From 5441d55ee191ca789ba86b7e477a64eb73398c80 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Wed, 22 May 2024 14:29:43 -0700 Subject: [PATCH 01/77] Gradle Check Optimization (#13786) Signed-off-by: Prudhvi Godithi --- .github/workflows/gradle-check.yml | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/.github/workflows/gradle-check.yml b/.github/workflows/gradle-check.yml index 6d8fe02c7c9c3..f610ff5c31049 100644 --- a/.github/workflows/gradle-check.yml +++ b/.github/workflows/gradle-check.yml @@ -39,6 +39,22 @@ jobs: echo "pr_number=$(jq --raw-output .pull_request.number $GITHUB_EVENT_PATH)" >> $GITHUB_ENV echo "pr_owner=$(jq --raw-output .pull_request.user.login $GITHUB_EVENT_PATH)" >> $GITHUB_ENV echo "pr_or_commit_description=$(jq --ascii-output .pull_request.body $GITHUB_EVENT_PATH)" >> $GITHUB_ENV + echo "post_merge_action=false" >> $GITHUB_ENV + + # to get the PR data that can be used for post merge actions + - uses: actions/github-script@v7 + if: github.event_name == 'push' + id: get_pr_data + with: + github-token: ${{secrets.GITHUB_TOKEN}} + script: | + return ( + await github.rest.repos.listPullRequestsAssociatedWithCommit({ + commit_sha: context.sha, + owner: context.repo.owner, + repo: context.repo.repo, + }) + ).data[0]; - name: Setup environment variables (Push) if: github.event_name == 'push' @@ -53,8 +69,9 @@ jobs: echo "pr_to_clone_url=$repo_url" >> $GITHUB_ENV echo "pr_title=Push trigger $branch_name $ref_id $repo_url" >> $GITHUB_ENV echo "pr_owner=$(jq --raw-output '.commits[0].author.username' $GITHUB_EVENT_PATH)" >> $GITHUB_ENV - echo "pr_number=Null" >> $GITHUB_ENV + echo 'pr_number=${{ fromJson(steps.get_pr_data.outputs.result).number }}' >> $GITHUB_ENV echo "pr_or_commit_description=$(jq --ascii-output .head_commit.message $GITHUB_EVENT_PATH)" >> $GITHUB_ENV + echo "post_merge_action=true" >> $GITHUB_ENV - name: Checkout opensearch-build repo uses: actions/checkout@v4 From b3049fb5ec860f63fdfe33c5b176869b1e4255e6 Mon Sep 17 00:00:00 2001 From: Varun Bansal Date: Thu, 23 May 2024 10:58:05 +0530 Subject: [PATCH 02/77] [Remote Store] Add segment transfer timeout dynamic setting (#13679) * [Remote Store] Add segment transfer timeout dynamic setting Signed-off-by: Varun Bansal --- CHANGELOG.md | 1 + .../RemoteStoreRefreshListenerIT.java | 1 - .../common/settings/ClusterSettings.java | 1 + .../opensearch/index/shard/IndexShard.java | 3 +- .../shard/RemoteStoreRefreshListener.java | 14 ++- .../shard/SegmentUploadFailedException.java | 28 ++++++ .../indices/RemoteStoreSettings.java | 26 ++++++ .../RemoteStoreRefreshListenerTests.java | 92 +++++++++++++++++-- 8 files changed, 154 insertions(+), 12 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/shard/SegmentUploadFailedException.java diff --git a/CHANGELOG.md b/CHANGELOG.md index a4e920f3a202d..4978c7aed3cfe 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add support for Azure Managed Identity in repository-azure ([#12423](https://github.com/opensearch-project/OpenSearch/issues/12423)) - Add useCompoundFile index setting ([#13478](https://github.com/opensearch-project/OpenSearch/pull/13478)) - Make outbound side of transport protocol dependent ([#13293](https://github.com/opensearch-project/OpenSearch/pull/13293)) +- [Remote Store] Add dynamic cluster settings to set timeout for segments upload to Remote Store ([#13679](https://github.com/opensearch-project/OpenSearch/pull/13679)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java index 65016c4976157..7ae08bf968ade 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRefreshListenerIT.java @@ -26,7 +26,6 @@ import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.SEGMENTS; import static org.opensearch.index.remote.RemoteStoreEnums.DataType.DATA; import static org.opensearch.index.remote.RemoteStorePressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED; -import static org.opensearch.test.OpenSearchTestCase.getShardLevelBlobPath; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteStoreRefreshListenerIT extends AbstractRemoteStoreMockRepositoryIntegTestCase { 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 4a5a45eb1a17a..33c0a498fc449 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -737,6 +737,7 @@ public void apply(Settings value, Settings current, Settings previous) { RemoteStoreSettings.CLUSTER_REMOTE_INDEX_SEGMENT_METADATA_RETENTION_MAX_COUNT_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_TRANSLOG_TRANSFER_TIMEOUT_SETTING, + RemoteStoreSettings.CLUSTER_REMOTE_SEGMENT_TRANSFER_TIMEOUT_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS 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 18d4a2ca6d639..42634d2129faa 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -3970,7 +3970,8 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro new RemoteStoreRefreshListener( this, this.checkpointPublisher, - remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId()) + remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId()), + remoteStoreSettings ) ); } 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 bfb841307af49..20afd7b2f3568 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -33,6 +33,7 @@ import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.translog.Translog; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.threadpool.ThreadPool; @@ -45,6 +46,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -89,11 +91,13 @@ public final class RemoteStoreRefreshListener extends ReleasableRetryableRefresh private volatile long primaryTerm; private volatile Iterator backoffDelayIterator; private final SegmentReplicationCheckpointPublisher checkpointPublisher; + private final RemoteStoreSettings remoteStoreSettings; public RemoteStoreRefreshListener( IndexShard indexShard, SegmentReplicationCheckpointPublisher checkpointPublisher, - RemoteSegmentTransferTracker segmentTracker + RemoteSegmentTransferTracker segmentTracker, + RemoteStoreSettings remoteStoreSettings ) { super(indexShard.getThreadPool()); logger = Loggers.getLogger(getClass(), indexShard.shardId()); @@ -116,6 +120,7 @@ public RemoteStoreRefreshListener( this.segmentTracker = segmentTracker; resetBackOffDelayIterator(); this.checkpointPublisher = checkpointPublisher; + this.remoteStoreSettings = remoteStoreSettings; } @Override @@ -286,7 +291,12 @@ public void onFailure(Exception e) { // Start the segments files upload uploadNewSegments(localSegmentsPostRefresh, localSegmentsSizeMap, segmentUploadsCompletedListener); - latch.await(); + if (latch.await( + remoteStoreSettings.getClusterRemoteSegmentTransferTimeout().millis(), + TimeUnit.MILLISECONDS + ) == false) { + throw new SegmentUploadFailedException("Timeout while waiting for remote segment transfer to complete"); + } } catch (EngineException e) { logger.warn("Exception while reading SegmentInfosSnapshot", e); } diff --git a/server/src/main/java/org/opensearch/index/shard/SegmentUploadFailedException.java b/server/src/main/java/org/opensearch/index/shard/SegmentUploadFailedException.java new file mode 100644 index 0000000000000..bbff399fb71ff --- /dev/null +++ b/server/src/main/java/org/opensearch/index/shard/SegmentUploadFailedException.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.index.shard; + +import java.io.IOException; + +/** + * Exception to be thrown when a segment upload fails. + * + * @opensearch.internal + */ +public class SegmentUploadFailedException extends IOException { + + /** + * Creates a new SegmentUploadFailedException. + * + * @param message error message + */ + public SegmentUploadFailedException(String message) { + super(message); + } +} diff --git a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java index 0bd4c7aedfc03..5234a09a58541 100644 --- a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java +++ b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java @@ -105,9 +105,21 @@ public class RemoteStoreSettings { Property.NodeScope ); + /** + * Controls timeout value while uploading segment files to remote segment store + */ + public static final Setting CLUSTER_REMOTE_SEGMENT_TRANSFER_TIMEOUT_SETTING = Setting.timeSetting( + "cluster.remote_store.segment.transfer_timeout", + TimeValue.timeValueMinutes(30), + TimeValue.timeValueMinutes(10), + Property.NodeScope, + Property.Dynamic + ); + private volatile TimeValue clusterRemoteTranslogBufferInterval; private volatile int minRemoteSegmentMetadataFiles; private volatile TimeValue clusterRemoteTranslogTransferTimeout; + private volatile TimeValue clusterRemoteSegmentTransferTimeout; private volatile RemoteStoreEnums.PathType pathType; private volatile RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm; private volatile int maxRemoteTranslogReaders; @@ -139,6 +151,12 @@ public RemoteStoreSettings(Settings settings, ClusterSettings clusterSettings) { maxRemoteTranslogReaders = CLUSTER_REMOTE_MAX_TRANSLOG_READERS.get(settings); clusterSettings.addSettingsUpdateConsumer(CLUSTER_REMOTE_MAX_TRANSLOG_READERS, this::setMaxRemoteTranslogReaders); + + clusterRemoteSegmentTransferTimeout = CLUSTER_REMOTE_SEGMENT_TRANSFER_TIMEOUT_SETTING.get(settings); + clusterSettings.addSettingsUpdateConsumer( + CLUSTER_REMOTE_SEGMENT_TRANSFER_TIMEOUT_SETTING, + this::setClusterRemoteSegmentTransferTimeout + ); } public TimeValue getClusterRemoteTranslogBufferInterval() { @@ -161,10 +179,18 @@ public TimeValue getClusterRemoteTranslogTransferTimeout() { return clusterRemoteTranslogTransferTimeout; } + public TimeValue getClusterRemoteSegmentTransferTimeout() { + return clusterRemoteSegmentTransferTimeout; + } + private void setClusterRemoteTranslogTransferTimeout(TimeValue clusterRemoteTranslogTransferTimeout) { this.clusterRemoteTranslogTransferTimeout = clusterRemoteTranslogTransferTimeout; } + private void setClusterRemoteSegmentTransferTimeout(TimeValue clusterRemoteSegmentTransferTimeout) { + this.clusterRemoteSegmentTransferTimeout = clusterRemoteSegmentTransferTimeout; + } + @ExperimentalApi public RemoteStoreEnums.PathType getPathType() { return pathType; diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index 8fbff4527ec7b..bb0776e0ced25 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -23,6 +23,7 @@ import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.engine.InternalEngineFactory; @@ -34,6 +35,7 @@ import org.opensearch.index.store.RemoteSegmentStoreDirectory.MetadataFilenameUtils; import org.opensearch.index.store.Store; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; +import org.opensearch.indices.DefaultRemoteStoreSettings; import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationType; @@ -91,7 +93,12 @@ public void setup(boolean primary, int numberOfDocs) throws IOException { remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, Settings.EMPTY); remoteStoreStatsTrackerFactory.afterIndexShardCreated(indexShard); RemoteSegmentTransferTracker tracker = remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId()); - remoteStoreRefreshListener = new RemoteStoreRefreshListener(indexShard, SegmentReplicationCheckpointPublisher.EMPTY, tracker); + remoteStoreRefreshListener = new RemoteStoreRefreshListener( + indexShard, + SegmentReplicationCheckpointPublisher.EMPTY, + tracker, + DefaultRemoteStoreSettings.INSTANCE + ); } private void indexDocs(int startDocId, int numberOfDocs) throws IOException { @@ -176,7 +183,12 @@ public void testRemoteDirectoryInitThrowsException() throws IOException { when(remoteStore.directory()).thenReturn(remoteStoreFilterDirectory); // Since the thrown IOException is caught in the constructor, ctor should be invoked successfully. - new RemoteStoreRefreshListener(shard, SegmentReplicationCheckpointPublisher.EMPTY, mock(RemoteSegmentTransferTracker.class)); + new RemoteStoreRefreshListener( + shard, + SegmentReplicationCheckpointPublisher.EMPTY, + mock(RemoteSegmentTransferTracker.class), + DefaultRemoteStoreSettings.INSTANCE + ); // Validate that the stream of metadata file of remoteMetadataDirectory has been opened only once and the // listFilesByPrefixInLexicographicOrder has been called twice. @@ -371,6 +383,33 @@ public void testRefreshSuccessOnSecondAttempt() throws Exception { assertNoLagAndTotalUploadsFailed(segmentTracker, 1); } + public void testSegmentUploadTimeout() throws Exception { + // This covers the case where segment upload fails due to timeout + int succeedOnAttempt = 1; + // We spy on IndexShard.isPrimaryStarted() to validate that we have tried running remote time as per the expectation. + CountDownLatch refreshCountLatch = new CountDownLatch(succeedOnAttempt); + CountDownLatch successLatch = new CountDownLatch(2); + Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh( + succeedOnAttempt, + refreshCountLatch, + successLatch, + 1, + new CountDownLatch(0), + true, + true + ); + assertBusy(() -> assertEquals(0, refreshCountLatch.getCount())); + assertBusy(() -> assertEquals(1, successLatch.getCount())); + RemoteStoreStatsTrackerFactory trackerFactory = tuple.v2(); + RemoteSegmentTransferTracker segmentTracker = trackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId()); + assertBusy(() -> { + assertTrue(segmentTracker.getTotalUploadsFailed() > 1); + assertTrue(segmentTracker.getTotalUploadsSucceeded() < 2); + }); + // shutdown threadpool for avoid leaking threads + indexShard.getThreadPool().shutdownNow(); + } + /** * Tests retry flow after snapshot and metadata files have been uploaded to remote store in the failed attempt. * Snapshot and metadata files created in failed attempt should not break retry. @@ -470,6 +509,7 @@ public void testRefreshFailedDueToPrimaryTermMisMatch() throws Exception { successLatch, checkpointPublishSucceedOnAttempt, reachedCheckpointPublishLatch, + false, false ); @@ -521,7 +561,8 @@ private Tuple mockIn successLatch, succeedCheckpointPublishOnAttempt, reachedCheckpointPublishLatch, - true + true, + false ); } @@ -531,7 +572,8 @@ private Tuple mockIn CountDownLatch successLatch, int succeedCheckpointPublishOnAttempt, CountDownLatch reachedCheckpointPublishLatch, - boolean mockPrimaryTerm + boolean mockPrimaryTerm, + boolean testUploadTimeout ) throws IOException { // Create index shard that we will be using to mock different methods in IndexShard for the unit test indexShard = newStartedShard( @@ -565,9 +607,22 @@ private Tuple mockIn // Mock (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()) Store remoteStore = mock(Store.class); when(shard.remoteStore()).thenReturn(remoteStore); - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = - (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()).getDelegate()) - .getDelegate(); + RemoteSegmentStoreDirectory remoteSegmentStoreDirectory; + RemoteDirectory remoteDirectory = mock(RemoteDirectory.class); + + if (testUploadTimeout) { + remoteSegmentStoreDirectory = new RemoteSegmentStoreDirectory( + remoteDirectory, + mock(RemoteDirectory.class), + mock(RemoteStoreLockManager.class), + indexShard.getThreadPool(), + indexShard.shardId + ); + } else { + remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore() + .directory()).getDelegate()).getDelegate(); + } + FilterDirectory remoteStoreFilterDirectory = new TestFilterDirectory(new TestFilterDirectory(remoteSegmentStoreDirectory)); when(remoteStore.directory()).thenReturn(remoteStoreFilterDirectory); @@ -639,7 +694,28 @@ private Tuple mockIn RemoteStoreSettings remoteStoreSettings = mock(RemoteStoreSettings.class); when(remoteStoreSettings.getMinRemoteSegmentMetadataFiles()).thenReturn(10); when(shard.getRemoteStoreSettings()).thenReturn(remoteStoreSettings); - RemoteStoreRefreshListener refreshListener = new RemoteStoreRefreshListener(shard, emptyCheckpointPublisher, tracker); + if (testUploadTimeout) { + when(remoteStoreSettings.getClusterRemoteSegmentTransferTimeout()).thenReturn(TimeValue.timeValueMillis(10)); + doAnswer(invocation -> { + ActionListener actionListener = invocation.getArgument(5); + indexShard.getThreadPool().executor(ThreadPool.Names.GENERIC).execute(() -> { + try { + Thread.sleep(30000); + } catch (InterruptedException e) { + logger.warn("copyFrom thread interrupted during sleep"); + } + actionListener.onResponse(null); + }); + return true; + }).when(remoteDirectory).copyFrom(any(), any(), any(), any(), any(), any(ActionListener.class), any(Boolean.class)); + } + + RemoteStoreRefreshListener refreshListener = new RemoteStoreRefreshListener( + shard, + emptyCheckpointPublisher, + tracker, + remoteStoreSettings + ); refreshListener.afterRefresh(true); return Tuple.tuple(refreshListener, remoteStoreStatsTrackerFactory); } From 76e2a5d9d70ab310033f33151fdaa0ab68c6537e Mon Sep 17 00:00:00 2001 From: Sandeep Kumawat <2025sandeepkumawat@gmail.com> Date: Thu, 23 May 2024 13:16:15 +0530 Subject: [PATCH 03/77] [Remote Store] Upload translog checkpoint as object metadata to translog (#13637) * Upload translog checkpoint as object metadata to translog Signed-off-by: Sandeep Kumawat <2025sandeepkumawat@gmail.com> --- CHANGELOG.md | 1 + .../repositories/s3/S3BlobContainer.java | 6 +- .../repositories/s3/S3BlobStore.java | 5 + .../indices/create/RemoteCloneIndexIT.java | 3 + .../indices/create/RemoteShrinkIndexIT.java | 9 + .../indices/create/RemoteSplitIndexIT.java | 2 + .../remote/RemoteClusterStateServiceIT.java | 6 + .../RemoteMigrationIndexMetadataUpdateIT.java | 1 + .../remotestore/BaseRemoteStoreRestoreIT.java | 7 +- .../remotestore/PrimaryTermValidationIT.java | 3 +- .../RemoteStoreBaseIntegTestCase.java | 47 +++- .../RemoteStoreClusterStateRestoreIT.java | 6 + .../remotestore/RemoteStoreForceMergeIT.java | 5 +- .../opensearch/remotestore/RemoteStoreIT.java | 26 +- .../RemoteStoreRepositoryRegistrationIT.java | 4 +- .../remotestore/RemoteStoreStatsIT.java | 2 +- .../RemoteStoreUploadIndexPathIT.java | 1 + .../MockFsMetadataSupportedBlobContainer.java | 92 +++++++ .../MockFsMetadataSupportedBlobStore.java | 44 +++ .../MockFsMetadataSupportedRepository.java | 51 ++++ ...ckFsMetadataSupportedRepositoryPlugin.java | 38 +++ .../cluster/metadata/IndexMetadata.java | 1 + .../metadata/MetadataCreateIndexService.java | 33 ++- .../allocation/IndexMetadataUpdater.java | 2 +- .../common/blobstore/BlobContainer.java | 6 +- .../common/blobstore/BlobStore.java | 7 + ...sult.java => InputStreamWithMetadata.java} | 12 +- .../common/settings/ClusterSettings.java | 3 +- .../org/opensearch/index/IndexSettings.java | 7 + .../RemoteMigrationIndexMetadataUpdater.java | 22 +- .../RemoteStoreCustomMetadataResolver.java | 74 ++++++ .../RemoteStorePathStrategyResolver.java | 44 --- .../index/remote/RemoteStoreUtils.java | 32 ++- .../opensearch/index/shard/IndexShard.java | 12 +- .../index/translog/RemoteFsTranslog.java | 31 ++- .../transfer/BlobStoreTransferService.java | 41 ++- .../index/translog/transfer/FileSnapshot.java | 10 + .../translog/transfer/TransferService.java | 6 +- .../translog/transfer/TransferSnapshot.java | 7 + .../TranslogCheckpointTransferSnapshot.java | 10 + .../transfer/TranslogTransferManager.java | 104 ++++++-- .../indices/RemoteStoreSettings.java | 24 ++ .../main/java/org/opensearch/node/Node.java | 3 +- .../opensearch/snapshots/RestoreService.java | 2 +- .../MetadataRolloverServiceTests.java | 9 +- .../MetadataCreateIndexServiceTests.java | 62 +++-- .../MetadataIndexTemplateServiceTests.java | 10 +- ...oteMigrationIndexMetadataUpdaterTests.java | 13 +- ...emoteStoreCustomMetadataResolverTests.java | 223 ++++++++++++++++ .../RemoteStorePathStrategyResolverTests.java | 148 ----------- .../index/remote/RemoteStoreUtilsTests.java | 57 ++++ .../BlobStoreTransferServiceTests.java | 40 +++ .../TranslogTransferManagerTests.java | 251 +++++++++++++----- .../indices/cluster/ClusterStateChanges.java | 3 +- .../snapshots/SnapshotResiliencyTests.java | 3 +- .../test/OpenSearchIntegTestCase.java | 1 + 56 files changed, 1287 insertions(+), 385 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedBlobContainer.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedBlobStore.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepository.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepositoryPlugin.java rename server/src/main/java/org/opensearch/common/blobstore/{FetchBlobResult.java => InputStreamWithMetadata.java} (74%) create mode 100644 server/src/main/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolver.java delete mode 100644 server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategyResolver.java create mode 100644 server/src/test/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolverTests.java delete mode 100644 server/src/test/java/org/opensearch/index/remote/RemoteStorePathStrategyResolverTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 4978c7aed3cfe..4cd610694a7b6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add useCompoundFile index setting ([#13478](https://github.com/opensearch-project/OpenSearch/pull/13478)) - Make outbound side of transport protocol dependent ([#13293](https://github.com/opensearch-project/OpenSearch/pull/13293)) - [Remote Store] Add dynamic cluster settings to set timeout for segments upload to Remote Store ([#13679](https://github.com/opensearch-project/OpenSearch/pull/13679)) +- [Remote Store] Upload translog checkpoint as object metadata to translog.tlog([#13637](https://github.com/opensearch-project/OpenSearch/pull/13637)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java index acf0c5e83a17b..b489a3cc85037 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java @@ -78,7 +78,7 @@ import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStoreException; import org.opensearch.common.blobstore.DeleteResult; -import org.opensearch.common.blobstore.FetchBlobResult; +import org.opensearch.common.blobstore.InputStreamWithMetadata; import org.opensearch.common.blobstore.stream.read.ReadContext; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; @@ -143,9 +143,9 @@ public boolean blobExists(String blobName) { @ExperimentalApi @Override - public FetchBlobResult readBlobWithMetadata(String blobName) throws IOException { + public InputStreamWithMetadata readBlobWithMetadata(String blobName) throws IOException { S3RetryingInputStream s3RetryingInputStream = new S3RetryingInputStream(blobStore, buildKey(blobName)); - return new FetchBlobResult(s3RetryingInputStream, s3RetryingInputStream.getMetadata()); + return new InputStreamWithMetadata(s3RetryingInputStream, s3RetryingInputStream.getMetadata()); } @Override diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobStore.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobStore.java index de815f9202f44..f688be9216b8f 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobStore.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobStore.java @@ -244,6 +244,11 @@ public Map> extendedStats() { return extendedStats; } + @Override + public boolean isBlobMetadataEnabled() { + return true; + } + public ObjectCannedACL getCannedACL() { return cannedACL; } diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java index 4be049c9a9109..a1122f279c7e4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java @@ -52,6 +52,7 @@ import org.opensearch.index.query.TermsQueryBuilder; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.VersionUtils; import java.util.concurrent.ExecutionException; @@ -60,6 +61,7 @@ import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; import static org.hamcrest.Matchers.equalTo; +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteCloneIndexIT extends RemoteStoreBaseIntegTestCase { @Override @@ -139,6 +141,7 @@ public void testCreateCloneIndex() { } public void testCreateCloneIndexFailure() throws ExecutionException, InterruptedException { + asyncUploadMockFsRepo = false; Version version = VersionUtils.randomIndexCompatibleVersion(random()); int numPrimaryShards = 1; prepareCreate("source").setSettings( diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteShrinkIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteShrinkIndexIT.java index 282eb9c6ad95e..cd19a0ee1ff77 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteShrinkIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteShrinkIndexIT.java @@ -48,7 +48,9 @@ import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.VersionUtils; +import org.junit.Before; import java.util.Arrays; import java.util.Map; @@ -61,12 +63,18 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteShrinkIndexIT extends RemoteStoreBaseIntegTestCase { @Override protected boolean forbidPrivateIndexSettings() { return false; } + @Before + public void setup() { + asyncUploadMockFsRepo = false; + } + public Settings indexSettings() { return Settings.builder() .put(super.indexSettings()) @@ -84,6 +92,7 @@ public void testCreateShrinkIndexToN() { int[] shardSplits = randomFrom(possibleShardSplits); assertEquals(shardSplits[0], (shardSplits[0] / shardSplits[1]) * shardSplits[1]); assertEquals(shardSplits[1], (shardSplits[1] / shardSplits[2]) * shardSplits[2]); + internalCluster().ensureAtLeastNumDataNodes(2); prepareCreate("source").setSettings(Settings.builder().put(indexSettings()).put("number_of_shards", shardSplits[0])).get(); for (int i = 0; i < 20; i++) { diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java index dd4252d24f314..dc3c8793a93f6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java @@ -67,6 +67,7 @@ import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.VersionUtils; import java.io.IOException; @@ -86,6 +87,7 @@ import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures; import static org.hamcrest.Matchers.equalTo; +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteSplitIndexIT extends RemoteStoreBaseIntegTestCase { @Override diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java index 61b34af5be3ba..42120aa32eb47 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java @@ -18,6 +18,7 @@ import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.Before; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -40,6 +41,11 @@ public class RemoteClusterStateServiceIT extends RemoteStoreBaseIntegTestCase { private static String INDEX_NAME = "test-index"; + @Before + public void setup() { + asyncUploadMockFsRepo = false; + } + @Override protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java index 45679598dc551..53f4ef3fe281f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java @@ -512,5 +512,6 @@ private void assertCustomIndexMetadata(String index) { logger.info("---> Asserting custom index metadata"); IndexMetadata iMd = internalCluster().client().admin().cluster().prepareState().get().getState().metadata().index(index); assertNotNull(iMd.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)); + assertNotNull(iMd.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY).get(IndexMetadata.TRANSLOG_METADATA_KEY)); } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/BaseRemoteStoreRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/BaseRemoteStoreRestoreIT.java index d29dacb001434..280fd13f0fdcf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/BaseRemoteStoreRestoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/BaseRemoteStoreRestoreIT.java @@ -11,15 +11,18 @@ import org.opensearch.action.index.IndexResponse; import org.opensearch.common.settings.Settings; import org.opensearch.plugins.Plugin; +import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.transport.MockTransportService; -import java.util.Arrays; import java.util.Collection; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class BaseRemoteStoreRestoreIT extends RemoteStoreBaseIntegTestCase { static final String INDEX_NAME = "remote-store-test-idx-1"; static final String INDEX_NAMES = "test-remote-store-1,test-remote-store-2,remote-store-test-index-1,remote-store-test-index-2"; @@ -39,7 +42,7 @@ public Settings indexSettings(int shards, int replicas) { @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class); + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockTransportService.TestPlugin.class)).collect(Collectors.toList()); } protected void restore(String... indices) { diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/PrimaryTermValidationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/PrimaryTermValidationIT.java index e14a4062f7775..6b94e638a6876 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/PrimaryTermValidationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/PrimaryTermValidationIT.java @@ -30,7 +30,6 @@ import org.junit.Before; import java.nio.file.Path; -import java.util.Arrays; import java.util.Collection; import java.util.HashSet; import java.util.Set; @@ -50,7 +49,7 @@ public class PrimaryTermValidationIT extends RemoteStoreBaseIntegTestCase { @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class); + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockTransportService.TestPlugin.class)).collect(Collectors.toList()); } @Before diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index d7ad0daa43524..740aee69f7d80 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -36,6 +36,9 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.plugins.Plugin; +import org.opensearch.remotestore.multipart.mocks.MockFsRepositoryPlugin; +import org.opensearch.remotestore.translogmetadata.mocks.MockFsMetadataSupportedRepositoryPlugin; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.repositories.fs.ReloadableFsRepository; @@ -48,6 +51,7 @@ import java.nio.file.Path; import java.nio.file.SimpleFileVisitor; import java.nio.file.attribute.BasicFileAttributes; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -55,6 +59,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import java.util.stream.Stream; 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; @@ -74,6 +79,8 @@ public class RemoteStoreBaseIntegTestCase extends OpenSearchIntegTestCase { protected Path segmentRepoPath; protected Path translogRepoPath; protected boolean clusterSettingsSuppliedByTest = false; + protected boolean asyncUploadMockFsRepo = randomBoolean(); + private boolean metadataSupportedType = randomBoolean(); private final List documentKeys = List.of( randomAlphaOfLength(5), randomAlphaOfLength(5), @@ -129,6 +136,19 @@ protected Map indexData(int numberOfIterations, boolean invokeFlus return indexingStats; } + @Override + protected Collection> nodePlugins() { + if (!clusterSettingsSuppliedByTest && asyncUploadMockFsRepo) { + if (metadataSupportedType) { + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockFsMetadataSupportedRepositoryPlugin.class)) + .collect(Collectors.toList()); + } else { + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockFsRepositoryPlugin.class)).collect(Collectors.toList()); + } + } + return super.nodePlugins(); + } + @Override protected Settings nodeSettings(int nodeOrdinal) { if (segmentRepoPath == null || translogRepoPath == null) { @@ -138,10 +158,27 @@ protected Settings nodeSettings(int nodeOrdinal) { if (clusterSettingsSuppliedByTest) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)).build(); } else { - return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - .put(remoteStoreClusterSettings(REPOSITORY_NAME, segmentRepoPath, REPOSITORY_2_NAME, translogRepoPath)) - .build(); + if (asyncUploadMockFsRepo) { + String repoType = metadataSupportedType ? MockFsMetadataSupportedRepositoryPlugin.TYPE_MD : MockFsRepositoryPlugin.TYPE; + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put( + remoteStoreClusterSettings( + REPOSITORY_NAME, + segmentRepoPath, + repoType, + REPOSITORY_2_NAME, + translogRepoPath, + repoType + ) + ) + .build(); + } else { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(remoteStoreClusterSettings(REPOSITORY_NAME, segmentRepoPath, REPOSITORY_2_NAME, translogRepoPath)) + .build(); + } } } @@ -221,6 +258,8 @@ protected Settings remoteStoreIndexSettings(int numberOfReplicas, long totalFiel @After public void teardown() { clusterSettingsSuppliedByTest = false; + asyncUploadMockFsRepo = randomBoolean(); + metadataSupportedType = randomBoolean(); assertRemoteStoreRepositoryOnAllNodes(REPOSITORY_NAME); assertRemoteStoreRepositoryOnAllNodes(REPOSITORY_2_NAME); clusterAdmin().prepareCleanupRepository(REPOSITORY_NAME).get(); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java index 8c8209b80bfd8..b22817ef19d1b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreClusterStateRestoreIT.java @@ -38,6 +38,7 @@ import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.Before; import java.io.IOException; import java.nio.file.Files; @@ -68,6 +69,11 @@ public class RemoteStoreClusterStateRestoreIT extends BaseRemoteStoreRestoreIT { static final Setting MOCK_SETTING = Setting.simpleString("mock-setting"); static final String[] EXCLUDED_NODES = { "ex-1", "ex-2" }; + @Before + public void setup() { + asyncUploadMockFsRepo = false; + } + @Override protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreForceMergeIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreForceMergeIT.java index 0bcde4b44c734..d957dda1ba04f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreForceMergeIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreForceMergeIT.java @@ -19,11 +19,12 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; @@ -37,7 +38,7 @@ public class RemoteStoreForceMergeIT extends RemoteStoreBaseIntegTestCase { @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class); + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockTransportService.TestPlugin.class)).collect(Collectors.toList()); } @Override diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java index ca0ae3ca9a700..7721b18a4fe6b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java @@ -38,7 +38,6 @@ import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.plugins.Plugin; -import org.opensearch.remotestore.multipart.mocks.MockFsRepositoryPlugin; import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.transport.MockTransportService; @@ -48,7 +47,6 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; @@ -56,6 +54,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import java.util.stream.Stream; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; @@ -81,7 +80,7 @@ public class RemoteStoreIT extends RemoteStoreBaseIntegTestCase { @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class, MockFsRepositoryPlugin.class); + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockTransportService.TestPlugin.class)).collect(Collectors.toList()); } @Override @@ -797,25 +796,8 @@ public void testResumeUploadAfterFailedPrimaryRelocation() throws ExecutionExcep // Test local only translog files which are not uploaded to remote store (no metadata present in remote) // Without the cleanup change in RemoteFsTranslog.createEmptyTranslog, this test fails with NPE. public void testLocalOnlyTranslogCleanupOnNodeRestart() throws Exception { - clusterSettingsSuppliedByTest = true; - - // Overriding settings to use AsyncMultiStreamBlobContainer - Settings settings = Settings.builder() - .put(super.nodeSettings(1)) - .put( - remoteStoreClusterSettings( - REPOSITORY_NAME, - segmentRepoPath, - MockFsRepositoryPlugin.TYPE, - REPOSITORY_2_NAME, - translogRepoPath, - MockFsRepositoryPlugin.TYPE - ) - ) - .build(); - - internalCluster().startClusterManagerOnlyNode(settings); - String dataNode = internalCluster().startDataOnlyNode(settings); + internalCluster().startClusterManagerOnlyNode(); + String dataNode = internalCluster().startDataOnlyNode(); // 1. Create index with 0 replica createIndex(INDEX_NAME, remoteStoreIndexSettings(0, 10000L, -1)); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRepositoryRegistrationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRepositoryRegistrationIT.java index ef2dcf3217df6..b0827dcfe4892 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRepositoryRegistrationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRepositoryRegistrationIT.java @@ -25,11 +25,11 @@ import org.opensearch.test.transport.MockTransportService; import java.io.IOException; -import java.util.Arrays; import java.util.Collection; import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.opensearch.repositories.blobstore.BlobStoreRepository.SYSTEM_REPOSITORY_SETTING; @@ -38,7 +38,7 @@ public class RemoteStoreRepositoryRegistrationIT extends RemoteStoreBaseIntegTes @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class); + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockTransportService.TestPlugin.class)).collect(Collectors.toList()); } public void testSingleNodeClusterRepositoryRegistration() throws Exception { diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java index 4a0af206b9d89..31c73e2fc03ae 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java @@ -54,7 +54,7 @@ public class RemoteStoreStatsIT extends RemoteStoreBaseIntegTestCase { @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class); + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockTransportService.TestPlugin.class)).collect(Collectors.toList()); } public void setup() { diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreUploadIndexPathIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreUploadIndexPathIT.java index 9b30dacfced13..44c02dbb6d611 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreUploadIndexPathIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreUploadIndexPathIT.java @@ -41,6 +41,7 @@ protected Settings nodeSettings(int nodeOrdinal) { * wherever not required. */ public void testRemoteIndexPathFileCreation() throws ExecutionException, InterruptedException, IOException { + asyncUploadMockFsRepo = false; String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); internalCluster().startDataOnlyNodes(2); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedBlobContainer.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedBlobContainer.java new file mode 100644 index 0000000000000..109a884ff6c5d --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedBlobContainer.java @@ -0,0 +1,92 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.remotestore.translogmetadata.mocks; + +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.InputStreamWithMetadata; +import org.opensearch.common.blobstore.fs.FsBlobStore; +import org.opensearch.common.blobstore.stream.write.WriteContext; +import org.opensearch.core.action.ActionListener; +import org.opensearch.remotestore.multipart.mocks.MockFsAsyncBlobContainer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Path; +import java.util.Base64; +import java.util.HashMap; +import java.util.Map; + +public class MockFsMetadataSupportedBlobContainer extends MockFsAsyncBlobContainer { + + private static String CHECKPOINT_FILE_DATA_KEY = "ckp-data"; + + public MockFsMetadataSupportedBlobContainer(FsBlobStore blobStore, BlobPath blobPath, Path path, boolean triggerDataIntegrityFailure) { + super(blobStore, blobPath, path, triggerDataIntegrityFailure); + } + + @Override + public void asyncBlobUpload(WriteContext writeContext, ActionListener completionListener) throws IOException { + // If the upload writeContext have a non-null metadata, we store the metadata content as translog.ckp file. + if (writeContext.getMetadata() != null) { + String base64String = writeContext.getMetadata().get(CHECKPOINT_FILE_DATA_KEY); + byte[] decodedBytes = Base64.getDecoder().decode(base64String); + ByteArrayInputStream inputStream = new ByteArrayInputStream(decodedBytes); + int length = decodedBytes.length; + String ckpFileName = getCheckpointFileName(writeContext.getFileName()); + writeBlob(ckpFileName, inputStream, length, true); + } + super.asyncBlobUpload(writeContext, completionListener); + } + + // This is utility to get the translog.ckp file name for a given translog.tlog file. + private String getCheckpointFileName(String translogFileName) { + if (!translogFileName.endsWith(".tlog")) { + throw new IllegalArgumentException("Invalid translog file name format: " + translogFileName); + } + + int dotIndex = translogFileName.lastIndexOf('.'); + String baseName = translogFileName.substring(0, dotIndex); + return baseName + ".ckp"; + } + + public static String convertToBase64(InputStream inputStream) throws IOException { + try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) { + byte[] buffer = new byte[128]; + int bytesRead; + int totalBytesRead = 0; + + while ((bytesRead = inputStream.read(buffer)) != -1) { + byteArrayOutputStream.write(buffer, 0, bytesRead); + totalBytesRead += bytesRead; + if (totalBytesRead > 1024) { + // We enforce a limit of 1KB on the size of the checkpoint file. + throw new AssertionError("Input stream exceeds 1KB limit"); + } + } + + byte[] bytes = byteArrayOutputStream.toByteArray(); + return Base64.getEncoder().encodeToString(bytes); + } + } + + // during readBlobWithMetadata call we separately download translog.ckp file and return it as metadata. + @Override + public InputStreamWithMetadata readBlobWithMetadata(String blobName) throws IOException { + String ckpFileName = getCheckpointFileName(blobName); + InputStream inputStream = readBlob(blobName); + try (InputStream ckpInputStream = readBlob(ckpFileName)) { + String ckpString = convertToBase64(ckpInputStream); + Map metadata = new HashMap<>(); + metadata.put(CHECKPOINT_FILE_DATA_KEY, ckpString); + return new InputStreamWithMetadata(inputStream, metadata); + } + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedBlobStore.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedBlobStore.java new file mode 100644 index 0000000000000..89dd91c8222ac --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedBlobStore.java @@ -0,0 +1,44 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.remotestore.translogmetadata.mocks; + +import org.opensearch.OpenSearchException; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.fs.FsBlobStore; + +import java.io.IOException; +import java.nio.file.Path; + +public class MockFsMetadataSupportedBlobStore extends FsBlobStore { + + private final boolean triggerDataIntegrityFailure; + + public MockFsMetadataSupportedBlobStore(int bufferSizeInBytes, Path path, boolean readonly, boolean triggerDataIntegrityFailure) + throws IOException { + super(bufferSizeInBytes, path, readonly); + this.triggerDataIntegrityFailure = triggerDataIntegrityFailure; + } + + @Override + public BlobContainer blobContainer(BlobPath path) { + try { + return new MockFsMetadataSupportedBlobContainer(this, path, buildAndCreate(path), triggerDataIntegrityFailure); + } catch (IOException ex) { + throw new OpenSearchException("failed to create blob container", ex); + } + } + + // Make MockFs metadata supported + @Override + public boolean isBlobMetadataEnabled() { + return true; + } + +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepository.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepository.java new file mode 100644 index 0000000000000..333fba413ce4e --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepository.java @@ -0,0 +1,51 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.remotestore.translogmetadata.mocks; + +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.fs.FsBlobStore; +import org.opensearch.common.settings.Setting; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.env.Environment; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.repositories.fs.FsRepository; + +public class MockFsMetadataSupportedRepository extends FsRepository { + + public static Setting TRIGGER_DATA_INTEGRITY_FAILURE = Setting.boolSetting( + "mock_fs_repository.trigger_data_integrity_failure", + false + ); + + private final boolean triggerDataIntegrityFailure; + + public MockFsMetadataSupportedRepository( + RepositoryMetadata metadata, + Environment environment, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + RecoverySettings recoverySettings + ) { + super(metadata, environment, namedXContentRegistry, clusterService, recoverySettings); + triggerDataIntegrityFailure = TRIGGER_DATA_INTEGRITY_FAILURE.get(metadata.settings()); + } + + @Override + protected BlobStore createBlobStore() throws Exception { + FsBlobStore fsBlobStore = (FsBlobStore) super.createBlobStore(); + return new MockFsMetadataSupportedBlobStore( + fsBlobStore.bufferSizeInBytes(), + fsBlobStore.path(), + isReadOnly(), + triggerDataIntegrityFailure + ); + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepositoryPlugin.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepositoryPlugin.java new file mode 100644 index 0000000000000..71ae652a6b23d --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepositoryPlugin.java @@ -0,0 +1,38 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.remotestore.translogmetadata.mocks; + +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.env.Environment; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.plugins.Plugin; +import org.opensearch.plugins.RepositoryPlugin; +import org.opensearch.repositories.Repository; + +import java.util.Collections; +import java.util.Map; + +public class MockFsMetadataSupportedRepositoryPlugin extends Plugin implements RepositoryPlugin { + + public static final String TYPE_MD = "fs_metadata_supported_repository"; + + @Override + public Map getRepositories( + Environment env, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + RecoverySettings recoverySettings + ) { + return Collections.singletonMap( + "fs_metadata_supported_repository", + metadata -> new MockFsMetadataSupportedRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings) + ); + } +} 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 80b78cfe154f1..8d2851c2e0561 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -636,6 +636,7 @@ public static APIBlock readFrom(StreamInput input) throws IOException { 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 TRANSLOG_METADATA_KEY = "translog_metadata"; 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 121f8d935cf48..16edec112f123 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -89,10 +89,10 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.MapperService.MergeReason; import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.remote.RemoteStoreCustomMetadataResolver; import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.index.remote.RemoteStorePathStrategy; -import org.opensearch.index.remote.RemoteStorePathStrategyResolver; import org.opensearch.index.shard.IndexSettingProvider; import org.opensearch.index.translog.Translog; import org.opensearch.indices.IndexCreationException; @@ -104,6 +104,7 @@ import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.node.remotestore.RemoteStoreNodeService; +import org.opensearch.repositories.RepositoriesService; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; @@ -177,7 +178,7 @@ public class MetadataCreateIndexService { private AwarenessReplicaBalance awarenessReplicaBalance; @Nullable - private final RemoteStorePathStrategyResolver remoteStorePathStrategyResolver; + private final RemoteStoreCustomMetadataResolver remoteStoreCustomMetadataResolver; public MetadataCreateIndexService( final Settings settings, @@ -193,7 +194,8 @@ public MetadataCreateIndexService( final SystemIndices systemIndices, final boolean forbidPrivateIndexSettings, final AwarenessReplicaBalance awarenessReplicaBalance, - final RemoteStoreSettings remoteStoreSettings + final RemoteStoreSettings remoteStoreSettings, + final Supplier repositoriesServiceSupplier ) { this.settings = settings; this.clusterService = clusterService; @@ -212,8 +214,8 @@ public MetadataCreateIndexService( // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction. createIndexTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.CREATE_INDEX_KEY, true); Supplier minNodeVersionSupplier = () -> clusterService.state().nodes().getMinNodeVersion(); - remoteStorePathStrategyResolver = isRemoteDataAttributePresent(settings) - ? new RemoteStorePathStrategyResolver(remoteStoreSettings, minNodeVersionSupplier) + remoteStoreCustomMetadataResolver = isRemoteDataAttributePresent(settings) + ? new RemoteStoreCustomMetadataResolver(remoteStoreSettings, minNodeVersionSupplier, repositoriesServiceSupplier, settings) : null; } @@ -562,7 +564,7 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata( tmpImdBuilder.setRoutingNumShards(routingNumShards); tmpImdBuilder.settings(indexSettings); tmpImdBuilder.system(isSystem); - addRemoteStorePathStrategyInCustomData(tmpImdBuilder, true); + addRemoteStoreCustomMetadata(tmpImdBuilder, true); // Set up everything, now locally create the index to see that things are ok, and apply IndexMetadata tempMetadata = tmpImdBuilder.build(); @@ -572,13 +574,13 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata( } /** - * Adds the remote store path type information in custom data of index metadata. + * Adds the 1) remote store path type 2) ckp as translog metadata information in custom data of index metadata. * * @param tmpImdBuilder index metadata builder. * @param assertNullOldType flag to verify that the old remote store path type is null */ - public void addRemoteStorePathStrategyInCustomData(IndexMetadata.Builder tmpImdBuilder, boolean assertNullOldType) { - if (remoteStorePathStrategyResolver == null) { + public void addRemoteStoreCustomMetadata(IndexMetadata.Builder tmpImdBuilder, boolean assertNullOldType) { + if (remoteStoreCustomMetadataResolver == null) { return; } // It is possible that remote custom data exists already. In such cases, we need to only update the path type @@ -586,14 +588,21 @@ public void addRemoteStorePathStrategyInCustomData(IndexMetadata.Builder tmpImdB Map existingCustomData = tmpImdBuilder.removeCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); assert assertNullOldType == false || Objects.isNull(existingCustomData); - // Determine the path type for use using the remoteStorePathResolver. - RemoteStorePathStrategy newPathStrategy = remoteStorePathStrategyResolver.get(); Map remoteCustomData = new HashMap<>(); + + // Determine if the ckp would be stored as translog metadata + boolean isTranslogMetadataEnabled = remoteStoreCustomMetadataResolver.isTranslogMetadataEnabled(); + remoteCustomData.put(IndexMetadata.TRANSLOG_METADATA_KEY, Boolean.toString(isTranslogMetadataEnabled)); + + // Determine the path type for use using the remoteStorePathResolver. + RemoteStorePathStrategy newPathStrategy = remoteStoreCustomMetadataResolver.getPathStrategy(); remoteCustomData.put(PathType.NAME, newPathStrategy.getType().name()); if (Objects.nonNull(newPathStrategy.getHashAlgorithm())) { remoteCustomData.put(PathHashAlgorithm.NAME, newPathStrategy.getHashAlgorithm().name()); } - logger.trace(() -> new ParameterizedMessage("Added newStrategy={}, replaced oldStrategy={}", remoteCustomData, existingCustomData)); + logger.trace( + () -> new ParameterizedMessage("Added newCustomData={}, replaced oldCustomData={}", remoteCustomData, existingCustomData) + ); tmpImdBuilder.putCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY, remoteCustomData); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java index ddcccd597e894..2431f57a6a1f9 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/IndexMetadataUpdater.java @@ -176,7 +176,7 @@ public Metadata applyChanges(Metadata oldMetadata, RoutingTable newRoutingTable, oldMetadata.settings(), logger ); - migrationImdUpdater.maybeUpdateRemoteStorePathStrategy(indexMetadataBuilder, index.getName()); + migrationImdUpdater.maybeUpdateRemoteStoreCustomMetadata(indexMetadataBuilder, index.getName()); migrationImdUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, index.getName()); } } diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index 4f5f8d4b1ef5f..a2e4199029ef4 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -80,16 +80,16 @@ public interface BlobContainer { InputStream readBlob(String blobName) throws IOException; /** - * Creates a new {@link FetchBlobResult} for the given blob name. + * Creates a new {@link InputStreamWithMetadata} for the given blob name. * * @param blobName * The name of the blob to get an {@link InputStream} for. - * @return The {@link FetchBlobResult} of the blob. + * @return The {@link InputStreamWithMetadata} of the blob. * @throws NoSuchFileException if the blob does not exist * @throws IOException if the blob can not be read. */ @ExperimentalApi - default FetchBlobResult readBlobWithMetadata(String blobName) throws IOException { + default InputStreamWithMetadata readBlobWithMetadata(String blobName) throws IOException { throw new UnsupportedOperationException("readBlobWithMetadata is not implemented yet"); }; diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobStore.java b/server/src/main/java/org/opensearch/common/blobstore/BlobStore.java index 8ce8ec8e01abe..406ccc6aa4a18 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobStore.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobStore.java @@ -71,6 +71,13 @@ default Map> extendedStats() { */ default void reload(RepositoryMetadata repositoryMetadata) {} + /** + * Returns a boolean indicating if blobStore has object metadata support enabled + */ + default boolean isBlobMetadataEnabled() { + return false; + } + /** * Metrics for BlobStore interactions */ diff --git a/server/src/main/java/org/opensearch/common/blobstore/FetchBlobResult.java b/server/src/main/java/org/opensearch/common/blobstore/InputStreamWithMetadata.java similarity index 74% rename from server/src/main/java/org/opensearch/common/blobstore/FetchBlobResult.java rename to server/src/main/java/org/opensearch/common/blobstore/InputStreamWithMetadata.java index 55aca771b586c..aa307e260e033 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/FetchBlobResult.java +++ b/server/src/main/java/org/opensearch/common/blobstore/InputStreamWithMetadata.java @@ -10,6 +10,8 @@ import org.opensearch.common.annotation.ExperimentalApi; +import java.io.Closeable; +import java.io.IOException; import java.io.InputStream; import java.util.Map; @@ -20,7 +22,7 @@ * @opensearch.experimental */ @ExperimentalApi -public class FetchBlobResult { +public class InputStreamWithMetadata implements Closeable { /** * Downloaded blob InputStream @@ -40,9 +42,15 @@ public Map getMetadata() { return metadata; } - public FetchBlobResult(InputStream inputStream, Map metadata) { + public InputStreamWithMetadata(InputStream inputStream, Map metadata) { this.inputStream = inputStream; this.metadata = metadata; } + @Override + public void close() throws IOException { + if (inputStream != null) { + inputStream.close(); + } + } } 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 33c0a498fc449..a2be2ea4510e0 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -740,7 +740,8 @@ public void apply(Settings value, Settings current, Settings previous) { RemoteStoreSettings.CLUSTER_REMOTE_SEGMENT_TRANSFER_TIMEOUT_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING, - RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS + RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS, + RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA ) ) ); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 9d8ab6815eecc..2b0a62c18d5a7 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -763,6 +763,7 @@ public static IndexMergePolicy fromString(String text) { private final boolean widenIndexSortType; private final boolean assignedOnRemoteNode; private final RemoteStorePathStrategy remoteStorePathStrategy; + private final boolean isTranslogMetadataEnabled; /** * The maximum age of a retention lease before it is considered expired. @@ -989,6 +990,8 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti assignedOnRemoteNode = RemoteStoreNodeAttribute.isRemoteDataAttributePresent(this.getNodeSettings()); remoteStorePathStrategy = RemoteStoreUtils.determineRemoteStorePathStrategy(indexMetadata); + isTranslogMetadataEnabled = RemoteStoreUtils.determineTranslogMetadataEnabled(indexMetadata); + setEnableFuzzySetForDocId(scopedSettings.get(INDEX_DOC_ID_FUZZY_SET_ENABLED_SETTING)); setDocIdFuzzySetFalsePositiveProbability(scopedSettings.get(INDEX_DOC_ID_FUZZY_SET_FALSE_POSITIVE_PROBABILITY_SETTING)); @@ -1911,4 +1914,8 @@ public void setDocIdFuzzySetFalsePositiveProbability(double docIdFuzzySetFalsePo public RemoteStorePathStrategy getRemoteStorePathStrategy() { return remoteStorePathStrategy; } + + public boolean isTranslogMetadataEnabled() { + return isTranslogMetadataEnabled; + } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java index 761fa20ea64e5..cc51fcd2f18f6 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdater.java @@ -28,7 +28,7 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; -import static org.opensearch.index.remote.RemoteStoreUtils.determineRemoteStorePathStrategyDuringMigration; +import static org.opensearch.index.remote.RemoteStoreUtils.determineRemoteStoreCustomMetadataDuringMigration; import static org.opensearch.index.remote.RemoteStoreUtils.getRemoteStoreRepoName; 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; @@ -118,7 +118,7 @@ private boolean needsRemoteIndexSettingsUpdate( } /** - * Updates the remote store path strategy metadata for the index when it is migrating to remote. + * Updates the remote store custom metadata for the index when it is migrating to remote. * This is run during state change of each shard copy when the cluster is in `MIXED` mode and the direction of migration is `REMOTE_STORE` * Should not interfere with docrep functionality even if the index is in docrep nodes since this metadata * is not used anywhere in the docrep flow @@ -127,20 +127,20 @@ private boolean needsRemoteIndexSettingsUpdate( * @param indexMetadataBuilder Mutated {@link IndexMetadata.Builder} having the previous state updates * @param index index name */ - public void maybeUpdateRemoteStorePathStrategy(IndexMetadata.Builder indexMetadataBuilder, String index) { - if (indexHasRemotePathMetadata(indexMetadata) == false) { - logger.info("Adding remote store path strategy for index [{}] during migration", index); + public void maybeUpdateRemoteStoreCustomMetadata(IndexMetadata.Builder indexMetadataBuilder, String index) { + if (indexHasRemoteCustomMetadata(indexMetadata) == false) { + logger.info("Adding remote store custom data for index [{}] during migration", index); indexMetadataBuilder.putCustom( REMOTE_STORE_CUSTOM_KEY, - determineRemoteStorePathStrategyDuringMigration(clusterSettings, discoveryNodes) + determineRemoteStoreCustomMetadataDuringMigration(clusterSettings, discoveryNodes) ); } else { - logger.debug("Index {} already has remote store path strategy", index); + logger.debug("Index {} already has remote store custom data", index); } } public static boolean indexHasAllRemoteStoreRelatedMetadata(IndexMetadata indexMetadata) { - return indexHasRemoteStoreSettings(indexMetadata.getSettings()) && indexHasRemotePathMetadata(indexMetadata); + return indexHasRemoteStoreSettings(indexMetadata.getSettings()) && indexHasRemoteCustomMetadata(indexMetadata); } /** @@ -167,9 +167,11 @@ public static boolean indexHasRemoteStoreSettings(Settings indexSettings) { * @param indexMetadata Current index metadata * @return true if all above conditions match. false otherwise */ - public static boolean indexHasRemotePathMetadata(IndexMetadata indexMetadata) { + public static boolean indexHasRemoteCustomMetadata(IndexMetadata indexMetadata) { Map customMetadata = indexMetadata.getCustomData(REMOTE_STORE_CUSTOM_KEY); - return Objects.nonNull(customMetadata) && Objects.nonNull(customMetadata.get(PathType.NAME)); + return Objects.nonNull(customMetadata) + && (Objects.nonNull(customMetadata.get(PathType.NAME)) + || Objects.nonNull(customMetadata.get(IndexMetadata.TRANSLOG_METADATA_KEY))); } public static void updateRemoteStoreSettings(Settings.Builder settingsBuilder, String segmentRepository, String translogRepository) { diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolver.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolver.java new file mode 100644 index 0000000000000..e8a0dda5a699e --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolver.java @@ -0,0 +1,74 @@ +/* + * 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.Version; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; +import org.opensearch.index.remote.RemoteStoreEnums.PathType; +import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.Repository; +import org.opensearch.repositories.RepositoryMissingException; +import org.opensearch.repositories.blobstore.BlobStoreRepository; + +import java.util.function.Supplier; + +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo; + +/** + * Determines the {@link RemoteStorePathStrategy} at the time of index metadata creation. + * + * @opensearch.internal + */ +@ExperimentalApi +public class RemoteStoreCustomMetadataResolver { + + private final RemoteStoreSettings remoteStoreSettings; + private final Supplier minNodeVersionSupplier; + private final Supplier repositoriesServiceSupplier; + private final Settings settings; + + public RemoteStoreCustomMetadataResolver( + RemoteStoreSettings remoteStoreSettings, + Supplier minNodeVersionSupplier, + Supplier repositoriesServiceSupplier, + Settings settings + ) { + this.remoteStoreSettings = remoteStoreSettings; + this.minNodeVersionSupplier = minNodeVersionSupplier; + this.repositoriesServiceSupplier = repositoriesServiceSupplier; + this.settings = settings; + } + + public RemoteStorePathStrategy getPathStrategy() { + PathType pathType; + PathHashAlgorithm pathHashAlgorithm; + // Min node version check ensures that we are enabling the new prefix type only when all the nodes understand it. + pathType = Version.V_2_14_0.compareTo(minNodeVersionSupplier.get()) <= 0 ? remoteStoreSettings.getPathType() : PathType.FIXED; + // If the path type is fixed, hash algorithm is not applicable. + pathHashAlgorithm = pathType == PathType.FIXED ? null : remoteStoreSettings.getPathHashAlgorithm(); + return new RemoteStorePathStrategy(pathType, pathHashAlgorithm); + } + + public boolean isTranslogMetadataEnabled() { + Repository repository; + try { + repository = repositoriesServiceSupplier.get().repository(getRemoteStoreTranslogRepo(settings)); + } catch (RepositoryMissingException ex) { + throw new IllegalArgumentException("Repository should be created before creating index with remote_store enabled setting", ex); + } + BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; + return Version.V_2_15_0.compareTo(minNodeVersionSupplier.get()) <= 0 + && remoteStoreSettings.isTranslogMetadataEnabled() + && blobStoreRepository.blobStore().isBlobMetadataEnabled(); + } + +} diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategyResolver.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategyResolver.java deleted file mode 100644 index 178de406ed681..0000000000000 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategyResolver.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.index.remote; - -import org.opensearch.Version; -import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; -import org.opensearch.index.remote.RemoteStoreEnums.PathType; -import org.opensearch.indices.RemoteStoreSettings; - -import java.util.function.Supplier; - -/** - * Determines the {@link RemoteStorePathStrategy} at the time of index metadata creation. - * - * @opensearch.internal - */ -@ExperimentalApi -public class RemoteStorePathStrategyResolver { - - private final RemoteStoreSettings remoteStoreSettings; - private final Supplier minNodeVersionSupplier; - - public RemoteStorePathStrategyResolver(RemoteStoreSettings remoteStoreSettings, Supplier minNodeVersionSupplier) { - this.remoteStoreSettings = remoteStoreSettings; - this.minNodeVersionSupplier = minNodeVersionSupplier; - } - - public RemoteStorePathStrategy get() { - PathType pathType; - PathHashAlgorithm pathHashAlgorithm; - // Min node version check ensures that we are enabling the new prefix type only when all the nodes understand it. - pathType = Version.V_2_14_0.compareTo(minNodeVersionSupplier.get()) <= 0 ? remoteStoreSettings.getPathType() : PathType.FIXED; - // If the path type is fixed, hash algorithm is not applicable. - pathHashAlgorithm = pathType == PathType.FIXED ? null : remoteStoreSettings.getPathHashAlgorithm(); - return new RemoteStorePathStrategy(pathType, pathHashAlgorithm); - } -} diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 27b1b88034573..9a9de6c819424 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -32,6 +32,7 @@ import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA; /** * Utils for remote store @@ -181,25 +182,50 @@ public static RemoteStorePathStrategy determineRemoteStorePathStrategy(IndexMeta return new RemoteStorePathStrategy(RemoteStoreEnums.PathType.FIXED); } + /** + * Determines if translog file object metadata can be used to store checkpoint file data. + */ + public static boolean determineTranslogMetadataEnabled(IndexMetadata indexMetadata) { + Map remoteCustomData = indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + assert remoteCustomData == null || remoteCustomData.containsKey(IndexMetadata.TRANSLOG_METADATA_KEY); + if (remoteCustomData != null && remoteCustomData.containsKey(IndexMetadata.TRANSLOG_METADATA_KEY)) { + return Boolean.parseBoolean(remoteCustomData.get(IndexMetadata.TRANSLOG_METADATA_KEY)); + } + return false; + } + /** * Generates the remote store path type information to be added to custom data of index metadata during migration * * @param clusterSettings Current Cluster settings from {@link ClusterState} - * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state + * @param discoveryNodes Current {@link DiscoveryNodes} from the cluster state * @return {@link Map} to be added as custom data in index metadata */ - public static Map determineRemoteStorePathStrategyDuringMigration( + public static Map determineRemoteStoreCustomMetadataDuringMigration( Settings clusterSettings, DiscoveryNodes discoveryNodes ) { + Map remoteCustomData = new HashMap<>(); Version minNodeVersion = discoveryNodes.getMinNodeVersion(); + + // TODO: During the document replication to a remote store migration, there should be a check to determine if the registered + // translog blobstore supports custom metadata or not. + // Currently, the blobStoreMetadataEnabled flag is set to false because the integration tests run on the local file system, which + // does not support custom metadata. + // https://github.com/opensearch-project/OpenSearch/issues/13745 + boolean blobStoreMetadataEnabled = false; + boolean translogMetadata = Version.CURRENT.compareTo(minNodeVersion) <= 0 + && CLUSTER_REMOTE_STORE_TRANSLOG_METADATA.get(clusterSettings) + && blobStoreMetadataEnabled; + + remoteCustomData.put(IndexMetadata.TRANSLOG_METADATA_KEY, Boolean.toString(translogMetadata)); + RemoteStoreEnums.PathType pathType = Version.CURRENT.compareTo(minNodeVersion) <= 0 ? CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.get(clusterSettings) : RemoteStoreEnums.PathType.FIXED; RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm = pathType == RemoteStoreEnums.PathType.FIXED ? null : CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.get(clusterSettings); - Map remoteCustomData = new HashMap<>(); remoteCustomData.put(RemoteStoreEnums.PathType.NAME, pathType.name()); if (Objects.nonNull(pathHashAlgorithm)) { remoteCustomData.put(RemoteStoreEnums.PathHashAlgorithm.NAME, pathHashAlgorithm.name()); 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 42634d2129faa..3517579856d43 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4977,7 +4977,14 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException { TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting); assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory; Repository repository = ((RemoteBlobStoreInternalTranslogFactory) translogFactory).getRepository(); - RemoteFsTranslog.cleanup(repository, shardId, getThreadPool(), indexSettings.getRemoteStorePathStrategy(), remoteStoreSettings); + RemoteFsTranslog.cleanup( + repository, + shardId, + getThreadPool(), + indexSettings.getRemoteStorePathStrategy(), + remoteStoreSettings, + indexSettings().isTranslogMetadataEnabled() + ); } /* @@ -5002,7 +5009,8 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { indexSettings.getRemoteStorePathStrategy(), remoteStoreSettings, logger, - shouldSeedRemoteStore() + shouldSeedRemoteStore(), + indexSettings().isTranslogMetadataEnabled() ); } 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 c7f756957076c..67549c86b7dd2 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -91,6 +91,7 @@ public class RemoteFsTranslog extends Translog { private static final int SYNC_PERMIT = 1; private final Semaphore syncPermit = new Semaphore(SYNC_PERMIT); private final AtomicBoolean pauseSync = new AtomicBoolean(false); + private final boolean isTranslogMetadataEnabled; public RemoteFsTranslog( TranslogConfig config, @@ -110,6 +111,7 @@ public RemoteFsTranslog( this.startedPrimarySupplier = startedPrimarySupplier; this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); + isTranslogMetadataEnabled = indexSettings().isTranslogMetadataEnabled(); this.translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, @@ -117,7 +119,8 @@ public RemoteFsTranslog( fileTransferTracker, remoteTranslogTransferTracker, indexSettings().getRemoteStorePathStrategy(), - remoteStoreSettings + remoteStoreSettings, + isTranslogMetadataEnabled ); try { download(translogTransferManager, location, logger, config.shouldSeedRemote()); @@ -169,7 +172,8 @@ public static void download( RemoteStorePathStrategy pathStrategy, RemoteStoreSettings remoteStoreSettings, Logger logger, - boolean seedRemote + boolean seedRemote, + boolean isTranslogMetadataEnabled ) throws IOException { assert repository instanceof BlobStoreRepository : String.format( Locale.ROOT, @@ -188,7 +192,8 @@ public static void download( fileTransferTracker, remoteTranslogTransferTracker, pathStrategy, - remoteStoreSettings + remoteStoreSettings, + isTranslogMetadataEnabled ); RemoteFsTranslog.download(translogTransferManager, location, logger, seedRemote); logger.trace(remoteTranslogTransferTracker.toString()); @@ -293,7 +298,8 @@ public static TranslogTransferManager buildTranslogTransferManager( FileTransferTracker fileTransferTracker, RemoteTranslogTransferTracker tracker, RemoteStorePathStrategy pathStrategy, - RemoteStoreSettings remoteStoreSettings + RemoteStoreSettings remoteStoreSettings, + boolean isTranslogMetadataEnabled ) { assert Objects.nonNull(pathStrategy); String indexUUID = shardId.getIndex().getUUID(); @@ -315,7 +321,16 @@ public static TranslogTransferManager buildTranslogTransferManager( .build(); BlobPath mdPath = pathStrategy.generatePath(mdPathInput); BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool); - return new TranslogTransferManager(shardId, transferService, dataPath, mdPath, fileTransferTracker, tracker, remoteStoreSettings); + return new TranslogTransferManager( + shardId, + transferService, + dataPath, + mdPath, + fileTransferTracker, + tracker, + remoteStoreSettings, + isTranslogMetadataEnabled + ); } @Override @@ -592,7 +607,8 @@ public static void cleanup( ShardId shardId, ThreadPool threadPool, RemoteStorePathStrategy pathStrategy, - RemoteStoreSettings remoteStoreSettings + RemoteStoreSettings remoteStoreSettings, + boolean isTranslogMetadataEnabled ) throws IOException { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; @@ -607,7 +623,8 @@ public static void cleanup( fileTransferTracker, remoteTranslogTransferTracker, pathStrategy, - remoteStoreSettings + remoteStoreSettings, + isTranslogMetadataEnabled ); // clean up all remote translog files translogTransferManager.deleteTranslogFiles(); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index bec2d78d9af62..704f6419da60a 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -18,7 +18,7 @@ import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; -import org.opensearch.common.blobstore.FetchBlobResult; +import org.opensearch.common.blobstore.InputStreamWithMetadata; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; @@ -28,16 +28,20 @@ import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.threadpool.ThreadPool; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; +import java.util.Base64; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; import static org.opensearch.common.blobstore.BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC; +import static org.opensearch.index.translog.transfer.TranslogTransferManager.CHECKPOINT_FILE_DATA_KEY; /** * Service that handles remote transfer of translog and checkpoint files @@ -104,6 +108,30 @@ public void uploadBlobs( } + // Builds a metadata map containing the Base64-encoded checkpoint file data associated with a translog file. + static Map buildTransferFileMetadata(InputStream metadataInputStream) throws IOException { + Map metadata = new HashMap<>(); + try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) { + byte[] buffer = new byte[128]; + int bytesRead; + int totalBytesRead = 0; + + while ((bytesRead = metadataInputStream.read(buffer)) != -1) { + byteArrayOutputStream.write(buffer, 0, bytesRead); + totalBytesRead += bytesRead; + if (totalBytesRead > 1024) { + // We enforce a limit of 1KB on the size of the checkpoint file. + throw new IOException("Input stream exceeds 1KB limit"); + } + } + + byte[] bytes = byteArrayOutputStream.toByteArray(); + String metadataString = Base64.getEncoder().encodeToString(bytes); + metadata.put(CHECKPOINT_FILE_DATA_KEY, metadataString); + } + return metadata; + } + private void uploadBlob( TransferFileSnapshot fileSnapshot, ActionListener listener, @@ -113,6 +141,11 @@ private void uploadBlob( try { ChannelFactory channelFactory = FileChannel::open; + Map metadata = null; + if (fileSnapshot.getMetadataFileInputStream() != null) { + metadata = buildTransferFileMetadata(fileSnapshot.getMetadataFileInputStream()); + } + long contentLength; try (FileChannel channel = channelFactory.open(fileSnapshot.getPath(), StandardOpenOption.READ)) { contentLength = channel.size(); @@ -130,7 +163,8 @@ private void uploadBlob( writePriority, (size, position) -> new OffsetRangeFileInputStream(fileSnapshot.getPath(), size, position), Objects.requireNonNull(fileSnapshot.getChecksum()), - remoteIntegrityEnabled + remoteIntegrityEnabled, + metadata ); ActionListener completionListener = ActionListener.wrap(resp -> listener.onResponse(fileSnapshot), ex -> { logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), ex); @@ -168,7 +202,8 @@ public InputStream downloadBlob(Iterable path, String fileName) throws I @Override @ExperimentalApi - public FetchBlobResult downloadBlobWithMetadata(Iterable path, String fileName) throws IOException { + public InputStreamWithMetadata downloadBlobWithMetadata(Iterable path, String fileName) throws IOException { + assert blobStore.isBlobMetadataEnabled(); return blobStore.blobContainer((BlobPath) path).readBlobWithMetadata(fileName); } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java b/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java index dcec94edd694f..86f042af0584b 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java @@ -108,6 +108,8 @@ public static class TransferFileSnapshot extends FileSnapshot { private final long primaryTerm; private Long checksum; + @Nullable + private InputStream metadataFileInputStream; public TransferFileSnapshot(Path path, long primaryTerm, Long checksum) throws IOException { super(path); @@ -128,6 +130,14 @@ public long getPrimaryTerm() { return primaryTerm; } + public void setMetadataFileInputStream(InputStream inputStream) { + this.metadataFileInputStream = inputStream; + } + + public InputStream getMetadataFileInputStream() { + return metadataFileInputStream; + } + @Override public int hashCode() { return Objects.hash(primaryTerm, super.hashCode()); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java index 0894ebf500ebd..0ff983739438b 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java @@ -11,7 +11,7 @@ import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; -import org.opensearch.common.blobstore.FetchBlobResult; +import org.opensearch.common.blobstore.InputStreamWithMetadata; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.core.action.ActionListener; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; @@ -131,11 +131,11 @@ void uploadBlobs( * * @param path the remote path from where download should be made * @param fileName the name of the file - * @return {@link FetchBlobResult} of the remote file + * @return {@link InputStreamWithMetadata} of the remote file * @throws IOException the exception while reading the data */ @ExperimentalApi - FetchBlobResult downloadBlobWithMetadata(Iterable path, String fileName) throws IOException; + InputStreamWithMetadata downloadBlobWithMetadata(Iterable path, String fileName) throws IOException; void listAllInSortedOrder(Iterable path, String filenamePrefix, int limit, ActionListener> listener); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TransferSnapshot.java b/server/src/main/java/org/opensearch/index/translog/transfer/TransferSnapshot.java index ef34fd31a296b..6dcdc8f8cf44a 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TransferSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TransferSnapshot.java @@ -12,6 +12,7 @@ import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.index.translog.transfer.FileSnapshot.TranslogFileSnapshot; +import java.io.IOException; import java.util.Set; /** @@ -39,4 +40,10 @@ public interface TransferSnapshot { * @return the translog transfer metadata */ TranslogTransferMetadata getTranslogTransferMetadata(); + + /** + * The snapshot of the translog generational files having checkpoint file inputStream as metadata + * @return the set of translog files having checkpoint file inputStream as metadata. + */ + Set getTranslogFileSnapshotWithMetadata() throws IOException; } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogCheckpointTransferSnapshot.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogCheckpointTransferSnapshot.java index fb78731246a07..ae007c0c33e1e 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogCheckpointTransferSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogCheckpointTransferSnapshot.java @@ -64,6 +64,16 @@ public Set getTranslogFileSnapshots() { return translogCheckpointFileInfoTupleSet.stream().map(Tuple::v1).collect(Collectors.toSet()); } + @Override + public Set getTranslogFileSnapshotWithMetadata() throws IOException { + for (Tuple tuple : translogCheckpointFileInfoTupleSet) { + TransferFileSnapshot translogFileSnapshot = tuple.v1(); + TransferFileSnapshot checkpointFileSnapshot = tuple.v2(); + translogFileSnapshot.setMetadataFileInputStream(checkpointFileSnapshot.inputStream()); + } + return getTranslogFileSnapshots(); + } + @Override public TranslogTransferMetadata getTranslogTransferMetadata() { return new TranslogTransferMetadata( diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 47638f44fd6fc..1cc39cdf442e2 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -16,6 +16,7 @@ import org.opensearch.common.SetOnce; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.InputStreamWithMetadata; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.common.io.stream.BytesStreamOutput; @@ -36,6 +37,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Base64; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -63,6 +65,9 @@ public class TranslogTransferManager { private final RemoteTranslogTransferTracker remoteTranslogTransferTracker; private final RemoteStoreSettings remoteStoreSettings; private static final int METADATA_FILES_TO_FETCH = 10; + // Flag to include checkpoint file data as translog file metadata during upload/download + private final boolean isTranslogMetadataEnabled; + final static String CHECKPOINT_FILE_DATA_KEY = "ckp-data"; private final Logger logger; @@ -79,7 +84,8 @@ public TranslogTransferManager( BlobPath remoteMetadataTransferPath, FileTransferTracker fileTransferTracker, RemoteTranslogTransferTracker remoteTranslogTransferTracker, - RemoteStoreSettings remoteStoreSettings + RemoteStoreSettings remoteStoreSettings, + boolean isTranslogMetadataEnabled ) { this.shardId = shardId; this.transferService = transferService; @@ -89,6 +95,7 @@ public TranslogTransferManager( this.logger = Loggers.getLogger(getClass(), shardId); this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; this.remoteStoreSettings = remoteStoreSettings; + this.isTranslogMetadataEnabled = isTranslogMetadataEnabled; } public RemoteTranslogTransferTracker getRemoteTranslogTransferTracker() { @@ -110,8 +117,12 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans long prevUploadTimeInMillis = remoteTranslogTransferTracker.getTotalUploadTimeInMillis(); try { - toUpload.addAll(fileTransferTracker.exclusionFilter(transferSnapshot.getTranslogFileSnapshots())); - toUpload.addAll(fileTransferTracker.exclusionFilter((transferSnapshot.getCheckpointFileSnapshots()))); + if (isTranslogMetadataEnabled) { + toUpload.addAll(fileTransferTracker.exclusionFilter(transferSnapshot.getTranslogFileSnapshotWithMetadata())); + } else { + toUpload.addAll(fileTransferTracker.exclusionFilter(transferSnapshot.getTranslogFileSnapshots())); + toUpload.addAll(fileTransferTracker.exclusionFilter((transferSnapshot.getCheckpointFileSnapshots()))); + } if (toUpload.isEmpty()) { logger.trace("Nothing to upload for transfer"); return true; @@ -236,30 +247,78 @@ public boolean downloadTranslog(String primaryTerm, String generation, Path loca generation, location ); - // Download Checkpoint file from remote to local FS String ckpFileName = Translog.getCommitCheckpointFileName(Long.parseLong(generation)); - downloadToFS(ckpFileName, location, primaryTerm); - // Download translog file from remote to local FS String translogFilename = Translog.getFilename(Long.parseLong(generation)); - downloadToFS(translogFilename, location, primaryTerm); + if (isTranslogMetadataEnabled == false) { + // Download Checkpoint file, translog file from remote to local FS + downloadToFS(ckpFileName, location, primaryTerm, false); + downloadToFS(translogFilename, location, primaryTerm, false); + } else { + // Download translog.tlog file with object metadata from remote to local FS + Map metadata = downloadToFS(translogFilename, location, primaryTerm, true); + try { + assert metadata != null && !metadata.isEmpty() && metadata.containsKey(CHECKPOINT_FILE_DATA_KEY); + recoverCkpFileUsingMetadata(metadata, location, generation, translogFilename); + } catch (Exception e) { + throw new IOException("Failed to recover checkpoint file from remote", e); + } + } return true; } - private void downloadToFS(String fileName, Path location, String primaryTerm) throws IOException { + /** + * Process the provided metadata and tries to recover translog.ckp file to the FS. + */ + private void recoverCkpFileUsingMetadata(Map metadata, Path location, String generation, String fileName) + throws IOException { + + String ckpFileName = Translog.getCommitCheckpointFileName(Long.parseLong(generation)); + Path filePath = location.resolve(ckpFileName); + // Here, we always override the existing file if present. + deleteFileIfExists(filePath); + + String ckpDataBase64 = metadata.get(CHECKPOINT_FILE_DATA_KEY); + if (ckpDataBase64 == null) { + logger.error("Error processing metadata for translog file: {}", fileName); + throw new IllegalStateException( + "Checkpoint file data key " + CHECKPOINT_FILE_DATA_KEY + " is expected but not found in metadata for file: " + fileName + ); + } + byte[] ckpFileBytes = Base64.getDecoder().decode(ckpDataBase64); + Files.write(filePath, ckpFileBytes); + } + + private Map downloadToFS(String fileName, Path location, String primaryTerm, boolean withMetadata) throws IOException { Path filePath = location.resolve(fileName); // Here, we always override the existing file if present. // We need to change this logic when we introduce incremental download - if (Files.exists(filePath)) { - Files.delete(filePath); - } + deleteFileIfExists(filePath); + Map metadata = null; boolean downloadStatus = false; long bytesToRead = 0, downloadStartTime = System.nanoTime(); - try (InputStream inputStream = transferService.downloadBlob(remoteDataTransferPath.add(primaryTerm), fileName)) { - // Capture number of bytes for stats before reading - bytesToRead = inputStream.available(); - Files.copy(inputStream, filePath); - downloadStatus = true; + try { + if (withMetadata) { + try ( + InputStreamWithMetadata inputStreamWithMetadata = transferService.downloadBlobWithMetadata( + remoteDataTransferPath.add(primaryTerm), + fileName + ) + ) { + InputStream inputStream = inputStreamWithMetadata.getInputStream(); + metadata = inputStreamWithMetadata.getMetadata(); + + bytesToRead = inputStream.available(); + Files.copy(inputStream, filePath); + downloadStatus = true; + } + } else { + try (InputStream inputStream = transferService.downloadBlob(remoteDataTransferPath.add(primaryTerm), fileName)) { + bytesToRead = inputStream.available(); + Files.copy(inputStream, filePath); + downloadStatus = true; + } + } } finally { remoteTranslogTransferTracker.addDownloadTimeInMillis((System.nanoTime() - downloadStartTime) / 1_000_000L); if (downloadStatus) { @@ -269,6 +328,13 @@ private void downloadToFS(String fileName, Path location, String primaryTerm) th // Mark in FileTransferTracker so that the same files are not uploaded at the time of translog sync fileTransferTracker.add(fileName, true); + return metadata; + } + + private void deleteFileIfExists(Path filePath) throws IOException { + if (Files.exists(filePath)) { + Files.delete(filePath); + } } public TranslogTransferMetadata readMetadata() throws IOException { @@ -391,7 +457,11 @@ public void deleteGenerationAsync(long primaryTerm, Set generations, Runna // Add .ckp and .tlog file to translog file list which is located in basePath/ String ckpFileName = Translog.getCommitCheckpointFileName(generation); String translogFileName = Translog.getFilename(generation); - translogFiles.addAll(List.of(ckpFileName, translogFileName)); + if (isTranslogMetadataEnabled == false) { + translogFiles.addAll(List.of(ckpFileName, translogFileName)); + } else { + translogFiles.add(translogFileName); + } }); // Delete the translog and checkpoint files asynchronously deleteTranslogFilesAsync(primaryTerm, translogFiles, onCompletion); diff --git a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java index 5234a09a58541..074186f64a75d 100644 --- a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java +++ b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java @@ -80,6 +80,18 @@ public class RemoteStoreSettings { Property.Dynamic ); + /** + * This setting is used to disable uploading translog.ckp file as metadata to translog.tlog. This setting is effective only for + * repositories that supports metadata read and write with metadata and is applicable for only remote store enabled clusters. + */ + @ExperimentalApi + public static final Setting CLUSTER_REMOTE_STORE_TRANSLOG_METADATA = Setting.boolSetting( + "cluster.remote_store.index.translog.translog_metadata", + true, + Property.NodeScope, + Property.Dynamic + ); + /** * This setting is used to set the remote store blob store path hash algorithm strategy. This setting is effective only for * remote store enabled cluster. This setting will come to effect if the {@link #CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING} @@ -123,6 +135,7 @@ public class RemoteStoreSettings { private volatile RemoteStoreEnums.PathType pathType; private volatile RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm; private volatile int maxRemoteTranslogReaders; + private volatile boolean isTranslogMetadataEnabled; public RemoteStoreSettings(Settings settings, ClusterSettings clusterSettings) { clusterRemoteTranslogBufferInterval = CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(settings); @@ -146,6 +159,9 @@ public RemoteStoreSettings(Settings settings, ClusterSettings clusterSettings) { pathType = clusterSettings.get(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING); clusterSettings.addSettingsUpdateConsumer(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING, this::setPathType); + isTranslogMetadataEnabled = clusterSettings.get(CLUSTER_REMOTE_STORE_TRANSLOG_METADATA); + clusterSettings.addSettingsUpdateConsumer(CLUSTER_REMOTE_STORE_TRANSLOG_METADATA, this::setTranslogMetadataEnabled); + pathHashAlgorithm = clusterSettings.get(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING); clusterSettings.addSettingsUpdateConsumer(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING, this::setPathHashAlgorithm); @@ -205,6 +221,14 @@ private void setPathType(RemoteStoreEnums.PathType pathType) { this.pathType = pathType; } + private void setTranslogMetadataEnabled(boolean isTranslogMetadataEnabled) { + this.isTranslogMetadataEnabled = isTranslogMetadataEnabled; + } + + public boolean isTranslogMetadataEnabled() { + return isTranslogMetadataEnabled; + } + private void setPathHashAlgorithm(RemoteStoreEnums.PathHashAlgorithm pathHashAlgorithm) { this.pathHashAlgorithm = pathHashAlgorithm; } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index c0ceba23ccbf6..9462aeddbd0e4 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -898,7 +898,8 @@ protected Node( systemIndices, forbidPrivateIndexSettings, awarenessReplicaBalance, - remoteStoreSettings + remoteStoreSettings, + repositoriesServiceReference::get ); pluginsService.filterPlugins(Plugin.class) .forEach( diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index 5883a8a37be71..93aac68eb898c 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -472,7 +472,7 @@ public ClusterState execute(ClusterState currentState) { .put(snapshotIndexMetadata.getSettings()) .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) ); - createIndexService.addRemoteStorePathStrategyInCustomData(indexMdBuilder, false); + createIndexService.addRemoteStoreCustomMetadata(indexMdBuilder, false); shardLimitValidator.validateShardLimit( renamedIndexName, snapshotIndexMetadata.getSettings(), diff --git a/server/src/test/java/org/opensearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java b/server/src/test/java/org/opensearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java index 50ffd7322544a..3d6a54055d3d5 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java @@ -740,7 +740,8 @@ public void testRolloverClusterState() throws Exception { systemIndices, false, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + null ); MetadataIndexAliasesService indexAliasesService = new MetadataIndexAliasesService( clusterService, @@ -879,7 +880,8 @@ public void testRolloverClusterStateForDataStream() throws Exception { systemIndices, false, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + null ); MetadataIndexAliasesService indexAliasesService = new MetadataIndexAliasesService( clusterService, @@ -1058,7 +1060,8 @@ public void testRolloverClusterStateForDataStreamNoTemplate() throws Exception { new SystemIndices(emptyMap()), false, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + null ); MetadataIndexAliasesService indexAliasesService = new MetadataIndexAliasesService( clusterService, 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 fad98a6609c3b..0072649e4ca72 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -56,6 +56,7 @@ import org.opensearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.UUIDs; +import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.IndexScopedSettings; @@ -87,6 +88,8 @@ import org.opensearch.indices.SystemIndices; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.remotestore.RemoteStoreNodeService; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.snapshots.EmptySnapshotsInfoService; import org.opensearch.test.ClusterServiceUtils; import org.opensearch.test.OpenSearchTestCase; @@ -116,6 +119,7 @@ import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -156,6 +160,7 @@ import static org.opensearch.node.Node.NODE_ATTRIBUTES; 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.node.remotestore.RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.hamcrest.Matchers.containsString; @@ -176,6 +181,9 @@ public class MetadataCreateIndexServiceTests extends OpenSearchTestCase { private CreateIndexClusterStateUpdateRequest request; private QueryShardContext queryShardContext; private ClusterSettings clusterSettings; + private IndicesService indicesServices; + private RepositoriesService repositoriesService; + private Supplier repositoriesServiceSupplier; private static final String segmentRepositoryNameAttributeKey = NODE_ATTRIBUTES.getKey() + REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; private static final String translogRepositoryNameAttributeKey = NODE_ATTRIBUTES.getKey() @@ -188,6 +196,9 @@ public class MetadataCreateIndexServiceTests extends OpenSearchTestCase { public void setup() throws Exception { super.setUp(); clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + indicesServices = mock(IndicesService.class); + repositoriesServiceSupplier = mock(Supplier.class); + repositoriesService = mock(RepositoriesService.class); } @Before @@ -694,7 +705,7 @@ public void testValidateIndexName() throws Exception { MetadataCreateIndexService checkerService = new MetadataCreateIndexService( Settings.EMPTY, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -705,7 +716,8 @@ public void testValidateIndexName() throws Exception { new SystemIndices(Collections.emptyMap()), false, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + repositoriesServiceSupplier ); validateIndexName( checkerService, @@ -781,7 +793,7 @@ public void testValidateDotIndex() { MetadataCreateIndexService checkerService = new MetadataCreateIndexService( Settings.EMPTY, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -792,7 +804,8 @@ public void testValidateDotIndex() { new SystemIndices(Collections.singletonMap("foo", systemIndexDescriptors)), false, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + repositoriesServiceSupplier ); // Check deprecations assertFalse(checkerService.validateDotIndex(".test2", false)); @@ -1207,7 +1220,7 @@ public void testvalidateIndexSettings() { MetadataCreateIndexService checkerService = new MetadataCreateIndexService( settings, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -1218,7 +1231,8 @@ public void testvalidateIndexSettings() { new SystemIndices(Collections.emptyMap()), true, new AwarenessReplicaBalance(settings, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + repositoriesServiceSupplier ); List validationErrors = checkerService.getIndexSettingsValidationErrors(settings, false, Optional.empty()); @@ -1327,7 +1341,7 @@ public void testClusterForceReplicationTypeInValidateIndexSettings() { final MetadataCreateIndexService checkerService = new MetadataCreateIndexService( forceClusterSettingEnabled, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -1338,7 +1352,8 @@ public void testClusterForceReplicationTypeInValidateIndexSettings() { new SystemIndices(Collections.emptyMap()), true, new AwarenessReplicaBalance(forceClusterSettingEnabled, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + repositoriesServiceSupplier ); // Use DOCUMENT replication type setting for index creation final Settings indexSettings = Settings.builder().put(INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.DOCUMENT).build(); @@ -1453,7 +1468,7 @@ public void testRemoteStoreDisabledByUserIndexSettings() { MetadataCreateIndexService checkerService = new MetadataCreateIndexService( Settings.EMPTY, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -1464,7 +1479,8 @@ public void testRemoteStoreDisabledByUserIndexSettings() { new SystemIndices(Collections.emptyMap()), true, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + repositoriesServiceSupplier ); final List validationErrors = checkerService.getIndexSettingsValidationErrors( @@ -1488,7 +1504,7 @@ public void testRemoteStoreOverrideSegmentRepoIndexSettings() { MetadataCreateIndexService checkerService = new MetadataCreateIndexService( Settings.EMPTY, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -1499,7 +1515,8 @@ public void testRemoteStoreOverrideSegmentRepoIndexSettings() { new SystemIndices(Collections.emptyMap()), true, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + repositoriesServiceSupplier ); final List validationErrors = checkerService.getIndexSettingsValidationErrors( @@ -1528,7 +1545,7 @@ public void testRemoteStoreOverrideTranslogRepoIndexSettings() { MetadataCreateIndexService checkerService = new MetadataCreateIndexService( Settings.EMPTY, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -1539,7 +1556,8 @@ public void testRemoteStoreOverrideTranslogRepoIndexSettings() { new SystemIndices(Collections.emptyMap()), true, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + repositoriesServiceSupplier ); final List validationErrors = checkerService.getIndexSettingsValidationErrors( @@ -1746,10 +1764,16 @@ private IndexMetadata testRemoteCustomData(boolean remoteStoreEnabled, PathType RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); ThreadPool threadPool = new TestThreadPool(getTestName()); + BlobStoreRepository repositoryMock = mock(BlobStoreRepository.class); + when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); + when(repositoriesService.repository(getRemoteStoreTranslogRepo(settings))).thenReturn(repositoryMock); + BlobStore blobStoreMock = mock(BlobStore.class); + when(repositoryMock.blobStore()).thenReturn(blobStoreMock); + when(blobStoreMock.isBlobMetadataEnabled()).thenReturn(randomBoolean()); MetadataCreateIndexService metadataCreateIndexService = new MetadataCreateIndexService( settings, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -1760,7 +1784,8 @@ private IndexMetadata testRemoteCustomData(boolean remoteStoreEnabled, PathType new SystemIndices(Collections.emptyMap()), true, new AwarenessReplicaBalance(settings, clusterService.getClusterSettings()), - remoteStoreSettings + remoteStoreSettings, + repositoriesServiceSupplier ); CreateIndexClusterStateUpdateRequest request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); Settings indexSettings = Settings.builder() @@ -1873,7 +1898,7 @@ public void testIndexLifecycleNameSetting() { MetadataCreateIndexService checkerService = new MetadataCreateIndexService( Settings.EMPTY, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false, clusterService), @@ -1884,7 +1909,8 @@ public void testIndexLifecycleNameSetting() { new SystemIndices(Collections.emptyMap()), true, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + repositoriesServiceSupplier ); final List validationErrors = checkerService.getIndexSettingsValidationErrors(ilnSetting, true, Optional.empty()); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexTemplateServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexTemplateServiceTests.java index 0b99ffac67ee8..6643d6e13289b 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexTemplateServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexTemplateServiceTests.java @@ -57,8 +57,10 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.indices.DefaultRemoteStoreSettings; import org.opensearch.indices.IndexTemplateMissingException; +import org.opensearch.indices.IndicesService; import org.opensearch.indices.InvalidIndexTemplateException; import org.opensearch.indices.SystemIndices; +import org.opensearch.repositories.RepositoriesService; import org.opensearch.test.OpenSearchSingleNodeTestCase; import java.io.IOException; @@ -2039,10 +2041,13 @@ private static List putTemplate(NamedXContentRegistry xContentRegistr when(clusterService.state()).thenReturn(clusterState); when(clusterService.getSettings()).thenReturn(settings); when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + IndicesService indicesServices = mock(IndicesService.class); + RepositoriesService repositoriesService = mock(RepositoriesService.class); + // when(indicesServices.getRepositoriesServiceSupplier()).thenReturn(() -> repositoriesService); MetadataCreateIndexService createIndexService = new MetadataCreateIndexService( Settings.EMPTY, clusterService, - null, + indicesServices, null, null, createTestShardLimitService(randomIntBetween(1, 1000), false), @@ -2053,7 +2058,8 @@ private static List putTemplate(NamedXContentRegistry xContentRegistr new SystemIndices(Collections.emptyMap()), true, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + null ); MetadataIndexTemplateService service = new MetadataIndexTemplateService( clusterService, diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java index d8220c93e4eeb..a2cd3f6b803e0 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteMigrationIndexMetadataUpdaterTests.java @@ -163,7 +163,7 @@ public void testMaybeUpdateRemoteStorePathStrategyExecutes() { .build(), logger ); - migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy(builder, indexName); + migrationIndexMetadataUpdater.maybeUpdateRemoteStoreCustomMetadata(builder, indexName); assertCustomPathMetadataIsPresent(builder.build()); } @@ -186,7 +186,7 @@ public void testMaybeUpdateRemoteStorePathStrategyDoesNotExecute() { logger ); - migrationIndexMetadataUpdater.maybeUpdateRemoteStorePathStrategy(builder, indexName); + migrationIndexMetadataUpdater.maybeUpdateRemoteStoreCustomMetadata(builder, indexName); assertCustomPathMetadataIsPresent(builder.build()); } @@ -298,7 +298,14 @@ public static Metadata createIndexMetadataWithRemoteStoreSettings(String indexNa ) .putCustom( REMOTE_STORE_CUSTOM_KEY, - Map.of(RemoteStoreEnums.PathType.NAME, "dummy", RemoteStoreEnums.PathHashAlgorithm.NAME, "dummy") + Map.of( + RemoteStoreEnums.PathType.NAME, + "dummy", + RemoteStoreEnums.PathHashAlgorithm.NAME, + "dummy", + IndexMetadata.TRANSLOG_METADATA_KEY, + "dummy" + ) ) .build(); return Metadata.builder().put(indexMetadata).build(); diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolverTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolverTests.java new file mode 100644 index 0000000000000..7e702ad3773e8 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreCustomMetadataResolverTests.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.index.remote; + +import org.opensearch.Version; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; +import org.opensearch.index.remote.RemoteStoreEnums.PathType; +import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; + +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.getRemoteStoreTranslogRepo; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class RemoteStoreCustomMetadataResolverTests extends OpenSearchTestCase { + + RepositoriesService repositoriesService = mock(RepositoriesService.class); + Settings settings = Settings.EMPTY; + + public void testGetPathStrategyMinVersionOlder() { + Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), randomFrom(PathType.values())).build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + RemoteStoreCustomMetadataResolver resolver = new RemoteStoreCustomMetadataResolver( + remoteStoreSettings, + () -> Version.V_2_13_0, + () -> repositoriesService, + settings + ); + assertEquals(PathType.FIXED, resolver.getPathStrategy().getType()); + assertNull(resolver.getPathStrategy().getHashAlgorithm()); + } + + public void testGetPathStrategyMinVersionNewer() { + PathType pathType = randomFrom(PathType.values()); + Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), pathType).build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + RemoteStoreCustomMetadataResolver resolver = new RemoteStoreCustomMetadataResolver( + remoteStoreSettings, + () -> Version.V_2_14_0, + () -> repositoriesService, + settings + ); + assertEquals(pathType, resolver.getPathStrategy().getType()); + if (pathType.requiresHashAlgorithm()) { + assertNotNull(resolver.getPathStrategy().getHashAlgorithm()); + } else { + assertNull(resolver.getPathStrategy().getHashAlgorithm()); + } + } + + public void testGetPathStrategyStrategy() { + // FIXED type + Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.FIXED).build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + RemoteStoreCustomMetadataResolver resolver = new RemoteStoreCustomMetadataResolver( + remoteStoreSettings, + () -> Version.V_2_14_0, + () -> repositoriesService, + settings + ); + assertEquals(PathType.FIXED, resolver.getPathStrategy().getType()); + + // FIXED type with hash algorithm + settings = Settings.builder() + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.FIXED) + .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), randomFrom(PathHashAlgorithm.values())) + .build(); + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + resolver = new RemoteStoreCustomMetadataResolver(remoteStoreSettings, () -> Version.V_2_14_0, () -> repositoriesService, settings); + assertEquals(PathType.FIXED, resolver.getPathStrategy().getType()); + + // HASHED_PREFIX type with FNV_1A_COMPOSITE + settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX).build(); + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + resolver = new RemoteStoreCustomMetadataResolver(remoteStoreSettings, () -> Version.V_2_14_0, () -> repositoriesService, settings); + assertEquals(PathType.HASHED_PREFIX, resolver.getPathStrategy().getType()); + assertEquals(PathHashAlgorithm.FNV_1A_COMPOSITE_1, resolver.getPathStrategy().getHashAlgorithm()); + + // HASHED_PREFIX type with FNV_1A_COMPOSITE + settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX).build(); + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + resolver = new RemoteStoreCustomMetadataResolver(remoteStoreSettings, () -> Version.V_2_14_0, () -> repositoriesService, settings); + assertEquals(PathType.HASHED_PREFIX, resolver.getPathStrategy().getType()); + assertEquals(PathHashAlgorithm.FNV_1A_COMPOSITE_1, resolver.getPathStrategy().getHashAlgorithm()); + + // HASHED_PREFIX type with FNV_1A_BASE64 + settings = Settings.builder() + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX) + .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), PathHashAlgorithm.FNV_1A_BASE64) + .build(); + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + resolver = new RemoteStoreCustomMetadataResolver(remoteStoreSettings, () -> Version.V_2_14_0, () -> repositoriesService, settings); + assertEquals(PathType.HASHED_PREFIX, resolver.getPathStrategy().getType()); + assertEquals(PathHashAlgorithm.FNV_1A_BASE64, resolver.getPathStrategy().getHashAlgorithm()); + + // HASHED_PREFIX type with FNV_1A_BASE64 + settings = Settings.builder() + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX) + .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), PathHashAlgorithm.FNV_1A_BASE64) + .build(); + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + resolver = new RemoteStoreCustomMetadataResolver(remoteStoreSettings, () -> Version.V_2_14_0, () -> repositoriesService, settings); + assertEquals(PathType.HASHED_PREFIX, resolver.getPathStrategy().getType()); + assertEquals(PathHashAlgorithm.FNV_1A_BASE64, resolver.getPathStrategy().getHashAlgorithm()); + } + + public void testGetPathStrategyStrategyWithDynamicUpdate() { + + // Default value + Settings settings = Settings.builder().build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + RemoteStoreCustomMetadataResolver resolver = new RemoteStoreCustomMetadataResolver( + remoteStoreSettings, + () -> Version.V_2_14_0, + () -> repositoriesService, + settings + ); + assertEquals(PathType.FIXED, resolver.getPathStrategy().getType()); + assertNull(resolver.getPathStrategy().getHashAlgorithm()); + + // Set HASHED_PREFIX with default hash algorithm + clusterSettings.applySettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX).build() + ); + assertEquals(PathType.HASHED_PREFIX, resolver.getPathStrategy().getType()); + assertEquals(PathHashAlgorithm.FNV_1A_COMPOSITE_1, resolver.getPathStrategy().getHashAlgorithm()); + + // Set HASHED_PREFIX with FNV_1A_BASE64 hash algorithm + clusterSettings.applySettings( + Settings.builder() + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX) + .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), PathHashAlgorithm.FNV_1A_BASE64) + .build() + ); + assertEquals(PathType.HASHED_PREFIX, resolver.getPathStrategy().getType()); + assertEquals(PathHashAlgorithm.FNV_1A_BASE64, resolver.getPathStrategy().getHashAlgorithm()); + + // Set HASHED_INFIX with default hash algorithm + clusterSettings.applySettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_INFIX).build() + ); + assertEquals(PathType.HASHED_INFIX, resolver.getPathStrategy().getType()); + assertEquals(PathHashAlgorithm.FNV_1A_COMPOSITE_1, resolver.getPathStrategy().getHashAlgorithm()); + + // Set HASHED_INFIX with FNV_1A_BASE64 hash algorithm + clusterSettings.applySettings( + Settings.builder() + .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_INFIX) + .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), PathHashAlgorithm.FNV_1A_BASE64) + .build() + ); + assertEquals(PathType.HASHED_INFIX, resolver.getPathStrategy().getType()); + assertEquals(PathHashAlgorithm.FNV_1A_BASE64, resolver.getPathStrategy().getHashAlgorithm()); + } + + public void testTranslogMetadataAllowedTrueWithMinVersionNewer() { + Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_TRANSLOG_METADATA.getKey(), true).build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + BlobStoreRepository repositoryMock = mock(BlobStoreRepository.class); + when(repositoriesService.repository(getRemoteStoreTranslogRepo(settings))).thenReturn(repositoryMock); + BlobStore blobStoreMock = mock(BlobStore.class); + when(repositoryMock.blobStore()).thenReturn(blobStoreMock); + when(blobStoreMock.isBlobMetadataEnabled()).thenReturn(true); + RemoteStoreCustomMetadataResolver resolver = new RemoteStoreCustomMetadataResolver( + remoteStoreSettings, + () -> Version.CURRENT, + () -> repositoriesService, + settings + ); + assertTrue(resolver.isTranslogMetadataEnabled()); + } + + public void testTranslogMetadataAllowedFalseWithMinVersionNewer() { + Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_TRANSLOG_METADATA.getKey(), false).build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + RemoteStoreCustomMetadataResolver resolver = new RemoteStoreCustomMetadataResolver( + remoteStoreSettings, + () -> Version.CURRENT, + () -> repositoriesService, + settings + ); + assertFalse(resolver.isTranslogMetadataEnabled()); + } + + public void testTranslogMetadataAllowedMinVersionOlder() { + Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_TRANSLOG_METADATA.getKey(), randomBoolean()).build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); + RemoteStoreCustomMetadataResolver resolver = new RemoteStoreCustomMetadataResolver( + remoteStoreSettings, + () -> Version.V_2_14_0, + () -> repositoriesService, + settings + ); + assertFalse(resolver.isTranslogMetadataEnabled()); + } + +} diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStorePathStrategyResolverTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStorePathStrategyResolverTests.java deleted file mode 100644 index de61c902bf13e..0000000000000 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStorePathStrategyResolverTests.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.index.remote; - -import org.opensearch.Version; -import org.opensearch.common.settings.ClusterSettings; -import org.opensearch.common.settings.Settings; -import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; -import org.opensearch.index.remote.RemoteStoreEnums.PathType; -import org.opensearch.indices.RemoteStoreSettings; -import org.opensearch.test.OpenSearchTestCase; - -import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; -import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; - -public class RemoteStorePathStrategyResolverTests extends OpenSearchTestCase { - - public void testGetMinVersionOlder() { - Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), randomFrom(PathType.values())).build(); - ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - RemoteStorePathStrategyResolver resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_13_0); - assertEquals(PathType.FIXED, resolver.get().getType()); - assertNull(resolver.get().getHashAlgorithm()); - } - - public void testGetMinVersionNewer() { - PathType pathType = randomFrom(PathType.values()); - Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), pathType).build(); - ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - RemoteStorePathStrategyResolver resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_14_0); - assertEquals(pathType, resolver.get().getType()); - if (pathType.requiresHashAlgorithm()) { - assertNotNull(resolver.get().getHashAlgorithm()); - } else { - assertNull(resolver.get().getHashAlgorithm()); - } - } - - public void testGetStrategy() { - // FIXED type - Settings settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.FIXED).build(); - ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - RemoteStorePathStrategyResolver resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_14_0); - assertEquals(PathType.FIXED, resolver.get().getType()); - - // FIXED type with hash algorithm - settings = Settings.builder() - .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.FIXED) - .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), randomFrom(PathHashAlgorithm.values())) - .build(); - clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_14_0); - assertEquals(PathType.FIXED, resolver.get().getType()); - - // HASHED_PREFIX type with FNV_1A_COMPOSITE - settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX).build(); - clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_14_0); - assertEquals(PathType.HASHED_PREFIX, resolver.get().getType()); - assertEquals(PathHashAlgorithm.FNV_1A_COMPOSITE_1, resolver.get().getHashAlgorithm()); - - // HASHED_PREFIX type with FNV_1A_COMPOSITE - settings = Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX).build(); - clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_14_0); - assertEquals(PathType.HASHED_PREFIX, resolver.get().getType()); - assertEquals(PathHashAlgorithm.FNV_1A_COMPOSITE_1, resolver.get().getHashAlgorithm()); - - // HASHED_PREFIX type with FNV_1A_BASE64 - settings = Settings.builder() - .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX) - .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), PathHashAlgorithm.FNV_1A_BASE64) - .build(); - clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_14_0); - assertEquals(PathType.HASHED_PREFIX, resolver.get().getType()); - assertEquals(PathHashAlgorithm.FNV_1A_BASE64, resolver.get().getHashAlgorithm()); - - // HASHED_PREFIX type with FNV_1A_BASE64 - settings = Settings.builder() - .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX) - .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), PathHashAlgorithm.FNV_1A_BASE64) - .build(); - clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_14_0); - assertEquals(PathType.HASHED_PREFIX, resolver.get().getType()); - assertEquals(PathHashAlgorithm.FNV_1A_BASE64, resolver.get().getHashAlgorithm()); - } - - public void testGetStrategyWithDynamicUpdate() { - - // Default value - Settings settings = Settings.builder().build(); - ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings(settings, clusterSettings); - RemoteStorePathStrategyResolver resolver = new RemoteStorePathStrategyResolver(remoteStoreSettings, () -> Version.V_2_14_0); - assertEquals(PathType.FIXED, resolver.get().getType()); - assertNull(resolver.get().getHashAlgorithm()); - - // Set HASHED_PREFIX with default hash algorithm - clusterSettings.applySettings( - Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX).build() - ); - assertEquals(PathType.HASHED_PREFIX, resolver.get().getType()); - assertEquals(PathHashAlgorithm.FNV_1A_COMPOSITE_1, resolver.get().getHashAlgorithm()); - - // Set HASHED_PREFIX with FNV_1A_BASE64 hash algorithm - clusterSettings.applySettings( - Settings.builder() - .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_PREFIX) - .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), PathHashAlgorithm.FNV_1A_BASE64) - .build() - ); - assertEquals(PathType.HASHED_PREFIX, resolver.get().getType()); - assertEquals(PathHashAlgorithm.FNV_1A_BASE64, resolver.get().getHashAlgorithm()); - - // Set HASHED_INFIX with default hash algorithm - clusterSettings.applySettings( - Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_INFIX).build() - ); - assertEquals(PathType.HASHED_INFIX, resolver.get().getType()); - assertEquals(PathHashAlgorithm.FNV_1A_COMPOSITE_1, resolver.get().getHashAlgorithm()); - - // Set HASHED_INFIX with FNV_1A_BASE64 hash algorithm - clusterSettings.applySettings( - Settings.builder() - .put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), PathType.HASHED_INFIX) - .put(CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING.getKey(), PathHashAlgorithm.FNV_1A_BASE64) - .build() - ); - assertEquals(PathType.HASHED_INFIX, resolver.get().getType()); - assertEquals(PathHashAlgorithm.FNV_1A_BASE64, resolver.get().getHashAlgorithm()); - } -} diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java index c1fc0cdaa0d3b..59c3d3dccdd0f 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java @@ -8,9 +8,13 @@ package org.opensearch.index.remote; +import org.opensearch.Version; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.support.PlainBlobMetadata; +import org.opensearch.common.settings.Settings; import org.opensearch.index.shard.IndexShardTestUtils; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.translog.transfer.TranslogTransferMetadata; @@ -26,7 +30,9 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; import static org.opensearch.index.remote.RemoteStoreUtils.URL_BASE64_CHARSET; +import static org.opensearch.index.remote.RemoteStoreUtils.determineTranslogMetadataEnabled; import static org.opensearch.index.remote.RemoteStoreUtils.longToCompositeBase64AndBinaryEncoding; import static org.opensearch.index.remote.RemoteStoreUtils.longToUrlBase64; import static org.opensearch.index.remote.RemoteStoreUtils.urlBase64ToLong; @@ -40,6 +46,7 @@ public class RemoteStoreUtilsTests extends OpenSearchTestCase { private static Map BASE64_CHARSET_IDX_MAP; + private static String index = "test-index"; static { Map charToIndexMap = new HashMap<>(); @@ -341,4 +348,54 @@ static long compositeUrlBase64BinaryEncodingToLong(String encodedValue) { String binaryString = base64PartBinary + encodedValue.substring(1); return new BigInteger(binaryString, 2).longValue(); } + + public void testDeterdetermineTranslogMetadataEnabledWhenTrue() { + Metadata metadata = createIndexMetadataWithRemoteStoreSettings(index, 1); + IndexMetadata indexMetadata = metadata.index(index); + assertTrue(determineTranslogMetadataEnabled(indexMetadata)); + } + + public void testDeterdetermineTranslogMetadataEnabledWhenFalse() { + Metadata metadata = createIndexMetadataWithRemoteStoreSettings(index, 0); + IndexMetadata indexMetadata = metadata.index(index); + assertFalse(determineTranslogMetadataEnabled(indexMetadata)); + } + + public void testDeterdetermineTranslogMetadataEnabledWhenKeyNotFound() { + Metadata metadata = createIndexMetadataWithRemoteStoreSettings(index, 2); + IndexMetadata indexMetadata = metadata.index(index); + assertThrows(AssertionError.class, () -> determineTranslogMetadataEnabled(indexMetadata)); + } + + private static Metadata createIndexMetadataWithRemoteStoreSettings(String indexName, int option) { + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(indexName); + indexMetadata.settings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.getKey(), true) + .put(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), "dummy-tlog-repo") + .put(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.getKey(), "dummy-segment-repo") + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "SEGMENT") + .build() + ).putCustom(REMOTE_STORE_CUSTOM_KEY, getCustomDataMap(option)).build(); + return Metadata.builder().put(indexMetadata).build(); + } + + private static Map getCustomDataMap(int option) { + if (option > 1) { + return Map.of(); + } + String value = (option == 1) ? "true" : "false"; + return Map.of( + RemoteStoreEnums.PathType.NAME, + "dummy", + RemoteStoreEnums.PathHashAlgorithm.NAME, + "dummy", + IndexMetadata.TRANSLOG_METADATA_KEY, + value + ); + } + } diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java index e4f5a454b15f6..3ed90c0837663 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java @@ -26,14 +26,21 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; +import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import java.util.Base64; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import static org.opensearch.index.translog.transfer.TranslogTransferManager.CHECKPOINT_FILE_DATA_KEY; + public class BlobStoreTransferServiceTests extends OpenSearchTestCase { private ThreadPool threadPool; @@ -144,4 +151,37 @@ private Environment createEnvironment() { .build() ); } + + public void testBuildTransferFileMetadata_EmptyInputStream() throws IOException { + InputStream emptyInputStream = new ByteArrayInputStream(new byte[0]); + Map metadata = BlobStoreTransferService.buildTransferFileMetadata(emptyInputStream); + assertTrue(metadata.containsKey(CHECKPOINT_FILE_DATA_KEY)); + assertEquals("", metadata.get(CHECKPOINT_FILE_DATA_KEY)); + } + + public void testBuildTransferFileMetadata_NonEmptyInputStream() throws IOException { + String inputData = "This is a test input stream."; + InputStream inputStream = new ByteArrayInputStream(inputData.getBytes(StandardCharsets.UTF_8)); + Map metadata = BlobStoreTransferService.buildTransferFileMetadata(inputStream); + assertTrue(metadata.containsKey(CHECKPOINT_FILE_DATA_KEY)); + String expectedBase64String = Base64.getEncoder().encodeToString(inputData.getBytes(StandardCharsets.UTF_8)); + assertEquals(expectedBase64String, metadata.get(CHECKPOINT_FILE_DATA_KEY)); + } + + public void testBuildTransferFileMetadata_InputStreamExceedsLimit() { + byte[] largeData = new byte[1025]; // 1025 bytes, exceeding the 1KB limit + InputStream largeInputStream = new ByteArrayInputStream(largeData); + IOException exception = assertThrows(IOException.class, () -> BlobStoreTransferService.buildTransferFileMetadata(largeInputStream)); + assertEquals(exception.getMessage(), "Input stream exceeds 1KB limit"); + } + + public void testBuildTransferFileMetadata_SmallInputStreamOptimization() throws IOException { + String inputData = "Small input"; + InputStream inputStream = new ByteArrayInputStream(inputData.getBytes(StandardCharsets.UTF_8)); + Map metadata = BlobStoreTransferService.buildTransferFileMetadata(inputStream); + assertTrue(metadata.containsKey(CHECKPOINT_FILE_DATA_KEY)); + String expectedBase64String = Base64.getEncoder().encodeToString(inputData.getBytes(StandardCharsets.UTF_8)); + assertEquals(expectedBase64String, metadata.get(CHECKPOINT_FILE_DATA_KEY)); + } + } diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java index 8b3fc6651a505..c6f9838ad2d52 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java @@ -15,6 +15,7 @@ import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.InputStreamWithMetadata; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.blobstore.support.PlainBlobMetadata; import org.opensearch.common.collect.Tuple; @@ -41,9 +42,12 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.Base64; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -53,15 +57,17 @@ import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.TRANSLOG; import static org.opensearch.index.remote.RemoteStoreEnums.DataType.METADATA; +import static org.opensearch.index.translog.transfer.TranslogTransferManager.CHECKPOINT_FILE_DATA_KEY; import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyMap; -import static org.mockito.ArgumentMatchers.anySet; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyMap; +import static org.mockito.Mockito.anySet; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -81,6 +87,7 @@ public class TranslogTransferManagerTests extends OpenSearchTestCase { FileTransferTracker tracker; TranslogTransferManager translogTransferManager; long delayForBlobDownload; + boolean isTranslogMetadataEnabled; @Override public void setUp() throws Exception { @@ -97,6 +104,7 @@ public void setUp() throws Exception { tlogBytes = "Hello Translog".getBytes(StandardCharsets.UTF_8); ckpBytes = "Hello Checkpoint".getBytes(StandardCharsets.UTF_8); tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0), remoteTranslogTransferTracker); + isTranslogMetadataEnabled = false; translogTransferManager = new TranslogTransferManager( shardId, transferService, @@ -104,7 +112,8 @@ public void setUp() throws Exception { remoteBaseTransferPath.add(METADATA.getName()), tracker, remoteTranslogTransferTracker, - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + isTranslogMetadataEnabled ); delayForBlobDownload = 1; @@ -170,7 +179,8 @@ public void onFailure(TransferFileSnapshot fileSnapshot, Exception e) { remoteBaseTransferPath.add(METADATA.getName()), fileTransferTracker, remoteTranslogTransferTracker, - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + isTranslogMetadataEnabled ); assertTrue(translogTransferManager.transferSnapshot(createTransferSnapshot(), new TranslogTransferListener() { @@ -222,7 +232,8 @@ public void testTransferSnapshotOnUploadTimeout() throws Exception { remoteBaseTransferPath.add(METADATA.getName()), fileTransferTracker, remoteTranslogTransferTracker, - remoteStoreSettings + remoteStoreSettings, + isTranslogMetadataEnabled ); SetOnce exception = new SetOnce<>(); translogTransferManager.transferSnapshot(createTransferSnapshot(), new TranslogTransferListener() { @@ -265,7 +276,8 @@ public void testTransferSnapshotOnThreadInterrupt() throws Exception { remoteBaseTransferPath.add(METADATA.getName()), fileTransferTracker, remoteTranslogTransferTracker, - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + isTranslogMetadataEnabled ); SetOnce exception = new SetOnce<>(); @@ -297,64 +309,66 @@ public void onUploadFailed(TransferSnapshot transferSnapshot, Exception ex) { uploadThread.get().interrupt(); } - private TransferSnapshot createTransferSnapshot() { - return new TransferSnapshot() { - @Override - public Set getCheckpointFileSnapshots() { - try { - return Set.of( - new CheckpointFileSnapshot( - primaryTerm, - generation, - minTranslogGeneration, - createTempFile(Translog.TRANSLOG_FILE_PREFIX + generation, Translog.CHECKPOINT_SUFFIX), - null - ), - new CheckpointFileSnapshot( - primaryTerm, - generation, - minTranslogGeneration, - createTempFile(Translog.TRANSLOG_FILE_PREFIX + (generation - 1), Translog.CHECKPOINT_SUFFIX), - null - ) - ); - } catch (IOException e) { - throw new AssertionError("Failed to create temp file", e); + private TransferSnapshot createTransferSnapshot() throws IOException { + try { + CheckpointFileSnapshot checkpointFileSnapshot1 = new CheckpointFileSnapshot( + primaryTerm, + generation, + minTranslogGeneration, + createTempFile(Translog.TRANSLOG_FILE_PREFIX + generation, Translog.CHECKPOINT_SUFFIX), + null + ); + CheckpointFileSnapshot checkpointFileSnapshot2 = new CheckpointFileSnapshot( + primaryTerm, + generation, + minTranslogGeneration, + createTempFile(Translog.TRANSLOG_FILE_PREFIX + (generation - 1), Translog.CHECKPOINT_SUFFIX), + null + ); + TranslogFileSnapshot translogFileSnapshot1 = new TranslogFileSnapshot( + primaryTerm, + generation, + createTempFile(Translog.TRANSLOG_FILE_PREFIX + generation, Translog.TRANSLOG_FILE_SUFFIX), + null + ); + TranslogFileSnapshot translogFileSnapshot2 = new TranslogFileSnapshot( + primaryTerm, + generation - 1, + createTempFile(Translog.TRANSLOG_FILE_PREFIX + (generation - 1), Translog.TRANSLOG_FILE_SUFFIX), + null + ); + + return new TransferSnapshot() { + @Override + public Set getCheckpointFileSnapshots() { + return Set.of(checkpointFileSnapshot1, checkpointFileSnapshot2); } - } - @Override - public Set getTranslogFileSnapshots() { - try { - return Set.of( - new TranslogFileSnapshot( - primaryTerm, - generation, - createTempFile(Translog.TRANSLOG_FILE_PREFIX + generation, Translog.TRANSLOG_FILE_SUFFIX), - null - ), - new TranslogFileSnapshot( - primaryTerm, - generation - 1, - createTempFile(Translog.TRANSLOG_FILE_PREFIX + (generation - 1), Translog.TRANSLOG_FILE_SUFFIX), - null - ) - ); - } catch (IOException e) { - throw new AssertionError("Failed to create temp file", e); + @Override + public Set getTranslogFileSnapshots() { + return Set.of(translogFileSnapshot1, translogFileSnapshot2); } - } - @Override - public TranslogTransferMetadata getTranslogTransferMetadata() { - return new TranslogTransferMetadata(primaryTerm, generation, minTranslogGeneration, randomInt(5)); - } + @Override + public TranslogTransferMetadata getTranslogTransferMetadata() { + return new TranslogTransferMetadata(primaryTerm, generation, minTranslogGeneration, randomInt(5)); + } - @Override - public String toString() { - return "test-to-string"; - } - }; + @Override + public Set getTranslogFileSnapshotWithMetadata() throws IOException { + translogFileSnapshot1.setMetadataFileInputStream(checkpointFileSnapshot1.inputStream()); + translogFileSnapshot2.setMetadataFileInputStream(checkpointFileSnapshot2.inputStream()); + return Set.of(translogFileSnapshot1, translogFileSnapshot2); + } + + @Override + public String toString() { + return "test-to-string"; + } + }; + } catch (Exception e) { + throw new IOException("Failed to create transfer snapshot"); + } } public void testReadMetadataNoFile() throws IOException { @@ -502,7 +516,8 @@ public void testDeleteTranslogSuccess() throws Exception { remoteBaseTransferPath.add(METADATA.getName()), tracker, remoteTranslogTransferTracker, - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + isTranslogMetadataEnabled ); String translogFile = "translog-19.tlog", checkpointFile = "translog-19.ckp"; tracker.add(translogFile, true); @@ -567,7 +582,8 @@ public void testDeleteTranslogFailure() throws Exception { remoteBaseTransferPath.add(METADATA.getName()), tracker, remoteTranslogTransferTracker, - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + isTranslogMetadataEnabled ); String translogFile = "translog-19.tlog", checkpointFile = "translog-19.ckp"; tracker.add(translogFile, true); @@ -624,4 +640,117 @@ public void testMetadataConflict() throws InterruptedException { assertThrows(RuntimeException.class, translogTransferManager::readMetadata); } + + // tests for cases when ckp is stored as translog metadata. + public void testTransferSnapshotWithTranslogMetadata() throws Exception { + AtomicInteger fileTransferSucceeded = new AtomicInteger(); + AtomicInteger fileTransferFailed = new AtomicInteger(); + AtomicInteger translogTransferSucceeded = new AtomicInteger(); + AtomicInteger translogTransferFailed = new AtomicInteger(); + + isTranslogMetadataEnabled = true; + + doNothing().when(transferService) + .uploadBlob( + any(TransferFileSnapshot.class), + Mockito.eq(remoteBaseTransferPath.add(String.valueOf(primaryTerm))), + any(WritePriority.class) + ); + doAnswer(invocationOnMock -> { + ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; + Set transferFileSnapshots = (Set) invocationOnMock.getArguments()[0]; + transferFileSnapshots.forEach(transferFileSnapshot -> { + assertNotNull(transferFileSnapshot.getMetadataFileInputStream()); + listener.onResponse(transferFileSnapshot); + }); + return null; + }).when(transferService).uploadBlobs(anySet(), anyMap(), any(ActionListener.class), any(WritePriority.class)); + + FileTransferTracker fileTransferTracker = new FileTransferTracker( + new ShardId("index", "indexUUid", 0), + remoteTranslogTransferTracker + ) { + @Override + public void onSuccess(TransferFileSnapshot fileSnapshot) { + fileTransferSucceeded.incrementAndGet(); + super.onSuccess(fileSnapshot); + } + + @Override + public void onFailure(TransferFileSnapshot fileSnapshot, Exception e) { + fileTransferFailed.incrementAndGet(); + super.onFailure(fileSnapshot, e); + } + + }; + + translogTransferManager = new TranslogTransferManager( + shardId, + transferService, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), + fileTransferTracker, + remoteTranslogTransferTracker, + DefaultRemoteStoreSettings.INSTANCE, + isTranslogMetadataEnabled + ); + + assertTrue(translogTransferManager.transferSnapshot(createTransferSnapshot(), new TranslogTransferListener() { + @Override + public void onUploadComplete(TransferSnapshot transferSnapshot) { + translogTransferSucceeded.incrementAndGet(); + } + + @Override + public void onUploadFailed(TransferSnapshot transferSnapshot, Exception ex) { + translogTransferFailed.incrementAndGet(); + } + })); + assertEquals(2, fileTransferSucceeded.get()); + assertEquals(0, fileTransferFailed.get()); + assertEquals(1, translogTransferSucceeded.get()); + assertEquals(0, translogTransferFailed.get()); + assertEquals(2, fileTransferTracker.allUploaded().size()); + } + + public void testDownloadTranslogWithMetadata() throws IOException { + isTranslogMetadataEnabled = true; + translogTransferManager = new TranslogTransferManager( + shardId, + transferService, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), + tracker, + remoteTranslogTransferTracker, + DefaultRemoteStoreSettings.INSTANCE, + isTranslogMetadataEnabled + ); + Path location = createTempDir(); + assertFalse(Files.exists(location.resolve("translog-23.tlog"))); + assertFalse(Files.exists(location.resolve("translog-23.ckp"))); + mockDownloadBlobWithMetadataResponse(); + translogTransferManager.downloadTranslog("12", "23", location); + verify(transferService, times(0)).downloadBlob(any(BlobPath.class), eq("translog-23.tlog")); + verify(transferService, times(0)).downloadBlob(any(BlobPath.class), eq("translog-23.ckp")); + verify(transferService, times(1)).downloadBlobWithMetadata(any(BlobPath.class), eq("translog-23.tlog")); + assertTrue(Files.exists(location.resolve("translog-23.tlog"))); + assertTrue(Files.exists(location.resolve("translog-23.ckp"))); + assertTlogCkpDownloadStatsWithMetadata(); + } + + private void mockDownloadBlobWithMetadataResponse() throws IOException { + Map metadata = new HashMap<>(); + String ckpDataString = Base64.getEncoder().encodeToString(ckpBytes); + metadata.put(CHECKPOINT_FILE_DATA_KEY, ckpDataString); + when(transferService.downloadBlobWithMetadata(any(BlobPath.class), eq("translog-23.tlog"))).thenAnswer(invocation -> { + Thread.sleep(delayForBlobDownload); + return new InputStreamWithMetadata(new ByteArrayInputStream(tlogBytes), metadata); + }); + } + + private void assertTlogCkpDownloadStatsWithMetadata() { + assertEquals(tlogBytes.length, remoteTranslogTransferTracker.getDownloadBytesSucceeded()); + // Expect delay for both tlog and ckp file + assertTrue(remoteTranslogTransferTracker.getTotalDownloadTimeInMillis() >= delayForBlobDownload); + } } diff --git a/server/src/test/java/org/opensearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/opensearch/indices/cluster/ClusterStateChanges.java index 17bd821ed0c8c..8f1d58cf201e9 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/opensearch/indices/cluster/ClusterStateChanges.java @@ -314,7 +314,8 @@ public IndexMetadata upgradeIndexMetadata(IndexMetadata indexMetadata, Version m systemIndices, true, awarenessReplicaBalance, - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + null ); transportCloseIndexAction = new TransportCloseIndexAction( diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 3a07aa413e2a2..460aaa08a224d 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2171,7 +2171,8 @@ public void onFailure(final Exception e) { systemIndices, false, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + null ); actions.put( CreateIndexAction.INSTANCE, diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index a9f6fdc86155d..0eca08a7678ae 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -2640,6 +2640,7 @@ private static Settings buildRemoteStoreNodeAttributes( .put(segmentRepoSettingsAttributeKeyPrefix + "chunk_size", 200, ByteSizeUnit.BYTES); } settings.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), randomFrom(PathType.values())); + settings.put(RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA.getKey(), randomBoolean()); return settings.build(); } From da0cf847a3639b45efc34b211854284ecf00ef8d Mon Sep 17 00:00:00 2001 From: Neetika Singhal Date: Thu, 23 May 2024 09:58:08 -0700 Subject: [PATCH 04/77] Refactor writable warm feature flag to include tiering (#13719) Signed-off-by: Neetika Singhal --- .../common/settings/FeatureFlagSettings.java | 2 +- .../org/opensearch/common/util/FeatureFlags.java | 12 ++++-------- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java b/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java index 985eb40711e16..7a364de1c5dc6 100644 --- a/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java @@ -34,7 +34,7 @@ protected FeatureFlagSettings( FeatureFlags.IDENTITY_SETTING, FeatureFlags.TELEMETRY_SETTING, FeatureFlags.DATETIME_FORMATTER_CACHING_SETTING, - FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING, + FeatureFlags.TIERED_REMOTE_INDEX_SETTING, FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL_SETTING, FeatureFlags.PLUGGABLE_CACHE_SETTING ); diff --git a/server/src/main/java/org/opensearch/common/util/FeatureFlags.java b/server/src/main/java/org/opensearch/common/util/FeatureFlags.java index a72583607ede0..62cfbd861d4d9 100644 --- a/server/src/main/java/org/opensearch/common/util/FeatureFlags.java +++ b/server/src/main/java/org/opensearch/common/util/FeatureFlags.java @@ -56,10 +56,10 @@ public class FeatureFlags { public static final String DATETIME_FORMATTER_CACHING = "opensearch.experimental.optimization.datetime_formatter_caching.enabled"; /** - * Gates the functionality of writeable remote index + * Gates the functionality of remote index having the capability to move across different tiers * Once the feature is ready for release, this feature flag can be removed. */ - public static final String WRITEABLE_REMOTE_INDEX = "opensearch.experimental.feature.writeable_remote_index.enabled"; + public static final String TIERED_REMOTE_INDEX = "opensearch.experimental.feature.tiered_remote_index.enabled"; /** * Gates the functionality of pluggable cache. @@ -85,11 +85,7 @@ public class FeatureFlags { Property.NodeScope ); - public static final Setting WRITEABLE_REMOTE_INDEX_SETTING = Setting.boolSetting( - WRITEABLE_REMOTE_INDEX, - false, - Property.NodeScope - ); + public static final Setting TIERED_REMOTE_INDEX_SETTING = Setting.boolSetting(TIERED_REMOTE_INDEX, false, Property.NodeScope); public static final Setting PLUGGABLE_CACHE_SETTING = Setting.boolSetting(PLUGGABLE_CACHE, false, Property.NodeScope); @@ -99,7 +95,7 @@ public class FeatureFlags { IDENTITY_SETTING, TELEMETRY_SETTING, DATETIME_FORMATTER_CACHING_SETTING, - WRITEABLE_REMOTE_INDEX_SETTING, + TIERED_REMOTE_INDEX_SETTING, PLUGGABLE_CACHE_SETTING ); /** From e67ced73226453d5a5504c78f3b7d5ae90b4914e Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Thu, 23 May 2024 12:02:22 -0700 Subject: [PATCH 05/77] [Bugfix] [Tiered Caching] Fixes issues when integrating tiered cache with disk cache (#13784) --------- Signed-off-by: Peter Alfonsi Co-authored-by: Peter Alfonsi --- .../common/tier/TieredSpilloverCache.java | 2 + .../cache/common/tier/MockDiskCache.java | 4 + .../tier/TieredSpilloverCacheTests.java | 35 +++++ plugins/cache-ehcache/build.gradle | 18 +-- .../licenses/slf4j-api-1.7.36.jar.sha1 | 1 + .../licenses/slf4j-api-LICENSE.txt | 21 +++ .../licenses/slf4j-api-NOTICE.txt | 0 .../cache/store/disk/EhcacheDiskCache.java | 138 +++++++++--------- .../plugin-metadata/plugin-security.policy | 3 + 9 files changed, 143 insertions(+), 79 deletions(-) create mode 100644 plugins/cache-ehcache/licenses/slf4j-api-1.7.36.jar.sha1 create mode 100644 plugins/cache-ehcache/licenses/slf4j-api-LICENSE.txt create mode 100644 plugins/cache-ehcache/licenses/slf4j-api-NOTICE.txt 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 9942651ccdd67..f40c35dde83de 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 @@ -119,6 +119,8 @@ public class TieredSpilloverCache implements ICache { .setValueType(builder.cacheConfig.getValueType()) .setSettings(builder.cacheConfig.getSettings()) .setWeigher(builder.cacheConfig.getWeigher()) + .setKeySerializer(builder.cacheConfig.getKeySerializer()) + .setValueSerializer(builder.cacheConfig.getValueSerializer()) .setDimensionNames(builder.cacheConfig.getDimensionNames()) .setStatsTrackingEnabled(false) .build(), diff --git a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/MockDiskCache.java b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/MockDiskCache.java index 2058faa5181b1..69e2060f7ea2f 100644 --- a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/MockDiskCache.java +++ b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/MockDiskCache.java @@ -141,6 +141,10 @@ public MockDiskCacheFactory(long delay, int maxSize, boolean statsTrackingEnable @Override @SuppressWarnings({ "unchecked" }) public ICache create(CacheConfig config, CacheType cacheType, Map cacheFactories) { + // As we can't directly IT with the tiered cache and ehcache, check that we receive non-null serializers, as an ehcache disk + // cache would require. + assert config.getKeySerializer() != null; + assert config.getValueSerializer() != null; return new Builder().setKeySerializer((Serializer) config.getKeySerializer()) .setValueSerializer((Serializer) config.getValueSerializer()) .setMaxSize(maxSize) 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 6d5ee91326338..6c49341591589 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 @@ -16,6 +16,7 @@ import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.policy.CachedQueryResult; +import org.opensearch.common.cache.serializer.Serializer; import org.opensearch.common.cache.settings.CacheSettings; import org.opensearch.common.cache.stats.ImmutableCacheStats; import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; @@ -32,6 +33,8 @@ import org.junit.Before; import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -166,6 +169,8 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception .setKeyType(String.class) .setWeigher((k, v) -> keyValueSize) .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) .setSettings(settings) .setDimensionNames(dimensionNames) .setCachedResultParser(s -> new CachedQueryResult.PolicyValues(20_000_000L)) // Values will always appear to have taken @@ -318,6 +323,8 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { .setKeyType(String.class) .setWeigher((k, v) -> keyValueSize) .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) .setDimensionNames(dimensionNames) .setSettings( Settings.builder() @@ -830,6 +837,8 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio .setKeyType(String.class) .setWeigher((k, v) -> 150) .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) .setSettings( Settings.builder() .put( @@ -1014,6 +1023,8 @@ public void testTookTimePolicyFromFactory() throws Exception { .setKeyType(String.class) .setWeigher((k, v) -> keyValueSize) .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) .setSettings(settings) .setMaxSizeInBytes(onHeapCacheSize * keyValueSize) .setDimensionNames(dimensionNames) @@ -1415,6 +1426,8 @@ private TieredSpilloverCache intializeTieredSpilloverCache( .setSettings(settings) .setDimensionNames(dimensionNames) .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) .setSettings( Settings.builder() .put( @@ -1479,4 +1492,26 @@ private ImmutableCacheStats getStatsSnapshotForTier(TieredSpilloverCache t } return snapshot; } + + // Duplicated here from EhcacheDiskCacheTests.java, we can't add a dependency on that plugin + static class StringSerializer implements Serializer { + private final Charset charset = StandardCharsets.UTF_8; + + @Override + public byte[] serialize(String object) { + return object.getBytes(charset); + } + + @Override + public String deserialize(byte[] bytes) { + if (bytes == null) { + return null; + } + return new String(bytes, charset); + } + + public boolean equals(String object, byte[] bytes) { + return object.equals(deserialize(bytes)); + } + } } diff --git a/plugins/cache-ehcache/build.gradle b/plugins/cache-ehcache/build.gradle index 4fc5e44f58c3a..5747624e2fb69 100644 --- a/plugins/cache-ehcache/build.gradle +++ b/plugins/cache-ehcache/build.gradle @@ -24,6 +24,7 @@ versions << [ dependencies { api "org.ehcache:ehcache:${versions.ehcache}" + api "org.slf4j:slf4j-api:${versions.slf4j}" } thirdPartyAudit { @@ -78,10 +79,9 @@ thirdPartyAudit { 'org.osgi.framework.BundleActivator', 'org.osgi.framework.BundleContext', 'org.osgi.framework.ServiceReference', - 'org.slf4j.Logger', - 'org.slf4j.LoggerFactory', - 'org.slf4j.Marker', - 'org.slf4j.event.Level' + 'org.slf4j.impl.StaticLoggerBinder', + 'org.slf4j.impl.StaticMDCBinder', + 'org.slf4j.impl.StaticMarkerBinder' ) } @@ -90,13 +90,3 @@ tasks.named("bundlePlugin").configure { into 'config' } } - -test { - // TODO: Adding permission in plugin-security.policy doesn't seem to work. - systemProperty 'tests.security.manager', 'false' -} - -internalClusterTest { - // TODO: Remove this later once we have a way. - systemProperty 'tests.security.manager', 'false' -} diff --git a/plugins/cache-ehcache/licenses/slf4j-api-1.7.36.jar.sha1 b/plugins/cache-ehcache/licenses/slf4j-api-1.7.36.jar.sha1 new file mode 100644 index 0000000000000..77b9917528382 --- /dev/null +++ b/plugins/cache-ehcache/licenses/slf4j-api-1.7.36.jar.sha1 @@ -0,0 +1 @@ +6c62681a2f655b49963a5983b8b0950a6120ae14 \ No newline at end of file diff --git a/plugins/cache-ehcache/licenses/slf4j-api-LICENSE.txt b/plugins/cache-ehcache/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..54512cc08d16b --- /dev/null +++ b/plugins/cache-ehcache/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ +Copyright (c) 2004-2022 QOS.ch +All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/plugins/cache-ehcache/licenses/slf4j-api-NOTICE.txt b/plugins/cache-ehcache/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d 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 9a4dce1067b61..b4c62fbf85cb8 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 @@ -42,6 +42,8 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.security.AccessController; +import java.security.PrivilegedAction; import java.time.Duration; import java.util.Arrays; import java.util.Iterator; @@ -175,57 +177,60 @@ private EhcacheDiskCache(Builder builder) { @SuppressWarnings({ "rawtypes" }) private Cache buildCache(Duration expireAfterAccess, Builder builder) { - try { - return this.cacheManager.createCache( - this.diskCacheAlias, - CacheConfigurationBuilder.newCacheConfigurationBuilder( - ICacheKey.class, - ByteArrayWrapper.class, - ResourcePoolsBuilder.newResourcePoolsBuilder().disk(maxWeightInBytes, MemoryUnit.B) - ).withExpiry(new ExpiryPolicy<>() { - @Override - public Duration getExpiryForCreation(ICacheKey key, ByteArrayWrapper value) { - return INFINITE; - } - - @Override - public Duration getExpiryForAccess(ICacheKey key, Supplier value) { - return expireAfterAccess; - } - - @Override - public Duration getExpiryForUpdate( - ICacheKey key, - Supplier oldValue, - ByteArrayWrapper newValue - ) { - return INFINITE; - } - }) - .withService(getListenerConfiguration(builder)) - .withService( - new OffHeapDiskStoreConfiguration( - this.threadPoolAlias, - (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType) - .get(DISK_WRITE_CONCURRENCY_KEY) - .get(settings), - (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType).get(DISK_SEGMENT_KEY).get(settings) + // Creating the cache requires permissions specified in plugin-security.policy + return AccessController.doPrivileged((PrivilegedAction>) () -> { + try { + return this.cacheManager.createCache( + this.diskCacheAlias, + CacheConfigurationBuilder.newCacheConfigurationBuilder( + ICacheKey.class, + ByteArrayWrapper.class, + ResourcePoolsBuilder.newResourcePoolsBuilder().disk(maxWeightInBytes, MemoryUnit.B) + ).withExpiry(new ExpiryPolicy<>() { + @Override + public Duration getExpiryForCreation(ICacheKey key, ByteArrayWrapper value) { + return INFINITE; + } + + @Override + public Duration getExpiryForAccess(ICacheKey key, Supplier value) { + return expireAfterAccess; + } + + @Override + public Duration getExpiryForUpdate( + ICacheKey key, + Supplier oldValue, + ByteArrayWrapper newValue + ) { + return INFINITE; + } + }) + .withService(getListenerConfiguration(builder)) + .withService( + new OffHeapDiskStoreConfiguration( + this.threadPoolAlias, + (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType) + .get(DISK_WRITE_CONCURRENCY_KEY) + .get(settings), + (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType).get(DISK_SEGMENT_KEY).get(settings) + ) ) - ) - .withKeySerializer(new KeySerializerWrapper(keySerializer)) - .withValueSerializer(new ByteArrayWrapperSerializer()) + .withKeySerializer(new KeySerializerWrapper(keySerializer)) + .withValueSerializer(new ByteArrayWrapperSerializer()) // We pass ByteArrayWrapperSerializer as ehcache's value serializer. If V is an interface, and we pass its // serializer directly to ehcache, ehcache requires the classes match exactly before/after serialization. // This is not always feasible or necessary, like for BytesReference. So, we handle the value serialization // before V hits ehcache. - ); - } catch (IllegalArgumentException ex) { - logger.error("Ehcache disk cache initialization failed due to illegal argument: {}", ex.getMessage()); - throw ex; - } catch (IllegalStateException ex) { - logger.error("Ehcache disk cache initialization failed: {}", ex.getMessage()); - throw ex; - } + ); + } catch (IllegalArgumentException ex) { + logger.error("Ehcache disk cache initialization failed due to illegal argument: {}", ex.getMessage()); + throw ex; + } catch (IllegalStateException ex) { + logger.error("Ehcache disk cache initialization failed: {}", ex.getMessage()); + throw ex; + } + }); } private CacheEventListenerConfigurationBuilder getListenerConfiguration(Builder builder) { @@ -252,25 +257,28 @@ Map, CompletableFuture, V>>> getCompletableFutur @SuppressForbidden(reason = "Ehcache uses File.io") private PersistentCacheManager buildCacheManager() { // In case we use multiple ehCaches, we can define this cache manager at a global level. - return CacheManagerBuilder.newCacheManagerBuilder() - .with(CacheManagerBuilder.persistence(new File(storagePath))) - - .using( - PooledExecutionServiceConfigurationBuilder.newPooledExecutionServiceConfigurationBuilder() - .defaultPool(THREAD_POOL_ALIAS_PREFIX + "Default#" + UNIQUE_ID, 1, 3) // Default pool used for other tasks - // like event listeners - .pool( - this.threadPoolAlias, - (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType) - .get(DISK_WRITE_MIN_THREADS_KEY) - .get(settings), - (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType) - .get(DISK_WRITE_MAXIMUM_THREADS_KEY) - .get(settings) - ) - .build() - ) - .build(true); + // Creating the cache manager also requires permissions specified in plugin-security.policy + return AccessController.doPrivileged((PrivilegedAction) () -> { + return CacheManagerBuilder.newCacheManagerBuilder() + .with(CacheManagerBuilder.persistence(new File(storagePath))) + + .using( + PooledExecutionServiceConfigurationBuilder.newPooledExecutionServiceConfigurationBuilder() + .defaultPool(THREAD_POOL_ALIAS_PREFIX + "Default#" + UNIQUE_ID, 1, 3) // Default pool used for other tasks + // like event listeners + .pool( + this.threadPoolAlias, + (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType) + .get(DISK_WRITE_MIN_THREADS_KEY) + .get(settings), + (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType) + .get(DISK_WRITE_MAXIMUM_THREADS_KEY) + .get(settings) + ) + .build() + ) + .build(true); + }); } @Override diff --git a/plugins/cache-ehcache/src/main/plugin-metadata/plugin-security.policy b/plugins/cache-ehcache/src/main/plugin-metadata/plugin-security.policy index 40007eea62dba..85c82824d5d65 100644 --- a/plugins/cache-ehcache/src/main/plugin-metadata/plugin-security.policy +++ b/plugins/cache-ehcache/src/main/plugin-metadata/plugin-security.policy @@ -9,5 +9,8 @@ grant { permission java.lang.RuntimePermission "accessClassInPackage.sun.misc"; permission java.lang.RuntimePermission "createClassLoader"; + permission java.lang.RuntimePermission "accessDeclaredMembers"; + permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; + permission java.lang.RuntimePermission "getClassLoader"; }; From 27003270adedebd43eedbd0186c0825f2c0e8613 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Thu, 23 May 2024 20:36:38 -0400 Subject: [PATCH 06/77] Bump netty from 4.1.109.Final to 4.1.110.Final (#13802) --- CHANGELOG.md | 1 + buildSrc/version.properties | 2 +- modules/transport-netty4/build.gradle | 3 +++ .../licenses/netty-buffer-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-buffer-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-common-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-common-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-handler-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-handler-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.110.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.109.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-dns-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-dns-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-socks-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-socks-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-handler-proxy-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.109.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.110.Final.jar.sha1 | 1 + .../repository-hdfs/licenses/netty-all-4.1.109.Final.jar.sha1 | 1 - .../repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 | 1 + plugins/repository-s3/build.gradle | 3 +++ .../repository-s3/licenses/netty-buffer-4.1.109.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-codec-4.1.109.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.110.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-common-4.1.109.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-handler-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-handler-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.110.Final.jar.sha1 | 1 + .../netty-transport-classes-epoll-4.1.109.Final.jar.sha1 | 1 - .../netty-transport-classes-epoll-4.1.110.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.109.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.110.Final.jar.sha1 | 1 + plugins/transport-nio/build.gradle | 3 +++ .../transport-nio/licenses/netty-buffer-4.1.109.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-codec-4.1.109.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.110.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-common-4.1.109.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-handler-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-handler-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.110.Final.jar.sha1 | 1 + plugins/transport-reactor-netty4/build.gradle | 3 +++ .../licenses/netty-buffer-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-buffer-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-dns-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-dns-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-common-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-common-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-handler-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-handler-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.109.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.110.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.109.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.110.Final.jar.sha1 | 1 + 94 files changed, 58 insertions(+), 45 deletions(-) delete mode 100644 modules/transport-netty4/licenses/netty-buffer-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-http-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-common-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-handler-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-resolver-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-transport-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-dns-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-socks-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-socks-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-handler-proxy-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-hdfs/licenses/netty-all-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-buffer-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-http-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-http-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-common-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-handler-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-resolver-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-resolver-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.110.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-buffer-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-codec-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-codec-http-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-codec-http-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-common-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-handler-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-resolver-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-resolver-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-transport-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-transport-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-common-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-handler-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-transport-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 4cd610694a7b6..4b4a5e5f4f981 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,6 +22,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `mockito` from 5.11.0 to 5.12.0 ([#13665](https://github.com/opensearch-project/OpenSearch/pull/13665)) - Bump `com.google.code.gson:gson` from 2.10.1 to 2.11.0 ([#13752](https://github.com/opensearch-project/OpenSearch/pull/13752)) - Bump `ch.qos.logback:logback-core` from 1.5.3 to 1.5.6 ([#13756](https://github.com/opensearch-project/OpenSearch/pull/13756)) +- Bump `netty` from 4.1.109.Final to 4.1.110.Final ([#13802](https://github.com/opensearch-project/OpenSearch/pull/13802)) ### Changed - Add ability for Boolean and date field queries to run when only doc_values are enabled ([#11650](https://github.com/opensearch-project/OpenSearch/pull/11650)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 1e097d586aded..d8779cf8c2778 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -27,7 +27,7 @@ google_http_client = 1.44.1 # when updating the JNA version, also update the version in buildSrc/build.gradle jna = 5.13.0 -netty = 4.1.109.Final +netty = 4.1.110.Final joda = 2.12.7 # project reactor diff --git a/modules/transport-netty4/build.gradle b/modules/transport-netty4/build.gradle index 83c4db80b7798..a8a165df637a2 100644 --- a/modules/transport-netty4/build.gradle +++ b/modules/transport-netty4/build.gradle @@ -235,11 +235,14 @@ thirdPartyAudit { 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueConsumerFields', 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueProducerFields', 'io.netty.util.internal.shaded.org.jctools.queues.LinkedQueueNode', + 'io.netty.util.internal.shaded.org.jctools.queues.MpmcArrayQueueConsumerIndexField', + 'io.netty.util.internal.shaded.org.jctools.queues.MpmcArrayQueueProducerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueConsumerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerLimitField', 'io.netty.util.internal.shaded.org.jctools.util.UnsafeAccess', 'io.netty.util.internal.shaded.org.jctools.util.UnsafeRefArrayAccess', + 'io.netty.util.internal.shaded.org.jctools.util.UnsafeLongArrayAccess', 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator', 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$1', 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$2', diff --git a/modules/transport-netty4/licenses/netty-buffer-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-buffer-4.1.109.Final.jar.sha1 deleted file mode 100644 index 76b51cdae3867..0000000000000 --- a/modules/transport-netty4/licenses/netty-buffer-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d21d602ad7c639fa16b1d26559065d310a34c51 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..faaf70c858a6e --- /dev/null +++ b/modules/transport-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +3d918a9ee057d995c362902b54634fc307132aac \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-4.1.109.Final.jar.sha1 deleted file mode 100644 index 1bccee872152d..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -16e0b2beb49318a549d3ba5d66d707bd5daa8c97 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..7affbc14fa93a --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +f1fa43b03e93ab88e805b6a4e3e83780c80b47d2 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http-4.1.109.Final.jar.sha1 deleted file mode 100644 index 3423fb94e8497..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-http-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6dca43cedc0b2dc6bf57bdc85fce6ffca3e6b72a \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..07730a5606ce2 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +9d05cd927209ea25bbf342962c00b8e5a828c2a4 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 deleted file mode 100644 index b83ad36222d07..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6bd4a54b69a81356393f6e4621bad40754f8a5a2 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..ebd1e0d52efb2 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +e0849843eb5b1c036b12551baca98a9f7ff847a0 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-common-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-common-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5172500557f8b..0000000000000 --- a/modules/transport-netty4/licenses/netty-common-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da63e54ee1ca69abf4206cb74fadef7f50850911 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..568c0aa2a2c03 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +ec361e7e025c029be50c55c8480080cabcbc01e7 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-handler-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-handler-4.1.109.Final.jar.sha1 deleted file mode 100644 index cabe61b300523..0000000000000 --- a/modules/transport-netty4/licenses/netty-handler-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9167863307b3c44cc12262e7b5512de3499b9c4a \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..2d6050dd1e3a5 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +168db749c22652ee7fed1ebf7ec46ce856d75e51 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-resolver-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-resolver-4.1.109.Final.jar.sha1 deleted file mode 100644 index 14e21cc0cdb60..0000000000000 --- a/modules/transport-netty4/licenses/netty-resolver-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -055485ac976e27c8bb67ee111a8490c58f67b70c \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..c3ee8087a8b5d --- /dev/null +++ b/modules/transport-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +66c15921104cda0159b34e316541bc765dfaf3c0 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-4.1.109.Final.jar.sha1 deleted file mode 100644 index 6b23d0883e31f..0000000000000 --- a/modules/transport-netty4/licenses/netty-transport-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -79e3b07d58ef03c7a860d48f932b720675aa8bd3 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..32c8fa2b876a2 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +b91f04c39ac14d6a29d07184ef305953ee6e0348 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5afeb9627c9b5..0000000000000 --- a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da7fe1e6943cbab8ee48df2beadc2c8304f347a2 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..2c468962b1b64 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +a7096e7c0a25a983647909d7513f5d4943d589c0 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-dns-4.1.109.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-dns-4.1.109.Final.jar.sha1 deleted file mode 100644 index b13a709f1c449..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-dns-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ee231baee2cc9f1300ecc0d9a1e8bb9b31db02fa \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..c4ca8f15e85c5 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +381c5bf8b7570c163fa7893a26d02b7ac36ff6eb \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 deleted file mode 100644 index b83ad36222d07..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6bd4a54b69a81356393f6e4621bad40754f8a5a2 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..ebd1e0d52efb2 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +e0849843eb5b1c036b12551baca98a9f7ff847a0 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-socks-4.1.109.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-socks-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5caf947d87a1b..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-socks-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7f4f0c0dd54c578af2c613a0db7172bf7dca9c79 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-socks-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-socks-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..9f6e95ba38d2e --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-socks-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +4d54c8d5b95b14756043efb59b8c3e62ec67aa43 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.109.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.109.Final.jar.sha1 deleted file mode 100644 index e0f52ab04ea84..0000000000000 --- a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a77224107f586a7f9e3dc5d12fc0d4d8f0c04803 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..f31396d94c2ec --- /dev/null +++ b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +b7fb401dd47c79e6b99f2319ac3b561c50c31c30 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 deleted file mode 100644 index b42cdc2835eb0..0000000000000 --- a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5f4d858234b557b73631a24e562bb89fc5399cad \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..18d122acd2c44 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +3e687cdc4ecdbbad07508a11b715bdf95fa20939 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5afeb9627c9b5..0000000000000 --- a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da7fe1e6943cbab8ee48df2beadc2c8304f347a2 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..2c468962b1b64 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +a7096e7c0a25a983647909d7513f5d4943d589c0 \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/netty-all-4.1.109.Final.jar.sha1 b/plugins/repository-hdfs/licenses/netty-all-4.1.109.Final.jar.sha1 deleted file mode 100644 index a874755cc29da..0000000000000 --- a/plugins/repository-hdfs/licenses/netty-all-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3ba1acc8ff088334f2ac5556663f8b737eb8b571 \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 b/plugins/repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..8f8d86e6065b2 --- /dev/null +++ b/plugins/repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +db3f4d3ad3d16e26991a64d50b749ae09e0e0c8e \ No newline at end of file diff --git a/plugins/repository-s3/build.gradle b/plugins/repository-s3/build.gradle index 560d12d14395d..00decbe4fa9cd 100644 --- a/plugins/repository-s3/build.gradle +++ b/plugins/repository-s3/build.gradle @@ -577,11 +577,14 @@ thirdPartyAudit { 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueConsumerFields', 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueProducerFields', 'io.netty.util.internal.shaded.org.jctools.queues.LinkedQueueNode', + 'io.netty.util.internal.shaded.org.jctools.queues.MpmcArrayQueueConsumerIndexField', + 'io.netty.util.internal.shaded.org.jctools.queues.MpmcArrayQueueProducerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueConsumerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerLimitField', 'io.netty.util.internal.shaded.org.jctools.util.UnsafeAccess', 'io.netty.util.internal.shaded.org.jctools.util.UnsafeRefArrayAccess', + 'io.netty.util.internal.shaded.org.jctools.util.UnsafeLongArrayAccess', ) } diff --git a/plugins/repository-s3/licenses/netty-buffer-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-buffer-4.1.109.Final.jar.sha1 deleted file mode 100644 index 76b51cdae3867..0000000000000 --- a/plugins/repository-s3/licenses/netty-buffer-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d21d602ad7c639fa16b1d26559065d310a34c51 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..faaf70c858a6e --- /dev/null +++ b/plugins/repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +3d918a9ee057d995c362902b54634fc307132aac \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-4.1.109.Final.jar.sha1 deleted file mode 100644 index 1bccee872152d..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -16e0b2beb49318a549d3ba5d66d707bd5daa8c97 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..7affbc14fa93a --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +f1fa43b03e93ab88e805b6a4e3e83780c80b47d2 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http-4.1.109.Final.jar.sha1 deleted file mode 100644 index 3423fb94e8497..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-http-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6dca43cedc0b2dc6bf57bdc85fce6ffca3e6b72a \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..07730a5606ce2 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-http-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +9d05cd927209ea25bbf342962c00b8e5a828c2a4 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 deleted file mode 100644 index b83ad36222d07..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6bd4a54b69a81356393f6e4621bad40754f8a5a2 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..ebd1e0d52efb2 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +e0849843eb5b1c036b12551baca98a9f7ff847a0 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-common-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-common-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5172500557f8b..0000000000000 --- a/plugins/repository-s3/licenses/netty-common-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da63e54ee1ca69abf4206cb74fadef7f50850911 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..568c0aa2a2c03 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +ec361e7e025c029be50c55c8480080cabcbc01e7 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-handler-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-handler-4.1.109.Final.jar.sha1 deleted file mode 100644 index cabe61b300523..0000000000000 --- a/plugins/repository-s3/licenses/netty-handler-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9167863307b3c44cc12262e7b5512de3499b9c4a \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..2d6050dd1e3a5 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +168db749c22652ee7fed1ebf7ec46ce856d75e51 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-resolver-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-resolver-4.1.109.Final.jar.sha1 deleted file mode 100644 index 14e21cc0cdb60..0000000000000 --- a/plugins/repository-s3/licenses/netty-resolver-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -055485ac976e27c8bb67ee111a8490c58f67b70c \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-resolver-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-resolver-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..c3ee8087a8b5d --- /dev/null +++ b/plugins/repository-s3/licenses/netty-resolver-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +66c15921104cda0159b34e316541bc765dfaf3c0 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-4.1.109.Final.jar.sha1 deleted file mode 100644 index 6b23d0883e31f..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -79e3b07d58ef03c7a860d48f932b720675aa8bd3 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..32c8fa2b876a2 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +b91f04c39ac14d6a29d07184ef305953ee6e0348 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.109.Final.jar.sha1 deleted file mode 100644 index 83fc39246ef0a..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7307c8acbc9b331fce3496750a5112bdc726fd2a \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..408f3aa5d1339 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +3ca1cff0bf82bfd38e89f6946e54f24cbb3424a2 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5afeb9627c9b5..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da7fe1e6943cbab8ee48df2beadc2c8304f347a2 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..2c468962b1b64 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +a7096e7c0a25a983647909d7513f5d4943d589c0 \ No newline at end of file diff --git a/plugins/transport-nio/build.gradle b/plugins/transport-nio/build.gradle index 8c0ee8ba718ac..ee557aa0efc79 100644 --- a/plugins/transport-nio/build.gradle +++ b/plugins/transport-nio/build.gradle @@ -173,11 +173,14 @@ thirdPartyAudit { 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueConsumerFields', 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueProducerFields', 'io.netty.util.internal.shaded.org.jctools.queues.LinkedQueueNode', + 'io.netty.util.internal.shaded.org.jctools.queues.MpmcArrayQueueConsumerIndexField', + 'io.netty.util.internal.shaded.org.jctools.queues.MpmcArrayQueueProducerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueConsumerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerLimitField', 'io.netty.util.internal.shaded.org.jctools.util.UnsafeAccess', 'io.netty.util.internal.shaded.org.jctools.util.UnsafeRefArrayAccess', + 'io.netty.util.internal.shaded.org.jctools.util.UnsafeLongArrayAccess', 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator', 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$1', diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.109.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.109.Final.jar.sha1 deleted file mode 100644 index 76b51cdae3867..0000000000000 --- a/plugins/transport-nio/licenses/netty-buffer-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d21d602ad7c639fa16b1d26559065d310a34c51 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..faaf70c858a6e --- /dev/null +++ b/plugins/transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +3d918a9ee057d995c362902b54634fc307132aac \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.109.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.109.Final.jar.sha1 deleted file mode 100644 index 1bccee872152d..0000000000000 --- a/plugins/transport-nio/licenses/netty-codec-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -16e0b2beb49318a549d3ba5d66d707bd5daa8c97 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..7affbc14fa93a --- /dev/null +++ b/plugins/transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +f1fa43b03e93ab88e805b6a4e3e83780c80b47d2 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.109.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.109.Final.jar.sha1 deleted file mode 100644 index 3423fb94e8497..0000000000000 --- a/plugins/transport-nio/licenses/netty-codec-http-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6dca43cedc0b2dc6bf57bdc85fce6ffca3e6b72a \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..07730a5606ce2 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-codec-http-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +9d05cd927209ea25bbf342962c00b8e5a828c2a4 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-common-4.1.109.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5172500557f8b..0000000000000 --- a/plugins/transport-nio/licenses/netty-common-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da63e54ee1ca69abf4206cb74fadef7f50850911 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..568c0aa2a2c03 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +ec361e7e025c029be50c55c8480080cabcbc01e7 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.109.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.109.Final.jar.sha1 deleted file mode 100644 index cabe61b300523..0000000000000 --- a/plugins/transport-nio/licenses/netty-handler-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9167863307b3c44cc12262e7b5512de3499b9c4a \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..2d6050dd1e3a5 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +168db749c22652ee7fed1ebf7ec46ce856d75e51 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.109.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.109.Final.jar.sha1 deleted file mode 100644 index 14e21cc0cdb60..0000000000000 --- a/plugins/transport-nio/licenses/netty-resolver-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -055485ac976e27c8bb67ee111a8490c58f67b70c \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..c3ee8087a8b5d --- /dev/null +++ b/plugins/transport-nio/licenses/netty-resolver-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +66c15921104cda0159b34e316541bc765dfaf3c0 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.109.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.109.Final.jar.sha1 deleted file mode 100644 index 6b23d0883e31f..0000000000000 --- a/plugins/transport-nio/licenses/netty-transport-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -79e3b07d58ef03c7a860d48f932b720675aa8bd3 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..32c8fa2b876a2 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-transport-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +b91f04c39ac14d6a29d07184ef305953ee6e0348 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/build.gradle b/plugins/transport-reactor-netty4/build.gradle index 7d7eb330b4a55..1a94def3fdff1 100644 --- a/plugins/transport-reactor-netty4/build.gradle +++ b/plugins/transport-reactor-netty4/build.gradle @@ -249,11 +249,14 @@ thirdPartyAudit { 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueConsumerFields', 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueProducerFields', 'io.netty.util.internal.shaded.org.jctools.queues.LinkedQueueNode', + 'io.netty.util.internal.shaded.org.jctools.queues.MpmcArrayQueueConsumerIndexField', + 'io.netty.util.internal.shaded.org.jctools.queues.MpmcArrayQueueProducerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueConsumerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerIndexField', 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerLimitField', 'io.netty.util.internal.shaded.org.jctools.util.UnsafeAccess', 'io.netty.util.internal.shaded.org.jctools.util.UnsafeRefArrayAccess', + 'io.netty.util.internal.shaded.org.jctools.util.UnsafeLongArrayAccess', 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator', 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$1', 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$2', diff --git a/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.109.Final.jar.sha1 deleted file mode 100644 index 76b51cdae3867..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d21d602ad7c639fa16b1d26559065d310a34c51 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..faaf70c858a6e --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +3d918a9ee057d995c362902b54634fc307132aac \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.109.Final.jar.sha1 deleted file mode 100644 index 1bccee872152d..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -16e0b2beb49318a549d3ba5d66d707bd5daa8c97 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..7affbc14fa93a --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +f1fa43b03e93ab88e805b6a4e3e83780c80b47d2 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.109.Final.jar.sha1 deleted file mode 100644 index b13a709f1c449..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ee231baee2cc9f1300ecc0d9a1e8bb9b31db02fa \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..c4ca8f15e85c5 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +381c5bf8b7570c163fa7893a26d02b7ac36ff6eb \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.109.Final.jar.sha1 deleted file mode 100644 index 3423fb94e8497..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6dca43cedc0b2dc6bf57bdc85fce6ffca3e6b72a \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..07730a5606ce2 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +9d05cd927209ea25bbf342962c00b8e5a828c2a4 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 deleted file mode 100644 index b83ad36222d07..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6bd4a54b69a81356393f6e4621bad40754f8a5a2 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..ebd1e0d52efb2 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +e0849843eb5b1c036b12551baca98a9f7ff847a0 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-common-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-common-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5172500557f8b..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-common-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da63e54ee1ca69abf4206cb74fadef7f50850911 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..568c0aa2a2c03 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +ec361e7e025c029be50c55c8480080cabcbc01e7 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.109.Final.jar.sha1 deleted file mode 100644 index cabe61b300523..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9167863307b3c44cc12262e7b5512de3499b9c4a \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..2d6050dd1e3a5 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +168db749c22652ee7fed1ebf7ec46ce856d75e51 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.109.Final.jar.sha1 deleted file mode 100644 index 14e21cc0cdb60..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -055485ac976e27c8bb67ee111a8490c58f67b70c \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..c3ee8087a8b5d --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +66c15921104cda0159b34e316541bc765dfaf3c0 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 deleted file mode 100644 index b42cdc2835eb0..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5f4d858234b557b73631a24e562bb89fc5399cad \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..18d122acd2c44 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +3e687cdc4ecdbbad07508a11b715bdf95fa20939 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.109.Final.jar.sha1 deleted file mode 100644 index 6b23d0883e31f..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -79e3b07d58ef03c7a860d48f932b720675aa8bd3 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..32c8fa2b876a2 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +b91f04c39ac14d6a29d07184ef305953ee6e0348 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 deleted file mode 100644 index 5afeb9627c9b5..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.109.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da7fe1e6943cbab8ee48df2beadc2c8304f347a2 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 new file mode 100644 index 0000000000000..2c468962b1b64 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 @@ -0,0 +1 @@ +a7096e7c0a25a983647909d7513f5d4943d589c0 \ No newline at end of file From 8ac6806ac275e932fb52d38aa1d5dfdaa0f74369 Mon Sep 17 00:00:00 2001 From: Lakshya Taragi <157457166+ltaragi@users.noreply.github.com> Date: Fri, 24 May 2024 10:10:45 +0530 Subject: [PATCH 07/77] Add trace level logging to RepositoryURLClientYamlTestSuiteIT (#13779) Signed-off-by: Lakshya Taragi --- .../repositories/url/RepositoryURLClientYamlTestSuiteIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/repository-url/src/yamlRestTest/java/org/opensearch/repositories/url/RepositoryURLClientYamlTestSuiteIT.java b/modules/repository-url/src/yamlRestTest/java/org/opensearch/repositories/url/RepositoryURLClientYamlTestSuiteIT.java index 6536d474f5abc..27cef3f7d7251 100644 --- a/modules/repository-url/src/yamlRestTest/java/org/opensearch/repositories/url/RepositoryURLClientYamlTestSuiteIT.java +++ b/modules/repository-url/src/yamlRestTest/java/org/opensearch/repositories/url/RepositoryURLClientYamlTestSuiteIT.java @@ -47,6 +47,7 @@ import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.repositories.fs.FsRepository; +import org.opensearch.test.junit.annotations.TestIssueLogging; import org.opensearch.test.rest.yaml.ClientYamlTestCandidate; import org.opensearch.test.rest.yaml.OpenSearchClientYamlSuiteTestCase; import org.junit.Before; @@ -63,6 +64,7 @@ import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.notNullValue; +@TestIssueLogging(value = "_root:TRACE", issueUrl = "https://github.com/opensearch-project/OpenSearch/issues/9117") public class RepositoryURLClientYamlTestSuiteIT extends OpenSearchClientYamlSuiteTestCase { public RepositoryURLClientYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { From 8883e62a8f3d862739250cff1eb8d3d18680cfa2 Mon Sep 17 00:00:00 2001 From: Craig Perkins Date: Fri, 24 May 2024 13:57:02 -0400 Subject: [PATCH 08/77] Add getMetadataFields to MapperService (#13819) * Add getMetadataFields to MapperService Signed-off-by: Craig Perkins * Add to CHANGELOG Signed-off-by: Craig Perkins * Unmodifiable Set Signed-off-by: Craig Perkins --------- Signed-off-by: Craig Perkins --- CHANGELOG.md | 1 + .../java/org/opensearch/index/mapper/MapperService.java | 7 +++++++ .../org/opensearch/index/mapper/MapperServiceTests.java | 6 ++++++ 3 files changed, 14 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4b4a5e5f4f981..e8d25e0337e30 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,6 +11,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Make outbound side of transport protocol dependent ([#13293](https://github.com/opensearch-project/OpenSearch/pull/13293)) - [Remote Store] Add dynamic cluster settings to set timeout for segments upload to Remote Store ([#13679](https://github.com/opensearch-project/OpenSearch/pull/13679)) - [Remote Store] Upload translog checkpoint as object metadata to translog.tlog([#13637](https://github.com/opensearch-project/OpenSearch/pull/13637)) +- Add getMetadataFields to MapperService ([#13819](https://github.com/opensearch-project/OpenSearch/pull/13819)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) diff --git a/server/src/main/java/org/opensearch/index/mapper/MapperService.java b/server/src/main/java/org/opensearch/index/mapper/MapperService.java index fc8654216e187..a1f3894c9f14c 100644 --- a/server/src/main/java/org/opensearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/opensearch/index/mapper/MapperService.java @@ -714,6 +714,13 @@ public boolean isMetadataField(String field) { return mapperRegistry.isMetadataField(field); } + /** + * Returns a set containing the registered metadata fields + */ + public Set getMetadataFields() { + return Collections.unmodifiableSet(mapperRegistry.getMetadataMapperParsers().keySet()); + } + /** * An analyzer wrapper that can lookup fields within the index mappings */ diff --git a/server/src/test/java/org/opensearch/index/mapper/MapperServiceTests.java b/server/src/test/java/org/opensearch/index/mapper/MapperServiceTests.java index adcfc9d7b17fc..b9c8866e69cbb 100644 --- a/server/src/test/java/org/opensearch/index/mapper/MapperServiceTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/MapperServiceTests.java @@ -52,6 +52,7 @@ import org.opensearch.index.mapper.KeywordFieldMapper.KeywordFieldType; import org.opensearch.index.mapper.MapperService.MergeReason; import org.opensearch.index.mapper.NumberFieldMapper.NumberFieldType; +import org.opensearch.indices.IndicesModule; import org.opensearch.indices.InvalidTypeNameException; import org.opensearch.indices.analysis.AnalysisModule.AnalysisProvider; import org.opensearch.plugins.AnalysisPlugin; @@ -90,6 +91,11 @@ public void testTypeValidation() { MapperService.validateTypeName("_doc"); // no exception } + public void testGetMetadataFieldsReturnsExpectedSet() throws Throwable { + final MapperService mapperService = createIndex("test1").mapperService(); + assertEquals(mapperService.getMetadataFields(), IndicesModule.getBuiltInMetadataFields()); + } + public void testPreflightUpdateDoesNotChangeMapping() throws Throwable { final MapperService mapperService = createIndex("test1").mapperService(); final CompressedXContent mapping = createMappingSpecifyingNumberOfFields(1); From 56d8dc65da163222ba9a3ba1b5c980422787b6f4 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Fri, 24 May 2024 14:54:22 -0400 Subject: [PATCH 09/77] Bump jackson from 2.17.0 to 2.17.1 (#13817) Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + buildSrc/version.properties | 4 ++-- client/sniffer/licenses/jackson-core-2.17.0.jar.sha1 | 1 - client/sniffer/licenses/jackson-core-2.17.1.jar.sha1 | 1 + .../upgrade-cli/licenses/jackson-annotations-2.17.0.jar.sha1 | 1 - .../upgrade-cli/licenses/jackson-annotations-2.17.1.jar.sha1 | 1 + .../upgrade-cli/licenses/jackson-databind-2.17.0.jar.sha1 | 1 - .../upgrade-cli/licenses/jackson-databind-2.17.1.jar.sha1 | 1 + libs/core/licenses/jackson-core-2.17.0.jar.sha1 | 1 - libs/core/licenses/jackson-core-2.17.1.jar.sha1 | 1 + libs/x-content/licenses/jackson-core-2.17.0.jar.sha1 | 1 - libs/x-content/licenses/jackson-core-2.17.1.jar.sha1 | 1 + .../licenses/jackson-dataformat-cbor-2.17.0.jar.sha1 | 1 - .../licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 | 1 + .../licenses/jackson-dataformat-smile-2.17.0.jar.sha1 | 1 - .../licenses/jackson-dataformat-smile-2.17.1.jar.sha1 | 1 + .../licenses/jackson-dataformat-yaml-2.17.0.jar.sha1 | 1 - .../licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 | 1 + .../ingest-geoip/licenses/jackson-annotations-2.17.0.jar.sha1 | 1 - .../ingest-geoip/licenses/jackson-annotations-2.17.1.jar.sha1 | 1 + .../ingest-geoip/licenses/jackson-databind-2.17.0.jar.sha1 | 1 - .../ingest-geoip/licenses/jackson-databind-2.17.1.jar.sha1 | 1 + .../crypto-kms/licenses/jackson-annotations-2.17.0.jar.sha1 | 1 - .../crypto-kms/licenses/jackson-annotations-2.17.1.jar.sha1 | 1 + plugins/crypto-kms/licenses/jackson-databind-2.17.0.jar.sha1 | 1 - plugins/crypto-kms/licenses/jackson-databind-2.17.1.jar.sha1 | 1 + .../licenses/jackson-annotations-2.17.0.jar.sha1 | 1 - .../licenses/jackson-annotations-2.17.1.jar.sha1 | 1 + .../discovery-ec2/licenses/jackson-databind-2.17.0.jar.sha1 | 1 - .../discovery-ec2/licenses/jackson-databind-2.17.1.jar.sha1 | 1 + .../licenses/jackson-annotations-2.17.0.jar.sha1 | 1 - .../licenses/jackson-annotations-2.17.1.jar.sha1 | 1 + .../licenses/jackson-databind-2.17.0.jar.sha1 | 1 - .../licenses/jackson-databind-2.17.1.jar.sha1 | 1 + .../licenses/jackson-dataformat-xml-2.17.0.jar.sha1 | 1 - .../licenses/jackson-dataformat-xml-2.17.1.jar.sha1 | 1 + .../licenses/jackson-datatype-jsr310-2.17.0.jar.sha1 | 1 - .../licenses/jackson-datatype-jsr310-2.17.1.jar.sha1 | 1 + .../licenses/jackson-module-jaxb-annotations-2.17.0.jar.sha1 | 1 - .../licenses/jackson-module-jaxb-annotations-2.17.1.jar.sha1 | 1 + .../licenses/jackson-annotations-2.17.0.jar.sha1 | 1 - .../licenses/jackson-annotations-2.17.1.jar.sha1 | 1 + .../repository-s3/licenses/jackson-databind-2.17.0.jar.sha1 | 1 - .../repository-s3/licenses/jackson-databind-2.17.1.jar.sha1 | 1 + 44 files changed, 24 insertions(+), 23 deletions(-) delete mode 100644 client/sniffer/licenses/jackson-core-2.17.0.jar.sha1 create mode 100644 client/sniffer/licenses/jackson-core-2.17.1.jar.sha1 delete mode 100644 distribution/tools/upgrade-cli/licenses/jackson-annotations-2.17.0.jar.sha1 create mode 100644 distribution/tools/upgrade-cli/licenses/jackson-annotations-2.17.1.jar.sha1 delete mode 100644 distribution/tools/upgrade-cli/licenses/jackson-databind-2.17.0.jar.sha1 create mode 100644 distribution/tools/upgrade-cli/licenses/jackson-databind-2.17.1.jar.sha1 delete mode 100644 libs/core/licenses/jackson-core-2.17.0.jar.sha1 create mode 100644 libs/core/licenses/jackson-core-2.17.1.jar.sha1 delete mode 100644 libs/x-content/licenses/jackson-core-2.17.0.jar.sha1 create mode 100644 libs/x-content/licenses/jackson-core-2.17.1.jar.sha1 delete mode 100644 libs/x-content/licenses/jackson-dataformat-cbor-2.17.0.jar.sha1 create mode 100644 libs/x-content/licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 delete mode 100644 libs/x-content/licenses/jackson-dataformat-smile-2.17.0.jar.sha1 create mode 100644 libs/x-content/licenses/jackson-dataformat-smile-2.17.1.jar.sha1 delete mode 100644 libs/x-content/licenses/jackson-dataformat-yaml-2.17.0.jar.sha1 create mode 100644 libs/x-content/licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 delete mode 100644 modules/ingest-geoip/licenses/jackson-annotations-2.17.0.jar.sha1 create mode 100644 modules/ingest-geoip/licenses/jackson-annotations-2.17.1.jar.sha1 delete mode 100644 modules/ingest-geoip/licenses/jackson-databind-2.17.0.jar.sha1 create mode 100644 modules/ingest-geoip/licenses/jackson-databind-2.17.1.jar.sha1 delete mode 100644 plugins/crypto-kms/licenses/jackson-annotations-2.17.0.jar.sha1 create mode 100644 plugins/crypto-kms/licenses/jackson-annotations-2.17.1.jar.sha1 delete mode 100644 plugins/crypto-kms/licenses/jackson-databind-2.17.0.jar.sha1 create mode 100644 plugins/crypto-kms/licenses/jackson-databind-2.17.1.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/jackson-annotations-2.17.0.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/jackson-annotations-2.17.1.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/jackson-databind-2.17.0.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/jackson-databind-2.17.1.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/jackson-annotations-2.17.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/jackson-annotations-2.17.1.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/jackson-databind-2.17.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/jackson-databind-2.17.1.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/jackson-dataformat-xml-2.17.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/jackson-dataformat-xml-2.17.1.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/jackson-datatype-jsr310-2.17.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/jackson-datatype-jsr310-2.17.1.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/jackson-module-jaxb-annotations-2.17.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/jackson-module-jaxb-annotations-2.17.1.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/jackson-annotations-2.17.0.jar.sha1 create mode 100644 plugins/repository-s3/licenses/jackson-annotations-2.17.1.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/jackson-databind-2.17.0.jar.sha1 create mode 100644 plugins/repository-s3/licenses/jackson-databind-2.17.1.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index e8d25e0337e30..937eae83c9e7c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -24,6 +24,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.google.code.gson:gson` from 2.10.1 to 2.11.0 ([#13752](https://github.com/opensearch-project/OpenSearch/pull/13752)) - Bump `ch.qos.logback:logback-core` from 1.5.3 to 1.5.6 ([#13756](https://github.com/opensearch-project/OpenSearch/pull/13756)) - Bump `netty` from 4.1.109.Final to 4.1.110.Final ([#13802](https://github.com/opensearch-project/OpenSearch/pull/13802)) +- Bump `jackson` from 2.17.0 to 2.17.1 ([#13817](https://github.com/opensearch-project/OpenSearch/pull/13817)) ### Changed - Add ability for Boolean and date field queries to run when only doc_values are enabled ([#11650](https://github.com/opensearch-project/OpenSearch/pull/11650)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index d8779cf8c2778..317e1a0129f1d 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -7,8 +7,8 @@ bundled_jdk = 21.0.3+9 # optional dependencies spatial4j = 0.7 jts = 1.15.0 -jackson = 2.17.0 -jackson_databind = 2.17.0 +jackson = 2.17.1 +jackson_databind = 2.17.1 snakeyaml = 2.1 icu4j = 70.1 supercsv = 2.4.0 diff --git a/client/sniffer/licenses/jackson-core-2.17.0.jar.sha1 b/client/sniffer/licenses/jackson-core-2.17.0.jar.sha1 deleted file mode 100644 index 9b906dbda1656..0000000000000 --- a/client/sniffer/licenses/jackson-core-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a6e5058ef9720623c517252d17162f845306ff3a \ No newline at end of file diff --git a/client/sniffer/licenses/jackson-core-2.17.1.jar.sha1 b/client/sniffer/licenses/jackson-core-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..82dab5981e652 --- /dev/null +++ b/client/sniffer/licenses/jackson-core-2.17.1.jar.sha1 @@ -0,0 +1 @@ +5e52a11644cd59a28ef79f02bddc2cc3bab45edb \ No newline at end of file diff --git a/distribution/tools/upgrade-cli/licenses/jackson-annotations-2.17.0.jar.sha1 b/distribution/tools/upgrade-cli/licenses/jackson-annotations-2.17.0.jar.sha1 deleted file mode 100644 index 66bf7ed6ecce8..0000000000000 --- a/distribution/tools/upgrade-cli/licenses/jackson-annotations-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -880a742337010da4c851f843d8cac150e22dff9f \ No newline at end of file diff --git a/distribution/tools/upgrade-cli/licenses/jackson-annotations-2.17.1.jar.sha1 b/distribution/tools/upgrade-cli/licenses/jackson-annotations-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..4ceead1b7ae4f --- /dev/null +++ b/distribution/tools/upgrade-cli/licenses/jackson-annotations-2.17.1.jar.sha1 @@ -0,0 +1 @@ +fca7ef6192c9ad05d07bc50da991bf937a84af3a \ No newline at end of file diff --git a/distribution/tools/upgrade-cli/licenses/jackson-databind-2.17.0.jar.sha1 b/distribution/tools/upgrade-cli/licenses/jackson-databind-2.17.0.jar.sha1 deleted file mode 100644 index c0e4bb0c56849..0000000000000 --- a/distribution/tools/upgrade-cli/licenses/jackson-databind-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7173e9e1d4bc6d7ca03bc4eeedcd548b8b580b34 \ No newline at end of file diff --git a/distribution/tools/upgrade-cli/licenses/jackson-databind-2.17.1.jar.sha1 b/distribution/tools/upgrade-cli/licenses/jackson-databind-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..7cf1ac1b60301 --- /dev/null +++ b/distribution/tools/upgrade-cli/licenses/jackson-databind-2.17.1.jar.sha1 @@ -0,0 +1 @@ +0524dcbcccdde7d45a679dfc333e4763feb09079 \ No newline at end of file diff --git a/libs/core/licenses/jackson-core-2.17.0.jar.sha1 b/libs/core/licenses/jackson-core-2.17.0.jar.sha1 deleted file mode 100644 index 9b906dbda1656..0000000000000 --- a/libs/core/licenses/jackson-core-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a6e5058ef9720623c517252d17162f845306ff3a \ No newline at end of file diff --git a/libs/core/licenses/jackson-core-2.17.1.jar.sha1 b/libs/core/licenses/jackson-core-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..82dab5981e652 --- /dev/null +++ b/libs/core/licenses/jackson-core-2.17.1.jar.sha1 @@ -0,0 +1 @@ +5e52a11644cd59a28ef79f02bddc2cc3bab45edb \ No newline at end of file diff --git a/libs/x-content/licenses/jackson-core-2.17.0.jar.sha1 b/libs/x-content/licenses/jackson-core-2.17.0.jar.sha1 deleted file mode 100644 index 9b906dbda1656..0000000000000 --- a/libs/x-content/licenses/jackson-core-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a6e5058ef9720623c517252d17162f845306ff3a \ No newline at end of file diff --git a/libs/x-content/licenses/jackson-core-2.17.1.jar.sha1 b/libs/x-content/licenses/jackson-core-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..82dab5981e652 --- /dev/null +++ b/libs/x-content/licenses/jackson-core-2.17.1.jar.sha1 @@ -0,0 +1 @@ +5e52a11644cd59a28ef79f02bddc2cc3bab45edb \ No newline at end of file diff --git a/libs/x-content/licenses/jackson-dataformat-cbor-2.17.0.jar.sha1 b/libs/x-content/licenses/jackson-dataformat-cbor-2.17.0.jar.sha1 deleted file mode 100644 index 382e20d3d31c1..0000000000000 --- a/libs/x-content/licenses/jackson-dataformat-cbor-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6833c8573452d583e4af650a7424d547606b2501 \ No newline at end of file diff --git a/libs/x-content/licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 b/libs/x-content/licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..ff42ed1f92cfe --- /dev/null +++ b/libs/x-content/licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 @@ -0,0 +1 @@ +ba5d8e6ecc62aa0e49c0ce935b8696352dbebc71 \ No newline at end of file diff --git a/libs/x-content/licenses/jackson-dataformat-smile-2.17.0.jar.sha1 b/libs/x-content/licenses/jackson-dataformat-smile-2.17.0.jar.sha1 deleted file mode 100644 index d117479166d17..0000000000000 --- a/libs/x-content/licenses/jackson-dataformat-smile-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f10183857607fde789490d33ea46372a2d2b0c72 \ No newline at end of file diff --git a/libs/x-content/licenses/jackson-dataformat-smile-2.17.1.jar.sha1 b/libs/x-content/licenses/jackson-dataformat-smile-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..47d19067cf2a6 --- /dev/null +++ b/libs/x-content/licenses/jackson-dataformat-smile-2.17.1.jar.sha1 @@ -0,0 +1 @@ +89683ac4f0a0c2c4f69ea56b90480ed40266dac8 \ No newline at end of file diff --git a/libs/x-content/licenses/jackson-dataformat-yaml-2.17.0.jar.sha1 b/libs/x-content/licenses/jackson-dataformat-yaml-2.17.0.jar.sha1 deleted file mode 100644 index 35242eed9b212..0000000000000 --- a/libs/x-content/licenses/jackson-dataformat-yaml-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -57a963c6258c49febc11390082d8503f71bb15a9 \ No newline at end of file diff --git a/libs/x-content/licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 b/libs/x-content/licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..7946e994c7104 --- /dev/null +++ b/libs/x-content/licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 @@ -0,0 +1 @@ +b4c7b8a9ea3f398116a75c146b982b22afebc4ee \ No newline at end of file diff --git a/modules/ingest-geoip/licenses/jackson-annotations-2.17.0.jar.sha1 b/modules/ingest-geoip/licenses/jackson-annotations-2.17.0.jar.sha1 deleted file mode 100644 index 66bf7ed6ecce8..0000000000000 --- a/modules/ingest-geoip/licenses/jackson-annotations-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -880a742337010da4c851f843d8cac150e22dff9f \ No newline at end of file diff --git a/modules/ingest-geoip/licenses/jackson-annotations-2.17.1.jar.sha1 b/modules/ingest-geoip/licenses/jackson-annotations-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..4ceead1b7ae4f --- /dev/null +++ b/modules/ingest-geoip/licenses/jackson-annotations-2.17.1.jar.sha1 @@ -0,0 +1 @@ +fca7ef6192c9ad05d07bc50da991bf937a84af3a \ No newline at end of file diff --git a/modules/ingest-geoip/licenses/jackson-databind-2.17.0.jar.sha1 b/modules/ingest-geoip/licenses/jackson-databind-2.17.0.jar.sha1 deleted file mode 100644 index c0e4bb0c56849..0000000000000 --- a/modules/ingest-geoip/licenses/jackson-databind-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7173e9e1d4bc6d7ca03bc4eeedcd548b8b580b34 \ No newline at end of file diff --git a/modules/ingest-geoip/licenses/jackson-databind-2.17.1.jar.sha1 b/modules/ingest-geoip/licenses/jackson-databind-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..7cf1ac1b60301 --- /dev/null +++ b/modules/ingest-geoip/licenses/jackson-databind-2.17.1.jar.sha1 @@ -0,0 +1 @@ +0524dcbcccdde7d45a679dfc333e4763feb09079 \ No newline at end of file diff --git a/plugins/crypto-kms/licenses/jackson-annotations-2.17.0.jar.sha1 b/plugins/crypto-kms/licenses/jackson-annotations-2.17.0.jar.sha1 deleted file mode 100644 index 66bf7ed6ecce8..0000000000000 --- a/plugins/crypto-kms/licenses/jackson-annotations-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -880a742337010da4c851f843d8cac150e22dff9f \ No newline at end of file diff --git a/plugins/crypto-kms/licenses/jackson-annotations-2.17.1.jar.sha1 b/plugins/crypto-kms/licenses/jackson-annotations-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..4ceead1b7ae4f --- /dev/null +++ b/plugins/crypto-kms/licenses/jackson-annotations-2.17.1.jar.sha1 @@ -0,0 +1 @@ +fca7ef6192c9ad05d07bc50da991bf937a84af3a \ No newline at end of file diff --git a/plugins/crypto-kms/licenses/jackson-databind-2.17.0.jar.sha1 b/plugins/crypto-kms/licenses/jackson-databind-2.17.0.jar.sha1 deleted file mode 100644 index c0e4bb0c56849..0000000000000 --- a/plugins/crypto-kms/licenses/jackson-databind-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7173e9e1d4bc6d7ca03bc4eeedcd548b8b580b34 \ No newline at end of file diff --git a/plugins/crypto-kms/licenses/jackson-databind-2.17.1.jar.sha1 b/plugins/crypto-kms/licenses/jackson-databind-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..7cf1ac1b60301 --- /dev/null +++ b/plugins/crypto-kms/licenses/jackson-databind-2.17.1.jar.sha1 @@ -0,0 +1 @@ +0524dcbcccdde7d45a679dfc333e4763feb09079 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/jackson-annotations-2.17.0.jar.sha1 b/plugins/discovery-ec2/licenses/jackson-annotations-2.17.0.jar.sha1 deleted file mode 100644 index 66bf7ed6ecce8..0000000000000 --- a/plugins/discovery-ec2/licenses/jackson-annotations-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -880a742337010da4c851f843d8cac150e22dff9f \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/jackson-annotations-2.17.1.jar.sha1 b/plugins/discovery-ec2/licenses/jackson-annotations-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..4ceead1b7ae4f --- /dev/null +++ b/plugins/discovery-ec2/licenses/jackson-annotations-2.17.1.jar.sha1 @@ -0,0 +1 @@ +fca7ef6192c9ad05d07bc50da991bf937a84af3a \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/jackson-databind-2.17.0.jar.sha1 b/plugins/discovery-ec2/licenses/jackson-databind-2.17.0.jar.sha1 deleted file mode 100644 index c0e4bb0c56849..0000000000000 --- a/plugins/discovery-ec2/licenses/jackson-databind-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7173e9e1d4bc6d7ca03bc4eeedcd548b8b580b34 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/jackson-databind-2.17.1.jar.sha1 b/plugins/discovery-ec2/licenses/jackson-databind-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..7cf1ac1b60301 --- /dev/null +++ b/plugins/discovery-ec2/licenses/jackson-databind-2.17.1.jar.sha1 @@ -0,0 +1 @@ +0524dcbcccdde7d45a679dfc333e4763feb09079 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-annotations-2.17.0.jar.sha1 b/plugins/repository-azure/licenses/jackson-annotations-2.17.0.jar.sha1 deleted file mode 100644 index 66bf7ed6ecce8..0000000000000 --- a/plugins/repository-azure/licenses/jackson-annotations-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -880a742337010da4c851f843d8cac150e22dff9f \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-annotations-2.17.1.jar.sha1 b/plugins/repository-azure/licenses/jackson-annotations-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..4ceead1b7ae4f --- /dev/null +++ b/plugins/repository-azure/licenses/jackson-annotations-2.17.1.jar.sha1 @@ -0,0 +1 @@ +fca7ef6192c9ad05d07bc50da991bf937a84af3a \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-databind-2.17.0.jar.sha1 b/plugins/repository-azure/licenses/jackson-databind-2.17.0.jar.sha1 deleted file mode 100644 index c0e4bb0c56849..0000000000000 --- a/plugins/repository-azure/licenses/jackson-databind-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7173e9e1d4bc6d7ca03bc4eeedcd548b8b580b34 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-databind-2.17.1.jar.sha1 b/plugins/repository-azure/licenses/jackson-databind-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..7cf1ac1b60301 --- /dev/null +++ b/plugins/repository-azure/licenses/jackson-databind-2.17.1.jar.sha1 @@ -0,0 +1 @@ +0524dcbcccdde7d45a679dfc333e4763feb09079 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-dataformat-xml-2.17.0.jar.sha1 b/plugins/repository-azure/licenses/jackson-dataformat-xml-2.17.0.jar.sha1 deleted file mode 100644 index 9dea3dfc55691..0000000000000 --- a/plugins/repository-azure/licenses/jackson-dataformat-xml-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -fbe3c274a39cef5538ca8688ac7e2ad0053a6ffa \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-dataformat-xml-2.17.1.jar.sha1 b/plugins/repository-azure/licenses/jackson-dataformat-xml-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..3915ab2616beb --- /dev/null +++ b/plugins/repository-azure/licenses/jackson-dataformat-xml-2.17.1.jar.sha1 @@ -0,0 +1 @@ +e6a168dba62aa63743b9e2b83f4e0f0dfdc143d3 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-datatype-jsr310-2.17.0.jar.sha1 b/plugins/repository-azure/licenses/jackson-datatype-jsr310-2.17.0.jar.sha1 deleted file mode 100644 index fe8e51b8e0869..0000000000000 --- a/plugins/repository-azure/licenses/jackson-datatype-jsr310-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3fab507bba9d477e52ed2302dc3ddbd23cbae339 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-datatype-jsr310-2.17.1.jar.sha1 b/plugins/repository-azure/licenses/jackson-datatype-jsr310-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..db26ebbf738f7 --- /dev/null +++ b/plugins/repository-azure/licenses/jackson-datatype-jsr310-2.17.1.jar.sha1 @@ -0,0 +1 @@ +0969b0c3cb8c75d759e9a6c585c44c9b9f3a4f75 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-module-jaxb-annotations-2.17.0.jar.sha1 b/plugins/repository-azure/licenses/jackson-module-jaxb-annotations-2.17.0.jar.sha1 deleted file mode 100644 index 3954ac9c39af3..0000000000000 --- a/plugins/repository-azure/licenses/jackson-module-jaxb-annotations-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e07032ce170277213ac4835169ca79fa0340c7b5 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jackson-module-jaxb-annotations-2.17.1.jar.sha1 b/plugins/repository-azure/licenses/jackson-module-jaxb-annotations-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..bb8ecfe34d295 --- /dev/null +++ b/plugins/repository-azure/licenses/jackson-module-jaxb-annotations-2.17.1.jar.sha1 @@ -0,0 +1 @@ +f77e7bf0e64dfcf53bfdcf2764ad7ab92b78a4da \ No newline at end of file diff --git a/plugins/repository-s3/licenses/jackson-annotations-2.17.0.jar.sha1 b/plugins/repository-s3/licenses/jackson-annotations-2.17.0.jar.sha1 deleted file mode 100644 index 66bf7ed6ecce8..0000000000000 --- a/plugins/repository-s3/licenses/jackson-annotations-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -880a742337010da4c851f843d8cac150e22dff9f \ No newline at end of file diff --git a/plugins/repository-s3/licenses/jackson-annotations-2.17.1.jar.sha1 b/plugins/repository-s3/licenses/jackson-annotations-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..4ceead1b7ae4f --- /dev/null +++ b/plugins/repository-s3/licenses/jackson-annotations-2.17.1.jar.sha1 @@ -0,0 +1 @@ +fca7ef6192c9ad05d07bc50da991bf937a84af3a \ No newline at end of file diff --git a/plugins/repository-s3/licenses/jackson-databind-2.17.0.jar.sha1 b/plugins/repository-s3/licenses/jackson-databind-2.17.0.jar.sha1 deleted file mode 100644 index c0e4bb0c56849..0000000000000 --- a/plugins/repository-s3/licenses/jackson-databind-2.17.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7173e9e1d4bc6d7ca03bc4eeedcd548b8b580b34 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/jackson-databind-2.17.1.jar.sha1 b/plugins/repository-s3/licenses/jackson-databind-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..7cf1ac1b60301 --- /dev/null +++ b/plugins/repository-s3/licenses/jackson-databind-2.17.1.jar.sha1 @@ -0,0 +1 @@ +0524dcbcccdde7d45a679dfc333e4763feb09079 \ No newline at end of file From 5cde176918744b9640d2ef84cf9ede4226950cea Mon Sep 17 00:00:00 2001 From: Muneer Kolarkunnu <33829651+akolarkunnu@users.noreply.github.com> Date: Sat, 25 May 2024 06:51:09 +0530 Subject: [PATCH 10/77] org.opensearch.repositories.s3.S3BlobStoreRepositoryTests.testRequestStats fails with NullPointerException (#13814) It's a NullPointerException from S3BlobStore.extendedStats() method, where 'genericStatsMetricPublisher' is null. This parameter sets through S3Repository constructor from test and in this test it sets as null. This is the root cause of the issue. If we set valid a GenericStatsMetricPublisher, test works fine without any issue. This was a consistent failure, not a random failure. Resolves #10735 Signed-off-by: akolarkunnu Co-authored-by: akolarkunnu --- .../repositories/s3/S3BlobStoreRepositoryTests.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java index 21184380d54a9..f72374d63808a 100644 --- a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java +++ b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java @@ -165,7 +165,6 @@ protected Settings nodeSettings(int nodeOrdinal) { return builder.build(); } - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/10735") @Override public void testRequestStats() throws Exception { final String repository = createRepository(randomName()); @@ -249,6 +248,8 @@ protected S3Repository createRepository( ClusterService clusterService, RecoverySettings recoverySettings ) { + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + return new S3Repository( metadata, registry, @@ -263,7 +264,7 @@ protected S3Repository createRepository( false, null, null, - null + genericStatsMetricPublisher ) { @Override From 1c0a274e2abe2ad353dcc3a5c5d0e787310bce1d Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Fri, 24 May 2024 22:58:22 -0400 Subject: [PATCH 11/77] Bump reactor from 3.5.15 to 3.5.17 and reactor-netty from 1.1.17 to 1.1.19 (#13825) Signed-off-by: Andriy Redko --- CHANGELOG.md | 2 ++ buildSrc/version.properties | 4 ++-- .../licenses/reactor-netty-core-1.1.17.jar.sha1 | 1 - .../licenses/reactor-netty-core-1.1.19.jar.sha1 | 1 + .../licenses/reactor-netty-http-1.1.17.jar.sha1 | 1 - .../licenses/reactor-netty-http-1.1.19.jar.sha1 | 1 + .../licenses/reactor-netty-core-1.1.17.jar.sha1 | 1 - .../licenses/reactor-netty-core-1.1.19.jar.sha1 | 1 + .../licenses/reactor-netty-http-1.1.17.jar.sha1 | 1 - .../licenses/reactor-netty-http-1.1.19.jar.sha1 | 1 + server/licenses/reactor-core-3.5.15.jar.sha1 | 1 - server/licenses/reactor-core-3.5.17.jar.sha1 | 1 + 12 files changed, 9 insertions(+), 7 deletions(-) delete mode 100644 plugins/repository-azure/licenses/reactor-netty-core-1.1.17.jar.sha1 create mode 100644 plugins/repository-azure/licenses/reactor-netty-core-1.1.19.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/reactor-netty-http-1.1.17.jar.sha1 create mode 100644 plugins/repository-azure/licenses/reactor-netty-http-1.1.19.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.17.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.19.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.17.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.19.jar.sha1 delete mode 100644 server/licenses/reactor-core-3.5.15.jar.sha1 create mode 100644 server/licenses/reactor-core-3.5.17.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 937eae83c9e7c..74f9b1b113248 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -25,6 +25,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `ch.qos.logback:logback-core` from 1.5.3 to 1.5.6 ([#13756](https://github.com/opensearch-project/OpenSearch/pull/13756)) - Bump `netty` from 4.1.109.Final to 4.1.110.Final ([#13802](https://github.com/opensearch-project/OpenSearch/pull/13802)) - Bump `jackson` from 2.17.0 to 2.17.1 ([#13817](https://github.com/opensearch-project/OpenSearch/pull/13817)) +- Bump `reactor` from 3.5.15 to 3.5.17 ([#13825](https://github.com/opensearch-project/OpenSearch/pull/13825)) +- Bump `reactor-netty` from 1.1.17 to 1.1.19 ([#13825](https://github.com/opensearch-project/OpenSearch/pull/13825)) ### Changed - Add ability for Boolean and date field queries to run when only doc_values are enabled ([#11650](https://github.com/opensearch-project/OpenSearch/pull/11650)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 317e1a0129f1d..3a8d97096d415 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -31,8 +31,8 @@ netty = 4.1.110.Final joda = 2.12.7 # project reactor -reactor_netty = 1.1.17 -reactor = 3.5.15 +reactor_netty = 1.1.19 +reactor = 3.5.17 # client dependencies httpclient5 = 5.2.1 diff --git a/plugins/repository-azure/licenses/reactor-netty-core-1.1.17.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-core-1.1.17.jar.sha1 deleted file mode 100644 index 3d631bc904f24..0000000000000 --- a/plugins/repository-azure/licenses/reactor-netty-core-1.1.17.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -319b1d41f28e92b31b7ca0f19183337f5539bb44 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/reactor-netty-core-1.1.19.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-core-1.1.19.jar.sha1 new file mode 100644 index 0000000000000..cbcbfcd87d682 --- /dev/null +++ b/plugins/repository-azure/licenses/reactor-netty-core-1.1.19.jar.sha1 @@ -0,0 +1 @@ +639e2c63ade6f2a49d7e501ca2264b74d240b448 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/reactor-netty-http-1.1.17.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-http-1.1.17.jar.sha1 deleted file mode 100644 index 9ceef6959744b..0000000000000 --- a/plugins/repository-azure/licenses/reactor-netty-http-1.1.17.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9ed949dcd050ef30d9eeedd53d95d1dce20ce832 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/reactor-netty-http-1.1.19.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-http-1.1.19.jar.sha1 new file mode 100644 index 0000000000000..1eeedfc0926f5 --- /dev/null +++ b/plugins/repository-azure/licenses/reactor-netty-http-1.1.19.jar.sha1 @@ -0,0 +1 @@ +b4bbb1aeb64ecb2b3949c38983032a7f0b0ebd07 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.17.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.17.jar.sha1 deleted file mode 100644 index 3d631bc904f24..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.17.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -319b1d41f28e92b31b7ca0f19183337f5539bb44 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.19.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.19.jar.sha1 new file mode 100644 index 0000000000000..cbcbfcd87d682 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.19.jar.sha1 @@ -0,0 +1 @@ +639e2c63ade6f2a49d7e501ca2264b74d240b448 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.17.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.17.jar.sha1 deleted file mode 100644 index 9ceef6959744b..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.17.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9ed949dcd050ef30d9eeedd53d95d1dce20ce832 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.19.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.19.jar.sha1 new file mode 100644 index 0000000000000..1eeedfc0926f5 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.19.jar.sha1 @@ -0,0 +1 @@ +b4bbb1aeb64ecb2b3949c38983032a7f0b0ebd07 \ No newline at end of file diff --git a/server/licenses/reactor-core-3.5.15.jar.sha1 b/server/licenses/reactor-core-3.5.15.jar.sha1 deleted file mode 100644 index 02df47ed58b9d..0000000000000 --- a/server/licenses/reactor-core-3.5.15.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4e07a24c671235a2a806e75e9b8ff23d7d1db3d4 \ No newline at end of file diff --git a/server/licenses/reactor-core-3.5.17.jar.sha1 b/server/licenses/reactor-core-3.5.17.jar.sha1 new file mode 100644 index 0000000000000..6663356bab047 --- /dev/null +++ b/server/licenses/reactor-core-3.5.17.jar.sha1 @@ -0,0 +1 @@ +2cf9b080e3a2d8a5a39948260db5fd1dae54c3ac \ No newline at end of file From 09c2c7d798bf12ee8e422720cc56d0bb86cea393 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 May 2024 10:14:40 -0400 Subject: [PATCH 12/77] Bump commons-cli:commons-cli from 1.7.0 to 1.8.0 in /plugins/repository-hdfs (#13840) * Bump commons-cli:commons-cli in /plugins/repository-hdfs Bumps commons-cli:commons-cli from 1.7.0 to 1.8.0. --- updated-dependencies: - dependency-name: commons-cli:commons-cli 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 +- plugins/repository-hdfs/licenses/commons-cli-1.7.0.jar.sha1 | 1 - plugins/repository-hdfs/licenses/commons-cli-1.8.0.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 2 deletions(-) delete mode 100644 plugins/repository-hdfs/licenses/commons-cli-1.7.0.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/commons-cli-1.8.0.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 74f9b1b113248..722850e3a5705 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -27,6 +27,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `jackson` from 2.17.0 to 2.17.1 ([#13817](https://github.com/opensearch-project/OpenSearch/pull/13817)) - Bump `reactor` from 3.5.15 to 3.5.17 ([#13825](https://github.com/opensearch-project/OpenSearch/pull/13825)) - Bump `reactor-netty` from 1.1.17 to 1.1.19 ([#13825](https://github.com/opensearch-project/OpenSearch/pull/13825)) +- Bump `commons-cli:commons-cli` from 1.7.0 to 1.8.0 ([#13840](https://github.com/opensearch-project/OpenSearch/pull/13840)) ### Changed - Add ability for Boolean and date field queries to run when only doc_values are enabled ([#11650](https://github.com/opensearch-project/OpenSearch/pull/11650)) diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle index d97ff72cf2ebb..eb50bd2d0615a 100644 --- a/plugins/repository-hdfs/build.gradle +++ b/plugins/repository-hdfs/build.gradle @@ -70,7 +70,7 @@ dependencies { api 'com.google.code.gson:gson:2.11.0' runtimeOnly "com.google.guava:guava:${versions.guava}" api "commons-logging:commons-logging:${versions.commonslogging}" - api 'commons-cli:commons-cli:1.7.0' + api 'commons-cli:commons-cli:1.8.0' api "commons-codec:commons-codec:${versions.commonscodec}" api 'commons-collections:commons-collections:3.2.2' api "org.apache.commons:commons-compress:${versions.commonscompress}" diff --git a/plugins/repository-hdfs/licenses/commons-cli-1.7.0.jar.sha1 b/plugins/repository-hdfs/licenses/commons-cli-1.7.0.jar.sha1 deleted file mode 100644 index 759bc9275d346..0000000000000 --- a/plugins/repository-hdfs/licenses/commons-cli-1.7.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6504b3f17e8bc5adc6b6c8deecc90144d0154075 \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/commons-cli-1.8.0.jar.sha1 b/plugins/repository-hdfs/licenses/commons-cli-1.8.0.jar.sha1 new file mode 100644 index 0000000000000..65102052409ea --- /dev/null +++ b/plugins/repository-hdfs/licenses/commons-cli-1.8.0.jar.sha1 @@ -0,0 +1 @@ +41a4bff12057eecb6daaf9c7f36c237815be3da1 \ No newline at end of file From 595959f3442070f797fcffd9bf079e5563575572 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 May 2024 12:14:04 -0400 Subject: [PATCH 13/77] Bump org.apache.xmlbeans:xmlbeans from 5.2.0 to 5.2.1 in /plugins/ingest-attachment (#13839) * Bump org.apache.xmlbeans:xmlbeans in /plugins/ingest-attachment Bumps org.apache.xmlbeans:xmlbeans from 5.2.0 to 5.2.1. --- updated-dependencies: - dependency-name: org.apache.xmlbeans:xmlbeans dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] * Updating SHAs 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: opensearch-trigger-bot[bot] Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + plugins/ingest-attachment/build.gradle | 2 +- plugins/ingest-attachment/licenses/xmlbeans-5.2.0.jar.sha1 | 1 - plugins/ingest-attachment/licenses/xmlbeans-5.2.1.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 2 deletions(-) delete mode 100644 plugins/ingest-attachment/licenses/xmlbeans-5.2.0.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/xmlbeans-5.2.1.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 722850e3a5705..a0b5dd289ec85 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -28,6 +28,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `reactor` from 3.5.15 to 3.5.17 ([#13825](https://github.com/opensearch-project/OpenSearch/pull/13825)) - Bump `reactor-netty` from 1.1.17 to 1.1.19 ([#13825](https://github.com/opensearch-project/OpenSearch/pull/13825)) - Bump `commons-cli:commons-cli` from 1.7.0 to 1.8.0 ([#13840](https://github.com/opensearch-project/OpenSearch/pull/13840)) +- Bump `org.apache.xmlbeans:xmlbeans` from 5.2.0 to 5.2.1 ([#13839](https://github.com/opensearch-project/OpenSearch/pull/13839)) ### Changed - Add ability for Boolean and date field queries to run when only doc_values are enabled ([#11650](https://github.com/opensearch-project/OpenSearch/pull/11650)) diff --git a/plugins/ingest-attachment/build.gradle b/plugins/ingest-attachment/build.gradle index 4749aa911886d..d631855013527 100644 --- a/plugins/ingest-attachment/build.gradle +++ b/plugins/ingest-attachment/build.gradle @@ -89,7 +89,7 @@ dependencies { api "org.apache.poi:poi:${versions.poi}" api "org.apache.poi:poi-ooxml-lite:${versions.poi}" api "commons-codec:commons-codec:${versions.commonscodec}" - api 'org.apache.xmlbeans:xmlbeans:5.2.0' + api 'org.apache.xmlbeans:xmlbeans:5.2.1' api 'org.apache.commons:commons-collections4:4.4' // MS Office api "org.apache.poi:poi-scratchpad:${versions.poi}" diff --git a/plugins/ingest-attachment/licenses/xmlbeans-5.2.0.jar.sha1 b/plugins/ingest-attachment/licenses/xmlbeans-5.2.0.jar.sha1 deleted file mode 100644 index f34274d593697..0000000000000 --- a/plugins/ingest-attachment/licenses/xmlbeans-5.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6198ac997b3f234f2b5393fa415f78fac2e06510 \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/xmlbeans-5.2.1.jar.sha1 b/plugins/ingest-attachment/licenses/xmlbeans-5.2.1.jar.sha1 new file mode 100644 index 0000000000000..eaab556163e5c --- /dev/null +++ b/plugins/ingest-attachment/licenses/xmlbeans-5.2.1.jar.sha1 @@ -0,0 +1 @@ +e16ddf17fe181c202b097e0dcc0ee2fed91cb7da \ No newline at end of file From 994bfae0c0beddd8416ab88ed6138e2792638c4e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 May 2024 14:17:05 -0400 Subject: [PATCH 14/77] Bump com.google.code.gson:gson in /test/fixtures/hdfs-fixture (#13842) Bumps [com.google.code.gson:gson](https://github.com/google/gson) from 2.10.1 to 2.11.0. - [Release notes](https://github.com/google/gson/releases) - [Changelog](https://github.com/google/gson/blob/main/CHANGELOG.md) - [Commits](https://github.com/google/gson/compare/gson-parent-2.10.1...gson-parent-2.11.0) --- updated-dependencies: - dependency-name: com.google.code.gson:gson dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- test/fixtures/hdfs-fixture/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/fixtures/hdfs-fixture/build.gradle b/test/fixtures/hdfs-fixture/build.gradle index d8d5d95651b5e..ddb876b46fd1c 100644 --- a/test/fixtures/hdfs-fixture/build.gradle +++ b/test/fixtures/hdfs-fixture/build.gradle @@ -58,7 +58,7 @@ dependencies { api "commons-codec:commons-codec:${versions.commonscodec}" api "org.apache.logging.log4j:log4j-core:${versions.log4j}" api "io.netty:netty-all:${versions.netty}" - api 'com.google.code.gson:gson:2.10.1' + api 'com.google.code.gson:gson:2.11.0' api "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:${versions.jackson}" api "com.fasterxml.jackson.core:jackson-databind:${versions.jackson_databind}" api "com.fasterxml.woodstox:woodstox-core:${versions.woodstox}" From 45360db1bcef97ae363463d1295a71332e31aef2 Mon Sep 17 00:00:00 2001 From: Navneet Verma Date: Tue, 28 May 2024 07:49:35 -0700 Subject: [PATCH 15/77] Update to Apache Lucene 9.11.0-snapshot-4be6531 (#13850) Signed-off-by: Navneet Verma --- buildSrc/version.properties | 2 +- .../lucene-core-9.11.0-snapshot-4be6531.jar.sha1 | 1 + .../lucene-core-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...ene-expressions-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...ene-expressions-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...ne-analysis-icu-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...ne-analysis-icu-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...alysis-kuromoji-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...alysis-kuromoji-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...e-analysis-nori-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...e-analysis-nori-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...alysis-phonetic-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...alysis-phonetic-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...nalysis-smartcn-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...nalysis-smartcn-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...nalysis-stempel-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...nalysis-stempel-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...ysis-morfologik-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...ysis-morfologik-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...analysis-common-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...analysis-common-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...backward-codecs-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...backward-codecs-9.11.0-snapshot-fb97840.jar.sha1 | 1 - .../lucene-core-9.11.0-snapshot-4be6531.jar.sha1 | 1 + .../lucene-core-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...lucene-grouping-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...lucene-grouping-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...ene-highlighter-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...ene-highlighter-9.11.0-snapshot-fb97840.jar.sha1 | 1 - .../lucene-join-9.11.0-snapshot-4be6531.jar.sha1 | 1 + .../lucene-join-9.11.0-snapshot-fb97840.jar.sha1 | 1 - .../lucene-memory-9.11.0-snapshot-4be6531.jar.sha1 | 1 + .../lucene-memory-9.11.0-snapshot-fb97840.jar.sha1 | 1 - .../lucene-misc-9.11.0-snapshot-4be6531.jar.sha1 | 1 + .../lucene-misc-9.11.0-snapshot-fb97840.jar.sha1 | 1 - .../lucene-queries-9.11.0-snapshot-4be6531.jar.sha1 | 1 + .../lucene-queries-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...ene-queryparser-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...ene-queryparser-9.11.0-snapshot-fb97840.jar.sha1 | 1 - .../lucene-sandbox-9.11.0-snapshot-4be6531.jar.sha1 | 1 + .../lucene-sandbox-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...-spatial-extras-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...-spatial-extras-9.11.0-snapshot-fb97840.jar.sha1 | 1 - ...ucene-spatial3d-9.11.0-snapshot-4be6531.jar.sha1 | 1 + ...ucene-spatial3d-9.11.0-snapshot-fb97840.jar.sha1 | 1 - .../lucene-suggest-9.11.0-snapshot-4be6531.jar.sha1 | 1 + .../lucene-suggest-9.11.0-snapshot-fb97840.jar.sha1 | 1 - .../search/uhighlight/CustomFieldHighlighter.java | 13 ++++++++++++- .../subphase/highlight/FastVectorHighlighter.java | 2 +- .../fetch/subphase/highlight/HighlightBuilder.java | 4 +++- 50 files changed, 40 insertions(+), 27 deletions(-) create mode 100644 libs/core/licenses/lucene-core-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 libs/core/licenses/lucene-core-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 modules/lang-expression/licenses/lucene-expressions-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 modules/lang-expression/licenses/lucene-expressions-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-analysis-common-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-analysis-common-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-backward-codecs-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-backward-codecs-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-core-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-core-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-grouping-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-grouping-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-highlighter-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-highlighter-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-join-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-join-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-memory-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-memory-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-misc-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-misc-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-queries-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-queries-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-queryparser-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-queryparser-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-sandbox-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-sandbox-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-spatial-extras-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-spatial-extras-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-spatial3d-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-spatial3d-9.11.0-snapshot-fb97840.jar.sha1 create mode 100644 server/licenses/lucene-suggest-9.11.0-snapshot-4be6531.jar.sha1 delete mode 100644 server/licenses/lucene-suggest-9.11.0-snapshot-fb97840.jar.sha1 diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 3a8d97096d415..0a36ed5e200f7 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -1,5 +1,5 @@ opensearch = 3.0.0 -lucene = 9.11.0-snapshot-fb97840 +lucene = 9.11.0-snapshot-4be6531 bundled_jdk_vendor = adoptium bundled_jdk = 21.0.3+9 diff --git a/libs/core/licenses/lucene-core-9.11.0-snapshot-4be6531.jar.sha1 b/libs/core/licenses/lucene-core-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..943a9b2fd214b --- /dev/null +++ b/libs/core/licenses/lucene-core-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +3c2361bd633374ae3814b175cc25ccf773f67026 \ No newline at end of file diff --git a/libs/core/licenses/lucene-core-9.11.0-snapshot-fb97840.jar.sha1 b/libs/core/licenses/lucene-core-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 88309bc46411a..0000000000000 --- a/libs/core/licenses/lucene-core-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -53a828e3e88f55c83979cd3df0704617cc9edb9a \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-9.11.0-snapshot-4be6531.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..ee00419f52066 --- /dev/null +++ b/modules/lang-expression/licenses/lucene-expressions-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +8752daf173a642ae02e081cc0398f2ce59278200 \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-9.11.0-snapshot-fb97840.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index f4da6e39aeeb8..0000000000000 --- a/modules/lang-expression/licenses/lucene-expressions-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ab914b48665f484315b78e4b6787aa42f5966bb6 \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0-snapshot-4be6531.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..04338d8933590 --- /dev/null +++ b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +12630ff9c56e2a372ba57f519c579ff9e728208a \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0-snapshot-fb97840.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 594733c11402c..0000000000000 --- a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f9cd7bec33c8cf3b891976cb674ffe9c97f8747f \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0-snapshot-4be6531.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..b8da0dacfe9f1 --- /dev/null +++ b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +752bfc61c7829be6c27d9c1764250196e2c6b06b \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0-snapshot-fb97840.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index c46e06b8c87e4..0000000000000 --- a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c244a56bf7cd171a19379c96f1d20c477a34578d \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0-snapshot-4be6531.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..b58adc03938f3 --- /dev/null +++ b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +5ca56d42b24498a226cf91f48b94e010b6af5fe2 \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0-snapshot-fb97840.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index a79c34a127920..0000000000000 --- a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -da26df43f2b0d7c2dfecbf208cae0772a5e382ca \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0-snapshot-4be6531.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..dea962647d995 --- /dev/null +++ b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +8eb59a89aa8984457798ccffb8e97e5351bebc1f \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0-snapshot-fb97840.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index f2b08020be1ad..0000000000000 --- a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f752ffa5ee4697b04643214236138f3defdee2f4 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0-snapshot-4be6531.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..1259b95a789a5 --- /dev/null +++ b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +851c1bd99eaef368e84335853dd448e4f56cdbc8 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0-snapshot-fb97840.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 969a05905eaf0..0000000000000 --- a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -73fe44fe755aef72e7293b2ffdd934beb631429d \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0-snapshot-4be6531.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..8c0d8fd278b89 --- /dev/null +++ b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +63647085d41ae231733580c20a498ce7c9134ce5 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0-snapshot-fb97840.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index fdf0bd39e217e..0000000000000 --- a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c2b48831b25e1c7e8f683a63d1505c2d133256d3 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0-snapshot-4be6531.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..0eb1fb5f2b31f --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +a3ba7dd03b1df9efed08eb544689d51d2be22aa5 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0-snapshot-fb97840.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 0042415700453..0000000000000 --- a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -757f8b29f103f82b6fb6948634e93dd497c9d7a8 \ No newline at end of file diff --git a/server/licenses/lucene-analysis-common-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-analysis-common-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..08339fa8a4ce1 --- /dev/null +++ b/server/licenses/lucene-analysis-common-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +9cc4e600289bf1171b47de74536bd34c476f85a8 \ No newline at end of file diff --git a/server/licenses/lucene-analysis-common-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-analysis-common-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index f229c373aa1af..0000000000000 --- a/server/licenses/lucene-analysis-common-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -cd0afb5da5dcb4c7498bd1ee7f7bab0e289404b8 \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-backward-codecs-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..3dce8a2162edd --- /dev/null +++ b/server/licenses/lucene-backward-codecs-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +8babfe85be7e36c893741e08072c11e71db09715 \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-backward-codecs-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index edaf28a7f6e76..0000000000000 --- a/server/licenses/lucene-backward-codecs-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -88888315cd60e565960ae2e6fed2af0df077a2a2 \ No newline at end of file diff --git a/server/licenses/lucene-core-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-core-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..943a9b2fd214b --- /dev/null +++ b/server/licenses/lucene-core-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +3c2361bd633374ae3814b175cc25ccf773f67026 \ No newline at end of file diff --git a/server/licenses/lucene-core-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-core-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 88309bc46411a..0000000000000 --- a/server/licenses/lucene-core-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -53a828e3e88f55c83979cd3df0704617cc9edb9a \ No newline at end of file diff --git a/server/licenses/lucene-grouping-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-grouping-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..8587c3ed5e82a --- /dev/null +++ b/server/licenses/lucene-grouping-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +d9f29b49cd1e0a061ff7fa4a53e8605bd49bd3d0 \ No newline at end of file diff --git a/server/licenses/lucene-grouping-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-grouping-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 13f1276e3b033..0000000000000 --- a/server/licenses/lucene-grouping-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d1f54a816c9d85e890a862a2dffdc734ece2770c \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-highlighter-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..25579432a9cbd --- /dev/null +++ b/server/licenses/lucene-highlighter-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +33bc26d46d62bb1cf3bf725db637226a43db7625 \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-highlighter-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 6cef51ac4453f..0000000000000 --- a/server/licenses/lucene-highlighter-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1e7c9336fa86fb866fcd76ea5d6283c804b4d580 \ No newline at end of file diff --git a/server/licenses/lucene-join-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-join-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..1bfef89965e67 --- /dev/null +++ b/server/licenses/lucene-join-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +82966698abdb8f0367a162f642560566a6085dc8 \ No newline at end of file diff --git a/server/licenses/lucene-join-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-join-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 2524ac35c2afe..0000000000000 --- a/server/licenses/lucene-join-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -17be4fc1f9feca0dac84a37d54dca4b32df4c619 \ No newline at end of file diff --git a/server/licenses/lucene-memory-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-memory-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..73adf3fcb2829 --- /dev/null +++ b/server/licenses/lucene-memory-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +746f392e7ec27a7cd6ca2add7dd8441d2a6085da \ No newline at end of file diff --git a/server/licenses/lucene-memory-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-memory-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index f5ef377300839..0000000000000 --- a/server/licenses/lucene-memory-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7350675a2cf386c0f003b667b61db614f03bb984 \ No newline at end of file diff --git a/server/licenses/lucene-misc-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-misc-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..7f7dfead4c329 --- /dev/null +++ b/server/licenses/lucene-misc-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +0eb06ecc39c0ec0db380a6e5aad1b16907e0bfd9 \ No newline at end of file diff --git a/server/licenses/lucene-misc-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-misc-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index e94fcf0f259a1..0000000000000 --- a/server/licenses/lucene-misc-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c5c8bd120d5985ab6bd4e5f89efe08c226c0a323 \ No newline at end of file diff --git a/server/licenses/lucene-queries-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-queries-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..e3d400003efd8 --- /dev/null +++ b/server/licenses/lucene-queries-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +0e56eb18cceffcd5ce2e47b679e873420254df74 \ No newline at end of file diff --git a/server/licenses/lucene-queries-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-queries-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index fc80394195fa9..0000000000000 --- a/server/licenses/lucene-queries-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2e1975ac26e9172722f734bf0f5583317e5eb16a \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-queryparser-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..8e8c7f5171107 --- /dev/null +++ b/server/licenses/lucene-queryparser-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +dee3997a72eeae905e92930f53e724b6bef279da \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-queryparser-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 10ef577bc1bdc..0000000000000 --- a/server/licenses/lucene-queryparser-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c8cf3d5dd4d0538b38e4e88bb865bc59d835d887 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-sandbox-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..2d1df051e30b4 --- /dev/null +++ b/server/licenses/lucene-sandbox-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +946bc45b87b3d770ab6828b0d0a5f8684f2c3624 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-sandbox-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 08a61ba30bc0d..0000000000000 --- a/server/licenses/lucene-sandbox-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5a3a7a138ff4978f3ddb186d9786e6cb4793b291 \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-spatial-extras-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..0f9b7c0e90218 --- /dev/null +++ b/server/licenses/lucene-spatial-extras-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +d73667f61fb5e7fde4cec52fcfbbfd9847068aec \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-spatial-extras-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index a244219c1de60..0000000000000 --- a/server/licenses/lucene-spatial-extras-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -dc71e0125c66d29a1bffc1ddeab4b96526e737c8 \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-spatial3d-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..87894603e0d84 --- /dev/null +++ b/server/licenses/lucene-spatial3d-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +a8e8ab80bfb6abd70932e50fe31e13ecf2e00987 \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-spatial3d-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index d2b3821bbf5f6..0000000000000 --- a/server/licenses/lucene-spatial3d-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -aef4c04d353092a438eee302521fe34188b7c4df \ No newline at end of file diff --git a/server/licenses/lucene-suggest-9.11.0-snapshot-4be6531.jar.sha1 b/server/licenses/lucene-suggest-9.11.0-snapshot-4be6531.jar.sha1 new file mode 100644 index 0000000000000..6100f6fe0d585 --- /dev/null +++ b/server/licenses/lucene-suggest-9.11.0-snapshot-4be6531.jar.sha1 @@ -0,0 +1 @@ +45d6f0facd45d4e49585f0dabfa62ed5a1883033 \ No newline at end of file diff --git a/server/licenses/lucene-suggest-9.11.0-snapshot-fb97840.jar.sha1 b/server/licenses/lucene-suggest-9.11.0-snapshot-fb97840.jar.sha1 deleted file mode 100644 index 2c147e4651a44..0000000000000 --- a/server/licenses/lucene-suggest-9.11.0-snapshot-fb97840.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -86f68cacd85f99b4ddcda3aff7c873349ba59381 \ No newline at end of file diff --git a/server/src/main/java/org/apache/lucene/search/uhighlight/CustomFieldHighlighter.java b/server/src/main/java/org/apache/lucene/search/uhighlight/CustomFieldHighlighter.java index d1748d7f80995..22d5146f5bd4f 100644 --- a/server/src/main/java/org/apache/lucene/search/uhighlight/CustomFieldHighlighter.java +++ b/server/src/main/java/org/apache/lucene/search/uhighlight/CustomFieldHighlighter.java @@ -51,6 +51,8 @@ class CustomFieldHighlighter extends FieldHighlighter { private static final Passage[] EMPTY_PASSAGE = new Passage[0]; + private static final Comparator DEFAULT_PASSAGE_SORT_COMPARATOR = Comparator.comparingInt(Passage::getStartOffset); + private final Locale breakIteratorLocale; private final int noMatchSize; private String fieldValue; @@ -66,7 +68,16 @@ class CustomFieldHighlighter extends FieldHighlighter { PassageFormatter passageFormatter, int noMatchSize ) { - super(field, fieldOffsetStrategy, breakIterator, passageScorer, maxPassages, maxNoHighlightPassages, passageFormatter); + super( + field, + fieldOffsetStrategy, + breakIterator, + passageScorer, + maxPassages, + maxNoHighlightPassages, + passageFormatter, + DEFAULT_PASSAGE_SORT_COMPARATOR + ); this.breakIteratorLocale = breakIteratorLocale; this.noMatchSize = noMatchSize; } diff --git a/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/FastVectorHighlighter.java b/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/FastVectorHighlighter.java index 69f86bb91cc6e..6ae90b0ef8434 100644 --- a/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/FastVectorHighlighter.java +++ b/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/FastVectorHighlighter.java @@ -304,7 +304,7 @@ private static BoundaryScanner getBoundaryScanner(Field field) { return DEFAULT_WORD_BOUNDARY_SCANNER; case CHARS: if (fieldOptions.boundaryMaxScan() != SimpleBoundaryScanner.DEFAULT_MAX_SCAN - || fieldOptions.boundaryChars() != SimpleBoundaryScanner.DEFAULT_BOUNDARY_CHARS) { + || fieldOptions.boundaryChars() != HighlightBuilder.DEFAULT_BOUNDARY_CHARS) { return new SimpleBoundaryScanner(fieldOptions.boundaryMaxScan(), fieldOptions.boundaryChars()); } return DEFAULT_SIMPLE_BOUNDARY_SCANNER; diff --git a/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/HighlightBuilder.java b/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/HighlightBuilder.java index 0e7c3cf30ccec..44ef3a90395b8 100644 --- a/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/HighlightBuilder.java +++ b/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/HighlightBuilder.java @@ -111,6 +111,8 @@ public class HighlightBuilder extends AbstractHighlighterBuilder" }; + static final Character[] DEFAULT_BOUNDARY_CHARS = HighlightBuilder.convertCharArray(SimpleBoundaryScanner.DEFAULT_BOUNDARY_CHARS); + /** * a {@link FieldOptions} with default settings */ @@ -124,7 +126,7 @@ public class HighlightBuilder extends AbstractHighlighterBuilder Date: Tue, 28 May 2024 09:52:33 -0700 Subject: [PATCH 16/77] Check changed files before running gradle check (#13498) Signed-off-by: Sayali Gaikawad --- .github/workflows/gradle-check.yml | 28 ++++++++++++++++++++++++++-- 1 file changed, 26 insertions(+), 2 deletions(-) diff --git a/.github/workflows/gradle-check.yml b/.github/workflows/gradle-check.yml index f610ff5c31049..0ba4dbe374fdd 100644 --- a/.github/workflows/gradle-check.yml +++ b/.github/workflows/gradle-check.yml @@ -12,13 +12,28 @@ permissions: contents: read # to fetch code (actions/checkout) jobs: + check-files: + runs-on: ubuntu-latest + outputs: + RUN_GRADLE_CHECK: ${{ steps.changed-files-specific.outputs.any_changed }} + steps: + - uses: actions/checkout@v3 + - name: Get changed files + id: changed-files-specific + uses: tj-actions/changed-files@v44 + with: + files_ignore: | + release-notes/*.md + .github/** + *.md + gradle-check: - if: github.repository == 'opensearch-project/OpenSearch' + needs: check-files + if: github.repository == 'opensearch-project/OpenSearch' && needs.check-files.outputs.RUN_GRADLE_CHECK == 'true' permissions: contents: read # to fetch code (actions/checkout) pull-requests: write # to create or update comment (peter-evans/create-or-update-comment) issues: write # To create an issue if check fails on push. - runs-on: ubuntu-latest timeout-minutes: 130 steps: @@ -151,3 +166,12 @@ jobs: with: assignees: ${{ github.event.head_commit.author.username }}, ${{ github.triggering_actor }} filename: .github/ISSUE_TEMPLATE/failed_check.md + + check-result: + needs: [check-files, gradle-check] + if: always() + runs-on: ubuntu-latest + steps: + - name: Fail if gradle-check fails + if: ${{ needs.check-files.outputs.RUN_GRADLE_CHECK && needs.gradle-check.result == 'failure' }} + run: exit 1 \ No newline at end of file From b9befaa12524744ffe71d3f12699cb27cb8fca5e Mon Sep 17 00:00:00 2001 From: Shivansh Arora <31575408+shiv0408@users.noreply.github.com> Date: Wed, 29 May 2024 12:24:22 +0530 Subject: [PATCH 17/77] Optimize remote state stale file deletion (#13131) * Optimize remote state stale file deletion Signed-off-by: Shivansh Arora * Added UT Signed-off-by: Shivansh Arora * Refactored into a clean up manager file Signed-off-by: Shivansh Arora * Add UT Signed-off-by: Shivansh Arora * Modify the Integ test Signed-off-by: Shivansh Arora * Address PR comment Signed-off-by: Shivansh Arora * Add changelog Signed-off-by: Shivansh Arora * apply spotless Signed-off-by: Shivansh Arora * Made minor requested changes Signed-off-by: Shivansh Arora * Fix default time interval Signed-off-by: Shivansh Arora * Clean up global metadata attribute objects from remote Signed-off-by: Shivansh Arora * fix build Signed-off-by: Shivansh Arora * apply spotless Signed-off-by: Shivansh Arora * remove MockAsyncUploadFSRepo to avoid flakiness Signed-off-by: Shivansh Arora * apply spotless Signed-off-by: Shivansh Arora --------- Signed-off-by: Shivansh Arora --- CHANGELOG.md | 1 + .../RemoteClusterStateCleanupManagerIT.java | 145 ++++++ .../remote/RemoteClusterStateServiceIT.java | 63 --- .../RemoteStoreBaseIntegTestCase.java | 10 + .../common/settings/ClusterSettings.java | 2 + .../RemoteClusterStateCleanupManager.java | 408 ++++++++++++++++ .../remote/RemoteClusterStateService.java | 275 ++--------- .../main/java/org/opensearch/node/Node.java | 7 +- .../GatewayMetaStatePersistedStateTests.java | 6 +- ...RemoteClusterStateCleanupManagerTests.java | 446 ++++++++++++++++++ .../RemoteClusterStateServiceTests.java | 136 +----- 11 files changed, 1057 insertions(+), 442 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java create mode 100644 server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index a0b5dd289ec85..a48c4635d8dec 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,6 +12,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Remote Store] Add dynamic cluster settings to set timeout for segments upload to Remote Store ([#13679](https://github.com/opensearch-project/OpenSearch/pull/13679)) - [Remote Store] Upload translog checkpoint as object metadata to translog.tlog([#13637](https://github.com/opensearch-project/OpenSearch/pull/13637)) - Add getMetadataFields to MapperService ([#13819](https://github.com/opensearch-project/OpenSearch/pull/13819)) +- [Remote State] Add async remote state deletion task running on an interval, configurable by a setting ([#13131](https://github.com/opensearch-project/OpenSearch/pull/13131)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java new file mode 100644 index 0000000000000..0f441fe01a368 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java @@ -0,0 +1,145 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.Settings; +import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.Before; + +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; +import static org.opensearch.indices.IndicesService.CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteClusterStateCleanupManagerIT extends RemoteStoreBaseIntegTestCase { + + private static final String INDEX_NAME = "test-index"; + + @Before + public void setup() { + asyncUploadMockFsRepo = false; + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); + } + + private Map initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount) { + prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount); + Map indexStats = indexData(1, false, INDEX_NAME); + assertEquals(shardCount * (replicaCount + 1), getNumShards(INDEX_NAME).totalNumShards); + ensureGreen(INDEX_NAME); + return indexStats; + } + + public void testRemoteCleanupTaskUpdated() { + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); + RemoteClusterStateCleanupManager remoteClusterStateCleanupManager = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateCleanupManager.class + ); + + assertEquals(CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval()); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + + // now disable + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), -1)) + .get(); + + assertEquals(-1, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMillis()); + assertFalse(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + + // now set Clean up interval to 1 min + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "1m")) + .get(); + assertEquals(1, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMinutes()); + } + + public void testRemoteCleanupDeleteStale() throws Exception { + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); + + // set cleanup interval to 100 ms to make the test faster + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "100ms")) + .get(); + + assertTrue(response.isAcknowledged()); + + // update cluster state 21 times to ensure that clean up has run after this will upload 42 manifest files + // to repository, if manifest files are less than that it means clean up has run + updateClusterStateNTimes(RETAINED_MANIFESTS + SKIP_CLEANUP_STATE_CHANGES + 1); + + RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); + BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); + BlobPath baseMetadataPath = repository.basePath() + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(getClusterState().metadata().clusterUUID()); + BlobPath manifestContainerPath = baseMetadataPath.add("manifest"); + + assertBusy(() -> { + int manifestFiles = repository.blobStore().blobContainer(manifestContainerPath).listBlobsByPrefix("manifest").size(); + logger.info("number of current manifest file: {}", manifestFiles); + // we can't guarantee that we have same number of manifest as Retained manifest in our repo as there can be other queued task + // other than replica count change which can upload new manifest files, that's why we check that number of manifests is between + // Retained manifests and Retained manifests + 2 * Skip cleanup state changes (each cluster state update uploads 2 manifests) + assertTrue( + "Current number of manifest files: " + manifestFiles, + manifestFiles >= RETAINED_MANIFESTS && manifestFiles < RETAINED_MANIFESTS + 2 * SKIP_CLEANUP_STATE_CHANGES + ); + }, 500, TimeUnit.MILLISECONDS); + } + + private void updateClusterStateNTimes(int n) { + int newReplicaCount = randomIntBetween(0, 3); + for (int i = n; i > 0; i--) { + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING.getKey(), i, TimeUnit.SECONDS)) + .get(); + assertTrue(response.isAcknowledged()); + } + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java index 42120aa32eb47..ab2f0f0080566 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java @@ -10,7 +10,6 @@ import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; import org.opensearch.discovery.DiscoveryStats; @@ -27,7 +26,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.CUSTOM_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; @@ -51,16 +49,6 @@ protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); } - private void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { - internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes); - internalCluster().startDataOnlyNodes(numDataOnlyNodes); - for (String index : indices.split(",")) { - createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); - ensureYellowAndNoInitializingShards(index); - ensureGreen(index); - } - } - private Map initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount) { prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount); Map indexStats = indexData(1, false, INDEX_NAME); @@ -69,49 +57,6 @@ private Map initialTestSetup(int shardCount, int replicaCount, int return indexStats; } - public void testFullClusterRestoreStaleDelete() throws Exception { - int shardCount = randomIntBetween(1, 2); - int replicaCount = 1; - int dataNodeCount = shardCount * (replicaCount + 1); - int clusterManagerNodeCount = 1; - - initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); - setReplicaCount(0); - setReplicaCount(2); - setReplicaCount(0); - setReplicaCount(1); - setReplicaCount(0); - setReplicaCount(1); - setReplicaCount(0); - setReplicaCount(2); - setReplicaCount(0); - - RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( - RemoteClusterStateService.class - ); - - RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); - - BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); - BlobPath baseMetadataPath = repository.basePath() - .add( - Base64.getUrlEncoder() - .withoutPadding() - .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) - ) - .add("cluster-state") - .add(getClusterState().metadata().clusterUUID()); - - assertEquals(10, repository.blobStore().blobContainer(baseMetadataPath.add("manifest")).listBlobsByPrefix("manifest").size()); - - Map indexMetadataMap = remoteClusterStateService.getLatestClusterState( - cluster().getClusterName(), - getClusterState().metadata().clusterUUID() - ).getMetadata().getIndices(); - assertEquals(0, indexMetadataMap.values().stream().findFirst().get().getNumberOfReplicas()); - assertEquals(shardCount, indexMetadataMap.values().stream().findFirst().get().getNumberOfShards()); - } - public void testRemoteStateStats() { int shardCount = randomIntBetween(1, 2); int replicaCount = 1; @@ -241,12 +186,4 @@ private void validateNodesStatsResponse(NodesStatsResponse nodesStatsResponse) { assertNotNull(nodesStatsResponse.getNodes().get(0)); assertNotNull(nodesStatsResponse.getNodes().get(0).getDiscoveryStats()); } - - private void setReplicaCount(int replicaCount) { - client().admin() - .indices() - .prepareUpdateSettings(INDEX_NAME) - .setSettings(Settings.builder().put(SETTING_NUMBER_OF_REPLICAS, replicaCount)) - .get(); - } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 740aee69f7d80..64efcee6ef1b5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -350,4 +350,14 @@ protected void restore(boolean restoreAllShards, String... indices) { PlainActionFuture.newFuture() ); } + + protected void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { + internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes); + internalCluster().startDataOnlyNodes(numDataOnlyNodes); + for (String index : indices.split(",")) { + createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); + ensureYellowAndNoInitializingShards(index); + ensureGreen(index); + } + } } 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 a2be2ea4510e0..d57ef7e780602 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -104,6 +104,7 @@ import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.PersistedClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateCleanupManager; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpTransportSettings; import org.opensearch.index.IndexModule; @@ -711,6 +712,7 @@ public void apply(Settings value, Settings current, Settings previous) { SearchRequestSlowLog.CLUSTER_SEARCH_REQUEST_SLOWLOG_LEVEL, // Remote cluster state settings + RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING, RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING, RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, RemoteClusterStateService.GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java new file mode 100644 index 0000000000000..8a106a25e5630 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java @@ -0,0 +1,408 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.logging.log4j.util.Strings; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.service.ClusterApplierService; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.AbstractAsyncTask; +import org.opensearch.core.action.ActionListener; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.threadpool.ThreadPool; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_FORMAT; +import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_METADATA_FORMAT; +import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; +import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_PATH_TOKEN; + +/** + * A Manager which provides APIs to clean up stale cluster state files and runs an async stale cleanup task + * + * @opensearch.internal + */ +public class RemoteClusterStateCleanupManager implements Closeable { + + public static final int RETAINED_MANIFESTS = 10; + public static final int SKIP_CLEANUP_STATE_CHANGES = 10; + public static final TimeValue CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT = TimeValue.timeValueMinutes(5); + public static final TimeValue CLUSTER_STATE_CLEANUP_INTERVAL_MINIMUM = TimeValue.MINUS_ONE; + + /** + * Setting to specify the interval to do run stale file cleanup job + * Min value -1 indicates that the stale file cleanup job should be disabled + */ + public static final Setting REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING = Setting.timeSetting( + "cluster.remote_store.state.cleanup_interval", + CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, + CLUSTER_STATE_CLEANUP_INTERVAL_MINIMUM, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private static final Logger logger = LogManager.getLogger(RemoteClusterStateCleanupManager.class); + private final RemoteClusterStateService remoteClusterStateService; + private final RemotePersistenceStats remoteStateStats; + private BlobStoreTransferService blobStoreTransferService; + private TimeValue staleFileCleanupInterval; + private final AtomicBoolean deleteStaleMetadataRunning = new AtomicBoolean(false); + private AsyncStaleFileDeletion staleFileDeletionTask; + private long lastCleanupAttemptStateVersion; + private final ThreadPool threadpool; + private final ClusterApplierService clusterApplierService; + + public RemoteClusterStateCleanupManager(RemoteClusterStateService remoteClusterStateService, ClusterService clusterService) { + this.remoteClusterStateService = remoteClusterStateService; + this.remoteStateStats = remoteClusterStateService.getStats(); + ClusterSettings clusterSettings = clusterService.getClusterSettings(); + this.clusterApplierService = clusterService.getClusterApplierService(); + this.staleFileCleanupInterval = clusterSettings.get(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING); + this.threadpool = remoteClusterStateService.getThreadpool(); + // initialize with 0, a cleanup will be done when this node is elected master node and version is incremented more than threshold + this.lastCleanupAttemptStateVersion = 0; + clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING, this::updateCleanupInterval); + } + + void start() { + staleFileDeletionTask = new AsyncStaleFileDeletion(this); + } + + @Override + public void close() throws IOException { + if (staleFileDeletionTask != null) { + staleFileDeletionTask.close(); + } + } + + private BlobStoreTransferService getBlobStoreTransferService() { + if (blobStoreTransferService == null) { + blobStoreTransferService = new BlobStoreTransferService(remoteClusterStateService.getBlobStore(), threadpool); + } + return blobStoreTransferService; + } + + private void updateCleanupInterval(TimeValue updatedInterval) { + this.staleFileCleanupInterval = updatedInterval; + logger.info("updated remote state cleanup interval to {}", updatedInterval); + // After updating the interval, we need to close the current task and create a new one which will run with updated interval + if (staleFileDeletionTask != null && !staleFileDeletionTask.getInterval().equals(updatedInterval)) { + staleFileDeletionTask.setInterval(updatedInterval); + } + } + + // visible for testing + void cleanUpStaleFiles() { + ClusterState currentAppliedState = clusterApplierService.state(); + if (currentAppliedState.nodes().isLocalNodeElectedClusterManager()) { + long cleanUpAttemptStateVersion = currentAppliedState.version(); + assert Strings.isNotEmpty(currentAppliedState.getClusterName().value()) : "cluster name is not set"; + assert Strings.isNotEmpty(currentAppliedState.metadata().clusterUUID()) : "cluster uuid is not set"; + if (cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion > SKIP_CLEANUP_STATE_CHANGES) { + logger.info( + "Cleaning up stale remote state files for cluster [{}] with uuid [{}]. Last clean was done before {} updates", + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion + ); + this.deleteStaleClusterMetadata( + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + RETAINED_MANIFESTS + ); + lastCleanupAttemptStateVersion = cleanUpAttemptStateVersion; + } else { + logger.debug( + "Skipping cleanup of stale remote state files for cluster [{}] with uuid [{}]. Last clean was done before {} updates, which is less than threshold {}", + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion, + SKIP_CLEANUP_STATE_CHANGES + ); + } + } else { + logger.debug("Skipping cleanup task as local node is not elected Cluster Manager"); + } + } + + private void addStaleGlobalMetadataPath(String fileName, Set filesToKeep, Set staleGlobalMetadataPaths) { + if (!filesToKeep.contains(fileName)) { + String[] splitPath = fileName.split("/"); + staleGlobalMetadataPaths.add( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( + splitPath[splitPath.length - 1] + ) + ); + } + } + + // visible for testing + void deleteClusterMetadata( + String clusterName, + String clusterUUID, + List activeManifestBlobMetadata, + List staleManifestBlobMetadata + ) { + try { + Set filesToKeep = new HashSet<>(); + Set staleManifestPaths = new HashSet<>(); + Set staleIndexMetadataPaths = new HashSet<>(); + Set staleGlobalMetadataPaths = new HashSet<>(); + activeManifestBlobMetadata.forEach(blobMetadata -> { + ClusterMetadataManifest clusterMetadataManifest = remoteClusterStateService.fetchRemoteClusterMetadataManifest( + clusterName, + clusterUUID, + blobMetadata.name() + ); + clusterMetadataManifest.getIndices() + .forEach(uploadedIndexMetadata -> filesToKeep.add(uploadedIndexMetadata.getUploadedFilename())); + if (clusterMetadataManifest.getCodecVersion() == ClusterMetadataManifest.CODEC_V1) { + filesToKeep.add(clusterMetadataManifest.getGlobalMetadataFileName()); + } else if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V2) { + filesToKeep.add(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); + filesToKeep.add(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); + filesToKeep.add(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); + clusterMetadataManifest.getCustomMetadataMap() + .values() + .forEach(attribute -> filesToKeep.add(attribute.getUploadedFilename())); + } + }); + staleManifestBlobMetadata.forEach(blobMetadata -> { + ClusterMetadataManifest clusterMetadataManifest = remoteClusterStateService.fetchRemoteClusterMetadataManifest( + clusterName, + clusterUUID, + blobMetadata.name() + ); + staleManifestPaths.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blobMetadata.name()); + if (clusterMetadataManifest.getCodecVersion() == ClusterMetadataManifest.CODEC_V1) { + addStaleGlobalMetadataPath(clusterMetadataManifest.getGlobalMetadataFileName(), filesToKeep, staleGlobalMetadataPaths); + } else if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V2) { + addStaleGlobalMetadataPath( + clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename(), + filesToKeep, + staleGlobalMetadataPaths + ); + addStaleGlobalMetadataPath( + clusterMetadataManifest.getSettingsMetadata().getUploadedFilename(), + filesToKeep, + staleGlobalMetadataPaths + ); + addStaleGlobalMetadataPath( + clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename(), + filesToKeep, + staleGlobalMetadataPaths + ); + clusterMetadataManifest.getCustomMetadataMap() + .values() + .forEach( + attribute -> addStaleGlobalMetadataPath(attribute.getUploadedFilename(), filesToKeep, staleGlobalMetadataPaths) + ); + } + + clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { + if (filesToKeep.contains(uploadedIndexMetadata.getUploadedFilename()) == false) { + staleIndexMetadataPaths.add( + new BlobPath().add(INDEX_PATH_TOKEN).add(uploadedIndexMetadata.getIndexUUID()).buildAsString() + + INDEX_METADATA_FORMAT.blobName(uploadedIndexMetadata.getUploadedFilename()) + ); + } + }); + }); + + if (staleManifestPaths.isEmpty()) { + logger.debug("No stale Remote Cluster Metadata files found"); + return; + } + + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleGlobalMetadataPaths)); + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleIndexMetadataPaths)); + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleManifestPaths)); + } catch (IllegalStateException e) { + logger.error("Error while fetching Remote Cluster Metadata manifests", e); + } catch (IOException e) { + logger.error("Error while deleting stale Remote Cluster Metadata files", e); + remoteStateStats.cleanUpAttemptFailed(); + } catch (Exception e) { + logger.error("Unexpected error while deleting stale Remote Cluster Metadata files", e); + remoteStateStats.cleanUpAttemptFailed(); + } + } + + /** + * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests + * + * @param clusterName name of the cluster + * @param clusterUUID uuid of cluster state to refer to in remote + * @param manifestsToRetain no of latest manifest files to keep in remote + */ + // package private for testing + void deleteStaleClusterMetadata(String clusterName, String clusterUUID, int manifestsToRetain) { + if (deleteStaleMetadataRunning.compareAndSet(false, true) == false) { + logger.info("Delete stale cluster metadata task is already in progress."); + return; + } + try { + getBlobStoreTransferService().listAllInSortedOrderAsync( + ThreadPool.Names.REMOTE_PURGE, + remoteClusterStateService.getManifestFolderPath(clusterName, clusterUUID), + MANIFEST_FILE_PREFIX, + Integer.MAX_VALUE, + new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + if (blobMetadata.size() > manifestsToRetain) { + deleteClusterMetadata( + clusterName, + clusterUUID, + blobMetadata.subList(0, manifestsToRetain), + blobMetadata.subList(manifestsToRetain, blobMetadata.size()) + ); + } + deleteStaleMetadataRunning.set(false); + } + + @Override + public void onFailure(Exception e) { + logger.error( + new ParameterizedMessage( + "Exception occurred while deleting Remote Cluster Metadata for clusterUUIDs {}", + clusterUUID + ) + ); + deleteStaleMetadataRunning.set(false); + } + } + ); + } catch (Exception e) { + deleteStaleMetadataRunning.set(false); + throw e; + } + } + + /** + * Purges all remote cluster state against provided cluster UUIDs + * + * @param clusterName name of the cluster + * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged + */ + void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { + clusterUUIDs.forEach( + clusterUUID -> getBlobStoreTransferService().deleteAsync( + ThreadPool.Names.REMOTE_PURGE, + remoteClusterStateService.getCusterMetadataBasePath(clusterName, clusterUUID), + new ActionListener<>() { + @Override + public void onResponse(Void unused) { + logger.info("Deleted all remote cluster metadata for cluster UUID - {}", clusterUUID); + } + + @Override + public void onFailure(Exception e) { + logger.error( + new ParameterizedMessage( + "Exception occurred while deleting all remote cluster metadata for cluster UUID {}", + clusterUUID + ), + e + ); + remoteStateStats.cleanUpAttemptFailed(); + } + } + ) + ); + } + + // package private for testing + void deleteStalePaths(String clusterName, String clusterUUID, List stalePaths) throws IOException { + logger.debug(String.format(Locale.ROOT, "Deleting stale files from remote - %s", stalePaths)); + getBlobStoreTransferService().deleteBlobs( + remoteClusterStateService.getCusterMetadataBasePath(clusterName, clusterUUID), + stalePaths + ); + } + + /** + * Purges all remote cluster state against provided cluster UUIDs + * @param clusterState current state of the cluster + * @param committedManifest last committed ClusterMetadataManifest + */ + public void deleteStaleClusterUUIDs(ClusterState clusterState, ClusterMetadataManifest committedManifest) { + threadpool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { + String clusterName = clusterState.getClusterName().value(); + logger.debug("Deleting stale cluster UUIDs data from remote [{}]", clusterName); + Set allClustersUUIDsInRemote; + try { + allClustersUUIDsInRemote = new HashSet<>( + remoteClusterStateService.getAllClusterUUIDs(clusterState.getClusterName().value()) + ); + } catch (IOException e) { + logger.info(String.format(Locale.ROOT, "Error while fetching all cluster UUIDs for [%s]", clusterName)); + return; + } + // Retain last 2 cluster uuids data + allClustersUUIDsInRemote.remove(committedManifest.getClusterUUID()); + allClustersUUIDsInRemote.remove(committedManifest.getPreviousClusterUUID()); + deleteStaleUUIDsClusterMetadata(clusterName, new ArrayList<>(allClustersUUIDsInRemote)); + }); + } + + public TimeValue getStaleFileCleanupInterval() { + return this.staleFileCleanupInterval; + } + + AsyncStaleFileDeletion getStaleFileDeletionTask() { // for testing + return this.staleFileDeletionTask; + } + + RemotePersistenceStats getStats() { + return this.remoteStateStats; + } + + static final class AsyncStaleFileDeletion extends AbstractAsyncTask { + private final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; + + AsyncStaleFileDeletion(RemoteClusterStateCleanupManager remoteClusterStateCleanupManager) { + super( + logger, + remoteClusterStateCleanupManager.threadpool, + remoteClusterStateCleanupManager.getStaleFileCleanupInterval(), + true + ); + this.remoteClusterStateCleanupManager = remoteClusterStateCleanupManager; + rescheduleIfNecessary(); + } + + @Override + protected boolean mustReschedule() { + return true; + } + + @Override + protected void runInternal() { + remoteClusterStateCleanupManager.cleanUpStaleFiles(); + } + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index ac821cd15a5b3..9bf1dff2359ca 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -18,11 +18,13 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.Nullable; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -59,7 +61,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.LongSupplier; @@ -81,8 +82,6 @@ public class RemoteClusterStateService implements Closeable { public static final String METADATA_MANIFEST_NAME_FORMAT = "%s"; - public static final int RETAINED_MANIFESTS = 10; - public static final String DELIMITER = "__"; public static final String CUSTOM_DELIMITER = "--"; @@ -207,8 +206,7 @@ public class RemoteClusterStateService implements Closeable { private volatile TimeValue indexMetadataUploadTimeout; private volatile TimeValue globalMetadataUploadTimeout; private volatile TimeValue metadataManifestUploadTimeout; - - private final AtomicBoolean deleteStaleMetadataRunning = new AtomicBoolean(false); + private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; private final RemotePersistenceStats remoteStateStats; private final String CLUSTER_STATE_UPLOAD_TIME_LOG_STRING = "writing cluster state for version [{}] took [{}ms]"; private final String METADATA_UPDATE_LOG_STRING = "wrote metadata for [{}] indices and skipped [{}] unchanged " @@ -232,7 +230,7 @@ public RemoteClusterStateService( String nodeId, Supplier repositoriesService, Settings settings, - ClusterSettings clusterSettings, + ClusterService clusterService, LongSupplier relativeTimeNanosSupplier, ThreadPool threadPool, List indexMetadataUploadListeners @@ -243,6 +241,7 @@ public RemoteClusterStateService( this.settings = settings; this.relativeTimeNanosSupplier = relativeTimeNanosSupplier; this.threadpool = threadPool; + ClusterSettings clusterSettings = clusterService.getClusterSettings(); this.slowWriteLoggingThreshold = clusterSettings.get(SLOW_WRITE_LOGGING_THRESHOLD); this.indexMetadataUploadTimeout = clusterSettings.get(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING); this.globalMetadataUploadTimeout = clusterSettings.get(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING); @@ -252,16 +251,10 @@ public RemoteClusterStateService( clusterSettings.addSettingsUpdateConsumer(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, this::setGlobalMetadataUploadTimeout); clusterSettings.addSettingsUpdateConsumer(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, this::setMetadataManifestUploadTimeout); this.remoteStateStats = new RemotePersistenceStats(); + this.remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(this, clusterService); this.indexMetadataUploadListeners = indexMetadataUploadListeners; } - private BlobStoreTransferService getBlobStoreTransferService() { - if (blobStoreTransferService == null) { - blobStoreTransferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadpool); - } - return blobStoreTransferService; - } - /** * This method uploads entire cluster state metadata to the configured blob store. For now only index metadata upload is supported. This method should be * invoked by the elected cluster manager when the remote cluster state is enabled. @@ -417,7 +410,6 @@ public ClusterMetadataManifest writeIncrementalMetadata( : previousManifest.getCustomMetadataMap(), false ); - deleteStaleClusterMetadata(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), RETAINED_MANIFESTS); final long durationMillis = TimeValue.nsecToMSec(relativeTimeNanosSupplier.getAsLong() - startTimeNanos); remoteStateStats.stateSucceeded(); @@ -721,6 +713,10 @@ private CheckedRunnable getAsyncMetadataWriteAction( ); } + public RemoteClusterStateCleanupManager getCleanupManager() { + return remoteClusterStateCleanupManager; + } + @Nullable public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterState, ClusterMetadataManifest previousManifest) throws IOException { @@ -740,12 +736,16 @@ public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterStat previousManifest.getCustomMetadataMap(), true ); - deleteStaleClusterUUIDs(clusterState, committedManifest); + if (!previousManifest.isClusterUUIDCommitted() && committedManifest.isClusterUUIDCommitted()) { + remoteClusterStateCleanupManager.deleteStaleClusterUUIDs(clusterState, committedManifest); + } + return committedManifest; } @Override public void close() throws IOException { + remoteClusterStateCleanupManager.close(); if (blobStoreRepository != null) { IOUtils.close(blobStoreRepository); } @@ -760,6 +760,7 @@ public void start() { final Repository repository = repositoriesService.get().repository(remoteStoreRepo); assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; blobStoreRepository = (BlobStoreRepository) repository; + remoteClusterStateCleanupManager.start(); } private ClusterMetadataManifest uploadManifest( @@ -850,6 +851,14 @@ private void writeMetadataManifest(String clusterName, String clusterUUID, Clust ); } + ThreadPool getThreadpool() { + return threadpool; + } + + BlobStore getBlobStore() { + return blobStoreRepository.blobStore(); + } + private BlobContainer indexMetadataContainer(String clusterName, String clusterUUID, String indexUUID) { // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index/ftqsCnn9TgOX return blobStoreRepository.blobStore() @@ -867,7 +876,7 @@ private BlobContainer manifestContainer(String clusterName, String clusterUUID) return blobStoreRepository.blobStore().blobContainer(getManifestFolderPath(clusterName, clusterUUID)); } - private BlobPath getCusterMetadataBasePath(String clusterName, String clusterUUID) { + BlobPath getCusterMetadataBasePath(String clusterName, String clusterUUID) { return blobStoreRepository.basePath().add(encodeString(clusterName)).add(CLUSTER_STATE_PATH_TOKEN).add(clusterUUID); } @@ -982,7 +991,7 @@ private static String metadataAttributeFileName(String componentPrefix, Long met ); } - private BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { + BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { return getCusterMetadataBasePath(clusterName, clusterUUID).add(MANIFEST_PATH_TOKEN); } @@ -1235,7 +1244,7 @@ public String getLastKnownUUIDFromRemote(String clusterName) { } } - private Set getAllClusterUUIDs(String clusterName) throws IOException { + Set getAllClusterUUIDs(String clusterName) throws IOException { Map clusterUUIDMetadata = clusterUUIDContainer(clusterName).children(); if (clusterUUIDMetadata == null) { return Collections.emptySet(); @@ -1426,7 +1435,7 @@ private Optional getLatestManifestFileName(String clusterName, String cl * @param clusterName name of the cluster * @return ClusterMetadataManifest */ - private ClusterMetadataManifest fetchRemoteClusterMetadataManifest(String clusterName, String clusterUUID, String filename) + ClusterMetadataManifest fetchRemoteClusterMetadataManifest(String clusterName, String clusterUUID, String filename) throws IllegalStateException { try { return getClusterMetadataManifestBlobStoreFormat(filename).read( @@ -1486,234 +1495,6 @@ public RemoteStateTransferException(String errorDesc, Throwable cause) { } } - /** - * Purges all remote cluster state against provided cluster UUIDs - * - * @param clusterName name of the cluster - * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged - */ - void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { - clusterUUIDs.forEach(clusterUUID -> { - getBlobStoreTransferService().deleteAsync( - ThreadPool.Names.REMOTE_PURGE, - getCusterMetadataBasePath(clusterName, clusterUUID), - new ActionListener<>() { - @Override - public void onResponse(Void unused) { - logger.info("Deleted all remote cluster metadata for cluster UUID - {}", clusterUUID); - } - - @Override - public void onFailure(Exception e) { - logger.error( - new ParameterizedMessage( - "Exception occurred while deleting all remote cluster metadata for cluster UUID {}", - clusterUUID - ), - e - ); - remoteStateStats.cleanUpAttemptFailed(); - } - } - ); - }); - } - - /** - * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests - * - * @param clusterName name of the cluster - * @param clusterUUID uuid of cluster state to refer to in remote - * @param manifestsToRetain no of latest manifest files to keep in remote - */ - // package private for testing - void deleteStaleClusterMetadata(String clusterName, String clusterUUID, int manifestsToRetain) { - if (deleteStaleMetadataRunning.compareAndSet(false, true) == false) { - logger.info("Delete stale cluster metadata task is already in progress."); - return; - } - try { - getBlobStoreTransferService().listAllInSortedOrderAsync( - ThreadPool.Names.REMOTE_PURGE, - getManifestFolderPath(clusterName, clusterUUID), - "manifest", - Integer.MAX_VALUE, - new ActionListener<>() { - @Override - public void onResponse(List blobMetadata) { - if (blobMetadata.size() > manifestsToRetain) { - deleteClusterMetadata( - clusterName, - clusterUUID, - blobMetadata.subList(0, manifestsToRetain - 1), - blobMetadata.subList(manifestsToRetain - 1, blobMetadata.size()) - ); - } - deleteStaleMetadataRunning.set(false); - } - - @Override - public void onFailure(Exception e) { - logger.error( - new ParameterizedMessage( - "Exception occurred while deleting Remote Cluster Metadata for clusterUUIDs {}", - clusterUUID - ) - ); - deleteStaleMetadataRunning.set(false); - } - } - ); - } catch (Exception e) { - deleteStaleMetadataRunning.set(false); - throw e; - } - } - - private void deleteClusterMetadata( - String clusterName, - String clusterUUID, - List activeManifestBlobMetadata, - List staleManifestBlobMetadata - ) { - try { - Set filesToKeep = new HashSet<>(); - Set staleManifestPaths = new HashSet<>(); - Set staleIndexMetadataPaths = new HashSet<>(); - Set staleGlobalMetadataPaths = new HashSet<>(); - activeManifestBlobMetadata.forEach(blobMetadata -> { - ClusterMetadataManifest clusterMetadataManifest = fetchRemoteClusterMetadataManifest( - clusterName, - clusterUUID, - blobMetadata.name() - ); - clusterMetadataManifest.getIndices() - .forEach(uploadedIndexMetadata -> filesToKeep.add(uploadedIndexMetadata.getUploadedFilename())); - if (clusterMetadataManifest.getGlobalMetadataFileName() != null) { - filesToKeep.add(clusterMetadataManifest.getGlobalMetadataFileName()); - } else { - filesToKeep.add(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); - filesToKeep.add(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); - filesToKeep.add(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); - clusterMetadataManifest.getCustomMetadataMap() - .forEach((key, value) -> { filesToKeep.add(value.getUploadedFilename()); }); - } - }); - staleManifestBlobMetadata.forEach(blobMetadata -> { - ClusterMetadataManifest clusterMetadataManifest = fetchRemoteClusterMetadataManifest( - clusterName, - clusterUUID, - blobMetadata.name() - ); - staleManifestPaths.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blobMetadata.name()); - if (clusterMetadataManifest.getGlobalMetadataFileName() != null) { - if (filesToKeep.contains(clusterMetadataManifest.getGlobalMetadataFileName()) == false) { - String[] globalMetadataSplitPath = clusterMetadataManifest.getGlobalMetadataFileName().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - globalMetadataSplitPath[globalMetadataSplitPath.length - 1] - ) - ); - } - } else { - if (filesToKeep.contains(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()) == false) { - String[] coordinationMetadataSplitPath = clusterMetadataManifest.getCoordinationMetadata() - .getUploadedFilename() - .split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - coordinationMetadataSplitPath[coordinationMetadataSplitPath.length - 1] - ) - ); - } - if (filesToKeep.contains(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()) == false) { - String[] templatesMetadataSplitPath = clusterMetadataManifest.getTemplatesMetadata() - .getUploadedFilename() - .split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - templatesMetadataSplitPath[templatesMetadataSplitPath.length - 1] - ) - ); - } - if (filesToKeep.contains(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()) == false) { - String[] settingsMetadataSplitPath = clusterMetadataManifest.getSettingsMetadata().getUploadedFilename().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - settingsMetadataSplitPath[settingsMetadataSplitPath.length - 1] - ) - ); - } - clusterMetadataManifest.getCustomMetadataMap().forEach((key, value) -> { - if (filesToKeep.contains(value.getUploadedFilename()) == false) { - String[] customMetadataSplitPath = value.getUploadedFilename().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - customMetadataSplitPath[customMetadataSplitPath.length - 1] - ) - ); - } - }); - } - - clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { - if (filesToKeep.contains(uploadedIndexMetadata.getUploadedFilename()) == false) { - staleIndexMetadataPaths.add( - new BlobPath().add(INDEX_PATH_TOKEN).add(uploadedIndexMetadata.getIndexUUID()).buildAsString() - + INDEX_METADATA_FORMAT.blobName(uploadedIndexMetadata.getUploadedFilename()) - ); - } - }); - }); - - if (staleManifestPaths.isEmpty()) { - logger.debug("No stale Remote Cluster Metadata files found"); - return; - } - - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleGlobalMetadataPaths)); - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleIndexMetadataPaths)); - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleManifestPaths)); - } catch (IllegalStateException e) { - logger.error("Error while fetching Remote Cluster Metadata manifests", e); - } catch (IOException e) { - logger.error("Error while deleting stale Remote Cluster Metadata files", e); - remoteStateStats.cleanUpAttemptFailed(); - } catch (Exception e) { - logger.error("Unexpected error while deleting stale Remote Cluster Metadata files", e); - remoteStateStats.cleanUpAttemptFailed(); - } - } - - private void deleteStalePaths(String clusterName, String clusterUUID, List stalePaths) throws IOException { - logger.debug(String.format(Locale.ROOT, "Deleting stale files from remote - %s", stalePaths)); - getBlobStoreTransferService().deleteBlobs(getCusterMetadataBasePath(clusterName, clusterUUID), stalePaths); - } - - /** - * Purges all remote cluster state against provided cluster UUIDs - * - * @param clusterState current state of the cluster - * @param committedManifest last committed ClusterMetadataManifest - */ - public void deleteStaleClusterUUIDs(ClusterState clusterState, ClusterMetadataManifest committedManifest) { - threadpool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { - String clusterName = clusterState.getClusterName().value(); - logger.debug("Deleting stale cluster UUIDs data from remote [{}]", clusterName); - Set allClustersUUIDsInRemote; - try { - allClustersUUIDsInRemote = new HashSet<>(getAllClusterUUIDs(clusterState.getClusterName().value())); - } catch (IOException e) { - logger.info(String.format(Locale.ROOT, "Error while fetching all cluster UUIDs for [%s]", clusterName)); - return; - } - // Retain last 2 cluster uuids data - allClustersUUIDsInRemote.remove(committedManifest.getClusterUUID()); - allClustersUUIDsInRemote.remove(committedManifest.getPreviousClusterUUID()); - deleteStaleUUIDsClusterMetadata(clusterName, new ArrayList<>(allClustersUUIDsInRemote)); - }); - } - public RemotePersistenceStats getStats() { return remoteStateStats; } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 9462aeddbd0e4..04bd31e6a5809 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -138,6 +138,7 @@ import org.opensearch.gateway.MetaStateService; import org.opensearch.gateway.PersistedClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateCleanupManager; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpServerTransport; import org.opensearch.identity.IdentityService; @@ -751,6 +752,7 @@ protected Node( threadPool::relativeTimeInMillis ); final RemoteClusterStateService remoteClusterStateService; + final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; final RemoteIndexPathUploader remoteIndexPathUploader; if (isRemoteStoreClusterStateEnabled(settings)) { remoteIndexPathUploader = new RemoteIndexPathUploader( @@ -763,14 +765,16 @@ protected Node( nodeEnvironment.nodeId(), repositoriesServiceReference::get, settings, - clusterService.getClusterSettings(), + clusterService, threadPool::preciseRelativeTimeInNanos, threadPool, List.of(remoteIndexPathUploader) ); + remoteClusterStateCleanupManager = remoteClusterStateService.getCleanupManager(); } else { remoteClusterStateService = null; remoteIndexPathUploader = null; + remoteClusterStateCleanupManager = null; } // collect engine factory providers from plugins @@ -1374,6 +1378,7 @@ protected Node( b.bind(MetricsRegistry.class).toInstance(metricsRegistry); b.bind(RemoteClusterStateService.class).toProvider(() -> remoteClusterStateService); b.bind(RemoteIndexPathUploader.class).toProvider(() -> remoteIndexPathUploader); + b.bind(RemoteClusterStateCleanupManager.class).toProvider(() -> remoteClusterStateCleanupManager); b.bind(PersistedStateRegistry.class).toInstance(persistedStateRegistry); b.bind(SegmentReplicationStatsTracker.class).toInstance(segmentReplicationStatsTracker); b.bind(SearchRequestOperationsCompositeListenerFactory.class).toInstance(searchRequestOperationsCompositeListenerFactory); diff --git a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java index 3ba98c44f8d3e..418e6d8de6adb 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java @@ -462,9 +462,7 @@ public void testDataOnlyNodePersistence() throws Exception { }); when(transportService.getThreadPool()).thenReturn(threadPool); ClusterService clusterService = mock(ClusterService.class); - when(clusterService.getClusterSettings()).thenReturn( - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); + when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); final PersistedClusterStateService persistedClusterStateService = new PersistedClusterStateService( nodeEnvironment, xContentRegistry(), @@ -487,7 +485,7 @@ public void testDataOnlyNodePersistence() throws Exception { nodeEnvironment.nodeId(), repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java new file mode 100644 index 0000000000000..24fd1b164a4ff --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java @@ -0,0 +1,446 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.service.ClusterApplierService; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.support.PlainBlobMetadata; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.AbstractAsyncTask; +import org.opensearch.core.action.ActionListener; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.VersionUtils; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.AsyncStaleFileDeletion; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; +import static org.opensearch.gateway.remote.RemoteClusterStateService.CLUSTER_STATE_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; +import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; +import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.SETTING_METADATA; +import static org.opensearch.gateway.remote.RemoteClusterStateService.TEMPLATES_METADATA; +import static org.opensearch.gateway.remote.RemoteClusterStateService.encodeString; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.nodesWithLocalNodeClusterManager; +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.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class RemoteClusterStateCleanupManagerTests extends OpenSearchTestCase { + private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; + private Supplier repositoriesServiceSupplier; + private RepositoriesService repositoriesService; + private BlobStoreRepository blobStoreRepository; + private BlobStore blobStore; + private ClusterSettings clusterSettings; + private ClusterApplierService clusterApplierService; + private ClusterState clusterState; + private Metadata metadata; + private RemoteClusterStateService remoteClusterStateService; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + repositoriesServiceSupplier = mock(Supplier.class); + repositoriesService = mock(RepositoriesService.class); + when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); + + String stateRepoTypeAttributeKey = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + "remote_store_repository" + ); + String stateRepoSettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + "remote_store_repository" + ); + + Settings settings = Settings.builder() + .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "remote_store_repository") + .put(stateRepoTypeAttributeKey, FsRepository.TYPE) + .put(stateRepoSettingsAttributeKeyPrefix + "location", "randomRepoPath") + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .build(); + + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterApplierService = mock(ClusterApplierService.class); + clusterState = mock(ClusterState.class); + metadata = mock(Metadata.class); + ClusterService clusterService = mock(ClusterService.class); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + when(clusterState.getClusterName()).thenReturn(new ClusterName("test")); + when(metadata.clusterUUID()).thenReturn("testUUID"); + when(clusterState.metadata()).thenReturn(metadata); + when(clusterApplierService.state()).thenReturn(clusterState); + when(clusterService.getClusterApplierService()).thenReturn(clusterApplierService); + + blobStoreRepository = mock(BlobStoreRepository.class); + blobStore = mock(BlobStore.class); + when(blobStoreRepository.blobStore()).thenReturn(blobStore); + when(repositoriesService.repository("remote_store_repository")).thenReturn(blobStoreRepository); + + remoteClusterStateService = mock(RemoteClusterStateService.class); + when(remoteClusterStateService.getStats()).thenReturn(new RemotePersistenceStats()); + when(remoteClusterStateService.getThreadpool()).thenReturn(threadPool); + when(remoteClusterStateService.getBlobStore()).thenReturn(blobStore); + remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(remoteClusterStateService, clusterService); + } + + @After + public void teardown() throws Exception { + super.tearDown(); + remoteClusterStateCleanupManager.close(); + threadPool.shutdown(); + } + + public void testDeleteClusterMetadata() throws IOException { + String clusterUUID = "clusterUUID"; + String clusterName = "test-cluster"; + List inactiveBlobs = Arrays.asList( + new PlainBlobMetadata("manifest1.dat", 1L), + new PlainBlobMetadata("manifest2.dat", 1L), + new PlainBlobMetadata("manifest3.dat", 1L) + ); + List activeBlobs = Arrays.asList( + new PlainBlobMetadata("manifest4.dat", 1L), + new PlainBlobMetadata("manifest5.dat", 1L) + ); + UploadedIndexMetadata index1Metadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1"); + UploadedIndexMetadata index2Metadata = new UploadedIndexMetadata("index2", "indexUUID2", "index_metadata2"); + UploadedIndexMetadata index1UpdatedMetadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1_updated"); + UploadedMetadataAttribute coordinationMetadata = new UploadedMetadataAttribute(COORDINATION_METADATA, "coordination_metadata"); + UploadedMetadataAttribute templateMetadata = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata"); + UploadedMetadataAttribute settingMetadata = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata"); + UploadedMetadataAttribute coordinationMetadataUpdated = new UploadedMetadataAttribute( + COORDINATION_METADATA, + "coordination_metadata_updated" + ); + UploadedMetadataAttribute templateMetadataUpdated = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata_updated"); + UploadedMetadataAttribute settingMetadataUpdated = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata_updated"); + ClusterMetadataManifest manifest1 = ClusterMetadataManifest.builder() + .indices(List.of(index1Metadata)) + .globalMetadataFileName("global_metadata") + .clusterTerm(1L) + .stateVersion(1L) + .codecVersion(CODEC_V1) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID(clusterUUID) + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .build(); + ClusterMetadataManifest manifest2 = ClusterMetadataManifest.builder(manifest1) + .indices(List.of(index1Metadata, index2Metadata)) + .codecVersion(CODEC_V2) + .globalMetadataFileName(null) + .coordinationMetadata(coordinationMetadata) + .templatesMetadata(templateMetadata) + .settingMetadata(settingMetadata) + .build(); + ClusterMetadataManifest manifest3 = ClusterMetadataManifest.builder(manifest2) + .indices(List.of(index1UpdatedMetadata, index2Metadata)) + .settingMetadata(settingMetadataUpdated) + .build(); + + // active manifest have reference to index1Updated, index2, settingsUpdated, coordinationUpdated, templates, templatesUpdated + ClusterMetadataManifest manifest4 = ClusterMetadataManifest.builder(manifest3) + .coordinationMetadata(coordinationMetadataUpdated) + .build(); + ClusterMetadataManifest manifest5 = ClusterMetadataManifest.builder(manifest4).templatesMetadata(templateMetadataUpdated).build(); + + when(remoteClusterStateService.fetchRemoteClusterMetadataManifest(eq(clusterName), eq(clusterUUID), any())).thenReturn( + manifest4, + manifest5, + manifest1, + manifest2, + manifest3 + ); + BlobContainer container = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).thenReturn(container); + doNothing().when(container).deleteBlobsIgnoringIfNotExists(any()); + + remoteClusterStateCleanupManager.deleteClusterMetadata(clusterName, clusterUUID, activeBlobs, inactiveBlobs); + verify(container).deleteBlobsIgnoringIfNotExists( + List.of( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + coordinationMetadata.getUploadedFilename() + ".dat", + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + settingMetadata.getUploadedFilename() + ".dat", + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + "global_metadata.dat" + ) + ); + verify(container).deleteBlobsIgnoringIfNotExists( + List.of( + new BlobPath().add(INDEX_PATH_TOKEN).add(index1Metadata.getIndexUUID()).buildAsString() + + index1Metadata.getUploadedFilePath() + + ".dat" + ) + ); + Set staleManifest = new HashSet<>(); + inactiveBlobs.forEach(blob -> staleManifest.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blob.name())); + verify(container).deleteBlobsIgnoringIfNotExists(new ArrayList<>(staleManifest)); + } + + public void testDeleteStaleClusterUUIDs() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + ClusterMetadataManifest clusterMetadataManifest = ClusterMetadataManifest.builder() + .indices(List.of()) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID("cluster-uuid1") + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .build(); + + BlobPath blobPath = new BlobPath().add("random-path"); + BlobContainer uuidContainerContainer = mock(BlobContainer.class); + BlobContainer manifest2Container = mock(BlobContainer.class); + BlobContainer manifest3Container = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).then(invocation -> { + BlobPath blobPath1 = invocation.getArgument(0); + if (blobPath1.buildAsString().endsWith("cluster-state/")) { + return uuidContainerContainer; + } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid2/")) { + return manifest2Container; + } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid3/")) { + return manifest3Container; + } else { + throw new IllegalArgumentException("Unexpected blob path " + blobPath1); + } + }); + when( + manifest2Container.listBlobsByPrefixInSortedOrder( + MANIFEST_FILE_PREFIX + DELIMITER, + Integer.MAX_VALUE, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ) + ).thenReturn(List.of(new PlainBlobMetadata("mainfest2", 1L))); + when( + manifest3Container.listBlobsByPrefixInSortedOrder( + MANIFEST_FILE_PREFIX + DELIMITER, + Integer.MAX_VALUE, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ) + ).thenReturn(List.of(new PlainBlobMetadata("mainfest3", 1L))); + Set uuids = new HashSet<>(Arrays.asList("cluster-uuid1", "cluster-uuid2", "cluster-uuid3")); + when(remoteClusterStateService.getAllClusterUUIDs(any())).thenReturn(uuids); + when(remoteClusterStateService.getCusterMetadataBasePath(any(), any())).then( + invocationOnMock -> blobPath.add(encodeString(invocationOnMock.getArgument(0))) + .add(CLUSTER_STATE_PATH_TOKEN) + .add((String) invocationOnMock.getArgument(1)) + ); + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleClusterUUIDs(clusterState, clusterMetadataManifest); + try { + assertBusy(() -> { + verify(manifest2Container, times(1)).delete(); + verify(manifest3Container, times(1)).delete(); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void testRemoteStateCleanupFailureStats() throws IOException { + BlobContainer blobContainer = mock(BlobContainer.class); + doThrow(IOException.class).when(blobContainer).delete(); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + BlobPath blobPath = new BlobPath().add("random-path"); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleUUIDsClusterMetadata("cluster1", List.of("cluster-uuid1")); + try { + assertBusy(() -> { + // wait for stats to get updated + assertNotNull(remoteClusterStateCleanupManager.getStats()); + assertEquals(0, remoteClusterStateCleanupManager.getStats().getSuccessCount()); + assertEquals(1, remoteClusterStateCleanupManager.getStats().getCleanupAttemptFailedCount()); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { + BlobContainer blobContainer = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> { + callCount.incrementAndGet(); + if (latch.await(5000, TimeUnit.SECONDS) == false) { + throw new Exception("Timed out waiting for delete task queuing to complete"); + } + return null; + }).when(blobContainer) + .listBlobsByPrefixInSortedOrder( + any(String.class), + any(int.class), + any(BlobContainer.BlobNameSortOrder.class), + any(ActionListener.class) + ); + + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); + remoteClusterStateCleanupManager.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); + + latch.countDown(); + assertBusy(() -> assertEquals(1, callCount.get())); + } + + public void testRemoteClusterStateCleanupSetting() { + remoteClusterStateCleanupManager.start(); + // verify default value + assertEquals(CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, remoteClusterStateCleanupManager.getStaleFileCleanupInterval()); + + // verify update interval + int cleanupInterval = randomIntBetween(1, 10); + Settings newSettings = Settings.builder().put("cluster.remote_store.state.cleanup_interval", cleanupInterval + "s").build(); + clusterSettings.applySettings(newSettings); + assertEquals(cleanupInterval, remoteClusterStateCleanupManager.getStaleFileCleanupInterval().seconds()); + } + + public void testRemoteCleanupTaskScheduled() { + AbstractAsyncTask cleanupTask = remoteClusterStateCleanupManager.getStaleFileDeletionTask(); + assertNull(cleanupTask); + // now the task should be initialized + remoteClusterStateCleanupManager.start(); + assertNotNull(remoteClusterStateCleanupManager.getStaleFileDeletionTask()); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().mustReschedule()); + assertEquals( + clusterSettings.get(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING), + remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval() + ); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + assertFalse(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isClosed()); + } + + public void testRemoteCleanupSkipsOnOnlyElectedClusterManager() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(false); + when(clusterState.nodes()).thenReturn(nodes); + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + spyManager.cleanUpStaleFiles(); + assertEquals(0, callCount.get()); + + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.version()).thenReturn(randomLongBetween(11, 20)); + spyManager.cleanUpStaleFiles(); + assertEquals(1, callCount.get()); + } + + public void testRemoteCleanupSkipsIfVersionIncrementLessThanThreshold() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + long version = randomLongBetween(1, SKIP_CLEANUP_STATE_CHANGES); + when(clusterApplierService.state()).thenReturn(clusterState); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.nodes()).thenReturn(nodes); + when(clusterState.version()).thenReturn(version); + + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + + remoteClusterStateCleanupManager.cleanUpStaleFiles(); + assertEquals(0, callCount.get()); + } + + public void testRemoteCleanupCallsDeleteIfVersionIncrementGreaterThanThreshold() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + long version = randomLongBetween(SKIP_CLEANUP_STATE_CHANGES + 1, SKIP_CLEANUP_STATE_CHANGES + 10); + when(clusterApplierService.state()).thenReturn(clusterState); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.nodes()).thenReturn(nodes); + when(clusterState.version()).thenReturn(version); + + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + + // using spied cleanup manager so that stubbed deleteStaleClusterMetadata is called + spyManager.cleanUpStaleFiles(); + assertEquals(1, callCount.get()); + } + + public void testRemoteCleanupSchedulesEvenAfterFailure() { + remoteClusterStateCleanupManager.start(); + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocationOnMock -> { + callCount.incrementAndGet(); + throw new RuntimeException("Test exception"); + }).when(spyManager).cleanUpStaleFiles(); + AsyncStaleFileDeletion task = new AsyncStaleFileDeletion(spyManager); + assertTrue(task.isScheduled()); + task.run(); + // Task is still scheduled after the failure + assertTrue(task.isScheduled()); + assertEquals(1, callCount.get()); + + task.run(); + // Task is still scheduled after the failure + assertTrue(task.isScheduled()); + assertEquals(2, callCount.get()); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index 1b242b921c0d7..5f0c371a3137e 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -19,6 +19,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; @@ -72,9 +73,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.Supplier; @@ -93,7 +91,6 @@ import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_CURRENT_CODEC_VERSION; import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; import static org.opensearch.gateway.remote.RemoteClusterStateService.METADATA_FILE_PREFIX; -import static org.opensearch.gateway.remote.RemoteClusterStateService.RETAINED_MANIFESTS; import static org.opensearch.gateway.remote.RemoteClusterStateService.SETTING_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.TEMPLATES_METADATA; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; @@ -109,13 +106,12 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class RemoteClusterStateServiceTests extends OpenSearchTestCase { private RemoteClusterStateService remoteClusterStateService; + private ClusterService clusterService; private ClusterSettings clusterSettings; private Supplier repositoriesServiceSupplier; private RepositoriesService repositoriesService; @@ -148,6 +144,8 @@ public void setup() { .build(); clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterService = mock(ClusterService.class); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( Stream.of( NetworkModule.getNamedXContents().stream(), @@ -165,7 +163,7 @@ public void setup() { "test-node-id", repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) @@ -187,14 +185,14 @@ public void testFailWriteFullMetadataNonClusterManagerNode() throws IOException public void testFailInitializationWhenRemoteStateDisabled() { final Settings settings = Settings.builder().build(); - ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); assertThrows( AssertionError.class, () -> new RemoteClusterStateService( "test-node-id", repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) @@ -1280,72 +1278,6 @@ public void testGetValidPreviousClusterUUIDWhenLastUUIDUncommitted() throws IOEx assertThat(previousClusterUUID, equalTo("cluster-uuid2")); } - public void testDeleteStaleClusterUUIDs() throws IOException { - final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); - ClusterMetadataManifest clusterMetadataManifest = ClusterMetadataManifest.builder() - .indices(List.of()) - .clusterTerm(1L) - .stateVersion(1L) - .stateUUID(randomAlphaOfLength(10)) - .clusterUUID("cluster-uuid1") - .nodeId("nodeA") - .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) - .previousClusterUUID(ClusterState.UNKNOWN_UUID) - .committed(true) - .build(); - - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - BlobContainer uuidContainerContainer = mock(BlobContainer.class); - BlobContainer manifest2Container = mock(BlobContainer.class); - BlobContainer manifest3Container = mock(BlobContainer.class); - when(blobStore.blobContainer(any())).then(invocation -> { - BlobPath blobPath1 = invocation.getArgument(0); - if (blobPath1.buildAsString().endsWith("cluster-state/")) { - return uuidContainerContainer; - } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid2/")) { - return manifest2Container; - } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid3/")) { - return manifest3Container; - } else { - throw new IllegalArgumentException("Unexpected blob path " + blobPath1); - } - }); - Map blobMetadataMap = Map.of( - "cluster-uuid1", - mock(BlobContainer.class), - "cluster-uuid2", - mock(BlobContainer.class), - "cluster-uuid3", - mock(BlobContainer.class) - ); - when(uuidContainerContainer.children()).thenReturn(blobMetadataMap); - when( - manifest2Container.listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - Integer.MAX_VALUE, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(List.of(new PlainBlobMetadata("mainfest2", 1L))); - when( - manifest3Container.listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - Integer.MAX_VALUE, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(List.of(new PlainBlobMetadata("mainfest3", 1L))); - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleClusterUUIDs(clusterState, clusterMetadataManifest); - try { - assertBusy(() -> { - verify(manifest2Container, times(1)).delete(); - verify(manifest3Container, times(1)).delete(); - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - public void testRemoteStateStats() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); mockBlobStoreObjects(); @@ -1358,26 +1290,6 @@ public void testRemoteStateStats() throws IOException { assertEquals(0, remoteClusterStateService.getStats().getFailedCount()); } - public void testRemoteStateCleanupFailureStats() throws IOException { - BlobContainer blobContainer = mock(BlobContainer.class); - doThrow(IOException.class).when(blobContainer).delete(); - when(blobStore.blobContainer(any())).thenReturn(blobContainer); - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleUUIDsClusterMetadata("cluster1", Arrays.asList("cluster-uuid1")); - try { - assertBusy(() -> { - // wait for stats to get updated - assertTrue(remoteClusterStateService.getStats() != null); - assertEquals(0, remoteClusterStateService.getStats().getSuccessCount()); - assertEquals(1, remoteClusterStateService.getStats().getCleanupAttemptFailedCount()); - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - public void testFileNames() { final Index index = new Index("test-index", "index-uuid"); final Settings idxSettings = Settings.builder() @@ -1418,36 +1330,6 @@ private void verifyManifestFileNameWithCodec(int codecVersion) { assertThat(splittedName[3], is("P")); } - public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { - BlobContainer blobContainer = mock(BlobContainer.class); - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - when(blobStore.blobContainer(any())).thenReturn(blobContainer); - - CountDownLatch latch = new CountDownLatch(1); - AtomicInteger callCount = new AtomicInteger(0); - doAnswer(invocation -> { - callCount.incrementAndGet(); - if (latch.await(5000, TimeUnit.SECONDS) == false) { - throw new Exception("Timed out waiting for delete task queuing to complete"); - } - return null; - }).when(blobContainer) - .listBlobsByPrefixInSortedOrder( - any(String.class), - any(int.class), - any(BlobContainer.BlobNameSortOrder.class), - any(ActionListener.class) - ); - - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); - remoteClusterStateService.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); - - latch.countDown(); - assertBusy(() -> assertEquals(1, callCount.get())); - } - public void testIndexMetadataUploadWaitTimeSetting() { // verify default value assertEquals( @@ -1891,7 +1773,7 @@ private static ClusterState.Builder generateClusterStateWithGlobalMetadata() { ); } - private static ClusterState.Builder generateClusterStateWithOneIndex() { + static ClusterState.Builder generateClusterStateWithOneIndex() { final Index index = new Index("test-index", "index-uuid"); final Settings idxSettings = Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) @@ -1921,7 +1803,7 @@ private static ClusterState.Builder generateClusterStateWithOneIndex() { ); } - private static DiscoveryNodes nodesWithLocalNodeClusterManager() { + static DiscoveryNodes nodesWithLocalNodeClusterManager() { return DiscoveryNodes.builder().clusterManagerNodeId("cluster-manager-id").localNodeId("cluster-manager-id").build(); } From 983297289325bdd1c832bfe18d105bd3e9727ab5 Mon Sep 17 00:00:00 2001 From: Craig Perkins Date: Wed, 29 May 2024 08:35:43 -0400 Subject: [PATCH 18/77] Remove deprecated constant META_FIELDS_BEFORE_7DOT8 (#13860) * Add _primary_term to builtInMetadataMappers Signed-off-by: Craig Perkins * Add to CHANGELOG Signed-off-by: Craig Perkins * Remove deprecated constant META_FIELDS_BEFORE_7DOT8 Signed-off-by: Craig Perkins * Update CHANGELOG Signed-off-by: Craig Perkins --------- Signed-off-by: Craig Perkins --- CHANGELOG.md | 1 + .../java/org/opensearch/index/mapper/MapperService.java | 6 ------ 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a48c4635d8dec..03292154be11c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -13,6 +13,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Remote Store] Upload translog checkpoint as object metadata to translog.tlog([#13637](https://github.com/opensearch-project/OpenSearch/pull/13637)) - Add getMetadataFields to MapperService ([#13819](https://github.com/opensearch-project/OpenSearch/pull/13819)) - [Remote State] Add async remote state deletion task running on an interval, configurable by a setting ([#13131](https://github.com/opensearch-project/OpenSearch/pull/13131)) +- Remove deprecated constant META_FIELDS_BEFORE_7DOT8 ([#13860](https://github.com/opensearch-project/OpenSearch/pull/13860)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) diff --git a/server/src/main/java/org/opensearch/index/mapper/MapperService.java b/server/src/main/java/org/opensearch/index/mapper/MapperService.java index a1f3894c9f14c..83fb743305702 100644 --- a/server/src/main/java/org/opensearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/opensearch/index/mapper/MapperService.java @@ -73,7 +73,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -201,11 +200,6 @@ public enum MergeReason { Property.IndexScope, Property.Deprecated ); - // Deprecated set of meta-fields, for checking if a field is meta, use an instance method isMetadataField instead - @Deprecated - public static final Set META_FIELDS_BEFORE_7DOT8 = Collections.unmodifiableSet( - new HashSet<>(Arrays.asList("_id", IgnoredFieldMapper.NAME, "_index", "_routing", "_size", "_timestamp", "_ttl", "_type")) - ); private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(MapperService.class); From 05fb780a3aec3bfb46c0cb76c65e340b11d42def Mon Sep 17 00:00:00 2001 From: Navneet Verma Date: Wed, 29 May 2024 07:14:36 -0700 Subject: [PATCH 19/77] Enabled reloading functionality for reloading KNNVectorsFormat SPIs from plugins (#13864) Signed-off-by: Navneet Verma --- server/src/main/java/org/opensearch/plugins/PluginsService.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/main/java/org/opensearch/plugins/PluginsService.java b/server/src/main/java/org/opensearch/plugins/PluginsService.java index a6eefd2f4fd17..f08c9c738f1b4 100644 --- a/server/src/main/java/org/opensearch/plugins/PluginsService.java +++ b/server/src/main/java/org/opensearch/plugins/PluginsService.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.DocValuesFormat; +import org.apache.lucene.codecs.KnnVectorsFormat; import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.util.SPIClassIterator; import org.opensearch.Build; @@ -762,6 +763,7 @@ static void reloadLuceneSPI(ClassLoader loader) { // Codecs: PostingsFormat.reloadPostingsFormats(loader); DocValuesFormat.reloadDocValuesFormats(loader); + KnnVectorsFormat.reloadKnnVectorsFormat(loader); Codec.reloadCodecs(loader); } From 3b14f0a88bb6cd74041e927b435436a1cbb26bc0 Mon Sep 17 00:00:00 2001 From: Craig Perkins Date: Wed, 29 May 2024 10:48:43 -0400 Subject: [PATCH 20/77] Move CHANGELOG entry for #13860 to CHANGELOG 3.0 (#13874) Signed-off-by: Craig Perkins --- CHANGELOG-3.0.md | 1 + CHANGELOG.md | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG-3.0.md b/CHANGELOG-3.0.md index 964383078c38d..af788c40ae304 100644 --- a/CHANGELOG-3.0.md +++ b/CHANGELOG-3.0.md @@ -24,6 +24,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add task completion count in search backpressure stats API ([#10028](https://github.com/opensearch-project/OpenSearch/pull/10028/)) - Deprecate CamelCase `PathHierarchy` tokenizer name in favor to lowercase `path_hierarchy` ([#10894](https://github.com/opensearch-project/OpenSearch/pull/10894)) - Breaking change: Do not request "search_pipelines" metrics by default in NodesInfoRequest ([#12497](https://github.com/opensearch-project/OpenSearch/pull/12497)) +- Remove deprecated constant META_FIELDS_BEFORE_7DOT8 ([#13860](https://github.com/opensearch-project/OpenSearch/pull/13860)) ### Deprecated diff --git a/CHANGELOG.md b/CHANGELOG.md index 03292154be11c..a48c4635d8dec 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -13,7 +13,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Remote Store] Upload translog checkpoint as object metadata to translog.tlog([#13637](https://github.com/opensearch-project/OpenSearch/pull/13637)) - Add getMetadataFields to MapperService ([#13819](https://github.com/opensearch-project/OpenSearch/pull/13819)) - [Remote State] Add async remote state deletion task running on an interval, configurable by a setting ([#13131](https://github.com/opensearch-project/OpenSearch/pull/13131)) -- Remove deprecated constant META_FIELDS_BEFORE_7DOT8 ([#13860](https://github.com/opensearch-project/OpenSearch/pull/13860)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) From 84ad3a7d2dc4410954b30d1c22d908047cfdd73b Mon Sep 17 00:00:00 2001 From: Sagar <99425694+sgup432@users.noreply.github.com> Date: Wed, 29 May 2024 08:09:27 -0700 Subject: [PATCH 21/77] Remove changelog entry for the released fix (#13598) Signed-off-by: Sagar Upadhyaya Signed-off-by: Sagar Upadhyaya Co-authored-by: Sagar Upadhyaya --- CHANGELOG.md | 1 - release-notes/opensearch.release-notes-2.14.0.md | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a48c4635d8dec..782c9d04be0b4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -42,7 +42,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Remove handling of index.mapper.dynamic in AutoCreateIndex([#13067](https://github.com/opensearch-project/OpenSearch/pull/13067)) ### Fixed -- Fix negative RequestStats metric issue ([#13553](https://github.com/opensearch-project/OpenSearch/pull/13553)) - Fix get field mapping API returns 404 error in mixed cluster with multiple versions ([#13624](https://github.com/opensearch-project/OpenSearch/pull/13624)) - Allow clearing `remote_store.compatibility_mode` setting ([#13646](https://github.com/opensearch-project/OpenSearch/pull/13646)) - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) diff --git a/release-notes/opensearch.release-notes-2.14.0.md b/release-notes/opensearch.release-notes-2.14.0.md index 8ef0215baa67a..c5fc3e895c45d 100644 --- a/release-notes/opensearch.release-notes-2.14.0.md +++ b/release-notes/opensearch.release-notes-2.14.0.md @@ -84,4 +84,5 @@ - Improve the error messages for _stats with closed indices ([#13012](https://github.com/opensearch-project/OpenSearch/pull/13012)) - Ignore BaseRestHandler unconsumed content check as it's always consumed. ([#13290](https://github.com/opensearch-project/OpenSearch/pull/13290)) - Fix mapper_parsing_exception when using flat_object fields with names longer than 11 characters ([#13259](https://github.com/opensearch-project/OpenSearch/pull/13259)) -- DATETIME_FORMATTER_CACHING_SETTING experimental feature should not default to 'true' ([#13532](https://github.com/opensearch-project/OpenSearch/pull/13532)) \ No newline at end of file +- DATETIME_FORMATTER_CACHING_SETTING experimental feature should not default to 'true' ([#13532](https://github.com/opensearch-project/OpenSearch/pull/13532)) +- Fix negative RequestStats metric issue ([#13553](https://github.com/opensearch-project/OpenSearch/pull/13553)) From 2fd2c3414c27b6f6d501130fd0b5a39d3e5a5139 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Wed, 29 May 2024 16:43:00 -0400 Subject: [PATCH 22/77] Attempt to stabilize MacOS GA runner caused by QEMU / Colima / Docker issues (#13877) Signed-off-by: Andriy Redko --- .github/workflows/assemble.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/assemble.yml b/.github/workflows/assemble.yml index d18170e9ea6b7..4bff5b7d60d1f 100644 --- a/.github/workflows/assemble.yml +++ b/.github/workflows/assemble.yml @@ -19,6 +19,8 @@ jobs: - name: Setup docker (missing on MacOS) if: runner.os == 'macos' uses: douglascamata/setup-docker-macos-action@main + with: + upgrade-qemu: true - name: Run Gradle (assemble) run: | ./gradlew assemble --parallel --no-build-cache -PDISABLE_BUILD_CACHE From db5240e06dc08e7d7d03432595bb4d93b0e2e32d Mon Sep 17 00:00:00 2001 From: Heemin Kim Date: Wed, 29 May 2024 22:13:28 -0700 Subject: [PATCH 23/77] Pass parent filter to inner hit query (#13770) Signed-off-by: Heemin Kim --- CHANGELOG.md | 1 + .../index/query/NestedQueryBuilder.java | 36 +++++++++++++------ .../index/query/NestedQueryBuilderTests.java | 35 ++++++++++++++++++ 3 files changed, 62 insertions(+), 10 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 782c9d04be0b4..5416005e4706c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -45,6 +45,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix get field mapping API returns 404 error in mixed cluster with multiple versions ([#13624](https://github.com/opensearch-project/OpenSearch/pull/13624)) - Allow clearing `remote_store.compatibility_mode` setting ([#13646](https://github.com/opensearch-project/OpenSearch/pull/13646)) - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) +- Pass parent filter to inner hit query ([#13770](https://github.com/opensearch-project/OpenSearch/pull/13770)) ### Security diff --git a/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java index 3f97b3918a126..b5ba79632b622 100644 --- a/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java @@ -401,16 +401,32 @@ protected void doBuild(SearchContext parentSearchContext, InnerHitsContext inner } } String name = innerHitBuilder.getName() != null ? innerHitBuilder.getName() : nestedObjectMapper.fullPath(); - ObjectMapper parentObjectMapper = queryShardContext.nestedScope().nextLevel(nestedObjectMapper); - NestedInnerHitSubContext nestedInnerHits = new NestedInnerHitSubContext( - name, - parentSearchContext, - parentObjectMapper, - nestedObjectMapper - ); - setupInnerHitsContext(queryShardContext, nestedInnerHits); - queryShardContext.nestedScope().previousLevel(); - innerHitsContext.addInnerHitDefinition(nestedInnerHits); + ObjectMapper parentObjectMapper = queryShardContext.nestedScope().getObjectMapper(); + BitSetProducer parentFilter; + if (parentObjectMapper == null) { + parentFilter = queryShardContext.bitsetFilter(Queries.newNonNestedFilter()); + } else { + parentFilter = queryShardContext.bitsetFilter(parentObjectMapper.nestedTypeFilter()); + } + BitSetProducer previousParentFilter = queryShardContext.getParentFilter(); + try { + queryShardContext.setParentFilter(parentFilter); + queryShardContext.nestedScope().nextLevel(nestedObjectMapper); + try { + NestedInnerHitSubContext nestedInnerHits = new NestedInnerHitSubContext( + name, + parentSearchContext, + parentObjectMapper, + nestedObjectMapper + ); + setupInnerHitsContext(queryShardContext, nestedInnerHits); + innerHitsContext.addInnerHitDefinition(nestedInnerHits); + } finally { + queryShardContext.nestedScope().previousLevel(); + } + } finally { + queryShardContext.setParentFilter(previousParentFilter); + } } } diff --git a/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java b/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java index 29efd64e5c751..096acd23429bd 100644 --- a/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java @@ -311,6 +311,41 @@ public void testInlineLeafInnerHitsNestedQuery() throws Exception { assertThat(innerHitBuilders.get(leafInnerHits.getName()), Matchers.notNullValue()); } + public void testParentFilterFromInlineLeafInnerHitsNestedQuery() throws Exception { + QueryShardContext queryShardContext = createShardContext(); + SearchContext searchContext = mock(SearchContext.class); + when(searchContext.getQueryShardContext()).thenReturn(queryShardContext); + + MapperService mapperService = mock(MapperService.class); + IndexSettings settings = new IndexSettings(newIndexMeta("index", Settings.EMPTY), Settings.EMPTY); + when(mapperService.getIndexSettings()).thenReturn(settings); + when(searchContext.mapperService()).thenReturn(mapperService); + + InnerHitBuilder leafInnerHits = randomNestedInnerHits(); + leafInnerHits.setScriptFields(null); + leafInnerHits.setHighlightBuilder(null); + + QueryBuilder innerQueryBuilder = spy(new MatchAllQueryBuilder()); + when(innerQueryBuilder.toQuery(queryShardContext)).thenAnswer(invoke -> { + QueryShardContext context = invoke.getArgument(0); + if (context.getParentFilter() == null) { + throw new Exception("Expect parent filter to be non-null"); + } + return invoke.callRealMethod(); + }); + NestedQueryBuilder query = new NestedQueryBuilder("nested1", innerQueryBuilder, ScoreMode.None); + query.innerHit(leafInnerHits); + final Map innerHitBuilders = new HashMap<>(); + final InnerHitsContext innerHitsContext = new InnerHitsContext(); + query.extractInnerHitBuilders(innerHitBuilders); + assertThat(innerHitBuilders.size(), Matchers.equalTo(1)); + assertTrue(innerHitBuilders.containsKey(leafInnerHits.getName())); + assertNull(queryShardContext.getParentFilter()); + innerHitBuilders.get(leafInnerHits.getName()).build(searchContext, innerHitsContext); + assertNull(queryShardContext.getParentFilter()); + verify(innerQueryBuilder).toQuery(queryShardContext); + } + public void testInlineLeafInnerHitsNestedQueryViaBoolQuery() { InnerHitBuilder leafInnerHits = randomNestedInnerHits(); NestedQueryBuilder nestedQueryBuilder = new NestedQueryBuilder("path", new MatchAllQueryBuilder(), ScoreMode.None).innerHit( From 2e497436cf9dd78aa3d06f86285aef897c962cac Mon Sep 17 00:00:00 2001 From: Shivansh Arora <31575408+shiv0408@users.noreply.github.com> Date: Thu, 30 May 2024 16:53:52 +0530 Subject: [PATCH 24/77] Remove conflicting static method from Metadata.Custom interface (#13869) Signed-off-by: Shivansh Arora --- .../main/java/org/opensearch/cluster/metadata/Metadata.java | 5 ----- .../opensearch/gateway/remote/RemoteClusterStateService.java | 2 +- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java index d016501dd0910..e1aa5626f36c1 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java @@ -180,11 +180,6 @@ static Custom fromXContent(XContentParser parser, String name) throws IOExceptio // handling any Exception is caller's responsibility return parser.namedObject(Custom.class, name, null); } - - static Custom fromXContent(XContentParser parser) throws IOException { - String currentFieldName = parser.currentName(); - return fromXContent(parser, currentFieldName); - } } public static final Setting DEFAULT_REPLICA_COUNT_SETTING = Setting.intSetting( diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index 9bf1dff2359ca..01ffd8f1cca46 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -147,7 +147,7 @@ public class RemoteClusterStateService implements Closeable { public static final ChecksumBlobStoreFormat CUSTOM_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( "custom", METADATA_NAME_FORMAT, - Metadata.Custom::fromXContent + null // no need of reader here, as this object is only used to write/serialize the object ); /** From c7e8421b67566eedfa9b6c640afceca7b30089a1 Mon Sep 17 00:00:00 2001 From: Michael Froh Date: Thu, 30 May 2024 17:12:57 +0000 Subject: [PATCH 25/77] Fix flaky test S3BlobStoreRepositoryTests.testRequestStats (#13887) If a BlobStoreRepository is created, but the blobStore() method is never called, then the actual blobStore instance won't be created. This means the repository will always emit non-detailed empty stats. When we would try to merge these stats with another repository that was initialized, it would fail the assertion that they either both have detailed stats or neither did. This fix looks like it's checking if the blobStore has been initialized, but by calling blobStore(), it's making sure that it gets initialized. Signed-off-by: Michael Froh --- .../repositories/s3/S3BlobStoreRepositoryTests.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java index f72374d63808a..c5438d58e679d 100644 --- a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java +++ b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java @@ -73,7 +73,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.stream.StreamSupport; import fixture.s3.S3HttpHandler; @@ -206,7 +205,12 @@ public void testRequestStats() throws Exception { } catch (RepositoryMissingException e) { return null; } - }).filter(Objects::nonNull).map(Repository::stats).reduce(RepositoryStats::merge).get(); + }).filter(b -> { + if (b instanceof BlobStoreRepository) { + return ((BlobStoreRepository) b).blobStore() != null; + } + return false; + }).map(Repository::stats).reduce(RepositoryStats::merge).get(); Map> extendedStats = repositoryStats.extendedStats; Map aggregatedStats = new HashMap<>(); From 9632bd6277131d1583b80641f3684e741ddf1544 Mon Sep 17 00:00:00 2001 From: Heemin Kim Date: Thu, 30 May 2024 11:37:01 -0700 Subject: [PATCH 26/77] Revert "Pass parent filter to inner hit query (#13770)" (#13896) This reverts commit db5240e06dc08e7d7d03432595bb4d93b0e2e32d. --- CHANGELOG.md | 1 - .../index/query/NestedQueryBuilder.java | 36 ++++++------------- .../index/query/NestedQueryBuilderTests.java | 35 ------------------ 3 files changed, 10 insertions(+), 62 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5416005e4706c..782c9d04be0b4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -45,7 +45,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix get field mapping API returns 404 error in mixed cluster with multiple versions ([#13624](https://github.com/opensearch-project/OpenSearch/pull/13624)) - Allow clearing `remote_store.compatibility_mode` setting ([#13646](https://github.com/opensearch-project/OpenSearch/pull/13646)) - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) -- Pass parent filter to inner hit query ([#13770](https://github.com/opensearch-project/OpenSearch/pull/13770)) ### Security diff --git a/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java index b5ba79632b622..3f97b3918a126 100644 --- a/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java @@ -401,32 +401,16 @@ protected void doBuild(SearchContext parentSearchContext, InnerHitsContext inner } } String name = innerHitBuilder.getName() != null ? innerHitBuilder.getName() : nestedObjectMapper.fullPath(); - ObjectMapper parentObjectMapper = queryShardContext.nestedScope().getObjectMapper(); - BitSetProducer parentFilter; - if (parentObjectMapper == null) { - parentFilter = queryShardContext.bitsetFilter(Queries.newNonNestedFilter()); - } else { - parentFilter = queryShardContext.bitsetFilter(parentObjectMapper.nestedTypeFilter()); - } - BitSetProducer previousParentFilter = queryShardContext.getParentFilter(); - try { - queryShardContext.setParentFilter(parentFilter); - queryShardContext.nestedScope().nextLevel(nestedObjectMapper); - try { - NestedInnerHitSubContext nestedInnerHits = new NestedInnerHitSubContext( - name, - parentSearchContext, - parentObjectMapper, - nestedObjectMapper - ); - setupInnerHitsContext(queryShardContext, nestedInnerHits); - innerHitsContext.addInnerHitDefinition(nestedInnerHits); - } finally { - queryShardContext.nestedScope().previousLevel(); - } - } finally { - queryShardContext.setParentFilter(previousParentFilter); - } + ObjectMapper parentObjectMapper = queryShardContext.nestedScope().nextLevel(nestedObjectMapper); + NestedInnerHitSubContext nestedInnerHits = new NestedInnerHitSubContext( + name, + parentSearchContext, + parentObjectMapper, + nestedObjectMapper + ); + setupInnerHitsContext(queryShardContext, nestedInnerHits); + queryShardContext.nestedScope().previousLevel(); + innerHitsContext.addInnerHitDefinition(nestedInnerHits); } } diff --git a/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java b/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java index 096acd23429bd..29efd64e5c751 100644 --- a/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java @@ -311,41 +311,6 @@ public void testInlineLeafInnerHitsNestedQuery() throws Exception { assertThat(innerHitBuilders.get(leafInnerHits.getName()), Matchers.notNullValue()); } - public void testParentFilterFromInlineLeafInnerHitsNestedQuery() throws Exception { - QueryShardContext queryShardContext = createShardContext(); - SearchContext searchContext = mock(SearchContext.class); - when(searchContext.getQueryShardContext()).thenReturn(queryShardContext); - - MapperService mapperService = mock(MapperService.class); - IndexSettings settings = new IndexSettings(newIndexMeta("index", Settings.EMPTY), Settings.EMPTY); - when(mapperService.getIndexSettings()).thenReturn(settings); - when(searchContext.mapperService()).thenReturn(mapperService); - - InnerHitBuilder leafInnerHits = randomNestedInnerHits(); - leafInnerHits.setScriptFields(null); - leafInnerHits.setHighlightBuilder(null); - - QueryBuilder innerQueryBuilder = spy(new MatchAllQueryBuilder()); - when(innerQueryBuilder.toQuery(queryShardContext)).thenAnswer(invoke -> { - QueryShardContext context = invoke.getArgument(0); - if (context.getParentFilter() == null) { - throw new Exception("Expect parent filter to be non-null"); - } - return invoke.callRealMethod(); - }); - NestedQueryBuilder query = new NestedQueryBuilder("nested1", innerQueryBuilder, ScoreMode.None); - query.innerHit(leafInnerHits); - final Map innerHitBuilders = new HashMap<>(); - final InnerHitsContext innerHitsContext = new InnerHitsContext(); - query.extractInnerHitBuilders(innerHitBuilders); - assertThat(innerHitBuilders.size(), Matchers.equalTo(1)); - assertTrue(innerHitBuilders.containsKey(leafInnerHits.getName())); - assertNull(queryShardContext.getParentFilter()); - innerHitBuilders.get(leafInnerHits.getName()).build(searchContext, innerHitsContext); - assertNull(queryShardContext.getParentFilter()); - verify(innerQueryBuilder).toQuery(queryShardContext); - } - public void testInlineLeafInnerHitsNestedQueryViaBoolQuery() { InnerHitBuilder leafInnerHits = randomNestedInnerHits(); NestedQueryBuilder nestedQueryBuilder = new NestedQueryBuilder("path", new MatchAllQueryBuilder(), ScoreMode.None).innerHit( From 6c9603a795cc98945af2d0d9771091ecdb09e61f Mon Sep 17 00:00:00 2001 From: Heemin Kim Date: Thu, 30 May 2024 17:37:39 -0700 Subject: [PATCH 27/77] Pass parent filter to inner hit query (#13903) Signed-off-by: Heemin Kim --- CHANGELOG.md | 1 + .../index/query/NestedQueryBuilder.java | 36 +++++++++++++----- .../index/query/NestedQueryBuilderTests.java | 37 +++++++++++++++++++ 3 files changed, 64 insertions(+), 10 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 782c9d04be0b4..f260799b0885e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -45,6 +45,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix get field mapping API returns 404 error in mixed cluster with multiple versions ([#13624](https://github.com/opensearch-project/OpenSearch/pull/13624)) - Allow clearing `remote_store.compatibility_mode` setting ([#13646](https://github.com/opensearch-project/OpenSearch/pull/13646)) - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) +- Pass parent filter to inner hit query ([#13903](https://github.com/opensearch-project/OpenSearch/pull/13903)) ### Security diff --git a/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java index 3f97b3918a126..b5ba79632b622 100644 --- a/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/NestedQueryBuilder.java @@ -401,16 +401,32 @@ protected void doBuild(SearchContext parentSearchContext, InnerHitsContext inner } } String name = innerHitBuilder.getName() != null ? innerHitBuilder.getName() : nestedObjectMapper.fullPath(); - ObjectMapper parentObjectMapper = queryShardContext.nestedScope().nextLevel(nestedObjectMapper); - NestedInnerHitSubContext nestedInnerHits = new NestedInnerHitSubContext( - name, - parentSearchContext, - parentObjectMapper, - nestedObjectMapper - ); - setupInnerHitsContext(queryShardContext, nestedInnerHits); - queryShardContext.nestedScope().previousLevel(); - innerHitsContext.addInnerHitDefinition(nestedInnerHits); + ObjectMapper parentObjectMapper = queryShardContext.nestedScope().getObjectMapper(); + BitSetProducer parentFilter; + if (parentObjectMapper == null) { + parentFilter = queryShardContext.bitsetFilter(Queries.newNonNestedFilter()); + } else { + parentFilter = queryShardContext.bitsetFilter(parentObjectMapper.nestedTypeFilter()); + } + BitSetProducer previousParentFilter = queryShardContext.getParentFilter(); + try { + queryShardContext.setParentFilter(parentFilter); + queryShardContext.nestedScope().nextLevel(nestedObjectMapper); + try { + NestedInnerHitSubContext nestedInnerHits = new NestedInnerHitSubContext( + name, + parentSearchContext, + parentObjectMapper, + nestedObjectMapper + ); + setupInnerHitsContext(queryShardContext, nestedInnerHits); + innerHitsContext.addInnerHitDefinition(nestedInnerHits); + } finally { + queryShardContext.nestedScope().previousLevel(); + } + } finally { + queryShardContext.setParentFilter(previousParentFilter); + } } } diff --git a/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java b/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java index 29efd64e5c751..f72bd76913c8f 100644 --- a/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/query/NestedQueryBuilderTests.java @@ -311,6 +311,43 @@ public void testInlineLeafInnerHitsNestedQuery() throws Exception { assertThat(innerHitBuilders.get(leafInnerHits.getName()), Matchers.notNullValue()); } + public void testParentFilterFromInlineLeafInnerHitsNestedQuery() throws Exception { + QueryShardContext queryShardContext = createShardContext(); + SearchContext searchContext = mock(SearchContext.class); + when(searchContext.getQueryShardContext()).thenReturn(queryShardContext); + + MapperService mapperService = mock(MapperService.class); + IndexSettings settings = new IndexSettings(newIndexMeta("index", Settings.EMPTY), Settings.EMPTY); + when(mapperService.getIndexSettings()).thenReturn(settings); + when(searchContext.mapperService()).thenReturn(mapperService); + + InnerHitBuilder leafInnerHits = randomNestedInnerHits(); + // Set null for values not related with this test case + leafInnerHits.setScriptFields(null); + leafInnerHits.setHighlightBuilder(null); + leafInnerHits.setSorts(null); + + QueryBuilder innerQueryBuilder = spy(new MatchAllQueryBuilder()); + when(innerQueryBuilder.toQuery(queryShardContext)).thenAnswer(invoke -> { + QueryShardContext context = invoke.getArgument(0); + if (context.getParentFilter() == null) { + throw new Exception("Expect parent filter to be non-null"); + } + return invoke.callRealMethod(); + }); + NestedQueryBuilder query = new NestedQueryBuilder("nested1", innerQueryBuilder, ScoreMode.None); + query.innerHit(leafInnerHits); + final Map innerHitBuilders = new HashMap<>(); + final InnerHitsContext innerHitsContext = new InnerHitsContext(); + query.extractInnerHitBuilders(innerHitBuilders); + assertThat(innerHitBuilders.size(), Matchers.equalTo(1)); + assertTrue(innerHitBuilders.containsKey(leafInnerHits.getName())); + assertNull(queryShardContext.getParentFilter()); + innerHitBuilders.get(leafInnerHits.getName()).build(searchContext, innerHitsContext); + assertNull(queryShardContext.getParentFilter()); + verify(innerQueryBuilder).toQuery(queryShardContext); + } + public void testInlineLeafInnerHitsNestedQueryViaBoolQuery() { InnerHitBuilder leafInnerHits = randomNestedInnerHits(); NestedQueryBuilder nestedQueryBuilder = new NestedQueryBuilder("path", new MatchAllQueryBuilder(), ScoreMode.None).innerHit( From a32859bb5a5acc54b62df5384c7b7c949c647c2b Mon Sep 17 00:00:00 2001 From: Oliver Lockwood Date: Fri, 31 May 2024 13:32:52 +0100 Subject: [PATCH 28/77] 13776: allow adding query parameters to RequestOptions (#13777) * 13776: allow adding query parameters to RequestOptions Signed-off-by: Oliver Lockwood * Fix bug highlighted by unit testing Signed-off-by: Oliver Lockwood * Address code review comments Signed-off-by: Oliver Lockwood * Run spotlessApply to satisfy formatting rules Signed-off-by: Oliver Lockwood * Fix more queryParams->parameters cases Signed-off-by: Oliver Lockwood * Apply code review feedback Signed-off-by: Oliver Lockwood --------- Signed-off-by: Oliver Lockwood --- CHANGELOG.md | 1 + .../java/org/opensearch/client/Request.java | 8 +++- .../org/opensearch/client/RequestOptions.java | 43 ++++++++++++++++++- .../client/RequestOptionsTests.java | 43 +++++++++++++++++++ 4 files changed, 92 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f260799b0885e..a98d0be56a658 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -13,6 +13,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Remote Store] Upload translog checkpoint as object metadata to translog.tlog([#13637](https://github.com/opensearch-project/OpenSearch/pull/13637)) - Add getMetadataFields to MapperService ([#13819](https://github.com/opensearch-project/OpenSearch/pull/13819)) - [Remote State] Add async remote state deletion task running on an interval, configurable by a setting ([#13131](https://github.com/opensearch-project/OpenSearch/pull/13131)) +- Allow setting query parameters on requests ([#13776](https://github.com/opensearch-project/OpenSearch/issues/13776)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) diff --git a/client/rest/src/main/java/org/opensearch/client/Request.java b/client/rest/src/main/java/org/opensearch/client/Request.java index 441b01b0891ad..32fedee0c97bf 100644 --- a/client/rest/src/main/java/org/opensearch/client/Request.java +++ b/client/rest/src/main/java/org/opensearch/client/Request.java @@ -110,7 +110,13 @@ public void addParameters(Map paramSource) { * will change it. */ public Map getParameters() { - return unmodifiableMap(parameters); + if (options.getParameters().isEmpty()) { + return unmodifiableMap(parameters); + } else { + Map combinedParameters = new HashMap<>(parameters); + combinedParameters.putAll(options.getParameters()); + return unmodifiableMap(combinedParameters); + } } /** diff --git a/client/rest/src/main/java/org/opensearch/client/RequestOptions.java b/client/rest/src/main/java/org/opensearch/client/RequestOptions.java index 189d785faaf45..bbc1f8bc85fcb 100644 --- a/client/rest/src/main/java/org/opensearch/client/RequestOptions.java +++ b/client/rest/src/main/java/org/opensearch/client/RequestOptions.java @@ -40,8 +40,11 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; /** * The portion of an HTTP request to OpenSearch that can be @@ -53,18 +56,21 @@ public final class RequestOptions { */ public static final RequestOptions DEFAULT = new Builder( Collections.emptyList(), + Collections.emptyMap(), HeapBufferedResponseConsumerFactory.DEFAULT, null, null ).build(); private final List
headers; + private final Map parameters; private final HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory; private final WarningsHandler warningsHandler; private final RequestConfig requestConfig; private RequestOptions(Builder builder) { this.headers = Collections.unmodifiableList(new ArrayList<>(builder.headers)); + this.parameters = Collections.unmodifiableMap(new HashMap<>(builder.parameters)); this.httpAsyncResponseConsumerFactory = builder.httpAsyncResponseConsumerFactory; this.warningsHandler = builder.warningsHandler; this.requestConfig = builder.requestConfig; @@ -74,7 +80,7 @@ private RequestOptions(Builder builder) { * Create a builder that contains these options but can be modified. */ public Builder toBuilder() { - return new Builder(headers, httpAsyncResponseConsumerFactory, warningsHandler, requestConfig); + return new Builder(headers, parameters, httpAsyncResponseConsumerFactory, warningsHandler, requestConfig); } /** @@ -84,6 +90,14 @@ public List
getHeaders() { return headers; } + /** + * Query parameters to attach to the request. Any parameters present here + * will override matching parameters in the {@link Request}, if they exist. + */ + public Map getParameters() { + return parameters; + } + /** * The {@link HttpAsyncResponseConsumerFactory} used to create one * {@link AsyncResponseConsumer} callback per retry. Controls how the @@ -142,6 +156,12 @@ public String toString() { b.append(headers.get(h).toString()); } } + if (parameters.size() > 0) { + if (comma) b.append(", "); + comma = true; + b.append("parameters="); + b.append(parameters.entrySet().stream().map(e -> e.getKey() + "=" + e.getValue()).collect(Collectors.joining(","))); + } if (httpAsyncResponseConsumerFactory != HttpAsyncResponseConsumerFactory.DEFAULT) { if (comma) b.append(", "); comma = true; @@ -170,6 +190,7 @@ public boolean equals(Object obj) { RequestOptions other = (RequestOptions) obj; return headers.equals(other.headers) + && parameters.equals(other.parameters) && httpAsyncResponseConsumerFactory.equals(other.httpAsyncResponseConsumerFactory) && Objects.equals(warningsHandler, other.warningsHandler); } @@ -179,7 +200,7 @@ public boolean equals(Object obj) { */ @Override public int hashCode() { - return Objects.hash(headers, httpAsyncResponseConsumerFactory, warningsHandler); + return Objects.hash(headers, parameters, httpAsyncResponseConsumerFactory, warningsHandler); } /** @@ -189,17 +210,20 @@ public int hashCode() { */ public static class Builder { private final List
headers; + private final Map parameters; private HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory; private WarningsHandler warningsHandler; private RequestConfig requestConfig; private Builder( List
headers, + Map parameters, HttpAsyncResponseConsumerFactory httpAsyncResponseConsumerFactory, WarningsHandler warningsHandler, RequestConfig requestConfig ) { this.headers = new ArrayList<>(headers); + this.parameters = new HashMap<>(parameters); this.httpAsyncResponseConsumerFactory = httpAsyncResponseConsumerFactory; this.warningsHandler = warningsHandler; this.requestConfig = requestConfig; @@ -226,6 +250,21 @@ public Builder addHeader(String name, String value) { return this; } + /** + * Add the provided query parameter to the request. Any parameters added here + * will override matching parameters in the {@link Request}, if they exist. + * + * @param name the query parameter name + * @param value the query parameter value + * @throws NullPointerException if {@code name} or {@code value} is null. + */ + public Builder addParameter(String name, String value) { + Objects.requireNonNull(name, "query parameter name cannot be null"); + Objects.requireNonNull(value, "query parameter value cannot be null"); + this.parameters.put(name, value); + return this; + } + /** * Set the {@link HttpAsyncResponseConsumerFactory} used to create one * {@link AsyncResponseConsumer} callback per retry. Controls how the diff --git a/client/rest/src/test/java/org/opensearch/client/RequestOptionsTests.java b/client/rest/src/test/java/org/opensearch/client/RequestOptionsTests.java index a7f9a48c73393..06fc92559c2d3 100644 --- a/client/rest/src/test/java/org/opensearch/client/RequestOptionsTests.java +++ b/client/rest/src/test/java/org/opensearch/client/RequestOptionsTests.java @@ -39,12 +39,15 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; @@ -90,6 +93,39 @@ public void testAddHeader() { } } + public void testAddParameter() { + assertThrows( + "query parameter name cannot be null", + NullPointerException.class, + () -> randomBuilder().addParameter(null, randomAsciiLettersOfLengthBetween(3, 10)) + ); + + assertThrows( + "query parameter value cannot be null", + NullPointerException.class, + () -> randomBuilder().addParameter(randomAsciiLettersOfLengthBetween(3, 10), null) + ); + + RequestOptions.Builder builder = RequestOptions.DEFAULT.toBuilder(); + int numParameters = between(0, 5); + Map parameters = new HashMap<>(); + for (int i = 0; i < numParameters; i++) { + String name = randomAsciiAlphanumOfLengthBetween(5, 10); + String value = randomAsciiAlphanumOfLength(3); + parameters.put(name, value); + builder.addParameter(name, value); + } + RequestOptions options = builder.build(); + assertEquals(parameters, options.getParameters()); + + try { + options.getParameters().put(randomAsciiAlphanumOfLengthBetween(5, 10), randomAsciiAlphanumOfLength(3)); + fail("expected failure"); + } catch (UnsupportedOperationException e) { + assertNull(e.getMessage()); + } + } + public void testSetHttpAsyncResponseConsumerFactory() { try { RequestOptions.DEFAULT.toBuilder().setHttpAsyncResponseConsumerFactory(null); @@ -145,6 +181,13 @@ static RequestOptions.Builder randomBuilder() { } } + if (randomBoolean()) { + int queryParamCount = between(1, 5); + for (int i = 0; i < queryParamCount; i++) { + builder.addParameter(randomAsciiAlphanumOfLength(3), randomAsciiAlphanumOfLength(3)); + } + } + if (randomBoolean()) { builder.setHttpAsyncResponseConsumerFactory(new HeapBufferedResponseConsumerFactory(1)); } From f50121a1c9ee2bc360428cec87bd1aab2db1824f Mon Sep 17 00:00:00 2001 From: Sarthak Aggarwal Date: Fri, 31 May 2024 18:38:18 +0530 Subject: [PATCH 29/77] fix flaky #5364 (#13910) Signed-off-by: Sarthak Aggarwal --- .../java/org/opensearch/index/engine/InternalEngineTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java index 54a562642d4ab..08a24a74fdc89 100644 --- a/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java @@ -1872,7 +1872,7 @@ public void testForceMergeWithSoftDeletesRetentionAndRecoverySource() throws Exc try ( Store store = createStore(); InternalEngine engine = createEngine( - config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get) + config(indexSettings, store, createTempDir(), newMergePolicy(random(), false), null, null, globalCheckpoint::get) ) ) { int numDocs = scaledRandomIntBetween(10, 100); From fffd1010e88baf332f2dd4169d69bfcf0a869780 Mon Sep 17 00:00:00 2001 From: Michael Froh Date: Fri, 31 May 2024 18:34:22 +0000 Subject: [PATCH 30/77] Fix negative scores returned from `multi_match` query with `cross_fields` (#13829) Under specific circumstances, when using `cross_fields` scoring on a `multi_match` query, we can end up with negative scores from the inverse document frequency calculation in the BM25 formula. Specifically, the IDF is calculated as: ``` log(1 + (N - n + 0.5) / (n + 0.5)) ``` where `N` is the number of documents containing the field and `n` is the number of documents containing the given term in the field. Obviously, `n` should always be less than or equal to `N`. Unfortunately, `cross_fields` makes up a new value for `n` and tries to use it across all fields. This change finds the (nonzero) value of `N` for each field and uses that as an upper bound for the new value of `n`. Signed-off-by: Michael Froh --------- Signed-off-by: Michael Froh --- CHANGELOG.md | 1 + .../test/search/50_multi_match.yml | 35 ++++++++++++++++++ .../lucene/queries/BlendedTermQuery.java | 8 +++- .../test/rest/yaml/section/Assertion.java | 37 +++++++++++++++++++ .../yaml/section/GreaterThanAssertion.java | 1 + .../section/GreaterThanEqualToAssertion.java | 1 + .../rest/yaml/section/LessThanAssertion.java | 1 + .../section/LessThanOrEqualToAssertion.java | 1 + 8 files changed, 84 insertions(+), 1 deletion(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search/50_multi_match.yml diff --git a/CHANGELOG.md b/CHANGELOG.md index a98d0be56a658..f43e8e40c7338 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -46,6 +46,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix get field mapping API returns 404 error in mixed cluster with multiple versions ([#13624](https://github.com/opensearch-project/OpenSearch/pull/13624)) - Allow clearing `remote_store.compatibility_mode` setting ([#13646](https://github.com/opensearch-project/OpenSearch/pull/13646)) - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) +- Don't return negative scores from `multi_match` query with `cross_fields` type ([#13829](https://github.com/opensearch-project/OpenSearch/pull/13829)) - Pass parent filter to inner hit query ([#13903](https://github.com/opensearch-project/OpenSearch/pull/13903)) ### Security diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/50_multi_match.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/50_multi_match.yml new file mode 100644 index 0000000000000..34acb5985b555 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/50_multi_match.yml @@ -0,0 +1,35 @@ +"Cross fields do not return negative scores": + - skip: + version: " - 2.99.99" + reason: "This fix is in 2.15. Until we do the BWC dance, we need to skip all pre-3.0, though." + - do: + index: + index: test + id: 1 + body: { "color" : "orange red yellow" } + - do: + index: + index: test + id: 2 + body: { "color": "orange red purple", "shape": "red square" } + - do: + index: + index: test + id: 3 + body: { "color" : "orange red yellow purple" } + - do: + indices.refresh: { } + - do: + search: + index: test + body: + query: + multi_match: + query: "red" + type: "cross_fields" + fields: [ "color", "shape^100"] + tie_breaker: 0.1 + explain: true + - match: { hits.total.value: 3 } + - match: { hits.hits.0._id: "2" } + - gt: { hits.hits.2._score: 0.0 } diff --git a/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java b/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java index b47b974b96fed..34e1e210d7137 100644 --- a/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java +++ b/server/src/main/java/org/apache/lucene/queries/BlendedTermQuery.java @@ -120,6 +120,7 @@ protected void blend(final TermStates[] contexts, int maxDoc, IndexReader reader } int max = 0; long minSumTTF = Long.MAX_VALUE; + int[] docCounts = new int[contexts.length]; for (int i = 0; i < contexts.length; i++) { TermStates ctx = contexts[i]; int df = ctx.docFreq(); @@ -133,6 +134,7 @@ protected void blend(final TermStates[] contexts, int maxDoc, IndexReader reader // we need to find out the minimum sumTTF to adjust the statistics // otherwise the statistics don't match minSumTTF = Math.min(minSumTTF, reader.getSumTotalTermFreq(terms[i].field())); + docCounts[i] = reader.getDocCount(terms[i].field()); } } if (maxDoc > minSumTTF) { @@ -175,7 +177,11 @@ protected int compare(int i, int j) { if (prev > current) { actualDf++; } - contexts[i] = ctx = adjustDF(reader.getContext(), ctx, Math.min(maxDoc, actualDf)); + // Per field, we want to guarantee that the adjusted df does not exceed the number of docs with the field. + // That is, in the IDF formula (log(1 + (N - n + 0.5) / (n + 0.5))), we need to make sure that n (the + // adjusted df) is never bigger than N (the number of docs with the field). + int fieldMaxDoc = Math.min(maxDoc, docCounts[i]); + contexts[i] = ctx = adjustDF(reader.getContext(), ctx, Math.min(fieldMaxDoc, actualDf)); prev = current; sumTTF += ctx.totalTermFreq(); } diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/Assertion.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/Assertion.java index b9cbaacdf8873..732d4291ae670 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/Assertion.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/Assertion.java @@ -37,6 +37,8 @@ import java.io.IOException; import java.util.Map; +import static org.junit.Assert.fail; + /** * Base class for executable sections that hold assertions */ @@ -79,6 +81,41 @@ protected final Object getActualValue(ClientYamlTestExecutionContext executionCo return executionContext.response(field); } + static Object convertActualValue(Object actualValue, Object expectedValue) { + if (actualValue == null || expectedValue.getClass().isAssignableFrom(actualValue.getClass())) { + return actualValue; + } + if (actualValue instanceof Number && expectedValue instanceof Number) { + if (expectedValue instanceof Float) { + return Float.parseFloat(actualValue.toString()); + } else if (expectedValue instanceof Double) { + return Double.parseDouble(actualValue.toString()); + } else if (expectedValue instanceof Integer) { + return Integer.parseInt(actualValue.toString()); + } else if (expectedValue instanceof Long) { + return Long.parseLong(actualValue.toString()); + } + } + // Force a class cast exception here, so developers can flesh out the above logic as needed. + try { + expectedValue.getClass().cast(actualValue); + } catch (ClassCastException e) { + fail( + "Type mismatch: Expected value (" + + expectedValue + + ") has type " + + expectedValue.getClass() + + ". " + + "Actual value (" + + actualValue + + ") has type " + + actualValue.getClass() + + "." + ); + } + return actualValue; + } + @Override public XContentLocation getLocation() { return location; diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/GreaterThanAssertion.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/GreaterThanAssertion.java index 4c2e70f37a33c..0d20dc7c326b0 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/GreaterThanAssertion.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/GreaterThanAssertion.java @@ -71,6 +71,7 @@ public GreaterThanAssertion(XContentLocation location, String field, Object expe @Override protected void doAssert(Object actualValue, Object expectedValue) { logger.trace("assert that [{}] is greater than [{}] (field: [{}])", actualValue, expectedValue, getField()); + actualValue = convertActualValue(actualValue, expectedValue); assertThat( "value of [" + getField() + "] is not comparable (got [" + safeClass(actualValue) + "])", actualValue, diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/GreaterThanEqualToAssertion.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/GreaterThanEqualToAssertion.java index 8e929eff44348..a6435c1303489 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/GreaterThanEqualToAssertion.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/GreaterThanEqualToAssertion.java @@ -72,6 +72,7 @@ public GreaterThanEqualToAssertion(XContentLocation location, String field, Obje @Override protected void doAssert(Object actualValue, Object expectedValue) { logger.trace("assert that [{}] is greater than or equal to [{}] (field: [{}])", actualValue, expectedValue, getField()); + actualValue = convertActualValue(actualValue, expectedValue); assertThat( "value of [" + getField() + "] is not comparable (got [" + safeClass(actualValue) + "])", actualValue, diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/LessThanAssertion.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/LessThanAssertion.java index d6e2ae1e23996..acffe03d34439 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/LessThanAssertion.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/LessThanAssertion.java @@ -72,6 +72,7 @@ public LessThanAssertion(XContentLocation location, String field, Object expecte @Override protected void doAssert(Object actualValue, Object expectedValue) { logger.trace("assert that [{}] is less than [{}] (field: [{}])", actualValue, expectedValue, getField()); + actualValue = convertActualValue(actualValue, expectedValue); assertThat( "value of [" + getField() + "] is not comparable (got [" + safeClass(actualValue) + "])", actualValue, diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/LessThanOrEqualToAssertion.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/LessThanOrEqualToAssertion.java index ee46c04496f32..d685d3e46a543 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/LessThanOrEqualToAssertion.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/LessThanOrEqualToAssertion.java @@ -72,6 +72,7 @@ public LessThanOrEqualToAssertion(XContentLocation location, String field, Objec @Override protected void doAssert(Object actualValue, Object expectedValue) { logger.trace("assert that [{}] is less than or equal to [{}] (field: [{}])", actualValue, expectedValue, getField()); + actualValue = convertActualValue(actualValue, expectedValue); assertThat( "value of [" + getField() + "] is not comparable (got [" + safeClass(actualValue) + "])", actualValue, From bb84ca7c08ead307d18c83cbb1c37a6768f8bde5 Mon Sep 17 00:00:00 2001 From: Craig Perkins Date: Fri, 31 May 2024 15:55:20 -0400 Subject: [PATCH 31/77] Revert "Remove deprecated constant META_FIELDS_BEFORE_7DOT8 (#13860)" (#13916) * Revert "Remove deprecated constant META_FIELDS_BEFORE_7DOT8 (#13860)" This reverts commit 983297289325bdd1c832bfe18d105bd3e9727ab5. * Update CHANGELOG-3.0 Signed-off-by: Craig Perkins --------- Signed-off-by: Craig Perkins --- CHANGELOG-3.0.md | 1 - .../java/org/opensearch/index/mapper/MapperService.java | 6 ++++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/CHANGELOG-3.0.md b/CHANGELOG-3.0.md index af788c40ae304..964383078c38d 100644 --- a/CHANGELOG-3.0.md +++ b/CHANGELOG-3.0.md @@ -24,7 +24,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add task completion count in search backpressure stats API ([#10028](https://github.com/opensearch-project/OpenSearch/pull/10028/)) - Deprecate CamelCase `PathHierarchy` tokenizer name in favor to lowercase `path_hierarchy` ([#10894](https://github.com/opensearch-project/OpenSearch/pull/10894)) - Breaking change: Do not request "search_pipelines" metrics by default in NodesInfoRequest ([#12497](https://github.com/opensearch-project/OpenSearch/pull/12497)) -- Remove deprecated constant META_FIELDS_BEFORE_7DOT8 ([#13860](https://github.com/opensearch-project/OpenSearch/pull/13860)) ### Deprecated diff --git a/server/src/main/java/org/opensearch/index/mapper/MapperService.java b/server/src/main/java/org/opensearch/index/mapper/MapperService.java index 83fb743305702..a1f3894c9f14c 100644 --- a/server/src/main/java/org/opensearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/opensearch/index/mapper/MapperService.java @@ -73,6 +73,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -200,6 +201,11 @@ public enum MergeReason { Property.IndexScope, Property.Deprecated ); + // Deprecated set of meta-fields, for checking if a field is meta, use an instance method isMetadataField instead + @Deprecated + public static final Set META_FIELDS_BEFORE_7DOT8 = Collections.unmodifiableSet( + new HashSet<>(Arrays.asList("_id", IgnoredFieldMapper.NAME, "_index", "_routing", "_size", "_timestamp", "_ttl", "_type")) + ); private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(MapperService.class); From 9a876245f01f15cf58d3587f6b651e8181658a2b Mon Sep 17 00:00:00 2001 From: panguixin Date: Sat, 1 Jun 2024 07:56:22 +0800 Subject: [PATCH 32/77] Fix NPE on restore searchable snapshot (#13911) Signed-off-by: panguixin Signed-off-by: Andrew Ross Co-authored-by: Andrew Ross --- CHANGELOG.md | 1 + .../snapshots/SearchableSnapshotIT.java | 51 +++++++++++++++++++ .../put/TransportUpdateSettingsAction.java | 7 +-- .../cluster/block/ClusterBlocks.java | 3 +- .../cluster/metadata/IndexMetadata.java | 7 +++ .../cluster/routing/OperationRouting.java | 5 +- .../cluster/routing/RoutingPool.java | 8 +-- .../decider/DiskThresholdDecider.java | 13 ++++- .../common/settings/ClusterSettings.java | 4 +- .../org/opensearch/index/IndexSettings.java | 6 +-- .../store/remote/filecache/FileCache.java | 16 ------ .../remote/filecache/FileCacheSettings.java | 50 ++++++++++++++++++ .../main/java/org/opensearch/node/Node.java | 4 +- .../opensearch/snapshots/RestoreService.java | 15 +++--- .../decider/DiskThresholdDeciderTests.java | 4 +- .../snapshots/SnapshotResiliencyTests.java | 5 +- .../opensearch/test/OpenSearchTestCase.java | 4 +- 17 files changed, 145 insertions(+), 58 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/store/remote/filecache/FileCacheSettings.java diff --git a/CHANGELOG.md b/CHANGELOG.md index f43e8e40c7338..78d294c8b4c97 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -48,6 +48,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) - Don't return negative scores from `multi_match` query with `cross_fields` type ([#13829](https://github.com/opensearch-project/OpenSearch/pull/13829)) - Pass parent filter to inner hit query ([#13903](https://github.com/opensearch-project/OpenSearch/pull/13903)) +- Fix NPE on restore searchable snapshot ([#13911](https://github.com/opensearch-project/OpenSearch/pull/13911)) ### Security diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java index 90bb2b501764e..b41dd99ff6d40 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java @@ -47,6 +47,7 @@ import org.opensearch.node.Node; import org.opensearch.repositories.fs.FsRepository; import org.hamcrest.MatcherAssert; +import org.junit.After; import java.io.IOException; import java.nio.file.Files; @@ -62,6 +63,10 @@ import static org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.FS; import static org.opensearch.core.common.util.CollectionUtils.iterableAsArrayList; +import static org.opensearch.index.store.remote.filecache.FileCacheSettings.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING; +import static org.opensearch.test.NodeRoles.clusterManagerOnlyNode; +import static org.opensearch.test.NodeRoles.dataNode; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -939,6 +944,52 @@ public void testRelocateSearchableSnapshotIndex() throws Exception { assertSearchableSnapshotIndexDirectoryExistence(searchNode2, index, false); } + public void testCreateSearchableSnapshotWithSpecifiedRemoteDataRatio() throws Exception { + final String snapshotName = "test-snap"; + final String repoName = "test-repo"; + final String indexName1 = "test-idx-1"; + final String restoredIndexName1 = indexName1 + "-copy"; + final String indexName2 = "test-idx-2"; + final String restoredIndexName2 = indexName2 + "-copy"; + final int numReplicasIndex1 = 1; + final int numReplicasIndex2 = 1; + + Settings clusterManagerNodeSettings = clusterManagerOnlyNode(); + internalCluster().startNodes(2, clusterManagerNodeSettings); + Settings dateNodeSettings = dataNode(); + internalCluster().startNodes(2, dateNodeSettings); + createIndexWithDocsAndEnsureGreen(numReplicasIndex1, 100, indexName1); + createIndexWithDocsAndEnsureGreen(numReplicasIndex2, 100, indexName2); + + final Client client = client(); + assertAcked( + client.admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.getKey(), 5)) + ); + + createRepositoryWithSettings(null, repoName); + takeSnapshot(client, snapshotName, repoName, indexName1, indexName2); + + internalCluster().ensureAtLeastNumSearchNodes(Math.max(numReplicasIndex1, numReplicasIndex2) + 1); + restoreSnapshotAndEnsureGreen(client, snapshotName, repoName); + + assertDocCount(restoredIndexName1, 100L); + assertDocCount(restoredIndexName2, 100L); + assertIndexDirectoryDoesNotExist(restoredIndexName1, restoredIndexName2); + } + + @After + public void cleanup() throws Exception { + assertAcked( + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.getKey())) + ); + } + private void assertSearchableSnapshotIndexDirectoryExistence(String nodeName, Index index, boolean exists) throws Exception { final Node node = internalCluster().getInstance(Node.class, nodeName); final ShardId shardId = new ShardId(index, 0); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java index 9265c6ae60678..09cceca52ce23 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/settings/put/TransportUpdateSettingsAction.java @@ -50,7 +50,6 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.index.Index; -import org.opensearch.index.IndexModule; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -59,8 +58,6 @@ import java.util.Set; import java.util.stream.Stream; -import static org.opensearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; - /** * Transport action for updating index settings * @@ -133,9 +130,7 @@ protected ClusterBlockException checkBlock(UpdateSettingsRequest request, Cluste for (Index index : requestIndices) { if (state.blocks().indexBlocked(ClusterBlockLevel.METADATA_WRITE, index.getName())) { allowSearchableSnapshotSettingsUpdate = allowSearchableSnapshotSettingsUpdate - && IndexModule.Type.REMOTE_SNAPSHOT.match( - state.getMetadata().getIndexSafe(index).getSettings().get(INDEX_STORE_TYPE_SETTING.getKey()) - ); + && state.getMetadata().getIndexSafe(index).isRemoteSnapshot(); } } // check if all settings in the request are in the allow list diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java index 304136166d515..02a20b7681ba7 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java @@ -42,7 +42,6 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.rest.RestStatus; -import org.opensearch.index.IndexModule; import java.io.IOException; import java.util.Collections; @@ -399,7 +398,7 @@ public Builder addBlocks(IndexMetadata indexMetadata) { if (IndexMetadata.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING.get(indexMetadata.getSettings())) { addIndexBlock(indexName, IndexMetadata.INDEX_READ_ONLY_ALLOW_DELETE_BLOCK); } - if (IndexModule.Type.REMOTE_SNAPSHOT.match(indexMetadata.getSettings().get(IndexModule.INDEX_STORE_TYPE_SETTING.getKey()))) { + if (indexMetadata.isRemoteSnapshot()) { addIndexBlock(indexName, IndexMetadata.REMOTE_READ_ONLY_ALLOW_DELETE); } return this; 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 8d2851c2e0561..9e7fe23f29872 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -65,6 +65,7 @@ import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.gateway.MetadataStateFormat; +import org.opensearch.index.IndexModule; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.indices.replication.common.ReplicationType; @@ -683,6 +684,7 @@ public static APIBlock readFrom(StreamInput input) throws IOException { private final ActiveShardCount waitForActiveShards; private final Map rolloverInfos; private final boolean isSystem; + private final boolean isRemoteSnapshot; private IndexMetadata( final Index index, @@ -743,6 +745,7 @@ private IndexMetadata( this.waitForActiveShards = waitForActiveShards; this.rolloverInfos = Collections.unmodifiableMap(rolloverInfos); this.isSystem = isSystem; + this.isRemoteSnapshot = IndexModule.Type.REMOTE_SNAPSHOT.match(this.settings); assert numberOfShards * routingFactor == routingNumShards : routingNumShards + " must be a multiple of " + numberOfShards; } @@ -1204,6 +1207,10 @@ public boolean isSystem() { return isSystem; } + public boolean isRemoteSnapshot() { + return isRemoteSnapshot; + } + public static Builder builder(String index) { return new Builder(index); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/OperationRouting.java b/server/src/main/java/org/opensearch/cluster/routing/OperationRouting.java index 6a95c98815698..6158461c7d4e9 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/OperationRouting.java +++ b/server/src/main/java/org/opensearch/cluster/routing/OperationRouting.java @@ -44,7 +44,6 @@ import org.opensearch.common.settings.Settings; import org.opensearch.core.common.Strings; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.IndexModule; import org.opensearch.index.IndexNotFoundException; import org.opensearch.node.ResponseCollectorService; @@ -242,9 +241,7 @@ public GroupShardsIterator searchShards( final Set set = new HashSet<>(shards.size()); for (IndexShardRoutingTable shard : shards) { IndexMetadata indexMetadataForShard = indexMetadata(clusterState, shard.shardId.getIndex().getName()); - if (IndexModule.Type.REMOTE_SNAPSHOT.match( - indexMetadataForShard.getSettings().get(IndexModule.INDEX_STORE_TYPE_SETTING.getKey()) - ) && (preference == null || preference.isEmpty())) { + if (indexMetadataForShard.isRemoteSnapshot() && (preference == null || preference.isEmpty())) { preference = Preference.PRIMARY.type(); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingPool.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingPool.java index a4ff237460e28..db10ad61c7d6d 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingPool.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingPool.java @@ -11,8 +11,6 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.allocation.RoutingAllocation; -import org.opensearch.common.settings.Settings; -import org.opensearch.index.IndexModule; /** * {@link RoutingPool} defines the different node types based on the assigned capabilities. The methods @@ -60,10 +58,6 @@ public static RoutingPool getShardPool(ShardRouting shard, RoutingAllocation all * @return {@link RoutingPool} for the given index. */ public static RoutingPool getIndexPool(IndexMetadata indexMetadata) { - Settings indexSettings = indexMetadata.getSettings(); - if (IndexModule.Type.REMOTE_SNAPSHOT.match(indexSettings.get(IndexModule.INDEX_STORE_TYPE_SETTING.getKey()))) { - return REMOTE_CAPABLE; - } - return LOCAL_ONLY; + return indexMetadata.isRemoteSnapshot() ? REMOTE_CAPABLE : LOCAL_ONLY; } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDecider.java index 2c7df6b81e676..efa5115939d3c 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDecider.java @@ -54,6 +54,7 @@ import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.store.remote.filecache.FileCacheSettings; import org.opensearch.index.store.remote.filecache.FileCacheStats; import org.opensearch.snapshots.SnapshotShardSizeInfo; @@ -68,7 +69,6 @@ import static org.opensearch.cluster.routing.RoutingPool.getShardPool; import static org.opensearch.cluster.routing.allocation.DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING; import static org.opensearch.cluster.routing.allocation.DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING; -import static org.opensearch.index.store.remote.filecache.FileCache.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING; /** * The {@link DiskThresholdDecider} checks that the node a shard is potentially @@ -109,11 +109,13 @@ public class DiskThresholdDecider extends AllocationDecider { private final DiskThresholdSettings diskThresholdSettings; private final boolean enableForSingleDataNode; + private final FileCacheSettings fileCacheSettings; public DiskThresholdDecider(Settings settings, ClusterSettings clusterSettings) { this.diskThresholdSettings = new DiskThresholdSettings(settings, clusterSettings); assert Version.CURRENT.major < 9 : "remove enable_for_single_data_node in 9"; this.enableForSingleDataNode = ENABLE_FOR_SINGLE_DATA_NODE.get(settings); + this.fileCacheSettings = new FileCacheSettings(settings, clusterSettings); } /** @@ -179,6 +181,12 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing The following block enables allocation for remote shards within safeguard limits of the filecache. */ if (REMOTE_CAPABLE.equals(getNodePool(node)) && REMOTE_CAPABLE.equals(getShardPool(shardRouting, allocation))) { + final double dataToFileCacheSizeRatio = fileCacheSettings.getRemoteDataRatio(); + // we don't need to check the ratio + if (dataToFileCacheSizeRatio <= 0.1f) { + return Decision.YES; + } + final List remoteShardsOnNode = StreamSupport.stream(node.spliterator(), false) .filter(shard -> shard.primary() && REMOTE_CAPABLE.equals(getShardPool(shard, allocation))) .collect(Collectors.toList()); @@ -199,7 +207,6 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing final FileCacheStats fileCacheStats = clusterInfo.getNodeFileCacheStats().getOrDefault(node.nodeId(), null); final long nodeCacheSize = fileCacheStats != null ? fileCacheStats.getTotal().getBytes() : 0; final long totalNodeRemoteShardSize = currentNodeRemoteShardSize + shardSize; - final double dataToFileCacheSizeRatio = DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.get(allocation.metadata().settings()); if (dataToFileCacheSizeRatio > 0.0f && totalNodeRemoteShardSize > dataToFileCacheSizeRatio * nodeCacheSize) { return allocation.decision( Decision.NO, @@ -208,6 +215,8 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing ); } return Decision.YES; + } else if (REMOTE_CAPABLE.equals(getShardPool(shardRouting, allocation))) { + return Decision.NO; } Map usages = clusterInfo.getNodeMostAvailableDiskUsages(); 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 d57ef7e780602..297fc98764d07 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -116,7 +116,7 @@ import org.opensearch.index.ShardIndexingPressureStore; import org.opensearch.index.remote.RemoteStorePressureSettings; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; -import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.filecache.FileCacheSettings; import org.opensearch.indices.IndexingMemoryController; import org.opensearch.indices.IndicesQueryCache; import org.opensearch.indices.IndicesRequestCache; @@ -689,7 +689,7 @@ public void apply(Settings value, Settings current, Settings previous) { // Settings related to Searchable Snapshots Node.NODE_SEARCH_CACHE_SIZE_SETTING, - FileCache.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING, + FileCacheSettings.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING, // Settings related to Remote Refresh Segment Pressure RemoteStorePressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED, diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 2b0a62c18d5a7..6c0ab2f6b0153 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -728,7 +728,6 @@ public static IndexMergePolicy fromString(String text) { private volatile TimeValue remoteTranslogUploadBufferInterval; private final String remoteStoreTranslogRepository; private final String remoteStoreRepository; - private final boolean isRemoteSnapshot; private int remoteTranslogKeepExtraGen; private Version extendedCompatibilitySnapshotVersion; @@ -919,9 +918,8 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti remoteTranslogUploadBufferInterval = INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(settings); remoteStoreRepository = settings.get(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY); this.remoteTranslogKeepExtraGen = INDEX_REMOTE_TRANSLOG_KEEP_EXTRA_GEN_SETTING.get(settings); - isRemoteSnapshot = IndexModule.Type.REMOTE_SNAPSHOT.match(this.settings); - if (isRemoteSnapshot && FeatureFlags.isEnabled(SEARCHABLE_SNAPSHOT_EXTENDED_COMPATIBILITY)) { + if (isRemoteSnapshot() && FeatureFlags.isEnabled(SEARCHABLE_SNAPSHOT_EXTENDED_COMPATIBILITY)) { extendedCompatibilitySnapshotVersion = SEARCHABLE_SNAPSHOT_EXTENDED_COMPATIBILITY_MINIMUM_VERSION; } else { extendedCompatibilitySnapshotVersion = Version.CURRENT.minimumIndexCompatibilityVersion(); @@ -1278,7 +1276,7 @@ public String getRemoteStoreTranslogRepository() { * Returns true if this is remote/searchable snapshot */ public boolean isRemoteSnapshot() { - return isRemoteSnapshot; + return indexMetadata.isRemoteSnapshot(); } /** diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCache.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCache.java index 2029b461674c7..e61e5ecd4084a 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCache.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCache.java @@ -10,7 +10,6 @@ import org.apache.lucene.store.IndexInput; import org.opensearch.common.annotation.PublicApi; -import org.opensearch.common.settings.Setting; import org.opensearch.core.common.breaker.CircuitBreaker; import org.opensearch.core.common.breaker.CircuitBreakingException; import org.opensearch.index.store.remote.utils.cache.CacheUsage; @@ -52,21 +51,6 @@ public class FileCache implements RefCountedCache { private final CircuitBreaker circuitBreaker; - /** - * Defines a limit of how much total remote data can be referenced as a ratio of the size of the disk reserved for - * the file cache. For example, if 100GB disk space is configured for use as a file cache and the - * remote_data_ratio of 5 is defined, then a total of 500GB of remote data can be loaded as searchable snapshots. - * This is designed to be a safeguard to prevent oversubscribing a cluster. - * Specify a value of zero for no limit, which is the default for compatibility reasons. - */ - public static final Setting DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING = Setting.doubleSetting( - "cluster.filecache.remote_data_ratio", - 0.0, - 0.0, - Setting.Property.NodeScope, - Setting.Property.Dynamic - ); - public FileCache(SegmentedCache cache, CircuitBreaker circuitBreaker) { this.theCache = cache; this.circuitBreaker = circuitBreaker; diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCacheSettings.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCacheSettings.java new file mode 100644 index 0000000000000..76086be932ecb --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCacheSettings.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.index.store.remote.filecache; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; + +/** + * Settings relate to file cache + * + * @opensearch.internal + */ +public class FileCacheSettings { + /** + * Defines a limit of how much total remote data can be referenced as a ratio of the size of the disk reserved for + * the file cache. For example, if 100GB disk space is configured for use as a file cache and the + * remote_data_ratio of 5 is defined, then a total of 500GB of remote data can be loaded as searchable snapshots. + * This is designed to be a safeguard to prevent oversubscribing a cluster. + * Specify a value of zero for no limit, which is the default for compatibility reasons. + */ + public static final Setting DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING = Setting.doubleSetting( + "cluster.filecache.remote_data_ratio", + 0.0, + 0.0, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + private volatile double remoteDataRatio; + + public FileCacheSettings(Settings settings, ClusterSettings clusterSettings) { + setRemoteDataRatio(DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.get(settings)); + clusterSettings.addSettingsUpdateConsumer(DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING, this::setRemoteDataRatio); + } + + public void setRemoteDataRatio(double remoteDataRatio) { + this.remoteDataRatio = remoteDataRatio; + } + + public double getRemoteDataRatio() { + return remoteDataRatio; + } +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 04bd31e6a5809..49545fa8a0c8b 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -155,6 +155,7 @@ import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.filecache.FileCacheCleaner; import org.opensearch.index.store.remote.filecache.FileCacheFactory; +import org.opensearch.index.store.remote.filecache.FileCacheSettings; import org.opensearch.indices.IndicesModule; import org.opensearch.indices.IndicesService; import org.opensearch.indices.RemoteStoreSettings; @@ -1159,7 +1160,8 @@ protected Node( metadataIndexUpgradeService, shardLimitValidator, indicesService, - clusterInfoService::getClusterInfo + clusterInfoService::getClusterInfo, + new FileCacheSettings(settings, clusterService.getClusterSettings())::getRemoteDataRatio ); RemoteStoreRestoreService remoteStoreRestoreService = new RemoteStoreRestoreService( diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index 93aac68eb898c..4cc8049b2b06b 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -129,7 +129,6 @@ import static org.opensearch.common.util.set.Sets.newHashSet; import static org.opensearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; import static org.opensearch.index.store.remote.directory.RemoteSnapshotDirectory.SEARCHABLE_SNAPSHOT_EXTENDED_COMPATIBILITY_MINIMUM_VERSION; -import static org.opensearch.index.store.remote.filecache.FileCache.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING; import static org.opensearch.node.Node.NODE_SEARCH_CACHE_SIZE_SETTING; /** @@ -204,6 +203,8 @@ public class RestoreService implements ClusterStateApplier { private final ClusterManagerTaskThrottler.ThrottlingKey restoreSnapshotTaskKey; + private final Supplier dataToFileCacheSizeRatioSupplier; + private static final CleanRestoreStateTaskExecutor cleanRestoreStateTaskExecutor = new CleanRestoreStateTaskExecutor(); public RestoreService( @@ -214,7 +215,8 @@ public RestoreService( MetadataIndexUpgradeService metadataIndexUpgradeService, ShardLimitValidator shardLimitValidator, IndicesService indicesService, - Supplier clusterInfoSupplier + Supplier clusterInfoSupplier, + Supplier dataToFileCacheSizeRatioSupplier ) { this.clusterService = clusterService; this.repositoriesService = repositoriesService; @@ -228,6 +230,7 @@ public RestoreService( this.shardLimitValidator = shardLimitValidator; this.indicesService = indicesService; this.clusterInfoSupplier = clusterInfoSupplier; + this.dataToFileCacheSizeRatioSupplier = dataToFileCacheSizeRatioSupplier; // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction. restoreSnapshotTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.RESTORE_SNAPSHOT_KEY, true); @@ -399,9 +402,7 @@ public ClusterState execute(ClusterState currentState) { if (isRemoteSnapshot) { snapshotIndexMetadata = addSnapshotToIndexSettings(snapshotIndexMetadata, snapshot, snapshotIndexId); } - final boolean isSearchableSnapshot = IndexModule.Type.REMOTE_SNAPSHOT.match( - snapshotIndexMetadata.getSettings().get(IndexModule.INDEX_STORE_TYPE_SETTING.getKey()) - ); + final boolean isSearchableSnapshot = snapshotIndexMetadata.isRemoteSnapshot(); final boolean isRemoteStoreShallowCopy = Boolean.TRUE.equals( snapshotInfo.isRemoteStoreIndexShallowCopyEnabled() ) && metadata.index(index).getSettings().getAsBoolean(SETTING_REMOTE_STORE_ENABLED, false); @@ -855,7 +856,7 @@ private IndexMetadata updateIndexSettings( private void validateSearchableSnapshotRestorable(long totalRestorableRemoteIndexesSize) { ClusterInfo clusterInfo = clusterInfoSupplier.get(); - double remoteDataToFileCacheRatio = DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.get(clusterService.getSettings()); + final double remoteDataToFileCacheRatio = dataToFileCacheSizeRatioSupplier.get(); Map nodeFileCacheStats = clusterInfo.getNodeFileCacheStats(); if (nodeFileCacheStats.isEmpty() || remoteDataToFileCacheRatio <= 0.01f) { return; @@ -869,7 +870,7 @@ private void validateSearchableSnapshotRestorable(long totalRestorableRemoteInde .sum(); Predicate isRemoteSnapshotShard = shardRouting -> shardRouting.primary() - && indicesService.indexService(shardRouting.index()).getIndexSettings().isRemoteSnapshot(); + && clusterService.state().getMetadata().getIndexSafe(shardRouting.index()).isRemoteSnapshot(); ShardsIterator shardsIterator = clusterService.state() .routingTable() diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index bde8a45359814..652633e689b93 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -69,7 +69,6 @@ import org.opensearch.common.settings.Settings; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.filecache.FileCacheStats; import org.opensearch.repositories.IndexId; import org.opensearch.snapshots.EmptySnapshotsInfoService; @@ -95,6 +94,7 @@ import static org.opensearch.cluster.routing.ShardRoutingState.STARTED; import static org.opensearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; +import static org.opensearch.index.store.remote.filecache.FileCacheSettings.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -381,6 +381,7 @@ public void testFileCacheRemoteShardsDecisions() { .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%") .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%") + .put(DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.getKey(), 5) .build(); // We have an index with 2 primary shards each taking 40 bytes. Each node has 100 bytes available @@ -406,7 +407,6 @@ public void testFileCacheRemoteShardsDecisions() { DiskThresholdDecider diskThresholdDecider = makeDecider(diskSettings); Metadata metadata = Metadata.builder() .put(IndexMetadata.builder("test").settings(remoteIndexSettings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0)) - .persistentSettings(Settings.builder().put(FileCache.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.getKey(), 5).build()) .build(); RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index("test")).build(); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 460aaa08a224d..5b39880930984 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -191,7 +191,6 @@ import org.opensearch.index.seqno.RetentionLeaseSyncer; import org.opensearch.index.shard.PrimaryReplicaSyncer; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; -import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.filecache.FileCacheStats; import org.opensearch.indices.DefaultRemoteStoreSettings; import org.opensearch.indices.IndicesModule; @@ -1681,7 +1680,6 @@ private Environment createEnvironment(String nodeName) { ClusterBootstrapService.INITIAL_CLUSTER_MANAGER_NODES_SETTING.getKey(), ClusterBootstrapService.INITIAL_CLUSTER_MANAGER_NODES_SETTING.get(Settings.EMPTY) ) - .put(FileCache.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.getKey(), 5) .put(MappingUpdatedAction.INDICES_MAX_IN_FLIGHT_UPDATES_SETTING.getKey(), 1000) // o.w. some tests might block .build() ); @@ -2252,7 +2250,8 @@ public void onFailure(final Exception e) { ), shardLimitValidator, indicesService, - clusterInfoService::getClusterInfo + clusterInfoService::getClusterInfo, + () -> 5.0 ); actions.put( PutMappingAction.INSTANCE, diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java index 5a3f3b5a07a8d..5ee65e7ea1a1c 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java @@ -125,7 +125,6 @@ import org.opensearch.index.analysis.TokenizerFactory; import org.opensearch.index.remote.RemoteStoreEnums; import org.opensearch.index.remote.RemoteStorePathStrategy; -import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.indices.analysis.AnalysisModule; import org.opensearch.monitor.jvm.JvmInfo; import org.opensearch.plugins.AnalysisPlugin; @@ -188,6 +187,7 @@ import static java.util.Collections.emptyMap; import static org.opensearch.core.common.util.CollectionUtils.arrayAsArrayList; +import static org.opensearch.index.store.remote.filecache.FileCacheSettings.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; @@ -1278,7 +1278,7 @@ public static Settings.Builder settings(Version version) { public static Settings.Builder remoteIndexSettings(Version version) { Settings.Builder builder = Settings.builder() - .put(FileCache.DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.getKey(), 5) + .put(DATA_TO_FILE_CACHE_SIZE_RATIO_SETTING.getKey(), 5) .put(IndexMetadata.SETTING_VERSION_CREATED, version) .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), IndexModule.Type.REMOTE_SNAPSHOT.getSettingsKey()); return builder; From 771f4ec2e4daebf04f9d94e6d8268a6b57b1b857 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Fri, 31 May 2024 17:17:52 -0700 Subject: [PATCH 33/77] Update the developer guide to add Gradle Check Metrics Dashboard details (#13919) Signed-off-by: Prudhvi Godithi --- DEVELOPER_GUIDE.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/DEVELOPER_GUIDE.md b/DEVELOPER_GUIDE.md index 92ef71b92da7e..bc11e7335af49 100644 --- a/DEVELOPER_GUIDE.md +++ b/DEVELOPER_GUIDE.md @@ -62,6 +62,7 @@ - [LineLint](#linelint) - [Lucene Snapshots](#lucene-snapshots) - [Flaky Tests](#flaky-tests) + - [Gradle Check Metrics Dashboard](#gradle-check-metrics-dashboard) # Developer Guide @@ -660,3 +661,7 @@ If you encounter a build/test failure in CI that is unrelated to the change in y 4. If an existing issue is found, paste a link to the known issue in a comment to your PR. 5. If no existing issue is found, open one. 6. Retry CI via the GitHub UX or by pushing an update to your PR. + +### Gradle Check Metrics Dashboard + +To get the comprehensive insights and analysis of the Gradle Check test failures, visit the [OpenSearch Gradle Check Metrics Dashboard](https://metrics.opensearch.org/_dashboards/app/dashboards#/view/e5e64d40-ed31-11ee-be99-69d1dbc75083). This dashboard is part of the [OpenSearch Metrics Project](https://github.com/opensearch-project/opensearch-metrics) initiative. The dashboard contains multiple data points that can help investigate and resolve flaky failures. Additionally, this dashboard can be used to drill down, slice, and dice the data using multiple supported filters, which further aids in troubleshooting and resolving issues. From 896b12759043ec993097694fc6409bdbf9cdcab1 Mon Sep 17 00:00:00 2001 From: Shubh Sahu Date: Mon, 3 Jun 2024 15:48:37 +0530 Subject: [PATCH 34/77] [Remote Store] Adding ITs to ensure RemoteIndexPath file gets created after migration for each index. (#13367) Signed-off-by: Shubh Sahu --- .../MigrationBaseTestCase.java | 4 +- .../RemoteMigrationIndexMetadataUpdateIT.java | 86 +++++++++++++++++++ 2 files changed, 89 insertions(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java index 611dfc2756b29..0493bcf800c97 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java @@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicLong; import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.opensearch.repositories.fs.ReloadableFsRepository.REPOSITORIES_FAILRATE_SETTING; @@ -78,10 +79,11 @@ protected Settings nodeSettings(int nodeOrdinal) { .put(super.nodeSettings(nodeOrdinal)) .put(extraSettings) .put(remoteStoreClusterSettings(REPOSITORY_NAME, segmentRepoPath, REPOSITORY_2_NAME, translogRepoPath)) + .put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) .build(); } else { logger.info("Adding docrep node"); - return Settings.builder().put(super.nodeSettings(nodeOrdinal)).build(); + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java index 53f4ef3fe281f..c72b6851c1125 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java @@ -15,14 +15,21 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.opensearch.common.settings.Settings; +import org.opensearch.core.util.FileSystemUtils; +import org.opensearch.index.remote.RemoteIndexPath; +import org.opensearch.index.remote.RemoteIndexPathUploader; +import org.opensearch.index.remote.RemoteStoreEnums; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; +import java.nio.file.Path; +import java.util.Arrays; import java.util.List; import java.util.function.Function; import java.util.stream.Collectors; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) @@ -454,6 +461,85 @@ public void testRemotePathMetadataAddedWithFirstPrimaryMovingToRemote() throws E assertRemoteProperties(indexName); } + /** + * Scenario: + * creates an index on docrep node with non-remote cluster-manager. + * make the cluster mixed, add remote cluster-manager and data nodes. + *

+ * exclude docrep nodes, assert that remote index path file exists + * when shards start relocating to the remote nodes. + */ + public void testRemoteIndexPathFileExistsAfterMigration() throws Exception { + String docrepClusterManager = internalCluster().startClusterManagerOnlyNode(); + + logger.info("---> Starting 2 docrep nodes"); + addRemote = false; + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "docrep").build()); + internalCluster().validateClusterFormed(); + + logger.info("---> Creating index with 1 primary and 1 replica"); + String indexName = "migration-index"; + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAndAssertDocrepProperties(indexName, oneReplica); + + String indexUUID = internalCluster().client() + .admin() + .indices() + .prepareGetSettings(indexName) + .get() + .getSetting(indexName, IndexMetadata.SETTING_INDEX_UUID); + + logger.info("---> Starting indexing in parallel"); + AsyncIndexingService indexingService = new AsyncIndexingService(indexName); + indexingService.startIndexing(); + + logger.info("---> Adding 2 remote enabled nodes to the cluster & cluster manager"); + initDocRepToRemoteMigration(); + addRemote = true; + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNodes(2, Settings.builder().put("node.attr._type", "remote").build()); + internalCluster().validateClusterFormed(); + + assertTrue( + internalCluster().client() + .admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX) + ) + .get() + .isAcknowledged() + ); + + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(docrepClusterManager)); + internalCluster().validateClusterFormed(); + + logger.info("---> Excluding docrep nodes from allocation"); + excludeNodeSet("type", "docrep"); + + waitForRelocation(); + waitNoPendingTasksOnAll(); + indexingService.stopIndexing(); + + // validate remote index path file exists + logger.info("---> Asserting remote index path file exists"); + String fileNamePrefix = String.join(RemoteIndexPathUploader.DELIMITER, indexUUID, "7", RemoteIndexPath.DEFAULT_VERSION); + + assertTrue(FileSystemUtils.exists(translogRepoPath.resolve(RemoteIndexPath.DIR))); + Path[] files = FileSystemUtils.files(translogRepoPath.resolve(RemoteIndexPath.DIR)); + assertEquals(1, files.length); + assertTrue(Arrays.stream(files).anyMatch(file -> file.toString().contains(fileNamePrefix))); + + assertTrue(FileSystemUtils.exists(segmentRepoPath.resolve(RemoteIndexPath.DIR))); + files = FileSystemUtils.files(segmentRepoPath.resolve(RemoteIndexPath.DIR)); + assertEquals(1, files.length); + assertTrue(Arrays.stream(files).anyMatch(file -> file.toString().contains(fileNamePrefix))); + } + private void createIndexAndAssertDocrepProperties(String index, Settings settings) { createIndexAssertHealthAndDocrepProperties(index, settings, this::ensureGreen); } From bf4367877eab27dff05a74d683d14e820130172d Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Mon, 3 Jun 2024 05:51:24 -0700 Subject: [PATCH 35/77] Add comment to XContentBuilder.toString() (#13809) * Add comment to XContentBuilder.toString Signed-off-by: Liyun Xiu * Fix typo Signed-off-by: Liyun Xiu --------- Signed-off-by: Liyun Xiu --- .../java/org/opensearch/core/xcontent/XContentBuilder.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java b/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java index 976f353100c55..552945d085884 100644 --- a/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java +++ b/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java @@ -157,6 +157,9 @@ public static XContentBuilder builder(XContent xContent, Set includes, S /** * Returns a string representation of the builder (only applicable for text based xcontent). + * Note: explicitly or implicitly (from debugger) calling toString() could cause XContentBuilder + * to close which is a side effect done by @see BytesReference#bytes(). + * Trying to write more contents after toString() will cause NPE. Use it with caution. */ @Override public String toString() { From 277456164761a51a5207252accbe6ec934ed8479 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 3 Jun 2024 12:50:02 -0400 Subject: [PATCH 36/77] Bump com.netflix.nebula.ospackage-base from 11.9.0 to 11.9.1 in /distribution/packages (#13933) * Bump com.netflix.nebula.ospackage-base in /distribution/packages Bumps com.netflix.nebula.ospackage-base from 11.9.0 to 11.9.1. --- updated-dependencies: - dependency-name: com.netflix.nebula.ospackage-base dependency-type: direct:production update-type: version-update:semver-patch ... 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 + distribution/packages/build.gradle | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 78d294c8b4c97..788b09928a8bc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -31,6 +31,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `reactor-netty` from 1.1.17 to 1.1.19 ([#13825](https://github.com/opensearch-project/OpenSearch/pull/13825)) - Bump `commons-cli:commons-cli` from 1.7.0 to 1.8.0 ([#13840](https://github.com/opensearch-project/OpenSearch/pull/13840)) - Bump `org.apache.xmlbeans:xmlbeans` from 5.2.0 to 5.2.1 ([#13839](https://github.com/opensearch-project/OpenSearch/pull/13839)) +- Bump `com.netflix.nebula.ospackage-base` from 11.9.0 to 11.9.1 ([#13933](https://github.com/opensearch-project/OpenSearch/pull/13933)) ### Changed - Add ability for Boolean and date field queries to run when only doc_values are enabled ([#11650](https://github.com/opensearch-project/OpenSearch/pull/11650)) diff --git a/distribution/packages/build.gradle b/distribution/packages/build.gradle index fbd13f03af814..211b3bd55da60 100644 --- a/distribution/packages/build.gradle +++ b/distribution/packages/build.gradle @@ -63,7 +63,7 @@ import java.util.regex.Pattern */ plugins { - id "com.netflix.nebula.ospackage-base" version "11.9.0" + id "com.netflix.nebula.ospackage-base" version "11.9.1" } void addProcessFilesTask(String type, boolean jdk) { From 194005ebcaf79cf4f0f9fecd96d4b9e90ca9d1c0 Mon Sep 17 00:00:00 2001 From: Rohit Ashiwal Date: Mon, 3 Jun 2024 23:01:52 +0530 Subject: [PATCH 37/77] Painless: ensure type "UnmodifiableMap" for params (#13885) Signed-off-by: Rohit Ashiwal --- CHANGELOG.md | 1 + .../painless/WhenThingsGoWrongTests.java | 3 ++ .../test/painless/17_update_error.yml | 47 +++++++++++++++++++ .../java/org/opensearch/script/Script.java | 2 +- .../org/opensearch/script/UpdateScript.java | 3 +- 5 files changed, 54 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 788b09928a8bc..3554565e737e8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -48,6 +48,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Allow clearing `remote_store.compatibility_mode` setting ([#13646](https://github.com/opensearch-project/OpenSearch/pull/13646)) - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) - Don't return negative scores from `multi_match` query with `cross_fields` type ([#13829](https://github.com/opensearch-project/OpenSearch/pull/13829)) +- Painless: ensure type "UnmodifiableMap" for params ([#13885](https://github.com/opensearch-project/OpenSearch/pull/13885)) - Pass parent filter to inner hit query ([#13903](https://github.com/opensearch-project/OpenSearch/pull/13903)) - Fix NPE on restore searchable snapshot ([#13911](https://github.com/opensearch-project/OpenSearch/pull/13911)) diff --git a/modules/lang-painless/src/test/java/org/opensearch/painless/WhenThingsGoWrongTests.java b/modules/lang-painless/src/test/java/org/opensearch/painless/WhenThingsGoWrongTests.java index 0d498e16154c8..3d48e96117a1c 100644 --- a/modules/lang-painless/src/test/java/org/opensearch/painless/WhenThingsGoWrongTests.java +++ b/modules/lang-painless/src/test/java/org/opensearch/painless/WhenThingsGoWrongTests.java @@ -354,6 +354,9 @@ public void testInvalidAssignment() { assertEquals(iae.getMessage(), "invalid assignment: cannot assign a value to addition operation [+]"); iae = expectScriptThrows(IllegalArgumentException.class, () -> exec("Double.x() = 1;")); assertEquals(iae.getMessage(), "invalid assignment: cannot assign a value to method call [x/0]"); + + expectScriptThrows(UnsupportedOperationException.class, () -> exec("params['modifyingParamsMap'] = 2;")); + expectScriptThrows(UnsupportedOperationException.class, () -> exec("params.modifyingParamsMap = 2;")); } public void testCannotResolveSymbol() { diff --git a/modules/lang-painless/src/yamlRestTest/resources/rest-api-spec/test/painless/17_update_error.yml b/modules/lang-painless/src/yamlRestTest/resources/rest-api-spec/test/painless/17_update_error.yml index 3d6db1b781caf..fdbc6de37e3ea 100644 --- a/modules/lang-painless/src/yamlRestTest/resources/rest-api-spec/test/painless/17_update_error.yml +++ b/modules/lang-painless/src/yamlRestTest/resources/rest-api-spec/test/painless/17_update_error.yml @@ -13,3 +13,50 @@ - match: { error.root_cause.0.position.offset: 13 } - match: { error.root_cause.0.position.start: 0 } - match: { error.root_cause.0.position.end: 38 } + +--- +"Test modifying params map from script leads to exception": + - skip: + features: "node_selector" + + - do: + put_script: + id: "except" + body: {"script": {"lang": "painless", "source": "params.that = 3"}} + + - do: + indices.create: + index: "test" + body: + settings: + index: + number_of_shards: 1 + number_of_replicas: 0 + mappings: + properties: + this: + type: "integer" + that: + type: "integer" + + - do: + index: + index: "test" + id: 1 + body: {"this": 1, "that": 2} + + - do: + catch: /unsupported_operation_exception/ + node_selector: + version: "2.15.0 - " + update: + index: "test" + id: 1 + body: + script: + id: "except" + params: {"this": 2} + + - match: { error.caused_by.position.offset: 6 } + - match: { error.caused_by.position.start: 0 } + - match: { error.caused_by.position.end: 15 } diff --git a/server/src/main/java/org/opensearch/script/Script.java b/server/src/main/java/org/opensearch/script/Script.java index 9e74314c281cd..f18bd992cb00d 100644 --- a/server/src/main/java/org/opensearch/script/Script.java +++ b/server/src/main/java/org/opensearch/script/Script.java @@ -589,7 +589,7 @@ public Script(StreamInput in) throws IOException { @SuppressWarnings("unchecked") Map options = (Map) (Map) in.readMap(); this.options = options; - this.params = in.readMap(); + this.params = Collections.unmodifiableMap(in.readMap()); } @Override diff --git a/server/src/main/java/org/opensearch/script/UpdateScript.java b/server/src/main/java/org/opensearch/script/UpdateScript.java index 86697e9ae550e..f6355fe24817b 100644 --- a/server/src/main/java/org/opensearch/script/UpdateScript.java +++ b/server/src/main/java/org/opensearch/script/UpdateScript.java @@ -32,6 +32,7 @@ package org.opensearch.script; +import java.util.Collections; import java.util.Map; /** @@ -53,7 +54,7 @@ public abstract class UpdateScript { private final Map ctx; public UpdateScript(Map params, Map ctx) { - this.params = params; + this.params = Collections.unmodifiableMap(params); this.ctx = ctx; } From bdf53b127ed695cd06e725181b8fcbd35008069f Mon Sep 17 00:00:00 2001 From: Shivansh Arora <31575408+shiv0408@users.noreply.github.com> Date: Mon, 3 Jun 2024 23:08:20 +0530 Subject: [PATCH 38/77] Bump japicmp plugin version (#13940) Signed-off-by: Shivansh Arora --- server/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/build.gradle b/server/build.gradle index 9714f13ec67d6..15301e68fca3d 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -36,7 +36,7 @@ plugins { id('opensearch.publish') id('opensearch.internal-cluster-test') id('opensearch.optional-dependencies') - id('me.champeau.gradle.japicmp') version '0.4.2' + id('me.champeau.gradle.japicmp') version '0.4.3' } publishing { From 6c1896bd11e18efeae51dc9ec4b8783987e5d1f8 Mon Sep 17 00:00:00 2001 From: Rishabh Maurya Date: Mon, 3 Jun 2024 13:42:15 -0700 Subject: [PATCH 39/77] [Derived Field] Dynamic FieldType inference based on random sampling of documents (#13592) --------- Signed-off-by: Rishabh Maurya --- .../index/mapper/FieldTypeInference.java | 181 +++++++++++++++ .../index/mapper/FieldTypeInferenceTests.java | 210 ++++++++++++++++++ 2 files changed, 391 insertions(+) create mode 100644 server/src/main/java/org/opensearch/index/mapper/FieldTypeInference.java create mode 100644 server/src/test/java/org/opensearch/index/mapper/FieldTypeInferenceTests.java diff --git a/server/src/main/java/org/opensearch/index/mapper/FieldTypeInference.java b/server/src/main/java/org/opensearch/index/mapper/FieldTypeInference.java new file mode 100644 index 0000000000000..713bdc4e691cd --- /dev/null +++ b/server/src/main/java/org/opensearch/index/mapper/FieldTypeInference.java @@ -0,0 +1,181 @@ +/* + * 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.mapper; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.ReaderUtil; +import org.opensearch.common.Randomness; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.search.lookup.SourceLookup; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.TreeSet; + +/** + * This class performs type inference by analyzing the _source documents. It uses a random sample of documents to infer the field type, similar to dynamic mapping type guessing logic. + * Unlike guessing based on the first document, where field could be missing, this method generates a random sample to make a more accurate inference. + * This approach is especially useful for handling missing fields, which is common in nested fields within derived fields of object types. + * + *

The sample size should be chosen carefully to ensure a high probability of selecting at least one document where the field is present. + * However, it's essential to strike a balance because a large sample size can lead to performance issues since each sample document's _source field is loaded and examined until the field is found. + * + *

Determining the sample size ({@code S}) is akin to deciding how many balls to draw from a bin, ensuring a high probability ({@code >=P}) of drawing at least one green ball (documents with the field) from a mixture of {@code R } red balls (documents without the field) and {@code G } green balls: + *

{@code
+ * P >= 1 - C(R, S) / C(R + G, S)
+ * }
+ * Here, {@code C()} represents the binomial coefficient. + * For a high confidence level, we aim for {@code P >= 0.95 }. For example, with {@code 10^7 } documents where the field is present in {@code 2% } of them, the sample size {@code S } should be around 149 to achieve a probability of {@code 0.95}. + */ +public class FieldTypeInference { + private final IndexReader indexReader; + private final String indexName; + private final MapperService mapperService; + // TODO expose using a index setting + private int sampleSize; + private static final int DEFAULT_SAMPLE_SIZE = 150; + private static final int MAX_SAMPLE_SIZE_ALLOWED = 1000; + + public FieldTypeInference(String indexName, MapperService mapperService, IndexReader indexReader) { + this.indexName = indexName; + this.mapperService = mapperService; + this.indexReader = indexReader; + this.sampleSize = DEFAULT_SAMPLE_SIZE; + } + + public void setSampleSize(int sampleSize) { + if (sampleSize > MAX_SAMPLE_SIZE_ALLOWED) { + throw new IllegalArgumentException("sample_size should be less than " + MAX_SAMPLE_SIZE_ALLOWED); + } + this.sampleSize = sampleSize; + } + + public int getSampleSize() { + return sampleSize; + } + + public Mapper infer(ValueFetcher valueFetcher) throws IOException { + RandomSourceValuesGenerator valuesGenerator = new RandomSourceValuesGenerator(sampleSize, indexReader, valueFetcher); + Mapper inferredMapper = null; + while (inferredMapper == null && valuesGenerator.hasNext()) { + List values = valuesGenerator.next(); + if (values == null || values.isEmpty()) { + continue; + } + // always use first value in case of multi value field to infer type + inferredMapper = inferTypeFromObject(values.get(0)); + } + return inferredMapper; + } + + private Mapper inferTypeFromObject(Object o) throws IOException { + if (o == null) { + return null; + } + DocumentMapper mapper = mapperService.documentMapper(); + XContentBuilder builder = XContentFactory.jsonBuilder().startObject().field("field", o).endObject(); + BytesReference bytesReference = BytesReference.bytes(builder); + SourceToParse sourceToParse = new SourceToParse(indexName, "_id", bytesReference, JsonXContent.jsonXContent.mediaType()); + ParsedDocument parsedDocument = mapper.parse(sourceToParse); + Mapping mapping = parsedDocument.dynamicMappingsUpdate(); + return mapping.root.getMapper("field"); + } + + private static class RandomSourceValuesGenerator implements Iterator> { + private final ValueFetcher valueFetcher; + private final IndexReader indexReader; + private final SourceLookup sourceLookup; + private final int[] docs; + private int iter; + private int leaf; + private final int MAX_ATTEMPTS_TO_GENERATE_RANDOM_SAMPLES = 10000; + + public RandomSourceValuesGenerator(int sampleSize, IndexReader indexReader, ValueFetcher valueFetcher) { + this.valueFetcher = valueFetcher; + this.indexReader = indexReader; + sampleSize = Math.min(sampleSize, indexReader.numDocs()); + this.docs = getSortedRandomNum( + sampleSize, + indexReader.numDocs(), + Math.max(sampleSize, MAX_ATTEMPTS_TO_GENERATE_RANDOM_SAMPLES) + ); + this.iter = 0; + this.leaf = -1; + this.sourceLookup = new SourceLookup(); + if (hasNext()) { + setNextLeaf(); + } + } + + @Override + public boolean hasNext() { + return iter < docs.length && leaf < indexReader.leaves().size(); + } + + /** + * Ensure hasNext() is called before calling next() + */ + @Override + public List next() { + int docID = docs[iter] - indexReader.leaves().get(leaf).docBase; + if (docID >= indexReader.leaves().get(leaf).reader().numDocs()) { + setNextLeaf(); + } + // deleted docs are getting used to infer type, which should be okay? + sourceLookup.setSegmentAndDocument(indexReader.leaves().get(leaf), docs[iter] - indexReader.leaves().get(leaf).docBase); + try { + iter++; + return valueFetcher.fetchValues(sourceLookup); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void setNextLeaf() { + int readerIndex = ReaderUtil.subIndex(docs[iter], indexReader.leaves()); + if (readerIndex != leaf) { + leaf = readerIndex; + } else { + // this will only happen when leaves are exhausted and readerIndex will be indexReader.leaves()-1. + leaf++; + } + if (leaf < indexReader.leaves().size()) { + valueFetcher.setNextReader(indexReader.leaves().get(leaf)); + } + } + + private static int[] getSortedRandomNum(int sampleSize, int upperBound, int attempts) { + Set generatedNumbers = new TreeSet<>(); + Random random = Randomness.get(); + int itr = 0; + if (upperBound <= 10 * sampleSize) { + List numberList = new ArrayList<>(); + for (int i = 0; i < upperBound; i++) { + numberList.add(i); + } + Collections.shuffle(numberList, random); + generatedNumbers.addAll(numberList.subList(0, sampleSize)); + } else { + while (generatedNumbers.size() < sampleSize && itr++ < attempts) { + int randomNumber = random.nextInt(upperBound); + generatedNumbers.add(randomNumber); + } + } + return generatedNumbers.stream().mapToInt(Integer::valueOf).toArray(); + } + } +} diff --git a/server/src/test/java/org/opensearch/index/mapper/FieldTypeInferenceTests.java b/server/src/test/java/org/opensearch/index/mapper/FieldTypeInferenceTests.java new file mode 100644 index 0000000000000..807d0e96ce5e3 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/mapper/FieldTypeInferenceTests.java @@ -0,0 +1,210 @@ +/* + * 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.mapper; + +import org.apache.lucene.document.Document; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.store.Directory; +import org.opensearch.common.lucene.Lucene; +import org.opensearch.core.index.Index; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.search.lookup.SourceLookup; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.mockito.Mockito.when; + +public class FieldTypeInferenceTests extends MapperServiceTestCase { + + private static final Map> documentMap; + static { + List listWithNull = new ArrayList<>(); + listWithNull.add(null); + documentMap = new HashMap<>(); + documentMap.put("text_field", List.of("The quick brown fox jumps over the lazy dog.")); + documentMap.put("int_field", List.of(789)); + documentMap.put("float_field", List.of(123.45)); + documentMap.put("date_field_1", List.of("2024-05-12T15:45:00Z")); + documentMap.put("date_field_2", List.of("2024-05-12")); + documentMap.put("boolean_field", List.of(true)); + documentMap.put("null_field", listWithNull); + documentMap.put("array_field_int", List.of(100, 200, 300, 400, 500)); + documentMap.put("array_field_text", List.of("100", "200")); + documentMap.put("object_type", List.of(Map.of("foo", Map.of("bar", 10)))); + } + + public void testJsonSupportedTypes() throws IOException { + MapperService mapperService = createMapperService(topMapping(b -> {})); + QueryShardContext queryShardContext = createQueryShardContext(mapperService); + when(queryShardContext.index()).thenReturn(new Index("test_index", "uuid")); + int totalDocs = 10000; + int docsPerLeafCount = 1000; + try (Directory dir = newDirectory()) { + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); + Document d = new Document(); + for (int i = 0; i < totalDocs; i++) { + iw.addDocument(d); + if ((i + 1) % docsPerLeafCount == 0) { + iw.commit(); + } + } + try (IndexReader reader = DirectoryReader.open(iw)) { + iw.close(); + FieldTypeInference typeInference = new FieldTypeInference("test_index", queryShardContext.getMapperService(), reader); + String[] fieldName = { "text_field" }; + Mapper mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("text", mapper.typeName()); + + fieldName[0] = "int_field"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("long", mapper.typeName()); + + fieldName[0] = "float_field"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("float", mapper.typeName()); + + fieldName[0] = "date_field_1"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("date", mapper.typeName()); + + fieldName[0] = "date_field_2"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("date", mapper.typeName()); + + fieldName[0] = "boolean_field"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("boolean", mapper.typeName()); + + fieldName[0] = "array_field_int"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("long", mapper.typeName()); + + fieldName[0] = "array_field_text"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("text", mapper.typeName()); + + fieldName[0] = "object_type"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertEquals("object", mapper.typeName()); + + fieldName[0] = "null_field"; + mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertNull(mapper); + + // If field is missing ensure that sample docIDs generated for inference are ordered and are in bounds + fieldName[0] = "missing_field"; + List> docsEvaluated = new ArrayList<>(); + int[] totalDocsEvaluated = { 0 }; + typeInference.setSampleSize(50); + mapper = typeInference.infer(new ValueFetcher() { + @Override + public List fetchValues(SourceLookup lookup) throws IOException { + docsEvaluated.get(docsEvaluated.size() - 1).add(lookup.docId()); + totalDocsEvaluated[0]++; + return documentMap.get(fieldName[0]); + } + + @Override + public void setNextReader(LeafReaderContext leafReaderContext) { + docsEvaluated.add(new ArrayList<>()); + } + }); + assertNull(mapper); + assertEquals(typeInference.getSampleSize(), totalDocsEvaluated[0]); + for (List docsPerLeaf : docsEvaluated) { + for (int j = 0; j < docsPerLeaf.size() - 1; j++) { + assertTrue(docsPerLeaf.get(j) < docsPerLeaf.get(j + 1)); + } + if (!docsPerLeaf.isEmpty()) { + assertTrue(docsPerLeaf.get(0) >= 0 && docsPerLeaf.get(docsPerLeaf.size() - 1) < docsPerLeafCount); + } + } + } + } + } + + public void testDeleteAllDocs() throws IOException { + MapperService mapperService = createMapperService(topMapping(b -> {})); + QueryShardContext queryShardContext = createQueryShardContext(mapperService); + when(queryShardContext.index()).thenReturn(new Index("test_index", "uuid")); + int totalDocs = 10000; + int docsPerLeafCount = 1000; + try (Directory dir = newDirectory()) { + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); + Document d = new Document(); + for (int i = 0; i < totalDocs; i++) { + iw.addDocument(d); + if ((i + 1) % docsPerLeafCount == 0) { + iw.commit(); + } + } + iw.deleteAll(); + iw.commit(); + + try (IndexReader reader = DirectoryReader.open(iw)) { + iw.close(); + FieldTypeInference typeInference = new FieldTypeInference("test_index", queryShardContext.getMapperService(), reader); + String[] fieldName = { "text_field" }; + Mapper mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertNull(mapper); + } + } + } + + public void testZeroDoc() throws IOException { + MapperService mapperService = createMapperService(topMapping(b -> {})); + QueryShardContext queryShardContext = createQueryShardContext(mapperService); + when(queryShardContext.index()).thenReturn(new Index("test_index", "uuid")); + try (Directory dir = newDirectory()) { + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); + try (IndexReader reader = DirectoryReader.open(iw)) { + iw.close(); + FieldTypeInference typeInference = new FieldTypeInference("test_index", queryShardContext.getMapperService(), reader); + String[] fieldName = { "text_field" }; + Mapper mapper = typeInference.infer(lookup -> documentMap.get(fieldName[0])); + assertNull(mapper); + } + } + } + + public void testSampleGeneration() throws IOException { + MapperService mapperService = createMapperService(topMapping(b -> {})); + QueryShardContext queryShardContext = createQueryShardContext(mapperService); + when(queryShardContext.index()).thenReturn(new Index("test_index", "uuid")); + int totalDocs = 10000; + int docsPerLeafCount = 1000; + try (Directory dir = newDirectory()) { + IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); + Document d = new Document(); + for (int i = 0; i < totalDocs; i++) { + iw.addDocument(d); + if ((i + 1) % docsPerLeafCount == 0) { + iw.commit(); + } + } + try (IndexReader reader = DirectoryReader.open(iw)) { + iw.close(); + FieldTypeInference typeInference = new FieldTypeInference("test_index", queryShardContext.getMapperService(), reader); + typeInference.setSampleSize(1000 - 1); + typeInference.infer(lookup -> documentMap.get("unknown_field")); + assertThrows(IllegalArgumentException.class, () -> typeInference.setSampleSize(1000 + 1)); + typeInference.setSampleSize(1000); + typeInference.infer(lookup -> documentMap.get("unknown_field")); + } + } + } +} From bb013dadc797bd3349a630444d59b6e9b6b96429 Mon Sep 17 00:00:00 2001 From: Rishabh Maurya Date: Mon, 3 Jun 2024 13:47:08 -0700 Subject: [PATCH 40/77] [DerivedField] object type support in mappings and new settings for derived field (#13717) --------- Signed-off-by: Rishabh Maurya --- .../opensearch/index/mapper/DerivedField.java | 81 ++++++++++++++++++- .../search/builder/SearchSourceBuilder.java | 31 +++++++ .../builder/SearchSourceBuilderTests.java | 64 +++++++++++++++ 3 files changed, 172 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/mapper/DerivedField.java b/server/src/main/java/org/opensearch/index/mapper/DerivedField.java index 7ebe4e5f0b0e8..b502e41cbb97b 100644 --- a/server/src/main/java/org/opensearch/index/mapper/DerivedField.java +++ b/server/src/main/java/org/opensearch/index/mapper/DerivedField.java @@ -8,6 +8,7 @@ package org.opensearch.index.mapper; +import org.opensearch.Version; import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -18,6 +19,7 @@ import org.opensearch.script.Script; import java.io.IOException; +import java.util.Map; import java.util.Objects; /** @@ -25,10 +27,13 @@ */ @PublicApi(since = "2.14.0") public class DerivedField implements Writeable, ToXContentFragment { - private final String name; private final String type; private final Script script; + private String sourceIndexedField; + private Map properties; + private Boolean ignoreMalformed; + private String format; public DerivedField(String name, String type, Script script) { this.name = name; @@ -40,6 +45,14 @@ public DerivedField(StreamInput in) throws IOException { name = in.readString(); type = in.readString(); script = new Script(in); + if (in.getVersion().onOrAfter(Version.V_2_15_0)) { + if (in.readBoolean()) { + properties = in.readMap(); + } + sourceIndexedField = in.readOptionalString(); + format = in.readOptionalString(); + ignoreMalformed = in.readOptionalBoolean(); + } } @Override @@ -47,6 +60,17 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(name); out.writeString(type); script.writeTo(out); + if (out.getVersion().onOrAfter(Version.V_2_15_0)) { + if (properties == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeMap(properties); + } + out.writeOptionalString(sourceIndexedField); + out.writeOptionalString(format); + out.writeOptionalBoolean(ignoreMalformed); + } } @Override @@ -54,6 +78,18 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par builder.startObject(name); builder.field("type", type); builder.field("script", script); + if (properties != null) { + builder.field("properties", properties); + } + if (sourceIndexedField != null) { + builder.field("source_indexed_field", sourceIndexedField); + } + if (format != null) { + builder.field("format", format); + } + if (ignoreMalformed != null) { + builder.field("ignore_malformed", ignoreMalformed); + } builder.endObject(); return builder; } @@ -70,9 +106,41 @@ public Script getScript() { return script; } + public Map getProperties() { + return properties; + } + + public String getSourceIndexedField() { + return sourceIndexedField; + } + + public String getFormat() { + return format; + } + + public boolean getIgnoreMalformed() { + return Boolean.TRUE.equals(ignoreMalformed); + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + public void setSourceIndexedField(String sourceIndexedField) { + this.sourceIndexedField = sourceIndexedField; + } + + public void setFormat(String format) { + this.format = format; + } + + public void setIgnoreMalformed(boolean ignoreMalformed) { + this.ignoreMalformed = ignoreMalformed; + } + @Override public int hashCode() { - return Objects.hash(name, type, script); + return Objects.hash(name, type, script, sourceIndexedField, properties, ignoreMalformed, format); } @Override @@ -84,7 +152,12 @@ public boolean equals(Object obj) { return false; } DerivedField other = (DerivedField) obj; - return Objects.equals(name, other.name) && Objects.equals(type, other.type) && Objects.equals(script, other.script); + return Objects.equals(name, other.name) + && Objects.equals(type, other.type) + && Objects.equals(script, other.script) + && Objects.equals(sourceIndexedField, other.sourceIndexedField) + && Objects.equals(properties, other.properties) + && Objects.equals(ignoreMalformed, other.ignoreMalformed) + && Objects.equals(format, other.format); } - } 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 07248a0719c3a..6c22567d8cf0d 100644 --- a/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java +++ b/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java @@ -1004,6 +1004,37 @@ public SearchSourceBuilder derivedField(String name, String type, Script script) return this; } + /** + * Adds a derived field with the given name with provided type, script and other parameters + * @param name name of the derived field + * @param type type of the derived field + * @param script script associated with derived field + * @param properties map of field name and type of field for nested fields within object derived field + * @param sourceIndexedField source text field which is indexed to filter documents for better performance + * @param format date format + * @param ignoreMalformed ignores malformed fields instead of failing search request + */ + public SearchSourceBuilder derivedField( + String name, + String type, + Script script, + Map properties, + String sourceIndexedField, + String format, + Boolean ignoreMalformed + ) { + if (derivedFields == null) { + derivedFields = new ArrayList<>(); + } + DerivedField derivedField = new DerivedField(name, type, script); + derivedField.setProperties(properties); + derivedField.setSourceIndexedField(sourceIndexedField); + derivedField.setFormat(format); + derivedField.setIgnoreMalformed(ignoreMalformed); + derivedFields.add(derivedField); + return this; + } + /** * Sets the boost a specific index or alias will receive when the query is executed * against it. diff --git a/server/src/test/java/org/opensearch/search/builder/SearchSourceBuilderTests.java b/server/src/test/java/org/opensearch/search/builder/SearchSourceBuilderTests.java index 5b1035e24185d..fd3dd8c12e84e 100644 --- a/server/src/test/java/org/opensearch/search/builder/SearchSourceBuilderTests.java +++ b/server/src/test/java/org/opensearch/search/builder/SearchSourceBuilderTests.java @@ -357,6 +357,70 @@ public void testDerivedFieldsParsingAndSerialization() throws IOException { } + public void testDerivedFieldsParsingAndSerializationObjectType() throws IOException { + { + String restContent = "{\n" + + " \"derived\": {\n" + + " \"duration\": {\n" + + " \"type\": \"long\",\n" + + " \"script\": \"emit(doc['test'])\"\n" + + " },\n" + + " \"ip_from_message\": {\n" + + " \"type\": \"keyword\",\n" + + " \"script\": \"emit(doc['message'])\"\n" + + " },\n" + + " \"object\": {\n" + + " \"type\": \"object\",\n" + + " \"script\": \"emit(doc['test'])\",\n" + + " \"format\": \"dd-MM-yyyy\",\n" + + " \"source_indexed_field\": \"test\",\n" + + " \"ignore_malformed\": true,\n" + + " \"properties\": {\n" + + " \"sub_field\": \"text\"\n" + + " }\n" + + " }\n" + + " },\n" + + " \"query\" : {\n" + + " \"match\": { \"content\": { \"query\": \"foo bar\" }}\n" + + " }\n" + + "}"; + + String expectedContent = + "{\"query\":{\"match\":{\"content\":{\"query\":\"foo bar\",\"operator\":\"OR\",\"prefix_length\":0,\"max_expansions\":50,\"fuzzy_transpositions\":true,\"lenient\":false,\"zero_terms_query\":\"NONE\",\"auto_generate_synonyms_phrase_query\":true,\"boost\":1.0}}},\"derived\":{\"duration\":{\"type\":\"long\",\"script\":\"emit(doc['test'])\"},\"ip_from_message\":{\"type\":\"keyword\",\"script\":\"emit(doc['message'])\"},\"object\":{\"format\":\"dd-MM-yyyy\",\"source_indexed_field\":\"test\",\"ignore_malformed\":true,\"type\":\"object\",\"script\":\"emit(doc['test'])\",\"properties\":{\"sub_field\":\"text\"}},\"derived_field\":{\"type\":\"object\",\"script\":{\"source\":\"emit(doc['message']\",\"lang\":\"painless\"},\"properties\":{\"sub_field_2\":\"keyword\"},\"source_indexed_field\":\"message\",\"format\":\"dd-MM-yyyy\",\"ignore_malformed\":true}}}"; + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, restContent)) { + SearchSourceBuilder searchSourceBuilder = SearchSourceBuilder.fromXContent(parser); + searchSourceBuilder.derivedField( + "derived_field", + "object", + new Script("emit(doc['message']"), + Map.of("sub_field_2", "keyword"), + "message", + "dd-MM-yyyy", + true + ); + searchSourceBuilder = rewrite(searchSourceBuilder); + assertEquals(3, searchSourceBuilder.getDerivedFieldsObject().size()); + assertEquals(1, searchSourceBuilder.getDerivedFields().size()); + assertEquals(1, searchSourceBuilder.getDerivedFields().get(0).getProperties().size()); + assertEquals("message", searchSourceBuilder.getDerivedFields().get(0).getSourceIndexedField()); + assertEquals("dd-MM-yyyy", searchSourceBuilder.getDerivedFields().get(0).getFormat()); + assertTrue(searchSourceBuilder.getDerivedFields().get(0).getIgnoreMalformed()); + + try (BytesStreamOutput output = new BytesStreamOutput()) { + searchSourceBuilder.writeTo(output); + try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), namedWriteableRegistry)) { + SearchSourceBuilder deserializedBuilder = new SearchSourceBuilder(in); + String actualContent = deserializedBuilder.toString(); + assertEquals(expectedContent, actualContent); + assertEquals(searchSourceBuilder.hashCode(), deserializedBuilder.hashCode()); + assertNotSame(searchSourceBuilder, deserializedBuilder); + } + } + } + } + } + public void testAggsParsing() throws IOException { { String restContent = "{\n" From e3d7e9988b643173babe0fbb82a31fe6c6c1eb86 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Mon, 3 Jun 2024 15:49:16 -0700 Subject: [PATCH 41/77] Update gradle config for develocity rebranding (#13942) The Gradle Enterprise plugin has been rebranded as the Develocity Plugin, resulting in a number of settings being deprecated. This commit updates the settings per [this migration guide][1] and removes all warnings from the build. [1]: https://docs.gradle.com/develocity/gradle-plugin/legacy/#develocity_migration Signed-off-by: Andrew Ross --- build.gradle | 11 ++++++++--- gradle.properties | 2 +- settings.gradle | 2 +- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/build.gradle b/build.gradle index e92f396e006f5..55b31ca816214 100644 --- a/build.gradle +++ b/build.gradle @@ -55,7 +55,6 @@ plugins { id 'opensearch.docker-support' id 'opensearch.global-build-info' id "com.diffplug.spotless" version "6.25.0" apply false - id "org.gradle.test-retry" version "1.5.9" apply false id "test-report-aggregation" id 'jacoco-report-aggregation' } @@ -71,6 +70,13 @@ apply from: 'gradle/run.gradle' apply from: 'gradle/missing-javadoc.gradle' apply from: 'gradle/code-coverage.gradle' +// Disable unconditional publishing of build scans +develocity { + buildScan { + publishing.onlyIf { false } + } +} + // common maven publishing configuration allprojects { group = 'org.opensearch' @@ -462,9 +468,8 @@ gradle.projectsEvaluated { // test retry configuration subprojects { - apply plugin: "org.gradle.test-retry" tasks.withType(Test).configureEach { - retry { + develocity.testRetry { if (BuildParams.isCi()) { maxRetries = 3 maxFailures = 10 diff --git a/gradle.properties b/gradle.properties index 7c359ed2b652c..4e8c5b98116c1 100644 --- a/gradle.properties +++ b/gradle.properties @@ -22,7 +22,7 @@ org.gradle.jvmargs=-Xmx3g -XX:+HeapDumpOnOutOfMemoryError -Xss2m \ options.forkOptions.memoryMaximumSize=3g # Disable Gradle Enterprise Gradle plugin's test retry -systemProp.gradle.enterprise.testretry.enabled=false +systemProp.develocity.testretry.enabled.enabled=false # Disable duplicate project id detection # See https://docs.gradle.org/current/userguide/upgrading_version_6.html#duplicate_project_names_may_cause_publication_to_fail diff --git a/settings.gradle b/settings.gradle index f58a8b35b10eb..ca8538a967ef7 100644 --- a/settings.gradle +++ b/settings.gradle @@ -10,7 +10,7 @@ */ plugins { - id "com.gradle.enterprise" version "3.17.4" + id "com.gradle.develocity" version "3.17.4" } ext.disableBuildCache = hasProperty('DISABLE_BUILD_CACHE') || System.getenv().containsKey('DISABLE_BUILD_CACHE') From 581fcd2c3204f84ff93c626be41698ec35641680 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 3 Jun 2024 18:50:01 -0400 Subject: [PATCH 42/77] Bump actions/checkout from 3 to 4 (#13935) * Bump actions/checkout from 3 to 4 Bumps [actions/checkout](https://github.com/actions/checkout) from 3 to 4. - [Release notes](https://github.com/actions/checkout/releases) - [Changelog](https://github.com/actions/checkout/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/checkout/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/checkout dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Signed-off-by: Peter Nied Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] Co-authored-by: Peter Nied --- .github/workflows/gradle-check.yml | 2 +- CHANGELOG.md | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/gradle-check.yml b/.github/workflows/gradle-check.yml index 0ba4dbe374fdd..07185ef4c65e3 100644 --- a/.github/workflows/gradle-check.yml +++ b/.github/workflows/gradle-check.yml @@ -17,7 +17,7 @@ jobs: outputs: RUN_GRADLE_CHECK: ${{ steps.changed-files-specific.outputs.any_changed }} steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 - name: Get changed files id: changed-files-specific uses: tj-actions/changed-files@v44 diff --git a/CHANGELOG.md b/CHANGELOG.md index 3554565e737e8..5ce11dabb6e9c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -31,6 +31,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `reactor-netty` from 1.1.17 to 1.1.19 ([#13825](https://github.com/opensearch-project/OpenSearch/pull/13825)) - Bump `commons-cli:commons-cli` from 1.7.0 to 1.8.0 ([#13840](https://github.com/opensearch-project/OpenSearch/pull/13840)) - Bump `org.apache.xmlbeans:xmlbeans` from 5.2.0 to 5.2.1 ([#13839](https://github.com/opensearch-project/OpenSearch/pull/13839)) +- Bump `actions/checkout` from 3 to 4 ([#13935](https://github.com/opensearch-project/OpenSearch/pull/13935)) - Bump `com.netflix.nebula.ospackage-base` from 11.9.0 to 11.9.1 ([#13933](https://github.com/opensearch-project/OpenSearch/pull/13933)) ### Changed From 293905af45b8eabb751fe1d89cde3bfcf3d54014 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Tue, 4 Jun 2024 13:02:17 -0700 Subject: [PATCH 43/77] Add DCO workflow (#13970) Signed-off-by: Prudhvi Godithi --- .github/workflows/dco.yml | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 .github/workflows/dco.yml diff --git a/.github/workflows/dco.yml b/.github/workflows/dco.yml new file mode 100644 index 0000000000000..9580d510fd108 --- /dev/null +++ b/.github/workflows/dco.yml @@ -0,0 +1,19 @@ +name: Developer Certificate of Origin Check + +on: [pull_request] + +jobs: + dco-check: + runs-on: ubuntu-latest + + steps: + - name: Get PR Commits + id: 'get-pr-commits' + uses: tim-actions/get-pr-commits@v1.1.0 + with: + token: ${{ secrets.GITHUB_TOKEN }} + - name: DCO Check + uses: tim-actions/dco@v1.1.0 + with: + commits: ${{ steps.get-pr-commits.outputs.commits }} + From 97340ed2616ef94e50f887718370e4e09c434bc0 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Tue, 4 Jun 2024 17:56:12 -0400 Subject: [PATCH 44/77] Revert "Optimize remote state stale file deletion (#13131)" (#13969) This reverts commit b9befaa12524744ffe71d3f12699cb27cb8fca5e. Signed-off-by: Andriy Redko --- .../RemoteClusterStateCleanupManagerIT.java | 145 ------ .../remote/RemoteClusterStateServiceIT.java | 63 +++ .../RemoteStoreBaseIntegTestCase.java | 10 - .../common/settings/ClusterSettings.java | 2 - .../RemoteClusterStateCleanupManager.java | 408 ---------------- .../remote/RemoteClusterStateService.java | 275 +++++++++-- .../main/java/org/opensearch/node/Node.java | 7 +- .../GatewayMetaStatePersistedStateTests.java | 6 +- ...RemoteClusterStateCleanupManagerTests.java | 446 ------------------ .../RemoteClusterStateServiceTests.java | 136 +++++- 10 files changed, 442 insertions(+), 1056 deletions(-) delete mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java delete mode 100644 server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java delete mode 100644 server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java deleted file mode 100644 index 0f441fe01a368..0000000000000 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.gateway.remote; - -import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; -import org.opensearch.common.blobstore.BlobPath; -import org.opensearch.common.settings.Settings; -import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; -import org.opensearch.repositories.RepositoriesService; -import org.opensearch.repositories.blobstore.BlobStoreRepository; -import org.opensearch.test.OpenSearchIntegTestCase; -import org.junit.Before; - -import java.nio.charset.StandardCharsets; -import java.util.Base64; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; -import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; -import static org.opensearch.indices.IndicesService.CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING; - -@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) -public class RemoteClusterStateCleanupManagerIT extends RemoteStoreBaseIntegTestCase { - - private static final String INDEX_NAME = "test-index"; - - @Before - public void setup() { - asyncUploadMockFsRepo = false; - } - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); - } - - private Map initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount) { - prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount); - Map indexStats = indexData(1, false, INDEX_NAME); - assertEquals(shardCount * (replicaCount + 1), getNumShards(INDEX_NAME).totalNumShards); - ensureGreen(INDEX_NAME); - return indexStats; - } - - public void testRemoteCleanupTaskUpdated() { - int shardCount = randomIntBetween(1, 2); - int replicaCount = 1; - int dataNodeCount = shardCount * (replicaCount + 1); - int clusterManagerNodeCount = 1; - - initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); - RemoteClusterStateCleanupManager remoteClusterStateCleanupManager = internalCluster().getClusterManagerNodeInstance( - RemoteClusterStateCleanupManager.class - ); - - assertEquals(CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval()); - assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); - - // now disable - client().admin() - .cluster() - .prepareUpdateSettings() - .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), -1)) - .get(); - - assertEquals(-1, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMillis()); - assertFalse(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); - - // now set Clean up interval to 1 min - client().admin() - .cluster() - .prepareUpdateSettings() - .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "1m")) - .get(); - assertEquals(1, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMinutes()); - } - - public void testRemoteCleanupDeleteStale() throws Exception { - int shardCount = randomIntBetween(1, 2); - int replicaCount = 1; - int dataNodeCount = shardCount * (replicaCount + 1); - int clusterManagerNodeCount = 1; - - initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); - - // set cleanup interval to 100 ms to make the test faster - ClusterUpdateSettingsResponse response = client().admin() - .cluster() - .prepareUpdateSettings() - .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "100ms")) - .get(); - - assertTrue(response.isAcknowledged()); - - // update cluster state 21 times to ensure that clean up has run after this will upload 42 manifest files - // to repository, if manifest files are less than that it means clean up has run - updateClusterStateNTimes(RETAINED_MANIFESTS + SKIP_CLEANUP_STATE_CHANGES + 1); - - RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); - BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); - BlobPath baseMetadataPath = repository.basePath() - .add( - Base64.getUrlEncoder() - .withoutPadding() - .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) - ) - .add("cluster-state") - .add(getClusterState().metadata().clusterUUID()); - BlobPath manifestContainerPath = baseMetadataPath.add("manifest"); - - assertBusy(() -> { - int manifestFiles = repository.blobStore().blobContainer(manifestContainerPath).listBlobsByPrefix("manifest").size(); - logger.info("number of current manifest file: {}", manifestFiles); - // we can't guarantee that we have same number of manifest as Retained manifest in our repo as there can be other queued task - // other than replica count change which can upload new manifest files, that's why we check that number of manifests is between - // Retained manifests and Retained manifests + 2 * Skip cleanup state changes (each cluster state update uploads 2 manifests) - assertTrue( - "Current number of manifest files: " + manifestFiles, - manifestFiles >= RETAINED_MANIFESTS && manifestFiles < RETAINED_MANIFESTS + 2 * SKIP_CLEANUP_STATE_CHANGES - ); - }, 500, TimeUnit.MILLISECONDS); - } - - private void updateClusterStateNTimes(int n) { - int newReplicaCount = randomIntBetween(0, 3); - for (int i = n; i > 0; i--) { - ClusterUpdateSettingsResponse response = client().admin() - .cluster() - .prepareUpdateSettings() - .setPersistentSettings(Settings.builder().put(CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING.getKey(), i, TimeUnit.SECONDS)) - .get(); - assertTrue(response.isAcknowledged()); - } - } -} diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java index ab2f0f0080566..42120aa32eb47 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java @@ -10,6 +10,7 @@ import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; import org.opensearch.discovery.DiscoveryStats; @@ -26,6 +27,7 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.CUSTOM_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; @@ -49,6 +51,16 @@ protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); } + private void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { + internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes); + internalCluster().startDataOnlyNodes(numDataOnlyNodes); + for (String index : indices.split(",")) { + createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); + ensureYellowAndNoInitializingShards(index); + ensureGreen(index); + } + } + private Map initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount) { prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount); Map indexStats = indexData(1, false, INDEX_NAME); @@ -57,6 +69,49 @@ private Map initialTestSetup(int shardCount, int replicaCount, int return indexStats; } + public void testFullClusterRestoreStaleDelete() throws Exception { + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); + setReplicaCount(0); + setReplicaCount(2); + setReplicaCount(0); + setReplicaCount(1); + setReplicaCount(0); + setReplicaCount(1); + setReplicaCount(0); + setReplicaCount(2); + setReplicaCount(0); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + + RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); + + BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); + BlobPath baseMetadataPath = repository.basePath() + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(getClusterState().metadata().clusterUUID()); + + assertEquals(10, repository.blobStore().blobContainer(baseMetadataPath.add("manifest")).listBlobsByPrefix("manifest").size()); + + Map indexMetadataMap = remoteClusterStateService.getLatestClusterState( + cluster().getClusterName(), + getClusterState().metadata().clusterUUID() + ).getMetadata().getIndices(); + assertEquals(0, indexMetadataMap.values().stream().findFirst().get().getNumberOfReplicas()); + assertEquals(shardCount, indexMetadataMap.values().stream().findFirst().get().getNumberOfShards()); + } + public void testRemoteStateStats() { int shardCount = randomIntBetween(1, 2); int replicaCount = 1; @@ -186,4 +241,12 @@ private void validateNodesStatsResponse(NodesStatsResponse nodesStatsResponse) { assertNotNull(nodesStatsResponse.getNodes().get(0)); assertNotNull(nodesStatsResponse.getNodes().get(0).getDiscoveryStats()); } + + private void setReplicaCount(int replicaCount) { + client().admin() + .indices() + .prepareUpdateSettings(INDEX_NAME) + .setSettings(Settings.builder().put(SETTING_NUMBER_OF_REPLICAS, replicaCount)) + .get(); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 64efcee6ef1b5..740aee69f7d80 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -350,14 +350,4 @@ protected void restore(boolean restoreAllShards, String... indices) { PlainActionFuture.newFuture() ); } - - protected void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { - internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes); - internalCluster().startDataOnlyNodes(numDataOnlyNodes); - for (String index : indices.split(",")) { - createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); - ensureYellowAndNoInitializingShards(index); - ensureGreen(index); - } - } } 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 297fc98764d07..7814518af471b 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -104,7 +104,6 @@ import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.PersistedClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; -import org.opensearch.gateway.remote.RemoteClusterStateCleanupManager; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpTransportSettings; import org.opensearch.index.IndexModule; @@ -712,7 +711,6 @@ public void apply(Settings value, Settings current, Settings previous) { SearchRequestSlowLog.CLUSTER_SEARCH_REQUEST_SLOWLOG_LEVEL, // Remote cluster state settings - RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING, RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING, RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, RemoteClusterStateService.GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java deleted file mode 100644 index 8a106a25e5630..0000000000000 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java +++ /dev/null @@ -1,408 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.gateway.remote; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.logging.log4j.util.Strings; -import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.service.ClusterApplierService; -import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.blobstore.BlobMetadata; -import org.opensearch.common.blobstore.BlobPath; -import org.opensearch.common.settings.ClusterSettings; -import org.opensearch.common.settings.Setting; -import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.concurrent.AbstractAsyncTask; -import org.opensearch.core.action.ActionListener; -import org.opensearch.index.translog.transfer.BlobStoreTransferService; -import org.opensearch.threadpool.ThreadPool; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_FORMAT; -import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_PATH_TOKEN; -import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_METADATA_FORMAT; -import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_PATH_TOKEN; -import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; -import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_PATH_TOKEN; - -/** - * A Manager which provides APIs to clean up stale cluster state files and runs an async stale cleanup task - * - * @opensearch.internal - */ -public class RemoteClusterStateCleanupManager implements Closeable { - - public static final int RETAINED_MANIFESTS = 10; - public static final int SKIP_CLEANUP_STATE_CHANGES = 10; - public static final TimeValue CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT = TimeValue.timeValueMinutes(5); - public static final TimeValue CLUSTER_STATE_CLEANUP_INTERVAL_MINIMUM = TimeValue.MINUS_ONE; - - /** - * Setting to specify the interval to do run stale file cleanup job - * Min value -1 indicates that the stale file cleanup job should be disabled - */ - public static final Setting REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING = Setting.timeSetting( - "cluster.remote_store.state.cleanup_interval", - CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, - CLUSTER_STATE_CLEANUP_INTERVAL_MINIMUM, - Setting.Property.NodeScope, - Setting.Property.Dynamic - ); - private static final Logger logger = LogManager.getLogger(RemoteClusterStateCleanupManager.class); - private final RemoteClusterStateService remoteClusterStateService; - private final RemotePersistenceStats remoteStateStats; - private BlobStoreTransferService blobStoreTransferService; - private TimeValue staleFileCleanupInterval; - private final AtomicBoolean deleteStaleMetadataRunning = new AtomicBoolean(false); - private AsyncStaleFileDeletion staleFileDeletionTask; - private long lastCleanupAttemptStateVersion; - private final ThreadPool threadpool; - private final ClusterApplierService clusterApplierService; - - public RemoteClusterStateCleanupManager(RemoteClusterStateService remoteClusterStateService, ClusterService clusterService) { - this.remoteClusterStateService = remoteClusterStateService; - this.remoteStateStats = remoteClusterStateService.getStats(); - ClusterSettings clusterSettings = clusterService.getClusterSettings(); - this.clusterApplierService = clusterService.getClusterApplierService(); - this.staleFileCleanupInterval = clusterSettings.get(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING); - this.threadpool = remoteClusterStateService.getThreadpool(); - // initialize with 0, a cleanup will be done when this node is elected master node and version is incremented more than threshold - this.lastCleanupAttemptStateVersion = 0; - clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING, this::updateCleanupInterval); - } - - void start() { - staleFileDeletionTask = new AsyncStaleFileDeletion(this); - } - - @Override - public void close() throws IOException { - if (staleFileDeletionTask != null) { - staleFileDeletionTask.close(); - } - } - - private BlobStoreTransferService getBlobStoreTransferService() { - if (blobStoreTransferService == null) { - blobStoreTransferService = new BlobStoreTransferService(remoteClusterStateService.getBlobStore(), threadpool); - } - return blobStoreTransferService; - } - - private void updateCleanupInterval(TimeValue updatedInterval) { - this.staleFileCleanupInterval = updatedInterval; - logger.info("updated remote state cleanup interval to {}", updatedInterval); - // After updating the interval, we need to close the current task and create a new one which will run with updated interval - if (staleFileDeletionTask != null && !staleFileDeletionTask.getInterval().equals(updatedInterval)) { - staleFileDeletionTask.setInterval(updatedInterval); - } - } - - // visible for testing - void cleanUpStaleFiles() { - ClusterState currentAppliedState = clusterApplierService.state(); - if (currentAppliedState.nodes().isLocalNodeElectedClusterManager()) { - long cleanUpAttemptStateVersion = currentAppliedState.version(); - assert Strings.isNotEmpty(currentAppliedState.getClusterName().value()) : "cluster name is not set"; - assert Strings.isNotEmpty(currentAppliedState.metadata().clusterUUID()) : "cluster uuid is not set"; - if (cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion > SKIP_CLEANUP_STATE_CHANGES) { - logger.info( - "Cleaning up stale remote state files for cluster [{}] with uuid [{}]. Last clean was done before {} updates", - currentAppliedState.getClusterName().value(), - currentAppliedState.metadata().clusterUUID(), - cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion - ); - this.deleteStaleClusterMetadata( - currentAppliedState.getClusterName().value(), - currentAppliedState.metadata().clusterUUID(), - RETAINED_MANIFESTS - ); - lastCleanupAttemptStateVersion = cleanUpAttemptStateVersion; - } else { - logger.debug( - "Skipping cleanup of stale remote state files for cluster [{}] with uuid [{}]. Last clean was done before {} updates, which is less than threshold {}", - currentAppliedState.getClusterName().value(), - currentAppliedState.metadata().clusterUUID(), - cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion, - SKIP_CLEANUP_STATE_CHANGES - ); - } - } else { - logger.debug("Skipping cleanup task as local node is not elected Cluster Manager"); - } - } - - private void addStaleGlobalMetadataPath(String fileName, Set filesToKeep, Set staleGlobalMetadataPaths) { - if (!filesToKeep.contains(fileName)) { - String[] splitPath = fileName.split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - splitPath[splitPath.length - 1] - ) - ); - } - } - - // visible for testing - void deleteClusterMetadata( - String clusterName, - String clusterUUID, - List activeManifestBlobMetadata, - List staleManifestBlobMetadata - ) { - try { - Set filesToKeep = new HashSet<>(); - Set staleManifestPaths = new HashSet<>(); - Set staleIndexMetadataPaths = new HashSet<>(); - Set staleGlobalMetadataPaths = new HashSet<>(); - activeManifestBlobMetadata.forEach(blobMetadata -> { - ClusterMetadataManifest clusterMetadataManifest = remoteClusterStateService.fetchRemoteClusterMetadataManifest( - clusterName, - clusterUUID, - blobMetadata.name() - ); - clusterMetadataManifest.getIndices() - .forEach(uploadedIndexMetadata -> filesToKeep.add(uploadedIndexMetadata.getUploadedFilename())); - if (clusterMetadataManifest.getCodecVersion() == ClusterMetadataManifest.CODEC_V1) { - filesToKeep.add(clusterMetadataManifest.getGlobalMetadataFileName()); - } else if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V2) { - filesToKeep.add(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); - filesToKeep.add(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); - filesToKeep.add(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); - clusterMetadataManifest.getCustomMetadataMap() - .values() - .forEach(attribute -> filesToKeep.add(attribute.getUploadedFilename())); - } - }); - staleManifestBlobMetadata.forEach(blobMetadata -> { - ClusterMetadataManifest clusterMetadataManifest = remoteClusterStateService.fetchRemoteClusterMetadataManifest( - clusterName, - clusterUUID, - blobMetadata.name() - ); - staleManifestPaths.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blobMetadata.name()); - if (clusterMetadataManifest.getCodecVersion() == ClusterMetadataManifest.CODEC_V1) { - addStaleGlobalMetadataPath(clusterMetadataManifest.getGlobalMetadataFileName(), filesToKeep, staleGlobalMetadataPaths); - } else if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V2) { - addStaleGlobalMetadataPath( - clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename(), - filesToKeep, - staleGlobalMetadataPaths - ); - addStaleGlobalMetadataPath( - clusterMetadataManifest.getSettingsMetadata().getUploadedFilename(), - filesToKeep, - staleGlobalMetadataPaths - ); - addStaleGlobalMetadataPath( - clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename(), - filesToKeep, - staleGlobalMetadataPaths - ); - clusterMetadataManifest.getCustomMetadataMap() - .values() - .forEach( - attribute -> addStaleGlobalMetadataPath(attribute.getUploadedFilename(), filesToKeep, staleGlobalMetadataPaths) - ); - } - - clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { - if (filesToKeep.contains(uploadedIndexMetadata.getUploadedFilename()) == false) { - staleIndexMetadataPaths.add( - new BlobPath().add(INDEX_PATH_TOKEN).add(uploadedIndexMetadata.getIndexUUID()).buildAsString() - + INDEX_METADATA_FORMAT.blobName(uploadedIndexMetadata.getUploadedFilename()) - ); - } - }); - }); - - if (staleManifestPaths.isEmpty()) { - logger.debug("No stale Remote Cluster Metadata files found"); - return; - } - - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleGlobalMetadataPaths)); - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleIndexMetadataPaths)); - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleManifestPaths)); - } catch (IllegalStateException e) { - logger.error("Error while fetching Remote Cluster Metadata manifests", e); - } catch (IOException e) { - logger.error("Error while deleting stale Remote Cluster Metadata files", e); - remoteStateStats.cleanUpAttemptFailed(); - } catch (Exception e) { - logger.error("Unexpected error while deleting stale Remote Cluster Metadata files", e); - remoteStateStats.cleanUpAttemptFailed(); - } - } - - /** - * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests - * - * @param clusterName name of the cluster - * @param clusterUUID uuid of cluster state to refer to in remote - * @param manifestsToRetain no of latest manifest files to keep in remote - */ - // package private for testing - void deleteStaleClusterMetadata(String clusterName, String clusterUUID, int manifestsToRetain) { - if (deleteStaleMetadataRunning.compareAndSet(false, true) == false) { - logger.info("Delete stale cluster metadata task is already in progress."); - return; - } - try { - getBlobStoreTransferService().listAllInSortedOrderAsync( - ThreadPool.Names.REMOTE_PURGE, - remoteClusterStateService.getManifestFolderPath(clusterName, clusterUUID), - MANIFEST_FILE_PREFIX, - Integer.MAX_VALUE, - new ActionListener<>() { - @Override - public void onResponse(List blobMetadata) { - if (blobMetadata.size() > manifestsToRetain) { - deleteClusterMetadata( - clusterName, - clusterUUID, - blobMetadata.subList(0, manifestsToRetain), - blobMetadata.subList(manifestsToRetain, blobMetadata.size()) - ); - } - deleteStaleMetadataRunning.set(false); - } - - @Override - public void onFailure(Exception e) { - logger.error( - new ParameterizedMessage( - "Exception occurred while deleting Remote Cluster Metadata for clusterUUIDs {}", - clusterUUID - ) - ); - deleteStaleMetadataRunning.set(false); - } - } - ); - } catch (Exception e) { - deleteStaleMetadataRunning.set(false); - throw e; - } - } - - /** - * Purges all remote cluster state against provided cluster UUIDs - * - * @param clusterName name of the cluster - * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged - */ - void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { - clusterUUIDs.forEach( - clusterUUID -> getBlobStoreTransferService().deleteAsync( - ThreadPool.Names.REMOTE_PURGE, - remoteClusterStateService.getCusterMetadataBasePath(clusterName, clusterUUID), - new ActionListener<>() { - @Override - public void onResponse(Void unused) { - logger.info("Deleted all remote cluster metadata for cluster UUID - {}", clusterUUID); - } - - @Override - public void onFailure(Exception e) { - logger.error( - new ParameterizedMessage( - "Exception occurred while deleting all remote cluster metadata for cluster UUID {}", - clusterUUID - ), - e - ); - remoteStateStats.cleanUpAttemptFailed(); - } - } - ) - ); - } - - // package private for testing - void deleteStalePaths(String clusterName, String clusterUUID, List stalePaths) throws IOException { - logger.debug(String.format(Locale.ROOT, "Deleting stale files from remote - %s", stalePaths)); - getBlobStoreTransferService().deleteBlobs( - remoteClusterStateService.getCusterMetadataBasePath(clusterName, clusterUUID), - stalePaths - ); - } - - /** - * Purges all remote cluster state against provided cluster UUIDs - * @param clusterState current state of the cluster - * @param committedManifest last committed ClusterMetadataManifest - */ - public void deleteStaleClusterUUIDs(ClusterState clusterState, ClusterMetadataManifest committedManifest) { - threadpool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { - String clusterName = clusterState.getClusterName().value(); - logger.debug("Deleting stale cluster UUIDs data from remote [{}]", clusterName); - Set allClustersUUIDsInRemote; - try { - allClustersUUIDsInRemote = new HashSet<>( - remoteClusterStateService.getAllClusterUUIDs(clusterState.getClusterName().value()) - ); - } catch (IOException e) { - logger.info(String.format(Locale.ROOT, "Error while fetching all cluster UUIDs for [%s]", clusterName)); - return; - } - // Retain last 2 cluster uuids data - allClustersUUIDsInRemote.remove(committedManifest.getClusterUUID()); - allClustersUUIDsInRemote.remove(committedManifest.getPreviousClusterUUID()); - deleteStaleUUIDsClusterMetadata(clusterName, new ArrayList<>(allClustersUUIDsInRemote)); - }); - } - - public TimeValue getStaleFileCleanupInterval() { - return this.staleFileCleanupInterval; - } - - AsyncStaleFileDeletion getStaleFileDeletionTask() { // for testing - return this.staleFileDeletionTask; - } - - RemotePersistenceStats getStats() { - return this.remoteStateStats; - } - - static final class AsyncStaleFileDeletion extends AbstractAsyncTask { - private final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; - - AsyncStaleFileDeletion(RemoteClusterStateCleanupManager remoteClusterStateCleanupManager) { - super( - logger, - remoteClusterStateCleanupManager.threadpool, - remoteClusterStateCleanupManager.getStaleFileCleanupInterval(), - true - ); - this.remoteClusterStateCleanupManager = remoteClusterStateCleanupManager; - rescheduleIfNecessary(); - } - - @Override - protected boolean mustReschedule() { - return true; - } - - @Override - protected void runInternal() { - remoteClusterStateCleanupManager.cleanUpStaleFiles(); - } - } -} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index 01ffd8f1cca46..0f862d1b68820 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -18,13 +18,11 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.TemplatesMetadata; -import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.Nullable; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; -import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -61,6 +59,7 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.LongSupplier; @@ -82,6 +81,8 @@ public class RemoteClusterStateService implements Closeable { public static final String METADATA_MANIFEST_NAME_FORMAT = "%s"; + public static final int RETAINED_MANIFESTS = 10; + public static final String DELIMITER = "__"; public static final String CUSTOM_DELIMITER = "--"; @@ -206,7 +207,8 @@ public class RemoteClusterStateService implements Closeable { private volatile TimeValue indexMetadataUploadTimeout; private volatile TimeValue globalMetadataUploadTimeout; private volatile TimeValue metadataManifestUploadTimeout; - private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; + + private final AtomicBoolean deleteStaleMetadataRunning = new AtomicBoolean(false); private final RemotePersistenceStats remoteStateStats; private final String CLUSTER_STATE_UPLOAD_TIME_LOG_STRING = "writing cluster state for version [{}] took [{}ms]"; private final String METADATA_UPDATE_LOG_STRING = "wrote metadata for [{}] indices and skipped [{}] unchanged " @@ -230,7 +232,7 @@ public RemoteClusterStateService( String nodeId, Supplier repositoriesService, Settings settings, - ClusterService clusterService, + ClusterSettings clusterSettings, LongSupplier relativeTimeNanosSupplier, ThreadPool threadPool, List indexMetadataUploadListeners @@ -241,7 +243,6 @@ public RemoteClusterStateService( this.settings = settings; this.relativeTimeNanosSupplier = relativeTimeNanosSupplier; this.threadpool = threadPool; - ClusterSettings clusterSettings = clusterService.getClusterSettings(); this.slowWriteLoggingThreshold = clusterSettings.get(SLOW_WRITE_LOGGING_THRESHOLD); this.indexMetadataUploadTimeout = clusterSettings.get(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING); this.globalMetadataUploadTimeout = clusterSettings.get(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING); @@ -251,10 +252,16 @@ public RemoteClusterStateService( clusterSettings.addSettingsUpdateConsumer(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, this::setGlobalMetadataUploadTimeout); clusterSettings.addSettingsUpdateConsumer(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, this::setMetadataManifestUploadTimeout); this.remoteStateStats = new RemotePersistenceStats(); - this.remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(this, clusterService); this.indexMetadataUploadListeners = indexMetadataUploadListeners; } + private BlobStoreTransferService getBlobStoreTransferService() { + if (blobStoreTransferService == null) { + blobStoreTransferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadpool); + } + return blobStoreTransferService; + } + /** * This method uploads entire cluster state metadata to the configured blob store. For now only index metadata upload is supported. This method should be * invoked by the elected cluster manager when the remote cluster state is enabled. @@ -410,6 +417,7 @@ public ClusterMetadataManifest writeIncrementalMetadata( : previousManifest.getCustomMetadataMap(), false ); + deleteStaleClusterMetadata(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), RETAINED_MANIFESTS); final long durationMillis = TimeValue.nsecToMSec(relativeTimeNanosSupplier.getAsLong() - startTimeNanos); remoteStateStats.stateSucceeded(); @@ -713,10 +721,6 @@ private CheckedRunnable getAsyncMetadataWriteAction( ); } - public RemoteClusterStateCleanupManager getCleanupManager() { - return remoteClusterStateCleanupManager; - } - @Nullable public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterState, ClusterMetadataManifest previousManifest) throws IOException { @@ -736,16 +740,12 @@ public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterStat previousManifest.getCustomMetadataMap(), true ); - if (!previousManifest.isClusterUUIDCommitted() && committedManifest.isClusterUUIDCommitted()) { - remoteClusterStateCleanupManager.deleteStaleClusterUUIDs(clusterState, committedManifest); - } - + deleteStaleClusterUUIDs(clusterState, committedManifest); return committedManifest; } @Override public void close() throws IOException { - remoteClusterStateCleanupManager.close(); if (blobStoreRepository != null) { IOUtils.close(blobStoreRepository); } @@ -760,7 +760,6 @@ public void start() { final Repository repository = repositoriesService.get().repository(remoteStoreRepo); assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; blobStoreRepository = (BlobStoreRepository) repository; - remoteClusterStateCleanupManager.start(); } private ClusterMetadataManifest uploadManifest( @@ -851,14 +850,6 @@ private void writeMetadataManifest(String clusterName, String clusterUUID, Clust ); } - ThreadPool getThreadpool() { - return threadpool; - } - - BlobStore getBlobStore() { - return blobStoreRepository.blobStore(); - } - private BlobContainer indexMetadataContainer(String clusterName, String clusterUUID, String indexUUID) { // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index/ftqsCnn9TgOX return blobStoreRepository.blobStore() @@ -876,7 +867,7 @@ private BlobContainer manifestContainer(String clusterName, String clusterUUID) return blobStoreRepository.blobStore().blobContainer(getManifestFolderPath(clusterName, clusterUUID)); } - BlobPath getCusterMetadataBasePath(String clusterName, String clusterUUID) { + private BlobPath getCusterMetadataBasePath(String clusterName, String clusterUUID) { return blobStoreRepository.basePath().add(encodeString(clusterName)).add(CLUSTER_STATE_PATH_TOKEN).add(clusterUUID); } @@ -991,7 +982,7 @@ private static String metadataAttributeFileName(String componentPrefix, Long met ); } - BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { + private BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { return getCusterMetadataBasePath(clusterName, clusterUUID).add(MANIFEST_PATH_TOKEN); } @@ -1244,7 +1235,7 @@ public String getLastKnownUUIDFromRemote(String clusterName) { } } - Set getAllClusterUUIDs(String clusterName) throws IOException { + private Set getAllClusterUUIDs(String clusterName) throws IOException { Map clusterUUIDMetadata = clusterUUIDContainer(clusterName).children(); if (clusterUUIDMetadata == null) { return Collections.emptySet(); @@ -1435,7 +1426,7 @@ private Optional getLatestManifestFileName(String clusterName, String cl * @param clusterName name of the cluster * @return ClusterMetadataManifest */ - ClusterMetadataManifest fetchRemoteClusterMetadataManifest(String clusterName, String clusterUUID, String filename) + private ClusterMetadataManifest fetchRemoteClusterMetadataManifest(String clusterName, String clusterUUID, String filename) throws IllegalStateException { try { return getClusterMetadataManifestBlobStoreFormat(filename).read( @@ -1495,6 +1486,234 @@ public RemoteStateTransferException(String errorDesc, Throwable cause) { } } + /** + * Purges all remote cluster state against provided cluster UUIDs + * + * @param clusterName name of the cluster + * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged + */ + void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { + clusterUUIDs.forEach(clusterUUID -> { + getBlobStoreTransferService().deleteAsync( + ThreadPool.Names.REMOTE_PURGE, + getCusterMetadataBasePath(clusterName, clusterUUID), + new ActionListener<>() { + @Override + public void onResponse(Void unused) { + logger.info("Deleted all remote cluster metadata for cluster UUID - {}", clusterUUID); + } + + @Override + public void onFailure(Exception e) { + logger.error( + new ParameterizedMessage( + "Exception occurred while deleting all remote cluster metadata for cluster UUID {}", + clusterUUID + ), + e + ); + remoteStateStats.cleanUpAttemptFailed(); + } + } + ); + }); + } + + /** + * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests + * + * @param clusterName name of the cluster + * @param clusterUUID uuid of cluster state to refer to in remote + * @param manifestsToRetain no of latest manifest files to keep in remote + */ + // package private for testing + void deleteStaleClusterMetadata(String clusterName, String clusterUUID, int manifestsToRetain) { + if (deleteStaleMetadataRunning.compareAndSet(false, true) == false) { + logger.info("Delete stale cluster metadata task is already in progress."); + return; + } + try { + getBlobStoreTransferService().listAllInSortedOrderAsync( + ThreadPool.Names.REMOTE_PURGE, + getManifestFolderPath(clusterName, clusterUUID), + "manifest", + Integer.MAX_VALUE, + new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + if (blobMetadata.size() > manifestsToRetain) { + deleteClusterMetadata( + clusterName, + clusterUUID, + blobMetadata.subList(0, manifestsToRetain - 1), + blobMetadata.subList(manifestsToRetain - 1, blobMetadata.size()) + ); + } + deleteStaleMetadataRunning.set(false); + } + + @Override + public void onFailure(Exception e) { + logger.error( + new ParameterizedMessage( + "Exception occurred while deleting Remote Cluster Metadata for clusterUUIDs {}", + clusterUUID + ) + ); + deleteStaleMetadataRunning.set(false); + } + } + ); + } catch (Exception e) { + deleteStaleMetadataRunning.set(false); + throw e; + } + } + + private void deleteClusterMetadata( + String clusterName, + String clusterUUID, + List activeManifestBlobMetadata, + List staleManifestBlobMetadata + ) { + try { + Set filesToKeep = new HashSet<>(); + Set staleManifestPaths = new HashSet<>(); + Set staleIndexMetadataPaths = new HashSet<>(); + Set staleGlobalMetadataPaths = new HashSet<>(); + activeManifestBlobMetadata.forEach(blobMetadata -> { + ClusterMetadataManifest clusterMetadataManifest = fetchRemoteClusterMetadataManifest( + clusterName, + clusterUUID, + blobMetadata.name() + ); + clusterMetadataManifest.getIndices() + .forEach(uploadedIndexMetadata -> filesToKeep.add(uploadedIndexMetadata.getUploadedFilename())); + if (clusterMetadataManifest.getGlobalMetadataFileName() != null) { + filesToKeep.add(clusterMetadataManifest.getGlobalMetadataFileName()); + } else { + filesToKeep.add(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); + filesToKeep.add(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); + filesToKeep.add(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); + clusterMetadataManifest.getCustomMetadataMap() + .forEach((key, value) -> { filesToKeep.add(value.getUploadedFilename()); }); + } + }); + staleManifestBlobMetadata.forEach(blobMetadata -> { + ClusterMetadataManifest clusterMetadataManifest = fetchRemoteClusterMetadataManifest( + clusterName, + clusterUUID, + blobMetadata.name() + ); + staleManifestPaths.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blobMetadata.name()); + if (clusterMetadataManifest.getGlobalMetadataFileName() != null) { + if (filesToKeep.contains(clusterMetadataManifest.getGlobalMetadataFileName()) == false) { + String[] globalMetadataSplitPath = clusterMetadataManifest.getGlobalMetadataFileName().split("/"); + staleGlobalMetadataPaths.add( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( + globalMetadataSplitPath[globalMetadataSplitPath.length - 1] + ) + ); + } + } else { + if (filesToKeep.contains(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()) == false) { + String[] coordinationMetadataSplitPath = clusterMetadataManifest.getCoordinationMetadata() + .getUploadedFilename() + .split("/"); + staleGlobalMetadataPaths.add( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( + coordinationMetadataSplitPath[coordinationMetadataSplitPath.length - 1] + ) + ); + } + if (filesToKeep.contains(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()) == false) { + String[] templatesMetadataSplitPath = clusterMetadataManifest.getTemplatesMetadata() + .getUploadedFilename() + .split("/"); + staleGlobalMetadataPaths.add( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( + templatesMetadataSplitPath[templatesMetadataSplitPath.length - 1] + ) + ); + } + if (filesToKeep.contains(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()) == false) { + String[] settingsMetadataSplitPath = clusterMetadataManifest.getSettingsMetadata().getUploadedFilename().split("/"); + staleGlobalMetadataPaths.add( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( + settingsMetadataSplitPath[settingsMetadataSplitPath.length - 1] + ) + ); + } + clusterMetadataManifest.getCustomMetadataMap().forEach((key, value) -> { + if (filesToKeep.contains(value.getUploadedFilename()) == false) { + String[] customMetadataSplitPath = value.getUploadedFilename().split("/"); + staleGlobalMetadataPaths.add( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( + customMetadataSplitPath[customMetadataSplitPath.length - 1] + ) + ); + } + }); + } + + clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { + if (filesToKeep.contains(uploadedIndexMetadata.getUploadedFilename()) == false) { + staleIndexMetadataPaths.add( + new BlobPath().add(INDEX_PATH_TOKEN).add(uploadedIndexMetadata.getIndexUUID()).buildAsString() + + INDEX_METADATA_FORMAT.blobName(uploadedIndexMetadata.getUploadedFilename()) + ); + } + }); + }); + + if (staleManifestPaths.isEmpty()) { + logger.debug("No stale Remote Cluster Metadata files found"); + return; + } + + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleGlobalMetadataPaths)); + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleIndexMetadataPaths)); + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleManifestPaths)); + } catch (IllegalStateException e) { + logger.error("Error while fetching Remote Cluster Metadata manifests", e); + } catch (IOException e) { + logger.error("Error while deleting stale Remote Cluster Metadata files", e); + remoteStateStats.cleanUpAttemptFailed(); + } catch (Exception e) { + logger.error("Unexpected error while deleting stale Remote Cluster Metadata files", e); + remoteStateStats.cleanUpAttemptFailed(); + } + } + + private void deleteStalePaths(String clusterName, String clusterUUID, List stalePaths) throws IOException { + logger.debug(String.format(Locale.ROOT, "Deleting stale files from remote - %s", stalePaths)); + getBlobStoreTransferService().deleteBlobs(getCusterMetadataBasePath(clusterName, clusterUUID), stalePaths); + } + + /** + * Purges all remote cluster state against provided cluster UUIDs + * + * @param clusterState current state of the cluster + * @param committedManifest last committed ClusterMetadataManifest + */ + public void deleteStaleClusterUUIDs(ClusterState clusterState, ClusterMetadataManifest committedManifest) { + threadpool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { + String clusterName = clusterState.getClusterName().value(); + logger.debug("Deleting stale cluster UUIDs data from remote [{}]", clusterName); + Set allClustersUUIDsInRemote; + try { + allClustersUUIDsInRemote = new HashSet<>(getAllClusterUUIDs(clusterState.getClusterName().value())); + } catch (IOException e) { + logger.info(String.format(Locale.ROOT, "Error while fetching all cluster UUIDs for [%s]", clusterName)); + return; + } + // Retain last 2 cluster uuids data + allClustersUUIDsInRemote.remove(committedManifest.getClusterUUID()); + allClustersUUIDsInRemote.remove(committedManifest.getPreviousClusterUUID()); + deleteStaleUUIDsClusterMetadata(clusterName, new ArrayList<>(allClustersUUIDsInRemote)); + }); + } + public RemotePersistenceStats getStats() { return remoteStateStats; } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 49545fa8a0c8b..76109ba10624a 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -138,7 +138,6 @@ import org.opensearch.gateway.MetaStateService; import org.opensearch.gateway.PersistedClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; -import org.opensearch.gateway.remote.RemoteClusterStateCleanupManager; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpServerTransport; import org.opensearch.identity.IdentityService; @@ -753,7 +752,6 @@ protected Node( threadPool::relativeTimeInMillis ); final RemoteClusterStateService remoteClusterStateService; - final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; final RemoteIndexPathUploader remoteIndexPathUploader; if (isRemoteStoreClusterStateEnabled(settings)) { remoteIndexPathUploader = new RemoteIndexPathUploader( @@ -766,16 +764,14 @@ protected Node( nodeEnvironment.nodeId(), repositoriesServiceReference::get, settings, - clusterService, + clusterService.getClusterSettings(), threadPool::preciseRelativeTimeInNanos, threadPool, List.of(remoteIndexPathUploader) ); - remoteClusterStateCleanupManager = remoteClusterStateService.getCleanupManager(); } else { remoteClusterStateService = null; remoteIndexPathUploader = null; - remoteClusterStateCleanupManager = null; } // collect engine factory providers from plugins @@ -1380,7 +1376,6 @@ protected Node( b.bind(MetricsRegistry.class).toInstance(metricsRegistry); b.bind(RemoteClusterStateService.class).toProvider(() -> remoteClusterStateService); b.bind(RemoteIndexPathUploader.class).toProvider(() -> remoteIndexPathUploader); - b.bind(RemoteClusterStateCleanupManager.class).toProvider(() -> remoteClusterStateCleanupManager); b.bind(PersistedStateRegistry.class).toInstance(persistedStateRegistry); b.bind(SegmentReplicationStatsTracker.class).toInstance(segmentReplicationStatsTracker); b.bind(SearchRequestOperationsCompositeListenerFactory.class).toInstance(searchRequestOperationsCompositeListenerFactory); diff --git a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java index 418e6d8de6adb..3ba98c44f8d3e 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java @@ -462,7 +462,9 @@ public void testDataOnlyNodePersistence() throws Exception { }); when(transportService.getThreadPool()).thenReturn(threadPool); ClusterService clusterService = mock(ClusterService.class); - when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); + when(clusterService.getClusterSettings()).thenReturn( + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); final PersistedClusterStateService persistedClusterStateService = new PersistedClusterStateService( nodeEnvironment, xContentRegistry(), @@ -485,7 +487,7 @@ public void testDataOnlyNodePersistence() throws Exception { nodeEnvironment.nodeId(), repositoriesServiceSupplier, settings, - clusterService, + clusterSettings, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java deleted file mode 100644 index 24fd1b164a4ff..0000000000000 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java +++ /dev/null @@ -1,446 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.gateway.remote; - -import org.opensearch.cluster.ClusterName; -import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.service.ClusterApplierService; -import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.blobstore.BlobContainer; -import org.opensearch.common.blobstore.BlobMetadata; -import org.opensearch.common.blobstore.BlobPath; -import org.opensearch.common.blobstore.BlobStore; -import org.opensearch.common.blobstore.support.PlainBlobMetadata; -import org.opensearch.common.settings.ClusterSettings; -import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.concurrent.AbstractAsyncTask; -import org.opensearch.core.action.ActionListener; -import org.opensearch.repositories.RepositoriesService; -import org.opensearch.repositories.blobstore.BlobStoreRepository; -import org.opensearch.repositories.fs.FsRepository; -import org.opensearch.test.OpenSearchTestCase; -import org.opensearch.test.VersionUtils; -import org.opensearch.threadpool.TestThreadPool; -import org.opensearch.threadpool.ThreadPool; -import org.junit.After; -import org.junit.Before; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; - -import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; -import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; -import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; -import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.AsyncStaleFileDeletion; -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; -import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; -import static org.opensearch.gateway.remote.RemoteClusterStateService.CLUSTER_STATE_PATH_TOKEN; -import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; -import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; -import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_PATH_TOKEN; -import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_PATH_TOKEN; -import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; -import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_PATH_TOKEN; -import static org.opensearch.gateway.remote.RemoteClusterStateService.SETTING_METADATA; -import static org.opensearch.gateway.remote.RemoteClusterStateService.TEMPLATES_METADATA; -import static org.opensearch.gateway.remote.RemoteClusterStateService.encodeString; -import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; -import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.nodesWithLocalNodeClusterManager; -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.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class RemoteClusterStateCleanupManagerTests extends OpenSearchTestCase { - private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; - private Supplier repositoriesServiceSupplier; - private RepositoriesService repositoriesService; - private BlobStoreRepository blobStoreRepository; - private BlobStore blobStore; - private ClusterSettings clusterSettings; - private ClusterApplierService clusterApplierService; - private ClusterState clusterState; - private Metadata metadata; - private RemoteClusterStateService remoteClusterStateService; - private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); - - @Before - public void setup() { - repositoriesServiceSupplier = mock(Supplier.class); - repositoriesService = mock(RepositoriesService.class); - when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); - - String stateRepoTypeAttributeKey = String.format( - Locale.getDefault(), - "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, - "remote_store_repository" - ); - String stateRepoSettingsAttributeKeyPrefix = String.format( - Locale.getDefault(), - "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, - "remote_store_repository" - ); - - Settings settings = Settings.builder() - .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "remote_store_repository") - .put(stateRepoTypeAttributeKey, FsRepository.TYPE) - .put(stateRepoSettingsAttributeKeyPrefix + "location", "randomRepoPath") - .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) - .build(); - - clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - clusterApplierService = mock(ClusterApplierService.class); - clusterState = mock(ClusterState.class); - metadata = mock(Metadata.class); - ClusterService clusterService = mock(ClusterService.class); - when(clusterService.getClusterSettings()).thenReturn(clusterSettings); - when(clusterState.getClusterName()).thenReturn(new ClusterName("test")); - when(metadata.clusterUUID()).thenReturn("testUUID"); - when(clusterState.metadata()).thenReturn(metadata); - when(clusterApplierService.state()).thenReturn(clusterState); - when(clusterService.getClusterApplierService()).thenReturn(clusterApplierService); - - blobStoreRepository = mock(BlobStoreRepository.class); - blobStore = mock(BlobStore.class); - when(blobStoreRepository.blobStore()).thenReturn(blobStore); - when(repositoriesService.repository("remote_store_repository")).thenReturn(blobStoreRepository); - - remoteClusterStateService = mock(RemoteClusterStateService.class); - when(remoteClusterStateService.getStats()).thenReturn(new RemotePersistenceStats()); - when(remoteClusterStateService.getThreadpool()).thenReturn(threadPool); - when(remoteClusterStateService.getBlobStore()).thenReturn(blobStore); - remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(remoteClusterStateService, clusterService); - } - - @After - public void teardown() throws Exception { - super.tearDown(); - remoteClusterStateCleanupManager.close(); - threadPool.shutdown(); - } - - public void testDeleteClusterMetadata() throws IOException { - String clusterUUID = "clusterUUID"; - String clusterName = "test-cluster"; - List inactiveBlobs = Arrays.asList( - new PlainBlobMetadata("manifest1.dat", 1L), - new PlainBlobMetadata("manifest2.dat", 1L), - new PlainBlobMetadata("manifest3.dat", 1L) - ); - List activeBlobs = Arrays.asList( - new PlainBlobMetadata("manifest4.dat", 1L), - new PlainBlobMetadata("manifest5.dat", 1L) - ); - UploadedIndexMetadata index1Metadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1"); - UploadedIndexMetadata index2Metadata = new UploadedIndexMetadata("index2", "indexUUID2", "index_metadata2"); - UploadedIndexMetadata index1UpdatedMetadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1_updated"); - UploadedMetadataAttribute coordinationMetadata = new UploadedMetadataAttribute(COORDINATION_METADATA, "coordination_metadata"); - UploadedMetadataAttribute templateMetadata = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata"); - UploadedMetadataAttribute settingMetadata = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata"); - UploadedMetadataAttribute coordinationMetadataUpdated = new UploadedMetadataAttribute( - COORDINATION_METADATA, - "coordination_metadata_updated" - ); - UploadedMetadataAttribute templateMetadataUpdated = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata_updated"); - UploadedMetadataAttribute settingMetadataUpdated = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata_updated"); - ClusterMetadataManifest manifest1 = ClusterMetadataManifest.builder() - .indices(List.of(index1Metadata)) - .globalMetadataFileName("global_metadata") - .clusterTerm(1L) - .stateVersion(1L) - .codecVersion(CODEC_V1) - .stateUUID(randomAlphaOfLength(10)) - .clusterUUID(clusterUUID) - .nodeId("nodeA") - .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) - .previousClusterUUID(ClusterState.UNKNOWN_UUID) - .committed(true) - .build(); - ClusterMetadataManifest manifest2 = ClusterMetadataManifest.builder(manifest1) - .indices(List.of(index1Metadata, index2Metadata)) - .codecVersion(CODEC_V2) - .globalMetadataFileName(null) - .coordinationMetadata(coordinationMetadata) - .templatesMetadata(templateMetadata) - .settingMetadata(settingMetadata) - .build(); - ClusterMetadataManifest manifest3 = ClusterMetadataManifest.builder(manifest2) - .indices(List.of(index1UpdatedMetadata, index2Metadata)) - .settingMetadata(settingMetadataUpdated) - .build(); - - // active manifest have reference to index1Updated, index2, settingsUpdated, coordinationUpdated, templates, templatesUpdated - ClusterMetadataManifest manifest4 = ClusterMetadataManifest.builder(manifest3) - .coordinationMetadata(coordinationMetadataUpdated) - .build(); - ClusterMetadataManifest manifest5 = ClusterMetadataManifest.builder(manifest4).templatesMetadata(templateMetadataUpdated).build(); - - when(remoteClusterStateService.fetchRemoteClusterMetadataManifest(eq(clusterName), eq(clusterUUID), any())).thenReturn( - manifest4, - manifest5, - manifest1, - manifest2, - manifest3 - ); - BlobContainer container = mock(BlobContainer.class); - when(blobStore.blobContainer(any())).thenReturn(container); - doNothing().when(container).deleteBlobsIgnoringIfNotExists(any()); - - remoteClusterStateCleanupManager.deleteClusterMetadata(clusterName, clusterUUID, activeBlobs, inactiveBlobs); - verify(container).deleteBlobsIgnoringIfNotExists( - List.of( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + coordinationMetadata.getUploadedFilename() + ".dat", - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + settingMetadata.getUploadedFilename() + ".dat", - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + "global_metadata.dat" - ) - ); - verify(container).deleteBlobsIgnoringIfNotExists( - List.of( - new BlobPath().add(INDEX_PATH_TOKEN).add(index1Metadata.getIndexUUID()).buildAsString() - + index1Metadata.getUploadedFilePath() - + ".dat" - ) - ); - Set staleManifest = new HashSet<>(); - inactiveBlobs.forEach(blob -> staleManifest.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blob.name())); - verify(container).deleteBlobsIgnoringIfNotExists(new ArrayList<>(staleManifest)); - } - - public void testDeleteStaleClusterUUIDs() throws IOException { - final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); - ClusterMetadataManifest clusterMetadataManifest = ClusterMetadataManifest.builder() - .indices(List.of()) - .clusterTerm(1L) - .stateVersion(1L) - .stateUUID(randomAlphaOfLength(10)) - .clusterUUID("cluster-uuid1") - .nodeId("nodeA") - .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) - .previousClusterUUID(ClusterState.UNKNOWN_UUID) - .committed(true) - .build(); - - BlobPath blobPath = new BlobPath().add("random-path"); - BlobContainer uuidContainerContainer = mock(BlobContainer.class); - BlobContainer manifest2Container = mock(BlobContainer.class); - BlobContainer manifest3Container = mock(BlobContainer.class); - when(blobStore.blobContainer(any())).then(invocation -> { - BlobPath blobPath1 = invocation.getArgument(0); - if (blobPath1.buildAsString().endsWith("cluster-state/")) { - return uuidContainerContainer; - } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid2/")) { - return manifest2Container; - } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid3/")) { - return manifest3Container; - } else { - throw new IllegalArgumentException("Unexpected blob path " + blobPath1); - } - }); - when( - manifest2Container.listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - Integer.MAX_VALUE, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(List.of(new PlainBlobMetadata("mainfest2", 1L))); - when( - manifest3Container.listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - Integer.MAX_VALUE, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(List.of(new PlainBlobMetadata("mainfest3", 1L))); - Set uuids = new HashSet<>(Arrays.asList("cluster-uuid1", "cluster-uuid2", "cluster-uuid3")); - when(remoteClusterStateService.getAllClusterUUIDs(any())).thenReturn(uuids); - when(remoteClusterStateService.getCusterMetadataBasePath(any(), any())).then( - invocationOnMock -> blobPath.add(encodeString(invocationOnMock.getArgument(0))) - .add(CLUSTER_STATE_PATH_TOKEN) - .add((String) invocationOnMock.getArgument(1)) - ); - remoteClusterStateCleanupManager.start(); - remoteClusterStateCleanupManager.deleteStaleClusterUUIDs(clusterState, clusterMetadataManifest); - try { - assertBusy(() -> { - verify(manifest2Container, times(1)).delete(); - verify(manifest3Container, times(1)).delete(); - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void testRemoteStateCleanupFailureStats() throws IOException { - BlobContainer blobContainer = mock(BlobContainer.class); - doThrow(IOException.class).when(blobContainer).delete(); - when(blobStore.blobContainer(any())).thenReturn(blobContainer); - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - remoteClusterStateCleanupManager.start(); - remoteClusterStateCleanupManager.deleteStaleUUIDsClusterMetadata("cluster1", List.of("cluster-uuid1")); - try { - assertBusy(() -> { - // wait for stats to get updated - assertNotNull(remoteClusterStateCleanupManager.getStats()); - assertEquals(0, remoteClusterStateCleanupManager.getStats().getSuccessCount()); - assertEquals(1, remoteClusterStateCleanupManager.getStats().getCleanupAttemptFailedCount()); - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { - BlobContainer blobContainer = mock(BlobContainer.class); - when(blobStore.blobContainer(any())).thenReturn(blobContainer); - - CountDownLatch latch = new CountDownLatch(1); - AtomicInteger callCount = new AtomicInteger(0); - doAnswer(invocation -> { - callCount.incrementAndGet(); - if (latch.await(5000, TimeUnit.SECONDS) == false) { - throw new Exception("Timed out waiting for delete task queuing to complete"); - } - return null; - }).when(blobContainer) - .listBlobsByPrefixInSortedOrder( - any(String.class), - any(int.class), - any(BlobContainer.BlobNameSortOrder.class), - any(ActionListener.class) - ); - - remoteClusterStateCleanupManager.start(); - remoteClusterStateCleanupManager.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); - remoteClusterStateCleanupManager.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); - - latch.countDown(); - assertBusy(() -> assertEquals(1, callCount.get())); - } - - public void testRemoteClusterStateCleanupSetting() { - remoteClusterStateCleanupManager.start(); - // verify default value - assertEquals(CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, remoteClusterStateCleanupManager.getStaleFileCleanupInterval()); - - // verify update interval - int cleanupInterval = randomIntBetween(1, 10); - Settings newSettings = Settings.builder().put("cluster.remote_store.state.cleanup_interval", cleanupInterval + "s").build(); - clusterSettings.applySettings(newSettings); - assertEquals(cleanupInterval, remoteClusterStateCleanupManager.getStaleFileCleanupInterval().seconds()); - } - - public void testRemoteCleanupTaskScheduled() { - AbstractAsyncTask cleanupTask = remoteClusterStateCleanupManager.getStaleFileDeletionTask(); - assertNull(cleanupTask); - // now the task should be initialized - remoteClusterStateCleanupManager.start(); - assertNotNull(remoteClusterStateCleanupManager.getStaleFileDeletionTask()); - assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().mustReschedule()); - assertEquals( - clusterSettings.get(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING), - remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval() - ); - assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); - assertFalse(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isClosed()); - } - - public void testRemoteCleanupSkipsOnOnlyElectedClusterManager() { - DiscoveryNodes nodes = mock(DiscoveryNodes.class); - when(nodes.isLocalNodeElectedClusterManager()).thenReturn(false); - when(clusterState.nodes()).thenReturn(nodes); - RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); - AtomicInteger callCount = new AtomicInteger(0); - doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); - spyManager.cleanUpStaleFiles(); - assertEquals(0, callCount.get()); - - when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); - when(clusterState.version()).thenReturn(randomLongBetween(11, 20)); - spyManager.cleanUpStaleFiles(); - assertEquals(1, callCount.get()); - } - - public void testRemoteCleanupSkipsIfVersionIncrementLessThanThreshold() { - DiscoveryNodes nodes = mock(DiscoveryNodes.class); - long version = randomLongBetween(1, SKIP_CLEANUP_STATE_CHANGES); - when(clusterApplierService.state()).thenReturn(clusterState); - when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); - when(clusterState.nodes()).thenReturn(nodes); - when(clusterState.version()).thenReturn(version); - - RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); - AtomicInteger callCount = new AtomicInteger(0); - doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); - - remoteClusterStateCleanupManager.cleanUpStaleFiles(); - assertEquals(0, callCount.get()); - } - - public void testRemoteCleanupCallsDeleteIfVersionIncrementGreaterThanThreshold() { - DiscoveryNodes nodes = mock(DiscoveryNodes.class); - long version = randomLongBetween(SKIP_CLEANUP_STATE_CHANGES + 1, SKIP_CLEANUP_STATE_CHANGES + 10); - when(clusterApplierService.state()).thenReturn(clusterState); - when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); - when(clusterState.nodes()).thenReturn(nodes); - when(clusterState.version()).thenReturn(version); - - RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); - AtomicInteger callCount = new AtomicInteger(0); - doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); - - // using spied cleanup manager so that stubbed deleteStaleClusterMetadata is called - spyManager.cleanUpStaleFiles(); - assertEquals(1, callCount.get()); - } - - public void testRemoteCleanupSchedulesEvenAfterFailure() { - remoteClusterStateCleanupManager.start(); - RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); - AtomicInteger callCount = new AtomicInteger(0); - doAnswer(invocationOnMock -> { - callCount.incrementAndGet(); - throw new RuntimeException("Test exception"); - }).when(spyManager).cleanUpStaleFiles(); - AsyncStaleFileDeletion task = new AsyncStaleFileDeletion(spyManager); - assertTrue(task.isScheduled()); - task.run(); - // Task is still scheduled after the failure - assertTrue(task.isScheduled()); - assertEquals(1, callCount.get()); - - task.run(); - // Task is still scheduled after the failure - assertTrue(task.isScheduled()); - assertEquals(2, callCount.get()); - } -} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index 5f0c371a3137e..1b242b921c0d7 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -19,7 +19,6 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; @@ -73,6 +72,9 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.Supplier; @@ -91,6 +93,7 @@ import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_CURRENT_CODEC_VERSION; import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; import static org.opensearch.gateway.remote.RemoteClusterStateService.METADATA_FILE_PREFIX; +import static org.opensearch.gateway.remote.RemoteClusterStateService.RETAINED_MANIFESTS; import static org.opensearch.gateway.remote.RemoteClusterStateService.SETTING_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.TEMPLATES_METADATA; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; @@ -106,12 +109,13 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class RemoteClusterStateServiceTests extends OpenSearchTestCase { private RemoteClusterStateService remoteClusterStateService; - private ClusterService clusterService; private ClusterSettings clusterSettings; private Supplier repositoriesServiceSupplier; private RepositoriesService repositoriesService; @@ -144,8 +148,6 @@ public void setup() { .build(); clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - clusterService = mock(ClusterService.class); - when(clusterService.getClusterSettings()).thenReturn(clusterSettings); NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( Stream.of( NetworkModule.getNamedXContents().stream(), @@ -163,7 +165,7 @@ public void setup() { "test-node-id", repositoriesServiceSupplier, settings, - clusterService, + clusterSettings, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) @@ -185,14 +187,14 @@ public void testFailWriteFullMetadataNonClusterManagerNode() throws IOException public void testFailInitializationWhenRemoteStateDisabled() { final Settings settings = Settings.builder().build(); - when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); assertThrows( AssertionError.class, () -> new RemoteClusterStateService( "test-node-id", repositoriesServiceSupplier, settings, - clusterService, + clusterSettings, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) @@ -1278,6 +1280,72 @@ public void testGetValidPreviousClusterUUIDWhenLastUUIDUncommitted() throws IOEx assertThat(previousClusterUUID, equalTo("cluster-uuid2")); } + public void testDeleteStaleClusterUUIDs() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + ClusterMetadataManifest clusterMetadataManifest = ClusterMetadataManifest.builder() + .indices(List.of()) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID("cluster-uuid1") + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .build(); + + BlobPath blobPath = new BlobPath().add("random-path"); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + BlobContainer uuidContainerContainer = mock(BlobContainer.class); + BlobContainer manifest2Container = mock(BlobContainer.class); + BlobContainer manifest3Container = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).then(invocation -> { + BlobPath blobPath1 = invocation.getArgument(0); + if (blobPath1.buildAsString().endsWith("cluster-state/")) { + return uuidContainerContainer; + } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid2/")) { + return manifest2Container; + } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid3/")) { + return manifest3Container; + } else { + throw new IllegalArgumentException("Unexpected blob path " + blobPath1); + } + }); + Map blobMetadataMap = Map.of( + "cluster-uuid1", + mock(BlobContainer.class), + "cluster-uuid2", + mock(BlobContainer.class), + "cluster-uuid3", + mock(BlobContainer.class) + ); + when(uuidContainerContainer.children()).thenReturn(blobMetadataMap); + when( + manifest2Container.listBlobsByPrefixInSortedOrder( + MANIFEST_FILE_PREFIX + DELIMITER, + Integer.MAX_VALUE, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ) + ).thenReturn(List.of(new PlainBlobMetadata("mainfest2", 1L))); + when( + manifest3Container.listBlobsByPrefixInSortedOrder( + MANIFEST_FILE_PREFIX + DELIMITER, + Integer.MAX_VALUE, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ) + ).thenReturn(List.of(new PlainBlobMetadata("mainfest3", 1L))); + remoteClusterStateService.start(); + remoteClusterStateService.deleteStaleClusterUUIDs(clusterState, clusterMetadataManifest); + try { + assertBusy(() -> { + verify(manifest2Container, times(1)).delete(); + verify(manifest3Container, times(1)).delete(); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + public void testRemoteStateStats() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); mockBlobStoreObjects(); @@ -1290,6 +1358,26 @@ public void testRemoteStateStats() throws IOException { assertEquals(0, remoteClusterStateService.getStats().getFailedCount()); } + public void testRemoteStateCleanupFailureStats() throws IOException { + BlobContainer blobContainer = mock(BlobContainer.class); + doThrow(IOException.class).when(blobContainer).delete(); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + BlobPath blobPath = new BlobPath().add("random-path"); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + remoteClusterStateService.start(); + remoteClusterStateService.deleteStaleUUIDsClusterMetadata("cluster1", Arrays.asList("cluster-uuid1")); + try { + assertBusy(() -> { + // wait for stats to get updated + assertTrue(remoteClusterStateService.getStats() != null); + assertEquals(0, remoteClusterStateService.getStats().getSuccessCount()); + assertEquals(1, remoteClusterStateService.getStats().getCleanupAttemptFailedCount()); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + public void testFileNames() { final Index index = new Index("test-index", "index-uuid"); final Settings idxSettings = Settings.builder() @@ -1330,6 +1418,36 @@ private void verifyManifestFileNameWithCodec(int codecVersion) { assertThat(splittedName[3], is("P")); } + public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { + BlobContainer blobContainer = mock(BlobContainer.class); + BlobPath blobPath = new BlobPath().add("random-path"); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> { + callCount.incrementAndGet(); + if (latch.await(5000, TimeUnit.SECONDS) == false) { + throw new Exception("Timed out waiting for delete task queuing to complete"); + } + return null; + }).when(blobContainer) + .listBlobsByPrefixInSortedOrder( + any(String.class), + any(int.class), + any(BlobContainer.BlobNameSortOrder.class), + any(ActionListener.class) + ); + + remoteClusterStateService.start(); + remoteClusterStateService.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); + remoteClusterStateService.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); + + latch.countDown(); + assertBusy(() -> assertEquals(1, callCount.get())); + } + public void testIndexMetadataUploadWaitTimeSetting() { // verify default value assertEquals( @@ -1773,7 +1891,7 @@ private static ClusterState.Builder generateClusterStateWithGlobalMetadata() { ); } - static ClusterState.Builder generateClusterStateWithOneIndex() { + private static ClusterState.Builder generateClusterStateWithOneIndex() { final Index index = new Index("test-index", "index-uuid"); final Settings idxSettings = Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) @@ -1803,7 +1921,7 @@ static ClusterState.Builder generateClusterStateWithOneIndex() { ); } - static DiscoveryNodes nodesWithLocalNodeClusterManager() { + private static DiscoveryNodes nodesWithLocalNodeClusterManager() { return DiscoveryNodes.builder().clusterManagerNodeId("cluster-manager-id").localNodeId("cluster-manager-id").build(); } From 39ae32a9347dc4dc0847ad0093b272b9eaec5ca0 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Tue, 4 Jun 2024 16:09:18 -0700 Subject: [PATCH 45/77] [Tiered Caching] Additional ITs for cache stats (#13655) * Adds cache clear IT Signed-off-by: Peter Alfonsi Cleaned up logic for cache stats ITs Signed-off-by: Peter Alfonsi Adds more tests around tiered spillover cache Signed-off-by: Peter Alfonsi Fixed cache stats behavior for overall /_nodes/stats call Signed-off-by: Peter Alfonsi cleanup Signed-off-by: Peter Alfonsi Fixed folder structure Signed-off-by: Peter Alfonsi Addressed Sagar's comments Signed-off-by: Peter Alfonsi Addressed Ankit's comments Signed-off-by: Peter Alfonsi Break horrifyingly long test case into many shorter cases Signed-off-by: Peter Alfonsi Added unsupported operation exception to TSC stats holder incrementEvictions() Signed-off-by: Peter Alfonsi Addressed Sorabh's comments Signed-off-by: Peter Alfonsi * rerun assemble Signed-off-by: Peter Alfonsi * rerun gradle Signed-off-by: Peter Alfonsi * rerun gradle Signed-off-by: Peter Alfonsi * rerun gradle Signed-off-by: Peter Alfonsi --------- Signed-off-by: Peter Alfonsi Co-authored-by: Peter Alfonsi --- .../common/tier}/TieredSpilloverCacheIT.java | 4 +- .../tier/TieredSpilloverCacheStatsIT.java | 501 ++++++++++++++++++ .../common/tier/TieredSpilloverCache.java | 16 +- .../tier/TieredSpilloverCacheStatsHolder.java | 29 +- .../tier/TieredSpilloverCacheTests.java | 17 +- .../CacheStatsAPIIndicesRequestCacheIT.java | 194 ++++--- .../admin/indices/stats/CommonStatsFlags.java | 2 +- .../common/cache/service/NodeCacheStats.java | 9 + .../indices/IndicesRequestCache.java | 27 +- 9 files changed, 700 insertions(+), 99 deletions(-) rename modules/cache-common/src/internalClusterTest/java/{org.opensearch.cache.common.tier => org/opensearch/cache/common/tier}/TieredSpilloverCacheIT.java (99%) create mode 100644 modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java diff --git a/modules/cache-common/src/internalClusterTest/java/org.opensearch.cache.common.tier/TieredSpilloverCacheIT.java b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheIT.java similarity index 99% rename from modules/cache-common/src/internalClusterTest/java/org.opensearch.cache.common.tier/TieredSpilloverCacheIT.java rename to modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheIT.java index bfc184cff0566..02be0990eb136 100644 --- a/modules/cache-common/src/internalClusterTest/java/org.opensearch.cache.common.tier/TieredSpilloverCacheIT.java +++ b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheIT.java @@ -65,7 +65,7 @@ protected Collection> nodePlugins() { return Arrays.asList(TieredSpilloverCachePlugin.class, MockDiskCachePlugin.class); } - private Settings defaultSettings(String onHeapCacheSizeInBytesOrPecentage) { + static Settings defaultSettings(String onHeapCacheSizeInBytesOrPercentage) { return Settings.builder() .put(FeatureFlags.PLUGGABLE_CACHE, "true") .put( @@ -88,7 +88,7 @@ private Settings defaultSettings(String onHeapCacheSizeInBytesOrPecentage) { OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) .get(MAXIMUM_SIZE_IN_BYTES_KEY) .getKey(), - onHeapCacheSizeInBytesOrPecentage + onHeapCacheSizeInBytesOrPercentage ) .build(); } diff --git a/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java new file mode 100644 index 0000000000000..537caccbac652 --- /dev/null +++ b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java @@ -0,0 +1,501 @@ +/* + * 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.common.tier; + +import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; +import org.opensearch.action.admin.indices.stats.CommonStatsFlags; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.client.Client; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.cache.CacheType; +import org.opensearch.common.cache.service.NodeCacheStats; +import org.opensearch.common.cache.stats.ImmutableCacheStats; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.index.cache.request.RequestCacheStats; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.indices.IndicesRequestCache; +import org.opensearch.plugins.Plugin; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.hamcrest.OpenSearchAssertions; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_NAME; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_DISK; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_ON_HEAP; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertSearchResponse; + +// Use a single data node to simplify accessing cache stats across different shards. +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class TieredSpilloverCacheStatsIT extends OpenSearchIntegTestCase { + @Override + protected Collection> nodePlugins() { + return Arrays.asList(TieredSpilloverCachePlugin.class, TieredSpilloverCacheIT.MockDiskCachePlugin.class); + } + + private final String HEAP_CACHE_SIZE_STRING = "10000B"; + private final int HEAP_CACHE_SIZE = 10_000; + private final String index1Name = "index1"; + private final String index2Name = "index2"; + + /** + * Test aggregating by indices + */ + public void testIndicesLevelAggregation() throws Exception { + internalCluster().startNodes( + 1, + Settings.builder() + .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put( + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), + new TimeValue(0, TimeUnit.SECONDS) + ) + .build() + ); + Client client = client(); + Map values = setupCacheForAggregationTests(client); + + ImmutableCacheStatsHolder allLevelsStatsHolder = getNodeCacheStatsResult( + client, + List.of(IndicesRequestCache.INDEX_DIMENSION_NAME, TIER_DIMENSION_NAME) + ); + ImmutableCacheStatsHolder indicesOnlyStatsHolder = getNodeCacheStatsResult( + client, + List.of(IndicesRequestCache.INDEX_DIMENSION_NAME) + ); + + // Get values for indices alone, assert these match for statsHolders that have additional dimensions vs. a statsHolder that only has + // the indices dimension + ImmutableCacheStats index1ExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats( + values.get("hitsOnHeapIndex1") + values.get("hitsOnDiskIndex1"), + values.get("itemsOnDiskIndex1AfterTest") + values.get("itemsOnHeapIndex1AfterTest"), + 0, + (values.get("itemsOnDiskIndex1AfterTest") + values.get("itemsOnHeapIndex1AfterTest")) * values.get("singleSearchSize"), + values.get("itemsOnDiskIndex1AfterTest") + values.get("itemsOnHeapIndex1AfterTest") + ) + ); + ImmutableCacheStats index2ExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats( + values.get("hitsOnHeapIndex2") + values.get("hitsOnDiskIndex2"), + values.get("itemsOnDiskIndex2AfterTest") + values.get("itemsOnHeapIndex2AfterTest"), + 0, + (values.get("itemsOnDiskIndex2AfterTest") + values.get("itemsOnHeapIndex2AfterTest")) * values.get("singleSearchSize"), + values.get("itemsOnDiskIndex2AfterTest") + values.get("itemsOnHeapIndex2AfterTest") + ) + ); + + for (ImmutableCacheStatsHolder statsHolder : List.of(allLevelsStatsHolder, indicesOnlyStatsHolder)) { + assertEquals(index1ExpectedStats, statsHolder.getStatsForDimensionValues(List.of(index1Name))); + assertEquals(index2ExpectedStats, statsHolder.getStatsForDimensionValues(List.of(index2Name))); + } + } + + /** + * Test aggregating by indices and tier + */ + public void testIndicesAndTierLevelAggregation() throws Exception { + internalCluster().startNodes( + 1, + Settings.builder() + .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put( + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), + new TimeValue(0, TimeUnit.SECONDS) + ) + .build() + ); + Client client = client(); + Map values = setupCacheForAggregationTests(client); + + ImmutableCacheStatsHolder allLevelsStatsHolder = getNodeCacheStatsResult( + client, + List.of(IndicesRequestCache.INDEX_DIMENSION_NAME, TIER_DIMENSION_NAME) + ); + + // Get values broken down by indices+tiers + ImmutableCacheStats index1HeapExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats( + values.get("hitsOnHeapIndex1"), + values.get("itemsOnHeapIndex1AfterTest") + values.get("itemsOnDiskIndex1AfterTest") + values.get("hitsOnDiskIndex1"), + values.get("itemsOnDiskIndex1AfterTest"), + values.get("itemsOnHeapIndex1AfterTest") * values.get("singleSearchSize"), + values.get("itemsOnHeapIndex1AfterTest") + ) + ); + assertEquals( + index1HeapExpectedStats, + allLevelsStatsHolder.getStatsForDimensionValues(List.of(index1Name, TIER_DIMENSION_VALUE_ON_HEAP)) + ); + + ImmutableCacheStats index2HeapExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats( + values.get("hitsOnHeapIndex2"), + values.get("itemsOnHeapIndex2AfterTest") + values.get("itemsOnDiskIndex2AfterTest") + values.get("hitsOnDiskIndex2"), + values.get("itemsOnDiskIndex2AfterTest"), + values.get("itemsOnHeapIndex2AfterTest") * values.get("singleSearchSize"), + values.get("itemsOnHeapIndex2AfterTest") + ) + ); + assertEquals( + index2HeapExpectedStats, + allLevelsStatsHolder.getStatsForDimensionValues(List.of(index2Name, TIER_DIMENSION_VALUE_ON_HEAP)) + ); + + ImmutableCacheStats index1DiskExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats( + values.get("hitsOnDiskIndex1"), + values.get("itemsOnHeapIndex1AfterTest") + values.get("itemsOnDiskIndex1AfterTest"), + 0, + values.get("itemsOnDiskIndex1AfterTest") * values.get("singleSearchSize"), + values.get("itemsOnDiskIndex1AfterTest") + ) + ); + assertEquals( + index1DiskExpectedStats, + allLevelsStatsHolder.getStatsForDimensionValues(List.of(index1Name, TIER_DIMENSION_VALUE_DISK)) + ); + + ImmutableCacheStats index2DiskExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats( + values.get("hitsOnDiskIndex2"), + values.get("itemsOnHeapIndex2AfterTest") + values.get("itemsOnDiskIndex2AfterTest"), + 0, + values.get("itemsOnDiskIndex2AfterTest") * values.get("singleSearchSize"), + values.get("itemsOnDiskIndex2AfterTest") + ) + ); + assertEquals( + index2DiskExpectedStats, + allLevelsStatsHolder.getStatsForDimensionValues(List.of(index2Name, TIER_DIMENSION_VALUE_DISK)) + ); + } + + /** + * Test aggregating by tier only + */ + public void testTierLevelAggregation() throws Exception { + internalCluster().startNodes( + 1, + Settings.builder() + .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put( + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), + new TimeValue(0, TimeUnit.SECONDS) + ) + .build() + ); + Client client = client(); + Map values = setupCacheForAggregationTests(client); + + // Get values for tiers alone and check they add correctly across indices + ImmutableCacheStatsHolder tiersOnlyStatsHolder = getNodeCacheStatsResult(client, List.of(TIER_DIMENSION_NAME)); + ImmutableCacheStats totalHeapExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats( + values.get("hitsOnHeapIndex1") + values.get("hitsOnHeapIndex2"), + values.get("itemsOnHeapAfterTest") + values.get("itemsOnDiskAfterTest") + values.get("hitsOnDiskIndex1") + values.get( + "hitsOnDiskIndex2" + ), + values.get("itemsOnDiskAfterTest"), + values.get("itemsOnHeapAfterTest") * values.get("singleSearchSize"), + values.get("itemsOnHeapAfterTest") + ) + ); + ImmutableCacheStats heapStats = tiersOnlyStatsHolder.getStatsForDimensionValues(List.of(TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(totalHeapExpectedStats, heapStats); + ImmutableCacheStats totalDiskExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats( + values.get("hitsOnDiskIndex1") + values.get("hitsOnDiskIndex2"), + values.get("itemsOnHeapAfterTest") + values.get("itemsOnDiskAfterTest"), + 0, + values.get("itemsOnDiskAfterTest") * values.get("singleSearchSize"), + values.get("itemsOnDiskAfterTest") + ) + ); + ImmutableCacheStats diskStats = tiersOnlyStatsHolder.getStatsForDimensionValues(List.of(TIER_DIMENSION_VALUE_DISK)); + assertEquals(totalDiskExpectedStats, diskStats); + } + + public void testInvalidLevelsAreIgnored() throws Exception { + internalCluster().startNodes( + 1, + Settings.builder() + .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put( + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), + new TimeValue(0, TimeUnit.SECONDS) + ) + .build() + ); + Client client = client(); + Map values = setupCacheForAggregationTests(client); + + ImmutableCacheStatsHolder allLevelsStatsHolder = getNodeCacheStatsResult( + client, + List.of(IndicesRequestCache.INDEX_DIMENSION_NAME, TIER_DIMENSION_NAME) + ); + ImmutableCacheStatsHolder indicesOnlyStatsHolder = getNodeCacheStatsResult( + client, + List.of(IndicesRequestCache.INDEX_DIMENSION_NAME) + ); + + // Test invalid levels are ignored and permuting the order of levels in the request doesn't matter + + // This should be equivalent to just "indices" + ImmutableCacheStatsHolder indicesEquivalentStatsHolder = getNodeCacheStatsResult( + client, + List.of(IndicesRequestCache.INDEX_DIMENSION_NAME, "unrecognized_dimension") + ); + assertEquals(indicesOnlyStatsHolder, indicesEquivalentStatsHolder); + + // This should be equivalent to "indices", "tier" + ImmutableCacheStatsHolder indicesAndTierEquivalentStatsHolder = getNodeCacheStatsResult( + client, + List.of(TIER_DIMENSION_NAME, "unrecognized_dimension_1", IndicesRequestCache.INDEX_DIMENSION_NAME, "unrecognized_dimension_2") + ); + assertEquals(allLevelsStatsHolder, indicesAndTierEquivalentStatsHolder); + + // This should be equivalent to no levels passed in + ImmutableCacheStatsHolder noLevelsEquivalentStatsHolder = getNodeCacheStatsResult( + client, + List.of("unrecognized_dimension_1", "unrecognized_dimension_2") + ); + ImmutableCacheStatsHolder noLevelsStatsHolder = getNodeCacheStatsResult(client, List.of()); + assertEquals(noLevelsStatsHolder, noLevelsEquivalentStatsHolder); + } + + /** + * Check the new stats API returns the same values as the old stats API. + */ + public void testStatsMatchOldApi() throws Exception { + internalCluster().startNodes( + 1, + Settings.builder() + .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put( + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), + new TimeValue(0, TimeUnit.SECONDS) + ) + .build() + ); + String index = "index"; + Client client = client(); + startIndex(client, index); + + // First search one time to see how big a single value will be + searchIndex(client, index, 0); + // get total stats + long singleSearchSize = getTotalStats(client).getSizeInBytes(); + // Select numbers so we get some values on both heap and disk + int itemsOnHeap = HEAP_CACHE_SIZE / (int) singleSearchSize; + int itemsOnDisk = 1 + randomInt(30); // The first one we search (to get the size) always goes to disk + int expectedEntries = itemsOnHeap + itemsOnDisk; + + for (int i = 1; i < expectedEntries; i++) { + // Cause misses + searchIndex(client, index, i); + } + int expectedMisses = itemsOnHeap + itemsOnDisk; + + // Cause some hits + int expectedHits = randomIntBetween(itemsOnHeap, expectedEntries); // Select it so some hits come from both tiers + for (int i = 0; i < expectedHits; i++) { + searchIndex(client, index, i); + } + + ImmutableCacheStats totalStats = getNodeCacheStatsResult(client, List.of()).getTotalStats(); + + // Check the new stats API values are as expected + assertEquals( + new ImmutableCacheStats(expectedHits, expectedMisses, 0, expectedEntries * singleSearchSize, expectedEntries), + totalStats + ); + // Now check the new stats API values for the cache as a whole match the old stats API values + RequestCacheStats oldAPIStats = client.admin() + .indices() + .prepareStats(index) + .setRequestCache(true) + .get() + .getTotal() + .getRequestCache(); + assertEquals(oldAPIStats.getHitCount(), totalStats.getHits()); + assertEquals(oldAPIStats.getMissCount(), totalStats.getMisses()); + assertEquals(oldAPIStats.getEvictions(), totalStats.getEvictions()); + assertEquals(oldAPIStats.getMemorySizeInBytes(), totalStats.getSizeInBytes()); + } + + private void startIndex(Client client, String indexName) throws InterruptedException { + assertAcked( + client.admin() + .indices() + .prepareCreate(indexName) + .setMapping("k", "type=keyword") + .setSettings( + Settings.builder() + .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build() + ) + .get() + ); + indexRandom(true, client.prepareIndex(indexName).setSource("k", "hello")); + ensureSearchable(indexName); + } + + private Map setupCacheForAggregationTests(Client client) throws Exception { + startIndex(client, index1Name); + startIndex(client, index2Name); + + // First search one time to see how big a single value will be + searchIndex(client, index1Name, 0); + // get total stats + long singleSearchSize = getTotalStats(client).getSizeInBytes(); + int itemsOnHeapAfterTest = HEAP_CACHE_SIZE / (int) singleSearchSize; // As the heap tier evicts, the items on it after the test will + // be the same as its max capacity + int itemsOnDiskAfterTest = 1 + randomInt(30); // The first one we search (to get the size) always goes to disk + + // Put some values on heap and disk for each index + int itemsOnHeapIndex1AfterTest = randomInt(itemsOnHeapAfterTest); + int itemsOnHeapIndex2AfterTest = itemsOnHeapAfterTest - itemsOnHeapIndex1AfterTest; + int itemsOnDiskIndex1AfterTest = 1 + randomInt(itemsOnDiskAfterTest - 1); + // The first one we search (to get the size) always goes to disk + int itemsOnDiskIndex2AfterTest = itemsOnDiskAfterTest - itemsOnDiskIndex1AfterTest; + int hitsOnHeapIndex1 = randomInt(itemsOnHeapIndex1AfterTest); + int hitsOnDiskIndex1 = randomInt(itemsOnDiskIndex1AfterTest); + int hitsOnHeapIndex2 = randomInt(itemsOnHeapIndex2AfterTest); + int hitsOnDiskIndex2 = randomInt(itemsOnDiskIndex2AfterTest); + + // Put these values into a map so tests can know what to expect in stats responses + Map expectedValues = new HashMap<>(); + expectedValues.put("itemsOnHeapIndex1AfterTest", itemsOnHeapIndex1AfterTest); + expectedValues.put("itemsOnHeapIndex2AfterTest", itemsOnHeapIndex2AfterTest); + expectedValues.put("itemsOnDiskIndex1AfterTest", itemsOnDiskIndex1AfterTest); + expectedValues.put("itemsOnDiskIndex2AfterTest", itemsOnDiskIndex2AfterTest); + expectedValues.put("hitsOnHeapIndex1", hitsOnHeapIndex1); + expectedValues.put("hitsOnDiskIndex1", hitsOnDiskIndex1); + expectedValues.put("hitsOnHeapIndex2", hitsOnHeapIndex2); + expectedValues.put("hitsOnDiskIndex2", hitsOnDiskIndex2); + expectedValues.put("singleSearchSize", (int) singleSearchSize); + expectedValues.put("itemsOnDiskAfterTest", itemsOnDiskAfterTest); + expectedValues.put("itemsOnHeapAfterTest", itemsOnHeapAfterTest); // Can only pass 10 keys in Map.of() constructor + + // The earliest items (0 - itemsOnDiskAfterTest) are the ones which get evicted to disk + for (int i = 1; i < itemsOnDiskIndex1AfterTest; i++) { // Start at 1 as 0 has already been searched + searchIndex(client, index1Name, i); + } + for (int i = itemsOnDiskIndex1AfterTest; i < itemsOnDiskIndex1AfterTest + itemsOnDiskIndex2AfterTest; i++) { + searchIndex(client, index2Name, i); + } + // The remaining items stay on heap + for (int i = itemsOnDiskAfterTest; i < itemsOnDiskAfterTest + itemsOnHeapIndex1AfterTest; i++) { + searchIndex(client, index1Name, i); + } + for (int i = itemsOnDiskAfterTest + itemsOnHeapIndex1AfterTest; i < itemsOnDiskAfterTest + itemsOnHeapAfterTest; i++) { + searchIndex(client, index2Name, i); + } + + // Get some hits on all combinations of indices and tiers + for (int i = itemsOnDiskAfterTest; i < itemsOnDiskAfterTest + hitsOnHeapIndex1; i++) { + // heap hits for index 1 + searchIndex(client, index1Name, i); + } + for (int i = itemsOnDiskAfterTest + itemsOnHeapIndex1AfterTest; i < itemsOnDiskAfterTest + itemsOnHeapIndex1AfterTest + + hitsOnHeapIndex2; i++) { + // heap hits for index 2 + searchIndex(client, index2Name, i); + } + for (int i = 0; i < hitsOnDiskIndex1; i++) { + // disk hits for index 1 + searchIndex(client, index1Name, i); + } + for (int i = itemsOnDiskIndex1AfterTest; i < itemsOnDiskIndex1AfterTest + hitsOnDiskIndex2; i++) { + // disk hits for index 2 + searchIndex(client, index2Name, i); + } + return expectedValues; + } + + private ImmutableCacheStats returnNullIfAllZero(ImmutableCacheStats expectedStats) { + // If the randomly chosen numbers are such that the expected stats would be 0, we actually have not interacted with the cache for + // this index. + // In this case, we expect the stats holder to have no stats for this node, and therefore we should get null from + // statsHolder.getStatsForDimensionValues(). + // We will not see it in the XContent response. + if (expectedStats.equals(new ImmutableCacheStats(0, 0, 0, 0, 0))) { + return null; + } + return expectedStats; + } + + // Duplicated from CacheStatsAPIIndicesRequestCacheIT.java, as we can't add a dependency on server.internalClusterTest + + private SearchResponse searchIndex(Client client, String index, int searchSuffix) { + SearchResponse resp = client.prepareSearch(index) + .setRequestCache(true) + .setQuery(QueryBuilders.termQuery("k", "hello" + padWithZeros(4, searchSuffix))) + // pad with zeros so request 0 and request 10 have the same size ("0000" and "0010" instead of "0" and "10") + .get(); + assertSearchResponse(resp); + OpenSearchAssertions.assertAllSuccessful(resp); + return resp; + } + + private String padWithZeros(int finalLength, int inputValue) { + // Avoid forbidden API String.format() + String input = String.valueOf(inputValue); + if (input.length() >= finalLength) { + return input; + } + StringBuilder sb = new StringBuilder(); + while (sb.length() < finalLength - input.length()) { + sb.append('0'); + } + sb.append(input); + return sb.toString(); + } + + private ImmutableCacheStats getTotalStats(Client client) throws IOException { + ImmutableCacheStatsHolder statsHolder = getNodeCacheStatsResult(client, List.of()); + return statsHolder.getStatsForDimensionValues(List.of()); + } + + private static ImmutableCacheStatsHolder getNodeCacheStatsResult(Client client, List aggregationLevels) throws IOException { + CommonStatsFlags statsFlags = new CommonStatsFlags(); + statsFlags.includeAllCacheTypes(); + String[] flagsLevels; + if (aggregationLevels == null) { + flagsLevels = null; + } else { + flagsLevels = aggregationLevels.toArray(new String[0]); + } + statsFlags.setLevels(flagsLevels); + + NodesStatsResponse nodeStatsResponse = client.admin() + .cluster() + .prepareNodesStats("data:true") + .addMetric(NodesStatsRequest.Metric.CACHE_STATS.metricName()) + .setIndices(statsFlags) + .get(); + // Can always get the first data node as there's only one in this test suite + assertEquals(1, nodeStatsResponse.getNodes().size()); + NodeCacheStats ncs = nodeStatsResponse.getNodes().get(0).getNodeCacheStats(); + return ncs.getStatsByCache(CacheType.INDICES_REQUEST_CACHE); + } +} 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 f40c35dde83de..63cdbca101f2a 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 @@ -327,6 +327,7 @@ private Function, Tuple> getValueFromTieredCache(boolean void handleRemovalFromHeapTier(RemovalNotification, V> notification) { ICacheKey key = notification.getKey(); boolean wasEvicted = SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()); + boolean countEvictionTowardsTotal = false; // Don't count this eviction towards the cache's total if it ends up in the disk tier if (caches.get(diskCache).isEnabled() && wasEvicted && evaluatePolicies(notification.getValue())) { try (ReleasableLock ignore = writeLock.acquire()) { diskCache.put(key, notification.getValue()); // spill over to the disk tier and increment its stats @@ -336,21 +337,28 @@ void handleRemovalFromHeapTier(RemovalNotification, V> notification // If the value is not going to the disk cache, send this notification to the TSC's removal listener // as the value is leaving the TSC entirely removalListener.onRemoval(notification); + countEvictionTowardsTotal = true; } - updateStatsOnRemoval(TIER_DIMENSION_VALUE_ON_HEAP, wasEvicted, key, notification.getValue()); + updateStatsOnRemoval(TIER_DIMENSION_VALUE_ON_HEAP, wasEvicted, key, notification.getValue(), countEvictionTowardsTotal); } void handleRemovalFromDiskTier(RemovalNotification, V> notification) { // Values removed from the disk tier leave the TSC entirely removalListener.onRemoval(notification); boolean wasEvicted = SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()); - updateStatsOnRemoval(TIER_DIMENSION_VALUE_DISK, wasEvicted, notification.getKey(), notification.getValue()); + updateStatsOnRemoval(TIER_DIMENSION_VALUE_DISK, wasEvicted, notification.getKey(), notification.getValue(), true); } - void updateStatsOnRemoval(String removedFromTierValue, boolean wasEvicted, ICacheKey key, V value) { + void updateStatsOnRemoval( + String removedFromTierValue, + boolean wasEvicted, + ICacheKey key, + V value, + boolean countEvictionTowardsTotal + ) { List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, removedFromTierValue); if (wasEvicted) { - statsHolder.incrementEvictions(dimensionValues); + statsHolder.incrementEvictions(dimensionValues, countEvictionTowardsTotal); } statsHolder.decrementItems(dimensionValues); statsHolder.decrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsHolder.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsHolder.java index d17059e8dee94..b40724430454b 100644 --- a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsHolder.java +++ b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsHolder.java @@ -105,20 +105,29 @@ public void incrementMisses(List dimensionValues) { internalIncrement(dimensionValues, missIncrementer, true); } + /** + * This method shouldn't be used in this class. Instead, use incrementEvictions(dimensionValues, includeInTotal) + * which specifies whether the eviction should be included in the cache's total evictions, or if it should + * just count towards that tier's evictions. + * @param dimensionValues The dimension values + */ @Override public void incrementEvictions(List dimensionValues) { - final String tierValue = validateTierDimensionValue(dimensionValues); + throw new UnsupportedOperationException( + "TieredSpilloverCacheHolder must specify whether to include an eviction in the total cache stats. Use incrementEvictions(List dimensionValues, boolean includeInTotal)" + ); + } - // If the disk tier is present, only evictions from the disk tier should be included in total values. + /** + * Increment evictions for this set of dimension values. + * @param dimensionValues The dimension values + * @param includeInTotal Whether to include this eviction in the total for the whole cache's evictions + */ + public void incrementEvictions(List dimensionValues, boolean includeInTotal) { + validateTierDimensionValue(dimensionValues); + // If we count this eviction towards the total, we should increment all ancestor nodes. If not, only increment the leaf node. Consumer evictionsIncrementer = (node) -> { - if (tierValue.equals(TIER_DIMENSION_VALUE_ON_HEAP) && diskCacheEnabled) { - // If on-heap tier, increment only the leaf node corresponding to the on heap values; not the total values in its parent - // nodes - if (node.isAtLowestLevel()) { - node.incrementEvictions(); - } - } else { - // If disk tier, or on-heap tier with a disabled disk tier, increment the leaf node and its parents + if (includeInTotal || node.isAtLowestLevel()) { node.incrementEvictions(); } }; 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 6c49341591589..54b15f236a418 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 @@ -926,14 +926,14 @@ public void testDiskTierPolicies() throws Exception { MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( keyValueSize, - 100, + keyValueSize * 100, removalListener, Settings.builder() .put( OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) .get(MAXIMUM_SIZE_IN_BYTES_KEY) .getKey(), - onHeapCacheSize * 50 + "b" + onHeapCacheSize * keyValueSize + "b" ) .build(), 0, @@ -955,6 +955,7 @@ public void testDiskTierPolicies() throws Exception { LoadAwareCacheLoader, String> loader = getLoadAwareCacheLoader(keyValuePairs); + int expectedEvictions = 0; for (String key : keyValuePairs.keySet()) { ICacheKey iCacheKey = getICacheKey(key); Boolean expectedOutput = expectedOutputs.get(key); @@ -967,8 +968,15 @@ public void testDiskTierPolicies() throws Exception { } else { // Should miss as heap tier size = 0 and the policy rejected it assertNull(result); + expectedEvictions++; } } + + // We expect values that were evicted from the heap tier and not allowed into the disk tier by the policy + // to count towards total evictions + assertEquals(keyValuePairs.size(), getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); // Disk tier is large enough for no evictions + assertEquals(expectedEvictions, getTotalStatsSnapshot(tieredSpilloverCache).getEvictions()); } public void testTookTimePolicyFromFactory() throws Exception { @@ -1493,6 +1501,11 @@ private ImmutableCacheStats getStatsSnapshotForTier(TieredSpilloverCache t return snapshot; } + private ImmutableCacheStats getTotalStatsSnapshot(TieredSpilloverCache tsc) throws IOException { + ImmutableCacheStatsHolder cacheStats = tsc.stats(new String[0]); + return cacheStats.getStatsForDimensionValues(List.of()); + } + // Duplicated here from EhcacheDiskCacheTests.java, we can't add a dependency on that plugin static class StringSerializer implements Serializer { private final Charset charset = StandardCharsets.UTF_8; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/CacheStatsAPIIndicesRequestCacheIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/CacheStatsAPIIndicesRequestCacheIT.java index de7a52761c77c..0539f96e429c1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/CacheStatsAPIIndicesRequestCacheIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/CacheStatsAPIIndicesRequestCacheIT.java @@ -12,6 +12,7 @@ import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; +import org.opensearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.action.search.SearchResponse; import org.opensearch.client.Client; @@ -20,7 +21,7 @@ import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.service.NodeCacheStats; import org.opensearch.common.cache.stats.ImmutableCacheStats; -import org.opensearch.common.cache.stats.ImmutableCacheStatsHolderTests; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentFactory; @@ -56,6 +57,10 @@ public static Collection parameters() { return Arrays.asList(new Object[] { Settings.builder().put(FeatureFlags.PLUGGABLE_CACHE, "true").build() }); } + /** + * Test aggregating by indices, indices+shards, shards, or no levels, and check the resulting stats + * are as we expect. + */ public void testCacheStatsAPIWIthOnHeapCache() throws Exception { String index1Name = "index1"; String index2Name = "index2"; @@ -73,84 +78,60 @@ public void testCacheStatsAPIWIthOnHeapCache() throws Exception { searchIndex(client, index2Name, ""); // First, aggregate by indices only - Map xContentMap = getNodeCacheStatsXContentMap(client, List.of(IndicesRequestCache.INDEX_DIMENSION_NAME)); + ImmutableCacheStatsHolder indicesStats = getNodeCacheStatsResult(client, List.of(IndicesRequestCache.INDEX_DIMENSION_NAME)); - List index1Keys = List.of(CacheType.INDICES_REQUEST_CACHE.getValue(), IndicesRequestCache.INDEX_DIMENSION_NAME, index1Name); + List index1Dimensions = List.of(index1Name); // Since we searched twice, we expect to see 1 hit, 1 miss and 1 entry for index 1 ImmutableCacheStats expectedStats = new ImmutableCacheStats(1, 1, 0, 0, 1); - checkCacheStatsAPIResponse(xContentMap, index1Keys, expectedStats, false, true); + checkCacheStatsAPIResponse(indicesStats, index1Dimensions, expectedStats, false, true); // Get the request size for one request, so we can reuse it for next index - int requestSize = (int) ((Map) ImmutableCacheStatsHolderTests.getValueFromNestedXContentMap( - xContentMap, - index1Keys - )).get(ImmutableCacheStats.Fields.SIZE_IN_BYTES); + long requestSize = indicesStats.getStatsForDimensionValues(List.of(index1Name)).getSizeInBytes(); assertTrue(requestSize > 0); - List index2Keys = List.of(CacheType.INDICES_REQUEST_CACHE.getValue(), IndicesRequestCache.INDEX_DIMENSION_NAME, index2Name); + List index2Dimensions = List.of(index2Name); // We searched once in index 2, we expect 1 miss + 1 entry expectedStats = new ImmutableCacheStats(0, 1, 0, requestSize, 1); - checkCacheStatsAPIResponse(xContentMap, index2Keys, expectedStats, true, true); + checkCacheStatsAPIResponse(indicesStats, index2Dimensions, expectedStats, true, true); // The total stats for the node should be 1 hit, 2 misses, and 2 entries expectedStats = new ImmutableCacheStats(1, 2, 0, 2 * requestSize, 2); - List totalStatsKeys = List.of(CacheType.INDICES_REQUEST_CACHE.getValue()); - checkCacheStatsAPIResponse(xContentMap, totalStatsKeys, expectedStats, true, true); + List totalStatsKeys = List.of(); + checkCacheStatsAPIResponse(indicesStats, totalStatsKeys, expectedStats, true, true); // Aggregate by shards only - xContentMap = getNodeCacheStatsXContentMap(client, List.of(IndicesRequestCache.SHARD_ID_DIMENSION_NAME)); + ImmutableCacheStatsHolder shardsStats = getNodeCacheStatsResult(client, List.of(IndicesRequestCache.SHARD_ID_DIMENSION_NAME)); - List index1Shard0Keys = List.of( - CacheType.INDICES_REQUEST_CACHE.getValue(), - IndicesRequestCache.SHARD_ID_DIMENSION_NAME, - "[" + index1Name + "][0]" - ); + List index1Shard0Dimensions = List.of("[" + index1Name + "][0]"); expectedStats = new ImmutableCacheStats(1, 1, 0, requestSize, 1); - checkCacheStatsAPIResponse(xContentMap, index1Shard0Keys, expectedStats, true, true); + checkCacheStatsAPIResponse(shardsStats, index1Shard0Dimensions, expectedStats, true, true); - List index2Shard0Keys = List.of( - CacheType.INDICES_REQUEST_CACHE.getValue(), - IndicesRequestCache.SHARD_ID_DIMENSION_NAME, - "[" + index2Name + "][0]" - ); + List index2Shard0Dimensions = List.of("[" + index2Name + "][0]"); expectedStats = new ImmutableCacheStats(0, 1, 0, requestSize, 1); - checkCacheStatsAPIResponse(xContentMap, index2Shard0Keys, expectedStats, true, true); + checkCacheStatsAPIResponse(shardsStats, index2Shard0Dimensions, expectedStats, true, true); // Aggregate by indices and shards - xContentMap = getNodeCacheStatsXContentMap( + ImmutableCacheStatsHolder indicesAndShardsStats = getNodeCacheStatsResult( client, List.of(IndicesRequestCache.INDEX_DIMENSION_NAME, IndicesRequestCache.SHARD_ID_DIMENSION_NAME) ); - index1Keys = List.of( - CacheType.INDICES_REQUEST_CACHE.getValue(), - IndicesRequestCache.INDEX_DIMENSION_NAME, - index1Name, - IndicesRequestCache.SHARD_ID_DIMENSION_NAME, - "[" + index1Name + "][0]" - ); + index1Dimensions = List.of(index1Name, "[" + index1Name + "][0]"); expectedStats = new ImmutableCacheStats(1, 1, 0, requestSize, 1); - checkCacheStatsAPIResponse(xContentMap, index1Keys, expectedStats, true, true); - - index2Keys = List.of( - CacheType.INDICES_REQUEST_CACHE.getValue(), - IndicesRequestCache.INDEX_DIMENSION_NAME, - index2Name, - IndicesRequestCache.SHARD_ID_DIMENSION_NAME, - "[" + index2Name + "][0]" - ); + checkCacheStatsAPIResponse(indicesAndShardsStats, index1Dimensions, expectedStats, true, true); + index2Dimensions = List.of(index2Name, "[" + index2Name + "][0]"); expectedStats = new ImmutableCacheStats(0, 1, 0, requestSize, 1); - checkCacheStatsAPIResponse(xContentMap, index2Keys, expectedStats, true, true); - + checkCacheStatsAPIResponse(indicesAndShardsStats, index2Dimensions, expectedStats, true, true); } - // TODO: Add testCacheStatsAPIWithTieredCache when TSC stats implementation PR is merged - + /** + * Check the new stats API returns the same values as the old stats API. In particular, + * check that the new and old APIs are both correctly estimating memory size, + * using the logic that includes the overhead memory in ICacheKey. + */ public void testStatsMatchOldApi() throws Exception { - // The main purpose of this test is to check that the new and old APIs are both correctly estimating memory size, - // using the logic that includes the overhead memory in ICacheKey. String index = "index"; Client client = client(); startIndex(client, index); @@ -173,8 +154,7 @@ public void testStatsMatchOldApi() throws Exception { .getRequestCache(); assertNotEquals(0, oldApiStats.getMemorySizeInBytes()); - List xContentMapKeys = List.of(CacheType.INDICES_REQUEST_CACHE.getValue()); - Map xContentMap = getNodeCacheStatsXContentMap(client, List.of()); + ImmutableCacheStatsHolder statsHolder = getNodeCacheStatsResult(client, List.of()); ImmutableCacheStats expected = new ImmutableCacheStats( oldApiStats.getHitCount(), oldApiStats.getMissCount(), @@ -183,9 +163,13 @@ public void testStatsMatchOldApi() throws Exception { 0 ); // Don't check entries, as the old API doesn't track this - checkCacheStatsAPIResponse(xContentMap, xContentMapKeys, expected, true, false); + checkCacheStatsAPIResponse(statsHolder, List.of(), expected, true, false); } + /** + * Test the XContent in the response behaves correctly when we pass null levels. + * Only the total cache stats should be returned. + */ public void testNullLevels() throws Exception { String index = "index"; Client client = client(); @@ -194,9 +178,81 @@ public void testNullLevels() throws Exception { for (int i = 0; i < numKeys; i++) { searchIndex(client, index, String.valueOf(i)); } - Map xContentMap = getNodeCacheStatsXContentMap(client, null); + Map xContentMap = getStatsXContent(getNodeCacheStatsResult(client, null)); // Null levels should result in only the total cache stats being returned -> 6 fields inside the response. - assertEquals(6, ((Map) xContentMap.get("request_cache")).size()); + assertEquals(6, xContentMap.size()); + } + + /** + * Test clearing the cache using API sets memory size and number of items to 0, but leaves other stats + * unaffected. + */ + public void testCacheClear() throws Exception { + String index = "index"; + Client client = client(); + + startIndex(client, index); + + int expectedHits = 2; + int expectedMisses = 7; + // Search for the same doc to give hits + for (int i = 0; i < expectedHits + 1; i++) { + searchIndex(client, index, ""); + } + // Search for new docs + for (int i = 0; i < expectedMisses - 1; i++) { + searchIndex(client, index, String.valueOf(i)); + } + + ImmutableCacheStats expectedTotal = new ImmutableCacheStats(expectedHits, expectedMisses, 0, 0, expectedMisses); + ImmutableCacheStatsHolder statsHolder = getNodeCacheStatsResult(client, List.of()); + // Don't check the memory size, just assert it's nonzero + checkCacheStatsAPIResponse(statsHolder, List.of(), expectedTotal, false, true); + long originalMemorySize = statsHolder.getTotalSizeInBytes(); + assertNotEquals(0, originalMemorySize); + + // Clear cache + ClearIndicesCacheRequest clearIndicesCacheRequest = new ClearIndicesCacheRequest(index); + client.admin().indices().clearCache(clearIndicesCacheRequest).actionGet(); + + // Now size and items should be 0 + expectedTotal = new ImmutableCacheStats(expectedHits, expectedMisses, 0, 0, 0); + statsHolder = getNodeCacheStatsResult(client, List.of()); + checkCacheStatsAPIResponse(statsHolder, List.of(), expectedTotal, true, true); + } + + /** + * Test the cache stats responses are in the expected place in XContent when we call the overall API + * GET /_nodes/stats. They should be at nodes.[node_id].caches.request_cache. + */ + public void testNodesStatsResponse() throws Exception { + String index = "index"; + Client client = client(); + + startIndex(client, index); + + NodesStatsResponse nodeStatsResponse = client.admin() + .cluster() + .prepareNodesStats("data:true") + .all() // This mimics /_nodes/stats + .get(); + XContentBuilder builder = XContentFactory.jsonBuilder(); + Map paramMap = new HashMap<>(); + ToXContent.Params params = new ToXContent.MapParams(paramMap); + + builder.startObject(); + nodeStatsResponse.toXContent(builder, params); + builder.endObject(); + Map xContentMap = XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), builder.toString(), true); + // Values should be at nodes.[node_id].caches.request_cache + // Get the node id + Map nodesResponse = (Map) xContentMap.get("nodes"); + assertEquals(1, nodesResponse.size()); + String nodeId = nodesResponse.keySet().toArray(String[]::new)[0]; + Map cachesResponse = (Map) ((Map) nodesResponse.get(nodeId)).get("caches"); + assertNotNull(cachesResponse); + // Request cache should be present in the response + assertTrue(cachesResponse.containsKey("request_cache")); } private void startIndex(Client client, String indexName) throws InterruptedException { @@ -227,8 +283,7 @@ private SearchResponse searchIndex(Client client, String index, String searchSuf return resp; } - private static Map getNodeCacheStatsXContentMap(Client client, List aggregationLevels) throws IOException { - + private static ImmutableCacheStatsHolder getNodeCacheStatsResult(Client client, List aggregationLevels) throws IOException { CommonStatsFlags statsFlags = new CommonStatsFlags(); statsFlags.includeAllCacheTypes(); String[] flagsLevels; @@ -248,16 +303,16 @@ private static Map getNodeCacheStatsXContentMap(Client client, L // Can always get the first data node as there's only one in this test suite assertEquals(1, nodeStatsResponse.getNodes().size()); NodeCacheStats ncs = nodeStatsResponse.getNodes().get(0).getNodeCacheStats(); + return ncs.getStatsByCache(CacheType.INDICES_REQUEST_CACHE); + } + private static Map getStatsXContent(ImmutableCacheStatsHolder statsHolder) throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder(); Map paramMap = new HashMap<>(); - if (aggregationLevels != null && !aggregationLevels.isEmpty()) { - paramMap.put("level", String.join(",", aggregationLevels)); - } ToXContent.Params params = new ToXContent.MapParams(paramMap); builder.startObject(); - ncs.toXContent(builder, params); + statsHolder.toXContent(builder, params); builder.endObject(); String resultString = builder.toString(); @@ -265,27 +320,22 @@ private static Map getNodeCacheStatsXContentMap(Client client, L } private static void checkCacheStatsAPIResponse( - Map xContentMap, - List xContentMapKeys, + ImmutableCacheStatsHolder statsHolder, + List dimensionValues, ImmutableCacheStats expectedStats, boolean checkMemorySize, boolean checkEntries ) { - // Assumes the keys point to a level whose keys are the field values ("size_in_bytes", "evictions", etc) and whose values store - // those stats - Map aggregatedStatsResponse = (Map) ImmutableCacheStatsHolderTests.getValueFromNestedXContentMap( - xContentMap, - xContentMapKeys - ); + ImmutableCacheStats aggregatedStatsResponse = statsHolder.getStatsForDimensionValues(dimensionValues); assertNotNull(aggregatedStatsResponse); - assertEquals(expectedStats.getHits(), (int) aggregatedStatsResponse.get(ImmutableCacheStats.Fields.HIT_COUNT)); - assertEquals(expectedStats.getMisses(), (int) aggregatedStatsResponse.get(ImmutableCacheStats.Fields.MISS_COUNT)); - assertEquals(expectedStats.getEvictions(), (int) aggregatedStatsResponse.get(ImmutableCacheStats.Fields.EVICTIONS)); + assertEquals(expectedStats.getHits(), (int) aggregatedStatsResponse.getHits()); + assertEquals(expectedStats.getMisses(), (int) aggregatedStatsResponse.getMisses()); + assertEquals(expectedStats.getEvictions(), (int) aggregatedStatsResponse.getEvictions()); if (checkMemorySize) { - assertEquals(expectedStats.getSizeInBytes(), (int) aggregatedStatsResponse.get(ImmutableCacheStats.Fields.SIZE_IN_BYTES)); + assertEquals(expectedStats.getSizeInBytes(), (int) aggregatedStatsResponse.getSizeInBytes()); } if (checkEntries) { - assertEquals(expectedStats.getItems(), (int) aggregatedStatsResponse.get(ImmutableCacheStats.Fields.ITEM_COUNT)); + assertEquals(expectedStats.getItems(), (int) aggregatedStatsResponse.getItems()); } } } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java index 4d108f8d78a69..ca2685e093d3f 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java @@ -138,7 +138,7 @@ public CommonStatsFlags all() { includeUnloadedSegments = false; includeAllShardIndexingPressureTrackers = false; includeOnlyTopIndexingPressureMetrics = false; - includeCaches = EnumSet.noneOf(CacheType.class); + includeCaches = EnumSet.allOf(CacheType.class); levels = new String[0]; return this; } diff --git a/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java b/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java index 07c75eab34194..dd94dbf61debb 100644 --- a/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java @@ -8,6 +8,7 @@ package org.opensearch.common.cache.service; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.cache.CacheType; @@ -51,6 +52,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(NodesStatsRequest.Metric.CACHE_STATS.metricName()); for (CacheType type : statsByCache.keySet()) { if (flags.getIncludeCaches().contains(type)) { builder.startObject(type.getValue()); @@ -58,6 +60,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); } } + builder.endObject(); return builder; } @@ -77,4 +80,10 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(statsByCache, flags); } + + // Get the immutable cache stats for a given cache, used to avoid having to process XContent in tests. + // Safe to expose publicly as the ImmutableCacheStatsHolder can't be modified after its creation. + public ImmutableCacheStatsHolder getStatsByCache(CacheType cacheType) { + return statsByCache.get(cacheType); + } } diff --git a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java index 44af83bb839c1..57f7e402536f2 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java @@ -715,23 +715,28 @@ private synchronized void cleanCache(double stalenessThreshold) { } // Contains CleanupKey objects with open shard but invalidated readerCacheKeyId. final Set cleanupKeysFromOutdatedReaders = new HashSet<>(); - // Contains CleanupKey objects of a closed shard. + // Contains CleanupKey objects for a full cache cleanup. + final Set> cleanupKeysFromFullClean = new HashSet<>(); + // Contains CleanupKey objects for a closed shard. final Set> cleanupKeysFromClosedShards = new HashSet<>(); for (Iterator iterator = keysToClean.iterator(); iterator.hasNext();) { CleanupKey cleanupKey = iterator.next(); iterator.remove(); - if (cleanupKey.readerCacheKeyId == null || !cleanupKey.entity.isOpen()) { - // null indicates full cleanup, as does a closed shard - IndexShard indexShard = (IndexShard) cleanupKey.entity.getCacheIdentity(); + final IndexShard indexShard = (IndexShard) cleanupKey.entity.getCacheIdentity(); + if (cleanupKey.readerCacheKeyId == null) { + // null indicates full cleanup // Add both shardId and indexShardHashCode to uniquely identify an indexShard. + cleanupKeysFromFullClean.add(new Tuple<>(indexShard.shardId(), indexShard.hashCode())); + } else if (!cleanupKey.entity.isOpen()) { + // The shard is closed cleanupKeysFromClosedShards.add(new Tuple<>(indexShard.shardId(), indexShard.hashCode())); } else { cleanupKeysFromOutdatedReaders.add(cleanupKey); } } - if (cleanupKeysFromOutdatedReaders.isEmpty() && cleanupKeysFromClosedShards.isEmpty()) { + if (cleanupKeysFromOutdatedReaders.isEmpty() && cleanupKeysFromFullClean.isEmpty() && cleanupKeysFromClosedShards.isEmpty()) { return; } @@ -740,15 +745,15 @@ private synchronized void cleanCache(double stalenessThreshold) { for (Iterator> iterator = cache.keys().iterator(); iterator.hasNext();) { ICacheKey key = iterator.next(); Key delegatingKey = key.key; - if (cleanupKeysFromClosedShards.contains(new Tuple<>(delegatingKey.shardId, delegatingKey.indexShardHashCode))) { - // Since the shard is closed, the cache should drop stats for this shard. - dimensionListsToDrop.add(key.dimensions); + Tuple shardIdInfo = new Tuple<>(delegatingKey.shardId, delegatingKey.indexShardHashCode); + if (cleanupKeysFromFullClean.contains(shardIdInfo) || cleanupKeysFromClosedShards.contains(shardIdInfo)) { iterator.remove(); } else { CacheEntity cacheEntity = cacheEntityLookup.apply(delegatingKey.shardId).orElse(null); if (cacheEntity == null) { // If cache entity is null, it means that index or shard got deleted/closed meanwhile. // So we will delete this key. + dimensionListsToDrop.add(key.dimensions); iterator.remove(); } else { CleanupKey cleanupKey = new CleanupKey(cacheEntity, delegatingKey.readerCacheKeyId); @@ -757,6 +762,12 @@ private synchronized void cleanCache(double stalenessThreshold) { } } } + + if (cleanupKeysFromClosedShards.contains(shardIdInfo)) { + // Since the shard is closed, the cache should drop stats for this shard. + // This should not happen on a full cache cleanup. + dimensionListsToDrop.add(key.dimensions); + } } for (List closedDimensions : dimensionListsToDrop) { // Invalidate a dummy key containing the dimensions we need to drop stats for From 10ae4cb97f0616b1728d1ba07b2d9ae8601071e4 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Wed, 5 Jun 2024 10:00:28 -0400 Subject: [PATCH 46/77] Another attempt to stabilize MacOS GA runner caused by QEMU / Colima / Docker issues (#13900) Signed-off-by: Andriy Redko --- .github/workflows/assemble.yml | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/.github/workflows/assemble.yml b/.github/workflows/assemble.yml index 4bff5b7d60d1f..51ae075ffa2c9 100644 --- a/.github/workflows/assemble.yml +++ b/.github/workflows/assemble.yml @@ -17,10 +17,23 @@ jobs: java-version: ${{ matrix.java }} distribution: temurin - name: Setup docker (missing on MacOS) + id: setup_docker if: runner.os == 'macos' uses: douglascamata/setup-docker-macos-action@main + continue-on-error: true with: upgrade-qemu: true + colima: v0.6.8 - name: Run Gradle (assemble) + if: runner.os == 'macos' && steps.setup_docker.outcome != 'success' + run: | + # Report success even if previous step failed (Docker on MacOS runner is very unstable) + exit 0; + - name: Run Gradle (assemble) + if: runner.os != 'macos' + run: | + ./gradlew assemble --parallel --no-build-cache -PDISABLE_BUILD_CACHE + - name: Run Gradle (assemble) + if: runner.os == 'macos' && steps.setup_docker.outcome == 'success' run: | ./gradlew assemble --parallel --no-build-cache -PDISABLE_BUILD_CACHE From 51214092ca532b557b6775ff8620025a51f92fcc Mon Sep 17 00:00:00 2001 From: Himshikha Gupta Date: Wed, 5 Jun 2024 22:03:54 +0530 Subject: [PATCH 47/77] [Remote Routing Table] Initial commit for RemoteRoutingTableService setup (#13304) * Initial commit for RemoteRoutingTableService setup Signed-off-by: Himshikha Gupta --- CHANGELOG.md | 1 + .../coordination/JoinTaskExecutor.java | 36 +++- .../metadata/RepositoriesMetadata.java | 36 ++++ .../remote/RemoteRoutingTableService.java | 67 +++++++ .../cluster/routing/remote/package-info.java | 10 ++ .../common/settings/FeatureFlagSettings.java | 3 +- .../opensearch/common/util/FeatureFlags.java | 14 +- .../remote/RemoteClusterStateService.java | 15 ++ .../remotestore/RemoteStoreNodeAttribute.java | 33 ++++ .../coordination/JoinTaskExecutorTests.java | 163 ++++++++++++++++++ .../RemoteRoutingTableServiceTests.java | 77 +++++++++ .../RemoteClusterStateServiceTests.java | 30 ++++ .../node/RemoteStoreNodeAttributeTests.java | 75 ++++++++ 13 files changed, 551 insertions(+), 9 deletions(-) create mode 100644 server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java create mode 100644 server/src/main/java/org/opensearch/cluster/routing/remote/package-info.java create mode 100644 server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 5ce11dabb6e9c..1ffb438172f50 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -14,6 +14,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add getMetadataFields to MapperService ([#13819](https://github.com/opensearch-project/OpenSearch/pull/13819)) - [Remote State] Add async remote state deletion task running on an interval, configurable by a setting ([#13131](https://github.com/opensearch-project/OpenSearch/pull/13131)) - Allow setting query parameters on requests ([#13776](https://github.com/opensearch-project/OpenSearch/issues/13776)) +- Add remote routing table for remote state publication with experimental feature flag ([#13304](https://github.com/opensearch-project/OpenSearch/pull/13304)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) 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 5475470b81b93..f77a7ffc8ce8e 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/JoinTaskExecutor.java @@ -511,11 +511,27 @@ private static void ensureRemoteStoreNodesCompatibility(DiscoveryNode joiningNod assert existingNodes.isEmpty() == false; CompatibilityMode remoteStoreCompatibilityMode = REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get(metadata.settings()); - if (STRICT.equals(remoteStoreCompatibilityMode)) { - DiscoveryNode existingNode = existingNodes.get(0); + List reposToSkip = new ArrayList<>(1); + Optional remoteRoutingTableNode = existingNodes.stream() + .filter( + node -> node.getAttributes().get(RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY) != null + ) + .findFirst(); + // If none of the existing nodes have routing table repo, then we skip this repo check if present in joining node. + // This ensures a new node with remote routing table repo is able to join the cluster. + if (remoteRoutingTableNode.isEmpty()) { + String joiningNodeRepoName = joiningNode.getAttributes() + .get(RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY); + if (joiningNodeRepoName != null) { + reposToSkip.add(joiningNodeRepoName); + } + } + + if (STRICT.equals(remoteStoreCompatibilityMode)) { + DiscoveryNode existingNode = remoteRoutingTableNode.orElseGet(() -> existingNodes.get(0)); if (joiningNode.isRemoteStoreNode()) { - ensureRemoteStoreNodesCompatibility(joiningNode, existingNode); + ensureRemoteStoreNodesCompatibility(joiningNode, existingNode, reposToSkip); } else { if (existingNode.isRemoteStoreNode()) { throw new IllegalStateException( @@ -537,19 +553,25 @@ private static void ensureRemoteStoreNodesCompatibility(DiscoveryNode joiningNod throw new IllegalStateException(reason); } if (joiningNode.isRemoteStoreNode()) { - Optional remoteDN = existingNodes.stream().filter(DiscoveryNode::isRemoteStoreNode).findFirst(); - remoteDN.ifPresent(discoveryNode -> ensureRemoteStoreNodesCompatibility(joiningNode, discoveryNode)); + Optional remoteDN = remoteRoutingTableNode.isPresent() + ? remoteRoutingTableNode + : existingNodes.stream().filter(DiscoveryNode::isRemoteStoreNode).findFirst(); + remoteDN.ifPresent(discoveryNode -> ensureRemoteStoreNodesCompatibility(joiningNode, discoveryNode, reposToSkip)); } } } } - private static void ensureRemoteStoreNodesCompatibility(DiscoveryNode joiningNode, DiscoveryNode existingNode) { + private static void ensureRemoteStoreNodesCompatibility( + DiscoveryNode joiningNode, + DiscoveryNode existingNode, + List reposToSkip + ) { if (joiningNode.isRemoteStoreNode()) { if (existingNode.isRemoteStoreNode()) { RemoteStoreNodeAttribute joiningRemoteStoreNodeAttribute = new RemoteStoreNodeAttribute(joiningNode); RemoteStoreNodeAttribute existingRemoteStoreNodeAttribute = new RemoteStoreNodeAttribute(existingNode); - if (existingRemoteStoreNodeAttribute.equals(joiningRemoteStoreNodeAttribute) == false) { + if (existingRemoteStoreNodeAttribute.equalsWithRepoSkip(joiningRemoteStoreNodeAttribute, reposToSkip) == false) { throw new IllegalStateException( "a remote store node [" + joiningNode diff --git a/server/src/main/java/org/opensearch/cluster/metadata/RepositoriesMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/RepositoriesMetadata.java index 9b52bdd1b16c5..4b3dc7964a87b 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/RepositoriesMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/RepositoriesMetadata.java @@ -51,8 +51,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.EnumSet; import java.util.List; +import java.util.stream.Collectors; import static org.opensearch.repositories.blobstore.BlobStoreRepository.SYSTEM_REPOSITORY_SETTING; @@ -164,6 +166,40 @@ public boolean equalsIgnoreGenerations(@Nullable RepositoriesMetadata other) { return true; } + /** + * Checks if this instance and the give instance share the same repositories, with option to skip checking for a list of repos. + * This will support + * @param other other repositories metadata + * @param reposToSkip list of repos to skip check for equality + * @return {@code true} iff both instances contain the same repositories apart from differences in generations, not including repos provided in reposToSkip. + */ + public boolean equalsIgnoreGenerationsWithRepoSkip(@Nullable RepositoriesMetadata other, List reposToSkip) { + if (other == null) { + return false; + } + List currentRepositories = repositories.stream() + .filter(repo -> !reposToSkip.contains(repo.name())) + .collect(Collectors.toList()); + List otherRepositories = other.repositories.stream() + .filter(repo -> !reposToSkip.contains(repo.name())) + .collect(Collectors.toList()); + + if (otherRepositories.size() != currentRepositories.size()) { + return false; + } + // Sort repos by name for ordered comparison + Comparator compareByName = (o1, o2) -> o1.name().compareTo(o2.name()); + currentRepositories.sort(compareByName); + otherRepositories.sort(compareByName); + + for (int i = 0; i < currentRepositories.size(); i++) { + if (currentRepositories.get(i).equalsIgnoreGenerations(otherRepositories.get(i)) == false) { + return false; + } + } + return true; + } + @Override public int hashCode() { return repositories.hashCode(); diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java new file mode 100644 index 0000000000000..ba2208e17df1f --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java @@ -0,0 +1,67 @@ +/* + * 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.routing.remote; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.lifecycle.AbstractLifecycleComponent; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.node.Node; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.Repository; +import org.opensearch.repositories.blobstore.BlobStoreRepository; + +import java.io.IOException; +import java.util.function.Supplier; + +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteRoutingTableEnabled; + +/** + * A Service which provides APIs to upload and download routing table from remote store. + * + * @opensearch.internal + */ +public class RemoteRoutingTableService extends AbstractLifecycleComponent { + + private static final Logger logger = LogManager.getLogger(RemoteRoutingTableService.class); + private final Settings settings; + private final Supplier repositoriesService; + private BlobStoreRepository blobStoreRepository; + + public RemoteRoutingTableService(Supplier repositoriesService, Settings settings) { + assert isRemoteRoutingTableEnabled(settings) : "Remote routing table is not enabled"; + this.repositoriesService = repositoriesService; + this.settings = settings; + } + + @Override + protected void doClose() throws IOException { + if (blobStoreRepository != null) { + IOUtils.close(blobStoreRepository); + } + } + + @Override + protected void doStart() { + assert isRemoteRoutingTableEnabled(settings) == true : "Remote routing table is not enabled"; + final String remoteStoreRepo = settings.get( + Node.NODE_ATTRIBUTES.getKey() + RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY + ); + assert remoteStoreRepo != null : "Remote routing table repository is not configured"; + final Repository repository = repositoriesService.get().repository(remoteStoreRepo); + assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; + blobStoreRepository = (BlobStoreRepository) repository; + } + + @Override + protected void doStop() {} + +} diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/package-info.java b/server/src/main/java/org/opensearch/cluster/routing/remote/package-info.java new file mode 100644 index 0000000000000..9fe016e783f20 --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/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. + */ + +/** Package containing class to perform operations on remote routing table */ +package org.opensearch.cluster.routing.remote; diff --git a/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java b/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java index 7a364de1c5dc6..238df1bd90113 100644 --- a/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java @@ -36,6 +36,7 @@ protected FeatureFlagSettings( FeatureFlags.DATETIME_FORMATTER_CACHING_SETTING, FeatureFlags.TIERED_REMOTE_INDEX_SETTING, FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL_SETTING, - FeatureFlags.PLUGGABLE_CACHE_SETTING + FeatureFlags.PLUGGABLE_CACHE_SETTING, + FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL_SETTING ); } diff --git a/server/src/main/java/org/opensearch/common/util/FeatureFlags.java b/server/src/main/java/org/opensearch/common/util/FeatureFlags.java index 62cfbd861d4d9..82f43921d2d28 100644 --- a/server/src/main/java/org/opensearch/common/util/FeatureFlags.java +++ b/server/src/main/java/org/opensearch/common/util/FeatureFlags.java @@ -67,6 +67,11 @@ public class FeatureFlags { */ public static final String PLUGGABLE_CACHE = "opensearch.experimental.feature.pluggable.caching.enabled"; + /** + * Gates the functionality of remote routing table. + */ + public static final String REMOTE_PUBLICATION_EXPERIMENTAL = "opensearch.experimental.feature.remote_store.publication.enabled"; + public static final Setting REMOTE_STORE_MIGRATION_EXPERIMENTAL_SETTING = Setting.boolSetting( REMOTE_STORE_MIGRATION_EXPERIMENTAL, false, @@ -89,6 +94,12 @@ public class FeatureFlags { public static final Setting PLUGGABLE_CACHE_SETTING = Setting.boolSetting(PLUGGABLE_CACHE, false, Property.NodeScope); + public static final Setting REMOTE_PUBLICATION_EXPERIMENTAL_SETTING = Setting.boolSetting( + REMOTE_PUBLICATION_EXPERIMENTAL, + false, + Property.NodeScope + ); + private static final List> ALL_FEATURE_FLAG_SETTINGS = List.of( REMOTE_STORE_MIGRATION_EXPERIMENTAL_SETTING, EXTENSIONS_SETTING, @@ -96,7 +107,8 @@ public class FeatureFlags { TELEMETRY_SETTING, DATETIME_FORMATTER_CACHING_SETTING, TIERED_REMOTE_INDEX_SETTING, - PLUGGABLE_CACHE_SETTING + PLUGGABLE_CACHE_SETTING, + REMOTE_PUBLICATION_EXPERIMENTAL_SETTING ); /** * Should store the settings from opensearch.yml. diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index 0f862d1b68820..1a67f3cf25bbf 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -18,6 +18,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.cluster.routing.remote.RemoteRoutingTableService; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.Nullable; import org.opensearch.common.blobstore.BlobContainer; @@ -68,6 +69,7 @@ import static java.util.Objects.requireNonNull; import static org.opensearch.gateway.PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteRoutingTableEnabled; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled; /** @@ -202,6 +204,7 @@ public class RemoteClusterStateService implements Closeable { private final List indexMetadataUploadListeners; private BlobStoreRepository blobStoreRepository; private BlobStoreTransferService blobStoreTransferService; + private Optional remoteRoutingTableService; private volatile TimeValue slowWriteLoggingThreshold; private volatile TimeValue indexMetadataUploadTimeout; @@ -253,6 +256,9 @@ public RemoteClusterStateService( clusterSettings.addSettingsUpdateConsumer(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, this::setMetadataManifestUploadTimeout); this.remoteStateStats = new RemotePersistenceStats(); this.indexMetadataUploadListeners = indexMetadataUploadListeners; + this.remoteRoutingTableService = isRemoteRoutingTableEnabled(settings) + ? Optional.of(new RemoteRoutingTableService(repositoriesService, settings)) + : Optional.empty(); } private BlobStoreTransferService getBlobStoreTransferService() { @@ -749,6 +755,9 @@ public void close() throws IOException { if (blobStoreRepository != null) { IOUtils.close(blobStoreRepository); } + if (this.remoteRoutingTableService.isPresent()) { + this.remoteRoutingTableService.get().close(); + } } public void start() { @@ -760,6 +769,7 @@ public void start() { final Repository repository = repositoriesService.get().repository(remoteStoreRepo); assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; blobStoreRepository = (BlobStoreRepository) repository; + this.remoteRoutingTableService.ifPresent(RemoteRoutingTableService::start); } private ClusterMetadataManifest uploadManifest( @@ -933,6 +943,11 @@ public TimeValue getMetadataManifestUploadTimeout() { return this.metadataManifestUploadTimeout; } + // Package private for unit test + Optional getRemoteRoutingTableService() { + return this.remoteRoutingTableService; + } + static String getManifestFileName(long term, long version, boolean committed, int codecVersion) { // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/manifest/manifest______C/P____ return String.join( 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 b10ec0d99c3d5..a0f745a4270c4 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeAttribute.java @@ -13,6 +13,7 @@ import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.node.Node; import org.opensearch.repositories.blobstore.BlobStoreRepository; @@ -28,6 +29,8 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; + /** * This is an abstraction for validating and storing information specific to remote backed storage nodes. * @@ -46,6 +49,8 @@ public class RemoteStoreNodeAttribute { + "." + CryptoMetadata.SETTINGS_KEY; public static final String REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX = "remote_store.repository.%s.settings."; + public static final String REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY = "remote_store.routing_table.repository"; + private final RepositoriesMetadata repositoriesMetadata; public static List SUPPORTED_DATA_REPO_NAME_ATTRIBUTES = List.of( @@ -157,6 +162,10 @@ private Set getValidatedRepositoryNames(DiscoveryNode node) { } else if (node.getAttributes().containsKey(REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY)) { repositoryNames.add(validateAttributeNonNull(node, REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY)); } + if (node.getAttributes().containsKey(REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY)) { + repositoryNames.add(validateAttributeNonNull(node, REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY)); + } + return repositoryNames; } @@ -187,6 +196,15 @@ public static boolean isRemoteStoreClusterStateEnabled(Settings settings) { && isRemoteClusterStateAttributePresent(settings); } + private static boolean isRemoteRoutingTableAttributePresent(Settings settings) { + return settings.getByPrefix(Node.NODE_ATTRIBUTES.getKey() + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY) + .isEmpty() == false; + } + + public static boolean isRemoteRoutingTableEnabled(Settings settings) { + return FeatureFlags.isEnabled(REMOTE_PUBLICATION_EXPERIMENTAL) && isRemoteRoutingTableAttributePresent(settings); + } + public RepositoriesMetadata getRepositoriesMetadata() { return this.repositoriesMetadata; } @@ -231,6 +249,21 @@ public int hashCode() { return hashCode; } + /** + * Checks if 2 instances are equal, with option to skip check for a list of repos. + * * + * @param o other instance + * @param reposToSkip list of repos to skip check for equality + * @return {@code true} iff both instances are equal, not including the repositories in both instances if they are part of reposToSkip. + */ + public boolean equalsWithRepoSkip(Object o, List reposToSkip) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + RemoteStoreNodeAttribute that = (RemoteStoreNodeAttribute) o; + return this.getRepositoriesMetadata().equalsIgnoreGenerationsWithRepoSkip(that.getRepositoriesMetadata(), reposToSkip); + } + @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/server/src/test/java/org/opensearch/cluster/coordination/JoinTaskExecutorTests.java b/server/src/test/java/org/opensearch/cluster/coordination/JoinTaskExecutorTests.java index 3e343e95f6c4b..9cb1bd0b57132 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/JoinTaskExecutorTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/JoinTaskExecutorTests.java @@ -72,6 +72,7 @@ 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_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; 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.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; @@ -944,6 +945,145 @@ public void testNodeJoinInMixedMode() { JoinTaskExecutor.ensureNodesCompatibility(joiningNode2, currentNodes, metadata); } + public void testRemoteRoutingTableRepoAbsentNodeJoin() { + + final DiscoveryNode existingNode = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + + ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(existingNode).localNodeId(existingNode.getId()).build()) + .build(); + + DiscoveryNode joiningNode = newDiscoveryNode(remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO)); + JoinTaskExecutor.ensureNodesCompatibility(joiningNode, currentState.getNodes(), currentState.metadata()); + } + + public void testRemoteRoutingTableNodeJoinRepoPresentInJoiningNode() { + final DiscoveryNode existingNode = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(existingNode).localNodeId(existingNode.getId()).build()) + .build(); + + Map attr = remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO); + attr.putAll(remoteRoutingTableAttributes(ROUTING_TABLE_REPO)); + DiscoveryNode joiningNode = newDiscoveryNode(attr); + JoinTaskExecutor.ensureNodesCompatibility(joiningNode, currentState.getNodes(), currentState.metadata()); + } + + public void testRemoteRoutingTableNodeJoinRepoPresentInExistingNode() { + Map attr = remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO); + attr.putAll(remoteRoutingTableAttributes(ROUTING_TABLE_REPO)); + final DiscoveryNode existingNode = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + attr, + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + + ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(existingNode).localNodeId(existingNode.getId()).build()) + .build(); + + DiscoveryNode joiningNode = newDiscoveryNode(remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO)); + assertThrows( + IllegalStateException.class, + () -> JoinTaskExecutor.ensureNodesCompatibility(joiningNode, currentState.getNodes(), currentState.metadata()) + ); + } + + public void testRemoteRoutingTableNodeJoinRepoPresentInBothNode() { + Map attr = remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO); + attr.putAll(remoteRoutingTableAttributes(ROUTING_TABLE_REPO)); + final DiscoveryNode existingNode = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + attr, + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + + ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(existingNode).localNodeId(existingNode.getId()).build()) + .build(); + + DiscoveryNode joiningNode = newDiscoveryNode(attr); + JoinTaskExecutor.ensureNodesCompatibility(joiningNode, currentState.getNodes(), currentState.metadata()); + } + + public void testRemoteRoutingTableNodeJoinNodeWithRemoteAndRoutingRepoDifference() { + Map attr = remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO); + attr.putAll(remoteRoutingTableAttributes(ROUTING_TABLE_REPO)); + final DiscoveryNode existingNode = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + attr, + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + + final DiscoveryNode existingNode2 = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + + ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(existingNode2).add(existingNode).localNodeId(existingNode.getId()).build()) + .build(); + + DiscoveryNode joiningNode = newDiscoveryNode(attr); + JoinTaskExecutor.ensureNodesCompatibility(joiningNode, currentState.getNodes(), currentState.metadata()); + } + + public void testRemoteRoutingTableNodeJoinNodeWithRemoteAndRoutingRepoDifferenceMixedMode() { + Map attr = remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO); + attr.putAll(remoteRoutingTableAttributes(ROUTING_TABLE_REPO)); + final DiscoveryNode existingNode = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + attr, + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + + final DiscoveryNode existingNode2 = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + remoteStoreNodeAttributes(SEGMENT_REPO, TRANSLOG_REPO), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + + final Settings settings = Settings.builder() + .put(MIGRATION_DIRECTION_SETTING.getKey(), RemoteStoreNodeService.Direction.REMOTE_STORE) + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed") + .build(); + final Settings nodeSettings = Settings.builder().put(REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); + FeatureFlags.initializeFeatureFlags(nodeSettings); + Metadata metadata = Metadata.builder().persistentSettings(settings).build(); + ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(existingNode2).add(existingNode).localNodeId(existingNode.getId()).build()) + .metadata(metadata) + .build(); + + DiscoveryNode joiningNode = newDiscoveryNode(attr); + JoinTaskExecutor.ensureNodesCompatibility(joiningNode, currentState.getNodes(), currentState.metadata()); + } + private void validateRepositoryMetadata(ClusterState updatedState, DiscoveryNode existingNode, int expectedRepositories) throws Exception { @@ -985,6 +1125,7 @@ private DiscoveryNode newDiscoveryNode(Map attributes) { private static final String TRANSLOG_REPO = "translog-repo"; private static final String CLUSTER_STATE_REPO = "cluster-state-repo"; private static final String COMMON_REPO = "remote-repo"; + private static final String ROUTING_TABLE_REPO = "routing-table-repo"; private Map remoteStoreNodeAttributes(String segmentRepoName, String translogRepoName) { return remoteStoreNodeAttributes(segmentRepoName, translogRepoName, CLUSTER_STATE_REPO); @@ -1049,6 +1190,28 @@ private Map remoteStateNodeAttributes(String clusterStateRepo) { }; } + private Map remoteRoutingTableAttributes(String repoName) { + String routingTableRepositoryTypeAttributeKey = String.format( + Locale.getDefault(), + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + repoName + ); + String routingTableRepositorySettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + repoName + ); + + return new HashMap<>() { + { + put(REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, repoName); + putIfAbsent(routingTableRepositoryTypeAttributeKey, "s3"); + putIfAbsent(routingTableRepositorySettingsAttributeKeyPrefix + "bucket", "state_bucket"); + putIfAbsent(routingTableRepositorySettingsAttributeKeyPrefix + "base_path", "/state/path"); + } + }; + } + private void validateAttributes(Map remoteStoreNodeAttributes, ClusterState currentState, DiscoveryNode existingNode) { DiscoveryNode joiningNode = newDiscoveryNode(remoteStoreNodeAttributes); Exception e = assertThrows( diff --git a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java new file mode 100644 index 0000000000000..9a9cbfa153259 --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java @@ -0,0 +1,77 @@ +/* + * 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.routing.remote; + +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.repositories.FilterRepository; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.RepositoryMissingException; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.After; +import org.junit.Before; + +import java.util.function.Supplier; + +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class RemoteRoutingTableServiceTests extends OpenSearchTestCase { + + private RemoteRoutingTableService remoteRoutingTableService; + private Supplier repositoriesServiceSupplier; + private RepositoriesService repositoriesService; + private BlobStoreRepository blobStoreRepository; + + @Before + public void setup() { + repositoriesServiceSupplier = mock(Supplier.class); + repositoriesService = mock(RepositoriesService.class); + when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); + + Settings settings = Settings.builder() + .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, "routing_repository") + .build(); + + blobStoreRepository = mock(BlobStoreRepository.class); + when(repositoriesService.repository("routing_repository")).thenReturn(blobStoreRepository); + + Settings nodeSettings = Settings.builder().put(REMOTE_PUBLICATION_EXPERIMENTAL, "true").build(); + FeatureFlags.initializeFeatureFlags(nodeSettings); + + remoteRoutingTableService = new RemoteRoutingTableService(repositoriesServiceSupplier, settings); + } + + @After + public void teardown() throws Exception { + super.tearDown(); + remoteRoutingTableService.close(); + } + + public void testFailInitializationWhenRemoteRoutingDisabled() { + final Settings settings = Settings.builder().build(); + assertThrows(AssertionError.class, () -> new RemoteRoutingTableService(repositoriesServiceSupplier, settings)); + } + + public void testFailStartWhenRepositoryNotSet() { + doThrow(new RepositoryMissingException("repository missing")).when(repositoriesService).repository("routing_repository"); + assertThrows(RepositoryMissingException.class, () -> remoteRoutingTableService.start()); + } + + public void testFailStartWhenNotBlobRepository() { + final FilterRepository filterRepository = mock(FilterRepository.class); + when(repositoriesService.repository("routing_repository")).thenReturn(filterRepository); + assertThrows(AssertionError.class, () -> remoteRoutingTableService.start()); + } + +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index 1b242b921c0d7..aa5996d734d27 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -33,6 +33,7 @@ import org.opensearch.common.network.NetworkModule; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.core.ParseField; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesArray; @@ -86,6 +87,7 @@ import org.mockito.ArgumentMatchers; import static java.util.stream.Collectors.toList; +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; import static org.opensearch.gateway.remote.RemoteClusterStateService.FORMAT_PARAMS; @@ -99,6 +101,7 @@ 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_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; @@ -1496,6 +1499,33 @@ public void testGlobalMetadataUploadWaitTimeSetting() { assertEquals(globalMetadataUploadTimeout, remoteClusterStateService.getGlobalMetadataUploadTimeout().seconds()); } + public void testRemoteRoutingTableNotInitializedWhenDisabled() { + assertFalse(remoteClusterStateService.getRemoteRoutingTableService().isPresent()); + } + + public void testRemoteRoutingTableInitializedWhenEnabled() { + Settings newSettings = Settings.builder() + .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, "routing_repository") + .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "remote_store_repository") + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .build(); + clusterSettings.applySettings(newSettings); + + Settings nodeSettings = Settings.builder().put(REMOTE_PUBLICATION_EXPERIMENTAL, "true").build(); + FeatureFlags.initializeFeatureFlags(nodeSettings); + + remoteClusterStateService = new RemoteClusterStateService( + "test-node-id", + repositoriesServiceSupplier, + newSettings, + clusterSettings, + () -> 0L, + threadPool, + List.of(new RemoteIndexPathUploader(threadPool, newSettings, repositoriesServiceSupplier, clusterSettings)) + ); + assertTrue(remoteClusterStateService.getRemoteRoutingTableService().isPresent()); + } + private void mockObjectsForGettingPreviousClusterUUID(Map clusterUUIDsPointers) throws IOException { mockObjectsForGettingPreviousClusterUUID(clusterUUIDsPointers, false, Collections.emptyMap()); } diff --git a/server/src/test/java/org/opensearch/node/RemoteStoreNodeAttributeTests.java b/server/src/test/java/org/opensearch/node/RemoteStoreNodeAttributeTests.java index c4ba271d27ae9..de7f8977686a7 100644 --- a/server/src/test/java/org/opensearch/node/RemoteStoreNodeAttributeTests.java +++ b/server/src/test/java/org/opensearch/node/RemoteStoreNodeAttributeTests.java @@ -19,6 +19,7 @@ import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.List; import java.util.Locale; import java.util.Map; @@ -28,6 +29,7 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_CRYPTO_SETTINGS_PREFIX; 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_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; 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; @@ -148,4 +150,77 @@ public void testNoCryptoMetadata() throws UnknownHostException { RepositoryMetadata repositoryMetadata = remoteStoreNodeAttribute.getRepositoriesMetadata().repositories().get(0); assertNull(repositoryMetadata.cryptoMetadata()); } + + public void testEqualsWithRepoSkip() throws UnknownHostException { + String repoName = "remote-store-A"; + String repoTypeSettingKey = String.format(Locale.ROOT, REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, repoName); + String repoSettingsKey = String.format(Locale.ROOT, REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, repoName); + Map attr = Map.of( + REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, + repoName, + REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, + repoName, + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, + repoName, + repoTypeSettingKey, + "s3", + repoSettingsKey, + "abc", + repoSettingsKey + "base_path", + "xyz" + ); + DiscoveryNode node = new DiscoveryNode( + "C", + new TransportAddress(InetAddress.getByName("localhost"), 9876), + attr, + emptySet(), + Version.CURRENT + ); + + RemoteStoreNodeAttribute remoteStoreNodeAttribute = new RemoteStoreNodeAttribute(node); + + String routingTableRepoName = "remote-store-B"; + String routingTableRepoTypeSettingKey = String.format( + Locale.ROOT, + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + routingTableRepoName + ); + String routingTableRepoSettingsKey = String.format( + Locale.ROOT, + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + routingTableRepoName + ); + + Map attr2 = Map.of( + REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, + repoName, + REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, + repoName, + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, + repoName, + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, + routingTableRepoName, + repoTypeSettingKey, + "s3", + repoSettingsKey, + "abc", + repoSettingsKey + "base_path", + "xyz", + routingTableRepoTypeSettingKey, + "s3", + routingTableRepoSettingsKey, + "xyz" + ); + DiscoveryNode node2 = new DiscoveryNode( + "C", + new TransportAddress(InetAddress.getByName("localhost"), 9876), + attr2, + emptySet(), + Version.CURRENT + ); + RemoteStoreNodeAttribute remoteStoreNodeAttribute2 = new RemoteStoreNodeAttribute(node2); + + assertFalse(remoteStoreNodeAttribute.equalsWithRepoSkip(remoteStoreNodeAttribute2, List.of())); + assertTrue(remoteStoreNodeAttribute.equalsWithRepoSkip(remoteStoreNodeAttribute2, List.of(routingTableRepoName))); + } } From ac78f0a4dc39f5b4be32adc8bf54999b2b44cd14 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Wed, 5 Jun 2024 10:03:56 -0700 Subject: [PATCH 48/77] Shorten PR template checklist (#13979) This commit removes items from the checklist that were redundant in order to reduce the work required to submit a PR. This also removes the check that enforces checklist completion as requested by issue #10970. Resolves #13965 Resolves #10970 Signed-off-by: Andrew Ross --- .github/pull_request_template.md | 12 +++------- .github/workflows/pull-request-checks.yml | 29 ----------------------- 2 files changed, 3 insertions(+), 38 deletions(-) delete mode 100644 .github/workflows/pull-request-checks.yml diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 6281fa0af3e36..5476637b84e92 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -13,15 +13,9 @@ Resolves #[Issue number to be closed when this PR is merged] ### Check List -- [ ] New functionality includes testing. - - [ ] All tests pass -- [ ] New functionality has been documented. - - [ ] New functionality has javadoc added -- [ ] API changes companion pull request [created](https://github.com/opensearch-project/opensearch-api-specification/blob/main/DEVELOPER_GUIDE.md). -- [ ] Failing checks are inspected and point to the corresponding known issue(s) (See: [Troubleshooting Failing Builds](../blob/main/CONTRIBUTING.md#troubleshooting-failing-builds)) -- [ ] Commits are signed per the DCO using --signoff -- [ ] Commit changes are listed out in CHANGELOG.md file (See: [Changelog](../blob/main/CONTRIBUTING.md#changelog)) -- [ ] Public documentation issue/PR [created](https://github.com/opensearch-project/documentation-website/issues/new/choose) +- [ ] Functionality includes testing. +- [ ] API changes companion pull request [created](https://github.com/opensearch-project/opensearch-api-specification/blob/main/DEVELOPER_GUIDE.md), if applicable. +- [ ] Public documentation issue/PR [created](https://github.com/opensearch-project/documentation-website/issues/new/choose), if applicable. By submitting this pull request, I confirm that my contribution is made under the terms of the Apache 2.0 license. For more information on following Developer Certificate of Origin and signing off your commits, please check [here](https://github.com/opensearch-project/OpenSearch/blob/main/CONTRIBUTING.md#developer-certificate-of-origin). diff --git a/.github/workflows/pull-request-checks.yml b/.github/workflows/pull-request-checks.yml deleted file mode 100644 index eec363572478c..0000000000000 --- a/.github/workflows/pull-request-checks.yml +++ /dev/null @@ -1,29 +0,0 @@ -name: Pull Request Checks - -on: - pull_request: - types: - [ - opened, - edited, - review_requested, - synchronize, - reopened, - ready_for_review, - ] - -jobs: - verify-description-checklist: - name: Verify Description Checklist - runs-on: ubuntu-latest - steps: - - uses: peternied/check-pull-request-description-checklist@v1.1 - if: github.event.pull_request.user.login != 'dependabot[bot]' - with: - checklist-items: | - New functionality includes testing. - All tests pass - New functionality has been documented. - New functionality has javadoc added - Commits are signed per the DCO using --signoff - Commit changes are listed out in CHANGELOG.md file (See: [Changelog](../blob/main/CONTRIBUTING.md#changelog)) From ba0df742037b49dc8c85efcf9ccec98fd3a1d69d Mon Sep 17 00:00:00 2001 From: Muneer Kolarkunnu <33829651+akolarkunnu@users.noreply.github.com> Date: Wed, 5 Jun 2024 22:40:39 +0530 Subject: [PATCH 49/77] COMPAT locale provider will be removed in a future release (#13988) * COMPAT locale provider will be removed in a future release Description: From JDK21 onwards a new warning has started to come, "WARNING: COMPAT locale provider will be removed in a future release". So, we have to avoid usage of COMPAT provider. We were setting exlpicitly to COMPAT locale provider in couple of places, this change is to convert COMPAT to CLDR locale provider. After this change, couple of tests started to fail becasue some locale data has minor changes in CLDR compared to COMPAT. For example, day and month short names of GERMAN "de" locale are different in CLDR and COMPAT, just need to add a . in the end for CLDR. Resolves #11550 Signed-off-by: Muneer Kolarkunnu Signed-off-by: akolarkunnu * COMPAT locale provider will be removed in a future release Description: From JDK21 onwards a new warning has started to come, "WARNING: COMPAT locale provider will be removed in a future release". So, we have to avoid usage of COMPAT provider. We were setting exlpicitly to COMPAT locale provider in couple of places, this change is to convert COMPAT to CLDR locale provider. After this change, couple of tests started to fail becasue some locale data has minor changes in CLDR compared to COMPAT. For example, day and month short names of GERMAN "de" locale are different in CLDR and COMPAT, just need to add a . in the end for CLDR. Resolves #11550 Signed-off-by: Abdul Muneer Kolarkunnu muneer.kolarkunnu@netapp.com Signed-off-by: akolarkunnu --------- Signed-off-by: Muneer Kolarkunnu Signed-off-by: akolarkunnu Signed-off-by: Abdul Muneer Kolarkunnu muneer.kolarkunnu@netapp.com Co-authored-by: akolarkunnu --- CHANGELOG-3.0.md | 1 + .../gradle/OpenSearchTestBasePlugin.java | 2 +- .../tools/launchers/SystemJvmOptions.java | 7 +------ gradle/ide.gradle | 2 +- .../opensearch/search/query/SearchQueryIT.java | 18 ++++++++---------- .../index/mapper/DateFieldMapperTests.java | 2 +- 6 files changed, 13 insertions(+), 19 deletions(-) diff --git a/CHANGELOG-3.0.md b/CHANGELOG-3.0.md index 964383078c38d..1cc12f66d52e1 100644 --- a/CHANGELOG-3.0.md +++ b/CHANGELOG-3.0.md @@ -17,6 +17,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Dependencies ### Changed +- Changed locale provider from COMPAT to CLDR ([13988](https://github.com/opensearch-project/OpenSearch/pull/13988)) - Migrate client transports to Apache HttpClient / Core 5.x ([#4459](https://github.com/opensearch-project/OpenSearch/pull/4459)) - Change http code on create index API with bad input raising NotXContentException from 500 to 400 ([#4773](https://github.com/opensearch-project/OpenSearch/pull/4773)) - Improve summary error message for invalid setting updates ([#4792](https://github.com/opensearch-project/OpenSearch/pull/4792)) diff --git a/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java index 2ea8c2d015ecc..d0cb2da9c1dd3 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java @@ -110,7 +110,7 @@ public void execute(Task t) { if (BuildParams.getRuntimeJavaVersion() == JavaVersion.VERSION_1_8) { test.systemProperty("java.locale.providers", "SPI,JRE"); } else { - test.systemProperty("java.locale.providers", "SPI,COMPAT"); + test.systemProperty("java.locale.providers", "SPI,CLDR"); if (test.getJavaVersion().compareTo(JavaVersion.VERSION_17) < 0) { test.jvmArgs("--illegal-access=warn"); } diff --git a/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java b/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java index 726c381db09f6..af7138569972a 100644 --- a/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java +++ b/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java @@ -105,13 +105,8 @@ private static String javaLocaleProviders() { SPI setting is used to allow loading custom CalendarDataProvider in jdk8 it has to be loaded from jre/lib/ext, in jdk9+ it is already within ES project and on a classpath - - Due to internationalization enhancements in JDK 9 OpenSearch need to set the provider to COMPAT otherwise time/date - parsing will break in an incompatible way for some date patterns and locales. - //TODO COMPAT will be deprecated in at some point, see please https://bugs.openjdk.java.net/browse/JDK-8232906 - See also: documentation in server/org.opensearch.common.time.IsoCalendarDataProvider */ - return "-Djava.locale.providers=SPI,COMPAT"; + return "-Djava.locale.providers=SPI,CLDR"; } } diff --git a/gradle/ide.gradle b/gradle/ide.gradle index 14d6b2982ccd0..4c4f3b07836c5 100644 --- a/gradle/ide.gradle +++ b/gradle/ide.gradle @@ -81,7 +81,7 @@ if (System.getProperty('idea.active') == 'true') { } runConfigurations { defaults(JUnit) { - vmParameters = '-ea -Djava.locale.providers=SPI,COMPAT' + vmParameters = '-ea -Djava.locale.providers=SPI,CLDR' if (BuildParams.runtimeJavaVersion > JavaVersion.VERSION_17) { vmParameters += ' -Djava.security.manager=allow' } diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java index a58db51780826..01ad06757640c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java @@ -1914,14 +1914,8 @@ public void testRangeQueryWithTimeZone() throws Exception { * Test range with a custom locale, e.g. "de" in this case. Documents here mention the day of week * as "Mi" for "Mittwoch (Wednesday" and "Do" for "Donnerstag (Thursday)" and the month in the query * as "Dez" for "Dezember (December)". - * Note: this test currently needs the JVM arg `-Djava.locale.providers=SPI,COMPAT` to be set. - * When running with gradle this is done implicitly through the BuildPlugin, but when running from - * an IDE this might need to be set manually in the run configuration. See also CONTRIBUTING.md section - * on "Configuring IDEs And Running Tests". */ public void testRangeQueryWithLocaleMapping() throws Exception { - assert ("SPI,COMPAT".equals(System.getProperty("java.locale.providers"))) : "`-Djava.locale.providers=SPI,COMPAT` needs to be set"; - assertAcked( prepareCreate("test").setMapping( jsonBuilder().startObject() @@ -1938,17 +1932,21 @@ public void testRangeQueryWithLocaleMapping() throws Exception { indexRandom( true, - client().prepareIndex("test").setId("1").setSource("date_field", "Mi, 06 Dez 2000 02:55:00 -0800"), - client().prepareIndex("test").setId("2").setSource("date_field", "Do, 07 Dez 2000 02:55:00 -0800") + client().prepareIndex("test").setId("1").setSource("date_field", "Mi., 06 Dez. 2000 02:55:00 -0800"), + client().prepareIndex("test").setId("2").setSource("date_field", "Do., 07 Dez. 2000 02:55:00 -0800") ); SearchResponse searchResponse = client().prepareSearch("test") - .setQuery(QueryBuilders.rangeQuery("date_field").gte("Di, 05 Dez 2000 02:55:00 -0800").lte("Do, 07 Dez 2000 00:00:00 -0800")) + .setQuery( + QueryBuilders.rangeQuery("date_field").gte("Di., 05 Dez. 2000 02:55:00 -0800").lte("Do., 07 Dez. 2000 00:00:00 -0800") + ) .get(); assertHitCount(searchResponse, 1L); searchResponse = client().prepareSearch("test") - .setQuery(QueryBuilders.rangeQuery("date_field").gte("Di, 05 Dez 2000 02:55:00 -0800").lte("Fr, 08 Dez 2000 00:00:00 -0800")) + .setQuery( + QueryBuilders.rangeQuery("date_field").gte("Di., 05 Dez. 2000 02:55:00 -0800").lte("Fr., 08 Dez. 2000 00:00:00 -0800") + ) .get(); assertHitCount(searchResponse, 2L); } diff --git a/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java index 2aa310ae959d9..98bcaa3a1a46b 100644 --- a/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java @@ -208,7 +208,7 @@ public void testChangeLocale() throws IOException { fieldMapping(b -> b.field("type", "date").field("format", "E, d MMM yyyy HH:mm:ss Z").field("locale", "de")) ); - mapper.parse(source(b -> b.field("field", "Mi, 06 Dez 2000 02:55:00 -0800"))); + mapper.parse(source(b -> b.field("field", "Mi., 06 Dez. 2000 02:55:00 -0800"))); } public void testNullValue() throws IOException { From 619554d8caebc602d946aa01805f3c93756e3382 Mon Sep 17 00:00:00 2001 From: Divya Madala <113469545+Divyaasm@users.noreply.github.com> Date: Wed, 5 Jun 2024 19:18:05 -0700 Subject: [PATCH 50/77] Add release notes for 1.3.17 (#14013) --- release-notes/opensearch.release-notes-1.3.17.md | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 release-notes/opensearch.release-notes-1.3.17.md diff --git a/release-notes/opensearch.release-notes-1.3.17.md b/release-notes/opensearch.release-notes-1.3.17.md new file mode 100644 index 0000000000000..5218b9e3be20c --- /dev/null +++ b/release-notes/opensearch.release-notes-1.3.17.md @@ -0,0 +1,6 @@ +## 2024-05-30 Version 1.3.17 Release Notes + +### Upgrades +- OpenJDK Update (April 2024 Patch releases), update to Eclipse Temurin 11.0.23+9 ([#13406](https://github.com/opensearch-project/OpenSearch/pull/13406)) +- Upgrade BouncyCastle dependencies from 1.75 to 1.78.1 resolving [CVE-2024-30172], [CVE-2024-30171] and [CVE-2024-29857] +- Bump `netty` from 4.1.109.Final to 4.1.110.Final ([#13802](https://github.com/opensearch-project/OpenSearch/pull/13802)) From 156eca3433ee0910f7fc21340f0052ada7e7533e Mon Sep 17 00:00:00 2001 From: Sooraj Sinha <81695996+soosinha@users.noreply.github.com> Date: Thu, 6 Jun 2024 08:23:46 +0530 Subject: [PATCH 51/77] [Remote Cluster State] Remote state interfaces (#13785) * Remote Writable Entity interfaces Signed-off-by: Sooraj Sinha --- .../AbstractRemoteWritableBlobEntity.java | 91 +++++++++++++++ .../common/remote/BlobPathParameters.java | 34 ++++++ .../remote/RemoteWritableEntityStore.java | 28 +++++ .../common/remote/RemoteWriteableEntity.java | 34 ++++++ .../common/remote/package-info.java | 11 ++ .../remote/RemoteClusterStateUtils.java | 23 ++++ .../model/RemoteClusterStateBlobStore.java | 107 ++++++++++++++++++ .../gateway/remote/model/package-info.java | 12 ++ 8 files changed, 340 insertions(+) create mode 100644 server/src/main/java/org/opensearch/common/remote/AbstractRemoteWritableBlobEntity.java create mode 100644 server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java create mode 100644 server/src/main/java/org/opensearch/common/remote/RemoteWritableEntityStore.java create mode 100644 server/src/main/java/org/opensearch/common/remote/RemoteWriteableEntity.java create mode 100644 server/src/main/java/org/opensearch/common/remote/package-info.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/model/package-info.java diff --git a/server/src/main/java/org/opensearch/common/remote/AbstractRemoteWritableBlobEntity.java b/server/src/main/java/org/opensearch/common/remote/AbstractRemoteWritableBlobEntity.java new file mode 100644 index 0000000000000..632b2b70d61df --- /dev/null +++ b/server/src/main/java/org/opensearch/common/remote/AbstractRemoteWritableBlobEntity.java @@ -0,0 +1,91 @@ +/* + * 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.common.remote; + +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.PATH_DELIMITER; + +/** + * An extension of {@link RemoteWriteableEntity} class which caters to the use case of writing to and reading from a blob storage + * + * @param The class type which can be uploaded to or downloaded from a blob storage. + */ +public abstract class AbstractRemoteWritableBlobEntity implements RemoteWriteableEntity { + + protected String blobFileName; + + protected String blobName; + private final String clusterUUID; + private final Compressor compressor; + private final NamedXContentRegistry namedXContentRegistry; + private String[] pathTokens; + + public AbstractRemoteWritableBlobEntity( + final String clusterUUID, + final Compressor compressor, + final NamedXContentRegistry namedXContentRegistry + ) { + this.clusterUUID = clusterUUID; + this.compressor = compressor; + this.namedXContentRegistry = namedXContentRegistry; + } + + public abstract BlobPathParameters getBlobPathParameters(); + + public String getFullBlobName() { + return blobName; + } + + public String getBlobFileName() { + if (blobFileName == null) { + String[] pathTokens = getBlobPathTokens(); + if (pathTokens == null || pathTokens.length < 1) { + return null; + } + blobFileName = pathTokens[pathTokens.length - 1]; + } + return blobFileName; + } + + public String[] getBlobPathTokens() { + if (pathTokens != null) { + return pathTokens; + } + if (blobName == null) { + return null; + } + pathTokens = blobName.split(PATH_DELIMITER); + return pathTokens; + } + + public abstract String generateBlobFileName(); + + public String clusterUUID() { + return clusterUUID; + } + + public abstract UploadedMetadata getUploadedMetadata(); + + public void setFullBlobName(BlobPath blobPath) { + this.blobName = blobPath.buildAsString() + blobFileName; + } + + public NamedXContentRegistry getNamedXContentRegistry() { + return namedXContentRegistry; + } + + protected Compressor getCompressor() { + return compressor; + } + +} diff --git a/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java b/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.java new file mode 100644 index 0000000000000..58c73a804b66a --- /dev/null +++ b/server/src/main/java/org/opensearch/common/remote/BlobPathParameters.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.common.remote; + +import java.util.List; + +/** + * Parameters which can be used to construct a blob path + * + */ +public class BlobPathParameters { + + private final List pathTokens; + private final String filePrefix; + + public BlobPathParameters(final List pathTokens, final String filePrefix) { + this.pathTokens = pathTokens; + this.filePrefix = filePrefix; + } + + public List getPathTokens() { + return pathTokens; + } + + public String getFilePrefix() { + return filePrefix; + } +} diff --git a/server/src/main/java/org/opensearch/common/remote/RemoteWritableEntityStore.java b/server/src/main/java/org/opensearch/common/remote/RemoteWritableEntityStore.java new file mode 100644 index 0000000000000..ccf7cafff1730 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/remote/RemoteWritableEntityStore.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.common.remote; + +import org.opensearch.core.action.ActionListener; + +import java.io.IOException; + +/** + * An interface to read/write an object from/to a remote storage. This interface is agnostic of the remote storage type. + * + * @param The object type which can be uploaded to or downloaded from remote storage. + * @param The wrapper entity which provides methods for serializing/deserializing entity T. + */ +public interface RemoteWritableEntityStore> { + + public void writeAsync(U entity, ActionListener listener); + + public T read(U entity) throws IOException; + + public void readAsync(U entity, ActionListener listener); +} diff --git a/server/src/main/java/org/opensearch/common/remote/RemoteWriteableEntity.java b/server/src/main/java/org/opensearch/common/remote/RemoteWriteableEntity.java new file mode 100644 index 0000000000000..778c24dce2e27 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/remote/RemoteWriteableEntity.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.common.remote; + +import java.io.IOException; +import java.io.InputStream; + +/** + * An interface to which provides defines the serialization/deserialization methods for objects to be uploaded to or downloaded from remote store. + * This interface is agnostic of the remote storage type. + * + * @param The object type which can be uploaded to or downloaded from remote storage. + */ +public interface RemoteWriteableEntity { + /** + * @return An InputStream created by serializing the entity T + * @throws IOException Exception encountered while serialization + */ + public InputStream serialize() throws IOException; + + /** + * @param inputStream The InputStream which is used to read the serialized entity + * @return The entity T after deserialization + * @throws IOException Exception encountered while deserialization + */ + public T deserialize(InputStream inputStream) throws IOException; + +} diff --git a/server/src/main/java/org/opensearch/common/remote/package-info.java b/server/src/main/java/org/opensearch/common/remote/package-info.java new file mode 100644 index 0000000000000..08ff9e910dc98 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/remote/package-info.java @@ -0,0 +1,11 @@ +/* + * 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. + */ +/** + * Common remote store package + */ +package org.opensearch.common.remote; diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java new file mode 100644 index 0000000000000..500d1af0211e8 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java @@ -0,0 +1,23 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import java.nio.charset.StandardCharsets; +import java.util.Base64; + +/** + * Utility class for Remote Cluster State + */ +public class RemoteClusterStateUtils { + public static final String PATH_DELIMITER = "/"; + + public static String encodeString(String content) { + return Base64.getUrlEncoder().withoutPadding().encodeToString(content.getBytes(StandardCharsets.UTF_8)); + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java new file mode 100644 index 0000000000000..1aeecc4e70382 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.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. + */ + +package org.opensearch.gateway.remote.model; + +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.RemoteWritableEntityStore; +import org.opensearch.common.remote.RemoteWriteableEntity; +import org.opensearch.core.action.ActionListener; +import org.opensearch.gateway.remote.RemoteClusterStateUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.io.InputStream; +import java.util.concurrent.ExecutorService; + +/** + * Abstract class for a blob type storage + * + * @param The entity which can be uploaded to / downloaded from blob store + * @param The concrete class implementing {@link RemoteWriteableEntity} which is used as a wrapper for T entity. + */ +public class RemoteClusterStateBlobStore> implements RemoteWritableEntityStore { + + private final BlobStoreTransferService transferService; + private final BlobStoreRepository blobStoreRepository; + private final String clusterName; + private final ExecutorService executorService; + + public RemoteClusterStateBlobStore( + final BlobStoreTransferService blobStoreTransferService, + final BlobStoreRepository blobStoreRepository, + final String clusterName, + final ThreadPool threadPool, + final String executor + ) { + this.transferService = blobStoreTransferService; + this.blobStoreRepository = blobStoreRepository; + this.clusterName = clusterName; + this.executorService = threadPool.executor(executor); + } + + @Override + public void writeAsync(final U entity, final ActionListener listener) { + try { + try (InputStream inputStream = entity.serialize()) { + BlobPath blobPath = getBlobPathForUpload(entity); + entity.setFullBlobName(blobPath); + // TODO uncomment below logic after merging PR https://github.com/opensearch-project/OpenSearch/pull/13836 + // transferService.uploadBlob(inputStream, getBlobPathForUpload(entity), entity.getBlobFileName(), WritePriority.URGENT, + // listener); + } + } catch (Exception e) { + listener.onFailure(e); + } + } + + public T read(final U entity) throws IOException { + // TODO Add timing logs and tracing + assert entity.getFullBlobName() != null; + return entity.deserialize(transferService.downloadBlob(getBlobPathForDownload(entity), entity.getBlobFileName())); + } + + @Override + public void readAsync(final U entity, final ActionListener listener) { + executorService.execute(() -> { + try { + listener.onResponse(read(entity)); + } catch (Exception e) { + listener.onFailure(e); + } + }); + } + + private BlobPath getBlobPathForUpload(final AbstractRemoteWritableBlobEntity obj) { + BlobPath blobPath = blobStoreRepository.basePath() + .add(RemoteClusterStateUtils.encodeString(clusterName)) + .add("cluster-state") + .add(obj.clusterUUID()); + for (String token : obj.getBlobPathParameters().getPathTokens()) { + blobPath = blobPath.add(token); + } + return blobPath; + } + + private BlobPath getBlobPathForDownload(final AbstractRemoteWritableBlobEntity obj) { + String[] pathTokens = obj.getBlobPathTokens(); + BlobPath blobPath = new BlobPath(); + if (pathTokens == null || pathTokens.length < 1) { + return blobPath; + } + // Iterate till second last path token to get the blob folder + for (int i = 0; i < pathTokens.length - 1; i++) { + blobPath = blobPath.add(pathTokens[i]); + } + return blobPath; + } + +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/package-info.java b/server/src/main/java/org/opensearch/gateway/remote/model/package-info.java new file mode 100644 index 0000000000000..c0d13d15cc885 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/model/package-info.java @@ -0,0 +1,12 @@ +/* + * 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 containing models for remote cluster state + */ +package org.opensearch.gateway.remote.model; From 3de5d3c273e744a76459c88f3a6341ed3173f5a0 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 6 Jun 2024 08:44:18 +0530 Subject: [PATCH 52/77] Optimize remote state stale file deletion (#13995) * Optimize remote state stale file deletion Signed-off-by: Shivansh Arora --- .../RemoteClusterStateCleanupManagerIT.java | 154 ++++++ .../remote/RemoteClusterStateServiceIT.java | 63 --- .../RemoteStoreBaseIntegTestCase.java | 10 + .../common/settings/ClusterSettings.java | 2 + .../RemoteClusterStateCleanupManager.java | 413 ++++++++++++++++ .../remote/RemoteClusterStateService.java | 275 ++--------- .../main/java/org/opensearch/node/Node.java | 7 +- .../GatewayMetaStatePersistedStateTests.java | 6 +- ...RemoteClusterStateCleanupManagerTests.java | 446 ++++++++++++++++++ .../RemoteClusterStateServiceTests.java | 138 +----- 10 files changed, 1071 insertions(+), 443 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java create mode 100644 server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java new file mode 100644 index 0000000000000..e96dedaa3e6a0 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java @@ -0,0 +1,154 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.Settings; +import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.Before; + +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; +import static org.opensearch.indices.IndicesService.CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteClusterStateCleanupManagerIT extends RemoteStoreBaseIntegTestCase { + + private static final String INDEX_NAME = "test-index"; + + @Before + public void setup() { + asyncUploadMockFsRepo = false; + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); + } + + private Map initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount) { + prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount); + Map indexStats = indexData(1, false, INDEX_NAME); + assertEquals(shardCount * (replicaCount + 1), getNumShards(INDEX_NAME).totalNumShards); + ensureGreen(INDEX_NAME); + return indexStats; + } + + public void testRemoteCleanupTaskUpdated() { + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); + RemoteClusterStateCleanupManager remoteClusterStateCleanupManager = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateCleanupManager.class + ); + + assertEquals(CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval()); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + + // now disable + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), -1)) + .get(); + + assertEquals(-1, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMillis()); + assertFalse(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + + // now set Clean up interval to 1 min + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "1m")) + .get(); + assertEquals(1, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMinutes()); + } + + public void testRemoteCleanupDeleteStale() throws Exception { + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); + + // update cluster state 21 times to ensure that clean up has run after this will upload 42 manifest files + // to repository, if manifest files are less than that it means clean up has run + updateClusterStateNTimes(RETAINED_MANIFESTS + SKIP_CLEANUP_STATE_CHANGES + 1); + + RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); + BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); + BlobPath baseMetadataPath = repository.basePath() + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(getClusterState().metadata().clusterUUID()); + BlobPath manifestContainerPath = baseMetadataPath.add("manifest"); + + // set cleanup interval to 100 ms to make the test faster + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "100ms")) + .get(); + + assertTrue(response.isAcknowledged()); + + assertBusy(() -> { + int manifestFiles = repository.blobStore().blobContainer(manifestContainerPath).listBlobsByPrefix("manifest").size(); + logger.info("number of current manifest file: {}", manifestFiles); + // we can't guarantee that we have same number of manifest as Retained manifest in our repo as there can be other queued task + // other than replica count change which can upload new manifest files, that's why we check that number of manifests is between + // Retained manifests and Retained manifests + 2 * Skip cleanup state changes (each cluster state update uploads 2 manifests) + assertTrue( + "Current number of manifest files: " + manifestFiles, + manifestFiles >= RETAINED_MANIFESTS && manifestFiles < RETAINED_MANIFESTS + 2 * SKIP_CLEANUP_STATE_CHANGES + ); + }, 500, TimeUnit.MILLISECONDS); + + // disable the clean up to avoid race condition during shutdown + response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "-1")) + .get(); + + assertTrue(response.isAcknowledged()); + } + + private void updateClusterStateNTimes(int n) { + int newReplicaCount = randomIntBetween(0, 3); + for (int i = n; i > 0; i--) { + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING.getKey(), i, TimeUnit.SECONDS)) + .get(); + assertTrue(response.isAcknowledged()); + } + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java index 42120aa32eb47..ab2f0f0080566 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateServiceIT.java @@ -10,7 +10,6 @@ import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; import org.opensearch.discovery.DiscoveryStats; @@ -27,7 +26,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.CUSTOM_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; @@ -51,16 +49,6 @@ protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); } - private void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { - internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes); - internalCluster().startDataOnlyNodes(numDataOnlyNodes); - for (String index : indices.split(",")) { - createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); - ensureYellowAndNoInitializingShards(index); - ensureGreen(index); - } - } - private Map initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount) { prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount); Map indexStats = indexData(1, false, INDEX_NAME); @@ -69,49 +57,6 @@ private Map initialTestSetup(int shardCount, int replicaCount, int return indexStats; } - public void testFullClusterRestoreStaleDelete() throws Exception { - int shardCount = randomIntBetween(1, 2); - int replicaCount = 1; - int dataNodeCount = shardCount * (replicaCount + 1); - int clusterManagerNodeCount = 1; - - initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount); - setReplicaCount(0); - setReplicaCount(2); - setReplicaCount(0); - setReplicaCount(1); - setReplicaCount(0); - setReplicaCount(1); - setReplicaCount(0); - setReplicaCount(2); - setReplicaCount(0); - - RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( - RemoteClusterStateService.class - ); - - RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); - - BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); - BlobPath baseMetadataPath = repository.basePath() - .add( - Base64.getUrlEncoder() - .withoutPadding() - .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) - ) - .add("cluster-state") - .add(getClusterState().metadata().clusterUUID()); - - assertEquals(10, repository.blobStore().blobContainer(baseMetadataPath.add("manifest")).listBlobsByPrefix("manifest").size()); - - Map indexMetadataMap = remoteClusterStateService.getLatestClusterState( - cluster().getClusterName(), - getClusterState().metadata().clusterUUID() - ).getMetadata().getIndices(); - assertEquals(0, indexMetadataMap.values().stream().findFirst().get().getNumberOfReplicas()); - assertEquals(shardCount, indexMetadataMap.values().stream().findFirst().get().getNumberOfShards()); - } - public void testRemoteStateStats() { int shardCount = randomIntBetween(1, 2); int replicaCount = 1; @@ -241,12 +186,4 @@ private void validateNodesStatsResponse(NodesStatsResponse nodesStatsResponse) { assertNotNull(nodesStatsResponse.getNodes().get(0)); assertNotNull(nodesStatsResponse.getNodes().get(0).getDiscoveryStats()); } - - private void setReplicaCount(int replicaCount) { - client().admin() - .indices() - .prepareUpdateSettings(INDEX_NAME) - .setSettings(Settings.builder().put(SETTING_NUMBER_OF_REPLICAS, replicaCount)) - .get(); - } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 740aee69f7d80..64efcee6ef1b5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -350,4 +350,14 @@ protected void restore(boolean restoreAllShards, String... indices) { PlainActionFuture.newFuture() ); } + + protected void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { + internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes); + internalCluster().startDataOnlyNodes(numDataOnlyNodes); + for (String index : indices.split(",")) { + createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); + ensureYellowAndNoInitializingShards(index); + ensureGreen(index); + } + } } 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 7814518af471b..297fc98764d07 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -104,6 +104,7 @@ import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.PersistedClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateCleanupManager; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpTransportSettings; import org.opensearch.index.IndexModule; @@ -711,6 +712,7 @@ public void apply(Settings value, Settings current, Settings previous) { SearchRequestSlowLog.CLUSTER_SEARCH_REQUEST_SLOWLOG_LEVEL, // Remote cluster state settings + RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING, RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING, RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, RemoteClusterStateService.GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java new file mode 100644 index 0000000000000..2fca239b10efd --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java @@ -0,0 +1,413 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.logging.log4j.util.Strings; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.service.ClusterApplierService; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.AbstractAsyncTask; +import org.opensearch.core.action.ActionListener; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.threadpool.ThreadPool; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_FORMAT; +import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_METADATA_FORMAT; +import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; +import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_PATH_TOKEN; + +/** + * A Manager which provides APIs to clean up stale cluster state files and runs an async stale cleanup task + * + * @opensearch.internal + */ +public class RemoteClusterStateCleanupManager implements Closeable { + + public static final int RETAINED_MANIFESTS = 10; + public static final int SKIP_CLEANUP_STATE_CHANGES = 10; + public static final TimeValue CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT = TimeValue.timeValueMinutes(5); + public static final TimeValue CLUSTER_STATE_CLEANUP_INTERVAL_MINIMUM = TimeValue.MINUS_ONE; + + /** + * Setting to specify the interval to do run stale file cleanup job + * Min value -1 indicates that the stale file cleanup job should be disabled + */ + public static final Setting REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING = Setting.timeSetting( + "cluster.remote_store.state.cleanup_interval", + CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, + CLUSTER_STATE_CLEANUP_INTERVAL_MINIMUM, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private static final Logger logger = LogManager.getLogger(RemoteClusterStateCleanupManager.class); + private final RemoteClusterStateService remoteClusterStateService; + private final RemotePersistenceStats remoteStateStats; + private BlobStoreTransferService blobStoreTransferService; + private TimeValue staleFileCleanupInterval; + private final AtomicBoolean deleteStaleMetadataRunning = new AtomicBoolean(false); + private volatile AsyncStaleFileDeletion staleFileDeletionTask; + private long lastCleanupAttemptStateVersion; + private final ThreadPool threadpool; + private final ClusterApplierService clusterApplierService; + + public RemoteClusterStateCleanupManager(RemoteClusterStateService remoteClusterStateService, ClusterService clusterService) { + this.remoteClusterStateService = remoteClusterStateService; + this.remoteStateStats = remoteClusterStateService.getStats(); + ClusterSettings clusterSettings = clusterService.getClusterSettings(); + this.clusterApplierService = clusterService.getClusterApplierService(); + this.staleFileCleanupInterval = clusterSettings.get(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING); + this.threadpool = remoteClusterStateService.getThreadpool(); + // initialize with 0, a cleanup will be done when this node is elected master node and version is incremented more than threshold + this.lastCleanupAttemptStateVersion = 0; + clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING, this::updateCleanupInterval); + } + + void start() { + staleFileDeletionTask = new AsyncStaleFileDeletion(this); + } + + @Override + public void close() throws IOException { + if (staleFileDeletionTask != null) { + staleFileDeletionTask.close(); + } + } + + private BlobStoreTransferService getBlobStoreTransferService() { + if (blobStoreTransferService == null) { + blobStoreTransferService = new BlobStoreTransferService(remoteClusterStateService.getBlobStore(), threadpool); + } + return blobStoreTransferService; + } + + private void updateCleanupInterval(TimeValue updatedInterval) { + this.staleFileCleanupInterval = updatedInterval; + logger.info("updated remote state cleanup interval to {}", updatedInterval); + // After updating the interval, we need to close the current task and create a new one which will run with updated interval + if (staleFileDeletionTask != null && !staleFileDeletionTask.getInterval().equals(updatedInterval)) { + staleFileDeletionTask.setInterval(updatedInterval); + } + } + + // visible for testing + void cleanUpStaleFiles() { + ClusterState currentAppliedState = clusterApplierService.state(); + if (currentAppliedState.nodes().isLocalNodeElectedClusterManager()) { + long cleanUpAttemptStateVersion = currentAppliedState.version(); + assert Strings.isNotEmpty(currentAppliedState.getClusterName().value()) : "cluster name is not set"; + assert Strings.isNotEmpty(currentAppliedState.metadata().clusterUUID()) : "cluster uuid is not set"; + if (cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion > SKIP_CLEANUP_STATE_CHANGES) { + logger.info( + "Cleaning up stale remote state files for cluster [{}] with uuid [{}]. Last clean was done before {} updates", + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion + ); + this.deleteStaleClusterMetadata( + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + RETAINED_MANIFESTS + ); + lastCleanupAttemptStateVersion = cleanUpAttemptStateVersion; + } else { + logger.debug( + "Skipping cleanup of stale remote state files for cluster [{}] with uuid [{}]. Last clean was done before {} updates, which is less than threshold {}", + currentAppliedState.getClusterName().value(), + currentAppliedState.metadata().clusterUUID(), + cleanUpAttemptStateVersion - lastCleanupAttemptStateVersion, + SKIP_CLEANUP_STATE_CHANGES + ); + } + } else { + logger.debug("Skipping cleanup task as local node is not elected Cluster Manager"); + } + } + + private void addStaleGlobalMetadataPath(String fileName, Set filesToKeep, Set staleGlobalMetadataPaths) { + if (!filesToKeep.contains(fileName)) { + String[] splitPath = fileName.split("/"); + staleGlobalMetadataPaths.add( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( + splitPath[splitPath.length - 1] + ) + ); + } + } + + // visible for testing + void deleteClusterMetadata( + String clusterName, + String clusterUUID, + List activeManifestBlobMetadata, + List staleManifestBlobMetadata + ) { + try { + Set filesToKeep = new HashSet<>(); + Set staleManifestPaths = new HashSet<>(); + Set staleIndexMetadataPaths = new HashSet<>(); + Set staleGlobalMetadataPaths = new HashSet<>(); + activeManifestBlobMetadata.forEach(blobMetadata -> { + ClusterMetadataManifest clusterMetadataManifest = remoteClusterStateService.fetchRemoteClusterMetadataManifest( + clusterName, + clusterUUID, + blobMetadata.name() + ); + clusterMetadataManifest.getIndices() + .forEach(uploadedIndexMetadata -> filesToKeep.add(uploadedIndexMetadata.getUploadedFilename())); + if (clusterMetadataManifest.getCodecVersion() == ClusterMetadataManifest.CODEC_V1) { + filesToKeep.add(clusterMetadataManifest.getGlobalMetadataFileName()); + } else if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V2) { + filesToKeep.add(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); + filesToKeep.add(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); + filesToKeep.add(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); + clusterMetadataManifest.getCustomMetadataMap() + .values() + .forEach(attribute -> filesToKeep.add(attribute.getUploadedFilename())); + } + }); + staleManifestBlobMetadata.forEach(blobMetadata -> { + ClusterMetadataManifest clusterMetadataManifest = remoteClusterStateService.fetchRemoteClusterMetadataManifest( + clusterName, + clusterUUID, + blobMetadata.name() + ); + staleManifestPaths.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blobMetadata.name()); + if (clusterMetadataManifest.getCodecVersion() == ClusterMetadataManifest.CODEC_V1) { + addStaleGlobalMetadataPath(clusterMetadataManifest.getGlobalMetadataFileName(), filesToKeep, staleGlobalMetadataPaths); + } else if (clusterMetadataManifest.getCodecVersion() >= ClusterMetadataManifest.CODEC_V2) { + addStaleGlobalMetadataPath( + clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename(), + filesToKeep, + staleGlobalMetadataPaths + ); + addStaleGlobalMetadataPath( + clusterMetadataManifest.getSettingsMetadata().getUploadedFilename(), + filesToKeep, + staleGlobalMetadataPaths + ); + addStaleGlobalMetadataPath( + clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename(), + filesToKeep, + staleGlobalMetadataPaths + ); + clusterMetadataManifest.getCustomMetadataMap() + .values() + .forEach( + attribute -> addStaleGlobalMetadataPath(attribute.getUploadedFilename(), filesToKeep, staleGlobalMetadataPaths) + ); + } + + clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { + if (filesToKeep.contains(uploadedIndexMetadata.getUploadedFilename()) == false) { + staleIndexMetadataPaths.add( + new BlobPath().add(INDEX_PATH_TOKEN).add(uploadedIndexMetadata.getIndexUUID()).buildAsString() + + INDEX_METADATA_FORMAT.blobName(uploadedIndexMetadata.getUploadedFilename()) + ); + } + }); + }); + + if (staleManifestPaths.isEmpty()) { + logger.debug("No stale Remote Cluster Metadata files found"); + return; + } + + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleGlobalMetadataPaths)); + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleIndexMetadataPaths)); + deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleManifestPaths)); + } catch (IllegalStateException e) { + logger.error("Error while fetching Remote Cluster Metadata manifests", e); + } catch (IOException e) { + logger.error("Error while deleting stale Remote Cluster Metadata files", e); + remoteStateStats.cleanUpAttemptFailed(); + } catch (Exception e) { + logger.error("Unexpected error while deleting stale Remote Cluster Metadata files", e); + remoteStateStats.cleanUpAttemptFailed(); + } + } + + /** + * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests + * + * @param clusterName name of the cluster + * @param clusterUUID uuid of cluster state to refer to in remote + * @param manifestsToRetain no of latest manifest files to keep in remote + */ + // package private for testing + void deleteStaleClusterMetadata(String clusterName, String clusterUUID, int manifestsToRetain) { + if (deleteStaleMetadataRunning.compareAndSet(false, true) == false) { + logger.info("Delete stale cluster metadata task is already in progress."); + return; + } + try { + getBlobStoreTransferService().listAllInSortedOrderAsync( + ThreadPool.Names.REMOTE_PURGE, + remoteClusterStateService.getManifestFolderPath(clusterName, clusterUUID), + MANIFEST_FILE_PREFIX, + Integer.MAX_VALUE, + new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + if (blobMetadata.size() > manifestsToRetain) { + deleteClusterMetadata( + clusterName, + clusterUUID, + blobMetadata.subList(0, manifestsToRetain), + blobMetadata.subList(manifestsToRetain, blobMetadata.size()) + ); + } + deleteStaleMetadataRunning.set(false); + } + + @Override + public void onFailure(Exception e) { + logger.error( + new ParameterizedMessage( + "Exception occurred while deleting Remote Cluster Metadata for clusterUUIDs {}", + clusterUUID + ) + ); + deleteStaleMetadataRunning.set(false); + } + } + ); + } catch (Exception e) { + deleteStaleMetadataRunning.set(false); + throw e; + } + } + + /** + * Purges all remote cluster state against provided cluster UUIDs + * + * @param clusterName name of the cluster + * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged + */ + void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { + clusterUUIDs.forEach( + clusterUUID -> getBlobStoreTransferService().deleteAsync( + ThreadPool.Names.REMOTE_PURGE, + remoteClusterStateService.getCusterMetadataBasePath(clusterName, clusterUUID), + new ActionListener<>() { + @Override + public void onResponse(Void unused) { + logger.info("Deleted all remote cluster metadata for cluster UUID - {}", clusterUUID); + } + + @Override + public void onFailure(Exception e) { + logger.error( + new ParameterizedMessage( + "Exception occurred while deleting all remote cluster metadata for cluster UUID {}", + clusterUUID + ), + e + ); + remoteStateStats.cleanUpAttemptFailed(); + } + } + ) + ); + } + + // package private for testing + void deleteStalePaths(String clusterName, String clusterUUID, List stalePaths) throws IOException { + logger.debug(String.format(Locale.ROOT, "Deleting stale files from remote - %s", stalePaths)); + getBlobStoreTransferService().deleteBlobs( + remoteClusterStateService.getCusterMetadataBasePath(clusterName, clusterUUID), + stalePaths + ); + } + + /** + * Purges all remote cluster state against provided cluster UUIDs + * @param clusterState current state of the cluster + * @param committedManifest last committed ClusterMetadataManifest + */ + public void deleteStaleClusterUUIDs(ClusterState clusterState, ClusterMetadataManifest committedManifest) { + threadpool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { + String clusterName = clusterState.getClusterName().value(); + logger.debug("Deleting stale cluster UUIDs data from remote [{}]", clusterName); + Set allClustersUUIDsInRemote; + try { + allClustersUUIDsInRemote = new HashSet<>( + remoteClusterStateService.getAllClusterUUIDs(clusterState.getClusterName().value()) + ); + } catch (IOException e) { + logger.info(String.format(Locale.ROOT, "Error while fetching all cluster UUIDs for [%s]", clusterName)); + return; + } + // Retain last 2 cluster uuids data + allClustersUUIDsInRemote.remove(committedManifest.getClusterUUID()); + allClustersUUIDsInRemote.remove(committedManifest.getPreviousClusterUUID()); + deleteStaleUUIDsClusterMetadata(clusterName, new ArrayList<>(allClustersUUIDsInRemote)); + }); + } + + public TimeValue getStaleFileCleanupInterval() { + return this.staleFileCleanupInterval; + } + + AsyncStaleFileDeletion getStaleFileDeletionTask() { // for testing + return this.staleFileDeletionTask; + } + + RemotePersistenceStats getStats() { + return this.remoteStateStats; + } + + static final class AsyncStaleFileDeletion extends AbstractAsyncTask { + private final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; + + AsyncStaleFileDeletion(RemoteClusterStateCleanupManager remoteClusterStateCleanupManager) { + super( + logger, + remoteClusterStateCleanupManager.threadpool, + remoteClusterStateCleanupManager.getStaleFileCleanupInterval(), + true + ); + this.remoteClusterStateCleanupManager = remoteClusterStateCleanupManager; + rescheduleIfNecessary(); + } + + @Override + protected boolean mustReschedule() { + return true; + } + + @Override + protected void runInternal() { + remoteClusterStateCleanupManager.cleanUpStaleFiles(); + } + + @Override + protected String getThreadPool() { + return ThreadPool.Names.REMOTE_PURGE; + } + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index 1a67f3cf25bbf..fdf6b2f94ba6b 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -19,11 +19,13 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.routing.remote.RemoteRoutingTableService; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.Nullable; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -60,7 +62,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.LongSupplier; @@ -83,8 +84,6 @@ public class RemoteClusterStateService implements Closeable { public static final String METADATA_MANIFEST_NAME_FORMAT = "%s"; - public static final int RETAINED_MANIFESTS = 10; - public static final String DELIMITER = "__"; public static final String CUSTOM_DELIMITER = "--"; @@ -210,8 +209,7 @@ public class RemoteClusterStateService implements Closeable { private volatile TimeValue indexMetadataUploadTimeout; private volatile TimeValue globalMetadataUploadTimeout; private volatile TimeValue metadataManifestUploadTimeout; - - private final AtomicBoolean deleteStaleMetadataRunning = new AtomicBoolean(false); + private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; private final RemotePersistenceStats remoteStateStats; private final String CLUSTER_STATE_UPLOAD_TIME_LOG_STRING = "writing cluster state for version [{}] took [{}ms]"; private final String METADATA_UPDATE_LOG_STRING = "wrote metadata for [{}] indices and skipped [{}] unchanged " @@ -235,7 +233,7 @@ public RemoteClusterStateService( String nodeId, Supplier repositoriesService, Settings settings, - ClusterSettings clusterSettings, + ClusterService clusterService, LongSupplier relativeTimeNanosSupplier, ThreadPool threadPool, List indexMetadataUploadListeners @@ -246,6 +244,7 @@ public RemoteClusterStateService( this.settings = settings; this.relativeTimeNanosSupplier = relativeTimeNanosSupplier; this.threadpool = threadPool; + ClusterSettings clusterSettings = clusterService.getClusterSettings(); this.slowWriteLoggingThreshold = clusterSettings.get(SLOW_WRITE_LOGGING_THRESHOLD); this.indexMetadataUploadTimeout = clusterSettings.get(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING); this.globalMetadataUploadTimeout = clusterSettings.get(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING); @@ -255,19 +254,13 @@ public RemoteClusterStateService( clusterSettings.addSettingsUpdateConsumer(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, this::setGlobalMetadataUploadTimeout); clusterSettings.addSettingsUpdateConsumer(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, this::setMetadataManifestUploadTimeout); this.remoteStateStats = new RemotePersistenceStats(); + this.remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(this, clusterService); this.indexMetadataUploadListeners = indexMetadataUploadListeners; this.remoteRoutingTableService = isRemoteRoutingTableEnabled(settings) ? Optional.of(new RemoteRoutingTableService(repositoriesService, settings)) : Optional.empty(); } - private BlobStoreTransferService getBlobStoreTransferService() { - if (blobStoreTransferService == null) { - blobStoreTransferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadpool); - } - return blobStoreTransferService; - } - /** * This method uploads entire cluster state metadata to the configured blob store. For now only index metadata upload is supported. This method should be * invoked by the elected cluster manager when the remote cluster state is enabled. @@ -423,7 +416,6 @@ public ClusterMetadataManifest writeIncrementalMetadata( : previousManifest.getCustomMetadataMap(), false ); - deleteStaleClusterMetadata(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), RETAINED_MANIFESTS); final long durationMillis = TimeValue.nsecToMSec(relativeTimeNanosSupplier.getAsLong() - startTimeNanos); remoteStateStats.stateSucceeded(); @@ -727,6 +719,10 @@ private CheckedRunnable getAsyncMetadataWriteAction( ); } + public RemoteClusterStateCleanupManager getCleanupManager() { + return remoteClusterStateCleanupManager; + } + @Nullable public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterState, ClusterMetadataManifest previousManifest) throws IOException { @@ -746,12 +742,16 @@ public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterStat previousManifest.getCustomMetadataMap(), true ); - deleteStaleClusterUUIDs(clusterState, committedManifest); + if (!previousManifest.isClusterUUIDCommitted() && committedManifest.isClusterUUIDCommitted()) { + remoteClusterStateCleanupManager.deleteStaleClusterUUIDs(clusterState, committedManifest); + } + return committedManifest; } @Override public void close() throws IOException { + remoteClusterStateCleanupManager.close(); if (blobStoreRepository != null) { IOUtils.close(blobStoreRepository); } @@ -769,6 +769,7 @@ public void start() { final Repository repository = repositoriesService.get().repository(remoteStoreRepo); assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; blobStoreRepository = (BlobStoreRepository) repository; + remoteClusterStateCleanupManager.start(); this.remoteRoutingTableService.ifPresent(RemoteRoutingTableService::start); } @@ -860,6 +861,14 @@ private void writeMetadataManifest(String clusterName, String clusterUUID, Clust ); } + ThreadPool getThreadpool() { + return threadpool; + } + + BlobStore getBlobStore() { + return blobStoreRepository.blobStore(); + } + private BlobContainer indexMetadataContainer(String clusterName, String clusterUUID, String indexUUID) { // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index/ftqsCnn9TgOX return blobStoreRepository.blobStore() @@ -877,7 +886,7 @@ private BlobContainer manifestContainer(String clusterName, String clusterUUID) return blobStoreRepository.blobStore().blobContainer(getManifestFolderPath(clusterName, clusterUUID)); } - private BlobPath getCusterMetadataBasePath(String clusterName, String clusterUUID) { + BlobPath getCusterMetadataBasePath(String clusterName, String clusterUUID) { return blobStoreRepository.basePath().add(encodeString(clusterName)).add(CLUSTER_STATE_PATH_TOKEN).add(clusterUUID); } @@ -997,7 +1006,7 @@ private static String metadataAttributeFileName(String componentPrefix, Long met ); } - private BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { + BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { return getCusterMetadataBasePath(clusterName, clusterUUID).add(MANIFEST_PATH_TOKEN); } @@ -1250,7 +1259,7 @@ public String getLastKnownUUIDFromRemote(String clusterName) { } } - private Set getAllClusterUUIDs(String clusterName) throws IOException { + Set getAllClusterUUIDs(String clusterName) throws IOException { Map clusterUUIDMetadata = clusterUUIDContainer(clusterName).children(); if (clusterUUIDMetadata == null) { return Collections.emptySet(); @@ -1441,7 +1450,7 @@ private Optional getLatestManifestFileName(String clusterName, String cl * @param clusterName name of the cluster * @return ClusterMetadataManifest */ - private ClusterMetadataManifest fetchRemoteClusterMetadataManifest(String clusterName, String clusterUUID, String filename) + ClusterMetadataManifest fetchRemoteClusterMetadataManifest(String clusterName, String clusterUUID, String filename) throws IllegalStateException { try { return getClusterMetadataManifestBlobStoreFormat(filename).read( @@ -1501,234 +1510,6 @@ public RemoteStateTransferException(String errorDesc, Throwable cause) { } } - /** - * Purges all remote cluster state against provided cluster UUIDs - * - * @param clusterName name of the cluster - * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged - */ - void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { - clusterUUIDs.forEach(clusterUUID -> { - getBlobStoreTransferService().deleteAsync( - ThreadPool.Names.REMOTE_PURGE, - getCusterMetadataBasePath(clusterName, clusterUUID), - new ActionListener<>() { - @Override - public void onResponse(Void unused) { - logger.info("Deleted all remote cluster metadata for cluster UUID - {}", clusterUUID); - } - - @Override - public void onFailure(Exception e) { - logger.error( - new ParameterizedMessage( - "Exception occurred while deleting all remote cluster metadata for cluster UUID {}", - clusterUUID - ), - e - ); - remoteStateStats.cleanUpAttemptFailed(); - } - } - ); - }); - } - - /** - * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests - * - * @param clusterName name of the cluster - * @param clusterUUID uuid of cluster state to refer to in remote - * @param manifestsToRetain no of latest manifest files to keep in remote - */ - // package private for testing - void deleteStaleClusterMetadata(String clusterName, String clusterUUID, int manifestsToRetain) { - if (deleteStaleMetadataRunning.compareAndSet(false, true) == false) { - logger.info("Delete stale cluster metadata task is already in progress."); - return; - } - try { - getBlobStoreTransferService().listAllInSortedOrderAsync( - ThreadPool.Names.REMOTE_PURGE, - getManifestFolderPath(clusterName, clusterUUID), - "manifest", - Integer.MAX_VALUE, - new ActionListener<>() { - @Override - public void onResponse(List blobMetadata) { - if (blobMetadata.size() > manifestsToRetain) { - deleteClusterMetadata( - clusterName, - clusterUUID, - blobMetadata.subList(0, manifestsToRetain - 1), - blobMetadata.subList(manifestsToRetain - 1, blobMetadata.size()) - ); - } - deleteStaleMetadataRunning.set(false); - } - - @Override - public void onFailure(Exception e) { - logger.error( - new ParameterizedMessage( - "Exception occurred while deleting Remote Cluster Metadata for clusterUUIDs {}", - clusterUUID - ) - ); - deleteStaleMetadataRunning.set(false); - } - } - ); - } catch (Exception e) { - deleteStaleMetadataRunning.set(false); - throw e; - } - } - - private void deleteClusterMetadata( - String clusterName, - String clusterUUID, - List activeManifestBlobMetadata, - List staleManifestBlobMetadata - ) { - try { - Set filesToKeep = new HashSet<>(); - Set staleManifestPaths = new HashSet<>(); - Set staleIndexMetadataPaths = new HashSet<>(); - Set staleGlobalMetadataPaths = new HashSet<>(); - activeManifestBlobMetadata.forEach(blobMetadata -> { - ClusterMetadataManifest clusterMetadataManifest = fetchRemoteClusterMetadataManifest( - clusterName, - clusterUUID, - blobMetadata.name() - ); - clusterMetadataManifest.getIndices() - .forEach(uploadedIndexMetadata -> filesToKeep.add(uploadedIndexMetadata.getUploadedFilename())); - if (clusterMetadataManifest.getGlobalMetadataFileName() != null) { - filesToKeep.add(clusterMetadataManifest.getGlobalMetadataFileName()); - } else { - filesToKeep.add(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); - filesToKeep.add(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()); - filesToKeep.add(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()); - clusterMetadataManifest.getCustomMetadataMap() - .forEach((key, value) -> { filesToKeep.add(value.getUploadedFilename()); }); - } - }); - staleManifestBlobMetadata.forEach(blobMetadata -> { - ClusterMetadataManifest clusterMetadataManifest = fetchRemoteClusterMetadataManifest( - clusterName, - clusterUUID, - blobMetadata.name() - ); - staleManifestPaths.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blobMetadata.name()); - if (clusterMetadataManifest.getGlobalMetadataFileName() != null) { - if (filesToKeep.contains(clusterMetadataManifest.getGlobalMetadataFileName()) == false) { - String[] globalMetadataSplitPath = clusterMetadataManifest.getGlobalMetadataFileName().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - globalMetadataSplitPath[globalMetadataSplitPath.length - 1] - ) - ); - } - } else { - if (filesToKeep.contains(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()) == false) { - String[] coordinationMetadataSplitPath = clusterMetadataManifest.getCoordinationMetadata() - .getUploadedFilename() - .split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - coordinationMetadataSplitPath[coordinationMetadataSplitPath.length - 1] - ) - ); - } - if (filesToKeep.contains(clusterMetadataManifest.getTemplatesMetadata().getUploadedFilename()) == false) { - String[] templatesMetadataSplitPath = clusterMetadataManifest.getTemplatesMetadata() - .getUploadedFilename() - .split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - templatesMetadataSplitPath[templatesMetadataSplitPath.length - 1] - ) - ); - } - if (filesToKeep.contains(clusterMetadataManifest.getSettingsMetadata().getUploadedFilename()) == false) { - String[] settingsMetadataSplitPath = clusterMetadataManifest.getSettingsMetadata().getUploadedFilename().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - settingsMetadataSplitPath[settingsMetadataSplitPath.length - 1] - ) - ); - } - clusterMetadataManifest.getCustomMetadataMap().forEach((key, value) -> { - if (filesToKeep.contains(value.getUploadedFilename()) == false) { - String[] customMetadataSplitPath = value.getUploadedFilename().split("/"); - staleGlobalMetadataPaths.add( - new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + GLOBAL_METADATA_FORMAT.blobName( - customMetadataSplitPath[customMetadataSplitPath.length - 1] - ) - ); - } - }); - } - - clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { - if (filesToKeep.contains(uploadedIndexMetadata.getUploadedFilename()) == false) { - staleIndexMetadataPaths.add( - new BlobPath().add(INDEX_PATH_TOKEN).add(uploadedIndexMetadata.getIndexUUID()).buildAsString() - + INDEX_METADATA_FORMAT.blobName(uploadedIndexMetadata.getUploadedFilename()) - ); - } - }); - }); - - if (staleManifestPaths.isEmpty()) { - logger.debug("No stale Remote Cluster Metadata files found"); - return; - } - - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleGlobalMetadataPaths)); - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleIndexMetadataPaths)); - deleteStalePaths(clusterName, clusterUUID, new ArrayList<>(staleManifestPaths)); - } catch (IllegalStateException e) { - logger.error("Error while fetching Remote Cluster Metadata manifests", e); - } catch (IOException e) { - logger.error("Error while deleting stale Remote Cluster Metadata files", e); - remoteStateStats.cleanUpAttemptFailed(); - } catch (Exception e) { - logger.error("Unexpected error while deleting stale Remote Cluster Metadata files", e); - remoteStateStats.cleanUpAttemptFailed(); - } - } - - private void deleteStalePaths(String clusterName, String clusterUUID, List stalePaths) throws IOException { - logger.debug(String.format(Locale.ROOT, "Deleting stale files from remote - %s", stalePaths)); - getBlobStoreTransferService().deleteBlobs(getCusterMetadataBasePath(clusterName, clusterUUID), stalePaths); - } - - /** - * Purges all remote cluster state against provided cluster UUIDs - * - * @param clusterState current state of the cluster - * @param committedManifest last committed ClusterMetadataManifest - */ - public void deleteStaleClusterUUIDs(ClusterState clusterState, ClusterMetadataManifest committedManifest) { - threadpool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { - String clusterName = clusterState.getClusterName().value(); - logger.debug("Deleting stale cluster UUIDs data from remote [{}]", clusterName); - Set allClustersUUIDsInRemote; - try { - allClustersUUIDsInRemote = new HashSet<>(getAllClusterUUIDs(clusterState.getClusterName().value())); - } catch (IOException e) { - logger.info(String.format(Locale.ROOT, "Error while fetching all cluster UUIDs for [%s]", clusterName)); - return; - } - // Retain last 2 cluster uuids data - allClustersUUIDsInRemote.remove(committedManifest.getClusterUUID()); - allClustersUUIDsInRemote.remove(committedManifest.getPreviousClusterUUID()); - deleteStaleUUIDsClusterMetadata(clusterName, new ArrayList<>(allClustersUUIDsInRemote)); - }); - } - public RemotePersistenceStats getStats() { return remoteStateStats; } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 76109ba10624a..49545fa8a0c8b 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -138,6 +138,7 @@ import org.opensearch.gateway.MetaStateService; import org.opensearch.gateway.PersistedClusterStateService; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import org.opensearch.gateway.remote.RemoteClusterStateCleanupManager; import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpServerTransport; import org.opensearch.identity.IdentityService; @@ -752,6 +753,7 @@ protected Node( threadPool::relativeTimeInMillis ); final RemoteClusterStateService remoteClusterStateService; + final RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; final RemoteIndexPathUploader remoteIndexPathUploader; if (isRemoteStoreClusterStateEnabled(settings)) { remoteIndexPathUploader = new RemoteIndexPathUploader( @@ -764,14 +766,16 @@ protected Node( nodeEnvironment.nodeId(), repositoriesServiceReference::get, settings, - clusterService.getClusterSettings(), + clusterService, threadPool::preciseRelativeTimeInNanos, threadPool, List.of(remoteIndexPathUploader) ); + remoteClusterStateCleanupManager = remoteClusterStateService.getCleanupManager(); } else { remoteClusterStateService = null; remoteIndexPathUploader = null; + remoteClusterStateCleanupManager = null; } // collect engine factory providers from plugins @@ -1376,6 +1380,7 @@ protected Node( b.bind(MetricsRegistry.class).toInstance(metricsRegistry); b.bind(RemoteClusterStateService.class).toProvider(() -> remoteClusterStateService); b.bind(RemoteIndexPathUploader.class).toProvider(() -> remoteIndexPathUploader); + b.bind(RemoteClusterStateCleanupManager.class).toProvider(() -> remoteClusterStateCleanupManager); b.bind(PersistedStateRegistry.class).toInstance(persistedStateRegistry); b.bind(SegmentReplicationStatsTracker.class).toInstance(segmentReplicationStatsTracker); b.bind(SearchRequestOperationsCompositeListenerFactory.class).toInstance(searchRequestOperationsCompositeListenerFactory); diff --git a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java index 3ba98c44f8d3e..418e6d8de6adb 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java @@ -462,9 +462,7 @@ public void testDataOnlyNodePersistence() throws Exception { }); when(transportService.getThreadPool()).thenReturn(threadPool); ClusterService clusterService = mock(ClusterService.class); - when(clusterService.getClusterSettings()).thenReturn( - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); + when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); final PersistedClusterStateService persistedClusterStateService = new PersistedClusterStateService( nodeEnvironment, xContentRegistry(), @@ -487,7 +485,7 @@ public void testDataOnlyNodePersistence() throws Exception { nodeEnvironment.nodeId(), repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java new file mode 100644 index 0000000000000..24fd1b164a4ff --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java @@ -0,0 +1,446 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.service.ClusterApplierService; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.support.PlainBlobMetadata; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.AbstractAsyncTask; +import org.opensearch.core.action.ActionListener; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.VersionUtils; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.AsyncStaleFileDeletion; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; +import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; +import static org.opensearch.gateway.remote.RemoteClusterStateService.CLUSTER_STATE_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.COORDINATION_METADATA; +import static org.opensearch.gateway.remote.RemoteClusterStateService.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateService.GLOBAL_METADATA_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; +import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateService.SETTING_METADATA; +import static org.opensearch.gateway.remote.RemoteClusterStateService.TEMPLATES_METADATA; +import static org.opensearch.gateway.remote.RemoteClusterStateService.encodeString; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.nodesWithLocalNodeClusterManager; +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.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class RemoteClusterStateCleanupManagerTests extends OpenSearchTestCase { + private RemoteClusterStateCleanupManager remoteClusterStateCleanupManager; + private Supplier repositoriesServiceSupplier; + private RepositoriesService repositoriesService; + private BlobStoreRepository blobStoreRepository; + private BlobStore blobStore; + private ClusterSettings clusterSettings; + private ClusterApplierService clusterApplierService; + private ClusterState clusterState; + private Metadata metadata; + private RemoteClusterStateService remoteClusterStateService; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + repositoriesServiceSupplier = mock(Supplier.class); + repositoriesService = mock(RepositoriesService.class); + when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); + + String stateRepoTypeAttributeKey = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + "remote_store_repository" + ); + String stateRepoSettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + "remote_store_repository" + ); + + Settings settings = Settings.builder() + .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, "remote_store_repository") + .put(stateRepoTypeAttributeKey, FsRepository.TYPE) + .put(stateRepoSettingsAttributeKeyPrefix + "location", "randomRepoPath") + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .build(); + + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterApplierService = mock(ClusterApplierService.class); + clusterState = mock(ClusterState.class); + metadata = mock(Metadata.class); + ClusterService clusterService = mock(ClusterService.class); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + when(clusterState.getClusterName()).thenReturn(new ClusterName("test")); + when(metadata.clusterUUID()).thenReturn("testUUID"); + when(clusterState.metadata()).thenReturn(metadata); + when(clusterApplierService.state()).thenReturn(clusterState); + when(clusterService.getClusterApplierService()).thenReturn(clusterApplierService); + + blobStoreRepository = mock(BlobStoreRepository.class); + blobStore = mock(BlobStore.class); + when(blobStoreRepository.blobStore()).thenReturn(blobStore); + when(repositoriesService.repository("remote_store_repository")).thenReturn(blobStoreRepository); + + remoteClusterStateService = mock(RemoteClusterStateService.class); + when(remoteClusterStateService.getStats()).thenReturn(new RemotePersistenceStats()); + when(remoteClusterStateService.getThreadpool()).thenReturn(threadPool); + when(remoteClusterStateService.getBlobStore()).thenReturn(blobStore); + remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(remoteClusterStateService, clusterService); + } + + @After + public void teardown() throws Exception { + super.tearDown(); + remoteClusterStateCleanupManager.close(); + threadPool.shutdown(); + } + + public void testDeleteClusterMetadata() throws IOException { + String clusterUUID = "clusterUUID"; + String clusterName = "test-cluster"; + List inactiveBlobs = Arrays.asList( + new PlainBlobMetadata("manifest1.dat", 1L), + new PlainBlobMetadata("manifest2.dat", 1L), + new PlainBlobMetadata("manifest3.dat", 1L) + ); + List activeBlobs = Arrays.asList( + new PlainBlobMetadata("manifest4.dat", 1L), + new PlainBlobMetadata("manifest5.dat", 1L) + ); + UploadedIndexMetadata index1Metadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1"); + UploadedIndexMetadata index2Metadata = new UploadedIndexMetadata("index2", "indexUUID2", "index_metadata2"); + UploadedIndexMetadata index1UpdatedMetadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1_updated"); + UploadedMetadataAttribute coordinationMetadata = new UploadedMetadataAttribute(COORDINATION_METADATA, "coordination_metadata"); + UploadedMetadataAttribute templateMetadata = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata"); + UploadedMetadataAttribute settingMetadata = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata"); + UploadedMetadataAttribute coordinationMetadataUpdated = new UploadedMetadataAttribute( + COORDINATION_METADATA, + "coordination_metadata_updated" + ); + UploadedMetadataAttribute templateMetadataUpdated = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata_updated"); + UploadedMetadataAttribute settingMetadataUpdated = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata_updated"); + ClusterMetadataManifest manifest1 = ClusterMetadataManifest.builder() + .indices(List.of(index1Metadata)) + .globalMetadataFileName("global_metadata") + .clusterTerm(1L) + .stateVersion(1L) + .codecVersion(CODEC_V1) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID(clusterUUID) + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .build(); + ClusterMetadataManifest manifest2 = ClusterMetadataManifest.builder(manifest1) + .indices(List.of(index1Metadata, index2Metadata)) + .codecVersion(CODEC_V2) + .globalMetadataFileName(null) + .coordinationMetadata(coordinationMetadata) + .templatesMetadata(templateMetadata) + .settingMetadata(settingMetadata) + .build(); + ClusterMetadataManifest manifest3 = ClusterMetadataManifest.builder(manifest2) + .indices(List.of(index1UpdatedMetadata, index2Metadata)) + .settingMetadata(settingMetadataUpdated) + .build(); + + // active manifest have reference to index1Updated, index2, settingsUpdated, coordinationUpdated, templates, templatesUpdated + ClusterMetadataManifest manifest4 = ClusterMetadataManifest.builder(manifest3) + .coordinationMetadata(coordinationMetadataUpdated) + .build(); + ClusterMetadataManifest manifest5 = ClusterMetadataManifest.builder(manifest4).templatesMetadata(templateMetadataUpdated).build(); + + when(remoteClusterStateService.fetchRemoteClusterMetadataManifest(eq(clusterName), eq(clusterUUID), any())).thenReturn( + manifest4, + manifest5, + manifest1, + manifest2, + manifest3 + ); + BlobContainer container = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).thenReturn(container); + doNothing().when(container).deleteBlobsIgnoringIfNotExists(any()); + + remoteClusterStateCleanupManager.deleteClusterMetadata(clusterName, clusterUUID, activeBlobs, inactiveBlobs); + verify(container).deleteBlobsIgnoringIfNotExists( + List.of( + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + coordinationMetadata.getUploadedFilename() + ".dat", + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + settingMetadata.getUploadedFilename() + ".dat", + new BlobPath().add(GLOBAL_METADATA_PATH_TOKEN).buildAsString() + "global_metadata.dat" + ) + ); + verify(container).deleteBlobsIgnoringIfNotExists( + List.of( + new BlobPath().add(INDEX_PATH_TOKEN).add(index1Metadata.getIndexUUID()).buildAsString() + + index1Metadata.getUploadedFilePath() + + ".dat" + ) + ); + Set staleManifest = new HashSet<>(); + inactiveBlobs.forEach(blob -> staleManifest.add(new BlobPath().add(MANIFEST_PATH_TOKEN).buildAsString() + blob.name())); + verify(container).deleteBlobsIgnoringIfNotExists(new ArrayList<>(staleManifest)); + } + + public void testDeleteStaleClusterUUIDs() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + ClusterMetadataManifest clusterMetadataManifest = ClusterMetadataManifest.builder() + .indices(List.of()) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID("cluster-uuid1") + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .build(); + + BlobPath blobPath = new BlobPath().add("random-path"); + BlobContainer uuidContainerContainer = mock(BlobContainer.class); + BlobContainer manifest2Container = mock(BlobContainer.class); + BlobContainer manifest3Container = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).then(invocation -> { + BlobPath blobPath1 = invocation.getArgument(0); + if (blobPath1.buildAsString().endsWith("cluster-state/")) { + return uuidContainerContainer; + } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid2/")) { + return manifest2Container; + } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid3/")) { + return manifest3Container; + } else { + throw new IllegalArgumentException("Unexpected blob path " + blobPath1); + } + }); + when( + manifest2Container.listBlobsByPrefixInSortedOrder( + MANIFEST_FILE_PREFIX + DELIMITER, + Integer.MAX_VALUE, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ) + ).thenReturn(List.of(new PlainBlobMetadata("mainfest2", 1L))); + when( + manifest3Container.listBlobsByPrefixInSortedOrder( + MANIFEST_FILE_PREFIX + DELIMITER, + Integer.MAX_VALUE, + BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC + ) + ).thenReturn(List.of(new PlainBlobMetadata("mainfest3", 1L))); + Set uuids = new HashSet<>(Arrays.asList("cluster-uuid1", "cluster-uuid2", "cluster-uuid3")); + when(remoteClusterStateService.getAllClusterUUIDs(any())).thenReturn(uuids); + when(remoteClusterStateService.getCusterMetadataBasePath(any(), any())).then( + invocationOnMock -> blobPath.add(encodeString(invocationOnMock.getArgument(0))) + .add(CLUSTER_STATE_PATH_TOKEN) + .add((String) invocationOnMock.getArgument(1)) + ); + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleClusterUUIDs(clusterState, clusterMetadataManifest); + try { + assertBusy(() -> { + verify(manifest2Container, times(1)).delete(); + verify(manifest3Container, times(1)).delete(); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void testRemoteStateCleanupFailureStats() throws IOException { + BlobContainer blobContainer = mock(BlobContainer.class); + doThrow(IOException.class).when(blobContainer).delete(); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + BlobPath blobPath = new BlobPath().add("random-path"); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleUUIDsClusterMetadata("cluster1", List.of("cluster-uuid1")); + try { + assertBusy(() -> { + // wait for stats to get updated + assertNotNull(remoteClusterStateCleanupManager.getStats()); + assertEquals(0, remoteClusterStateCleanupManager.getStats().getSuccessCount()); + assertEquals(1, remoteClusterStateCleanupManager.getStats().getCleanupAttemptFailedCount()); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { + BlobContainer blobContainer = mock(BlobContainer.class); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> { + callCount.incrementAndGet(); + if (latch.await(5000, TimeUnit.SECONDS) == false) { + throw new Exception("Timed out waiting for delete task queuing to complete"); + } + return null; + }).when(blobContainer) + .listBlobsByPrefixInSortedOrder( + any(String.class), + any(int.class), + any(BlobContainer.BlobNameSortOrder.class), + any(ActionListener.class) + ); + + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); + remoteClusterStateCleanupManager.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); + + latch.countDown(); + assertBusy(() -> assertEquals(1, callCount.get())); + } + + public void testRemoteClusterStateCleanupSetting() { + remoteClusterStateCleanupManager.start(); + // verify default value + assertEquals(CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT, remoteClusterStateCleanupManager.getStaleFileCleanupInterval()); + + // verify update interval + int cleanupInterval = randomIntBetween(1, 10); + Settings newSettings = Settings.builder().put("cluster.remote_store.state.cleanup_interval", cleanupInterval + "s").build(); + clusterSettings.applySettings(newSettings); + assertEquals(cleanupInterval, remoteClusterStateCleanupManager.getStaleFileCleanupInterval().seconds()); + } + + public void testRemoteCleanupTaskScheduled() { + AbstractAsyncTask cleanupTask = remoteClusterStateCleanupManager.getStaleFileDeletionTask(); + assertNull(cleanupTask); + // now the task should be initialized + remoteClusterStateCleanupManager.start(); + assertNotNull(remoteClusterStateCleanupManager.getStaleFileDeletionTask()); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().mustReschedule()); + assertEquals( + clusterSettings.get(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING), + remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval() + ); + assertTrue(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isScheduled()); + assertFalse(remoteClusterStateCleanupManager.getStaleFileDeletionTask().isClosed()); + } + + public void testRemoteCleanupSkipsOnOnlyElectedClusterManager() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(false); + when(clusterState.nodes()).thenReturn(nodes); + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + spyManager.cleanUpStaleFiles(); + assertEquals(0, callCount.get()); + + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.version()).thenReturn(randomLongBetween(11, 20)); + spyManager.cleanUpStaleFiles(); + assertEquals(1, callCount.get()); + } + + public void testRemoteCleanupSkipsIfVersionIncrementLessThanThreshold() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + long version = randomLongBetween(1, SKIP_CLEANUP_STATE_CHANGES); + when(clusterApplierService.state()).thenReturn(clusterState); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.nodes()).thenReturn(nodes); + when(clusterState.version()).thenReturn(version); + + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + + remoteClusterStateCleanupManager.cleanUpStaleFiles(); + assertEquals(0, callCount.get()); + } + + public void testRemoteCleanupCallsDeleteIfVersionIncrementGreaterThanThreshold() { + DiscoveryNodes nodes = mock(DiscoveryNodes.class); + long version = randomLongBetween(SKIP_CLEANUP_STATE_CHANGES + 1, SKIP_CLEANUP_STATE_CHANGES + 10); + when(clusterApplierService.state()).thenReturn(clusterState); + when(nodes.isLocalNodeElectedClusterManager()).thenReturn(true); + when(clusterState.nodes()).thenReturn(nodes); + when(clusterState.version()).thenReturn(version); + + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocation -> callCount.incrementAndGet()).when(spyManager).deleteStaleClusterMetadata(any(), any(), anyInt()); + + // using spied cleanup manager so that stubbed deleteStaleClusterMetadata is called + spyManager.cleanUpStaleFiles(); + assertEquals(1, callCount.get()); + } + + public void testRemoteCleanupSchedulesEvenAfterFailure() { + remoteClusterStateCleanupManager.start(); + RemoteClusterStateCleanupManager spyManager = spy(remoteClusterStateCleanupManager); + AtomicInteger callCount = new AtomicInteger(0); + doAnswer(invocationOnMock -> { + callCount.incrementAndGet(); + throw new RuntimeException("Test exception"); + }).when(spyManager).cleanUpStaleFiles(); + AsyncStaleFileDeletion task = new AsyncStaleFileDeletion(spyManager); + assertTrue(task.isScheduled()); + task.run(); + // Task is still scheduled after the failure + assertTrue(task.isScheduled()); + assertEquals(1, callCount.get()); + + task.run(); + // Task is still scheduled after the failure + assertTrue(task.isScheduled()); + assertEquals(2, callCount.get()); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index aa5996d734d27..324023061ac46 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -19,6 +19,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; @@ -73,9 +74,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.Supplier; @@ -95,7 +93,6 @@ import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_CURRENT_CODEC_VERSION; import static org.opensearch.gateway.remote.RemoteClusterStateService.MANIFEST_FILE_PREFIX; import static org.opensearch.gateway.remote.RemoteClusterStateService.METADATA_FILE_PREFIX; -import static org.opensearch.gateway.remote.RemoteClusterStateService.RETAINED_MANIFESTS; import static org.opensearch.gateway.remote.RemoteClusterStateService.SETTING_METADATA; import static org.opensearch.gateway.remote.RemoteClusterStateService.TEMPLATES_METADATA; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; @@ -112,13 +109,12 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class RemoteClusterStateServiceTests extends OpenSearchTestCase { private RemoteClusterStateService remoteClusterStateService; + private ClusterService clusterService; private ClusterSettings clusterSettings; private Supplier repositoriesServiceSupplier; private RepositoriesService repositoriesService; @@ -151,6 +147,8 @@ public void setup() { .build(); clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterService = mock(ClusterService.class); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( Stream.of( NetworkModule.getNamedXContents().stream(), @@ -168,7 +166,7 @@ public void setup() { "test-node-id", repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) @@ -190,14 +188,14 @@ public void testFailWriteFullMetadataNonClusterManagerNode() throws IOException public void testFailInitializationWhenRemoteStateDisabled() { final Settings settings = Settings.builder().build(); - ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); assertThrows( AssertionError.class, () -> new RemoteClusterStateService( "test-node-id", repositoriesServiceSupplier, settings, - clusterSettings, + clusterService, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) @@ -1283,72 +1281,6 @@ public void testGetValidPreviousClusterUUIDWhenLastUUIDUncommitted() throws IOEx assertThat(previousClusterUUID, equalTo("cluster-uuid2")); } - public void testDeleteStaleClusterUUIDs() throws IOException { - final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); - ClusterMetadataManifest clusterMetadataManifest = ClusterMetadataManifest.builder() - .indices(List.of()) - .clusterTerm(1L) - .stateVersion(1L) - .stateUUID(randomAlphaOfLength(10)) - .clusterUUID("cluster-uuid1") - .nodeId("nodeA") - .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) - .previousClusterUUID(ClusterState.UNKNOWN_UUID) - .committed(true) - .build(); - - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - BlobContainer uuidContainerContainer = mock(BlobContainer.class); - BlobContainer manifest2Container = mock(BlobContainer.class); - BlobContainer manifest3Container = mock(BlobContainer.class); - when(blobStore.blobContainer(any())).then(invocation -> { - BlobPath blobPath1 = invocation.getArgument(0); - if (blobPath1.buildAsString().endsWith("cluster-state/")) { - return uuidContainerContainer; - } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid2/")) { - return manifest2Container; - } else if (blobPath1.buildAsString().contains("cluster-state/cluster-uuid3/")) { - return manifest3Container; - } else { - throw new IllegalArgumentException("Unexpected blob path " + blobPath1); - } - }); - Map blobMetadataMap = Map.of( - "cluster-uuid1", - mock(BlobContainer.class), - "cluster-uuid2", - mock(BlobContainer.class), - "cluster-uuid3", - mock(BlobContainer.class) - ); - when(uuidContainerContainer.children()).thenReturn(blobMetadataMap); - when( - manifest2Container.listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - Integer.MAX_VALUE, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(List.of(new PlainBlobMetadata("mainfest2", 1L))); - when( - manifest3Container.listBlobsByPrefixInSortedOrder( - MANIFEST_FILE_PREFIX + DELIMITER, - Integer.MAX_VALUE, - BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC - ) - ).thenReturn(List.of(new PlainBlobMetadata("mainfest3", 1L))); - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleClusterUUIDs(clusterState, clusterMetadataManifest); - try { - assertBusy(() -> { - verify(manifest2Container, times(1)).delete(); - verify(manifest3Container, times(1)).delete(); - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - public void testRemoteStateStats() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); mockBlobStoreObjects(); @@ -1361,26 +1293,6 @@ public void testRemoteStateStats() throws IOException { assertEquals(0, remoteClusterStateService.getStats().getFailedCount()); } - public void testRemoteStateCleanupFailureStats() throws IOException { - BlobContainer blobContainer = mock(BlobContainer.class); - doThrow(IOException.class).when(blobContainer).delete(); - when(blobStore.blobContainer(any())).thenReturn(blobContainer); - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleUUIDsClusterMetadata("cluster1", Arrays.asList("cluster-uuid1")); - try { - assertBusy(() -> { - // wait for stats to get updated - assertTrue(remoteClusterStateService.getStats() != null); - assertEquals(0, remoteClusterStateService.getStats().getSuccessCount()); - assertEquals(1, remoteClusterStateService.getStats().getCleanupAttemptFailedCount()); - }); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - public void testFileNames() { final Index index = new Index("test-index", "index-uuid"); final Settings idxSettings = Settings.builder() @@ -1421,36 +1333,6 @@ private void verifyManifestFileNameWithCodec(int codecVersion) { assertThat(splittedName[3], is("P")); } - public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { - BlobContainer blobContainer = mock(BlobContainer.class); - BlobPath blobPath = new BlobPath().add("random-path"); - when((blobStoreRepository.basePath())).thenReturn(blobPath); - when(blobStore.blobContainer(any())).thenReturn(blobContainer); - - CountDownLatch latch = new CountDownLatch(1); - AtomicInteger callCount = new AtomicInteger(0); - doAnswer(invocation -> { - callCount.incrementAndGet(); - if (latch.await(5000, TimeUnit.SECONDS) == false) { - throw new Exception("Timed out waiting for delete task queuing to complete"); - } - return null; - }).when(blobContainer) - .listBlobsByPrefixInSortedOrder( - any(String.class), - any(int.class), - any(BlobContainer.BlobNameSortOrder.class), - any(ActionListener.class) - ); - - remoteClusterStateService.start(); - remoteClusterStateService.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); - remoteClusterStateService.deleteStaleClusterMetadata("cluster-name", "cluster-uuid", RETAINED_MANIFESTS); - - latch.countDown(); - assertBusy(() -> assertEquals(1, callCount.get())); - } - public void testIndexMetadataUploadWaitTimeSetting() { // verify default value assertEquals( @@ -1518,7 +1400,7 @@ public void testRemoteRoutingTableInitializedWhenEnabled() { "test-node-id", repositoriesServiceSupplier, newSettings, - clusterSettings, + clusterService, () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, newSettings, repositoriesServiceSupplier, clusterSettings)) @@ -1921,7 +1803,7 @@ private static ClusterState.Builder generateClusterStateWithGlobalMetadata() { ); } - private static ClusterState.Builder generateClusterStateWithOneIndex() { + static ClusterState.Builder generateClusterStateWithOneIndex() { final Index index = new Index("test-index", "index-uuid"); final Settings idxSettings = Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) @@ -1951,7 +1833,7 @@ private static ClusterState.Builder generateClusterStateWithOneIndex() { ); } - private static DiscoveryNodes nodesWithLocalNodeClusterManager() { + static DiscoveryNodes nodesWithLocalNodeClusterManager() { return DiscoveryNodes.builder().clusterManagerNodeId("cluster-manager-id").localNodeId("cluster-manager-id").build(); } From 0b2e012c266f4d954e3c3875276880c15efd5e2c Mon Sep 17 00:00:00 2001 From: gargharsh3134 <51459091+gargharsh3134@users.noreply.github.com> Date: Thu, 6 Jun 2024 12:33:42 +0530 Subject: [PATCH 53/77] Adds counter metrics for leader and follower check failures (#12439) * Adds counter metrics for leader and follower check failures Signed-off-by: Harsh Garg --- CHANGELOG.md | 1 + .../cluster/ClusterManagerMetrics.java | 27 +++++++ .../cluster/coordination/Coordinator.java | 16 ++++- .../coordination/FollowersChecker.java | 7 +- .../cluster/coordination/LeaderChecker.java | 11 +-- .../opensearch/discovery/DiscoveryModule.java | 7 +- .../main/java/org/opensearch/node/Node.java | 3 +- .../coordination/FollowersCheckerTests.java | 65 +++++++++++++---- .../coordination/LeaderCheckerTests.java | 25 +++++-- .../cluster/coordination/NodeJoinTests.java | 3 +- .../discovery/DiscoveryModuleTests.java | 5 +- .../snapshots/SnapshotResiliencyTests.java | 3 +- .../telemetry/TestInMemoryCounter.java | 52 ++++++++++++++ .../telemetry/TestInMemoryHistogram.java | 47 ++++++++++++ .../TestInMemoryMetricsRegistry.java | 71 +++++++++++++++++++ .../AbstractCoordinatorTestCase.java | 3 +- 16 files changed, 313 insertions(+), 33 deletions(-) create mode 100644 server/src/test/java/org/opensearch/telemetry/TestInMemoryCounter.java create mode 100644 server/src/test/java/org/opensearch/telemetry/TestInMemoryHistogram.java create mode 100644 server/src/test/java/org/opensearch/telemetry/TestInMemoryMetricsRegistry.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 1ffb438172f50..759cf53adf8b8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,6 +5,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ## [Unreleased 2.x] ### Added +- Add leader and follower check failure counter metrics ([#12439](https://github.com/opensearch-project/OpenSearch/pull/12439)) - Add latency metrics for instrumenting critical clusterManager code paths ([#12333](https://github.com/opensearch-project/OpenSearch/pull/12333)) - Add support for Azure Managed Identity in repository-azure ([#12423](https://github.com/opensearch-project/OpenSearch/issues/12423)) - Add useCompoundFile index setting ([#13478](https://github.com/opensearch-project/OpenSearch/pull/13478)) diff --git a/server/src/main/java/org/opensearch/cluster/ClusterManagerMetrics.java b/server/src/main/java/org/opensearch/cluster/ClusterManagerMetrics.java index d48f82a388245..a98349a4af5cd 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterManagerMetrics.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterManagerMetrics.java @@ -8,6 +8,7 @@ package org.opensearch.cluster; +import org.opensearch.telemetry.metrics.Counter; import org.opensearch.telemetry.metrics.Histogram; import org.opensearch.telemetry.metrics.MetricsRegistry; import org.opensearch.telemetry.metrics.tags.Tags; @@ -23,6 +24,7 @@ public final class ClusterManagerMetrics { private static final String LATENCY_METRIC_UNIT_MS = "ms"; + private static final String COUNTER_METRICS_UNIT = "1"; public final Histogram clusterStateAppliersHistogram; public final Histogram clusterStateListenersHistogram; @@ -30,6 +32,9 @@ public final class ClusterManagerMetrics { public final Histogram clusterStateComputeHistogram; public final Histogram clusterStatePublishHistogram; + public final Counter leaderCheckFailureCounter; + public final Counter followerChecksFailureCounter; + public ClusterManagerMetrics(MetricsRegistry metricsRegistry) { clusterStateAppliersHistogram = metricsRegistry.createHistogram( "cluster.state.appliers.latency", @@ -56,6 +61,16 @@ public ClusterManagerMetrics(MetricsRegistry metricsRegistry) { "Histogram for recording time taken to publish a new cluster state", LATENCY_METRIC_UNIT_MS ); + followerChecksFailureCounter = metricsRegistry.createCounter( + "followers.checker.failure.count", + "Counter for number of failed follower checks", + COUNTER_METRICS_UNIT + ); + leaderCheckFailureCounter = metricsRegistry.createCounter( + "leader.checker.failure.count", + "Counter for number of failed leader checks", + COUNTER_METRICS_UNIT + ); } public void recordLatency(Histogram histogram, Double value) { @@ -69,4 +84,16 @@ public void recordLatency(Histogram histogram, Double value, Optional tags } histogram.record(value, tags.get()); } + + public void incrementCounter(Counter counter, Double value) { + incrementCounter(counter, value, Optional.empty()); + } + + public void incrementCounter(Counter counter, Double value, Optional tags) { + if (Objects.isNull(tags) || tags.isEmpty()) { + counter.add(value); + return; + } + counter.add(value, tags.get()); + } } 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 3d74feddfa261..f53e6837a67f5 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.cluster.ClusterChangedEvent; +import org.opensearch.cluster.ClusterManagerMetrics; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.ClusterStateTaskConfig; @@ -207,7 +208,8 @@ public Coordinator( ElectionStrategy electionStrategy, NodeHealthService nodeHealthService, PersistedStateRegistry persistedStateRegistry, - RemoteStoreNodeService remoteStoreNodeService + RemoteStoreNodeService remoteStoreNodeService, + ClusterManagerMetrics clusterManagerMetrics ) { this.settings = settings; this.transportService = transportService; @@ -261,14 +263,22 @@ public Coordinator( this::handlePublishRequest, this::handleApplyCommit ); - this.leaderChecker = new LeaderChecker(settings, clusterSettings, transportService, this::onLeaderFailure, nodeHealthService); + this.leaderChecker = new LeaderChecker( + settings, + clusterSettings, + transportService, + this::onLeaderFailure, + nodeHealthService, + clusterManagerMetrics + ); this.followersChecker = new FollowersChecker( settings, clusterSettings, transportService, this::onFollowerCheckRequest, this::removeNode, - nodeHealthService + nodeHealthService, + clusterManagerMetrics ); this.nodeRemovalExecutor = new NodeRemovalClusterStateTaskExecutor(allocationService, logger); this.clusterApplier = clusterApplier; diff --git a/server/src/main/java/org/opensearch/cluster/coordination/FollowersChecker.java b/server/src/main/java/org/opensearch/cluster/coordination/FollowersChecker.java index 70bb0515bb022..2ec0dabd91786 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/FollowersChecker.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/FollowersChecker.java @@ -35,6 +35,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.cluster.ClusterManagerMetrics; import org.opensearch.cluster.coordination.Coordinator.Mode; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; @@ -127,6 +128,7 @@ public class FollowersChecker { private final TransportService transportService; private final NodeHealthService nodeHealthService; private volatile FastResponseState fastResponseState; + private ClusterManagerMetrics clusterManagerMetrics; public FollowersChecker( Settings settings, @@ -134,7 +136,8 @@ public FollowersChecker( TransportService transportService, Consumer handleRequestAndUpdateState, BiConsumer onNodeFailure, - NodeHealthService nodeHealthService + NodeHealthService nodeHealthService, + ClusterManagerMetrics clusterManagerMetrics ) { this.settings = settings; this.transportService = transportService; @@ -161,6 +164,7 @@ public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connecti handleDisconnectedNode(node); } }); + this.clusterManagerMetrics = clusterManagerMetrics; } private void setFollowerCheckTimeout(TimeValue followerCheckTimeout) { @@ -413,6 +417,7 @@ public String executor() { } void failNode(String reason) { + clusterManagerMetrics.incrementCounter(clusterManagerMetrics.followerChecksFailureCounter, 1.0); transportService.getThreadPool().generic().execute(new Runnable() { @Override public void run() { diff --git a/server/src/main/java/org/opensearch/cluster/coordination/LeaderChecker.java b/server/src/main/java/org/opensearch/cluster/coordination/LeaderChecker.java index 8d4373b865f62..4fd2c0eb13073 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/LeaderChecker.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/LeaderChecker.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchException; +import org.opensearch.cluster.ClusterManagerMetrics; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.Nullable; @@ -119,17 +120,17 @@ public class LeaderChecker { private final TransportService transportService; private final Consumer onLeaderFailure; private final NodeHealthService nodeHealthService; - private AtomicReference currentChecker = new AtomicReference<>(); - private volatile DiscoveryNodes discoveryNodes; + private final ClusterManagerMetrics clusterManagerMetrics; LeaderChecker( final Settings settings, final ClusterSettings clusterSettings, final TransportService transportService, final Consumer onLeaderFailure, - NodeHealthService nodeHealthService + NodeHealthService nodeHealthService, + final ClusterManagerMetrics clusterManagerMetrics ) { this.settings = settings; leaderCheckInterval = LEADER_CHECK_INTERVAL_SETTING.get(settings); @@ -138,6 +139,7 @@ public class LeaderChecker { this.transportService = transportService; this.onLeaderFailure = onLeaderFailure; this.nodeHealthService = nodeHealthService; + this.clusterManagerMetrics = clusterManagerMetrics; clusterSettings.addSettingsUpdateConsumer(LEADER_CHECK_TIMEOUT_SETTING, this::setLeaderCheckTimeout); transportService.registerRequestHandler( @@ -293,7 +295,6 @@ public void handleResponse(Empty response) { logger.debug("closed check scheduler received a response, doing nothing"); return; } - failureCountSinceLastSuccess.set(0); scheduleNextWakeUp(); // logs trace message indicating success } @@ -304,7 +305,6 @@ public void handleException(TransportException exp) { logger.debug("closed check scheduler received a response, doing nothing"); return; } - if (exp instanceof ConnectTransportException || exp.getCause() instanceof ConnectTransportException) { logger.debug(new ParameterizedMessage("leader [{}] disconnected during check", leader), exp); leaderFailed(new ConnectTransportException(leader, "disconnected during check", exp)); @@ -355,6 +355,7 @@ public String executor() { void leaderFailed(Exception e) { if (isClosed.compareAndSet(false, true)) { + clusterManagerMetrics.incrementCounter(clusterManagerMetrics.leaderCheckFailureCounter, 1.0); transportService.getThreadPool().generic().execute(new Runnable() { @Override public void run() { diff --git a/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java b/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java index 288371aa240a0..538dea5b2e60b 100644 --- a/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/opensearch/discovery/DiscoveryModule.java @@ -34,6 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.ClusterManagerMetrics; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.coordination.Coordinator; import org.opensearch.cluster.coordination.ElectionStrategy; @@ -133,7 +134,8 @@ public DiscoveryModule( RerouteService rerouteService, NodeHealthService nodeHealthService, PersistedStateRegistry persistedStateRegistry, - RemoteStoreNodeService remoteStoreNodeService + RemoteStoreNodeService remoteStoreNodeService, + ClusterManagerMetrics clusterManagerMetrics ) { final Collection> joinValidators = new ArrayList<>(); final Map> hostProviders = new HashMap<>(); @@ -211,7 +213,8 @@ public DiscoveryModule( electionStrategy, nodeHealthService, persistedStateRegistry, - remoteStoreNodeService + remoteStoreNodeService, + clusterManagerMetrics ); } else { throw new IllegalArgumentException("Unknown discovery type [" + discoveryType + "]"); diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 49545fa8a0c8b..cb1f2caa082fc 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -1199,7 +1199,8 @@ protected Node( rerouteService, fsHealthService, persistedStateRegistry, - remoteStoreNodeService + remoteStoreNodeService, + clusterManagerMetrics ); final SearchPipelineService searchPipelineService = new SearchPipelineService( clusterService, diff --git a/server/src/test/java/org/opensearch/cluster/coordination/FollowersCheckerTests.java b/server/src/test/java/org/opensearch/cluster/coordination/FollowersCheckerTests.java index a106706c00732..d0bc41b459cc3 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/FollowersCheckerTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/FollowersCheckerTests.java @@ -33,6 +33,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.Version; +import org.opensearch.cluster.ClusterManagerMetrics; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.coordination.Coordinator.Mode; import org.opensearch.cluster.coordination.FollowersChecker.FollowerCheckRequest; @@ -47,6 +48,9 @@ import org.opensearch.core.transport.TransportResponse.Empty; import org.opensearch.monitor.NodeHealthService; import org.opensearch.monitor.StatusInfo; +import org.opensearch.telemetry.TestInMemoryMetricsRegistry; +import org.opensearch.telemetry.metrics.MetricsRegistry; +import org.opensearch.telemetry.metrics.noop.NoopMetricsRegistry; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.EqualsHashCodeTestUtils; import org.opensearch.test.EqualsHashCodeTestUtils.CopyFunction; @@ -131,6 +135,8 @@ protected void onSendRequest(long requestId, String action, TransportRequest req transportService.start(); transportService.acceptIncomingRequests(); + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); + final FollowersChecker followersChecker = new FollowersChecker( settings, clusterSettings, @@ -139,7 +145,8 @@ protected void onSendRequest(long requestId, String action, TransportRequest req (node, reason) -> { assert false : node; }, - () -> new StatusInfo(StatusInfo.Status.HEALTHY, "healthy-info") + () -> new StatusInfo(StatusInfo.Status.HEALTHY, "healthy-info"), + new ClusterManagerMetrics(metricsRegistry) ); followersChecker.setCurrentNodes(discoveryNodesHolder[0]); @@ -193,35 +200,43 @@ protected void onSendRequest(long requestId, String action, TransportRequest req followersChecker.clearCurrentNodes(); deterministicTaskQueue.runAllTasks(); assertThat(checkedNodes, empty()); + assertEquals(Integer.valueOf(0), metricsRegistry.getCounterStore().get("followers.checker.failure.count").getCounterValue()); } public void testFailsNodeThatDoesNotRespond() { final Settings settings = randomSettings(); + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); testBehaviourOfFailingNode( settings, () -> null, "followers check retry count exceeded", (FOLLOWER_CHECK_RETRY_COUNT_SETTING.get(settings) - 1) * FOLLOWER_CHECK_INTERVAL_SETTING.get(settings).millis() + FOLLOWER_CHECK_RETRY_COUNT_SETTING.get(settings) * FOLLOWER_CHECK_TIMEOUT_SETTING.get(settings).millis(), - () -> new StatusInfo(HEALTHY, "healthy-info") + () -> new StatusInfo(HEALTHY, "healthy-info"), + metricsRegistry ); + assertEquals(Integer.valueOf(2), metricsRegistry.getCounterStore().get("followers.checker.failure.count").getCounterValue()); } public void testFailsNodeThatRejectsCheck() { final Settings settings = randomSettings(); + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); testBehaviourOfFailingNode( settings, () -> { throw new OpenSearchException("simulated exception"); }, "followers check retry count exceeded", (FOLLOWER_CHECK_RETRY_COUNT_SETTING.get(settings) - 1) * FOLLOWER_CHECK_INTERVAL_SETTING.get(settings).millis(), - () -> new StatusInfo(HEALTHY, "healthy-info") + () -> new StatusInfo(HEALTHY, "healthy-info"), + metricsRegistry ); + assertEquals(Integer.valueOf(2), metricsRegistry.getCounterStore().get("followers.checker.failure.count").getCounterValue()); } public void testFailureCounterResetsOnSuccess() { final Settings settings = randomSettings(); final int retryCount = FOLLOWER_CHECK_RETRY_COUNT_SETTING.get(settings); final int maxRecoveries = randomIntBetween(3, 10); + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); // passes just enough checks to keep it alive, up to maxRecoveries, and then fails completely testBehaviourOfFailingNode(settings, new Supplier() { @@ -241,18 +256,23 @@ public Empty get() { "followers check retry count exceeded", (FOLLOWER_CHECK_RETRY_COUNT_SETTING.get(settings) * (maxRecoveries + 1) - 1) * FOLLOWER_CHECK_INTERVAL_SETTING.get(settings) .millis(), - () -> new StatusInfo(HEALTHY, "healthy-info") + () -> new StatusInfo(HEALTHY, "healthy-info"), + metricsRegistry ); + assertEquals(Integer.valueOf(2), metricsRegistry.getCounterStore().get("followers.checker.failure.count").getCounterValue()); } public void testFailsNodeThatIsDisconnected() { + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); testBehaviourOfFailingNode( Settings.EMPTY, () -> { throw new ConnectTransportException(null, "simulated exception"); }, "disconnected", 0, - () -> new StatusInfo(HEALTHY, "healthy-info") + () -> new StatusInfo(HEALTHY, "healthy-info"), + metricsRegistry ); + assertEquals(Integer.valueOf(2), metricsRegistry.getCounterStore().get("followers.checker.failure.count").getCounterValue()); } public void testFailsNodeThatDisconnects() { @@ -297,6 +317,7 @@ public String toString() { transportService.acceptIncomingRequests(); final AtomicBoolean nodeFailed = new AtomicBoolean(); + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); final FollowersChecker followersChecker = new FollowersChecker( settings, @@ -307,7 +328,8 @@ public String toString() { assertTrue(nodeFailed.compareAndSet(false, true)); assertThat(reason, equalTo("disconnected")); }, - () -> new StatusInfo(HEALTHY, "healthy-info") + () -> new StatusInfo(HEALTHY, "healthy-info"), + new ClusterManagerMetrics(metricsRegistry) ); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(localNode).add(otherNode).localNodeId(localNode.getId()).build(); @@ -318,16 +340,20 @@ public String toString() { deterministicTaskQueue.runAllRunnableTasks(); assertTrue(nodeFailed.get()); assertThat(followersChecker.getFaultyNodes(), contains(otherNode)); + assertEquals(Integer.valueOf(1), metricsRegistry.getCounterStore().get("followers.checker.failure.count").getCounterValue()); } public void testFailsNodeThatIsUnhealthy() { + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); testBehaviourOfFailingNode( randomSettings(), () -> { throw new NodeHealthCheckFailureException("non writable exception"); }, "health check failed", 0, - () -> new StatusInfo(HEALTHY, "healthy-info") + () -> new StatusInfo(HEALTHY, "healthy-info"), + metricsRegistry ); + assertEquals(Integer.valueOf(2), metricsRegistry.getCounterStore().get("followers.checker.failure.count").getCounterValue()); } private void testBehaviourOfFailingNode( @@ -335,7 +361,8 @@ private void testBehaviourOfFailingNode( Supplier responder, String failureReason, long expectedFailureTime, - NodeHealthService nodeHealthService + NodeHealthService nodeHealthService, + MetricsRegistry metricsRegistry ) { final DiscoveryNode localNode = new DiscoveryNode("local-node", buildNewFakeTransportAddress(), Version.CURRENT); final DiscoveryNode otherNode = new DiscoveryNode("other-node", buildNewFakeTransportAddress(), Version.CURRENT); @@ -386,7 +413,6 @@ public String toString() { transportService.acceptIncomingRequests(); final AtomicBoolean nodeFailed = new AtomicBoolean(); - final FollowersChecker followersChecker = new FollowersChecker( settings, clusterSettings, @@ -396,7 +422,8 @@ public String toString() { assertTrue(nodeFailed.compareAndSet(false, true)); assertThat(reason, equalTo(failureReason)); }, - nodeHealthService + nodeHealthService, + new ClusterManagerMetrics(metricsRegistry) ); DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(localNode).add(otherNode).localNodeId(localNode.getId()).build(); @@ -501,7 +528,11 @@ protected void onSendRequest(long requestId, String action, TransportRequest req if (exception != null) { throw exception; } - }, (node, reason) -> { assert false : node; }, () -> new StatusInfo(UNHEALTHY, "unhealthy-info")); + }, + (node, reason) -> { assert false : node; }, + () -> new StatusInfo(UNHEALTHY, "unhealthy-info"), + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) + ); final long leaderTerm = randomLongBetween(2, Long.MAX_VALUE); final long followerTerm = randomLongBetween(1, leaderTerm - 1); @@ -574,7 +605,11 @@ protected void onSendRequest(long requestId, String action, TransportRequest req if (exception != null) { throw exception; } - }, (node, reason) -> { assert false : node; }, () -> new StatusInfo(HEALTHY, "healthy-info")); + }, + (node, reason) -> { assert false : node; }, + () -> new StatusInfo(HEALTHY, "healthy-info"), + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) + ); { // Does not call into the coordinator in the normal case @@ -721,7 +756,11 @@ public void testPreferClusterManagerNodes() { ); final FollowersChecker followersChecker = new FollowersChecker(Settings.EMPTY, clusterSettings, transportService, fcr -> { assert false : fcr; - }, (node, reason) -> { assert false : node; }, () -> new StatusInfo(HEALTHY, "healthy-info")); + }, + (node, reason) -> { assert false : node; }, + () -> new StatusInfo(HEALTHY, "healthy-info"), + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) + ); followersChecker.setCurrentNodes(discoveryNodes); List followerTargets = Stream.of(capturingTransport.getCapturedRequestsAndClear()) .map(cr -> cr.node) diff --git a/server/src/test/java/org/opensearch/cluster/coordination/LeaderCheckerTests.java b/server/src/test/java/org/opensearch/cluster/coordination/LeaderCheckerTests.java index fe65058333116..decdeddfa37a1 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/LeaderCheckerTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/LeaderCheckerTests.java @@ -34,6 +34,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.Version; +import org.opensearch.cluster.ClusterManagerMetrics; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.coordination.LeaderChecker.LeaderCheckRequest; import org.opensearch.cluster.node.DiscoveryNode; @@ -44,6 +45,7 @@ import org.opensearch.core.transport.TransportResponse; import org.opensearch.core.transport.TransportResponse.Empty; import org.opensearch.monitor.StatusInfo; +import org.opensearch.telemetry.TestInMemoryMetricsRegistry; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.EqualsHashCodeTestUtils; import org.opensearch.test.EqualsHashCodeTestUtils.CopyFunction; @@ -175,11 +177,13 @@ public String toString() { transportService.acceptIncomingRequests(); final AtomicBoolean leaderFailed = new AtomicBoolean(); + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); + final ClusterManagerMetrics clusterManagerMetrics = new ClusterManagerMetrics(metricsRegistry); final LeaderChecker leaderChecker = new LeaderChecker(settings, clusterSettings, transportService, e -> { assertThat(e.getMessage(), matchesRegex("node \\[.*\\] failed \\[[1-9][0-9]*\\] consecutive checks")); assertTrue(leaderFailed.compareAndSet(false, true)); - }, () -> new StatusInfo(StatusInfo.Status.HEALTHY, "healthy-info")); + }, () -> new StatusInfo(StatusInfo.Status.HEALTHY, "healthy-info"), clusterManagerMetrics); logger.info("--> creating first checker"); leaderChecker.updateLeader(leader1); @@ -229,6 +233,7 @@ public String toString() { ); } leaderChecker.updateLeader(null); + assertEquals(Integer.valueOf(1), metricsRegistry.getCounterStore().get("leader.checker.failure.count").getCounterValue()); } enum Response { @@ -293,10 +298,13 @@ public String toString() { transportService.acceptIncomingRequests(); final AtomicBoolean leaderFailed = new AtomicBoolean(); + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); + final ClusterManagerMetrics clusterManagerMetrics = new ClusterManagerMetrics(metricsRegistry); + final LeaderChecker leaderChecker = new LeaderChecker(settings, clusterSettings, transportService, e -> { assertThat(e.getMessage(), anyOf(endsWith("disconnected"), endsWith("disconnected during check"))); assertTrue(leaderFailed.compareAndSet(false, true)); - }, () -> new StatusInfo(StatusInfo.Status.HEALTHY, "healthy-info")); + }, () -> new StatusInfo(StatusInfo.Status.HEALTHY, "healthy-info"), clusterManagerMetrics); leaderChecker.updateLeader(leader); { @@ -351,6 +359,7 @@ public String toString() { deterministicTaskQueue.runAllRunnableTasks(); assertTrue(leaderFailed.get()); } + assertEquals(Integer.valueOf(3), metricsRegistry.getCounterStore().get("leader.checker.failure.count").getCounterValue()); } public void testFollowerFailsImmediatelyOnHealthCheckFailure() { @@ -407,10 +416,12 @@ public String toString() { transportService.acceptIncomingRequests(); final AtomicBoolean leaderFailed = new AtomicBoolean(); + TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); + final ClusterManagerMetrics clusterManagerMetrics = new ClusterManagerMetrics(metricsRegistry); final LeaderChecker leaderChecker = new LeaderChecker(settings, clusterSettings, transportService, e -> { assertThat(e.getMessage(), endsWith("failed health checks")); assertTrue(leaderFailed.compareAndSet(false, true)); - }, () -> new StatusInfo(StatusInfo.Status.HEALTHY, "healthy-info")); + }, () -> new StatusInfo(StatusInfo.Status.HEALTHY, "healthy-info"), clusterManagerMetrics); leaderChecker.updateLeader(leader); @@ -430,6 +441,8 @@ public String toString() { assertTrue(leaderFailed.get()); } + + assertEquals(Integer.valueOf(1), metricsRegistry.getCounterStore().get("leader.checker.failure.count").getCounterValue()); } public void testLeaderBehaviour() { @@ -453,12 +466,15 @@ public void testLeaderBehaviour() { transportService.start(); transportService.acceptIncomingRequests(); + final TestInMemoryMetricsRegistry metricsRegistry = new TestInMemoryMetricsRegistry(); + final ClusterManagerMetrics clusterManagerMetrics = new ClusterManagerMetrics(metricsRegistry); final LeaderChecker leaderChecker = new LeaderChecker( settings, clusterSettings, transportService, e -> fail("shouldn't be checking anything"), - () -> nodeHealthServiceStatus.get() + () -> nodeHealthServiceStatus.get(), + clusterManagerMetrics ); final DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() @@ -523,6 +539,7 @@ public void testLeaderBehaviour() { equalTo("rejecting leader check from [" + otherNode + "] sent to a node that is no longer the cluster-manager") ); } + assertEquals(Integer.valueOf(0), metricsRegistry.getCounterStore().get("leader.checker.failure.count").getCounterValue()); } private class CapturingTransportResponseHandler implements TransportResponseHandler { diff --git a/server/src/test/java/org/opensearch/cluster/coordination/NodeJoinTests.java b/server/src/test/java/org/opensearch/cluster/coordination/NodeJoinTests.java index 10d5dceb74f55..f84f0326f4a9d 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/NodeJoinTests.java @@ -273,7 +273,8 @@ protected void onSendRequest( ElectionStrategy.DEFAULT_INSTANCE, nodeHealthService, persistedStateRegistry, - Mockito.mock(RemoteStoreNodeService.class) + Mockito.mock(RemoteStoreNodeService.class), + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) ); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/opensearch/discovery/DiscoveryModuleTests.java b/server/src/test/java/org/opensearch/discovery/DiscoveryModuleTests.java index b33ebf8333b36..5539b3237c2bf 100644 --- a/server/src/test/java/org/opensearch/discovery/DiscoveryModuleTests.java +++ b/server/src/test/java/org/opensearch/discovery/DiscoveryModuleTests.java @@ -32,6 +32,7 @@ package org.opensearch.discovery; import org.opensearch.Version; +import org.opensearch.cluster.ClusterManagerMetrics; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.coordination.Coordinator; import org.opensearch.cluster.coordination.PersistedStateRegistry; @@ -48,6 +49,7 @@ import org.opensearch.gateway.GatewayMetaState; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.plugins.DiscoveryPlugin; +import org.opensearch.telemetry.metrics.noop.NoopMetricsRegistry; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.transport.MockTransportService; @@ -128,7 +130,8 @@ private DiscoveryModule newModule(Settings settings, List plugi mock(RerouteService.class), null, new PersistedStateRegistry(), - remoteStoreNodeService + remoteStoreNodeService, + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) ); } diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 5b39880930984..86de008b5dee5 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2559,7 +2559,8 @@ public void start(ClusterState initialState) { ElectionStrategy.DEFAULT_INSTANCE, () -> new StatusInfo(HEALTHY, "healthy-info"), persistedStateRegistry, - remoteStoreNodeService + remoteStoreNodeService, + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) ); clusterManagerService.setClusterStatePublisher(coordinator); coordinator.start(); diff --git a/server/src/test/java/org/opensearch/telemetry/TestInMemoryCounter.java b/server/src/test/java/org/opensearch/telemetry/TestInMemoryCounter.java new file mode 100644 index 0000000000000..d9aee5ebfa941 --- /dev/null +++ b/server/src/test/java/org/opensearch/telemetry/TestInMemoryCounter.java @@ -0,0 +1,52 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry; + +import org.opensearch.telemetry.metrics.Counter; +import org.opensearch.telemetry.metrics.tags.Tags; + +import java.util.HashMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * This is a simple implementation of Counter which is utilized by TestInMemoryMetricsRegistry for + * Unit Tests. It initializes an atomic integer to add the values of counter which doesn't have any tags + * along with a map to store the values recorded against the tags. + * The map and atomic integer can then be used to get the added values. + */ +public class TestInMemoryCounter implements Counter { + + private AtomicInteger counterValue = new AtomicInteger(0); + private ConcurrentHashMap, Double> counterValueForTags = new ConcurrentHashMap<>(); + + public Integer getCounterValue() { + return this.counterValue.get(); + } + + public ConcurrentHashMap, Double> getCounterValueForTags() { + return this.counterValueForTags; + } + + @Override + public void add(double value) { + counterValue.addAndGet((int) value); + } + + @Override + public synchronized void add(double value, Tags tags) { + HashMap hashMap = (HashMap) tags.getTagsMap(); + if (counterValueForTags.get(hashMap) == null) { + counterValueForTags.put(hashMap, value); + } else { + value = counterValueForTags.get(hashMap) + value; + counterValueForTags.put(hashMap, value); + } + } +} diff --git a/server/src/test/java/org/opensearch/telemetry/TestInMemoryHistogram.java b/server/src/test/java/org/opensearch/telemetry/TestInMemoryHistogram.java new file mode 100644 index 0000000000000..ff28df2b6529d --- /dev/null +++ b/server/src/test/java/org/opensearch/telemetry/TestInMemoryHistogram.java @@ -0,0 +1,47 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry; + +import org.opensearch.telemetry.metrics.Histogram; +import org.opensearch.telemetry.metrics.tags.Tags; + +import java.util.HashMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * This is a simple implementation of Histogram which is utilized by TestInMemoryMetricsRegistry for + * Unit Tests. It initializes an atomic integer to record the value of histogram which doesn't have any tags + * along with a map to store the values recorded against the tags. + * The map and atomic integer can then be used to get the recorded values. + */ +public class TestInMemoryHistogram implements Histogram { + + private AtomicInteger histogramValue = new AtomicInteger(0); + private ConcurrentHashMap, Double> histogramValueForTags = new ConcurrentHashMap<>(); + + public Integer getHistogramValue() { + return this.histogramValue.get(); + } + + public ConcurrentHashMap, Double> getHistogramValueForTags() { + return this.histogramValueForTags; + } + + @Override + public void record(double value) { + histogramValue.addAndGet((int) value); + } + + @Override + public synchronized void record(double value, Tags tags) { + HashMap hashMap = (HashMap) tags.getTagsMap(); + histogramValueForTags.put(hashMap, value); + } +} diff --git a/server/src/test/java/org/opensearch/telemetry/TestInMemoryMetricsRegistry.java b/server/src/test/java/org/opensearch/telemetry/TestInMemoryMetricsRegistry.java new file mode 100644 index 0000000000000..6d395085b12ea --- /dev/null +++ b/server/src/test/java/org/opensearch/telemetry/TestInMemoryMetricsRegistry.java @@ -0,0 +1,71 @@ +/* + * 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.telemetry; + +import org.opensearch.telemetry.metrics.Counter; +import org.opensearch.telemetry.metrics.Histogram; +import org.opensearch.telemetry.metrics.MetricsRegistry; +import org.opensearch.telemetry.metrics.tags.Tags; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; + +/** + * This is a simple implementation of MetricsRegistry which can be utilized by Unit Tests. + * It just initializes and stores counters/histograms within a map, once created. + * The maps can then be used to get the counters/histograms by their names. + */ +public class TestInMemoryMetricsRegistry implements MetricsRegistry { + + private ConcurrentHashMap counterStore = new ConcurrentHashMap<>(); + private ConcurrentHashMap histogramStore = new ConcurrentHashMap<>(); + + public ConcurrentHashMap getCounterStore() { + return this.counterStore; + } + + public ConcurrentHashMap getHistogramStore() { + return this.histogramStore; + } + + @Override + public Counter createCounter(String name, String description, String unit) { + TestInMemoryCounter counter = new TestInMemoryCounter(); + counterStore.putIfAbsent(name, counter); + return counter; + } + + @Override + public Counter createUpDownCounter(String name, String description, String unit) { + /** + * ToDo: To be implemented when required. + */ + return null; + } + + @Override + public Histogram createHistogram(String name, String description, String unit) { + TestInMemoryHistogram histogram = new TestInMemoryHistogram(); + histogramStore.putIfAbsent(name, histogram); + return histogram; + } + + @Override + public Closeable createGauge(String name, String description, String unit, Supplier valueProvider, Tags tags) { + /** + * ToDo: To be implemented when required. + */ + return null; + } + + @Override + public void close() throws IOException {} +} 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 3ae737bf63923..1c2270bab1260 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 @@ -1183,7 +1183,8 @@ protected Optional getDisruptableMockTransport(Transpo getElectionStrategy(), nodeHealthService, persistedStateRegistry, - remoteStoreNodeService + remoteStoreNodeService, + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) ); clusterManagerService.setClusterStatePublisher(coordinator); final GatewayService gatewayService = new GatewayService( From 7faae255c2593921638e4a91898dce8d2867bede Mon Sep 17 00:00:00 2001 From: Himshikha Gupta Date: Thu, 6 Jun 2024 14:20:00 +0530 Subject: [PATCH 54/77] [Remote Routing Table] Initial commit for index routing table manifest (#13577) * Initial commit for index routing table manifest Co-authored-by: Bukhtawar Khan Co-authored-by: Himshikha Gupta Co-authored-by: Arpit Bandejiya --- .../stream}/BufferedChecksumStreamInput.java | 4 +- .../stream}/BufferedChecksumStreamOutput.java | 3 +- .../remote/ClusterMetadataManifest.java | 112 ++++++++++++++++-- .../remote/RemoteClusterStateService.java | 7 +- .../routingtable/IndexRoutingTableHeader.java | 81 +++++++++++++ .../routingtable/RemoteIndexRoutingTable.java | 100 ++++++++++++++++ .../remote/routingtable/package-info.java | 12 ++ .../index/translog/BaseTranslogReader.java | 1 + .../opensearch/index/translog/Translog.java | 2 + .../index/translog/TranslogHeader.java | 2 + .../index/translog/TranslogSnapshot.java | 1 + .../index/translog/TranslogWriter.java | 1 + .../remote/ClusterMetadataManifestTests.java | 110 ++++++++++++++++- .../RemoteClusterStateServiceTests.java | 10 +- .../IndexRoutingTableHeaderTests.java | 32 +++++ .../RemoteIndexRoutingTableTests.java | 87 ++++++++++++++ .../snapshots/BlobStoreFormatTests.java | 2 +- 17 files changed, 543 insertions(+), 24 deletions(-) rename {server/src/main/java/org/opensearch/index/translog => libs/core/src/main/java/org/opensearch/core/common/io/stream}/BufferedChecksumStreamInput.java (96%) rename {server/src/main/java/org/opensearch/index/translog => libs/core/src/main/java/org/opensearch/core/common/io/stream}/BufferedChecksumStreamOutput.java (96%) create mode 100644 server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeader.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTable.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/routingtable/package-info.java create mode 100644 server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeaderTests.java create mode 100644 server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableTests.java diff --git a/server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamInput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamInput.java similarity index 96% rename from server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamInput.java rename to libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamInput.java index f75f27b7bcb91..41680961b36e9 100644 --- a/server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamInput.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamInput.java @@ -30,12 +30,10 @@ * GitHub history for details. */ -package org.opensearch.index.translog; +package org.opensearch.core.common.io.stream; import org.apache.lucene.store.BufferedChecksum; import org.apache.lucene.util.BitUtil; -import org.opensearch.core.common.io.stream.FilterStreamInput; -import org.opensearch.core.common.io.stream.StreamInput; import java.io.EOFException; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamOutput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamOutput.java similarity index 96% rename from server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamOutput.java rename to libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamOutput.java index 9e96664c79cc5..422f956c0cd47 100644 --- a/server/src/main/java/org/opensearch/index/translog/BufferedChecksumStreamOutput.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/BufferedChecksumStreamOutput.java @@ -30,11 +30,10 @@ * GitHub history for details. */ -package org.opensearch.index.translog; +package org.opensearch.core.common.io.stream; import org.apache.lucene.store.BufferedChecksum; import org.opensearch.common.annotation.PublicApi; -import org.opensearch.core.common.io.stream.StreamOutput; import java.io.IOException; import java.util.zip.CRC32; diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java index bf02c73ca560b..b3b1bf37f8696 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java @@ -41,6 +41,7 @@ public class ClusterMetadataManifest implements Writeable, ToXContentFragment { public static final int CODEC_V0 = 0; // Older codec version, where we haven't introduced codec versions for manifest. public static final int CODEC_V1 = 1; // In Codec V1 we have introduced global-metadata and codec version in Manifest file. public static final int CODEC_V2 = 2; // In Codec V2, there are seperate metadata files rather than a single global metadata file. + public static final int CODEC_V3 = 3; // In Codec V3, we introduce index routing-metadata in manifest file. private static final ParseField CLUSTER_TERM_FIELD = new ParseField("cluster_term"); private static final ParseField STATE_VERSION_FIELD = new ParseField("state_version"); @@ -58,6 +59,8 @@ public class ClusterMetadataManifest implements Writeable, ToXContentFragment { private static final ParseField UPLOADED_SETTINGS_METADATA = new ParseField("uploaded_settings_metadata"); private static final ParseField UPLOADED_TEMPLATES_METADATA = new ParseField("uploaded_templates_metadata"); private static final ParseField UPLOADED_CUSTOM_METADATA = new ParseField("uploaded_custom_metadata"); + private static final ParseField ROUTING_TABLE_VERSION_FIELD = new ParseField("routing_table_version"); + private static final ParseField INDICES_ROUTING_FIELD = new ParseField("indices_routing"); private static ClusterMetadataManifest.Builder manifestV0Builder(Object[] fields) { return ClusterMetadataManifest.builder() @@ -86,6 +89,12 @@ private static ClusterMetadataManifest.Builder manifestV2Builder(Object[] fields .customMetadataMap(customMetadata(fields)); } + private static ClusterMetadataManifest.Builder manifestV3Builder(Object[] fields) { + return manifestV2Builder(fields).codecVersion(codecVersion(fields)) + .routingTableVersion(routingTableVersion(fields)) + .indicesRouting(indicesRouting(fields)); + } + private static long term(Object[] fields) { return (long) fields[0]; } @@ -151,6 +160,14 @@ private static Map customMetadata(Object[] fi return customs.stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())); } + private static long routingTableVersion(Object[] fields) { + return (long) fields[15]; + } + + private static List indicesRouting(Object[] fields) { + return (List) fields[16]; + } + private static final ConstructingObjectParser PARSER_V0 = new ConstructingObjectParser<>( "cluster_metadata_manifest", fields -> manifestV0Builder(fields).build() @@ -166,12 +183,18 @@ private static Map customMetadata(Object[] fi fields -> manifestV2Builder(fields).build() ); - private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V2; + private static final ConstructingObjectParser PARSER_V3 = new ConstructingObjectParser<>( + "cluster_metadata_manifest", + fields -> manifestV3Builder(fields).build() + ); + + private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V3; static { declareParser(PARSER_V0, CODEC_V0); declareParser(PARSER_V1, CODEC_V1); declareParser(PARSER_V2, CODEC_V2); + declareParser(PARSER_V3, CODEC_V3); } private static void declareParser(ConstructingObjectParser parser, long codec_version) { @@ -216,6 +239,14 @@ private static void declareParser(ConstructingObjectParser= CODEC_V3) { + parser.declareLong(ConstructingObjectParser.constructorArg(), ROUTING_TABLE_VERSION_FIELD); + parser.declareObjectArray( + ConstructingObjectParser.constructorArg(), + (p, c) -> UploadedIndexMetadata.fromXContent(p), + INDICES_ROUTING_FIELD + ); + } } private final int codecVersion; @@ -234,6 +265,8 @@ private static void declareParser(ConstructingObjectParser indicesRouting; public List getIndices() { return indices; @@ -306,6 +339,14 @@ public boolean hasMetadataAttributesFiles() { || !uploadedCustomMetadataMap.isEmpty(); } + public long getRoutingTableVersion() { + return routingTableVersion; + } + + public List getIndicesRouting() { + return indicesRouting; + } + public ClusterMetadataManifest( long clusterTerm, long version, @@ -322,7 +363,9 @@ public ClusterMetadataManifest( UploadedMetadataAttribute uploadedCoordinationMetadata, UploadedMetadataAttribute uploadedSettingsMetadata, UploadedMetadataAttribute uploadedTemplatesMetadata, - Map uploadedCustomMetadataMap + Map uploadedCustomMetadataMap, + long routingTableVersion, + List indicesRouting ) { this.clusterTerm = clusterTerm; this.stateVersion = version; @@ -336,6 +379,8 @@ public ClusterMetadataManifest( this.indices = Collections.unmodifiableList(indices); this.previousClusterUUID = previousClusterUUID; this.clusterUUIDCommitted = clusterUUIDCommitted; + this.routingTableVersion = routingTableVersion; + this.indicesRouting = Collections.unmodifiableList(indicesRouting); this.uploadedCoordinationMetadata = uploadedCoordinationMetadata; this.uploadedSettingsMetadata = uploadedSettingsMetadata; this.uploadedTemplatesMetadata = uploadedTemplatesMetadata; @@ -364,6 +409,8 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { in.readMap(StreamInput::readString, UploadedMetadataAttribute::new) ); this.globalMetadataFileName = null; + this.routingTableVersion = in.readLong(); + this.indicesRouting = Collections.unmodifiableList(in.readList(UploadedIndexMetadata::new)); } else if (in.getVersion().onOrAfter(Version.V_2_12_0)) { this.codecVersion = in.readInt(); this.globalMetadataFileName = in.readString(); @@ -371,6 +418,8 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { this.uploadedSettingsMetadata = null; this.uploadedTemplatesMetadata = null; this.uploadedCustomMetadataMap = null; + this.routingTableVersion = -1; + this.indicesRouting = null; } else { this.codecVersion = CODEC_V0; // Default codec this.globalMetadataFileName = null; @@ -378,6 +427,8 @@ public ClusterMetadataManifest(StreamInput in) throws IOException { this.uploadedSettingsMetadata = null; this.uploadedTemplatesMetadata = null; this.uploadedCustomMetadataMap = null; + this.routingTableVersion = -1; + this.indicesRouting = null; } } @@ -401,7 +452,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startArray(INDICES_FIELD.getPreferredName()); { for (UploadedIndexMetadata uploadedIndexMetadata : indices) { + builder.startObject(); uploadedIndexMetadata.toXContent(builder, params); + builder.endObject(); } } builder.endArray(); @@ -433,6 +486,18 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(CODEC_VERSION_FIELD.getPreferredName(), getCodecVersion()); builder.field(GLOBAL_METADATA_FIELD.getPreferredName(), getGlobalMetadataFileName()); } + if (onOrAfterCodecVersion(CODEC_V3)) { + builder.field(ROUTING_TABLE_VERSION_FIELD.getPreferredName(), getRoutingTableVersion()); + builder.startArray(INDICES_ROUTING_FIELD.getPreferredName()); + { + for (UploadedIndexMetadata uploadedIndexMetadata : indicesRouting) { + builder.startObject(); + uploadedIndexMetadata.toXContent(builder, params); + builder.endObject(); + } + } + builder.endArray(); + } return builder; } @@ -454,6 +519,8 @@ public void writeTo(StreamOutput out) throws IOException { uploadedSettingsMetadata.writeTo(out); uploadedTemplatesMetadata.writeTo(out); out.writeMap(uploadedCustomMetadataMap, StreamOutput::writeString, (o, v) -> v.writeTo(o)); + out.writeLong(routingTableVersion); + out.writeCollection(indicesRouting); } else if (out.getVersion().onOrAfter(Version.V_2_12_0)) { out.writeInt(codecVersion); out.writeString(globalMetadataFileName); @@ -480,7 +547,9 @@ public boolean equals(Object o) { && Objects.equals(previousClusterUUID, that.previousClusterUUID) && Objects.equals(clusterUUIDCommitted, that.clusterUUIDCommitted) && Objects.equals(globalMetadataFileName, that.globalMetadataFileName) - && Objects.equals(codecVersion, that.codecVersion); + && Objects.equals(codecVersion, that.codecVersion) + && Objects.equals(routingTableVersion, that.routingTableVersion) + && Objects.equals(indicesRouting, that.indicesRouting); } @Override @@ -497,7 +566,9 @@ public int hashCode() { nodeId, committed, previousClusterUUID, - clusterUUIDCommitted + clusterUUIDCommitted, + routingTableVersion, + indicesRouting ); } @@ -518,6 +589,10 @@ public static ClusterMetadataManifest fromXContentV1(XContentParser parser) thro return PARSER_V1.parse(parser, null); } + public static ClusterMetadataManifest fromXContentV2(XContentParser parser) throws IOException { + return PARSER_V2.parse(parser, null); + } + public static ClusterMetadataManifest fromXContent(XContentParser parser) throws IOException { return CURRENT_PARSER.parse(parser, null); } @@ -545,12 +620,24 @@ public static class Builder { private String previousClusterUUID; private boolean committed; private boolean clusterUUIDCommitted; + private long routingTableVersion; + private List indicesRouting; public Builder indices(List indices) { this.indices = indices; return this; } + public Builder routingTableVersion(long routingTableVersion) { + this.routingTableVersion = routingTableVersion; + return this; + } + + public Builder indicesRouting(List indicesRouting) { + this.indicesRouting = indicesRouting; + return this; + } + public Builder codecVersion(int codecVersion) { this.codecVersion = codecVersion; return this; @@ -625,6 +712,10 @@ public List getIndices() { return indices; } + public List getIndicesRouting() { + return indicesRouting; + } + public Builder previousClusterUUID(String previousClusterUUID) { this.previousClusterUUID = previousClusterUUID; return this; @@ -638,6 +729,7 @@ public Builder clusterUUIDCommitted(boolean clusterUUIDCommitted) { public Builder() { indices = new ArrayList<>(); customMetadataMap = new HashMap<>(); + indicesRouting = new ArrayList<>(); } public Builder(ClusterMetadataManifest manifest) { @@ -657,6 +749,8 @@ public Builder(ClusterMetadataManifest manifest) { this.indices = new ArrayList<>(manifest.indices); this.previousClusterUUID = manifest.previousClusterUUID; this.clusterUUIDCommitted = manifest.clusterUUIDCommitted; + this.routingTableVersion = manifest.routingTableVersion; + this.indicesRouting = new ArrayList<>(manifest.indicesRouting); } public ClusterMetadataManifest build() { @@ -676,7 +770,9 @@ public ClusterMetadataManifest build() { coordinationMetadata, settingsMetadata, templatesMetadata, - customMetadataMap + customMetadataMap, + routingTableVersion, + indicesRouting ); } @@ -776,11 +872,9 @@ public String getIndexUUID() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - return builder.startObject() - .field(INDEX_NAME_FIELD.getPreferredName(), getIndexName()) + return builder.field(INDEX_NAME_FIELD.getPreferredName(), getIndexName()) .field(INDEX_UUID_FIELD.getPreferredName(), getIndexUUID()) - .field(UPLOADED_FILENAME_FIELD.getPreferredName(), getUploadedFilePath()) - .endObject(); + .field(UPLOADED_FILENAME_FIELD.getPreferredName(), getUploadedFilePath()); } @Override diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index fdf6b2f94ba6b..d0593dcd51475 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -216,7 +216,7 @@ public class RemoteClusterStateService implements Closeable { + "indices, coordination metadata updated : [{}], settings metadata updated : [{}], templates metadata " + "updated : [{}], custom metadata updated : [{}]"; public static final int INDEX_METADATA_CURRENT_CODEC_VERSION = 1; - public static final int MANIFEST_CURRENT_CODEC_VERSION = ClusterMetadataManifest.CODEC_V2; + public static final int MANIFEST_CURRENT_CODEC_VERSION = ClusterMetadataManifest.CODEC_V3; public static final int GLOBAL_METADATA_CURRENT_CODEC_VERSION = 2; // ToXContent Params with gateway mode. @@ -806,7 +806,10 @@ private ClusterMetadataManifest uploadManifest( uploadedCoordinationMetadata, uploadedSettingsMetadata, uploadedTemplatesMetadata, - uploadedCustomMetadataMap + uploadedCustomMetadataMap, + clusterState.routingTable().version(), + // TODO: Add actual list of changed indices routing with index routing upload flow. + new ArrayList<>() ); writeMetadataManifest(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), manifest, manifestFileName); return manifest; diff --git a/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeader.java b/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeader.java new file mode 100644 index 0000000000000..5baea6adba0c7 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeader.java @@ -0,0 +1,81 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote.routingtable; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexFormatTooNewException; +import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.store.InputStreamDataInput; +import org.apache.lucene.store.OutputStreamDataOutput; +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.EOFException; +import java.io.IOException; + +/** + * The stored header information for the individual index routing table + */ +public class IndexRoutingTableHeader implements Writeable { + + public static final String INDEX_ROUTING_HEADER_CODEC = "index_routing_header_codec"; + public static final int INITIAL_VERSION = 1; + public static final int CURRENT_VERSION = INITIAL_VERSION; + private final String indexName; + + public IndexRoutingTableHeader(String indexName) { + this.indexName = indexName; + } + + /** + * Reads the contents on the stream into the corresponding {@link IndexRoutingTableHeader} + * + * @param in streamInput + * @throws IOException exception thrown on failing to read from stream. + */ + public IndexRoutingTableHeader(StreamInput in) throws IOException { + try { + readHeaderVersion(in); + indexName = in.readString(); + } catch (EOFException e) { + throw new IOException("index routing header truncated", e); + } + } + + private void readHeaderVersion(final StreamInput in) throws IOException { + try { + CodecUtil.checkHeader(new InputStreamDataInput(in), INDEX_ROUTING_HEADER_CODEC, INITIAL_VERSION, CURRENT_VERSION); + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException e) { + throw new IOException("index routing table header corrupted", e); + } + } + + /** + * Write the IndexRoutingTable to given stream. + * + * @param out stream to write + * @throws IOException exception thrown on failing to write to stream. + */ + public void writeTo(StreamOutput out) throws IOException { + try { + CodecUtil.writeHeader(new OutputStreamDataOutput(out), INDEX_ROUTING_HEADER_CODEC, CURRENT_VERSION); + out.writeString(indexName); + out.flush(); + } catch (IOException e) { + throw new IOException("Failed to write IndexRoutingTable header", e); + } + } + + public String getIndexName() { + return indexName; + } + +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTable.java b/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTable.java new file mode 100644 index 0000000000000..17c55190da07f --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTable.java @@ -0,0 +1,100 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote.routingtable; + +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamInput; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; +import org.opensearch.core.common.io.stream.InputStreamStreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.index.Index; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; + +/** + * Remote store object for IndexRoutingTable + */ +public class RemoteIndexRoutingTable implements Writeable { + + private final IndexRoutingTable indexRoutingTable; + + public RemoteIndexRoutingTable(IndexRoutingTable indexRoutingTable) { + this.indexRoutingTable = indexRoutingTable; + } + + /** + * Reads data from inputStream and creates RemoteIndexRoutingTable object with the {@link IndexRoutingTable} + * @param inputStream input stream with index routing data + * @param index index for the current routing data + * @throws IOException exception thrown on failing to read from stream. + */ + public RemoteIndexRoutingTable(InputStream inputStream, Index index) throws IOException { + try { + try (BufferedChecksumStreamInput in = new BufferedChecksumStreamInput(new InputStreamStreamInput(inputStream), "assertion")) { + // Read the Table Header first and confirm the index + IndexRoutingTableHeader indexRoutingTableHeader = new IndexRoutingTableHeader(in); + assert indexRoutingTableHeader.getIndexName().equals(index.getName()); + + int numberOfShardRouting = in.readVInt(); + IndexRoutingTable.Builder indicesRoutingTable = IndexRoutingTable.builder(index); + for (int idx = 0; idx < numberOfShardRouting; idx++) { + IndexShardRoutingTable indexShardRoutingTable = IndexShardRoutingTable.Builder.readFrom(in); + indicesRoutingTable.addIndexShard(indexShardRoutingTable); + } + verifyCheckSum(in); + indexRoutingTable = indicesRoutingTable.build(); + } + } catch (EOFException e) { + throw new IOException("Indices Routing table is corrupted", e); + } + } + + public IndexRoutingTable getIndexRoutingTable() { + return indexRoutingTable; + } + + /** + * Writes {@link IndexRoutingTable} to the given stream + * @param streamOutput output stream to write + * @throws IOException exception thrown on failing to write to stream. + */ + @Override + public void writeTo(StreamOutput streamOutput) throws IOException { + try { + BufferedChecksumStreamOutput out = new BufferedChecksumStreamOutput(streamOutput); + IndexRoutingTableHeader indexRoutingTableHeader = new IndexRoutingTableHeader(indexRoutingTable.getIndex().getName()); + indexRoutingTableHeader.writeTo(out); + out.writeVInt(indexRoutingTable.shards().size()); + for (IndexShardRoutingTable next : indexRoutingTable) { + IndexShardRoutingTable.Builder.writeTo(next, out); + } + out.writeLong(out.getChecksum()); + out.flush(); + } catch (IOException e) { + throw new IOException("Failed to write IndexRoutingTable to stream", e); + } + } + + private void verifyCheckSum(BufferedChecksumStreamInput in) throws IOException { + long expectedChecksum = in.getChecksum(); + long readChecksum = in.readLong(); + if (readChecksum != expectedChecksum) { + throw new IOException( + "checksum verification failed - expected: 0x" + + Long.toHexString(expectedChecksum) + + ", got: 0x" + + Long.toHexString(readChecksum) + ); + } + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/routingtable/package-info.java b/server/src/main/java/org/opensearch/gateway/remote/routingtable/package-info.java new file mode 100644 index 0000000000000..a6cb2251a5dd7 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/routingtable/package-info.java @@ -0,0 +1,12 @@ +/* + * 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 containing class to perform operations on remote routing table. + */ +package org.opensearch.gateway.remote.routingtable; diff --git a/server/src/main/java/org/opensearch/index/translog/BaseTranslogReader.java b/server/src/main/java/org/opensearch/index/translog/BaseTranslogReader.java index d6fa2a2e53de3..37af1dcbeab8b 100644 --- a/server/src/main/java/org/opensearch/index/translog/BaseTranslogReader.java +++ b/server/src/main/java/org/opensearch/index/translog/BaseTranslogReader.java @@ -32,6 +32,7 @@ package org.opensearch.index.translog; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamInput; import org.opensearch.core.common.io.stream.ByteBufferStreamInput; import org.opensearch.index.seqno.SequenceNumbers; 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 842e9c77d2350..87e0c21b8203c 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -48,6 +48,8 @@ import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamInput; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java b/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java index 7b5be9505f27a..66a9fe08d06b5 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java @@ -40,6 +40,8 @@ import org.apache.lucene.store.OutputStreamDataOutput; import org.apache.lucene.util.BytesRef; import org.opensearch.common.io.Channels; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamInput; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.InputStreamStreamInput; import org.opensearch.core.common.io.stream.OutputStreamStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogSnapshot.java b/server/src/main/java/org/opensearch/index/translog/TranslogSnapshot.java index 89718156cbbe8..521472f4d64a0 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogSnapshot.java @@ -32,6 +32,7 @@ package org.opensearch.index.translog; import org.opensearch.common.io.Channels; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamInput; import org.opensearch.index.seqno.SequenceNumbers; import java.io.EOFException; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java index 86f7567f3333d..b0c7d51c3e43b 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java @@ -50,6 +50,7 @@ import org.opensearch.core.Assertions; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamInput; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.seqno.SequenceNumbers; diff --git a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java index 0b3cd49140939..d1f559eb75f85 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java @@ -99,7 +99,7 @@ public void testClusterMetadataManifestXContent() throws IOException { Version.CURRENT, "test-node-id", false, - ClusterMetadataManifest.CODEC_V2, + ClusterMetadataManifest.CODEC_V3, null, Collections.singletonList(uploadedIndexMetadata), "prev-cluster-uuid", @@ -123,7 +123,9 @@ public void testClusterMetadataManifestXContent() throws IOException { "custom--weighted_routing_netadata-file" ) ) - ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())) + ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())), + 1L, + randomUploadedIndexMetadataList() ); final XContentBuilder builder = JsonXContent.contentBuilder(); builder.startObject(); @@ -169,7 +171,9 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { "custom--weighted_routing_netadata-file" ) ) - ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())) + ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())), + 1L, + randomUploadedIndexMetadataList() ); { // Mutate Cluster Term EqualsHashCodeTestUtils.checkEqualsAndHashCode( @@ -309,6 +313,106 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { } } + public void testClusterMetadataManifestXContentV2() throws IOException { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + UploadedMetadataAttribute uploadedMetadataAttribute = new UploadedMetadataAttribute("attribute_name", "testing_attribute"); + ClusterMetadataManifest originalManifest = new ClusterMetadataManifest( + 1L, + 1L, + "test-cluster-uuid", + "test-state-uuid", + Version.CURRENT, + "test-node-id", + false, + ClusterMetadataManifest.CODEC_V2, + null, + Collections.singletonList(uploadedIndexMetadata), + "prev-cluster-uuid", + true, + uploadedMetadataAttribute, + uploadedMetadataAttribute, + uploadedMetadataAttribute, + Collections.unmodifiableList( + Arrays.asList( + new UploadedMetadataAttribute( + RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + RepositoriesMetadata.TYPE, + "custom--repositories-file" + ), + new UploadedMetadataAttribute( + RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + IndexGraveyard.TYPE, + "custom--index_graveyard-file" + ), + new UploadedMetadataAttribute( + RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + + WeightedRoutingMetadata.TYPE, + "custom--weighted_routing_netadata-file" + ) + ) + ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())), + 0, + new ArrayList<>() + ); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + originalManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterMetadataManifest fromXContentManifest = ClusterMetadataManifest.fromXContentV2(parser); + assertEquals(originalManifest, fromXContentManifest); + } + } + + public void testClusterMetadataManifestXContentV3() throws IOException { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + UploadedMetadataAttribute uploadedMetadataAttribute = new UploadedMetadataAttribute("attribute_name", "testing_attribute"); + ClusterMetadataManifest originalManifest = new ClusterMetadataManifest( + 1L, + 1L, + "test-cluster-uuid", + "test-state-uuid", + Version.CURRENT, + "test-node-id", + false, + ClusterMetadataManifest.CODEC_V3, + null, + Collections.singletonList(uploadedIndexMetadata), + "prev-cluster-uuid", + true, + uploadedMetadataAttribute, + uploadedMetadataAttribute, + uploadedMetadataAttribute, + Collections.unmodifiableList( + Arrays.asList( + new UploadedMetadataAttribute( + RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + RepositoriesMetadata.TYPE, + "custom--repositories-file" + ), + new UploadedMetadataAttribute( + RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + IndexGraveyard.TYPE, + "custom--index_graveyard-file" + ), + new UploadedMetadataAttribute( + RemoteClusterStateService.CUSTOM_METADATA + RemoteClusterStateService.CUSTOM_DELIMITER + + WeightedRoutingMetadata.TYPE, + "custom--weighted_routing_netadata-file" + ) + ) + ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())), + 1L, + Collections.singletonList(uploadedIndexMetadata) + ); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + originalManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterMetadataManifest fromXContentManifest = ClusterMetadataManifest.fromXContent(parser); + assertEquals(originalManifest, fromXContentManifest); + } + } + private List randomUploadedIndexMetadataList() { final int size = randomIntBetween(1, 10); final List uploadedIndexMetadataList = new ArrayList<>(size); diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index 324023061ac46..4a53770c76d88 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -548,7 +548,7 @@ private void verifyWriteIncrementalGlobalMetadataFromOlderCodecSuccess(ClusterMe ); final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() - .codecVersion(2) + .codecVersion(3) .indices(Collections.emptyList()) .clusterTerm(1L) .stateVersion(1L) @@ -1072,6 +1072,8 @@ public void testReadGlobalMetadata() throws IOException { .nodeId("nodeA") .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) .previousClusterUUID("prev-cluster-uuid") + .routingTableVersion(1) + .indicesRouting(List.of()) .build(); Metadata expactedMetadata = Metadata.builder().persistentSettings(Settings.builder().put("readonly", true).build()).build(); @@ -1469,7 +1471,7 @@ private void mockObjectsForGettingPreviousClusterUUID( .build(); Map indexMetadataMap1 = Map.of("index-uuid1", indexMetadata1, "index-uuid2", indexMetadata2); mockBlobContainerForGlobalMetadata(blobContainer1, clusterManifest1, metadata1); - mockBlobContainer(blobContainer1, clusterManifest1, indexMetadataMap1, ClusterMetadataManifest.CODEC_V2); + mockBlobContainer(blobContainer1, clusterManifest1, indexMetadataMap1, ClusterMetadataManifest.CODEC_V3); List uploadedIndexMetadataList2 = List.of( new UploadedIndexMetadata("index1", "index-uuid1", "key1"), @@ -1501,7 +1503,7 @@ private void mockObjectsForGettingPreviousClusterUUID( .build(); Map indexMetadataMap2 = Map.of("index-uuid1", indexMetadata3, "index-uuid2", indexMetadata4); mockBlobContainerForGlobalMetadata(blobContainer2, clusterManifest2, metadata2); - mockBlobContainer(blobContainer2, clusterManifest2, indexMetadataMap2, ClusterMetadataManifest.CODEC_V2); + mockBlobContainer(blobContainer2, clusterManifest2, indexMetadataMap2, ClusterMetadataManifest.CODEC_V3); // differGlobalMetadata controls which one of IndexMetadata or Metadata object would be different // when comparing cluster-uuid3 and cluster-uuid1 state. @@ -1535,7 +1537,7 @@ private void mockObjectsForGettingPreviousClusterUUID( clusterUUIDCommitted.getOrDefault("cluster-uuid3", true) ); mockBlobContainerForGlobalMetadata(blobContainer3, clusterManifest3, metadata3); - mockBlobContainer(blobContainer3, clusterManifest3, indexMetadataMap3, ClusterMetadataManifest.CODEC_V2); + mockBlobContainer(blobContainer3, clusterManifest3, indexMetadataMap3, ClusterMetadataManifest.CODEC_V3); ArrayList mockBlobContainerOrderedList = new ArrayList<>( List.of(blobContainer1, blobContainer1, blobContainer3, blobContainer3, blobContainer2, blobContainer2) diff --git a/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeaderTests.java b/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeaderTests.java new file mode 100644 index 0000000000000..a3f0ac36a40f1 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/routingtable/IndexRoutingTableHeaderTests.java @@ -0,0 +1,32 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote.routingtable; + +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; + +public class IndexRoutingTableHeaderTests extends OpenSearchTestCase { + + public void testIndexRoutingTableHeader() throws IOException { + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + IndexRoutingTableHeader header = new IndexRoutingTableHeader(indexName); + try (BytesStreamOutput out = new BytesStreamOutput()) { + header.writeTo(out); + + BytesStreamInput in = new BytesStreamInput(out.bytes().toBytesRef().bytes); + IndexRoutingTableHeader headerRead = new IndexRoutingTableHeader(in); + assertEquals(indexName, headerRead.getIndexName()); + + } + } + +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableTests.java b/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableTests.java new file mode 100644 index 0000000000000..72066d8afb45b --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableTests.java @@ -0,0 +1,87 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote.routingtable; + +import org.opensearch.Version; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +public class RemoteIndexRoutingTableTests extends OpenSearchTestCase { + + public void testRoutingTableInput() { + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + int numberOfShards = randomIntBetween(1, 10); + int numberOfReplicas = randomIntBetween(1, 10); + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + ) + .build(); + + RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index(indexName)).build(); + + initialRoutingTable.getIndicesRouting().values().forEach(indexShardRoutingTables -> { + RemoteIndexRoutingTable indexRouting = new RemoteIndexRoutingTable(indexShardRoutingTables); + try (BytesStreamOutput streamOutput = new BytesStreamOutput();) { + indexRouting.writeTo(streamOutput); + RemoteIndexRoutingTable remoteIndexRoutingTable = new RemoteIndexRoutingTable( + streamOutput.bytes().streamInput(), + metadata.index(indexName).getIndex() + ); + IndexRoutingTable indexRoutingTable = remoteIndexRoutingTable.getIndexRoutingTable(); + assertEquals(numberOfShards, indexRoutingTable.getShards().size()); + assertEquals(metadata.index(indexName).getIndex(), indexRoutingTable.getIndex()); + assertEquals( + numberOfShards * (1 + numberOfReplicas), + indexRoutingTable.shardsWithState(ShardRoutingState.UNASSIGNED).size() + ); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + public void testRoutingTableInputStreamWithInvalidIndex() { + Metadata metadata = Metadata.builder() + .put(IndexMetadata.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) + .put(IndexMetadata.builder("invalid-index").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) + .build(); + + RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index("test")).build(); + AtomicInteger assertionError = new AtomicInteger(); + initialRoutingTable.getIndicesRouting().values().forEach(indexShardRoutingTables -> { + RemoteIndexRoutingTable indexRouting = new RemoteIndexRoutingTable(indexShardRoutingTables); + try (BytesStreamOutput streamOutput = new BytesStreamOutput()) { + indexRouting.writeTo(streamOutput); + RemoteIndexRoutingTable remoteIndexRoutingTable = new RemoteIndexRoutingTable( + streamOutput.bytes().streamInput(), + metadata.index("invalid-index").getIndex() + ); + } catch (AssertionError e) { + assertionError.getAndIncrement(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + assertEquals(1, assertionError.get()); + } + +} diff --git a/server/src/test/java/org/opensearch/snapshots/BlobStoreFormatTests.java b/server/src/test/java/org/opensearch/snapshots/BlobStoreFormatTests.java index c5f36fcc01983..95a8267734a07 100644 --- a/server/src/test/java/org/opensearch/snapshots/BlobStoreFormatTests.java +++ b/server/src/test/java/org/opensearch/snapshots/BlobStoreFormatTests.java @@ -49,13 +49,13 @@ import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.core.common.io.stream.BufferedChecksumStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.compress.CompressorRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.index.translog.BufferedChecksumStreamOutput; import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; import org.opensearch.test.OpenSearchTestCase; From b01b6e87227e29b2ec922577760f4bb5fa6de0bb Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Thu, 6 Jun 2024 17:51:10 +0530 Subject: [PATCH 55/77] [Remote Store] Add checks to skip remote uploads after shard is closed (#13998) Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../shard/RemoteStoreRefreshListener.java | 16 ++++ .../RemoteStoreRefreshListenerTests.java | 80 ++++++++++++++++++- 2 files changed, 93 insertions(+), 3 deletions(-) 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 20afd7b2f3568..77556f8391473 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -170,6 +170,13 @@ protected boolean performAfterRefreshWithPermit(boolean didRefresh) { * @return true if sync is needed */ private boolean shouldSync(boolean didRefresh, boolean skipPrimaryTermCheck) { + // Ignore syncing segments if the underlying shard is closed + // This also makes sure that retries are not scheduled for shards + // with failed syncSegments invocation after they are closed + if (shardClosed()) { + logger.info("Shard is already closed. Not attempting sync to remote store"); + return false; + } boolean shouldSync = didRefresh // If the readers change, didRefresh is always true. // The third condition exists for uploading the zero state segments where the refresh has not changed the reader // reference, but it is important to upload the zero state segments so that the restore does not break. @@ -607,6 +614,15 @@ public void onFailure(String file) { }; } + /** + * Checks if the underlying IndexShard instance is closed + * + * @return true if it is closed, false otherwise + */ + private boolean shardClosed() { + return indexShard.state() == IndexShardState.CLOSED; + } + @Override protected Logger getLogger() { return logger; diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index bb0776e0ced25..67787e8583930 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -48,6 +48,7 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; @@ -470,6 +471,25 @@ public void testRefreshPersistentFailure() throws Exception { assertFalse("remote store should not in sync", tuple.v1().isRemoteSegmentStoreInSync()); } + public void testRefreshPersistentFailureAndIndexShardClosed() throws Exception { + int succeedOnAttempt = 3; + int closeShardOnAttempt = 1; + CountDownLatch refreshCountLatch = new CountDownLatch(1); + CountDownLatch successLatch = new CountDownLatch(10); + Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh( + succeedOnAttempt, + refreshCountLatch, + successLatch, + true, + closeShardOnAttempt + ); + // Giving 10ms for some iterations of remote refresh upload + Thread.sleep(TimeUnit.SECONDS.toMillis(2)); + RemoteStoreRefreshListener listener = tuple.v1(); + assertFalse("remote store should not in sync", listener.isRemoteSegmentStoreInSync()); + assertFalse(listener.getRetryScheduledStatus()); + } + private void assertNoLagAndTotalUploadsFailed(RemoteSegmentTransferTracker segmentTracker, long totalUploadsFailed) throws Exception { assertBusy(() -> { assertEquals(0, segmentTracker.getBytesLag()); @@ -548,6 +568,49 @@ private Tuple mockIn return mockIndexShardWithRetryAndScheduleRefresh(succeedOnAttempt, refreshCountLatch, successLatch, 1, noOpLatch); } + private Tuple mockIndexShardWithRetryAndScheduleRefresh( + int totalAttempt, + CountDownLatch refreshCountLatch, + CountDownLatch successLatch, + int checkpointPublishSucceedOnAttempt, + CountDownLatch reachedCheckpointPublishLatch, + boolean mockPrimaryTerm, + boolean testUploadTimeout + ) throws IOException { + return mockIndexShardWithRetryAndScheduleRefresh( + totalAttempt, + refreshCountLatch, + successLatch, + checkpointPublishSucceedOnAttempt, + reachedCheckpointPublishLatch, + mockPrimaryTerm, + testUploadTimeout, + false, + 0 + ); + } + + private Tuple mockIndexShardWithRetryAndScheduleRefresh( + int succeedOnAttempt, + CountDownLatch refreshCountLatch, + CountDownLatch successLatch, + boolean closedShard, + int closeShardAfterAttempt + ) throws IOException { + CountDownLatch noOpLatch = new CountDownLatch(0); + return mockIndexShardWithRetryAndScheduleRefresh( + succeedOnAttempt, + refreshCountLatch, + successLatch, + 1, + noOpLatch, + true, + false, + closedShard, + closeShardAfterAttempt + ); + } + private Tuple mockIndexShardWithRetryAndScheduleRefresh( int succeedOnAttempt, CountDownLatch refreshCountLatch, @@ -562,7 +625,9 @@ private Tuple mockIn succeedCheckpointPublishOnAttempt, reachedCheckpointPublishLatch, true, - false + false, + false, + 0 ); } @@ -573,7 +638,9 @@ private Tuple mockIn int succeedCheckpointPublishOnAttempt, CountDownLatch reachedCheckpointPublishLatch, boolean mockPrimaryTerm, - boolean testUploadTimeout + boolean testUploadTimeout, + boolean closeShard, + int closeShardAfterAttempt ) throws IOException { // Create index shard that we will be using to mock different methods in IndexShard for the unit test indexShard = newStartedShard( @@ -601,7 +668,6 @@ private Tuple mockIn IndexShard shard = mock(IndexShard.class); Store store = mock(Store.class); when(shard.store()).thenReturn(store); - when(shard.state()).thenReturn(IndexShardState.STARTED); when(store.directory()).thenReturn(indexShard.store().directory()); // Mock (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()) @@ -663,6 +729,14 @@ private Tuple mockIn return indexShard.getLatestReplicationCheckpoint(); })).when(shard).computeReplicationCheckpoint(any()); + doAnswer((invocationOnMock -> { + if (closeShard && counter.get() == closeShardAfterAttempt) { + logger.info("Closing shard..."); + return IndexShardState.CLOSED; + } + return IndexShardState.STARTED; + })).when(shard).state(); + doAnswer(invocation -> { if (Objects.nonNull(successLatch)) { successLatch.countDown(); From 859906587b3ba220b08c94d0b5387dbe23af0ffe Mon Sep 17 00:00:00 2001 From: Muneer Kolarkunnu <33829651+akolarkunnu@users.noreply.github.com> Date: Thu, 6 Jun 2024 18:07:16 +0530 Subject: [PATCH 56/77] A documentation issue, used CONTRIBUTING.md instead of DEVELOPER_GUIDE.md (#14028) Description: Replaced CONTRIBUTING.md with DEVELOPER_GUIDE.md Resolves #13984 Signed-off-by: Abdul Muneer Kolarkunnu --- gradle/ide.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/ide.gradle b/gradle/ide.gradle index 4c4f3b07836c5..e266d9add172d 100644 --- a/gradle/ide.gradle +++ b/gradle/ide.gradle @@ -28,7 +28,7 @@ allprojects { apply plugin: 'idea' tasks.named('idea').configure { - doFirst { throw new GradleException("Use of the 'idea' task has been deprecated. For details on importing into IntelliJ see CONTRIBUTING.md.") } + doFirst { throw new GradleException("Use of the 'idea' task has been deprecated. For details on importing into IntelliJ see DEVELOPER_GUIDE.md.") } } } From 298aeda5fdffaa4f6a707d7b0248f7871cf83d80 Mon Sep 17 00:00:00 2001 From: Lantao Jin Date: Thu, 6 Jun 2024 21:06:54 +0800 Subject: [PATCH 57/77] Bump HdrHistogram to 2.2.2 and move the dependency version to version.properties (#13986) Signed-off-by: LantaoJin --- CHANGELOG.md | 2 +- buildSrc/version.properties | 2 ++ server/build.gradle | 6 +++--- server/licenses/HdrHistogram-2.2.1.jar.sha1 | 1 - server/licenses/HdrHistogram-2.2.2.jar.sha1 | 1 + 5 files changed, 7 insertions(+), 5 deletions(-) delete mode 100644 server/licenses/HdrHistogram-2.2.1.jar.sha1 create mode 100644 server/licenses/HdrHistogram-2.2.2.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 759cf53adf8b8..b122d062fe8c5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,7 +21,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) - Bump `org.apache.commons:commons-text` from 1.11.0 to 1.12.0 ([#13557](https://github.com/opensearch-project/OpenSearch/pull/13557)) -- Bump `org.hdrhistogram:HdrHistogram` from 2.1.12 to 2.2.1 ([#13556](https://github.com/opensearch-project/OpenSearch/pull/13556)) +- Bump `org.hdrhistogram:HdrHistogram` from 2.1.12 to 2.2.2 ([#13556](https://github.com/opensearch-project/OpenSearch/pull/13556), [#13986](https://github.com/opensearch-project/OpenSearch/pull/13986)) - Bump `com.gradle.enterprise` from 3.17.2 to 3.17.4 ([#13641](https://github.com/opensearch-project/OpenSearch/pull/13641), [#13753](https://github.com/opensearch-project/OpenSearch/pull/13753)) - Bump `org.apache.hadoop:hadoop-minicluster` from 3.3.6 to 3.4.0 ([#13642](https://github.com/opensearch-project/OpenSearch/pull/13642)) - Bump `mockito` from 5.11.0 to 5.12.0 ([#13665](https://github.com/opensearch-project/OpenSearch/pull/13665)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 0a36ed5e200f7..3a76cf6e9b7ad 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -23,6 +23,8 @@ guava = 32.1.1-jre protobuf = 3.22.3 jakarta_annotation = 1.3.5 google_http_client = 1.44.1 +tdigest = 3.3 +hdrhistogram = 2.2.2 # when updating the JNA version, also update the version in buildSrc/build.gradle jna = 5.13.0 diff --git a/server/build.gradle b/server/build.gradle index 15301e68fca3d..624e5fe332662 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -96,9 +96,9 @@ dependencies { api "joda-time:joda-time:${versions.joda}" // percentiles aggregation - api 'com.tdunning:t-digest:3.3' - // precentil ranks aggregation - api 'org.hdrhistogram:HdrHistogram:2.2.1' + api "com.tdunning:t-digest:${versions.tdigest}" + // percentile ranks aggregation + api "org.hdrhistogram:HdrHistogram:${versions.hdrhistogram}" // lucene spatial api "org.locationtech.spatial4j:spatial4j:${versions.spatial4j}", optional diff --git a/server/licenses/HdrHistogram-2.2.1.jar.sha1 b/server/licenses/HdrHistogram-2.2.1.jar.sha1 deleted file mode 100644 index 68225950d4744..0000000000000 --- a/server/licenses/HdrHistogram-2.2.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0eb1feb351f64176c377772a30174e582c0274d5 \ No newline at end of file diff --git a/server/licenses/HdrHistogram-2.2.2.jar.sha1 b/server/licenses/HdrHistogram-2.2.2.jar.sha1 new file mode 100644 index 0000000000000..2c895841bce81 --- /dev/null +++ b/server/licenses/HdrHistogram-2.2.2.jar.sha1 @@ -0,0 +1 @@ +7959933ebcc0f05b2eaa5af0a0c8689fa257b15c From b9ca5a8e24673ed38cab736ffbd57479de241553 Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Thu, 6 Jun 2024 19:20:10 +0530 Subject: [PATCH 58/77] [Remote Store] Add support to disable flush based on translog reader count (#14027) Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- CHANGELOG.md | 1 + .../opensearch/remotestore/RemoteStoreIT.java | 26 ++++++++++++++++++- .../index/translog/RemoteFsTranslog.java | 6 ++++- .../indices/RemoteStoreSettings.java | 8 +++++- ...RemoteStoreSettingsDynamicUpdateTests.java | 11 ++++++++ 5 files changed, 49 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b122d062fe8c5..098405f8f1d44 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,6 +16,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Remote State] Add async remote state deletion task running on an interval, configurable by a setting ([#13131](https://github.com/opensearch-project/OpenSearch/pull/13131)) - Allow setting query parameters on requests ([#13776](https://github.com/opensearch-project/OpenSearch/issues/13776)) - Add remote routing table for remote state publication with experimental feature flag ([#13304](https://github.com/opensearch-project/OpenSearch/pull/13304)) +- [Remote Store] Add support to disable flush based on translog reader count ([#14027](https://github.com/opensearch-project/OpenSearch/pull/14027)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java index 7721b18a4fe6b..96d6338e5913b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java @@ -852,7 +852,9 @@ public void testFlushOnTooManyRemoteTranslogFiles() throws Exception { ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); updateSettingsRequest.persistentSettings( - Settings.builder().put(RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS.getKey(), "100") + Settings.builder() + .put(RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS.getKey(), "100") + .put(CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "0ms") ); assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); @@ -883,5 +885,27 @@ public void testFlushOnTooManyRemoteTranslogFiles() throws Exception { assertEquals(totalFiles, 1L); } }, 30, TimeUnit.SECONDS); + + // Disabling max translog readers + assertAcked( + internalCluster().client() + .admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS.getKey(), "-1")) + .get() + ); + + // Indexing 500 more docs + for (int i = 0; i < 500; i++) { + indexBulk(INDEX_NAME, 1); + } + + // No flush is triggered since max_translog_readers is set to -1 + // Total tlog files would be incremented by 500 + try (Stream files = Files.list(translogLocation)) { + long totalFiles = files.filter(f -> f.getFileName().toString().endsWith(Translog.TRANSLOG_FILE_SUFFIX)).count(); + assertEquals(totalFiles, 501L); + } } } 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 67549c86b7dd2..f29b6fba6537f 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -706,6 +706,10 @@ int availablePermits() { */ @Override protected boolean shouldFlush() { - return readers.size() >= translogTransferManager.getMaxRemoteTranslogReadersSettings(); + int maxRemoteTlogReaders = translogTransferManager.getMaxRemoteTranslogReadersSettings(); + if (maxRemoteTlogReaders == -1) { + return false; + } + return readers.size() >= maxRemoteTlogReaders; } } diff --git a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java index 074186f64a75d..8cb482c8d8681 100644 --- a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java +++ b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java @@ -25,6 +25,7 @@ */ @PublicApi(since = "2.14.0") public class RemoteStoreSettings { + private static final int MIN_CLUSTER_REMOTE_MAX_TRANSLOG_READERS = 100; /** * Used to specify the default translog buffer interval for remote store backed indexes. @@ -112,7 +113,12 @@ public class RemoteStoreSettings { public static final Setting CLUSTER_REMOTE_MAX_TRANSLOG_READERS = Setting.intSetting( "cluster.remote_store.translog.max_readers", 1000, - 100, + -1, + v -> { + if (v != -1 && v < MIN_CLUSTER_REMOTE_MAX_TRANSLOG_READERS) { + throw new IllegalArgumentException("Cannot set value lower than " + MIN_CLUSTER_REMOTE_MAX_TRANSLOG_READERS); + } + }, Property.Dynamic, Property.NodeScope ); diff --git a/server/src/test/java/org/opensearch/indices/RemoteStoreSettingsDynamicUpdateTests.java b/server/src/test/java/org/opensearch/indices/RemoteStoreSettingsDynamicUpdateTests.java index f89fd3df6e340..cc9096ee41315 100644 --- a/server/src/test/java/org/opensearch/indices/RemoteStoreSettingsDynamicUpdateTests.java +++ b/server/src/test/java/org/opensearch/indices/RemoteStoreSettingsDynamicUpdateTests.java @@ -116,4 +116,15 @@ public void testMaxRemoteReferencedTranslogFiles() { ); assertEquals(500, remoteStoreSettings.getMaxRemoteTranslogReaders()); } + + public void testDisableMaxRemoteReferencedTranslogFiles() { + // Test default value + assertEquals(1000, remoteStoreSettings.getMaxRemoteTranslogReaders()); + + // Test override with valid value + clusterSettings.applySettings( + Settings.builder().put(RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS.getKey(), "-1").build() + ); + assertEquals(-1, remoteStoreSettings.getMaxRemoteTranslogReaders()); + } } From e9b6a8d652d51e26fe10159d0ebf4c8b429b5c28 Mon Sep 17 00:00:00 2001 From: Navneet Verma Date: Thu, 6 Jun 2024 09:02:56 -0700 Subject: [PATCH 59/77] Add capability to disable source recovery_source for an index (#13590) Signed-off-by: Navneet Verma --- CHANGELOG.md | 1 + .../recovery/FullRollingRestartIT.java | 143 ++++++++++++++++++ .../index/mapper/SourceFieldMapper.java | 117 ++++++++++++-- .../index/mapper/SourceFieldMapperTests.java | 137 ++++++++++++++++- 4 files changed, 385 insertions(+), 13 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 098405f8f1d44..597e0ae31b054 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -15,6 +15,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add getMetadataFields to MapperService ([#13819](https://github.com/opensearch-project/OpenSearch/pull/13819)) - [Remote State] Add async remote state deletion task running on an interval, configurable by a setting ([#13131](https://github.com/opensearch-project/OpenSearch/pull/13131)) - Allow setting query parameters on requests ([#13776](https://github.com/opensearch-project/OpenSearch/issues/13776)) +- Add capability to disable source recovery_source for an index ([#13590](https://github.com/opensearch-project/OpenSearch/pull/13590)) - Add remote routing table for remote state publication with experimental feature flag ([#13304](https://github.com/opensearch-project/OpenSearch/pull/13304)) - [Remote Store] Add support to disable flush based on translog reader count ([#14027](https://github.com/opensearch-project/OpenSearch/pull/14027)) diff --git a/server/src/internalClusterTest/java/org/opensearch/recovery/FullRollingRestartIT.java b/server/src/internalClusterTest/java/org/opensearch/recovery/FullRollingRestartIT.java index 0752ab7c9d0f1..d9e3cec426edf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/recovery/FullRollingRestartIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/recovery/FullRollingRestartIT.java @@ -36,6 +36,7 @@ import org.opensearch.action.admin.cluster.health.ClusterHealthRequestBuilder; import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; +import org.opensearch.action.admin.indices.create.CreateIndexResponse; import org.opensearch.action.admin.indices.recovery.RecoveryResponse; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -45,6 +46,8 @@ import org.opensearch.common.collect.MapBuilder; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.test.OpenSearchIntegTestCase.ClusterScope; import org.opensearch.test.OpenSearchIntegTestCase.Scope; @@ -253,4 +256,144 @@ public void testNoRebalanceOnRollingRestart() throws Exception { ); } } + + public void testFullRollingRestart_withNoRecoveryPayloadAndSource() throws Exception { + internalCluster().startNode(); + XContentBuilder builder = XContentFactory.jsonBuilder() + .startObject() + .startObject("_source") + .field("enabled") + .value(false) + .field("recovery_source_enabled") + .value(false) + .endObject() + .endObject(); + CreateIndexResponse response = prepareCreate("test").setMapping(builder).get(); + logger.info("Create index response is : {}", response); + + final String healthTimeout = "1m"; + + for (int i = 0; i < 1000; i++) { + client().prepareIndex("test") + .setId(Long.toString(i)) + .setSource(MapBuilder.newMapBuilder().put("test", "value" + i).map()) + .execute() + .actionGet(); + } + + for (int i = 1000; i < 2000; i++) { + client().prepareIndex("test") + .setId(Long.toString(i)) + .setSource(MapBuilder.newMapBuilder().put("test", "value" + i).map()) + .execute() + .actionGet(); + } + // ensuring all docs are committed to file system + flush(); + + logger.info("--> now start adding nodes"); + internalCluster().startNode(); + internalCluster().startNode(); + + // make sure the cluster state is green, and all has been recovered + assertTimeout( + client().admin() + .cluster() + .prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setTimeout(healthTimeout) + .setWaitForGreenStatus() + .setWaitForNoRelocatingShards(true) + .setWaitForNodes("3") + ); + + logger.info("--> add two more nodes"); + internalCluster().startNode(); + internalCluster().startNode(); + + // make sure the cluster state is green, and all has been recovered + assertTimeout( + client().admin() + .cluster() + .prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setTimeout(healthTimeout) + .setWaitForGreenStatus() + .setWaitForNoRelocatingShards(true) + .setWaitForNodes("5") + ); + + logger.info("--> refreshing and checking data"); + refreshAndWaitForReplication(); + for (int i = 0; i < 10; i++) { + assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(), 2000L); + } + + // now start shutting nodes down + internalCluster().stopRandomDataNode(); + // make sure the cluster state is green, and all has been recovered + assertTimeout( + client().admin() + .cluster() + .prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setTimeout(healthTimeout) + .setWaitForGreenStatus() + .setWaitForNoRelocatingShards(true) + .setWaitForNodes("4") + ); + + internalCluster().stopRandomDataNode(); + // make sure the cluster state is green, and all has been recovered + assertTimeout( + client().admin() + .cluster() + .prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setTimeout(healthTimeout) + .setWaitForGreenStatus() + .setWaitForNoRelocatingShards(true) + .setWaitForNodes("3") + ); + + logger.info("--> stopped two nodes, verifying data"); + refreshAndWaitForReplication(); + for (int i = 0; i < 10; i++) { + assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(), 2000L); + } + + // closing the 3rd node + internalCluster().stopRandomDataNode(); + // make sure the cluster state is green, and all has been recovered + assertTimeout( + client().admin() + .cluster() + .prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setTimeout(healthTimeout) + .setWaitForGreenStatus() + .setWaitForNoRelocatingShards(true) + .setWaitForNodes("2") + ); + + internalCluster().stopRandomDataNode(); + + // make sure the cluster state is yellow, and all has been recovered + assertTimeout( + client().admin() + .cluster() + .prepareHealth() + .setWaitForEvents(Priority.LANGUID) + .setTimeout(healthTimeout) + .setWaitForYellowStatus() + .setWaitForNoRelocatingShards(true) + .setWaitForNodes("1") + ); + + logger.info("--> one node left, verifying data"); + refreshAndWaitForReplication(); + for (int i = 0; i < 10; i++) { + assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(), 2000L); + } + } } diff --git a/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java index a2d769d486a0f..54f1528c04bf5 100644 --- a/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java @@ -74,6 +74,7 @@ public class SourceFieldMapper extends MetadataFieldMapper { public static final String CONTENT_TYPE = "_source"; private final Function, Map> filter; + private final Function, Map> recoverySourceFilter; /** * Default parameters for source fields @@ -119,13 +120,64 @@ public static class Builder extends MetadataFieldMapper.Builder { Collections.emptyList() ); + /** + * A mapping parameter which define whether the recovery_source should be added or not. Default value is true. + *

+ * Recovery source gets added if source is disabled or there are filters that are applied on _source using + * {@link #includes}/{@link #excludes}, which has the possibility to change the original document provided by + * customer. Recovery source is not a permanent field and gets removed during merges. Refer this merge + * policy: org.opensearch.index.engine.RecoverySourcePruneMergePolicy + *

+ * The main reason for adding the _recovery_source was to ensure Peer to Peer recovery if segments + * are not flushed to the disk. If you are disabling the recovery source, then ensure that you are calling + * flush operation of Opensearch periodically to ensure that segments are flushed to the disk and if required + * Peer to Peer recovery can happen using segment files rather than replaying traffic by querying Lucene + * snapshot. + * + *

+ * This is an expert mapping parameter. + * + */ + private final Parameter recoverySourceEnabled = Parameter.boolParam( + "recovery_source_enabled", + false, + m -> toType(m).recoverySourceEnabled, + Defaults.ENABLED + ); + + /** + * Provides capability to add specific fields in the recovery_source. + *

+ * Refer {@link #recoverySourceEnabled} for more details + * This is an expert parameter. + */ + private final Parameter> recoverySourceIncludes = Parameter.stringArrayParam( + "recovery_source_includes", + false, + m -> Arrays.asList(toType(m).recoverySourceIncludes), + Collections.emptyList() + ); + + /** + * Provides capability to remove specific fields in the recovery_source. + * + * Refer {@link #recoverySourceEnabled} for more details + * This is an expert parameter. + */ + private final Parameter> recoverySourceExcludes = Parameter.stringArrayParam( + "recovery_source_excludes", + false, + m -> Arrays.asList(toType(m).recoverySourceExcludes), + Collections.emptyList() + ); + public Builder() { super(Defaults.NAME); } @Override protected List> getParameters() { - return Arrays.asList(enabled, includes, excludes); + return Arrays.asList(enabled, includes, excludes, recoverySourceEnabled, recoverySourceIncludes, recoverySourceExcludes); } @Override @@ -133,7 +185,10 @@ public SourceFieldMapper build(BuilderContext context) { return new SourceFieldMapper( enabled.getValue(), includes.getValue().toArray(new String[0]), - excludes.getValue().toArray(new String[0]) + excludes.getValue().toArray(new String[0]), + recoverySourceEnabled.getValue(), + recoverySourceIncludes.getValue().toArray(new String[0]), + recoverySourceExcludes.getValue().toArray(new String[0]) ); } } @@ -173,17 +228,27 @@ public Query termQuery(Object value, QueryShardContext context) { } private final boolean enabled; + private final boolean recoverySourceEnabled; /** indicates whether the source will always exist and be complete, for use by features like the update API */ private final boolean complete; private final String[] includes; private final String[] excludes; + private final String[] recoverySourceIncludes; + private final String[] recoverySourceExcludes; private SourceFieldMapper() { - this(Defaults.ENABLED, Strings.EMPTY_ARRAY, Strings.EMPTY_ARRAY); + this(Defaults.ENABLED, Strings.EMPTY_ARRAY, Strings.EMPTY_ARRAY, Defaults.ENABLED, Strings.EMPTY_ARRAY, Strings.EMPTY_ARRAY); } - private SourceFieldMapper(boolean enabled, String[] includes, String[] excludes) { + private SourceFieldMapper( + boolean enabled, + String[] includes, + String[] excludes, + boolean recoverySourceEnabled, + String[] recoverySourceIncludes, + String[] recoverySourceExcludes + ) { super(new SourceFieldType(enabled)); this.enabled = enabled; this.includes = includes; @@ -191,6 +256,16 @@ private SourceFieldMapper(boolean enabled, String[] includes, String[] excludes) final boolean filtered = CollectionUtils.isEmpty(includes) == false || CollectionUtils.isEmpty(excludes) == false; this.filter = enabled && filtered ? XContentMapValues.filter(includes, excludes) : null; this.complete = enabled && CollectionUtils.isEmpty(includes) && CollectionUtils.isEmpty(excludes); + + // Set parameters for recovery source + this.recoverySourceEnabled = recoverySourceEnabled; + this.recoverySourceIncludes = recoverySourceIncludes; + this.recoverySourceExcludes = recoverySourceExcludes; + final boolean recoverySourcefiltered = CollectionUtils.isEmpty(recoverySourceIncludes) == false + || CollectionUtils.isEmpty(recoverySourceExcludes) == false; + this.recoverySourceFilter = this.recoverySourceEnabled && recoverySourcefiltered + ? XContentMapValues.filter(recoverySourceIncludes, recoverySourceExcludes) + : null; } public boolean enabled() { @@ -212,22 +287,40 @@ public void preParse(ParseContext context) throws IOException { context.doc().add(new StoredField(fieldType().name(), ref.bytes, ref.offset, ref.length)); } - if (originalSource != null && adaptedSource != originalSource) { - // if we omitted source or modified it we add the _recovery_source to ensure we have it for ops based recovery - BytesRef ref = originalSource.toBytesRef(); - context.doc().add(new StoredField(RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); - context.doc().add(new NumericDocValuesField(RECOVERY_SOURCE_NAME, 1)); + if (recoverySourceEnabled) { + if (originalSource != null && adaptedSource != originalSource) { + final BytesReference adaptedRecoverySource = applyFilters( + originalSource, + contentType, + recoverySourceEnabled, + recoverySourceFilter + ); + // if we omitted source or modified it we add the _recovery_source to ensure we have it for ops based recovery + BytesRef ref = adaptedRecoverySource.toBytesRef(); + context.doc().add(new StoredField(RECOVERY_SOURCE_NAME, ref.bytes, ref.offset, ref.length)); + context.doc().add(new NumericDocValuesField(RECOVERY_SOURCE_NAME, 1)); + } } } @Nullable public BytesReference applyFilters(@Nullable BytesReference originalSource, @Nullable MediaType contentType) throws IOException { - if (enabled && originalSource != null) { + return applyFilters(originalSource, contentType, enabled, filter); + } + + @Nullable + private BytesReference applyFilters( + @Nullable BytesReference originalSource, + @Nullable MediaType contentType, + boolean isProvidedSourceEnabled, + @Nullable final Function, Map> filters + ) throws IOException { + if (isProvidedSourceEnabled && originalSource != null) { // Percolate and tv APIs may not set the source and that is ok, because these APIs will not index any data - if (filter != null) { + if (filters != null) { // we don't update the context source if we filter, we want to keep it as is... Tuple> mapTuple = XContentHelper.convertToMap(originalSource, true, contentType); - Map filteredSource = filter.apply(mapTuple.v2()); + Map filteredSource = filters.apply(mapTuple.v2()); BytesStreamOutput bStream = new BytesStreamOutput(); MediaType actualContentType = mapTuple.v1(); XContentBuilder builder = MediaTypeRegistry.contentBuilder(actualContentType, bStream).map(filteredSource); diff --git a/server/src/test/java/org/opensearch/index/mapper/SourceFieldMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/SourceFieldMapperTests.java index 83d42fd423f08..4f3a4530b5475 100644 --- a/server/src/test/java/org/opensearch/index/mapper/SourceFieldMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/SourceFieldMapperTests.java @@ -90,7 +90,8 @@ public void testNoFormat() throws Exception { XContentType.SMILE ) ); - + final IndexableField recoverySourceIndexableField = doc.rootDoc().getField("_recovery_source"); + assertNull(recoverySourceIndexableField); assertThat(MediaTypeRegistry.xContentType(doc.source()), equalTo(XContentType.SMILE)); } @@ -128,6 +129,52 @@ public void testIncludes() throws Exception { ) ); + IndexableField sourceField = doc.rootDoc().getField("_source"); + Map sourceAsMap; + try (XContentParser parser = createParser(JsonXContent.jsonXContent, new BytesArray(sourceField.binaryValue()))) { + sourceAsMap = parser.map(); + } + final IndexableField recoverySourceIndexableField = doc.rootDoc().getField("_recovery_source"); + assertNotNull(recoverySourceIndexableField); + assertThat(sourceAsMap.containsKey("path1"), equalTo(true)); + assertThat(sourceAsMap.containsKey("path2"), equalTo(false)); + } + + public void testIncludesForRecoverySource() throws Exception { + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("_source") + .array("includes", new String[] { "path1*" }) + .array("recovery_source_includes", new String[] { "path2*" }) + .endObject() + .endObject() + .endObject() + .toString(); + + DocumentMapper documentMapper = createIndex("test").mapperService() + .documentMapperParser() + .parse("type", new CompressedXContent(mapping)); + + ParsedDocument doc = documentMapper.parse( + new SourceToParse( + "test", + "1", + BytesReference.bytes( + XContentFactory.jsonBuilder() + .startObject() + .startObject("path1") + .field("field1", "value1") + .endObject() + .startObject("path2") + .field("field2", "value2") + .endObject() + .endObject() + ), + MediaTypeRegistry.JSON + ) + ); + IndexableField sourceField = doc.rootDoc().getField("_source"); Map sourceAsMap; try (XContentParser parser = createParser(JsonXContent.jsonXContent, new BytesArray(sourceField.binaryValue()))) { @@ -135,6 +182,39 @@ public void testIncludes() throws Exception { } assertThat(sourceAsMap.containsKey("path1"), equalTo(true)); assertThat(sourceAsMap.containsKey("path2"), equalTo(false)); + + final IndexableField recoverySourceIndexableField = doc.rootDoc().getField("_recovery_source"); + assertNotNull(recoverySourceIndexableField); + Map recoverySourceAsMap; + try (XContentParser parser = createParser(JsonXContent.jsonXContent, new BytesArray(recoverySourceIndexableField.binaryValue()))) { + recoverySourceAsMap = parser.map(); + } + + assertThat(recoverySourceAsMap.containsKey("path1"), equalTo(false)); + assertThat(recoverySourceAsMap.containsKey("path2"), equalTo(true)); + } + + public void testNoRecoverySourceAndNoSource_whenBothAreDisabled() throws Exception { + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("_source") + .field("enabled", "false") + .field("recovery_source_enabled", "false") + .endObject() + .endObject() + .endObject() + .toString(); + + DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser(); + DocumentMapper documentMapper = parser.parse("type", new CompressedXContent(mapping)); + BytesReference source = BytesReference.bytes(XContentFactory.jsonBuilder().startObject().field("field", "value").endObject()); + ParsedDocument doc = documentMapper.parse(new SourceToParse("test", "1", source, MediaTypeRegistry.JSON)); + + final IndexableField sourceIndexableField = doc.rootDoc().getField("_source"); + final IndexableField recoverySourceIndexableField = doc.rootDoc().getField("_recovery_source"); + assertNull(recoverySourceIndexableField); + assertNull(sourceIndexableField); } public void testExcludes() throws Exception { @@ -171,6 +251,52 @@ public void testExcludes() throws Exception { ) ); + IndexableField sourceField = doc.rootDoc().getField("_source"); + Map sourceAsMap; + try (XContentParser parser = createParser(JsonXContent.jsonXContent, new BytesArray(sourceField.binaryValue()))) { + sourceAsMap = parser.map(); + } + final IndexableField recoverySourceIndexableField = doc.rootDoc().getField("_recovery_source"); + assertNotNull(recoverySourceIndexableField); + assertThat(sourceAsMap.containsKey("path1"), equalTo(false)); + assertThat(sourceAsMap.containsKey("path2"), equalTo(true)); + } + + public void testExcludesForRecoverySource() throws Exception { + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("_source") + .array("excludes", "path1*") + .array("recovery_source_excludes", "path2*") + .endObject() + .endObject() + .endObject() + .toString(); + + DocumentMapper documentMapper = createIndex("test").mapperService() + .documentMapperParser() + .parse("type", new CompressedXContent(mapping)); + + ParsedDocument doc = documentMapper.parse( + new SourceToParse( + "test", + "1", + BytesReference.bytes( + XContentFactory.jsonBuilder() + .startObject() + .startObject("path1") + .field("field1", "value1") + .endObject() + .startObject("path2") + .field("field2", "value2") + .endObject() + .endObject() + ), + MediaTypeRegistry.JSON + ) + ); + IndexableField sourceField = doc.rootDoc().getField("_source"); Map sourceAsMap; try (XContentParser parser = createParser(JsonXContent.jsonXContent, new BytesArray(sourceField.binaryValue()))) { @@ -178,6 +304,15 @@ public void testExcludes() throws Exception { } assertThat(sourceAsMap.containsKey("path1"), equalTo(false)); assertThat(sourceAsMap.containsKey("path2"), equalTo(true)); + + final IndexableField recoverySourceIndexableField = doc.rootDoc().getField("_recovery_source"); + assertNotNull(recoverySourceIndexableField); + Map recoverySourceAsMap; + try (XContentParser parser = createParser(JsonXContent.jsonXContent, new BytesArray(recoverySourceIndexableField.binaryValue()))) { + recoverySourceAsMap = parser.map(); + } + assertThat(recoverySourceAsMap.containsKey("path1"), equalTo(true)); + assertThat(recoverySourceAsMap.containsKey("path2"), equalTo(false)); } public void testEnabledNotUpdateable() throws Exception { From c89a17cecaf8348d936cd42d3000c1a1fa7cf120 Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Thu, 6 Jun 2024 22:22:28 +0530 Subject: [PATCH 60/77] Update checkpoint from remote nodes replicas (#13888) Signed-off-by: Gaurav Bafna --- .../MigrationBaseTestCase.java | 3 +- .../RemotePrimaryRelocationIT.java | 40 +++------- .../RemoteReplicaRecoveryIT.java | 79 ++++++------------- .../opensearch/index/shard/IndexShard.java | 1 - .../SegmentReplicationTargetService.java | 2 +- .../replication/common/ReplicationTarget.java | 4 +- 6 files changed, 41 insertions(+), 88 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java index 0493bcf800c97..b65f6f056aae6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java @@ -186,10 +186,11 @@ private Thread getIndexingThread() { indexSingleDoc(indexName); long currentDocCount = indexedDocs.incrementAndGet(); if (currentDocCount > 0 && currentDocCount % refreshFrequency == 0) { - logger.info("--> [iteration {}] flushing index", currentDocCount); if (rarely()) { + logger.info("--> [iteration {}] flushing index", currentDocCount); client().admin().indices().prepareFlush(indexName).get(); } else { + logger.info("--> [iteration {}] refreshing index", currentDocCount); client().admin().indices().prepareRefresh(indexName).get(); } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java index 293691ace2edd..cea653c0ead4b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java @@ -8,14 +8,11 @@ package org.opensearch.remotemigration; -import org.opensearch.action.DocWriteResponse; import org.opensearch.action.admin.cluster.health.ClusterHealthRequest; 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.client.Requests; import org.opensearch.cluster.routing.allocation.command.MoveAllocationCommand; @@ -66,8 +63,8 @@ public void testRemotePrimaryRelocation() throws Exception { AtomicInteger numAutoGenDocs = new AtomicInteger(); final AtomicBoolean finished = new AtomicBoolean(false); - Thread indexingThread = getIndexingThread(finished, numAutoGenDocs); - + AsyncIndexingService asyncIndexingService = new AsyncIndexingService("test"); + asyncIndexingService.startIndexing(); refresh("test"); // add remote node in mixed mode cluster @@ -141,17 +138,19 @@ public void testRemotePrimaryRelocation() throws Exception { logger.info("--> relocation from remote to remote complete"); finished.set(true); - indexingThread.join(); + asyncIndexingService.stopIndexing(); refresh("test"); - OpenSearchAssertions.assertHitCount(client().prepareSearch("test").setTrackTotalHits(true).get(), numAutoGenDocs.get()); + OpenSearchAssertions.assertHitCount( + client().prepareSearch("test").setTrackTotalHits(true).get(), + asyncIndexingService.getIndexedDocs() + ); OpenSearchAssertions.assertHitCount( client().prepareSearch("test") .setTrackTotalHits(true)// extra paranoia ;) .setQuery(QueryBuilders.termQuery("auto", true)) .get(), - numAutoGenDocs.get() + asyncIndexingService.getIndexedDocs() ); - } public void testMixedModeRelocation_RemoteSeedingFail() throws Exception { @@ -165,9 +164,8 @@ public void testMixedModeRelocation_RemoteSeedingFail() throws Exception { 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); + AsyncIndexingService asyncIndexingService = new AsyncIndexingService("test"); + asyncIndexingService.startIndexing(); refresh("test"); @@ -209,27 +207,11 @@ public void testMixedModeRelocation_RemoteSeedingFail() throws Exception { assertEquals(actionGet.getRelocatingShards(), 0); assertEquals(docRepNode, primaryNodeName("test")); - finished.set(true); - indexingThread.join(); + asyncIndexingService.stopIndexing(); client().admin() .cluster() .prepareUpdateSettings() .setTransientSettings(Settings.builder().put(RecoverySettings.INDICES_INTERNAL_REMOTE_UPLOAD_TIMEOUT.getKey(), (String) null)) .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/remotemigration/RemoteReplicaRecoveryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteReplicaRecoveryIT.java index 196ecb991bbc0..7270341202990 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteReplicaRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteReplicaRecoveryIT.java @@ -8,32 +8,27 @@ 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.settings.ClusterUpdateSettingsRequest; +import org.opensearch.action.admin.indices.replication.SegmentReplicationStatsResponse; import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest; -import org.opensearch.action.delete.DeleteResponse; -import org.opensearch.action.index.IndexResponse; import org.opensearch.cluster.metadata.IndexMetadata; 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.SegmentReplicationPerGroupStats; import org.opensearch.index.query.QueryBuilders; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.hamcrest.OpenSearchAssertions; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.TimeUnit; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; 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, autoManageMasterNodes = false) - public class RemoteReplicaRecoveryIT extends MigrationBaseTestCase { protected int maximumNumberOfShards() { @@ -63,10 +58,8 @@ public void testReplicaRecovery() throws Exception { client().admin().indices().prepareCreate("test").setSettings(indexSettings()).setMapping("field", "type=text").get(); String replicaNode = internalCluster().startNode(); ensureGreen("test"); - - AtomicInteger numAutoGenDocs = new AtomicInteger(); - final AtomicBoolean finished = new AtomicBoolean(false); - Thread indexingThread = getThread(finished, numAutoGenDocs); + AsyncIndexingService asyncIndexingService = new AsyncIndexingService("test"); + asyncIndexingService.startIndexing(); refresh("test"); @@ -78,12 +71,10 @@ public void testReplicaRecovery() throws Exception { updateSettingsRequest.persistentSettings(Settings.builder().put(MIGRATION_DIRECTION_SETTING.getKey(), "remote_store")); assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); - String remoteNode2 = internalCluster().startNode(); + internalCluster().startNode(); internalCluster().validateClusterFormed(); // identify the primary - - Thread.sleep(RandomNumbers.randomIntBetween(random(), 0, 2000)); logger.info("--> relocating primary from {} to {} ", primaryNode, remoteNode); client().admin() .cluster() @@ -102,7 +93,6 @@ public void testReplicaRecovery() throws Exception { assertEquals(0, clusterHealthResponse.getRelocatingShards()); logger.info("--> relocation of primary from docrep to remote complete"); - Thread.sleep(RandomNumbers.randomIntBetween(random(), 0, 2000)); logger.info("--> getting up the new replicas now to doc rep node as well as remote node "); // Increase replica count to 3 @@ -129,52 +119,33 @@ public void testReplicaRecovery() throws Exception { logger.info("--> replica is up now on another docrep now as well as remote node"); assertEquals(0, clusterHealthResponse.getRelocatingShards()); + asyncIndexingService.stopIndexing(); + refresh("test"); - Thread.sleep(RandomNumbers.randomIntBetween(random(), 0, 2000)); + // segrep lag should be zero + assertBusy(() -> { + SegmentReplicationStatsResponse segmentReplicationStatsResponse = dataNodeClient().admin() + .indices() + .prepareSegmentReplicationStats("test") + .setDetailed(true) + .execute() + .actionGet(); + SegmentReplicationPerGroupStats perGroupStats = segmentReplicationStatsResponse.getReplicationStats().get("test").get(0); + assertEquals(segmentReplicationStatsResponse.getReplicationStats().size(), 1); + perGroupStats.getReplicaStats().stream().forEach(e -> assertEquals(e.getCurrentReplicationLagMillis(), 0)); + }, 20, TimeUnit.SECONDS); - // Stop replicas on docrep now. - // ToDo : Remove once we have dual replication enabled - client().admin() - .indices() - .updateSettings( - new UpdateSettingsRequest("test").settings( - Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put("index.routing.allocation.exclude._name", primaryNode + "," + replicaNode) - .build() - ) - ) - .get(); - - finished.set(true); - indexingThread.join(); - refresh("test"); - OpenSearchAssertions.assertHitCount(client().prepareSearch("test").setTrackTotalHits(true).get(), numAutoGenDocs.get()); + OpenSearchAssertions.assertHitCount( + client().prepareSearch("test").setTrackTotalHits(true).get(), + asyncIndexingService.getIndexedDocs() + ); OpenSearchAssertions.assertHitCount( client().prepareSearch("test") .setTrackTotalHits(true)// extra paranoia ;) .setQuery(QueryBuilders.termQuery("auto", true)) - // .setPreference("_prefer_nodes:" + (remoteNode+ "," + remoteNode2)) .get(), - numAutoGenDocs.get() + asyncIndexingService.getIndexedDocs() ); } - - private Thread getThread(AtomicBoolean finished, AtomicInteger numAutoGenDocs) { - Thread indexingThread = new Thread(() -> { - while (finished.get() == false && numAutoGenDocs.get() < 100) { - 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(); - logger.info("Indexed {} docs here", numAutoGenDocs.get()); - } - }); - indexingThread.start(); - return indexingThread; - } - } 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 3517579856d43..49cb710c915fc 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -523,7 +523,6 @@ public boolean shouldSeedRemoteStore() { public Function isShardOnRemoteEnabledNode = nodeId -> { DiscoveryNode node = discoveryNodes.get(nodeId); if (node != null) { - logger.trace("Node {} has remote_enabled as {}", nodeId, node.isRemoteStoreNode()); return node.isRemoteStoreNode(); } return false; 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 fbd7ab7cea346..f6ed113019897 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -384,7 +384,7 @@ private void logReplicationFailure(SegmentReplicationState state, ReplicationFai protected void updateVisibleCheckpoint(long replicationId, IndexShard replicaShard) { // Update replication checkpoint on source via transport call only supported for remote store integration. For node- // node communication, checkpoint update is piggy-backed to GET_SEGMENT_FILES transport call - if (replicaShard.indexSettings().isRemoteStoreEnabled() == false) { + if (replicaShard.indexSettings().isAssignedOnRemoteNode() == false) { return; } ShardRouting primaryShard = clusterService.state().routingTable().shardRoutingTable(replicaShard.shardId()).primaryShard(); diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java index aac59df4f6573..76401eaabbf39 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java @@ -91,7 +91,7 @@ public ReplicationTarget(String name, IndexShard indexShard, ReplicationLuceneIn // make sure the store is not released until we are done. this.cancellableThreads = new CancellableThreads(); store.incRef(); - if (indexShard.indexSettings().isRemoteStoreEnabled()) { + if (indexShard.indexSettings().isAssignedOnRemoteNode()) { indexShard.remoteStore().incRef(); } } @@ -284,7 +284,7 @@ protected void closeInternal() { try { store.decRef(); } finally { - if (indexShard.indexSettings().isRemoteStoreEnabled()) { + if (indexShard.indexSettings().isAssignedOnRemoteNode()) { indexShard.remoteStore().decRef(); } } From ecf3c3d8b137a82af199698df00c6013ba4349cd Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Thu, 6 Jun 2024 12:53:17 -0400 Subject: [PATCH 61/77] Revert "COMPAT locale provider will be removed in a future release (#13988)" (#14039) This reverts commit ba0df742037b49dc8c85efcf9ccec98fd3a1d69d. Signed-off-by: Andriy Redko --- CHANGELOG-3.0.md | 1 - .../gradle/OpenSearchTestBasePlugin.java | 2 +- .../tools/launchers/SystemJvmOptions.java | 7 ++++++- gradle/ide.gradle | 2 +- .../opensearch/search/query/SearchQueryIT.java | 18 ++++++++++-------- .../index/mapper/DateFieldMapperTests.java | 2 +- 6 files changed, 19 insertions(+), 13 deletions(-) diff --git a/CHANGELOG-3.0.md b/CHANGELOG-3.0.md index 1cc12f66d52e1..964383078c38d 100644 --- a/CHANGELOG-3.0.md +++ b/CHANGELOG-3.0.md @@ -17,7 +17,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Dependencies ### Changed -- Changed locale provider from COMPAT to CLDR ([13988](https://github.com/opensearch-project/OpenSearch/pull/13988)) - Migrate client transports to Apache HttpClient / Core 5.x ([#4459](https://github.com/opensearch-project/OpenSearch/pull/4459)) - Change http code on create index API with bad input raising NotXContentException from 500 to 400 ([#4773](https://github.com/opensearch-project/OpenSearch/pull/4773)) - Improve summary error message for invalid setting updates ([#4792](https://github.com/opensearch-project/OpenSearch/pull/4792)) diff --git a/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java index d0cb2da9c1dd3..2ea8c2d015ecc 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java @@ -110,7 +110,7 @@ public void execute(Task t) { if (BuildParams.getRuntimeJavaVersion() == JavaVersion.VERSION_1_8) { test.systemProperty("java.locale.providers", "SPI,JRE"); } else { - test.systemProperty("java.locale.providers", "SPI,CLDR"); + test.systemProperty("java.locale.providers", "SPI,COMPAT"); if (test.getJavaVersion().compareTo(JavaVersion.VERSION_17) < 0) { test.jvmArgs("--illegal-access=warn"); } diff --git a/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java b/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java index af7138569972a..726c381db09f6 100644 --- a/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java +++ b/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java @@ -105,8 +105,13 @@ private static String javaLocaleProviders() { SPI setting is used to allow loading custom CalendarDataProvider in jdk8 it has to be loaded from jre/lib/ext, in jdk9+ it is already within ES project and on a classpath + + Due to internationalization enhancements in JDK 9 OpenSearch need to set the provider to COMPAT otherwise time/date + parsing will break in an incompatible way for some date patterns and locales. + //TODO COMPAT will be deprecated in at some point, see please https://bugs.openjdk.java.net/browse/JDK-8232906 + See also: documentation in server/org.opensearch.common.time.IsoCalendarDataProvider */ - return "-Djava.locale.providers=SPI,CLDR"; + return "-Djava.locale.providers=SPI,COMPAT"; } } diff --git a/gradle/ide.gradle b/gradle/ide.gradle index e266d9add172d..ea353f8d92bdd 100644 --- a/gradle/ide.gradle +++ b/gradle/ide.gradle @@ -81,7 +81,7 @@ if (System.getProperty('idea.active') == 'true') { } runConfigurations { defaults(JUnit) { - vmParameters = '-ea -Djava.locale.providers=SPI,CLDR' + vmParameters = '-ea -Djava.locale.providers=SPI,COMPAT' if (BuildParams.runtimeJavaVersion > JavaVersion.VERSION_17) { vmParameters += ' -Djava.security.manager=allow' } diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java index 01ad06757640c..a58db51780826 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java @@ -1914,8 +1914,14 @@ public void testRangeQueryWithTimeZone() throws Exception { * Test range with a custom locale, e.g. "de" in this case. Documents here mention the day of week * as "Mi" for "Mittwoch (Wednesday" and "Do" for "Donnerstag (Thursday)" and the month in the query * as "Dez" for "Dezember (December)". + * Note: this test currently needs the JVM arg `-Djava.locale.providers=SPI,COMPAT` to be set. + * When running with gradle this is done implicitly through the BuildPlugin, but when running from + * an IDE this might need to be set manually in the run configuration. See also CONTRIBUTING.md section + * on "Configuring IDEs And Running Tests". */ public void testRangeQueryWithLocaleMapping() throws Exception { + assert ("SPI,COMPAT".equals(System.getProperty("java.locale.providers"))) : "`-Djava.locale.providers=SPI,COMPAT` needs to be set"; + assertAcked( prepareCreate("test").setMapping( jsonBuilder().startObject() @@ -1932,21 +1938,17 @@ public void testRangeQueryWithLocaleMapping() throws Exception { indexRandom( true, - client().prepareIndex("test").setId("1").setSource("date_field", "Mi., 06 Dez. 2000 02:55:00 -0800"), - client().prepareIndex("test").setId("2").setSource("date_field", "Do., 07 Dez. 2000 02:55:00 -0800") + client().prepareIndex("test").setId("1").setSource("date_field", "Mi, 06 Dez 2000 02:55:00 -0800"), + client().prepareIndex("test").setId("2").setSource("date_field", "Do, 07 Dez 2000 02:55:00 -0800") ); SearchResponse searchResponse = client().prepareSearch("test") - .setQuery( - QueryBuilders.rangeQuery("date_field").gte("Di., 05 Dez. 2000 02:55:00 -0800").lte("Do., 07 Dez. 2000 00:00:00 -0800") - ) + .setQuery(QueryBuilders.rangeQuery("date_field").gte("Di, 05 Dez 2000 02:55:00 -0800").lte("Do, 07 Dez 2000 00:00:00 -0800")) .get(); assertHitCount(searchResponse, 1L); searchResponse = client().prepareSearch("test") - .setQuery( - QueryBuilders.rangeQuery("date_field").gte("Di., 05 Dez. 2000 02:55:00 -0800").lte("Fr., 08 Dez. 2000 00:00:00 -0800") - ) + .setQuery(QueryBuilders.rangeQuery("date_field").gte("Di, 05 Dez 2000 02:55:00 -0800").lte("Fr, 08 Dez 2000 00:00:00 -0800")) .get(); assertHitCount(searchResponse, 2L); } diff --git a/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java index 98bcaa3a1a46b..2aa310ae959d9 100644 --- a/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java @@ -208,7 +208,7 @@ public void testChangeLocale() throws IOException { fieldMapping(b -> b.field("type", "date").field("format", "E, d MMM yyyy HH:mm:ss Z").field("locale", "de")) ); - mapper.parse(source(b -> b.field("field", "Mi., 06 Dez. 2000 02:55:00 -0800"))); + mapper.parse(source(b -> b.field("field", "Mi, 06 Dez 2000 02:55:00 -0800"))); } public void testNullValue() throws IOException { From 1cded6523d738ed7536579e2634347fcced285a4 Mon Sep 17 00:00:00 2001 From: Jay Deng Date: Thu, 6 Jun 2024 09:56:41 -0700 Subject: [PATCH 62/77] Fix double invocation of postCollection when MultiBucketCollector is present (#14015) Signed-off-by: Jay Deng --- CHANGELOG.md | 1 + .../bucket/terms/StringTermsIT.java | 68 +++++++++++++++++++ .../BucketCollectorProcessor.java | 4 +- .../bucket/BestBucketsDeferringCollector.java | 2 + 4 files changed, 73 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 597e0ae31b054..2e150edd0a41b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -56,6 +56,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Painless: ensure type "UnmodifiableMap" for params ([#13885](https://github.com/opensearch-project/OpenSearch/pull/13885)) - Pass parent filter to inner hit query ([#13903](https://github.com/opensearch-project/OpenSearch/pull/13903)) - Fix NPE on restore searchable snapshot ([#13911](https://github.com/opensearch-project/OpenSearch/pull/13911)) +- Fix double invocation of postCollection when MultiBucketCollector is present ([#14015](https://github.com/opensearch-project/OpenSearch/pull/14015)) ### Security diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/StringTermsIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/StringTermsIT.java index edf9cd432dda2..f5d018b2ef491 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/StringTermsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/StringTermsIT.java @@ -42,10 +42,12 @@ import org.opensearch.index.query.QueryBuilders; import org.opensearch.script.Script; import org.opensearch.script.ScriptType; +import org.opensearch.search.aggregations.AggregationBuilders; import org.opensearch.search.aggregations.AggregationExecutionException; import org.opensearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.opensearch.search.aggregations.BucketOrder; import org.opensearch.search.aggregations.bucket.filter.Filter; +import org.opensearch.search.aggregations.bucket.filter.InternalFilters; import org.opensearch.search.aggregations.bucket.terms.Terms.Bucket; import org.opensearch.search.aggregations.metrics.Avg; import org.opensearch.search.aggregations.metrics.ExtendedStats; @@ -999,6 +1001,72 @@ public void testOtherDocCount() { testOtherDocCount(SINGLE_VALUED_FIELD_NAME, MULTI_VALUED_FIELD_NAME); } + public void testDeferredSubAggs() { + // Tests subAgg doc count is the same with different collection modes and additional top level aggs + SearchResponse r1 = client().prepareSearch("idx") + .setSize(0) + .addAggregation( + terms("terms1").collectMode(SubAggCollectionMode.BREADTH_FIRST) + .field("s_value") + .size(2) + .subAggregation(AggregationBuilders.filters("filter", QueryBuilders.boolQuery())) + ) + .addAggregation(AggregationBuilders.min("min").field("constant")) + .get(); + + SearchResponse r2 = client().prepareSearch("idx") + .setSize(0) + .addAggregation( + terms("terms1").collectMode(SubAggCollectionMode.DEPTH_FIRST) + .field("s_value") + .size(2) + .subAggregation(AggregationBuilders.filters("filter", QueryBuilders.boolQuery())) + ) + .addAggregation(AggregationBuilders.min("min").field("constant")) + .get(); + + SearchResponse r3 = client().prepareSearch("idx") + .setSize(0) + .addAggregation( + terms("terms1").collectMode(SubAggCollectionMode.BREADTH_FIRST) + .field("s_value") + .size(2) + .subAggregation(AggregationBuilders.filters("filter", QueryBuilders.boolQuery())) + ) + .get(); + + SearchResponse r4 = client().prepareSearch("idx") + .setSize(0) + .addAggregation( + terms("terms1").collectMode(SubAggCollectionMode.DEPTH_FIRST) + .field("s_value") + .size(2) + .subAggregation(AggregationBuilders.filters("filter", QueryBuilders.boolQuery())) + ) + .get(); + + assertNotNull(r1.getAggregations().get("terms1")); + assertNotNull(r2.getAggregations().get("terms1")); + assertNotNull(r3.getAggregations().get("terms1")); + assertNotNull(r4.getAggregations().get("terms1")); + + Terms terms = r1.getAggregations().get("terms1"); + Bucket b1 = terms.getBucketByKey("val0"); + InternalFilters f1 = b1.getAggregations().get("filter"); + long docCount1 = f1.getBuckets().get(0).getDocCount(); + Bucket b2 = terms.getBucketByKey("val1"); + InternalFilters f2 = b2.getAggregations().get("filter"); + long docCount2 = f1.getBuckets().get(0).getDocCount(); + + for (SearchResponse response : new SearchResponse[] { r2, r3, r4 }) { + terms = response.getAggregations().get("terms1"); + f1 = terms.getBucketByKey(b1.getKeyAsString()).getAggregations().get("filter"); + f2 = terms.getBucketByKey(b2.getKeyAsString()).getAggregations().get("filter"); + assertEquals(docCount1, f1.getBuckets().get(0).getDocCount()); + assertEquals(docCount2, f2.getBuckets().get(0).getDocCount()); + } + } + /** * Make sure that a request using a deterministic script or not using a script get cached. * Ensure requests using nondeterministic scripts do not get cached. diff --git a/server/src/main/java/org/opensearch/search/aggregations/BucketCollectorProcessor.java b/server/src/main/java/org/opensearch/search/aggregations/BucketCollectorProcessor.java index df05ce3f5c049..32c243cc12aa6 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/BucketCollectorProcessor.java +++ b/server/src/main/java/org/opensearch/search/aggregations/BucketCollectorProcessor.java @@ -71,10 +71,10 @@ public void processPostCollection(Collector collectorTree) throws IOException { collectors.offer(innerCollector); } } else if (currentCollector instanceof BucketCollector) { - ((BucketCollector) currentCollector).postCollection(); - // Perform build aggregation during post collection if (currentCollector instanceof Aggregator) { + // Do not perform postCollection for MultiBucketCollector as we are unwrapping that below + ((BucketCollector) currentCollector).postCollection(); ((Aggregator) currentCollector).buildTopLevel(); } else if (currentCollector instanceof MultiBucketCollector) { for (Collector innerCollector : ((MultiBucketCollector) currentCollector).getCollectors()) { diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/BestBucketsDeferringCollector.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/BestBucketsDeferringCollector.java index 223be3ba2d1ae..69d0fcd6c96f2 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/BestBucketsDeferringCollector.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/BestBucketsDeferringCollector.java @@ -124,6 +124,7 @@ private void finishLeaf() { if (context != null) { assert docDeltasBuilder != null && bucketsBuilder != null; entries.add(new Entry(context, docDeltasBuilder.build(), bucketsBuilder.build())); + context = null; } } @@ -161,6 +162,7 @@ public void preCollection() throws IOException { @Override public void postCollection() throws IOException { + assert searchContext.searcher().getLeafContexts().isEmpty() || finished != true; finishLeaf(); finished = true; } From 0ddf4bdd9ef14b2d8e928695bef08429579a6a5b Mon Sep 17 00:00:00 2001 From: Chenyang Ji Date: Thu, 6 Jun 2024 11:28:57 -0700 Subject: [PATCH 63/77] Query insights exporters implementation (#12982) --------- Signed-off-by: Chenyang Ji --- CHANGELOG.md | 1 + .../plugin/insights/QueryInsightsPlugin.java | 5 +- .../insights/core/exporter/DebugExporter.java | 61 ++++++++ .../core/exporter/LocalIndexExporter.java | 113 ++++++++++++++ .../core/exporter/QueryInsightsExporter.java | 26 ++++ .../QueryInsightsExporterFactory.java | 143 ++++++++++++++++++ .../insights/core/exporter/SinkType.java | 66 ++++++++ .../insights/core/exporter/package-info.java | 12 ++ .../core/service/QueryInsightsService.java | 37 ++++- .../core/service/TopQueriesService.java | 97 +++++++++++- .../rules/model/SearchQueryRecord.java | 7 + .../settings/QueryInsightsSettings.java | 33 ++++ .../insights/QueryInsightsPluginTests.java | 4 +- .../core/exporter/DebugExporterTests.java | 37 +++++ .../exporter/LocalIndexExporterTests.java | 99 ++++++++++++ .../QueryInsightsExporterFactoryTests.java | 89 +++++++++++ .../service/QueryInsightsServiceTests.java | 18 ++- .../core/service/TopQueriesServiceTests.java | 8 +- 18 files changed, 845 insertions(+), 11 deletions(-) create mode 100644 plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/DebugExporter.java create mode 100644 plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/LocalIndexExporter.java create mode 100644 plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporter.java create mode 100644 plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporterFactory.java create mode 100644 plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/SinkType.java create mode 100644 plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/package-info.java create mode 100644 plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/DebugExporterTests.java create mode 100644 plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/LocalIndexExporterTests.java create mode 100644 plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporterFactoryTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 2e150edd0a41b..30a7c9d8e9ab0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,6 +18,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add capability to disable source recovery_source for an index ([#13590](https://github.com/opensearch-project/OpenSearch/pull/13590)) - Add remote routing table for remote state publication with experimental feature flag ([#13304](https://github.com/opensearch-project/OpenSearch/pull/13304)) - [Remote Store] Add support to disable flush based on translog reader count ([#14027](https://github.com/opensearch-project/OpenSearch/pull/14027)) +- [Query Insights] Add exporter support for top n queries ([#12982](https://github.com/opensearch-project/OpenSearch/pull/12982)) ### Dependencies - Bump `com.github.spullara.mustache.java:compiler` from 0.9.10 to 0.9.13 ([#13329](https://github.com/opensearch-project/OpenSearch/pull/13329), [#13559](https://github.com/opensearch-project/OpenSearch/pull/13559)) diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/QueryInsightsPlugin.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/QueryInsightsPlugin.java index 4d7e0d486068a..22831c3e0f8ba 100644 --- a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/QueryInsightsPlugin.java +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/QueryInsightsPlugin.java @@ -70,7 +70,7 @@ public Collection createComponents( final Supplier repositoriesServiceSupplier ) { // create top n queries service - final QueryInsightsService queryInsightsService = new QueryInsightsService(threadPool); + final QueryInsightsService queryInsightsService = new QueryInsightsService(clusterService.getClusterSettings(), threadPool, client); return List.of(queryInsightsService, new QueryInsightsListener(clusterService, queryInsightsService)); } @@ -110,7 +110,8 @@ public List> getSettings() { // Settings for top N queries QueryInsightsSettings.TOP_N_LATENCY_QUERIES_ENABLED, QueryInsightsSettings.TOP_N_LATENCY_QUERIES_SIZE, - QueryInsightsSettings.TOP_N_LATENCY_QUERIES_WINDOW_SIZE + QueryInsightsSettings.TOP_N_LATENCY_QUERIES_WINDOW_SIZE, + QueryInsightsSettings.TOP_N_LATENCY_EXPORTER_SETTINGS ); } } diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/DebugExporter.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/DebugExporter.java new file mode 100644 index 0000000000000..116bd26e1f9bc --- /dev/null +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/DebugExporter.java @@ -0,0 +1,61 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.insights.core.exporter; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; + +import java.util.List; + +/** + * Debug exporter for development purpose + */ +public final class DebugExporter implements QueryInsightsExporter { + /** + * Logger of the debug exporter + */ + private final Logger logger = LogManager.getLogger(); + + /** + * Constructor of DebugExporter + */ + private DebugExporter() {} + + private static class InstanceHolder { + private static final DebugExporter INSTANCE = new DebugExporter(); + } + + /** + Get the singleton instance of DebugExporter + * + @return DebugExporter instance + */ + public static DebugExporter getInstance() { + return InstanceHolder.INSTANCE; + } + + /** + * Write the list of SearchQueryRecord to debug log + * + * @param records list of {@link SearchQueryRecord} + */ + @Override + public void export(final List records) { + logger.debug("QUERY_INSIGHTS_RECORDS: " + records.toString()); + } + + /** + * Close the debugger exporter sink + */ + @Override + public void close() { + logger.debug("Closing the DebugExporter.."); + } +} diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/LocalIndexExporter.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/LocalIndexExporter.java new file mode 100644 index 0000000000000..c19fe3655098b --- /dev/null +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/LocalIndexExporter.java @@ -0,0 +1,113 @@ +/* + * 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.plugin.insights.core.exporter; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.bulk.BulkRequestBuilder; +import org.opensearch.action.bulk.BulkResponse; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.client.Client; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormatter; + +import java.util.List; + +/** + * Local index exporter for exporting query insights data to local OpenSearch indices. + */ +public final class LocalIndexExporter implements QueryInsightsExporter { + /** + * Logger of the local index exporter + */ + private final Logger logger = LogManager.getLogger(); + private final Client client; + private DateTimeFormatter indexPattern; + + /** + * Constructor of LocalIndexExporter + * + * @param client OS client + * @param indexPattern the pattern of index to export to + */ + public LocalIndexExporter(final Client client, final DateTimeFormatter indexPattern) { + this.indexPattern = indexPattern; + this.client = client; + } + + /** + * Getter of indexPattern + * + * @return indexPattern + */ + public DateTimeFormatter getIndexPattern() { + return indexPattern; + } + + /** + * Setter of indexPattern + * + * @param indexPattern index pattern + * @return the current LocalIndexExporter + */ + public LocalIndexExporter setIndexPattern(DateTimeFormatter indexPattern) { + this.indexPattern = indexPattern; + return this; + } + + /** + * Export a list of SearchQueryRecord to a local index + * + * @param records list of {@link SearchQueryRecord} + */ + @Override + public void export(final List records) { + if (records == null || records.size() == 0) { + return; + } + try { + final String index = getDateTimeFromFormat(); + final BulkRequestBuilder bulkRequestBuilder = client.prepareBulk().setTimeout(TimeValue.timeValueMinutes(1)); + for (SearchQueryRecord record : records) { + bulkRequestBuilder.add( + new IndexRequest(index).source(record.toXContent(XContentFactory.jsonBuilder(), ToXContent.EMPTY_PARAMS)) + ); + } + bulkRequestBuilder.execute(new ActionListener() { + @Override + public void onResponse(BulkResponse bulkItemResponses) {} + + @Override + public void onFailure(Exception e) { + logger.error("Failed to execute bulk operation for query insights data: ", e); + } + }); + } catch (final Exception e) { + logger.error("Unable to index query insights data: ", e); + } + } + + /** + * Close the exporter sink + */ + @Override + public void close() { + logger.debug("Closing the LocalIndexExporter.."); + } + + private String getDateTimeFromFormat() { + return indexPattern.print(DateTime.now(DateTimeZone.UTC)); + } +} diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporter.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporter.java new file mode 100644 index 0000000000000..42e5354eb1640 --- /dev/null +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporter.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.plugin.insights.core.exporter; + +import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; + +import java.io.Closeable; +import java.util.List; + +/** + * Base interface for Query Insights exporters + */ +public interface QueryInsightsExporter extends Closeable { + /** + * Export a list of SearchQueryRecord to the exporter sink + * + * @param records list of {@link SearchQueryRecord} + */ + void export(final List records); +} diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporterFactory.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporterFactory.java new file mode 100644 index 0000000000000..7324590c9f582 --- /dev/null +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporterFactory.java @@ -0,0 +1,143 @@ +/* + * 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.plugin.insights.core.exporter; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.client.Client; +import org.opensearch.common.settings.Settings; +import org.joda.time.format.DateTimeFormat; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Locale; +import java.util.Set; + +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.DEFAULT_TOP_N_LATENCY_QUERIES_INDEX_PATTERN; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.DEFAULT_TOP_QUERIES_EXPORTER_TYPE; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.EXPORTER_TYPE; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.EXPORT_INDEX; + +/** + * Factory class for validating and creating exporters based on provided settings + */ +public class QueryInsightsExporterFactory { + /** + * Logger of the query insights exporter factory + */ + private final Logger logger = LogManager.getLogger(); + final private Client client; + final private Set exporters; + + /** + * Constructor of QueryInsightsExporterFactory + * + * @param client OS client + */ + public QueryInsightsExporterFactory(final Client client) { + this.client = client; + this.exporters = new HashSet<>(); + } + + /** + * Validate exporter sink config + * + * @param settings exporter sink config {@link Settings} + * @throws IllegalArgumentException if provided exporter sink config settings are invalid + */ + public void validateExporterConfig(final Settings settings) throws IllegalArgumentException { + // Disable exporter if the EXPORTER_TYPE setting is null + if (settings.get(EXPORTER_TYPE) == null) { + return; + } + SinkType type; + try { + type = SinkType.parse(settings.get(EXPORTER_TYPE, DEFAULT_TOP_QUERIES_EXPORTER_TYPE)); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "Invalid exporter type [%s], type should be one of %s", + settings.get(EXPORTER_TYPE), + SinkType.allSinkTypes() + ) + ); + } + switch (type) { + case LOCAL_INDEX: + final String indexPattern = settings.get(EXPORT_INDEX, DEFAULT_TOP_N_LATENCY_QUERIES_INDEX_PATTERN); + if (indexPattern.length() == 0) { + throw new IllegalArgumentException("Empty index pattern configured for the exporter"); + } + try { + DateTimeFormat.forPattern(indexPattern); + } catch (Exception e) { + throw new IllegalArgumentException( + String.format(Locale.ROOT, "Invalid index pattern [%s] configured for the exporter", indexPattern) + ); + } + } + } + + /** + * Create an exporter based on provided parameters + * + * @param type The type of exporter to create + * @param indexPattern the index pattern if creating a index exporter + * @return QueryInsightsExporter the created exporter sink + */ + public QueryInsightsExporter createExporter(SinkType type, String indexPattern) { + if (SinkType.LOCAL_INDEX.equals(type)) { + QueryInsightsExporter exporter = new LocalIndexExporter(client, DateTimeFormat.forPattern(indexPattern)); + this.exporters.add(exporter); + return exporter; + } + return DebugExporter.getInstance(); + } + + /** + * Update an exporter based on provided parameters + * + * @param exporter The exporter to update + * @param indexPattern the index pattern if creating a index exporter + * @return QueryInsightsExporter the updated exporter sink + */ + public QueryInsightsExporter updateExporter(QueryInsightsExporter exporter, String indexPattern) { + if (exporter.getClass() == LocalIndexExporter.class) { + ((LocalIndexExporter) exporter).setIndexPattern(DateTimeFormat.forPattern(indexPattern)); + } + return exporter; + } + + /** + * Close an exporter + * + * @param exporter the exporter to close + */ + public void closeExporter(QueryInsightsExporter exporter) throws IOException { + if (exporter != null) { + exporter.close(); + this.exporters.remove(exporter); + } + } + + /** + * Close all exporters + * + */ + public void closeAllExporters() { + for (QueryInsightsExporter exporter : exporters) { + try { + closeExporter(exporter); + } catch (IOException e) { + logger.error("Fail to close query insights exporter, error: ", e); + } + } + } +} diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/SinkType.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/SinkType.java new file mode 100644 index 0000000000000..c90c9c76b6706 --- /dev/null +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/SinkType.java @@ -0,0 +1,66 @@ +/* + * 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.plugin.insights.core.exporter; + +import java.util.Arrays; +import java.util.Locale; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Type of supported sinks + */ +public enum SinkType { + /** debug exporter */ + DEBUG("debug"), + /** local index exporter */ + LOCAL_INDEX("local_index"); + + private final String type; + + SinkType(String type) { + this.type = type; + } + + @Override + public String toString() { + return type; + } + + /** + * Parse SinkType from String + * @param type the String representation of the SinkType + * @return SinkType + */ + public static SinkType parse(final String type) { + return valueOf(type.toUpperCase(Locale.ROOT)); + } + + /** + * Get all valid SinkTypes + * + * @return A set contains all valid SinkTypes + */ + public static Set allSinkTypes() { + return Arrays.stream(values()).collect(Collectors.toSet()); + } + + /** + * Get Sink type from exporter + * + * @param exporter the {@link QueryInsightsExporter} + * @return SinkType associated with this exporter + */ + public static SinkType getSinkTypeFromExporter(QueryInsightsExporter exporter) { + if (exporter.getClass().equals(LocalIndexExporter.class)) { + return SinkType.LOCAL_INDEX; + } + return SinkType.DEBUG; + } +} diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/package-info.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/package-info.java new file mode 100644 index 0000000000000..7164411194f85 --- /dev/null +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/exporter/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * Query Insights exporter + */ +package org.opensearch.plugin.insights.core.exporter; diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/service/QueryInsightsService.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/service/QueryInsightsService.java index 525ca0d4a3d33..a83bb2094f165 100644 --- a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/service/QueryInsightsService.java +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/service/QueryInsightsService.java @@ -8,14 +8,18 @@ package org.opensearch.plugin.insights.core.service; +import org.opensearch.client.Client; import org.opensearch.common.inject.Inject; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.plugin.insights.core.exporter.QueryInsightsExporterFactory; import org.opensearch.plugin.insights.rules.model.MetricType; import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; import org.opensearch.plugin.insights.settings.QueryInsightsSettings; import org.opensearch.threadpool.Scheduler; import org.opensearch.threadpool.ThreadPool; +import java.io.IOException; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -23,6 +27,8 @@ import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.TOP_N_LATENCY_EXPORTER_SETTINGS; + /** * Service responsible for gathering, analyzing, storing and exporting * information related to search queries @@ -56,21 +62,35 @@ public class QueryInsightsService extends AbstractLifecycleComponent { */ protected volatile Scheduler.Cancellable scheduledFuture; + /** + * Query Insights exporter factory + */ + final QueryInsightsExporterFactory queryInsightsExporterFactory; + /** * Constructor of the QueryInsightsService * - * @param threadPool The OpenSearch thread pool to run async tasks + * @param clusterSettings OpenSearch cluster level settings + * @param threadPool The OpenSearch thread pool to run async tasks + * @param client OS client */ @Inject - public QueryInsightsService(final ThreadPool threadPool) { + public QueryInsightsService(final ClusterSettings clusterSettings, final ThreadPool threadPool, final Client client) { enableCollect = new HashMap<>(); queryRecordsQueue = new LinkedBlockingQueue<>(QueryInsightsSettings.QUERY_RECORD_QUEUE_CAPACITY); + this.threadPool = threadPool; + this.queryInsightsExporterFactory = new QueryInsightsExporterFactory(client); + // initialize top n queries services and configurations consumers topQueriesServices = new HashMap<>(); for (MetricType metricType : MetricType.allMetricTypes()) { enableCollect.put(metricType, false); - topQueriesServices.put(metricType, new TopQueriesService(metricType)); + topQueriesServices.put(metricType, new TopQueriesService(metricType, threadPool, queryInsightsExporterFactory)); } - this.threadPool = threadPool; + clusterSettings.addSettingsUpdateConsumer( + TOP_N_LATENCY_EXPORTER_SETTINGS, + (settings -> getTopQueriesService(MetricType.LATENCY).setExporter(settings)), + (settings -> getTopQueriesService(MetricType.LATENCY).validateExporterConfig(settings)) + ); } /** @@ -176,5 +196,12 @@ protected void doStop() { } @Override - protected void doClose() {} + protected void doClose() throws IOException { + // close all top n queries service + for (TopQueriesService topQueriesService : topQueriesServices.values()) { + topQueriesService.close(); + } + // close any unclosed resources + queryInsightsExporterFactory.closeAllExporters(); + } } diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/service/TopQueriesService.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/service/TopQueriesService.java index d2c30cbdf98e7..ff90edf1ec33d 100644 --- a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/service/TopQueriesService.java +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/core/service/TopQueriesService.java @@ -8,11 +8,19 @@ package org.opensearch.plugin.insights.core.service; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.plugin.insights.core.exporter.QueryInsightsExporter; +import org.opensearch.plugin.insights.core.exporter.QueryInsightsExporterFactory; +import org.opensearch.plugin.insights.core.exporter.SinkType; import org.opensearch.plugin.insights.rules.model.MetricType; import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; import org.opensearch.plugin.insights.settings.QueryInsightsSettings; +import org.opensearch.threadpool.ThreadPool; +import java.io.IOException; import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; @@ -27,6 +35,12 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.DEFAULT_TOP_N_LATENCY_QUERIES_INDEX_PATTERN; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.DEFAULT_TOP_QUERIES_EXPORTER_TYPE; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.EXPORTER_TYPE; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.EXPORT_INDEX; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.QUERY_INSIGHTS_EXECUTOR; + /** * Service responsible for gathering and storing top N queries * with high latency or resource usage @@ -34,6 +48,10 @@ * @opensearch.internal */ public class TopQueriesService { + /** + * Logger of the local index exporter + */ + private final Logger logger = LogManager.getLogger(); private boolean enabled; /** * The metric type to measure top n queries @@ -63,12 +81,34 @@ public class TopQueriesService { */ private final AtomicReference> topQueriesHistorySnapshot; - TopQueriesService(final MetricType metricType) { + /** + * Factory for validating and creating exporters + */ + private final QueryInsightsExporterFactory queryInsightsExporterFactory; + + /** + * The internal OpenSearch thread pool that execute async processing and exporting tasks + */ + private final ThreadPool threadPool; + + /** + * Exporter for exporting top queries data + */ + private QueryInsightsExporter exporter; + + TopQueriesService( + final MetricType metricType, + final ThreadPool threadPool, + final QueryInsightsExporterFactory queryInsightsExporterFactory + ) { this.enabled = false; this.metricType = metricType; + this.threadPool = threadPool; + this.queryInsightsExporterFactory = queryInsightsExporterFactory; this.topNSize = QueryInsightsSettings.DEFAULT_TOP_N_SIZE; this.windowSize = QueryInsightsSettings.DEFAULT_WINDOW_SIZE; this.windowStart = -1L; + this.exporter = null; topQueriesStore = new PriorityQueue<>(topNSize, (a, b) -> SearchQueryRecord.compare(a, b, metricType)); topQueriesCurrentSnapshot = new AtomicReference<>(new ArrayList<>()); topQueriesHistorySnapshot = new AtomicReference<>(new ArrayList<>()); @@ -169,6 +209,50 @@ public void validateWindowSize(final TimeValue windowSize) { } } + /** + * Set up the top queries exporter based on provided settings + * + * @param settings exporter config {@link Settings} + */ + public void setExporter(final Settings settings) { + if (settings.get(EXPORTER_TYPE) != null) { + SinkType expectedType = SinkType.parse(settings.get(EXPORTER_TYPE, DEFAULT_TOP_QUERIES_EXPORTER_TYPE)); + if (exporter != null && expectedType == SinkType.getSinkTypeFromExporter(exporter)) { + queryInsightsExporterFactory.updateExporter( + exporter, + settings.get(EXPORT_INDEX, DEFAULT_TOP_N_LATENCY_QUERIES_INDEX_PATTERN) + ); + } else { + try { + queryInsightsExporterFactory.closeExporter(this.exporter); + } catch (IOException e) { + logger.error("Fail to close the current exporter when updating exporter, error: ", e); + } + this.exporter = queryInsightsExporterFactory.createExporter( + SinkType.parse(settings.get(EXPORTER_TYPE, DEFAULT_TOP_QUERIES_EXPORTER_TYPE)), + settings.get(EXPORT_INDEX, DEFAULT_TOP_N_LATENCY_QUERIES_INDEX_PATTERN) + ); + } + } else { + // Disable exporter if exporter type is set to null + try { + queryInsightsExporterFactory.closeExporter(this.exporter); + this.exporter = null; + } catch (IOException e) { + logger.error("Fail to close the current exporter when disabling exporter, error: ", e); + } + } + } + + /** + * Validate provided settings for top queries exporter + * + * @param settings settings exporter config {@link Settings} + */ + public void validateExporterConfig(Settings settings) { + queryInsightsExporterFactory.validateExporterConfig(settings); + } + /** * Get all top queries records that are in the current top n queries store * Optionally include top N records from the last window. @@ -254,6 +338,10 @@ private void rotateWindowIfNecessary(final long newWindowStart) { topQueriesStore.clear(); topQueriesCurrentSnapshot.set(new ArrayList<>()); windowStart = newWindowStart; + // export to the configured sink + if (exporter != null) { + threadPool.executor(QUERY_INSIGHTS_EXECUTOR).execute(() -> exporter.export(history)); + } } } @@ -279,4 +367,11 @@ private long calculateWindowStart(final long timestamp) { public List getTopQueriesCurrentSnapshot() { return topQueriesCurrentSnapshot.get(); } + + /** + * Close the top n queries service + */ + public void close() throws IOException { + queryInsightsExporterFactory.closeExporter(this.exporter); + } } diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/rules/model/SearchQueryRecord.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/rules/model/SearchQueryRecord.java index 060711edb5580..fec00a680ae58 100644 --- a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/rules/model/SearchQueryRecord.java +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/rules/model/SearchQueryRecord.java @@ -8,9 +8,11 @@ package org.opensearch.plugin.insights.rules.model; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; @@ -173,4 +175,9 @@ public boolean equals(final Object o) { public int hashCode() { return Objects.hash(timestamp, measurements, attributes); } + + @Override + public String toString() { + return Strings.toString(MediaTypeRegistry.JSON, this); + } } diff --git a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/settings/QueryInsightsSettings.java b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/settings/QueryInsightsSettings.java index 52cc1fbde790f..b2e01062e334c 100644 --- a/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/settings/QueryInsightsSettings.java +++ b/plugins/query-insights/src/main/java/org/opensearch/plugin/insights/settings/QueryInsightsSettings.java @@ -9,7 +9,9 @@ package org.opensearch.plugin.insights.settings; import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.plugin.insights.core.exporter.SinkType; import java.util.Arrays; import java.util.HashSet; @@ -109,6 +111,37 @@ public class QueryInsightsSettings { Setting.Property.Dynamic ); + /** + * Config key for exporter type + */ + public static final String EXPORTER_TYPE = "type"; + /** + * Config key for export index + */ + public static final String EXPORT_INDEX = "config.index"; + + /** + * Settings and defaults for top queries exporters + */ + private static final String TOP_N_LATENCY_QUERIES_EXPORTER_PREFIX = TOP_N_LATENCY_QUERIES_PREFIX + ".exporter."; + /** + * Default index pattern of top n queries by latency + */ + public static final String DEFAULT_TOP_N_LATENCY_QUERIES_INDEX_PATTERN = "'top_queries_by_latency-'YYYY.MM.dd"; + /** + * Default exporter type of top queries + */ + public static final String DEFAULT_TOP_QUERIES_EXPORTER_TYPE = SinkType.LOCAL_INDEX.toString(); + + /** + * Settings for the exporter of top latency queries + */ + public static final Setting TOP_N_LATENCY_EXPORTER_SETTINGS = Setting.groupSetting( + TOP_N_LATENCY_QUERIES_EXPORTER_PREFIX, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + /** * Default constructor */ diff --git a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/QueryInsightsPluginTests.java b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/QueryInsightsPluginTests.java index 2f353f2a53329..8b8856e3e305c 100644 --- a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/QueryInsightsPluginTests.java +++ b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/QueryInsightsPluginTests.java @@ -50,6 +50,7 @@ public void setup() { clusterSettings.registerSetting(QueryInsightsSettings.TOP_N_LATENCY_QUERIES_ENABLED); clusterSettings.registerSetting(QueryInsightsSettings.TOP_N_LATENCY_QUERIES_SIZE); clusterSettings.registerSetting(QueryInsightsSettings.TOP_N_LATENCY_QUERIES_WINDOW_SIZE); + clusterSettings.registerSetting(QueryInsightsSettings.TOP_N_LATENCY_EXPORTER_SETTINGS); clusterService = ClusterServiceUtils.createClusterService(settings, clusterSettings, threadPool); } @@ -59,7 +60,8 @@ public void testGetSettings() { Arrays.asList( QueryInsightsSettings.TOP_N_LATENCY_QUERIES_ENABLED, QueryInsightsSettings.TOP_N_LATENCY_QUERIES_SIZE, - QueryInsightsSettings.TOP_N_LATENCY_QUERIES_WINDOW_SIZE + QueryInsightsSettings.TOP_N_LATENCY_QUERIES_WINDOW_SIZE, + QueryInsightsSettings.TOP_N_LATENCY_EXPORTER_SETTINGS ), queryInsightsPlugin.getSettings() ); diff --git a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/DebugExporterTests.java b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/DebugExporterTests.java new file mode 100644 index 0000000000000..736e406289b2c --- /dev/null +++ b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/DebugExporterTests.java @@ -0,0 +1,37 @@ +/* + * 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.plugin.insights.core.exporter; + +import org.opensearch.plugin.insights.QueryInsightsTestUtils; +import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.util.List; + +/** + * Granular tests for the {@link DebugExporterTests} class. + */ +public class DebugExporterTests extends OpenSearchTestCase { + private DebugExporter debugExporter; + + @Before + public void setup() { + debugExporter = DebugExporter.getInstance(); + } + + public void testExport() { + List records = QueryInsightsTestUtils.generateQueryInsightRecords(2); + try { + debugExporter.export(records); + } catch (Exception e) { + fail("No exception should be thrown when exporting query insights data"); + } + } +} diff --git a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/LocalIndexExporterTests.java b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/LocalIndexExporterTests.java new file mode 100644 index 0000000000000..9ea864a7083f4 --- /dev/null +++ b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/LocalIndexExporterTests.java @@ -0,0 +1,99 @@ +/* + * 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.plugin.insights.core.exporter; + +import org.opensearch.action.bulk.BulkAction; +import org.opensearch.action.bulk.BulkRequestBuilder; +import org.opensearch.action.bulk.BulkResponse; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.client.Client; +import org.opensearch.plugin.insights.QueryInsightsTestUtils; +import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; +import org.opensearch.test.OpenSearchTestCase; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.junit.Before; + +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +/** + * Granular tests for the {@link LocalIndexExporterTests} class. + */ +public class LocalIndexExporterTests extends OpenSearchTestCase { + private final DateTimeFormatter format = DateTimeFormat.forPattern("YYYY.MM.dd"); + private final Client client = mock(Client.class); + private LocalIndexExporter localIndexExporter; + + @Before + public void setup() { + localIndexExporter = new LocalIndexExporter(client, format); + } + + public void testExportEmptyRecords() { + List records = List.of(); + try { + localIndexExporter.export(records); + } catch (Exception e) { + fail("No exception should be thrown when exporting empty query insights data"); + } + } + + @SuppressWarnings("unchecked") + public void testExportRecords() { + BulkRequestBuilder bulkRequestBuilder = spy(new BulkRequestBuilder(client, BulkAction.INSTANCE)); + final PlainActionFuture future = mock(PlainActionFuture.class); + when(future.actionGet()).thenReturn(null); + doAnswer(invocation -> future).when(bulkRequestBuilder).execute(); + when(client.prepareBulk()).thenReturn(bulkRequestBuilder); + + List records = QueryInsightsTestUtils.generateQueryInsightRecords(2); + try { + localIndexExporter.export(records); + } catch (Exception e) { + fail("No exception should be thrown when exporting query insights data"); + } + assertEquals(2, bulkRequestBuilder.numberOfActions()); + } + + @SuppressWarnings("unchecked") + public void testExportRecordsWithError() { + BulkRequestBuilder bulkRequestBuilder = spy(new BulkRequestBuilder(client, BulkAction.INSTANCE)); + final PlainActionFuture future = mock(PlainActionFuture.class); + when(future.actionGet()).thenReturn(null); + doThrow(new RuntimeException()).when(bulkRequestBuilder).execute(); + when(client.prepareBulk()).thenReturn(bulkRequestBuilder); + + List records = QueryInsightsTestUtils.generateQueryInsightRecords(2); + try { + localIndexExporter.export(records); + } catch (Exception e) { + fail("No exception should be thrown when exporting query insights data"); + } + } + + public void testClose() { + try { + localIndexExporter.close(); + } catch (Exception e) { + fail("No exception should be thrown when closing local index exporter"); + } + } + + public void testGetAndSetIndexPattern() { + DateTimeFormatter newFormatter = mock(DateTimeFormatter.class); + localIndexExporter.setIndexPattern(newFormatter); + assert (localIndexExporter.getIndexPattern() == newFormatter); + } +} diff --git a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporterFactoryTests.java b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporterFactoryTests.java new file mode 100644 index 0000000000000..f01dd2c17509c --- /dev/null +++ b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/exporter/QueryInsightsExporterFactoryTests.java @@ -0,0 +1,89 @@ +/* + * 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.plugin.insights.core.exporter; + +import org.opensearch.client.Client; +import org.opensearch.common.settings.Settings; +import org.opensearch.test.OpenSearchTestCase; +import org.joda.time.format.DateTimeFormat; +import org.junit.Before; + +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.DEFAULT_TOP_QUERIES_EXPORTER_TYPE; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.EXPORTER_TYPE; +import static org.opensearch.plugin.insights.settings.QueryInsightsSettings.EXPORT_INDEX; +import static org.mockito.Mockito.mock; + +/** + * Granular tests for the {@link QueryInsightsExporterFactoryTests} class. + */ +public class QueryInsightsExporterFactoryTests extends OpenSearchTestCase { + private final String format = "YYYY.MM.dd"; + + private final Client client = mock(Client.class); + private QueryInsightsExporterFactory queryInsightsExporterFactory; + + @Before + public void setup() { + queryInsightsExporterFactory = new QueryInsightsExporterFactory(client); + } + + public void testValidateConfigWhenResetExporter() { + Settings.Builder settingsBuilder = Settings.builder(); + // empty settings + Settings settings = settingsBuilder.build(); + try { + queryInsightsExporterFactory.validateExporterConfig(settings); + } catch (Exception e) { + fail("No exception should be thrown when setting is null"); + } + } + + public void testInvalidExporterTypeConfig() { + Settings.Builder settingsBuilder = Settings.builder(); + Settings settings = settingsBuilder.put(EXPORTER_TYPE, "some_invalid_type").build(); + assertThrows(IllegalArgumentException.class, () -> { queryInsightsExporterFactory.validateExporterConfig(settings); }); + } + + public void testInvalidLocalIndexConfig() { + Settings.Builder settingsBuilder = Settings.builder(); + assertThrows(IllegalArgumentException.class, () -> { + queryInsightsExporterFactory.validateExporterConfig( + settingsBuilder.put(EXPORTER_TYPE, DEFAULT_TOP_QUERIES_EXPORTER_TYPE).put(EXPORT_INDEX, "").build() + ); + }); + assertThrows(IllegalArgumentException.class, () -> { + queryInsightsExporterFactory.validateExporterConfig( + settingsBuilder.put(EXPORTER_TYPE, DEFAULT_TOP_QUERIES_EXPORTER_TYPE).put(EXPORT_INDEX, "some_invalid_pattern").build() + ); + }); + } + + public void testCreateAndCloseExporter() { + QueryInsightsExporter exporter1 = queryInsightsExporterFactory.createExporter(SinkType.LOCAL_INDEX, format); + assertTrue(exporter1 instanceof LocalIndexExporter); + QueryInsightsExporter exporter2 = queryInsightsExporterFactory.createExporter(SinkType.DEBUG, format); + assertTrue(exporter2 instanceof DebugExporter); + QueryInsightsExporter exporter3 = queryInsightsExporterFactory.createExporter(SinkType.DEBUG, format); + assertTrue(exporter3 instanceof DebugExporter); + try { + queryInsightsExporterFactory.closeExporter(exporter1); + queryInsightsExporterFactory.closeExporter(exporter2); + queryInsightsExporterFactory.closeAllExporters(); + } catch (Exception e) { + fail("No exception should be thrown when closing exporter"); + } + } + + public void testUpdateExporter() { + LocalIndexExporter exporter = new LocalIndexExporter(client, DateTimeFormat.forPattern("yyyy-MM-dd")); + queryInsightsExporterFactory.updateExporter(exporter, "yyyy-MM-dd-HH"); + assertEquals(DateTimeFormat.forPattern("yyyy-MM-dd-HH"), exporter.getIndexPattern()); + } + +} diff --git a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/service/QueryInsightsServiceTests.java b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/service/QueryInsightsServiceTests.java index c29b48b9690d1..428f615ce2f90 100644 --- a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/service/QueryInsightsServiceTests.java +++ b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/service/QueryInsightsServiceTests.java @@ -8,6 +8,9 @@ package org.opensearch.plugin.insights.core.service; +import org.opensearch.client.Client; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; import org.opensearch.plugin.insights.QueryInsightsTestUtils; import org.opensearch.plugin.insights.rules.model.MetricType; import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; @@ -23,11 +26,16 @@ */ public class QueryInsightsServiceTests extends OpenSearchTestCase { private final ThreadPool threadPool = mock(ThreadPool.class); + private final Client client = mock(Client.class); private QueryInsightsService queryInsightsService; @Before public void setup() { - queryInsightsService = new QueryInsightsService(threadPool); + Settings.Builder settingsBuilder = Settings.builder(); + Settings settings = settingsBuilder.build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterSettings.registerSetting(QueryInsightsSettings.TOP_N_LATENCY_EXPORTER_SETTINGS); + queryInsightsService = new QueryInsightsService(clusterSettings, threadPool, client); queryInsightsService.enableCollection(MetricType.LATENCY, true); queryInsightsService.enableCollection(MetricType.CPU, true); queryInsightsService.enableCollection(MetricType.JVM, true); @@ -46,4 +54,12 @@ public void testAddRecordToLimitAndDrain() { queryInsightsService.getTopQueriesService(MetricType.LATENCY).getTopQueriesRecords(false).size() ); } + + public void testClose() { + try { + queryInsightsService.doClose(); + } catch (Exception e) { + fail("No exception expected when closing query insights service"); + } + } } diff --git a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/service/TopQueriesServiceTests.java b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/service/TopQueriesServiceTests.java index 060df84a89485..3efd4c86833cc 100644 --- a/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/service/TopQueriesServiceTests.java +++ b/plugins/query-insights/src/test/java/org/opensearch/plugin/insights/core/service/TopQueriesServiceTests.java @@ -11,24 +11,30 @@ import org.opensearch.cluster.coordination.DeterministicTaskQueue; import org.opensearch.common.unit.TimeValue; import org.opensearch.plugin.insights.QueryInsightsTestUtils; +import org.opensearch.plugin.insights.core.exporter.QueryInsightsExporterFactory; import org.opensearch.plugin.insights.rules.model.MetricType; import org.opensearch.plugin.insights.rules.model.SearchQueryRecord; import org.opensearch.plugin.insights.settings.QueryInsightsSettings; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.ThreadPool; import org.junit.Before; import java.util.List; import java.util.concurrent.TimeUnit; +import static org.mockito.Mockito.mock; + /** * Unit Tests for {@link QueryInsightsService}. */ public class TopQueriesServiceTests extends OpenSearchTestCase { private TopQueriesService topQueriesService; + private final ThreadPool threadPool = mock(ThreadPool.class); + private final QueryInsightsExporterFactory queryInsightsExporterFactory = mock(QueryInsightsExporterFactory.class); @Before public void setup() { - topQueriesService = new TopQueriesService(MetricType.LATENCY); + topQueriesService = new TopQueriesService(MetricType.LATENCY, threadPool, queryInsightsExporterFactory); topQueriesService.setTopNSize(Integer.MAX_VALUE); topQueriesService.setWindowSize(new TimeValue(Long.MAX_VALUE)); topQueriesService.setEnabled(true); From 9da6170fc38653284cfeee0e5c1800d41999ba08 Mon Sep 17 00:00:00 2001 From: Rishabh Maurya Date: Thu, 6 Jun 2024 11:36:03 -0700 Subject: [PATCH 64/77] [DerivedField] Support DerivedField object type and integration with Mapper (#13720) * Support DerivedField object type * ValueFetcher for DerivedField of object type to parse and fetch nested fields. * DerivedFieldResolver to resolve DerivedField type on the fly. * Introduction of new support types - text, float. * Use index analyzer for derived fields set in index settings. * Support of derived fields from query string * Unit tests Signed-off-by: Rishabh Maurya * Derived field object type properties to also support Object as a value Signed-off-by: Rishabh Maurya * Index and cluster setting to control derived fields Signed-off-by: Rishabh Maurya * Fix test Signed-off-by: Rishabh Maurya * Fixed missing javadoc Signed-off-by: Rishabh Maurya * Fix the rewrite issue when profile is set as true Signed-off-by: Rishabh Maurya * Address PR comments Signed-off-by: Rishabh Maurya * Address PR comments Signed-off-by: Rishabh Maurya * Use conjuction query with filter on source_indexed_field query and DerivedFieldQuery Signed-off-by: Rishabh Maurya * support source_indexed_field for non-object type derived fields Signed-off-by: Rishabh Maurya * rename source_indexed_field to prefilter_field Signed-off-by: Rishabh Maurya --------- Signed-off-by: Rishabh Maurya --- .../common/settings/ClusterSettings.java | 3 +- .../common/settings/IndexScopedSettings.java | 2 +- .../org/opensearch/index/IndexSettings.java | 22 + .../mapper/DefaultDerivedFieldResolver.java | 229 ++++++ .../opensearch/index/mapper/DerivedField.java | 29 +- .../index/mapper/DerivedFieldMapper.java | 180 +++-- .../index/mapper/DerivedFieldResolver.java | 37 + .../mapper/DerivedFieldResolverFactory.java | 58 ++ .../mapper/DerivedFieldSupportedTypes.java | 95 ++- .../index/mapper/DerivedFieldType.java | 304 +++++++- .../mapper/DerivedFieldValueFetcher.java | 18 +- .../mapper/NoOpDerivedFieldResolver.java | 28 + .../index/mapper/ObjectDerivedFieldType.java | 168 +++++ .../index/query/DerivedFieldQuery.java | 34 +- .../index/query/QueryShardContext.java | 50 +- .../VectorGeoPointShapeQueryProcessor.java | 2 +- .../index/search/QueryParserHelper.java | 3 + .../opensearch/script/DerivedFieldScript.java | 4 +- .../org/opensearch/search/SearchService.java | 51 +- .../search/builder/SearchSourceBuilder.java | 6 +- .../subphase/highlight/HighlightPhase.java | 4 +- .../highlight/UnifiedHighlighter.java | 6 +- .../mapper/DerivedFieldMapperQueryTests.java | 662 +++++++++++++++++- .../mapper/DerivedFieldResolverTests.java | 537 ++++++++++++++ .../index/mapper/DerivedFieldTypeTests.java | 62 +- .../index/query/DerivedFieldQueryTests.java | 90 ++- .../index/query/QueryShardContextTests.java | 37 +- .../opensearch/search/SearchServiceTests.java | 87 ++- .../builder/SearchSourceBuilderTests.java | 6 +- .../DerivedFieldFetchAndHighlightTests.java | 242 +++++-- 30 files changed, 2736 insertions(+), 320 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/mapper/DefaultDerivedFieldResolver.java create mode 100644 server/src/main/java/org/opensearch/index/mapper/DerivedFieldResolver.java create mode 100644 server/src/main/java/org/opensearch/index/mapper/DerivedFieldResolverFactory.java create mode 100644 server/src/main/java/org/opensearch/index/mapper/NoOpDerivedFieldResolver.java create mode 100644 server/src/main/java/org/opensearch/index/mapper/ObjectDerivedFieldType.java create mode 100644 server/src/test/java/org/opensearch/index/mapper/DerivedFieldResolverTests.java 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 297fc98764d07..95b4462d4f62e 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -743,7 +743,8 @@ public void apply(Settings value, Settings current, Settings previous) { RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING, RemoteStoreSettings.CLUSTER_REMOTE_MAX_TRANSLOG_READERS, - RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA + RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA, + SearchService.CLUSTER_ALLOW_DERIVED_FIELD_SETTING ) ) ); 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 980c432774f6e..6fe8dec9c21b1 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -237,7 +237,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { // Settings for concurrent segment search IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING, - + IndexSettings.ALLOW_DERIVED_FIELDS, // validate that built-in similarities don't get redefined Setting.groupSetting("index.similarity.", (s) -> { Map groups = s.getAsGroups(); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 6c0ab2f6b0153..613e93698d683 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -151,6 +151,14 @@ public static IndexMergePolicy fromString(String text) { true, Property.IndexScope ); + + public static final Setting ALLOW_DERIVED_FIELDS = Setting.boolSetting( + "index.query.derived_field.enabled", + true, + Property.Dynamic, + Property.IndexScope + ); + public static final Setting INDEX_TRANSLOG_SYNC_INTERVAL_SETTING = Setting.timeSetting( "index.translog.sync_interval", TimeValue.timeValueSeconds(5), @@ -763,6 +771,7 @@ public static IndexMergePolicy fromString(String text) { private final boolean assignedOnRemoteNode; private final RemoteStorePathStrategy remoteStorePathStrategy; private final boolean isTranslogMetadataEnabled; + private volatile boolean allowDerivedField; /** * The maximum age of a retention lease before it is considered expired. @@ -856,6 +865,10 @@ private void setDefaultFields(List defaultFields) { this.defaultFields = defaultFields; } + private void setAllowDerivedField(boolean allowDerivedField) { + this.allowDerivedField = allowDerivedField; + } + /** * Returns true if query string parsing should be lenient. The default is false */ @@ -884,6 +897,13 @@ public boolean isDefaultAllowUnmappedFields() { return defaultAllowUnmappedFields; } + /** + * Returns true if queries are allowed to define and use derived fields. The default is true + */ + public boolean isDerivedFieldAllowed() { + return allowDerivedField; + } + /** * Creates a new {@link IndexSettings} instance. The given node settings will be merged with the settings in the metadata * while index level settings will overwrite node settings. @@ -931,6 +951,7 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti this.queryStringAnalyzeWildcard = QUERY_STRING_ANALYZE_WILDCARD.get(nodeSettings); this.queryStringAllowLeadingWildcard = QUERY_STRING_ALLOW_LEADING_WILDCARD.get(nodeSettings); this.defaultAllowUnmappedFields = scopedSettings.get(ALLOW_UNMAPPED); + this.allowDerivedField = scopedSettings.get(ALLOW_DERIVED_FIELDS); this.durability = scopedSettings.get(INDEX_TRANSLOG_DURABILITY_SETTING); defaultFields = scopedSettings.get(DEFAULT_FIELD_SETTING); syncInterval = INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.get(settings); @@ -1105,6 +1126,7 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti INDEX_DOC_ID_FUZZY_SET_FALSE_POSITIVE_PROBABILITY_SETTING, this::setDocIdFuzzySetFalsePositiveProbability ); + scopedSettings.addSettingsUpdateConsumer(ALLOW_DERIVED_FIELDS, this::setAllowDerivedField); } private void setSearchIdleAfter(TimeValue searchIdleAfter) { diff --git a/server/src/main/java/org/opensearch/index/mapper/DefaultDerivedFieldResolver.java b/server/src/main/java/org/opensearch/index/mapper/DefaultDerivedFieldResolver.java new file mode 100644 index 0000000000000..c577a4117247b --- /dev/null +++ b/server/src/main/java/org/opensearch/index/mapper/DefaultDerivedFieldResolver.java @@ -0,0 +1,229 @@ +/* + * 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.mapper; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.regex.Regex; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.script.Script; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static org.opensearch.index.mapper.FieldMapper.IGNORE_MALFORMED_SETTING; + +/** + * Accepts definition of DerivedField from search request in both forms: map parsed from SearchRequest and {@link DerivedField} defined using client. + * The object is initialized per search request and is responsible to resolve {@link DerivedFieldType} given a field name. + * It uses {@link FieldTypeInference} to infer field type for a nested field within DerivedField of {@link DerivedFieldSupportedTypes#OBJECT} type. + */ +public class DefaultDerivedFieldResolver implements DerivedFieldResolver { + private final QueryShardContext queryShardContext; + private final Map derivedFieldTypeMap = new ConcurrentHashMap<>(); + private final FieldTypeInference typeInference; + private static final Logger logger = LogManager.getLogger(DefaultDerivedFieldResolver.class); + + DefaultDerivedFieldResolver( + QueryShardContext queryShardContext, + Map derivedFieldsObject, + List derivedFields + ) { + this( + queryShardContext, + derivedFieldsObject, + derivedFields, + new FieldTypeInference( + queryShardContext.index().getName(), + queryShardContext.getMapperService(), + queryShardContext.getIndexReader() + ) + ); + } + + DefaultDerivedFieldResolver( + QueryShardContext queryShardContext, + Map derivedFieldsObject, + List derivedFields, + FieldTypeInference typeInference + ) { + this.queryShardContext = queryShardContext; + initDerivedFieldTypes(derivedFieldsObject, derivedFields); + this.typeInference = typeInference; + } + + @Override + public Set resolvePattern(String pattern) { + Set derivedFields = new HashSet<>(); + if (queryShardContext != null && queryShardContext.getMapperService() != null) { + for (MappedFieldType fieldType : queryShardContext.getMapperService().fieldTypes()) { + if (Regex.simpleMatch(pattern, fieldType.name()) && fieldType instanceof DerivedFieldType) { + derivedFields.add(fieldType.name()); + } + } + } + for (String fieldName : derivedFieldTypeMap.keySet()) { + if (Regex.simpleMatch(pattern, fieldName)) { + derivedFields.add(fieldName); + } + } + return derivedFields; + } + + /** + * Resolves the fieldName. The search request definitions are given precedence over derived fields definitions in the index mapping. + * It caches the response for previously resolved field names + * @param fieldName name of the field. It also accepts nested derived field + * @return DerivedFieldType if resolved successfully, a null otherwise. + */ + @Override + public DerivedFieldType resolve(String fieldName) { + return Optional.ofNullable(resolveUsingSearchDefinitions(fieldName)).orElseGet(() -> resolveUsingMappings(fieldName)); + } + + private DerivedFieldType resolveUsingSearchDefinitions(String fieldName) { + return Optional.ofNullable(derivedFieldTypeMap.get(fieldName)) + .orElseGet( + () -> Optional.ofNullable((DerivedFieldType) getParentDerivedField(fieldName)) + .map( + // compute and cache nested derived field + parentDerivedField -> derivedFieldTypeMap.computeIfAbsent( + fieldName, + f -> this.resolveNestedField(f, parentDerivedField) + ) + ) + .orElse(null) + ); + } + + private DerivedFieldType resolveNestedField(String fieldName, DerivedFieldType parentDerivedField) { + Objects.requireNonNull(parentDerivedField); + try { + Script script = parentDerivedField.derivedField.getScript(); + String nestedType = explicitTypeFromParent(parentDerivedField.derivedField, fieldName.substring(fieldName.indexOf(".") + 1)); + if (nestedType == null) { + Mapper inferredFieldMapper = typeInference.infer( + getValueFetcher(fieldName, script, parentDerivedField.derivedField.getIgnoreMalformed()) + ); + if (inferredFieldMapper != null) { + nestedType = inferredFieldMapper.typeName(); + } + } + if (nestedType != null) { + DerivedField derivedField = new DerivedField(fieldName, nestedType, script); + if (parentDerivedField.derivedField.getProperties() != null) { + derivedField.setProperties(parentDerivedField.derivedField.getProperties()); + } + if (parentDerivedField.derivedField.getPrefilterField() != null) { + derivedField.setPrefilterField(parentDerivedField.derivedField.getPrefilterField()); + } + if (parentDerivedField.derivedField.getFormat() != null) { + derivedField.setFormat(parentDerivedField.derivedField.getFormat()); + } + if (parentDerivedField.derivedField.getIgnoreMalformed()) { + derivedField.setIgnoreMalformed(parentDerivedField.derivedField.getIgnoreMalformed()); + } + return getDerivedFieldType(derivedField); + } else { + logger.warn( + "Field type cannot be inferred. Ensure the field {} is not rare across entire index or provide explicit mapping using [properties] under parent object [{}] ", + fieldName, + parentDerivedField.derivedField.getName() + ); + } + } catch (IOException e) { + logger.warn(e.getMessage()); + } + return null; + } + + private MappedFieldType getParentDerivedField(String fieldName) { + if (fieldName.contains(".")) { + return resolve(fieldName.split("\\.")[0]); + } + return null; + } + + private static String explicitTypeFromParent(DerivedField parentDerivedField, String subField) { + if (parentDerivedField == null) { + return null; + } + return parentDerivedField.getNestedFieldType(subField); + } + + ValueFetcher getValueFetcher(String fieldName, Script script, boolean ignoreMalformed) { + String subFieldName = fieldName.substring(fieldName.indexOf(".") + 1); + return new ObjectDerivedFieldType.ObjectDerivedFieldValueFetcher( + subFieldName, + DerivedFieldType.getDerivedFieldLeafFactory(script, queryShardContext, queryShardContext.lookup()), + o -> o, // raw object returned will be used to infer the type without modifying it + ignoreMalformed + ); + } + + private void initDerivedFieldTypes(Map derivedFieldsObject, List derivedFields) { + if (derivedFieldsObject != null && !derivedFieldsObject.isEmpty()) { + Map derivedFieldObject = new HashMap<>(); + derivedFieldObject.put(DerivedFieldMapper.CONTENT_TYPE, derivedFieldsObject); + derivedFieldTypeMap.putAll(getAllDerivedFieldTypeFromObject(derivedFieldObject)); + } + if (derivedFields != null) { + for (DerivedField derivedField : derivedFields) { + derivedFieldTypeMap.put(derivedField.getName(), getDerivedFieldType(derivedField)); + } + } + } + + private Map getAllDerivedFieldTypeFromObject(Map derivedFieldObject) { + Map derivedFieldTypes = new HashMap<>(); + DocumentMapper documentMapper = queryShardContext.getMapperService() + .documentMapperParser() + .parse(DerivedFieldMapper.CONTENT_TYPE, derivedFieldObject); + if (documentMapper != null && documentMapper.mappers() != null) { + for (Mapper mapper : documentMapper.mappers()) { + if (mapper instanceof DerivedFieldMapper) { + DerivedFieldType derivedFieldType = ((DerivedFieldMapper) mapper).fieldType(); + derivedFieldTypes.put(derivedFieldType.name(), derivedFieldType); + } + } + } + return derivedFieldTypes; + } + + private DerivedFieldType getDerivedFieldType(DerivedField derivedField) { + Mapper.BuilderContext builderContext = new Mapper.BuilderContext( + queryShardContext.getMapperService().getIndexSettings().getSettings(), + new ContentPath(1) + ); + DerivedFieldMapper.Builder builder = new DerivedFieldMapper.Builder( + derivedField, + queryShardContext.getMapperService().getIndexAnalyzers(), + null, + IGNORE_MALFORMED_SETTING.getDefault(queryShardContext.getIndexSettings().getSettings()) + ); + return builder.build(builderContext).fieldType(); + } + + private DerivedFieldType resolveUsingMappings(String name) { + if (queryShardContext != null && queryShardContext.getMapperService() != null) { + MappedFieldType mappedFieldType = queryShardContext.getMapperService().fieldType(name); + if (mappedFieldType instanceof DerivedFieldType) { + return (DerivedFieldType) mappedFieldType; + } + } + return null; + } +} diff --git a/server/src/main/java/org/opensearch/index/mapper/DerivedField.java b/server/src/main/java/org/opensearch/index/mapper/DerivedField.java index b502e41cbb97b..249b60a1c4ec5 100644 --- a/server/src/main/java/org/opensearch/index/mapper/DerivedField.java +++ b/server/src/main/java/org/opensearch/index/mapper/DerivedField.java @@ -30,7 +30,7 @@ public class DerivedField implements Writeable, ToXContentFragment { private final String name; private final String type; private final Script script; - private String sourceIndexedField; + private String prefilterField; private Map properties; private Boolean ignoreMalformed; private String format; @@ -49,7 +49,7 @@ public DerivedField(StreamInput in) throws IOException { if (in.readBoolean()) { properties = in.readMap(); } - sourceIndexedField = in.readOptionalString(); + prefilterField = in.readOptionalString(); format = in.readOptionalString(); ignoreMalformed = in.readOptionalBoolean(); } @@ -67,7 +67,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(true); out.writeMap(properties); } - out.writeOptionalString(sourceIndexedField); + out.writeOptionalString(prefilterField); out.writeOptionalString(format); out.writeOptionalBoolean(ignoreMalformed); } @@ -81,8 +81,8 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par if (properties != null) { builder.field("properties", properties); } - if (sourceIndexedField != null) { - builder.field("source_indexed_field", sourceIndexedField); + if (prefilterField != null) { + builder.field("prefilter_field", prefilterField); } if (format != null) { builder.field("format", format); @@ -110,8 +110,15 @@ public Map getProperties() { return properties; } - public String getSourceIndexedField() { - return sourceIndexedField; + public String getNestedFieldType(String fieldName) { + if (properties == null || properties.isEmpty() || fieldName == null || fieldName.isEmpty()) { + return null; + } + return (String) properties.get(fieldName); + } + + public String getPrefilterField() { + return prefilterField; } public String getFormat() { @@ -126,8 +133,8 @@ public void setProperties(Map properties) { this.properties = properties; } - public void setSourceIndexedField(String sourceIndexedField) { - this.sourceIndexedField = sourceIndexedField; + public void setPrefilterField(String prefilterField) { + this.prefilterField = prefilterField; } public void setFormat(String format) { @@ -140,7 +147,7 @@ public void setIgnoreMalformed(boolean ignoreMalformed) { @Override public int hashCode() { - return Objects.hash(name, type, script, sourceIndexedField, properties, ignoreMalformed, format); + return Objects.hash(name, type, script, prefilterField, properties, ignoreMalformed, format); } @Override @@ -155,7 +162,7 @@ public boolean equals(Object obj) { return Objects.equals(name, other.name) && Objects.equals(type, other.type) && Objects.equals(script, other.script) - && Objects.equals(sourceIndexedField, other.sourceIndexedField) + && Objects.equals(prefilterField, other.prefilterField) && Objects.equals(properties, other.properties) && Objects.equals(ignoreMalformed, other.ignoreMalformed) && Objects.equals(format, other.format); diff --git a/server/src/main/java/org/opensearch/index/mapper/DerivedFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/DerivedFieldMapper.java index c6ae71320c35c..e08e46e1ea969 100644 --- a/server/src/main/java/org/opensearch/index/mapper/DerivedFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/DerivedFieldMapper.java @@ -9,16 +9,20 @@ package org.opensearch.index.mapper; import org.apache.lucene.index.IndexableField; +import org.opensearch.common.time.DateFormatter; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.analysis.IndexAnalyzers; import org.opensearch.script.Script; import java.io.IOException; import java.util.Arrays; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.function.Function; +import static org.opensearch.index.mapper.DateFieldMapper.getDefaultDateTimeFormatter; + /** * A field mapper for derived fields * @@ -28,6 +32,8 @@ public class DerivedFieldMapper extends ParametrizedFieldMapper { public static final String CONTENT_TYPE = "derived"; + protected final IndexAnalyzers indexAnalyzers; + private static DerivedFieldMapper toType(FieldMapper in) { return (DerivedFieldMapper) in; } @@ -38,62 +44,180 @@ private static DerivedFieldMapper toType(FieldMapper in) { * @opensearch.internal */ public static class Builder extends ParametrizedFieldMapper.Builder { - // TODO: The type of parameter may change here if the actual underlying FieldType object is needed - private final Parameter type = Parameter.stringParam("type", false, m -> toType(m).type, ""); + private final Parameter type = Parameter.stringParam("type", true, m -> toType(m).type, ""); + private final IndexAnalyzers indexAnalyzers; + private final boolean defaultIgnoreMalformed; + private final DateFormatter defaultDateFormatter; private final Parameter