From 83997fdc864eeb6fed4495d3365b086e376cbb74 Mon Sep 17 00:00:00 2001 From: Varun Lodaya Date: Mon, 13 May 2024 17:05:25 +0530 Subject: [PATCH 1/7] Updating security reach out email address (#13633) Signed-off-by: varun-lodaya --- README.md | 2 +- SECURITY.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 748f8a366ecc8..aff18018960d5 100644 --- a/README.md +++ b/README.md @@ -45,7 +45,7 @@ This project has adopted the [Amazon Open Source Code of Conduct](CODE_OF_CONDUCT.md). For more information see the [Code of Conduct FAQ](https://aws.github.io/code-of-conduct-faq), or contact [opensource-codeofconduct@amazon.com](mailto:opensource-codeofconduct@amazon.com) with any additional questions or comments. ## Security -If you discover a potential security issue in this project we ask that you notify AWS/Amazon Security via our [vulnerability reporting page](http://aws.amazon.com/security/vulnerability-reporting/) or directly via email to aws-security@amazon.com. Please do **not** create a public GitHub issue. +If you discover a potential security issue in this project we ask that you notify OpenSearch Security directly via email to security@opensearch.org. Please do **not** create a public GitHub issue. ## License diff --git a/SECURITY.md b/SECURITY.md index b86292104335f..be4ac7463864a 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -1,3 +1,3 @@ ## Reporting a Vulnerability -If you discover a potential security issue in this project we ask that you notify AWS/Amazon Security via our [vulnerability reporting page](http://aws.amazon.com/security/vulnerability-reporting/) or directly via email to aws-security@amazon.com. Please do **not** create a public GitHub issue. +If you discover a potential security issue in this project we ask that you notify OpenSearch Security directly via email to security@opensearch.org. Please do **not** create a public GitHub issue. From c328c18f69c08b21aa8a76af270b04a70c5a8069 Mon Sep 17 00:00:00 2001 From: Vikas Bansal <43470111+vikasvb90@users.noreply.github.com> Date: Mon, 13 May 2024 17:06:18 +0530 Subject: [PATCH 2/7] [Remote Store] Permit backed futures to prevent timeouts during upload bursts (#12159) Signed-off-by: vikasvb90 --- .../s3/S3BlobStoreRepositoryTests.java | 17 +- .../s3/GenericStatsMetricPublisher.java | 90 ++++++ .../repositories/s3/S3BlobContainer.java | 62 +++- .../repositories/s3/S3BlobStore.java | 34 ++- .../repositories/s3/S3Repository.java | 68 ++++- .../repositories/s3/S3RepositoryPlugin.java | 116 +++++++- .../s3/S3TransferRejectedException.java | 20 ++ .../s3/async/AsyncPartsHandler.java | 135 ++++++--- .../s3/async/AsyncTransferManager.java | 134 ++++++--- .../s3/async/SizeBasedBlockingQ.java | 230 +++++++++++++++ .../s3/async/TransferSemaphoresHolder.java | 186 ++++++++++++ .../repositories/s3/async/UploadRequest.java | 2 +- .../s3/RepositoryCredentialsTests.java | 17 +- .../s3/S3BlobContainerMockClientTests.java | 88 +++++- .../s3/S3BlobContainerRetriesTests.java | 51 +++- .../repositories/s3/S3RepositoryTests.java | 6 +- .../s3/async/AsyncTransferManagerTests.java | 12 +- .../s3/async/SizeBasedBlockingQTests.java | 102 +++++++ .../async/TransferSemaphoresHolderTests.java | 276 ++++++++++++++++++ .../common/blobstore/BlobStore.java | 1 + .../blobstore/stream/write/WritePriority.java | 7 +- .../shard/RemoteStoreRefreshListener.java | 6 +- .../index/store/RemoteDirectory.java | 12 +- .../store/RemoteSegmentStoreDirectory.java | 4 +- .../index/store/RemoteDirectoryTests.java | 6 +- .../RemoteSegmentStoreDirectoryTests.java | 4 +- 26 files changed, 1555 insertions(+), 131 deletions(-) create mode 100644 plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/GenericStatsMetricPublisher.java create mode 100644 plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3TransferRejectedException.java create mode 100644 plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQ.java create mode 100644 plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolder.java create mode 100644 plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQTests.java create mode 100644 plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolderTests.java 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 da2c6e8c1b0ee..21184380d54a9 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 @@ -249,7 +249,22 @@ protected S3Repository createRepository( ClusterService clusterService, RecoverySettings recoverySettings ) { - return new S3Repository(metadata, registry, service, clusterService, recoverySettings, null, null, null, null, null, false) { + return new S3Repository( + metadata, + registry, + service, + clusterService, + recoverySettings, + null, + null, + null, + null, + null, + false, + null, + null, + null + ) { @Override public BlobStore blobStore() { diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/GenericStatsMetricPublisher.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/GenericStatsMetricPublisher.java new file mode 100644 index 0000000000000..136fd68223354 --- /dev/null +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/GenericStatsMetricPublisher.java @@ -0,0 +1,90 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.repositories.s3; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Generic stats of repository-s3 plugin. + */ +public class GenericStatsMetricPublisher { + + private final AtomicLong normalPriorityQSize = new AtomicLong(); + private final AtomicInteger normalPriorityPermits = new AtomicInteger(); + private final AtomicLong lowPriorityQSize = new AtomicLong(); + private final AtomicInteger lowPriorityPermits = new AtomicInteger(); + private final long normalPriorityQCapacity; + private final int maxNormalPriorityPermits; + private final long lowPriorityQCapacity; + private final int maxLowPriorityPermits; + + public GenericStatsMetricPublisher( + long normalPriorityQCapacity, + int maxNormalPriorityPermits, + long lowPriorityQCapacity, + int maxLowPriorityPermits + ) { + this.normalPriorityQCapacity = normalPriorityQCapacity; + this.maxNormalPriorityPermits = maxNormalPriorityPermits; + this.lowPriorityQCapacity = lowPriorityQCapacity; + this.maxLowPriorityPermits = maxLowPriorityPermits; + } + + public void updateNormalPriorityQSize(long qSize) { + normalPriorityQSize.addAndGet(qSize); + } + + public void updateLowPriorityQSize(long qSize) { + lowPriorityQSize.addAndGet(qSize); + } + + public void updateNormalPermits(boolean increment) { + if (increment) { + normalPriorityPermits.incrementAndGet(); + } else { + normalPriorityPermits.decrementAndGet(); + } + } + + public void updateLowPermits(boolean increment) { + if (increment) { + lowPriorityPermits.incrementAndGet(); + } else { + lowPriorityPermits.decrementAndGet(); + } + } + + public long getNormalPriorityQSize() { + return normalPriorityQSize.get(); + } + + public int getAcquiredNormalPriorityPermits() { + return normalPriorityPermits.get(); + } + + public long getLowPriorityQSize() { + return lowPriorityQSize.get(); + } + + public int getAcquiredLowPriorityPermits() { + return lowPriorityPermits.get(); + } + + Map stats() { + final Map results = new HashMap<>(); + results.put("NormalPriorityQUtilization", (normalPriorityQSize.get() * 100) / normalPriorityQCapacity); + results.put("LowPriorityQUtilization", (lowPriorityQSize.get() * 100) / lowPriorityQCapacity); + results.put("NormalPriorityPermitsUtilization", (normalPriorityPermits.get() * 100L) / maxNormalPriorityPermits); + results.put("LowPriorityPermitsUtilization", (lowPriorityPermits.get() * 100L) / maxLowPriorityPermits); + return results; + } +} 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 14829a066ca3a..acf0c5e83a17b 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 @@ -90,6 +90,7 @@ import org.opensearch.core.common.Strings; import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.repositories.s3.async.SizeBasedBlockingQ; import org.opensearch.repositories.s3.async.UploadRequest; import org.opensearch.repositories.s3.utils.HttpRangeUtils; @@ -218,7 +219,14 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp writeContext.getMetadata() ); try { - if (uploadRequest.getContentLength() > ByteSizeUnit.GB.toBytes(10) && blobStore.isRedirectLargeUploads()) { + // If file size is greater than the queue capacity than SizeBasedBlockingQ will always reject the upload. + // Therefore, redirecting it to slow client. + if ((uploadRequest.getWritePriority() == WritePriority.LOW + && blobStore.getLowPrioritySizeBasedBlockingQ().isMaxCapacityBelowContentLength(uploadRequest.getContentLength()) == false) + || (uploadRequest.getWritePriority() != WritePriority.HIGH + && uploadRequest.getWritePriority() != WritePriority.URGENT + && blobStore.getNormalPrioritySizeBasedBlockingQ() + .isMaxCapacityBelowContentLength(uploadRequest.getContentLength()) == false)) { StreamContext streamContext = SocketAccess.doPrivileged( () -> writeContext.getStreamProvider(uploadRequest.getContentLength()) ); @@ -258,16 +266,30 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp } else { s3AsyncClient = amazonS3Reference.get().client(); } - CompletableFuture completableFuture = blobStore.getAsyncTransferManager() - .uploadObject(s3AsyncClient, uploadRequest, streamContext, blobStore.getStatsMetricPublisher()); - completableFuture.whenComplete((response, throwable) -> { - if (throwable == null) { - completionListener.onResponse(response); - } else { - Exception ex = throwable instanceof Error ? new Exception(throwable) : (Exception) throwable; - completionListener.onFailure(ex); - } - }); + + if (writeContext.getWritePriority() == WritePriority.URGENT + || writeContext.getWritePriority() == WritePriority.HIGH + || blobStore.isPermitBackedTransferEnabled() == false) { + createFileCompletableFuture(s3AsyncClient, uploadRequest, streamContext, completionListener); + } else if (writeContext.getWritePriority() == WritePriority.LOW) { + blobStore.getLowPrioritySizeBasedBlockingQ() + .produce( + new SizeBasedBlockingQ.Item( + writeContext.getFileSize(), + () -> createFileCompletableFuture(s3AsyncClient, uploadRequest, streamContext, completionListener) + ) + ); + } else if (writeContext.getWritePriority() == WritePriority.NORMAL) { + blobStore.getNormalPrioritySizeBasedBlockingQ() + .produce( + new SizeBasedBlockingQ.Item( + writeContext.getFileSize(), + () -> createFileCompletableFuture(s3AsyncClient, uploadRequest, streamContext, completionListener) + ) + ); + } else { + throw new IllegalStateException("Cannot perform upload for other priority types."); + } } } catch (Exception e) { logger.info("exception error from blob container for file {}", writeContext.getFileName()); @@ -275,6 +297,24 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp } } + private CompletableFuture createFileCompletableFuture( + S3AsyncClient s3AsyncClient, + UploadRequest uploadRequest, + StreamContext streamContext, + ActionListener completionListener + ) { + CompletableFuture completableFuture = blobStore.getAsyncTransferManager() + .uploadObject(s3AsyncClient, uploadRequest, streamContext, blobStore.getStatsMetricPublisher()); + return completableFuture.whenComplete((response, throwable) -> { + if (throwable == null) { + completionListener.onResponse(response); + } else { + Exception ex = throwable instanceof Error ? new Exception(throwable) : (Exception) throwable; + completionListener.onFailure(ex); + } + }); + } + @ExperimentalApi @Override public void readBlobAsync(String blobName, ActionListener listener) { 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 fc70fbb0db00e..de815f9202f44 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 @@ -45,6 +45,7 @@ import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.repositories.s3.async.AsyncExecutorContainer; import org.opensearch.repositories.s3.async.AsyncTransferManager; +import org.opensearch.repositories.s3.async.SizeBasedBlockingQ; import java.io.IOException; import java.util.Collections; @@ -56,6 +57,7 @@ import static org.opensearch.repositories.s3.S3Repository.BUFFER_SIZE_SETTING; import static org.opensearch.repositories.s3.S3Repository.BULK_DELETE_SIZE; import static org.opensearch.repositories.s3.S3Repository.CANNED_ACL_SETTING; +import static org.opensearch.repositories.s3.S3Repository.PERMIT_BACKED_TRANSFER_ENABLED; import static org.opensearch.repositories.s3.S3Repository.REDIRECT_LARGE_S3_UPLOAD; import static org.opensearch.repositories.s3.S3Repository.SERVER_SIDE_ENCRYPTION_SETTING; import static org.opensearch.repositories.s3.S3Repository.STORAGE_CLASS_SETTING; @@ -77,6 +79,8 @@ class S3BlobStore implements BlobStore { private volatile boolean uploadRetryEnabled; + private volatile boolean permitBackedTransferEnabled; + private volatile boolean serverSideEncryption; private volatile ObjectCannedACL cannedACL; @@ -94,6 +98,9 @@ class S3BlobStore implements BlobStore { private final AsyncExecutorContainer priorityExecutorBuilder; private final AsyncExecutorContainer normalExecutorBuilder; private final boolean multipartUploadEnabled; + private final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; + private final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; + private final GenericStatsMetricPublisher genericStatsMetricPublisher; S3BlobStore( S3Service service, @@ -109,7 +116,10 @@ class S3BlobStore implements BlobStore { AsyncTransferManager asyncTransferManager, AsyncExecutorContainer urgentExecutorBuilder, AsyncExecutorContainer priorityExecutorBuilder, - AsyncExecutorContainer normalExecutorBuilder + AsyncExecutorContainer normalExecutorBuilder, + SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, + SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ, + GenericStatsMetricPublisher genericStatsMetricPublisher ) { this.service = service; this.s3AsyncService = s3AsyncService; @@ -128,6 +138,10 @@ class S3BlobStore implements BlobStore { // Settings to initialize blobstore with. this.redirectLargeUploads = REDIRECT_LARGE_S3_UPLOAD.get(repositoryMetadata.settings()); this.uploadRetryEnabled = UPLOAD_RETRY_ENABLED.get(repositoryMetadata.settings()); + this.normalPrioritySizeBasedBlockingQ = normalPrioritySizeBasedBlockingQ; + this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; + this.genericStatsMetricPublisher = genericStatsMetricPublisher; + this.permitBackedTransferEnabled = PERMIT_BACKED_TRANSFER_ENABLED.get(repositoryMetadata.settings()); } @Override @@ -141,6 +155,7 @@ public void reload(RepositoryMetadata repositoryMetadata) { this.bulkDeletesSize = BULK_DELETE_SIZE.get(repositoryMetadata.settings()); this.redirectLargeUploads = REDIRECT_LARGE_S3_UPLOAD.get(repositoryMetadata.settings()); this.uploadRetryEnabled = UPLOAD_RETRY_ENABLED.get(repositoryMetadata.settings()); + this.permitBackedTransferEnabled = PERMIT_BACKED_TRANSFER_ENABLED.get(repositoryMetadata.settings()); } @Override @@ -168,6 +183,10 @@ public boolean isUploadRetryEnabled() { return uploadRetryEnabled; } + public boolean isPermitBackedTransferEnabled() { + return permitBackedTransferEnabled; + } + public String bucket() { return bucket; } @@ -184,6 +203,14 @@ public int getBulkDeletesSize() { return bulkDeletesSize; } + public SizeBasedBlockingQ getNormalPrioritySizeBasedBlockingQ() { + return normalPrioritySizeBasedBlockingQ; + } + + public SizeBasedBlockingQ getLowPrioritySizeBasedBlockingQ() { + return lowPrioritySizeBasedBlockingQ; + } + @Override public BlobContainer blobContainer(BlobPath path) { return new S3BlobContainer(path, this); @@ -201,7 +228,9 @@ public void close() throws IOException { @Override public Map stats() { - return statsMetricPublisher.getStats().toMap(); + Map stats = statsMetricPublisher.getStats().toMap(); + stats.putAll(genericStatsMetricPublisher.stats()); + return stats; } @Override @@ -211,6 +240,7 @@ public Map> extendedStats() { } Map> extendedStats = new HashMap<>(); statsMetricPublisher.getExtendedStats().forEach((k, v) -> extendedStats.put(k, v.toMap())); + extendedStats.put(Metric.GENERIC_STATS, genericStatsMetricPublisher.stats()); return extendedStats; } diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java index f7772a57c9afd..01b75c0b915f2 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java @@ -49,6 +49,7 @@ import org.opensearch.common.settings.SecureSetting; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.Strings; import org.opensearch.core.common.settings.SecureString; @@ -63,6 +64,7 @@ import org.opensearch.repositories.blobstore.MeteredBlobStoreRepository; import org.opensearch.repositories.s3.async.AsyncExecutorContainer; import org.opensearch.repositories.s3.async.AsyncTransferManager; +import org.opensearch.repositories.s3.async.SizeBasedBlockingQ; import org.opensearch.snapshots.SnapshotId; import org.opensearch.snapshots.SnapshotInfo; import org.opensearch.threadpool.Scheduler; @@ -156,6 +158,15 @@ class S3Repository extends MeteredBlobStoreRepository { Setting.Property.NodeScope ); + /** + * Whether large uploads need to be redirected to slow sync s3 client. + */ + static final Setting PERMIT_BACKED_TRANSFER_ENABLED = Setting.boolSetting( + "permit_backed_transfer_enabled", + true, + Setting.Property.NodeScope + ); + /** * Whether retry on uploads are enabled. This setting wraps inputstream with buffered stream to enable retries. */ @@ -193,6 +204,37 @@ class S3Repository extends MeteredBlobStoreRepository { true, Setting.Property.NodeScope ); + /** + * Percentage of total available permits to be available for priority transfers. + */ + public static Setting S3_PRIORITY_PERMIT_ALLOCATION_PERCENT = Setting.intSetting( + "s3_priority_permit_alloc_perc", + 70, + 21, + 80, + Setting.Property.NodeScope + ); + + /** + * Duration in minutes to wait for a permit in case no permit is available. + */ + public static Setting S3_PERMIT_WAIT_DURATION_MIN = Setting.intSetting( + "s3_permit_wait_duration_min", + 5, + 1, + 10, + Setting.Property.NodeScope + ); + + /** + * Number of transfer queue consumers + */ + public static Setting S3_TRANSFER_QUEUE_CONSUMERS = new Setting<>( + "s3_transfer_queue_consumers", + (s) -> Integer.toString(Math.max(5, OpenSearchExecutors.allocatedProcessors(s) * 2)), + (s) -> Setting.parseInt(s, 5, "s3_transfer_queue_consumers"), + Setting.Property.NodeScope + ); /** * Big files can be broken down into chunks during snapshotting if needed. Defaults to 1g. @@ -252,6 +294,9 @@ class S3Repository extends MeteredBlobStoreRepository { private final AsyncExecutorContainer priorityExecutorBuilder; private final AsyncExecutorContainer normalExecutorBuilder; private final Path pluginConfigPath; + private final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; + private final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; + private final GenericStatsMetricPublisher genericStatsMetricPublisher; private volatile int bulkDeletesSize; @@ -267,7 +312,10 @@ class S3Repository extends MeteredBlobStoreRepository { final AsyncExecutorContainer priorityExecutorBuilder, final AsyncExecutorContainer normalExecutorBuilder, final S3AsyncService s3AsyncService, - final boolean multipartUploadEnabled + final boolean multipartUploadEnabled, + final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, + final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ, + final GenericStatsMetricPublisher genericStatsMetricPublisher ) { this( metadata, @@ -281,7 +329,10 @@ class S3Repository extends MeteredBlobStoreRepository { normalExecutorBuilder, s3AsyncService, multipartUploadEnabled, - Path.of("") + Path.of(""), + normalPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ, + genericStatsMetricPublisher ); } @@ -300,7 +351,10 @@ class S3Repository extends MeteredBlobStoreRepository { final AsyncExecutorContainer normalExecutorBuilder, final S3AsyncService s3AsyncService, final boolean multipartUploadEnabled, - Path pluginConfigPath + Path pluginConfigPath, + final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, + final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ, + final GenericStatsMetricPublisher genericStatsMetricPublisher ) { super(metadata, namedXContentRegistry, clusterService, recoverySettings, buildLocation(metadata)); this.service = service; @@ -311,6 +365,9 @@ class S3Repository extends MeteredBlobStoreRepository { this.urgentExecutorBuilder = urgentExecutorBuilder; this.priorityExecutorBuilder = priorityExecutorBuilder; this.normalExecutorBuilder = normalExecutorBuilder; + this.normalPrioritySizeBasedBlockingQ = normalPrioritySizeBasedBlockingQ; + this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; + this.genericStatsMetricPublisher = genericStatsMetricPublisher; validateRepositoryMetadata(metadata); readRepositoryMetadata(); @@ -373,7 +430,10 @@ protected S3BlobStore createBlobStore() { asyncUploadUtils, urgentExecutorBuilder, priorityExecutorBuilder, - normalExecutorBuilder + normalExecutorBuilder, + normalPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ, + genericStatsMetricPublisher ); } diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RepositoryPlugin.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RepositoryPlugin.java index e7d2a4d024e60..110d91bfbd822 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RepositoryPlugin.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RepositoryPlugin.java @@ -41,6 +41,9 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; import org.opensearch.env.NodeEnvironment; @@ -53,6 +56,8 @@ import org.opensearch.repositories.s3.async.AsyncExecutorContainer; import org.opensearch.repositories.s3.async.AsyncTransferEventLoopGroup; import org.opensearch.repositories.s3.async.AsyncTransferManager; +import org.opensearch.repositories.s3.async.SizeBasedBlockingQ; +import org.opensearch.repositories.s3.async.TransferSemaphoresHolder; import org.opensearch.script.ScriptService; import org.opensearch.threadpool.ExecutorBuilder; import org.opensearch.threadpool.FixedExecutorBuilder; @@ -69,6 +74,8 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.function.Supplier; /** @@ -82,6 +89,8 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private static final String PRIORITY_STREAM_READER = "priority_stream_reader"; private static final String FUTURE_COMPLETION = "future_completion"; private static final String STREAM_READER = "stream_reader"; + private static final String LOW_TRANSFER_QUEUE_CONSUMER = "low_transfer_queue_consumer"; + private static final String NORMAL_TRANSFER_QUEUE_CONSUMER = "normal_transfer_queue_consumer"; protected final S3Service service; private final S3AsyncService s3AsyncService; @@ -91,6 +100,12 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private AsyncExecutorContainer urgentExecutorBuilder; private AsyncExecutorContainer priorityExecutorBuilder; private AsyncExecutorContainer normalExecutorBuilder; + private ExecutorService lowTransferQConsumerService; + private ExecutorService normalTransferQConsumerService; + private SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; + private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; + private TransferSemaphoresHolder transferSemaphoresHolder; + private GenericStatsMetricPublisher genericStatsMetricPublisher; public S3RepositoryPlugin(final Settings settings, final Path configPath) { this(settings, configPath, new S3Service(configPath), new S3AsyncService(configPath)); @@ -120,9 +135,36 @@ public List> getExecutorBuilders(Settings settings) { TimeValue.timeValueMinutes(5) ) ); + executorBuilders.add( + new FixedExecutorBuilder( + settings, + LOW_TRANSFER_QUEUE_CONSUMER, + lowPriorityTransferQConsumers(settings), + 10, + "thread_pool." + LOW_TRANSFER_QUEUE_CONSUMER + ) + ); + executorBuilders.add( + new FixedExecutorBuilder( + settings, + NORMAL_TRANSFER_QUEUE_CONSUMER, + normalPriorityTransferQConsumers(settings), + 10, + "thread_pool." + NORMAL_TRANSFER_QUEUE_CONSUMER + ) + ); return executorBuilders; } + private int lowPriorityTransferQConsumers(Settings settings) { + double lowPriorityAllocation = ((double) (100 - S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT.get(settings))) / 100; + return Math.max(2, (int) (lowPriorityAllocation * S3Repository.S3_TRANSFER_QUEUE_CONSUMERS.get(settings))); + } + + private int normalPriorityTransferQConsumers(Settings settings) { + return S3Repository.S3_TRANSFER_QUEUE_CONSUMERS.get(settings); + } + static int halfNumberOfProcessors(int numberOfProcessors) { return (numberOfProcessors + 1) / 2; } @@ -189,7 +231,67 @@ public Collection createComponents( threadPool.executor(STREAM_READER), new AsyncTransferEventLoopGroup(normalEventLoopThreads) ); - return Collections.emptyList(); + + this.lowTransferQConsumerService = threadPool.executor(LOW_TRANSFER_QUEUE_CONSUMER); + this.normalTransferQConsumerService = threadPool.executor(NORMAL_TRANSFER_QUEUE_CONSUMER); + + // High number of permit allocation because each op acquiring permit performs disk IO, computation and network IO. + int availablePermits = Math.max(allocatedProcessors(clusterService.getSettings()) * 4, 10); + double priorityPermitAllocation = ((double) S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT.get(clusterService.getSettings())) + / 100; + int normalPriorityPermits = (int) (priorityPermitAllocation * availablePermits); + int lowPriorityPermits = availablePermits - normalPriorityPermits; + + int normalPriorityConsumers = normalPriorityTransferQConsumers(clusterService.getSettings()); + int lowPriorityConsumers = lowPriorityTransferQConsumers(clusterService.getSettings()); + + ByteSizeValue normalPriorityQCapacity = new ByteSizeValue(normalPriorityConsumers * 10L, ByteSizeUnit.GB); + ByteSizeValue lowPriorityQCapacity = new ByteSizeValue(lowPriorityConsumers * 20L, ByteSizeUnit.GB); + + this.genericStatsMetricPublisher = new GenericStatsMetricPublisher( + normalPriorityQCapacity.getBytes(), + normalPriorityPermits, + lowPriorityQCapacity.getBytes(), + lowPriorityPermits + ); + + this.normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + normalPriorityQCapacity, + normalTransferQConsumerService, + normalPriorityConsumers, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL + ); + + LowPrioritySizeBasedBlockingQ lowPrioritySizeBasedBlockingQ = new LowPrioritySizeBasedBlockingQ( + lowPriorityQCapacity, + lowTransferQConsumerService, + lowPriorityConsumers, + genericStatsMetricPublisher + ); + this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; + this.transferSemaphoresHolder = new TransferSemaphoresHolder( + normalPriorityPermits, + lowPriorityPermits, + S3Repository.S3_PERMIT_WAIT_DURATION_MIN.get(clusterService.getSettings()), + TimeUnit.MINUTES, + genericStatsMetricPublisher + ); + + return CollectionUtils.arrayAsArrayList(this.normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ); + } + + // New class because in core, components are injected via guice only by instance creation due to which + // same binding types fail. + private static final class LowPrioritySizeBasedBlockingQ extends SizeBasedBlockingQ { + public LowPrioritySizeBasedBlockingQ( + ByteSizeValue capacity, + ExecutorService executorService, + int consumers, + GenericStatsMetricPublisher genericStatsMetricPublisher + ) { + super(capacity, executorService, consumers, genericStatsMetricPublisher, QueueEventType.LOW); + } } // proxy method for testing @@ -204,7 +306,8 @@ protected S3Repository createRepository( S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.get(clusterService.getSettings()).getBytes(), normalExecutorBuilder.getStreamReader(), priorityExecutorBuilder.getStreamReader(), - urgentExecutorBuilder.getStreamReader() + urgentExecutorBuilder.getStreamReader(), + transferSemaphoresHolder ); return new S3Repository( metadata, @@ -218,7 +321,10 @@ protected S3Repository createRepository( normalExecutorBuilder, s3AsyncService, S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING.get(clusterService.getSettings()), - configPath + configPath, + normalPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ, + genericStatsMetricPublisher ); } @@ -263,7 +369,9 @@ public List> getSettings() { S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING, S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING, S3Repository.REDIRECT_LARGE_S3_UPLOAD, - S3Repository.UPLOAD_RETRY_ENABLED + S3Repository.UPLOAD_RETRY_ENABLED, + S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT, + S3Repository.PERMIT_BACKED_TRANSFER_ENABLED ); } diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3TransferRejectedException.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3TransferRejectedException.java new file mode 100644 index 0000000000000..c9fa93ea0f5c3 --- /dev/null +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3TransferRejectedException.java @@ -0,0 +1,20 @@ +/* + * 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.repositories.s3; + +import org.opensearch.OpenSearchException; + +/** + * Thrown when transfer event is rejected due to breach in event queue size. + */ +public class S3TransferRejectedException extends OpenSearchException { + public S3TransferRejectedException(String msg) { + super(msg); + } +} diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncPartsHandler.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncPartsHandler.java index b4c4ed0ecaa75..4c95a0ffc5ec3 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncPartsHandler.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncPartsHandler.java @@ -23,6 +23,7 @@ import org.opensearch.common.StreamContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.io.InputStreamContainer; +import org.opensearch.repositories.s3.S3TransferRejectedException; import org.opensearch.repositories.s3.SocketAccess; import org.opensearch.repositories.s3.StatsMetricPublisher; import org.opensearch.repositories.s3.io.CheckedContainer; @@ -34,6 +35,8 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReferenceArray; /** @@ -41,7 +44,7 @@ */ public class AsyncPartsHandler { - private static Logger log = LogManager.getLogger(AsyncPartsHandler.class); + private static final Logger log = LogManager.getLogger(AsyncPartsHandler.class); /** * Uploads parts of the upload multipart request* @@ -55,9 +58,10 @@ public class AsyncPartsHandler { * @param completedParts Reference of completed parts * @param inputStreamContainers Checksum containers * @param statsMetricPublisher sdk metric publisher + * @param maxRetryablePartSize Max content size which can be used for retries in buffered streams. * @return list of completable futures - * @throws IOException thrown in case of an IO error */ + @SuppressWarnings({ "rawtypes", "unchecked" }) public static List> uploadParts( S3AsyncClient s3AsyncClient, ExecutorService executorService, @@ -69,35 +73,52 @@ public static List> uploadParts( AtomicReferenceArray completedParts, AtomicReferenceArray inputStreamContainers, StatsMetricPublisher statsMetricPublisher, - boolean uploadRetryEnabled - ) throws IOException { + boolean uploadRetryEnabled, + TransferSemaphoresHolder transferSemaphoresHolder, + long maxRetryablePartSize + ) throws InterruptedException { List> futures = new ArrayList<>(); + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); for (int partIdx = 0; partIdx < streamContext.getNumberOfParts(); partIdx++) { - InputStreamContainer inputStreamContainer = streamContext.provideStream(partIdx); - inputStreamContainers.set(partIdx, new CheckedContainer(inputStreamContainer.getContentLength())); - UploadPartRequest.Builder uploadPartRequestBuilder = UploadPartRequest.builder() - .bucket(uploadRequest.getBucket()) - .partNumber(partIdx + 1) - .key(uploadRequest.getKey()) - .uploadId(uploadId) - .overrideConfiguration(o -> o.addMetricPublisher(statsMetricPublisher.multipartUploadMetricCollector)) - .contentLength(inputStreamContainer.getContentLength()); - if (uploadRequest.doRemoteDataIntegrityCheck()) { - uploadPartRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32); - } - uploadPart( - s3AsyncClient, - executorService, - priorityExecutorService, - urgentExecutorService, - completedParts, - inputStreamContainers, - futures, - uploadPartRequestBuilder.build(), - inputStreamContainer, - uploadRequest, - uploadRetryEnabled + Semaphore semaphore = maybeAcquireSemaphore( + transferSemaphoresHolder, + requestContext, + uploadRequest.getWritePriority(), + uploadRequest.getKey() ); + try { + InputStreamContainer inputStreamContainer = streamContext.provideStream(partIdx); + inputStreamContainers.set(partIdx, new CheckedContainer(inputStreamContainer.getContentLength())); + UploadPartRequest.Builder uploadPartRequestBuilder = UploadPartRequest.builder() + .bucket(uploadRequest.getBucket()) + .partNumber(partIdx + 1) + .key(uploadRequest.getKey()) + .uploadId(uploadId) + .overrideConfiguration(o -> o.addMetricPublisher(statsMetricPublisher.multipartUploadMetricCollector)) + .contentLength(inputStreamContainer.getContentLength()); + if (uploadRequest.doRemoteDataIntegrityCheck()) { + uploadPartRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32); + } + uploadPart( + s3AsyncClient, + executorService, + priorityExecutorService, + urgentExecutorService, + completedParts, + inputStreamContainers, + futures, + uploadPartRequestBuilder.build(), + inputStreamContainer, + uploadRequest, + uploadRetryEnabled, + maxRetryablePartSize, + semaphore + ); + } catch (Exception ex) { + if (semaphore != null) { + semaphore.release(); + } + } } return futures; @@ -137,14 +158,54 @@ public static InputStream maybeRetryInputStream( InputStream inputStream, WritePriority writePriority, boolean uploadRetryEnabled, - long contentLength + long contentLength, + long maxRetryablePartSize ) { - if (uploadRetryEnabled == true && (writePriority == WritePriority.HIGH || writePriority == WritePriority.URGENT)) { - return new BufferedInputStream(inputStream, (int) (contentLength + 1)); + // Since we are backing uploads with limited permits, it is ok to use buffered stream. Maximum in-memory buffer + // would be (max permits * maxRetryablePartSize) excluding urgent + if (uploadRetryEnabled == true + && (contentLength <= maxRetryablePartSize || writePriority == WritePriority.HIGH || writePriority == WritePriority.URGENT)) { + return new UploadTrackedBufferedInputStream(inputStream, (int) (contentLength + 1)); } return inputStream; } + public static Semaphore maybeAcquireSemaphore( + TransferSemaphoresHolder transferSemaphoresHolder, + TransferSemaphoresHolder.RequestContext requestContext, + WritePriority writePriority, + String file + ) throws InterruptedException { + final TransferSemaphoresHolder.TypeSemaphore semaphore; + if (writePriority != WritePriority.HIGH && writePriority != WritePriority.URGENT) { + semaphore = transferSemaphoresHolder.acquirePermit(writePriority, requestContext); + if (semaphore == null) { + throw new S3TransferRejectedException("Permit not available for transfer of file " + file); + } + } else { + semaphore = null; + } + + return semaphore; + } + + /** + * Overridden stream to identify upload streams among all buffered stream instances for triaging. + */ + static class UploadTrackedBufferedInputStream extends BufferedInputStream { + AtomicBoolean closed = new AtomicBoolean(); + + public UploadTrackedBufferedInputStream(InputStream in, int length) { + super(in, length); + } + + @Override + public void close() throws IOException { + super.close(); + closed.set(true); + } + } + private static void uploadPart( S3AsyncClient s3AsyncClient, ExecutorService executorService, @@ -156,8 +217,11 @@ private static void uploadPart( UploadPartRequest uploadPartRequest, InputStreamContainer inputStreamContainer, UploadRequest uploadRequest, - boolean uploadRetryEnabled + boolean uploadRetryEnabled, + long maxRetryablePartSize, + Semaphore semaphore ) { + Integer partNumber = uploadPartRequest.partNumber(); ExecutorService streamReadExecutor; @@ -173,7 +237,8 @@ private static void uploadPart( inputStreamContainer.getInputStream(), uploadRequest.getWritePriority(), uploadRetryEnabled, - uploadPartRequest.contentLength() + uploadPartRequest.contentLength(), + maxRetryablePartSize ); CompletableFuture uploadPartResponseFuture = SocketAccess.doPrivileged( () -> s3AsyncClient.uploadPart( @@ -183,6 +248,10 @@ private static void uploadPart( ); CompletableFuture convertFuture = uploadPartResponseFuture.whenComplete((resp, throwable) -> { + if (semaphore != null) { + semaphore.release(); + } + try { inputStream.close(); } catch (IOException ex) { diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java index 80538059d17b8..0f9bf3be77d73 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java @@ -21,6 +21,7 @@ import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse; import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; import software.amazon.awssdk.services.s3.model.S3Exception; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.awssdk.utils.CompletableFutureUtils; @@ -48,6 +49,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.BiFunction; import java.util.function.Supplier; @@ -64,6 +66,10 @@ public final class AsyncTransferManager { private final ExecutorService priorityExecutorService; private final ExecutorService urgentExecutorService; private final long minimumPartSize; + private final long maxRetryablePartSize; + + @SuppressWarnings("rawtypes") + private final TransferSemaphoresHolder transferSemaphoresHolder; /** * The max number of parts on S3 side is 10,000 @@ -74,19 +80,22 @@ public final class AsyncTransferManager { * Construct a new object of AsyncTransferManager * * @param minimumPartSize The minimum part size for parallel multipart uploads - * @param executorService The stream reader {@link ExecutorService} for normal priority uploads - * @param priorityExecutorService The stream read {@link ExecutorService} for high priority uploads */ + @SuppressWarnings("rawtypes") public AsyncTransferManager( long minimumPartSize, ExecutorService executorService, ExecutorService priorityExecutorService, - ExecutorService urgentExecutorService + ExecutorService urgentExecutorService, + TransferSemaphoresHolder transferSemaphoresHolder ) { this.executorService = executorService; this.priorityExecutorService = priorityExecutorService; this.minimumPartSize = minimumPartSize; + // 10% buffer to allow additional metadata size in content such as encryption. + this.maxRetryablePartSize = (long) (minimumPartSize + 0.1 * minimumPartSize); this.urgentExecutorService = urgentExecutorService; + this.transferSemaphoresHolder = transferSemaphoresHolder; } /** @@ -108,7 +117,21 @@ public CompletableFuture uploadObject( try { if (streamContext.getNumberOfParts() == 1) { log.debug(() -> "Starting the upload as a single upload part request"); - uploadInOneChunk(s3AsyncClient, uploadRequest, streamContext.provideStream(0), returnFuture, statsMetricPublisher); + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + Semaphore semaphore = AsyncPartsHandler.maybeAcquireSemaphore( + transferSemaphoresHolder, + requestContext, + uploadRequest.getWritePriority(), + uploadRequest.getKey() + ); + try { + uploadInOneChunk(s3AsyncClient, uploadRequest, streamContext, returnFuture, statsMetricPublisher, semaphore); + } catch (Exception ex) { + if (semaphore != null) { + semaphore.release(); + } + throw ex; + } } else { log.debug(() -> "Starting the upload as multipart upload request"); uploadInParts(s3AsyncClient, uploadRequest, streamContext, returnFuture, statsMetricPublisher); @@ -146,21 +169,19 @@ private void uploadInParts( // Ensure cancellations are forwarded to the createMultipartUploadFuture future CompletableFutureUtils.forwardExceptionTo(returnFuture, createMultipartUploadFuture); - createMultipartUploadFuture.whenComplete((createMultipartUploadResponse, throwable) -> { - if (throwable != null) { - handleException(returnFuture, () -> "Failed to initiate multipart upload", throwable); - } else { - log.debug(() -> "Initiated new multipart upload, uploadId: " + createMultipartUploadResponse.uploadId()); - doUploadInParts( - s3AsyncClient, - uploadRequest, - streamContext, - returnFuture, - createMultipartUploadResponse.uploadId(), - statsMetricPublisher - ); - } - }); + String uploadId; + try { + // Block main thread here so that upload of parts doesn't get executed in future completion thread. + // We should never execute latent operation like acquisition of permit in future completion pool. + CreateMultipartUploadResponse createMultipartUploadResponse = createMultipartUploadFuture.get(); + uploadId = createMultipartUploadResponse.uploadId(); + log.debug(() -> "Initiated new multipart upload, uploadId: " + createMultipartUploadResponse.uploadId()); + } catch (Exception ex) { + handleException(returnFuture, () -> "Failed to initiate multipart upload", ex); + return; + } + + doUploadInParts(s3AsyncClient, uploadRequest, streamContext, returnFuture, uploadId, statsMetricPublisher); } private void doUploadInParts( @@ -189,7 +210,9 @@ private void doUploadInParts( completedParts, inputStreamContainers, statsMetricPublisher, - uploadRequest.isUploadRetryEnabled() + uploadRequest.isUploadRetryEnabled(), + transferSemaphoresHolder, + maxRetryablePartSize ); } catch (Exception ex) { try { @@ -320,12 +343,14 @@ public long calculateOptimalPartSize(long contentLengthOfSource, WritePriority w return (long) Math.max(optimalPartSize, minimumPartSize); } + @SuppressWarnings("unchecked") private void uploadInOneChunk( S3AsyncClient s3AsyncClient, UploadRequest uploadRequest, - InputStreamContainer inputStreamContainer, + StreamContext streamContext, CompletableFuture returnFuture, - StatsMetricPublisher statsMetricPublisher + StatsMetricPublisher statsMetricPublisher, + Semaphore semaphore ) { PutObjectRequest.Builder putObjectRequestBuilder = PutObjectRequest.builder() .bucket(uploadRequest.getBucket()) @@ -340,6 +365,7 @@ private void uploadInOneChunk( putObjectRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32); putObjectRequestBuilder.checksumCRC32(base64StringFromLong(uploadRequest.getExpectedChecksum())); } + PutObjectRequest putObjectRequest = putObjectRequestBuilder.build(); ExecutorService streamReadExecutor; if (uploadRequest.getWritePriority() == WritePriority.URGENT) { streamReadExecutor = urgentExecutorService; @@ -349,25 +375,33 @@ private void uploadInOneChunk( streamReadExecutor = executorService; } - InputStream inputStream = AsyncPartsHandler.maybeRetryInputStream( - inputStreamContainer.getInputStream(), - uploadRequest.getWritePriority(), - uploadRequest.isUploadRetryEnabled(), - uploadRequest.getContentLength() - ); - CompletableFuture putObjectFuture = SocketAccess.doPrivileged( - () -> s3AsyncClient.putObject( - putObjectRequestBuilder.build(), - AsyncRequestBody.fromInputStream(inputStream, inputStreamContainer.getContentLength(), streamReadExecutor) - ).handle((resp, throwable) -> { - try { - inputStream.close(); - } catch (IOException e) { - log.error( - () -> new ParameterizedMessage("Failed to close stream while uploading single file {}.", uploadRequest.getKey()), - e - ); - } + CompletableFuture putObjectFuture = SocketAccess.doPrivileged(() -> { + InputStream inputStream = null; + CompletableFuture putObjectRespFuture; + try { + InputStreamContainer inputStreamContainer = streamContext.provideStream(0); + inputStream = AsyncPartsHandler.maybeRetryInputStream( + inputStreamContainer.getInputStream(), + uploadRequest.getWritePriority(), + uploadRequest.isUploadRetryEnabled(), + uploadRequest.getContentLength(), + maxRetryablePartSize + ); + AsyncRequestBody asyncRequestBody = AsyncRequestBody.fromInputStream( + inputStream, + inputStreamContainer.getContentLength(), + streamReadExecutor + ); + putObjectRespFuture = s3AsyncClient.putObject(putObjectRequest, asyncRequestBody); + } catch (Exception e) { + releaseResourcesSafely(semaphore, inputStream, uploadRequest.getKey()); + return CompletableFuture.failedFuture(e); + } + + InputStream finalInputStream = inputStream; + return putObjectRespFuture.handle((resp, throwable) -> { + releaseResourcesSafely(semaphore, finalInputStream, uploadRequest.getKey()); + if (throwable != null) { Throwable unwrappedThrowable = ExceptionsHelper.unwrap(throwable, S3Exception.class); if (unwrappedThrowable != null) { @@ -395,13 +429,27 @@ private void uploadInOneChunk( } return null; - }) - ); + }); + }); CompletableFutureUtils.forwardExceptionTo(returnFuture, putObjectFuture); CompletableFutureUtils.forwardResultTo(putObjectFuture, returnFuture); } + private void releaseResourcesSafely(Semaphore semaphore, InputStream inputStream, String file) { + if (semaphore != null) { + semaphore.release(); + } + + if (inputStream != null) { + try { + inputStream.close(); + } catch (IOException e) { + log.error(() -> new ParameterizedMessage("Failed to close stream while uploading single file {}.", file), e); + } + } + } + private void deleteUploadedObject(S3AsyncClient s3AsyncClient, UploadRequest uploadRequest) { DeleteObjectRequest deleteObjectRequest = DeleteObjectRequest.builder() .bucket(uploadRequest.getBucket()) diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQ.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQ.java new file mode 100644 index 0000000000000..170c80f5d4db6 --- /dev/null +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQ.java @@ -0,0 +1,230 @@ +/* + * 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.repositories.s3.async; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.AlreadyClosedException; +import org.opensearch.common.lifecycle.AbstractLifecycleComponent; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.repositories.s3.GenericStatsMetricPublisher; +import org.opensearch.repositories.s3.S3TransferRejectedException; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Queue implementation to accept events based on their storage attribute. If size of queue is breached, then transfer + * event is rejected. + */ +public class SizeBasedBlockingQ extends AbstractLifecycleComponent { + private static final Logger log = LogManager.getLogger(SizeBasedBlockingQ.class); + + protected final LinkedBlockingQueue queue; + protected final Lock lock; + protected final Condition notEmpty; + + protected final AtomicLong currentSize; + protected final ByteSizeValue capacity; + protected final AtomicBoolean closed; + protected final ExecutorService executorService; + protected final int consumers; + private final GenericStatsMetricPublisher genericStatsMetricPublisher; + private final QueueEventType queueEventType; + + /** + * Constructor to create sized based blocking queue. + */ + public SizeBasedBlockingQ( + ByteSizeValue capacity, + ExecutorService executorService, + int consumers, + GenericStatsMetricPublisher genericStatsMetricPublisher, + QueueEventType queueEventType + ) { + this.queue = new LinkedBlockingQueue<>(); + this.lock = new ReentrantLock(); + this.notEmpty = lock.newCondition(); + this.currentSize = new AtomicLong(); + this.capacity = capacity; + this.closed = new AtomicBoolean(); + this.executorService = executorService; + this.consumers = consumers; + this.genericStatsMetricPublisher = genericStatsMetricPublisher; + this.queueEventType = queueEventType; + } + + public enum QueueEventType { + NORMAL, + LOW; + } + + @Override + protected void doStart() { + for (int worker = 0; worker < consumers; worker++) { + Thread consumer = new Consumer(queue, currentSize, lock, notEmpty, closed, genericStatsMetricPublisher, queueEventType); + executorService.submit(consumer); + } + } + + /** + * Add an item to the queue + */ + public void produce(Item item) throws InterruptedException { + if (item == null || item.size <= 0) { + throw new IllegalStateException("Invalid item input to produce."); + } + log.debug(() -> "Transfer queue event received of size: " + item.size + ". Current queue utilisation: " + currentSize.get()); + + if (currentSize.get() + item.size >= capacity.getBytes()) { + throw new S3TransferRejectedException("S3 Transfer queue capacity reached"); + } + + final Lock lock = this.lock; + final AtomicLong currentSize = this.currentSize; + lock.lock(); + try { + if (currentSize.get() + item.size >= capacity.getBytes()) { + throw new S3TransferRejectedException("S3 Transfer queue capacity reached"); + } + if (closed.get()) { + throw new AlreadyClosedException("Transfer queue is already closed."); + } + queue.put(item); + currentSize.addAndGet(item.size); + notEmpty.signalAll(); + updateStats(item.size, queueEventType, genericStatsMetricPublisher); + } finally { + lock.unlock(); + } + + } + + private static void updateStats(long itemSize, QueueEventType queueEventType, GenericStatsMetricPublisher genericStatsMetricPublisher) { + if (queueEventType == QueueEventType.NORMAL) { + genericStatsMetricPublisher.updateNormalPriorityQSize(itemSize); + } else if (queueEventType == QueueEventType.LOW) { + genericStatsMetricPublisher.updateLowPriorityQSize(itemSize); + } + } + + public int getSize() { + return queue.size(); + } + + public boolean isMaxCapacityBelowContentLength(long contentLength) { + return contentLength < capacity.getBytes(); + } + + protected static class Consumer extends Thread { + private final LinkedBlockingQueue queue; + private final Lock lock; + private final Condition notEmpty; + private final AtomicLong currentSize; + private final AtomicBoolean closed; + private final GenericStatsMetricPublisher genericStatsMetricPublisher; + private final QueueEventType queueEventType; + + public Consumer( + LinkedBlockingQueue queue, + AtomicLong currentSize, + Lock lock, + Condition notEmpty, + AtomicBoolean closed, + GenericStatsMetricPublisher genericStatsMetricPublisher, + QueueEventType queueEventType + ) { + this.queue = queue; + this.lock = lock; + this.notEmpty = notEmpty; + this.currentSize = currentSize; + this.closed = closed; + this.genericStatsMetricPublisher = genericStatsMetricPublisher; + this.queueEventType = queueEventType; + } + + @Override + public void run() { + while (true) { + try { + consume(); + } catch (AlreadyClosedException ex) { + return; + } catch (Exception ex) { + log.error("Failed to consume transfer event", ex); + } + } + } + + private void consume() throws InterruptedException { + final Lock lock = this.lock; + final AtomicLong currentSize = this.currentSize; + lock.lock(); + Item item; + try { + if (closed.get()) { + throw new AlreadyClosedException("transfer queue closed"); + } + while (currentSize.get() == 0) { + notEmpty.await(); + if (closed.get()) { + throw new AlreadyClosedException("transfer queue closed"); + } + } + + item = queue.take(); + currentSize.addAndGet(-item.size); + updateStats(-item.size, queueEventType, genericStatsMetricPublisher); + } finally { + lock.unlock(); + } + + try { + item.consumable.run(); + } catch (Exception ex) { + log.error("Exception on executing item consumable", ex); + } + } + + } + + public static class Item { + private final long size; + private final Runnable consumable; + + public Item(long size, Runnable consumable) { + this.size = size; + this.consumable = consumable; + } + } + + @Override + protected void doStop() { + doClose(); + } + + @Override + protected void doClose() { + lock.lock(); + try { + if (closed.get() == true) { + return; + } + closed.set(true); + notEmpty.signalAll(); + } finally { + lock.unlock(); + } + } +} diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolder.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolder.java new file mode 100644 index 0000000000000..7dccedb8d5278 --- /dev/null +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolder.java @@ -0,0 +1,186 @@ +/* + * 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.repositories.s3.async; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.repositories.s3.GenericStatsMetricPublisher; + +import java.util.Objects; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +/** + * Transfer semaphore holder for controlled transfer of data to remote. + */ +public class TransferSemaphoresHolder { + private static final Logger log = LogManager.getLogger(TransferSemaphoresHolder.class); + // For tests + protected TypeSemaphore lowPrioritySemaphore; + protected TypeSemaphore normalPrioritySemaphore; + private final int normalPriorityPermits; + private final int lowPriorityPermits; + private final int acquireWaitDuration; + private final TimeUnit acquireWaitDurationUnit; + + /** + * Constructor to create semaphores holder. + */ + public TransferSemaphoresHolder( + int normalPriorityPermits, + int lowPriorityPermits, + int acquireWaitDuration, + TimeUnit timeUnit, + GenericStatsMetricPublisher genericStatsPublisher + ) { + + this.normalPriorityPermits = normalPriorityPermits; + this.lowPriorityPermits = lowPriorityPermits; + this.normalPrioritySemaphore = new TypeSemaphore( + normalPriorityPermits, + TypeSemaphore.PermitType.NORMAL, + genericStatsPublisher::updateNormalPermits + ); + this.lowPrioritySemaphore = new TypeSemaphore( + lowPriorityPermits, + TypeSemaphore.PermitType.LOW, + genericStatsPublisher::updateLowPermits + ); + this.acquireWaitDuration = acquireWaitDuration; + this.acquireWaitDurationUnit = timeUnit; + } + + /** + * Overridden semaphore to identify transfer semaphores among all other semaphores for triaging. + */ + public static class TypeSemaphore extends Semaphore { + private final PermitType permitType; + private final Consumer permitChangeConsumer; + + public enum PermitType { + NORMAL, + LOW; + } + + public TypeSemaphore(int permits, PermitType permitType, Consumer permitChangeConsumer) { + super(permits); + this.permitType = permitType; + this.permitChangeConsumer = permitChangeConsumer; + } + + public PermitType getType() { + return permitType; + } + + @Override + public boolean tryAcquire() { + boolean acquired = super.tryAcquire(); + if (acquired) { + permitChangeConsumer.accept(true); + } + return acquired; + } + + @Override + public boolean tryAcquire(long timeout, TimeUnit unit) throws InterruptedException { + boolean acquired = super.tryAcquire(timeout, unit); + if (acquired) { + permitChangeConsumer.accept(true); + } + return acquired; + } + + @Override + public void release() { + super.release(); + permitChangeConsumer.accept(false); + } + } + + /** + * For multiple part requests of a single file, request context object will be set with the decision if low + * priority permits can also be utilized in high priority transfers of parts of the file. If high priority get fully + * consumed then low priority permits will be acquired for transfer. + * + * If a low priority transfer request comes in and a high priority transfer is in progress then till current + * high priority transfer finishes, low priority transfer may have to compete. This is an acceptable side effect + * because low priority transfers are generally heavy and it is ok to have slow progress in the beginning. + * + */ + public static class RequestContext { + + private final boolean lowPriorityPermitsConsumable; + + private RequestContext(boolean lowPriorityPermitsConsumable) { + this.lowPriorityPermitsConsumable = lowPriorityPermitsConsumable; + } + + } + + public RequestContext createRequestContext() { + return new RequestContext(this.lowPrioritySemaphore.availablePermits() == lowPriorityPermits); + } + + /** + * Acquire permit based on the availability and based on the transfer priority. + * A high priority event can acquire a low priority semaphore if all low permits are available. + * A low priority event can acquire a high priority semaphore if at least 40% of high permits are available. We + * reserve this bandwidth to ensure that high priority events never wait for permits in case of ongoing low priority + * transfers. + */ + public TypeSemaphore acquirePermit(WritePriority writePriority, RequestContext requestContext) throws InterruptedException { + log.debug( + () -> "Acquire permit request for transfer type: " + + writePriority + + ". Available high priority permits: " + + normalPrioritySemaphore.availablePermits() + + " and low priority permits: " + + lowPrioritySemaphore.availablePermits() + ); + // Try acquiring low priority permit or high priority permit immediately if available. + // Otherwise, we wait for low priority permit. + if (Objects.requireNonNull(writePriority) == WritePriority.LOW) { + if (lowPrioritySemaphore.tryAcquire()) { + return lowPrioritySemaphore; + } else if (normalPrioritySemaphore.availablePermits() > 0.4 * normalPriorityPermits && normalPrioritySemaphore.tryAcquire()) { + return normalPrioritySemaphore; + } else if (lowPrioritySemaphore.tryAcquire(acquireWaitDuration, acquireWaitDurationUnit)) { + return lowPrioritySemaphore; + } + return null; + } + + // Try acquiring high priority permit or low priority permit immediately if available. + // Otherwise, we wait for high priority permit. + if (normalPrioritySemaphore.tryAcquire()) { + return normalPrioritySemaphore; + } else if (requestContext.lowPriorityPermitsConsumable && lowPrioritySemaphore.tryAcquire()) { + return lowPrioritySemaphore; + } else if (normalPrioritySemaphore.tryAcquire(acquireWaitDuration, acquireWaitDurationUnit)) { + return normalPrioritySemaphore; + } + return null; + } + + /** + * Used in tests. + */ + public int getNormalPriorityPermits() { + return normalPriorityPermits; + } + + /** + * Used in tests. + */ + public int getLowPriorityPermits() { + return lowPriorityPermits; + } +} diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/UploadRequest.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/UploadRequest.java index b944a72225d36..79b58ff215c54 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/UploadRequest.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/UploadRequest.java @@ -26,8 +26,8 @@ public class UploadRequest { private final CheckedConsumer uploadFinalizer; private final boolean doRemoteDataIntegrityCheck; private final Long expectedChecksum; - private boolean uploadRetryEnabled; private final Map metadata; + private final boolean uploadRetryEnabled; /** * Construct a new UploadRequest object diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/RepositoryCredentialsTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/RepositoryCredentialsTests.java index f84d953baae8e..573a4f3f51a41 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/RepositoryCredentialsTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/RepositoryCredentialsTests.java @@ -303,7 +303,22 @@ protected S3Repository createRepository( ClusterService clusterService, RecoverySettings recoverySettings ) { - return new S3Repository(metadata, registry, service, clusterService, recoverySettings, null, null, null, null, null, false) { + return new S3Repository( + metadata, + registry, + service, + clusterService, + recoverySettings, + null, + null, + null, + null, + null, + false, + null, + null, + null + ) { @Override protected void assertSnapshotOrGenericThread() { // eliminate thread name check as we create repo manually on test/main threads diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java index 4173f8b66387f..9b413ac81d766 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java @@ -47,7 +47,10 @@ import org.opensearch.repositories.s3.async.AsyncExecutorContainer; import org.opensearch.repositories.s3.async.AsyncTransferEventLoopGroup; import org.opensearch.repositories.s3.async.AsyncTransferManager; +import org.opensearch.repositories.s3.async.SizeBasedBlockingQ; +import org.opensearch.repositories.s3.async.TransferSemaphoresHolder; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.Scheduler; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -65,6 +68,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -91,8 +95,13 @@ public class S3BlobContainerMockClientTests extends OpenSearchTestCase implement private MockS3AsyncService asyncService; private ExecutorService futureCompletionService; private ExecutorService streamReaderService; + private ExecutorService remoteTransferRetry; + private ExecutorService transferQueueConsumerService; + private ScheduledExecutorService scheduler; private AsyncTransferEventLoopGroup transferNIOGroup; private S3BlobContainer blobContainer; + private SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; + private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; static class MockS3AsyncService extends S3AsyncService { @@ -364,7 +373,27 @@ public void setUp() throws Exception { asyncService = new MockS3AsyncService(configPath(), 1000); futureCompletionService = Executors.newSingleThreadExecutor(); streamReaderService = Executors.newSingleThreadExecutor(); + remoteTransferRetry = Executors.newFixedThreadPool(20); + transferQueueConsumerService = Executors.newFixedThreadPool(20); + scheduler = new Scheduler.SafeScheduledThreadPoolExecutor(1); transferNIOGroup = new AsyncTransferEventLoopGroup(1); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 10L, ByteSizeUnit.GB), + transferQueueConsumerService, + 10, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL + ); + lowPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 20L, ByteSizeUnit.GB), + transferQueueConsumerService, + 5, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL + ); + normalPrioritySizeBasedBlockingQ.start(); + lowPrioritySizeBasedBlockingQ.start(); blobContainer = createBlobContainer(); super.setUp(); } @@ -373,6 +402,14 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { IOUtils.close(asyncService); + futureCompletionService.shutdown(); + streamReaderService.shutdown(); + remoteTransferRetry.shutdown(); + transferQueueConsumerService.shutdown(); + normalPrioritySizeBasedBlockingQ.close(); + lowPrioritySizeBasedBlockingQ.close(); + scheduler.shutdown(); + transferNIOGroup.close(); super.tearDown(); } @@ -394,7 +431,7 @@ private S3BlobStore createBlobStore() { streamReaderService, transferNIOGroup ); - + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); return new S3BlobStore( null, asyncService, @@ -410,11 +447,21 @@ private S3BlobStore createBlobStore() { S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.getDefault(Settings.EMPTY).getBytes(), asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), - asyncExecutorContainer.getStreamReader() + asyncExecutorContainer.getStreamReader(), + new TransferSemaphoresHolder( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 5, + TimeUnit.MINUTES, + genericStatsMetricPublisher + ) ), asyncExecutorContainer, asyncExecutorContainer, - asyncExecutorContainer + asyncExecutorContainer, + normalPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ, + genericStatsMetricPublisher ); } @@ -574,19 +621,32 @@ private int calculateNumberOfParts(long contentLength, long partSize) { return (int) ((contentLength % partSize) == 0 ? contentLength / partSize : (contentLength / partSize) + 1); } - public void testFailureWhenLargeFileRedirected() throws IOException, ExecutionException, InterruptedException { - testLargeFilesRedirectedToSlowSyncClient(true); + public void testFailureWhenLargeFileRedirected() throws IOException, InterruptedException { + testLargeFilesRedirectedToSlowSyncClient(true, WritePriority.LOW); + testLargeFilesRedirectedToSlowSyncClient(true, WritePriority.NORMAL); } - public void testLargeFileRedirected() throws IOException, ExecutionException, InterruptedException { - testLargeFilesRedirectedToSlowSyncClient(false); + public void testLargeFileRedirected() throws IOException, InterruptedException { + testLargeFilesRedirectedToSlowSyncClient(false, WritePriority.LOW); + testLargeFilesRedirectedToSlowSyncClient(false, WritePriority.NORMAL); } - private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException) throws IOException, InterruptedException { - final ByteSizeValue partSize = new ByteSizeValue(1024, ByteSizeUnit.MB); - + private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException, WritePriority writePriority) throws IOException, + InterruptedException { + ByteSizeValue capacity = new ByteSizeValue(1, ByteSizeUnit.GB); int numberOfParts = 20; - final long lastPartSize = new ByteSizeValue(20, ByteSizeUnit.MB).getBytes(); + final ByteSizeValue partSize = new ByteSizeValue(capacity.getBytes() / numberOfParts + 1, ByteSizeUnit.BYTES); + + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + SizeBasedBlockingQ sizeBasedBlockingQ = new SizeBasedBlockingQ( + capacity, + transferQueueConsumerService, + 10, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL + ); + + final long lastPartSize = new ByteSizeValue(200, ByteSizeUnit.MB).getBytes(); final long blobSize = ((numberOfParts - 1) * partSize.getBytes()) + lastPartSize; CountDownLatch countDownLatch = new CountDownLatch(1); AtomicReference exceptionRef = new AtomicReference<>(); @@ -609,6 +669,9 @@ private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException) t when(blobStore.getStatsMetricPublisher()).thenReturn(new StatsMetricPublisher()); when(blobStore.bufferSizeInBytes()).thenReturn(bufferSize); + when(blobStore.getLowPrioritySizeBasedBlockingQ()).thenReturn(sizeBasedBlockingQ); + when(blobStore.getNormalPrioritySizeBasedBlockingQ()).thenReturn(sizeBasedBlockingQ); + final boolean serverSideEncryption = randomBoolean(); when(blobStore.serverSideEncryption()).thenReturn(serverSideEncryption); @@ -658,7 +721,7 @@ private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException) t .streamContextSupplier(streamContextSupplier) .fileSize(blobSize) .failIfAlreadyExists(false) - .writePriority(WritePriority.HIGH) + .writePriority(writePriority) .uploadFinalizer(Assert::assertTrue) .doRemoteDataIntegrityCheck(false) .metadata(new HashMap<>()) @@ -693,5 +756,4 @@ private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException) t } }); } - } diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java index 10578090da75c..96ef28d24c14f 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -67,6 +67,8 @@ import org.opensearch.repositories.s3.async.AsyncExecutorContainer; import org.opensearch.repositories.s3.async.AsyncTransferEventLoopGroup; import org.opensearch.repositories.s3.async.AsyncTransferManager; +import org.opensearch.repositories.s3.async.SizeBasedBlockingQ; +import org.opensearch.repositories.s3.async.TransferSemaphoresHolder; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -87,6 +89,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -114,7 +118,12 @@ public class S3BlobContainerRetriesTests extends AbstractBlobContainerRetriesTes private S3AsyncService asyncService; private ExecutorService futureCompletionService; private ExecutorService streamReaderService; + private ExecutorService remoteTransferRetry; + private ExecutorService transferQueueConsumerService; + private ScheduledExecutorService scheduler; private AsyncTransferEventLoopGroup transferNIOGroup; + private SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; + private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; @Before public void setUp() throws Exception { @@ -125,7 +134,26 @@ public void setUp() throws Exception { futureCompletionService = Executors.newSingleThreadExecutor(); streamReaderService = Executors.newSingleThreadExecutor(); transferNIOGroup = new AsyncTransferEventLoopGroup(1); - + remoteTransferRetry = Executors.newFixedThreadPool(20); + transferQueueConsumerService = Executors.newFixedThreadPool(2); + scheduler = new ScheduledThreadPoolExecutor(1); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 5L, ByteSizeUnit.GB), + transferQueueConsumerService, + 2, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL + ); + lowPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 5L, ByteSizeUnit.GB), + transferQueueConsumerService, + 2, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.LOW + ); + normalPrioritySizeBasedBlockingQ.start(); + lowPrioritySizeBasedBlockingQ.start(); // needed by S3AsyncService SocketAccess.doPrivileged(() -> System.setProperty("opensearch.path.conf", configPath().toString())); super.setUp(); @@ -137,6 +165,11 @@ public void tearDown() throws Exception { streamReaderService.shutdown(); futureCompletionService.shutdown(); + remoteTransferRetry.shutdown(); + transferQueueConsumerService.shutdown(); + scheduler.shutdown(); + normalPrioritySizeBasedBlockingQ.close(); + lowPrioritySizeBasedBlockingQ.close(); IOUtils.close(transferNIOGroup); if (previousOpenSearchPathConf != null) { @@ -205,7 +238,7 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( streamReaderService, transferNIOGroup ); - + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); return new S3BlobContainer( BlobPath.cleanPath(), new S3BlobStore( @@ -223,11 +256,21 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.getDefault(Settings.EMPTY).getBytes(), asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), - asyncExecutorContainer.getStreamReader() + asyncExecutorContainer.getStreamReader(), + new TransferSemaphoresHolder( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 5, + TimeUnit.MINUTES, + genericStatsMetricPublisher + ) ), asyncExecutorContainer, asyncExecutorContainer, - asyncExecutorContainer + asyncExecutorContainer, + normalPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ, + genericStatsMetricPublisher ) ) { @Override diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java index 6fec535ae6301..f8e9903bb3577 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RepositoryTests.java @@ -169,7 +169,11 @@ private S3Repository createS3Repo(RepositoryMetadata metadata) { null, null, null, - false + false, + null, + null, + null, + null ) { @Override protected void assertSnapshotOrGenericThread() { diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java index 04d1819bef02b..89add3cdbfc60 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java @@ -33,6 +33,7 @@ import org.opensearch.common.io.InputStreamContainer; import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.repositories.blobstore.ZeroInputStream; +import org.opensearch.repositories.s3.GenericStatsMetricPublisher; import org.opensearch.repositories.s3.StatsMetricPublisher; import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; @@ -46,6 +47,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import static org.mockito.ArgumentMatchers.any; @@ -63,11 +65,19 @@ public class AsyncTransferManagerTests extends OpenSearchTestCase { @Before public void setUp() throws Exception { s3AsyncClient = mock(S3AsyncClient.class); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); asyncTransferManager = new AsyncTransferManager( ByteSizeUnit.MB.toBytes(5), Executors.newSingleThreadExecutor(), Executors.newSingleThreadExecutor(), - Executors.newSingleThreadExecutor() + Executors.newSingleThreadExecutor(), + new TransferSemaphoresHolder( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 5, + TimeUnit.MINUTES, + genericStatsMetricPublisher + ) ); super.setUp(); } diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQTests.java new file mode 100644 index 0000000000000..5be4037407d23 --- /dev/null +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQTests.java @@ -0,0 +1,102 @@ +/* + * 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.repositories.s3.async; + +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.repositories.s3.GenericStatsMetricPublisher; +import org.opensearch.repositories.s3.S3TransferRejectedException; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.After; +import org.junit.Before; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; + +public class SizeBasedBlockingQTests extends OpenSearchTestCase { + private ExecutorService consumerService; + private ExecutorService producerService; + + @Override + @Before + public void setUp() throws Exception { + this.consumerService = Executors.newFixedThreadPool(10); + this.producerService = Executors.newFixedThreadPool(100); + super.setUp(); + } + + @After + public void tearDown() throws Exception { + consumerService.shutdown(); + producerService.shutdown(); + super.tearDown(); + } + + public void testProducerConsumerOfBulkItems() throws InterruptedException { + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + SizeBasedBlockingQ.QueueEventType queueEventType = randomBoolean() + ? SizeBasedBlockingQ.QueueEventType.NORMAL + : SizeBasedBlockingQ.QueueEventType.LOW; + SizeBasedBlockingQ sizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(ByteSizeUnit.BYTES.toBytes(10)), + consumerService, + 10, + genericStatsMetricPublisher, + queueEventType + ); + sizeBasedBlockingQ.start(); + int numOfItems = randomIntBetween(100, 1000); + CountDownLatch latch = new CountDownLatch(numOfItems); + AtomicBoolean unknownError = new AtomicBoolean(); + for (int i = 0; i < numOfItems; i++) { + final int idx = i; + producerService.submit(() -> { + boolean throwException = randomBoolean(); + + SizeBasedBlockingQ.Item item = new TestItemToStr(randomIntBetween(1, 5), () -> { + latch.countDown(); + if (throwException) { + throw new RuntimeException("throwing random exception"); + } + }, idx); + + try { + sizeBasedBlockingQ.produce(item); + } catch (InterruptedException e) { + latch.countDown(); + unknownError.set(true); + throw new RuntimeException(e); + } catch (S3TransferRejectedException ex) { + latch.countDown(); + } + }); + } + latch.await(); + sizeBasedBlockingQ.close(); + assertFalse(unknownError.get()); + assertEquals(0L, genericStatsMetricPublisher.getNormalPriorityQSize()); + assertEquals(0L, genericStatsMetricPublisher.getLowPriorityQSize()); + } + + static class TestItemToStr extends SizeBasedBlockingQ.Item { + private final int id; + + public TestItemToStr(long size, Runnable consumable, int id) { + super(size, consumable); + this.id = id; + } + + @Override + public String toString() { + return String.valueOf(id); + } + } +} diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolderTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolderTests.java new file mode 100644 index 0000000000000..236f02c5eb1f7 --- /dev/null +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolderTests.java @@ -0,0 +1,276 @@ +/* + * 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.repositories.s3.async; + +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.repositories.s3.GenericStatsMetricPublisher; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.mockito.Mockito; + +import static org.opensearch.repositories.s3.async.TransferSemaphoresHolder.TypeSemaphore.PermitType; + +public class TransferSemaphoresHolderTests extends OpenSearchTestCase { + + public void testAllocation() { + int availablePermits = randomIntBetween(5, 20); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int normalPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - normalPermits; + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( + normalPermits, + lowPermits, + 1, + TimeUnit.NANOSECONDS, + genericStatsPublisher + ); + assertEquals(normalPermits, transferSemaphoresHolder.getNormalPriorityPermits()); + assertEquals(lowPermits, transferSemaphoresHolder.getLowPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredNormalPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredLowPriorityPermits()); + } + + public void testLowPriorityEventPermitAcquisition() throws InterruptedException { + int availablePermits = randomIntBetween(5, 50); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int normalPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - normalPermits; + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( + normalPermits, + lowPermits, + 1, + TimeUnit.NANOSECONDS, + genericStatsPublisher + ); + + List semaphores = new ArrayList<>(); + int normalPermitsEligibleForLowEvents = normalPermits - (int) (normalPermits * 0.4); + + int lowAcquisitionsExpected = (normalPermitsEligibleForLowEvents + lowPermits); + for (int i = 0; i < lowAcquisitionsExpected; i++) { + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.LOW, + requestContext + ); + semaphores.add(acquiredSemaphore); + if (i >= lowPermits) { + assertEquals(PermitType.NORMAL, acquiredSemaphore.getType()); + } else { + assertEquals(PermitType.LOW, acquiredSemaphore.getType()); + } + } + + for (int i = 0; i < normalPermits - normalPermitsEligibleForLowEvents; i++) { + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.NORMAL, + requestContext + ); + assertEquals(PermitType.NORMAL, acquiredSemaphore.getType()); + semaphores.add(acquiredSemaphore); + } + + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.LOW, + requestContext + ); + assertNull(acquiredSemaphore); + + assertEquals(availablePermits, semaphores.size()); + semaphores.forEach(Semaphore::release); + assertEquals(normalPermits, transferSemaphoresHolder.getNormalPriorityPermits()); + assertEquals(lowPermits, transferSemaphoresHolder.getLowPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredNormalPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredLowPriorityPermits()); + + } + + public void testNormalPermitEventAcquisition() throws InterruptedException { + int availablePermits = randomIntBetween(5, 50); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int normalPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - normalPermits; + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( + normalPermits, + lowPermits, + 1, + TimeUnit.NANOSECONDS, + genericStatsPublisher + ); + + List semaphores = new ArrayList<>(); + List lowSemaphores = new ArrayList<>(); + int normalAcquisitionsExpected = normalPermits + lowPermits; + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + for (int i = 0; i < normalAcquisitionsExpected; i++) { + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.NORMAL, + requestContext + ); + semaphores.add(acquiredSemaphore); + if (i >= normalPermits) { + assertEquals(PermitType.LOW, acquiredSemaphore.getType()); + lowSemaphores.add(acquiredSemaphore); + } else { + assertEquals(PermitType.NORMAL, acquiredSemaphore.getType()); + } + } + assertEquals(availablePermits, semaphores.size()); + + int lowAcquired = lowPermits; + + Semaphore removedLowSemaphore = lowSemaphores.remove(0); + removedLowSemaphore.release(); + semaphores.remove(removedLowSemaphore); + + requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.LOW, + requestContext + ); + semaphores.add(acquiredSemaphore); + lowSemaphores.add(acquiredSemaphore); + while (lowAcquired > 1) { + requestContext = transferSemaphoresHolder.createRequestContext(); + acquiredSemaphore = transferSemaphoresHolder.acquirePermit(WritePriority.NORMAL, requestContext); + assertNull(acquiredSemaphore); + lowAcquired--; + } + + semaphores.forEach(Semaphore::release); + assertEquals(normalPermits, transferSemaphoresHolder.getNormalPriorityPermits()); + assertEquals(lowPermits, transferSemaphoresHolder.getLowPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredNormalPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredLowPriorityPermits()); + } + + private static class TestTransferSemaphoresHolder extends TransferSemaphoresHolder { + AtomicInteger normalWaitCount = new AtomicInteger(); + AtomicInteger lowWaitCount = new AtomicInteger(); + + /** + * Constructor to create semaphores holder. + */ + public TestTransferSemaphoresHolder( + int normalPermits, + int lowPermits, + int acquireWaitDuration, + TimeUnit timeUnit, + GenericStatsMetricPublisher genericStatsMetricPublisher + ) throws InterruptedException { + super(normalPermits, lowPermits, acquireWaitDuration, timeUnit, genericStatsMetricPublisher); + TypeSemaphore executingNormalSemaphore = normalPrioritySemaphore; + TypeSemaphore executingLowSemaphore = lowPrioritySemaphore; + + this.normalPrioritySemaphore = Mockito.spy(normalPrioritySemaphore); + this.lowPrioritySemaphore = Mockito.spy(lowPrioritySemaphore); + Mockito.doAnswer(invocation -> { + normalWaitCount.incrementAndGet(); + return false; + }).when(normalPrioritySemaphore).tryAcquire(Mockito.anyLong(), Mockito.any(TimeUnit.class)); + Mockito.doAnswer(invocation -> executingNormalSemaphore.availablePermits()).when(normalPrioritySemaphore).availablePermits(); + Mockito.doAnswer(invocation -> executingNormalSemaphore.tryAcquire()).when(normalPrioritySemaphore).tryAcquire(); + + Mockito.doAnswer(invocation -> { + lowWaitCount.incrementAndGet(); + return false; + }).when(lowPrioritySemaphore).tryAcquire(Mockito.anyLong(), Mockito.any(TimeUnit.class)); + Mockito.doAnswer(invocation -> executingLowSemaphore.availablePermits()).when(lowPrioritySemaphore).availablePermits(); + Mockito.doAnswer(invocation -> executingLowSemaphore.tryAcquire()).when(lowPrioritySemaphore).tryAcquire(); + } + } + + public void testNormalSemaphoreAcquiredWait() throws InterruptedException { + int availablePermits = randomIntBetween(10, 50); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int normalPermits = (int) (availablePermits * priorityAllocation); + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + TestTransferSemaphoresHolder transferSemaphoresHolder = new TestTransferSemaphoresHolder( + normalPermits, + availablePermits - normalPermits, + 5, + TimeUnit.MINUTES, + genericStatsPublisher + ); + + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore lowSemaphore = transferSemaphoresHolder.acquirePermit(WritePriority.LOW, requestContext); + assertEquals(PermitType.LOW, lowSemaphore.getType()); + for (int i = 0; i < normalPermits; i++) { + requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.NORMAL, + requestContext + ); + assertEquals(PermitType.NORMAL, acquiredSemaphore.getType()); + } + + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.NORMAL, + requestContext + ); + assertNull(acquiredSemaphore); + assertEquals(1, transferSemaphoresHolder.normalWaitCount.get()); + assertEquals(0, transferSemaphoresHolder.lowWaitCount.get()); + } + + public void testLowSemaphoreAcquiredWait() throws InterruptedException { + int availablePermits = randomIntBetween(10, 50); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int normalPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - normalPermits; + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); + TestTransferSemaphoresHolder transferSemaphoresHolder = new TestTransferSemaphoresHolder( + normalPermits, + lowPermits, + 5, + TimeUnit.MINUTES, + genericStatsPublisher + ); + + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + int normalPermitsEligibleForLowEvents = normalPermits - (int) (normalPermits * 0.4); + for (int i = 0; i < normalPermitsEligibleForLowEvents; i++) { + TransferSemaphoresHolder.TypeSemaphore lowSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.NORMAL, + requestContext + ); + assertEquals(PermitType.NORMAL, lowSemaphore.getType()); + } + + for (int i = 0; i < lowPermits; i++) { + requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.LOW, + requestContext + ); + assertEquals(PermitType.LOW, acquiredSemaphore.getType()); + } + + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.LOW, + requestContext + ); + assertNull(acquiredSemaphore); + assertEquals(1, transferSemaphoresHolder.lowWaitCount.get()); + assertEquals(0, transferSemaphoresHolder.normalWaitCount.get()); + } + +} 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 0f6646d37f950..8ce8ec8e01abe 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobStore.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobStore.java @@ -75,6 +75,7 @@ default void reload(RepositoryMetadata repositoryMetadata) {} * Metrics for BlobStore interactions */ enum Metric { + GENERIC_STATS("generic_stats"), REQUEST_SUCCESS("request_success_total"), REQUEST_FAILURE("request_failures_total"), REQUEST_LATENCY("request_time_in_millis"), diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java index 3f341c878c3c7..4e8db0a3a8c69 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WritePriority.java @@ -14,7 +14,12 @@ * @opensearch.internal */ public enum WritePriority { + // Used for segment transfers during refresh, flush or merges NORMAL, + // Used for transfer of translog or ckp files. HIGH, - URGENT + // Used for transfer of remote cluster state + URGENT, + // All other background transfers such as in snapshot recovery, recovery from local store or index etc. + LOW } 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 351aec6e3af6c..bfb841307af49 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -437,10 +437,14 @@ private void uploadNewSegments( batchUploadListener.onFailure(ex); }); statsListener.beforeUpload(src); - remoteDirectory.copyFrom(storeDirectory, src, IOContext.DEFAULT, aggregatedListener); + remoteDirectory.copyFrom(storeDirectory, src, IOContext.DEFAULT, aggregatedListener, isLowPriorityUpload()); } } + private boolean isLowPriorityUpload() { + return isLocalOrSnapshotRecovery(); + } + /** * Whether to upload a file or not depending on whether file is in excluded list or has been already uploaded. * diff --git a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java index 345583bbbd1be..ab76150f8f83d 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -29,6 +29,7 @@ import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeInputStream; import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.index.store.exception.ChecksumCombinationException; import java.io.FileNotFoundException; @@ -323,11 +324,12 @@ public boolean copyFrom( String remoteFileName, IOContext context, Runnable postUploadRunner, - ActionListener listener + ActionListener listener, + boolean lowPriorityUpload ) { if (blobContainer instanceof AsyncMultiStreamBlobContainer) { try { - uploadBlob(from, src, remoteFileName, context, postUploadRunner, listener); + uploadBlob(from, src, remoteFileName, context, postUploadRunner, listener, lowPriorityUpload); } catch (Exception e) { listener.onFailure(e); } @@ -342,7 +344,8 @@ private void uploadBlob( String remoteFileName, IOContext ioContext, Runnable postUploadRunner, - ActionListener listener + ActionListener listener, + boolean lowPriorityUpload ) throws Exception { long expectedChecksum = calculateChecksumOfChecksum(from, src); long contentLength; @@ -353,12 +356,13 @@ private void uploadBlob( if (getBlobContainer() instanceof AsyncMultiStreamBlobContainer) { remoteIntegrityEnabled = ((AsyncMultiStreamBlobContainer) getBlobContainer()).remoteIntegrityCheckSupported(); } + lowPriorityUpload = lowPriorityUpload || contentLength > ByteSizeUnit.GB.toBytes(15); RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( src, remoteFileName, contentLength, true, - WritePriority.NORMAL, + lowPriorityUpload ? WritePriority.LOW : WritePriority.NORMAL, (size, position) -> uploadRateLimiter.apply(new OffsetRangeIndexInputStream(from.openInput(src, ioContext), size, position)), expectedChecksum, remoteIntegrityEnabled diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index ec1163fe91b6c..8c0ecb4cc783a 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -453,7 +453,7 @@ public IndexInput openInput(String name, IOContext context) throws IOException { * @param context IOContext to be used to open IndexInput of file during remote upload * @param listener Listener to handle upload callback events */ - public void copyFrom(Directory from, String src, IOContext context, ActionListener listener) { + public void copyFrom(Directory from, String src, IOContext context, ActionListener listener, boolean lowPriorityUpload) { try { final String remoteFileName = getNewRemoteSegmentFilename(src); boolean uploaded = remoteDataDirectory.copyFrom(from, src, remoteFileName, context, () -> { @@ -462,7 +462,7 @@ public void copyFrom(Directory from, String src, IOContext context, ActionListen } catch (IOException e) { throw new RuntimeException("Exception in segment postUpload for file " + src, e); } - }, listener); + }, listener, lowPriorityUpload); if (uploaded == false) { copyFrom(from, src, src, context); listener.onResponse(null); diff --git a/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java index 9e38e1749d434..ee81369725e6f 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java @@ -104,7 +104,8 @@ public void onResponse(Void t) { public void onFailure(Exception e) { fail("Listener responded with exception" + e); } - } + }, + false ); assertTrue(countDownLatch.await(10, TimeUnit.SECONDS)); assertTrue(postUploadInvoked.get()); @@ -141,7 +142,8 @@ public void onResponse(Void t) { public void onFailure(Exception e) { countDownLatch.countDown(); } - } + }, + false ); assertTrue(countDownLatch.await(10, TimeUnit.SECONDS)); assertFalse(postUploadInvoked.get()); diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index b1e2028d761f0..567199cf64cd8 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -639,7 +639,7 @@ public void onResponse(Void unused) { @Override public void onFailure(Exception e) {} }; - remoteSegmentStoreDirectory.copyFrom(storeDirectory, filename, IOContext.DEFAULT, completionListener); + remoteSegmentStoreDirectory.copyFrom(storeDirectory, filename, IOContext.DEFAULT, completionListener, false); assertTrue(latch.await(5000, TimeUnit.SECONDS)); assertTrue(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); storeDirectory.close(); @@ -683,7 +683,7 @@ public void onFailure(Exception e) { latch.countDown(); } }; - remoteSegmentStoreDirectory.copyFrom(storeDirectory, filename, IOContext.DEFAULT, completionListener); + remoteSegmentStoreDirectory.copyFrom(storeDirectory, filename, IOContext.DEFAULT, completionListener, false); assertTrue(latch.await(5000, TimeUnit.SECONDS)); assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); From a07858fd0f7e563ed2e4c17934dbbe37f6557f96 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 May 2024 10:44:50 -0400 Subject: [PATCH 3/7] Bump com.gradle.enterprise from 3.17.2 to 3.17.3 (#13641) * Bump com.gradle.enterprise from 3.17.2 to 3.17.3 Bumps com.gradle.enterprise from 3.17.2 to 3.17.3. --- updated-dependencies: - dependency-name: com.gradle.enterprise 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 + settings.gradle | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b24cc5347e768..f5d0ab4f7af38 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/), - 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 `com.gradle.enterprise` from 3.17.2 to 3.17.3 ([#13641](https://github.com/opensearch-project/OpenSearch/pull/13641)) ### 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/settings.gradle b/settings.gradle index 8e961b9d4179f..065a8dc3d0a8a 100644 --- a/settings.gradle +++ b/settings.gradle @@ -10,7 +10,7 @@ */ plugins { - id "com.gradle.enterprise" version "3.17.2" + id "com.gradle.enterprise" version "3.17.3" } ext.disableBuildCache = hasProperty('DISABLE_BUILD_CACHE') || System.getenv().containsKey('DISABLE_BUILD_CACHE') From 079cef5445b89b8072ad0ff811561e1d12c84603 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 May 2024 14:02:45 -0700 Subject: [PATCH 4/7] Bump org.apache.hadoop:hadoop-minicluster from 3.3.6 to 3.4.0 in /test/fixtures/hdfs-fixture (#13642) * Bump org.apache.hadoop:hadoop-minicluster in /test/fixtures/hdfs-fixture Bumps org.apache.hadoop:hadoop-minicluster from 3.3.6 to 3.4.0. --- updated-dependencies: - dependency-name: org.apache.hadoop:hadoop-minicluster dependency-type: direct:production update-type: version-update:semver-minor ... 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 + test/fixtures/hdfs-fixture/build.gradle | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f5d0ab4f7af38..afb0d0840b405 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/), - 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 `com.gradle.enterprise` from 3.17.2 to 3.17.3 ([#13641](https://github.com/opensearch-project/OpenSearch/pull/13641)) +- Bump `org.apache.hadoop:hadoop-minicluster` from 3.3.6 to 3.4.0 ([#13642](https://github.com/opensearch-project/OpenSearch/pull/13642)) ### 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/test/fixtures/hdfs-fixture/build.gradle b/test/fixtures/hdfs-fixture/build.gradle index 8ac13d897e0b5..33635f2053a21 100644 --- a/test/fixtures/hdfs-fixture/build.gradle +++ b/test/fixtures/hdfs-fixture/build.gradle @@ -37,7 +37,7 @@ versions << [ ] dependencies { - api("org.apache.hadoop:hadoop-minicluster:3.3.6") { + api("org.apache.hadoop:hadoop-minicluster:3.4.0") { exclude module: 'websocket-client' exclude module: 'jettison' exclude module: 'netty' From 14f1c43c108f378b13d109ade364216c082fb858 Mon Sep 17 00:00:00 2001 From: Vacha Shah Date: Mon, 13 May 2024 15:06:27 -0700 Subject: [PATCH 5/7] Abstracting outbound side of transport (#13293) * Abstracting outbound side of transport Signed-off-by: Vacha Shah * Making outbound handler protocol dependent via inbound handler Signed-off-by: Vacha Shah * Fixing precommit Signed-off-by: Vacha Shah * Addressing comments Signed-off-by: Vacha Shah * Fixing precommit Signed-off-by: Vacha Shah * Cleaning up code Signed-off-by: Vacha Shah * Addressing comments Signed-off-by: Vacha Shah * Cleaning up Signed-off-by: Vacha Shah * Addressing comments Signed-off-by: Vacha Shah * Abstracting InboundHandlerTests Signed-off-by: Vacha Shah * Abstracting TransportLoggerTests Signed-off-by: Vacha Shah --------- Signed-off-by: Vacha Shah --- CHANGELOG.md | 1 + .../opensearch/transport/InboundHandler.java | 13 + .../transport/NativeMessageHandler.java | 16 +- .../opensearch/transport/OutboundHandler.java | 171 +--------- .../transport/ProtocolMessageHandler.java | 14 + .../transport/ProtocolOutboundHandler.java | 70 ++++ .../opensearch/transport/TcpTransport.java | 24 +- .../transport/TcpTransportChannel.java | 4 +- .../opensearch/transport/TransportStatus.java | 4 +- .../CompressibleBytesOutputStream.java | 2 +- .../nativeprotocol/NativeOutboundHandler.java | 224 +++++++++++++ .../NativeOutboundMessage.java} | 18 +- .../{ => nativeprotocol}/NetworkMessage.java | 3 +- .../transport/InboundDecoderTests.java | 108 +++---- .../transport/InboundHandlerTests.java | 89 +++--- .../transport/InboundPipelineTests.java | 111 ++----- .../transport/NativeOutboundHandlerTests.java | 302 ++++++++++++++++++ .../transport/OutboundHandlerTests.java | 226 +------------ .../transport/TcpTransportTests.java | 15 +- .../transport/TransportLoggerTests.java | 25 +- .../CompressibleBytesOutputStreamTests.java | 2 +- .../NativeInboundDecoderTests.java | 59 ++++ .../NativeInboundHandlerTests.java | 45 +++ .../NativeInboundPipelineTests.java | 60 ++++ .../NativeOutboundMessageTests.java | 50 +++ .../NativeTransportLoggerTests.java | 42 +++ .../org/opensearch/transport/TestRequest.java | 4 + .../opensearch/transport/TestResponse.java | 4 + 28 files changed, 1061 insertions(+), 645 deletions(-) create mode 100644 server/src/main/java/org/opensearch/transport/ProtocolOutboundHandler.java rename server/src/main/java/org/opensearch/transport/{ => nativeprotocol}/CompressibleBytesOutputStream.java (98%) create mode 100644 server/src/main/java/org/opensearch/transport/nativeprotocol/NativeOutboundHandler.java rename server/src/main/java/org/opensearch/transport/{OutboundMessage.java => nativeprotocol/NativeOutboundMessage.java} (91%) rename server/src/main/java/org/opensearch/transport/{ => nativeprotocol}/NetworkMessage.java (96%) create mode 100644 server/src/test/java/org/opensearch/transport/NativeOutboundHandlerTests.java rename server/src/test/java/org/opensearch/transport/{ => nativeprotocol}/CompressibleBytesOutputStreamTests.java (99%) create mode 100644 server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundDecoderTests.java create mode 100644 server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundHandlerTests.java create mode 100644 server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundPipelineTests.java create mode 100644 server/src/test/java/org/opensearch/transport/nativeprotocol/NativeOutboundMessageTests.java create mode 100644 server/src/test/java/org/opensearch/transport/nativeprotocol/NativeTransportLoggerTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index afb0d0840b405..b858ede4b78cf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ## [Unreleased 2.x] ### Added - 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)) ### 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/transport/InboundHandler.java b/server/src/main/java/org/opensearch/transport/InboundHandler.java index 6492900c49a0e..f77c44ea362cf 100644 --- a/server/src/main/java/org/opensearch/transport/InboundHandler.java +++ b/server/src/main/java/org/opensearch/transport/InboundHandler.java @@ -32,7 +32,9 @@ package org.opensearch.transport; +import org.opensearch.Version; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.BigArrays; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.threadpool.ThreadPool; @@ -57,7 +59,12 @@ public class InboundHandler { private final Map protocolMessageHandlers; InboundHandler( + String nodeName, + Version version, + String[] features, + StatsTracker statsTracker, ThreadPool threadPool, + BigArrays bigArrays, OutboundHandler outboundHandler, NamedWriteableRegistry namedWriteableRegistry, TransportHandshaker handshaker, @@ -70,7 +77,12 @@ public class InboundHandler { this.protocolMessageHandlers = Map.of( NativeInboundMessage.NATIVE_PROTOCOL, new NativeMessageHandler( + nodeName, + version, + features, + statsTracker, threadPool, + bigArrays, outboundHandler, namedWriteableRegistry, handshaker, @@ -83,6 +95,7 @@ public class InboundHandler { } void setMessageListener(TransportMessageListener listener) { + protocolMessageHandlers.values().forEach(handler -> handler.setMessageListener(listener)); if (messageListener == TransportMessageListener.NOOP_LISTENER) { messageListener = listener; } else { diff --git a/server/src/main/java/org/opensearch/transport/NativeMessageHandler.java b/server/src/main/java/org/opensearch/transport/NativeMessageHandler.java index c5b65f9eb7a11..4c972fdc14fa5 100644 --- a/server/src/main/java/org/opensearch/transport/NativeMessageHandler.java +++ b/server/src/main/java/org/opensearch/transport/NativeMessageHandler.java @@ -37,6 +37,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.BytesRef; import org.opensearch.Version; +import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.core.common.io.stream.ByteBufferStreamInput; @@ -52,6 +53,7 @@ import org.opensearch.telemetry.tracing.channels.TraceableTcpTransportChannel; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.nativeprotocol.NativeInboundMessage; +import org.opensearch.transport.nativeprotocol.NativeOutboundHandler; import java.io.EOFException; import java.io.IOException; @@ -72,7 +74,7 @@ public class NativeMessageHandler implements ProtocolMessageHandler { private static final Logger logger = LogManager.getLogger(NativeMessageHandler.class); private final ThreadPool threadPool; - private final OutboundHandler outboundHandler; + private final NativeOutboundHandler outboundHandler; private final NamedWriteableRegistry namedWriteableRegistry; private final TransportHandshaker handshaker; private final TransportKeepAlive keepAlive; @@ -82,7 +84,12 @@ public class NativeMessageHandler implements ProtocolMessageHandler { private final Tracer tracer; NativeMessageHandler( + String nodeName, + Version version, + String[] features, + StatsTracker statsTracker, ThreadPool threadPool, + BigArrays bigArrays, OutboundHandler outboundHandler, NamedWriteableRegistry namedWriteableRegistry, TransportHandshaker handshaker, @@ -92,7 +99,7 @@ public class NativeMessageHandler implements ProtocolMessageHandler { TransportKeepAlive keepAlive ) { this.threadPool = threadPool; - this.outboundHandler = outboundHandler; + this.outboundHandler = new NativeOutboundHandler(nodeName, version, features, statsTracker, threadPool, bigArrays, outboundHandler); this.namedWriteableRegistry = namedWriteableRegistry; this.handshaker = handshaker; this.requestHandlers = requestHandlers; @@ -492,4 +499,9 @@ public void onFailure(Exception e) { } } + @Override + public void setMessageListener(TransportMessageListener listener) { + outboundHandler.setMessageListener(listener); + } + } diff --git a/server/src/main/java/org/opensearch/transport/OutboundHandler.java b/server/src/main/java/org/opensearch/transport/OutboundHandler.java index b83dbdd0effe4..43f53e4011260 100644 --- a/server/src/main/java/org/opensearch/transport/OutboundHandler.java +++ b/server/src/main/java/org/opensearch/transport/OutboundHandler.java @@ -35,164 +35,47 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.Version; -import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.CheckedSupplier; -import org.opensearch.common.io.stream.ReleasableBytesStreamOutput; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lease.Releasables; import org.opensearch.common.network.CloseableChannel; import org.opensearch.common.transport.NetworkExceptionHelper; -import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.action.ActionListener; import org.opensearch.core.action.NotifyOnceListener; import org.opensearch.core.common.bytes.BytesReference; -import org.opensearch.core.common.transport.TransportAddress; -import org.opensearch.core.transport.TransportResponse; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; -import java.util.Set; /** * Outbound data handler * * @opensearch.internal */ -final class OutboundHandler { +public final class OutboundHandler { private static final Logger logger = LogManager.getLogger(OutboundHandler.class); - private final String nodeName; - private final Version version; - private final String[] features; private final StatsTracker statsTracker; private final ThreadPool threadPool; - private final BigArrays bigArrays; - private volatile TransportMessageListener messageListener = TransportMessageListener.NOOP_LISTENER; - OutboundHandler( - String nodeName, - Version version, - String[] features, - StatsTracker statsTracker, - ThreadPool threadPool, - BigArrays bigArrays - ) { - this.nodeName = nodeName; - this.version = version; - this.features = features; + public OutboundHandler(StatsTracker statsTracker, ThreadPool threadPool) { this.statsTracker = statsTracker; this.threadPool = threadPool; - this.bigArrays = bigArrays; } void sendBytes(TcpChannel channel, BytesReference bytes, ActionListener listener) { - SendContext sendContext = new SendContext(channel, () -> bytes, listener); + SendContext sendContext = new SendContext(statsTracker, channel, () -> bytes, listener); try { - internalSend(channel, sendContext); + sendBytes(channel, sendContext); } catch (IOException e) { // This should not happen as the bytes are already serialized throw new AssertionError(e); } } - /** - * Sends the request to the given channel. This method should be used to send {@link TransportRequest} - * objects back to the caller. - */ - void sendRequest( - final DiscoveryNode node, - final TcpChannel channel, - final long requestId, - final String action, - final TransportRequest request, - final TransportRequestOptions options, - final Version channelVersion, - final boolean compressRequest, - final boolean isHandshake - ) throws IOException, TransportException { - Version version = Version.min(this.version, channelVersion); - OutboundMessage.Request message = new OutboundMessage.Request( - threadPool.getThreadContext(), - features, - request, - version, - action, - requestId, - isHandshake, - compressRequest - ); - ActionListener listener = ActionListener.wrap(() -> messageListener.onRequestSent(node, requestId, action, request, options)); - sendMessage(channel, message, listener); - } - - /** - * Sends the response to the given channel. This method should be used to send {@link TransportResponse} - * objects back to the caller. - * - * @see #sendErrorResponse(Version, Set, TcpChannel, long, String, Exception) for sending error responses - */ - void sendResponse( - final Version nodeVersion, - final Set features, - final TcpChannel channel, - final long requestId, - final String action, - final TransportResponse response, - final boolean compress, - final boolean isHandshake - ) throws IOException { - Version version = Version.min(this.version, nodeVersion); - OutboundMessage.Response message = new OutboundMessage.Response( - threadPool.getThreadContext(), - features, - response, - version, - requestId, - isHandshake, - compress - ); - ActionListener listener = ActionListener.wrap(() -> messageListener.onResponseSent(requestId, action, response)); - sendMessage(channel, message, listener); - } - - /** - * Sends back an error response to the caller via the given channel - */ - void sendErrorResponse( - final Version nodeVersion, - final Set features, - final TcpChannel channel, - final long requestId, - final String action, - final Exception error - ) throws IOException { - Version version = Version.min(this.version, nodeVersion); - TransportAddress address = new TransportAddress(channel.getLocalAddress()); - RemoteTransportException tx = new RemoteTransportException(nodeName, address, action, error); - OutboundMessage.Response message = new OutboundMessage.Response( - threadPool.getThreadContext(), - features, - tx, - version, - requestId, - false, - false - ); - ActionListener listener = ActionListener.wrap(() -> messageListener.onResponseSent(requestId, action, error)); - sendMessage(channel, message, listener); - } - - private void sendMessage(TcpChannel channel, OutboundMessage networkMessage, ActionListener listener) throws IOException { - MessageSerializer serializer = new MessageSerializer(networkMessage, bigArrays); - SendContext sendContext = new SendContext(channel, serializer, listener, serializer); - internalSend(channel, sendContext); - } - - private void internalSend(TcpChannel channel, SendContext sendContext) throws IOException { + public void sendBytes(TcpChannel channel, SendContext sendContext) throws IOException { channel.getChannelStats().markAccessed(threadPool.relativeTimeInMillis()); BytesReference reference = sendContext.get(); // stash thread context so that channel event loop is not polluted by thread context @@ -205,59 +88,30 @@ private void internalSend(TcpChannel channel, SendContext sendContext) throws IO } } - void setMessageListener(TransportMessageListener listener) { - if (messageListener == TransportMessageListener.NOOP_LISTENER) { - messageListener = listener; - } else { - throw new IllegalStateException("Cannot set message listener twice"); - } - } - /** * Internal message serializer * * @opensearch.internal */ - private static class MessageSerializer implements CheckedSupplier, Releasable { - - private final OutboundMessage message; - private final BigArrays bigArrays; - private volatile ReleasableBytesStreamOutput bytesStreamOutput; - - private MessageSerializer(OutboundMessage message, BigArrays bigArrays) { - this.message = message; - this.bigArrays = bigArrays; - } - - @Override - public BytesReference get() throws IOException { - bytesStreamOutput = new ReleasableBytesStreamOutput(bigArrays); - return message.serialize(bytesStreamOutput); - } - - @Override - public void close() { - IOUtils.closeWhileHandlingException(bytesStreamOutput); - } - } - - private class SendContext extends NotifyOnceListener implements CheckedSupplier { - + public static class SendContext extends NotifyOnceListener implements CheckedSupplier { + private final StatsTracker statsTracker; private final TcpChannel channel; private final CheckedSupplier messageSupplier; private final ActionListener listener; private final Releasable optionalReleasable; private long messageSize = -1; - private SendContext( + SendContext( + StatsTracker statsTracker, TcpChannel channel, CheckedSupplier messageSupplier, ActionListener listener ) { - this(channel, messageSupplier, listener, null); + this(statsTracker, channel, messageSupplier, listener, null); } - private SendContext( + public SendContext( + StatsTracker statsTracker, TcpChannel channel, CheckedSupplier messageSupplier, ActionListener listener, @@ -267,6 +121,7 @@ private SendContext( this.messageSupplier = messageSupplier; this.listener = listener; this.optionalReleasable = optionalReleasable; + this.statsTracker = statsTracker; } public BytesReference get() throws IOException { diff --git a/server/src/main/java/org/opensearch/transport/ProtocolMessageHandler.java b/server/src/main/java/org/opensearch/transport/ProtocolMessageHandler.java index 714d91d1e74c7..3c3fafebc34df 100644 --- a/server/src/main/java/org/opensearch/transport/ProtocolMessageHandler.java +++ b/server/src/main/java/org/opensearch/transport/ProtocolMessageHandler.java @@ -17,6 +17,14 @@ */ public interface ProtocolMessageHandler { + /** + * Handles the message received on the channel. + * @param channel the channel on which the message was received + * @param message the message received + * @param startTime the start time + * @param slowLogThresholdMs the threshold for slow logs + * @param messageListener the message listener + */ public void messageReceived( TcpChannel channel, ProtocolInboundMessage message, @@ -24,4 +32,10 @@ public void messageReceived( long slowLogThresholdMs, TransportMessageListener messageListener ) throws IOException; + + /** + * Sets the message listener to be used by the handler. + * @param listener the message listener + */ + public void setMessageListener(TransportMessageListener listener); } diff --git a/server/src/main/java/org/opensearch/transport/ProtocolOutboundHandler.java b/server/src/main/java/org/opensearch/transport/ProtocolOutboundHandler.java new file mode 100644 index 0000000000000..42c5462fddf80 --- /dev/null +++ b/server/src/main/java/org/opensearch/transport/ProtocolOutboundHandler.java @@ -0,0 +1,70 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.transport; + +import org.opensearch.Version; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.core.transport.TransportResponse; + +import java.io.IOException; +import java.util.Set; + +/** + * Protocol based outbound data handler. + * Different transport protocols can have different implementations of this class. + * + * @opensearch.internal + */ +public abstract class ProtocolOutboundHandler { + + /** + * Sends the request to the given channel. This method should be used to send {@link TransportRequest} + * objects back to the caller. + */ + public abstract void sendRequest( + final DiscoveryNode node, + final TcpChannel channel, + final long requestId, + final String action, + final TransportRequest request, + final TransportRequestOptions options, + final Version channelVersion, + final boolean compressRequest, + final boolean isHandshake + ) throws IOException, TransportException; + + /** + * Sends the response to the given channel. This method should be used to send {@link TransportResponse} + * objects back to the caller. + * + * @see #sendErrorResponse(Version, Set, TcpChannel, long, String, Exception) for sending error responses + */ + public abstract void sendResponse( + final Version nodeVersion, + final Set features, + final TcpChannel channel, + final long requestId, + final String action, + final TransportResponse response, + final boolean compress, + final boolean isHandshake + ) throws IOException; + + /** + * Sends back an error response to the caller via the given channel + */ + public abstract void sendErrorResponse( + final Version nodeVersion, + final Set features, + final TcpChannel channel, + final long requestId, + final String action, + final Exception error + ) throws IOException; +} diff --git a/server/src/main/java/org/opensearch/transport/TcpTransport.java b/server/src/main/java/org/opensearch/transport/TcpTransport.java index 8ba0178577232..ffa3168da0b3e 100644 --- a/server/src/main/java/org/opensearch/transport/TcpTransport.java +++ b/server/src/main/java/org/opensearch/transport/TcpTransport.java @@ -71,6 +71,7 @@ import org.opensearch.node.Node; import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.nativeprotocol.NativeOutboundHandler; import java.io.IOException; import java.io.StreamCorruptedException; @@ -150,6 +151,7 @@ public abstract class TcpTransport extends AbstractLifecycleComponent implements private final TransportKeepAlive keepAlive; private final OutboundHandler outboundHandler; private final InboundHandler inboundHandler; + private final NativeOutboundHandler handshakerHandler; private final ResponseHandlers responseHandlers = new ResponseHandlers(); private final RequestHandlers requestHandlers = new RequestHandlers(); @@ -188,11 +190,20 @@ public TcpTransport( } BigArrays bigArrays = new BigArrays(pageCacheRecycler, circuitBreakerService, CircuitBreaker.IN_FLIGHT_REQUESTS); - this.outboundHandler = new OutboundHandler(nodeName, version, features, statsTracker, threadPool, bigArrays); + this.outboundHandler = new OutboundHandler(statsTracker, threadPool); + this.handshakerHandler = new NativeOutboundHandler( + nodeName, + version, + features, + statsTracker, + threadPool, + bigArrays, + outboundHandler + ); this.handshaker = new TransportHandshaker( version, threadPool, - (node, channel, requestId, v) -> outboundHandler.sendRequest( + (node, channel, requestId, v) -> handshakerHandler.sendRequest( node, channel, requestId, @@ -206,7 +217,12 @@ public TcpTransport( ); this.keepAlive = new TransportKeepAlive(threadPool, this.outboundHandler::sendBytes); this.inboundHandler = new InboundHandler( + nodeName, + version, + features, + statsTracker, threadPool, + bigArrays, outboundHandler, namedWriteableRegistry, handshaker, @@ -238,7 +254,7 @@ protected void doStart() {} @Override public synchronized void setMessageListener(TransportMessageListener listener) { - outboundHandler.setMessageListener(listener); + handshakerHandler.setMessageListener(listener); inboundHandler.setMessageListener(listener); } @@ -319,7 +335,7 @@ public void sendRequest(long requestId, String action, TransportRequest request, throw new NodeNotConnectedException(node, "connection already closed"); } TcpChannel channel = channel(options.type()); - outboundHandler.sendRequest(node, channel, requestId, action, request, options, getVersion(), compress, false); + handshakerHandler.sendRequest(node, channel, requestId, action, request, options, getVersion(), compress, false); } } diff --git a/server/src/main/java/org/opensearch/transport/TcpTransportChannel.java b/server/src/main/java/org/opensearch/transport/TcpTransportChannel.java index 81de0af07ea7c..750fd50a4c44c 100644 --- a/server/src/main/java/org/opensearch/transport/TcpTransportChannel.java +++ b/server/src/main/java/org/opensearch/transport/TcpTransportChannel.java @@ -50,7 +50,7 @@ public final class TcpTransportChannel extends BaseTcpTransportChannel { private final AtomicBoolean released = new AtomicBoolean(); - private final OutboundHandler outboundHandler; + private final ProtocolOutboundHandler outboundHandler; private final String action; private final long requestId; private final Version version; @@ -60,7 +60,7 @@ public final class TcpTransportChannel extends BaseTcpTransportChannel { private final Releasable breakerRelease; TcpTransportChannel( - OutboundHandler outboundHandler, + ProtocolOutboundHandler outboundHandler, TcpChannel channel, String action, long requestId, diff --git a/server/src/main/java/org/opensearch/transport/TransportStatus.java b/server/src/main/java/org/opensearch/transport/TransportStatus.java index dab572949e688..76377468535b9 100644 --- a/server/src/main/java/org/opensearch/transport/TransportStatus.java +++ b/server/src/main/java/org/opensearch/transport/TransportStatus.java @@ -76,11 +76,11 @@ public static byte setCompress(byte value) { return value; } - static boolean isHandshake(byte value) { // pkg private since it's only used internally + public static boolean isHandshake(byte value) { return (value & STATUS_HANDSHAKE) != 0; } - static byte setHandshake(byte value) { // pkg private since it's only used internally + public static byte setHandshake(byte value) { value |= STATUS_HANDSHAKE; return value; } diff --git a/server/src/main/java/org/opensearch/transport/CompressibleBytesOutputStream.java b/server/src/main/java/org/opensearch/transport/nativeprotocol/CompressibleBytesOutputStream.java similarity index 98% rename from server/src/main/java/org/opensearch/transport/CompressibleBytesOutputStream.java rename to server/src/main/java/org/opensearch/transport/nativeprotocol/CompressibleBytesOutputStream.java index 57707d3b44477..92b682370bcd5 100644 --- a/server/src/main/java/org/opensearch/transport/CompressibleBytesOutputStream.java +++ b/server/src/main/java/org/opensearch/transport/nativeprotocol/CompressibleBytesOutputStream.java @@ -30,7 +30,7 @@ * GitHub history for details. */ -package org.opensearch.transport; +package org.opensearch.transport.nativeprotocol; import org.opensearch.common.io.Streams; import org.opensearch.common.util.io.IOUtils; diff --git a/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeOutboundHandler.java b/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeOutboundHandler.java new file mode 100644 index 0000000000000..66ed0d8e3eb2b --- /dev/null +++ b/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeOutboundHandler.java @@ -0,0 +1,224 @@ +/* +* SPDX-License-Identifier: Apache-2.0 +* +* The OpenSearch Contributors require contributions made to +* this file be licensed under the Apache-2.0 license or a +* compatible open source license. +*/ + +/* +* Licensed to Elasticsearch under one or more contributor +* license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright +* ownership. Elasticsearch licenses this file to you under +* the Apache License, Version 2.0 (the "License"); you may +* not use this file except in compliance with the License. +* You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +/* +* Modifications Copyright OpenSearch Contributors. See +* GitHub history for details. +*/ + +package org.opensearch.transport.nativeprotocol; + +import org.opensearch.Version; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.CheckedSupplier; +import org.opensearch.common.io.stream.ReleasableBytesStreamOutput; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.transport.TransportResponse; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.OutboundHandler; +import org.opensearch.transport.ProtocolOutboundHandler; +import org.opensearch.transport.RemoteTransportException; +import org.opensearch.transport.StatsTracker; +import org.opensearch.transport.TcpChannel; +import org.opensearch.transport.TransportException; +import org.opensearch.transport.TransportMessageListener; +import org.opensearch.transport.TransportRequest; +import org.opensearch.transport.TransportRequestOptions; + +import java.io.IOException; +import java.util.Set; + +/** + * Outbound data handler + * + * @opensearch.internal + */ +public final class NativeOutboundHandler extends ProtocolOutboundHandler { + private final String nodeName; + private final Version version; + private final String[] features; + private final StatsTracker statsTracker; + private final ThreadPool threadPool; + private final BigArrays bigArrays; + private volatile TransportMessageListener messageListener = TransportMessageListener.NOOP_LISTENER; + private final OutboundHandler handler; + + public NativeOutboundHandler( + String nodeName, + Version version, + String[] features, + StatsTracker statsTracker, + ThreadPool threadPool, + BigArrays bigArrays, + OutboundHandler handler + ) { + this.nodeName = nodeName; + this.version = version; + this.features = features; + this.statsTracker = statsTracker; + this.threadPool = threadPool; + this.bigArrays = bigArrays; + this.handler = handler; + } + + /** + * Sends the request to the given channel. This method should be used to send {@link TransportRequest} + * objects back to the caller. + */ + @Override + public void sendRequest( + final DiscoveryNode node, + final TcpChannel channel, + final long requestId, + final String action, + final TransportRequest request, + final TransportRequestOptions options, + final Version channelVersion, + final boolean compressRequest, + final boolean isHandshake + ) throws IOException, TransportException { + Version version = Version.min(this.version, channelVersion); + NativeOutboundMessage.Request message = new NativeOutboundMessage.Request( + threadPool.getThreadContext(), + features, + request, + version, + action, + requestId, + isHandshake, + compressRequest + ); + ActionListener listener = ActionListener.wrap(() -> messageListener.onRequestSent(node, requestId, action, request, options)); + sendMessage(channel, message, listener); + } + + /** + * Sends the response to the given channel. This method should be used to send {@link TransportResponse} + * objects back to the caller. + * + * @see #sendErrorResponse(Version, Set, TcpChannel, long, String, Exception) for sending error responses + */ + @Override + public void sendResponse( + final Version nodeVersion, + final Set features, + final TcpChannel channel, + final long requestId, + final String action, + final TransportResponse response, + final boolean compress, + final boolean isHandshake + ) throws IOException { + Version version = Version.min(this.version, nodeVersion); + NativeOutboundMessage.Response message = new NativeOutboundMessage.Response( + threadPool.getThreadContext(), + features, + response, + version, + requestId, + isHandshake, + compress + ); + ActionListener listener = ActionListener.wrap(() -> messageListener.onResponseSent(requestId, action, response)); + sendMessage(channel, message, listener); + } + + /** + * Sends back an error response to the caller via the given channel + */ + @Override + public void sendErrorResponse( + final Version nodeVersion, + final Set features, + final TcpChannel channel, + final long requestId, + final String action, + final Exception error + ) throws IOException { + Version version = Version.min(this.version, nodeVersion); + TransportAddress address = new TransportAddress(channel.getLocalAddress()); + RemoteTransportException tx = new RemoteTransportException(nodeName, address, action, error); + NativeOutboundMessage.Response message = new NativeOutboundMessage.Response( + threadPool.getThreadContext(), + features, + tx, + version, + requestId, + false, + false + ); + ActionListener listener = ActionListener.wrap(() -> messageListener.onResponseSent(requestId, action, error)); + sendMessage(channel, message, listener); + } + + private void sendMessage(TcpChannel channel, NativeOutboundMessage networkMessage, ActionListener listener) throws IOException { + MessageSerializer serializer = new MessageSerializer(networkMessage, bigArrays); + OutboundHandler.SendContext sendContext = new OutboundHandler.SendContext(statsTracker, channel, serializer, listener, serializer); + handler.sendBytes(channel, sendContext); + } + + public void setMessageListener(TransportMessageListener listener) { + if (messageListener == TransportMessageListener.NOOP_LISTENER) { + messageListener = listener; + } else { + throw new IllegalStateException("Cannot set message listener twice"); + } + } + + /** + * Internal message serializer + * + * @opensearch.internal + */ + private static class MessageSerializer implements CheckedSupplier, Releasable { + + private final NativeOutboundMessage message; + private final BigArrays bigArrays; + private volatile ReleasableBytesStreamOutput bytesStreamOutput; + + private MessageSerializer(NativeOutboundMessage message, BigArrays bigArrays) { + this.message = message; + this.bigArrays = bigArrays; + } + + @Override + public BytesReference get() throws IOException { + bytesStreamOutput = new ReleasableBytesStreamOutput(bigArrays); + return message.serialize(bytesStreamOutput); + } + + @Override + public void close() { + IOUtils.closeWhileHandlingException(bytesStreamOutput); + } + } +} diff --git a/server/src/main/java/org/opensearch/transport/OutboundMessage.java b/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeOutboundMessage.java similarity index 91% rename from server/src/main/java/org/opensearch/transport/OutboundMessage.java rename to server/src/main/java/org/opensearch/transport/nativeprotocol/NativeOutboundMessage.java index 3dafc6fb5eea9..d7590fb9e03ab 100644 --- a/server/src/main/java/org/opensearch/transport/OutboundMessage.java +++ b/server/src/main/java/org/opensearch/transport/nativeprotocol/NativeOutboundMessage.java @@ -29,7 +29,7 @@ * GitHub history for details. */ -package org.opensearch.transport; +package org.opensearch.transport.nativeprotocol; import org.opensearch.Version; import org.opensearch.common.io.stream.BytesStreamOutput; @@ -39,6 +39,10 @@ import org.opensearch.core.common.bytes.CompositeBytesReference; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.transport.BytesTransportRequest; +import org.opensearch.transport.RemoteTransportException; +import org.opensearch.transport.TcpHeader; +import org.opensearch.transport.TransportStatus; import java.io.IOException; import java.util.Set; @@ -48,11 +52,11 @@ * * @opensearch.internal */ -abstract class OutboundMessage extends NetworkMessage { +abstract class NativeOutboundMessage extends NetworkMessage { private final Writeable message; - OutboundMessage(ThreadContext threadContext, Version version, byte status, long requestId, Writeable message) { + NativeOutboundMessage(ThreadContext threadContext, Version version, byte status, long requestId, Writeable message) { super(threadContext, version, status, requestId); this.message = message; } @@ -93,7 +97,7 @@ protected BytesReference writeMessage(CompressibleBytesOutputStream stream) thro if (message instanceof BytesTransportRequest) { BytesTransportRequest bRequest = (BytesTransportRequest) message; bRequest.writeThin(stream); - zeroCopyBuffer = bRequest.bytes; + zeroCopyBuffer = bRequest.bytes(); } else if (message instanceof RemoteTransportException) { stream.writeException((RemoteTransportException) message); zeroCopyBuffer = BytesArray.EMPTY; @@ -119,7 +123,7 @@ protected BytesReference writeMessage(CompressibleBytesOutputStream stream) thro * * @opensearch.internal */ - static class Request extends OutboundMessage { + static class Request extends NativeOutboundMessage { private final String[] features; private final String action; @@ -149,7 +153,7 @@ protected void writeVariableHeader(StreamOutput stream) throws IOException { private static byte setStatus(boolean compress, boolean isHandshake, Writeable message) { byte status = 0; status = TransportStatus.setRequest(status); - if (compress && OutboundMessage.canCompress(message)) { + if (compress && NativeOutboundMessage.canCompress(message)) { status = TransportStatus.setCompress(status); } if (isHandshake) { @@ -165,7 +169,7 @@ private static byte setStatus(boolean compress, boolean isHandshake, Writeable m * * @opensearch.internal */ - static class Response extends OutboundMessage { + static class Response extends NativeOutboundMessage { private final Set features; diff --git a/server/src/main/java/org/opensearch/transport/NetworkMessage.java b/server/src/main/java/org/opensearch/transport/nativeprotocol/NetworkMessage.java similarity index 96% rename from server/src/main/java/org/opensearch/transport/NetworkMessage.java rename to server/src/main/java/org/opensearch/transport/nativeprotocol/NetworkMessage.java index f02d664b65929..c197539d2e009 100644 --- a/server/src/main/java/org/opensearch/transport/NetworkMessage.java +++ b/server/src/main/java/org/opensearch/transport/nativeprotocol/NetworkMessage.java @@ -29,11 +29,12 @@ * GitHub history for details. */ -package org.opensearch.transport; +package org.opensearch.transport.nativeprotocol; import org.opensearch.Version; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.transport.TransportStatus; /** * Represents a transport message sent over the network. Subclasses implement serialization and diff --git a/server/src/test/java/org/opensearch/transport/InboundDecoderTests.java b/server/src/test/java/org/opensearch/transport/InboundDecoderTests.java index 4d671443f396e..47ae8f9783ea3 100644 --- a/server/src/test/java/org/opensearch/transport/InboundDecoderTests.java +++ b/server/src/test/java/org/opensearch/transport/InboundDecoderTests.java @@ -39,19 +39,29 @@ import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.transport.TransportMessage; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.VersionUtils; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import static org.hamcrest.Matchers.hasItems; -public class InboundDecoderTests extends OpenSearchTestCase { +public abstract class InboundDecoderTests extends OpenSearchTestCase { - private ThreadContext threadContext; + protected ThreadContext threadContext; + + protected abstract BytesReference serialize( + boolean isRequest, + Version version, + boolean handshake, + boolean compress, + String action, + long requestId, + Writeable transportMessage + ) throws IOException; @Override public void setUp() throws Exception { @@ -65,36 +75,16 @@ public void testDecode() throws IOException { long requestId = randomNonNegativeLong(); final String headerKey = randomAlphaOfLength(10); final String headerValue = randomAlphaOfLength(20); + TransportMessage transportMessage; if (isRequest) { threadContext.putHeader(headerKey, headerValue); + transportMessage = new TestRequest(randomAlphaOfLength(100)); } else { threadContext.addResponseHeader(headerKey, headerValue); - } - OutboundMessage message; - if (isRequest) { - message = new OutboundMessage.Request( - threadContext, - new String[0], - new TestRequest(randomAlphaOfLength(100)), - Version.CURRENT, - action, - requestId, - false, - false - ); - } else { - message = new OutboundMessage.Response( - threadContext, - Collections.emptySet(), - new TestResponse(randomAlphaOfLength(100)), - Version.CURRENT, - requestId, - false, - false - ); + transportMessage = new TestResponse(randomAlphaOfLength(100)); } - final BytesReference totalBytes = message.serialize(new BytesStreamOutput()); + final BytesReference totalBytes = serialize(isRequest, Version.CURRENT, false, false, action, requestId, transportMessage); int totalHeaderSize = TcpHeader.headerSize(Version.CURRENT) + totalBytes.getInt(TcpHeader.VARIABLE_HEADER_SIZE_POSITION); final BytesReference messageBytes = totalBytes.slice(totalHeaderSize, totalBytes.length() - totalHeaderSize); @@ -142,18 +132,16 @@ public void testDecodeHandshakeCompatibility() throws IOException { final String headerValue = randomAlphaOfLength(20); threadContext.putHeader(headerKey, headerValue); Version handshakeCompatVersion = Version.CURRENT.minimumCompatibilityVersion().minimumCompatibilityVersion(); - OutboundMessage message = new OutboundMessage.Request( - threadContext, - new String[0], - new TestRequest(randomAlphaOfLength(100)), + + final BytesReference bytes = serialize( + true, handshakeCompatVersion, + true, + false, action, requestId, - true, - false + new TestRequest(randomAlphaOfLength(100)) ); - - final BytesReference bytes = message.serialize(new BytesStreamOutput()); int totalHeaderSize = TcpHeader.headerSize(handshakeCompatVersion) + bytes.getInt(TcpHeader.VARIABLE_HEADER_SIZE_POSITION); InboundDecoder decoder = new InboundDecoder(Version.CURRENT, PageCacheRecycler.NON_RECYCLING_INSTANCE); @@ -183,34 +171,14 @@ public void testCompressedDecode() throws IOException { } else { threadContext.addResponseHeader(headerKey, headerValue); } - OutboundMessage message; TransportMessage transportMessage; if (isRequest) { transportMessage = new TestRequest(randomAlphaOfLength(100)); - message = new OutboundMessage.Request( - threadContext, - new String[0], - transportMessage, - Version.CURRENT, - action, - requestId, - false, - true - ); } else { transportMessage = new TestResponse(randomAlphaOfLength(100)); - message = new OutboundMessage.Response( - threadContext, - Collections.emptySet(), - transportMessage, - Version.CURRENT, - requestId, - false, - true - ); } - final BytesReference totalBytes = message.serialize(new BytesStreamOutput()); + final BytesReference totalBytes = serialize(isRequest, Version.CURRENT, false, true, action, requestId, transportMessage); final BytesStreamOutput out = new BytesStreamOutput(); transportMessage.writeTo(out); final BytesReference uncompressedBytes = out.bytes(); @@ -260,18 +228,16 @@ public void testCompressedDecodeHandshakeCompatibility() throws IOException { final String headerValue = randomAlphaOfLength(20); threadContext.putHeader(headerKey, headerValue); Version handshakeCompatVersion = Version.CURRENT.minimumCompatibilityVersion().minimumCompatibilityVersion(); - OutboundMessage message = new OutboundMessage.Request( - threadContext, - new String[0], - new TestRequest(randomAlphaOfLength(100)), + + final BytesReference bytes = serialize( + true, handshakeCompatVersion, + true, + true, action, requestId, - true, - true + new TestRequest(randomAlphaOfLength(100)) ); - - final BytesReference bytes = message.serialize(new BytesStreamOutput()); int totalHeaderSize = TcpHeader.headerSize(handshakeCompatVersion) + bytes.getInt(TcpHeader.VARIABLE_HEADER_SIZE_POSITION); InboundDecoder decoder = new InboundDecoder(Version.CURRENT, PageCacheRecycler.NON_RECYCLING_INSTANCE); @@ -294,19 +260,17 @@ public void testVersionIncompatibilityDecodeException() throws IOException { String action = "test-request"; long requestId = randomNonNegativeLong(); Version incompatibleVersion = Version.CURRENT.minimumCompatibilityVersion().minimumCompatibilityVersion(); - OutboundMessage message = new OutboundMessage.Request( - threadContext, - new String[0], - new TestRequest(randomAlphaOfLength(100)), + + final BytesReference bytes = serialize( + true, incompatibleVersion, + false, + true, action, requestId, - false, - true + new TestRequest(randomAlphaOfLength(100)) ); - final BytesReference bytes = message.serialize(new BytesStreamOutput()); - InboundDecoder decoder = new InboundDecoder(Version.CURRENT, PageCacheRecycler.NON_RECYCLING_INSTANCE); final ArrayList fragments = new ArrayList<>(); final ReleasableBytesReference releasable1 = ReleasableBytesReference.wrap(bytes); diff --git a/server/src/test/java/org/opensearch/transport/InboundHandlerTests.java b/server/src/test/java/org/opensearch/transport/InboundHandlerTests.java index 2dde27d62e759..2553e7740990b 100644 --- a/server/src/test/java/org/opensearch/transport/InboundHandlerTests.java +++ b/server/src/test/java/org/opensearch/transport/InboundHandlerTests.java @@ -39,16 +39,17 @@ import org.opensearch.Version; import org.opensearch.common.bytes.ReleasableBytesReference; import org.opensearch.common.collect.Tuple; -import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.InputStreamStreamInput; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.tasks.TaskManager; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.MockLogAppender; @@ -75,7 +76,17 @@ import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.Matchers.instanceOf; -public class InboundHandlerTests extends OpenSearchTestCase { +public abstract class InboundHandlerTests extends OpenSearchTestCase { + + public abstract BytesReference serializeOutboundRequest( + ThreadContext threadContext, + Writeable message, + Version version, + String action, + long requestId, + boolean compress, + boolean handshake + ) throws IOException; private final TestThreadPool threadPool = new TestThreadPool(getClass().getName()); private final Version version = Version.CURRENT; @@ -101,19 +112,17 @@ public void sendMessage(BytesReference reference, ActionListener listener) }; NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList()); TransportHandshaker handshaker = new TransportHandshaker(version, threadPool, (n, c, r, v) -> {}); - outboundHandler = new OutboundHandler( - "node", - version, - new String[0], - new StatsTracker(), - threadPool, - BigArrays.NON_RECYCLING_INSTANCE - ); + outboundHandler = new OutboundHandler(new StatsTracker(), threadPool); TransportKeepAlive keepAlive = new TransportKeepAlive(threadPool, outboundHandler::sendBytes); requestHandlers = new Transport.RequestHandlers(); responseHandlers = new Transport.ResponseHandlers(); handler = new InboundHandler( + "node", + version, + new String[0], + new StatsTracker(), threadPool, + BigArrays.NON_RECYCLING_INSTANCE, outboundHandler, namedWriteableRegistry, handshaker, @@ -195,9 +204,9 @@ public TestResponse read(StreamInput in) throws IOException { ); requestHandlers.registerHandler(registry); String requestValue = randomAlphaOfLength(10); - OutboundMessage.Request request = new OutboundMessage.Request( + + BytesReference fullRequestBytes = serializeOutboundRequest( threadPool.getThreadContext(), - new String[0], new TestRequest(requestValue), version, action, @@ -205,8 +214,6 @@ public TestResponse read(StreamInput in) throws IOException { false, false ); - - BytesReference fullRequestBytes = request.serialize(new BytesStreamOutput()); BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize); Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); NativeInboundMessage requestMessage = new NativeInboundMessage( @@ -393,18 +400,8 @@ public TestResponse read(StreamInput in) throws IOException { requestHandlers.registerHandler(registry); String requestValue = randomAlphaOfLength(10); - OutboundMessage.Request request = new OutboundMessage.Request( - threadPool.getThreadContext(), - new String[0], - new TestRequest(requestValue), - version, - action, - requestId, - false, - false - ); - outboundHandler.setMessageListener(new TransportMessageListener() { + handler.setMessageListener(new TransportMessageListener() { @Override public void onResponseSent(long requestId, String action, Exception error) { exceptionCaptor.set(error); @@ -412,7 +409,15 @@ public void onResponseSent(long requestId, String action, Exception error) { }); // Create the request payload with 1 byte overflow - final BytesRef bytes = request.serialize(new BytesStreamOutput()).toBytesRef(); + final BytesRef bytes = serializeOutboundRequest( + threadPool.getThreadContext(), + new TestRequest(requestValue), + version, + action, + requestId, + false, + false + ).toBytesRef(); final ByteBuffer buffer = ByteBuffer.allocate(bytes.length + 1); buffer.put(bytes.bytes, 0, bytes.length); buffer.put((byte) 1); @@ -469,9 +474,16 @@ public TestResponse read(StreamInput in) throws IOException { requestHandlers.registerHandler(registry); String requestValue = randomAlphaOfLength(10); - OutboundMessage.Request request = new OutboundMessage.Request( + + handler.setMessageListener(new TransportMessageListener() { + @Override + public void onResponseSent(long requestId, String action, Exception error) { + exceptionCaptor.set(error); + } + }); + + final BytesReference fullRequestBytes = serializeOutboundRequest( threadPool.getThreadContext(), - new String[0], new TestRequest(requestValue), version, action, @@ -479,15 +491,6 @@ public TestResponse read(StreamInput in) throws IOException { false, false ); - - outboundHandler.setMessageListener(new TransportMessageListener() { - @Override - public void onResponseSent(long requestId, String action, Exception error) { - exceptionCaptor.set(error); - } - }); - - final BytesReference fullRequestBytes = request.serialize(new BytesStreamOutput()); // Create the request payload by intentionally stripping 1 byte away BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize - 1); Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); @@ -547,9 +550,9 @@ public TestResponse read(StreamInput in) throws IOException { ); requestHandlers.registerHandler(registry); String requestValue = randomAlphaOfLength(10); - OutboundMessage.Request request = new OutboundMessage.Request( + + BytesReference fullRequestBytes = serializeOutboundRequest( threadPool.getThreadContext(), - new String[0], new TestRequest(requestValue), version, action, @@ -557,8 +560,6 @@ public TestResponse read(StreamInput in) throws IOException { false, false ); - - BytesReference fullRequestBytes = request.serialize(new BytesStreamOutput()); BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize); Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); NativeInboundMessage requestMessage = new NativeInboundMessage( @@ -643,9 +644,9 @@ public TestResponse read(StreamInput in) throws IOException { ); requestHandlers.registerHandler(registry); String requestValue = randomAlphaOfLength(10); - OutboundMessage.Request request = new OutboundMessage.Request( + + BytesReference fullRequestBytes = serializeOutboundRequest( threadPool.getThreadContext(), - new String[0], new TestRequest(requestValue), version, action, @@ -653,8 +654,6 @@ public TestResponse read(StreamInput in) throws IOException { false, false ); - - BytesReference fullRequestBytes = request.serialize(new BytesStreamOutput()); BytesReference requestContent = fullRequestBytes.slice(headerSize, fullRequestBytes.length() - headerSize); Header requestHeader = new Header(fullRequestBytes.length() - 6, requestId, TransportStatus.setRequest((byte) 0), version); NativeInboundMessage requestMessage = new NativeInboundMessage( diff --git a/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java b/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java index d54f7e6fd2c2b..5a89bf1e0ead3 100644 --- a/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java +++ b/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java @@ -53,7 +53,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; @@ -64,12 +63,22 @@ import static org.hamcrest.Matchers.instanceOf; -public class InboundPipelineTests extends OpenSearchTestCase { +public abstract class InboundPipelineTests extends OpenSearchTestCase { private static final int BYTE_THRESHOLD = 128 * 1024; - private final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); - - public void testPipelineHandling() throws IOException { + public final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + + protected abstract BytesReference serialize( + boolean isRequest, + Version version, + boolean handshake, + boolean compress, + String action, + long requestId, + String value + ) throws IOException; + + public void testPipelineHandlingForNativeProtocol() throws IOException { final List> expected = new ArrayList<>(); final List> actual = new ArrayList<>(); final List toRelease = new ArrayList<>(); @@ -86,10 +95,10 @@ public void testPipelineHandling() throws IOException { actualData = new MessageData(version, requestId, isRequest, isCompressed, header.getActionName(), null); } else if (isRequest) { final TestRequest request = new TestRequest(message.openOrGetStreamInput()); - actualData = new MessageData(version, requestId, isRequest, isCompressed, header.getActionName(), request.value); + actualData = new MessageData(version, requestId, isRequest, isCompressed, header.getActionName(), request.getValue()); } else { final TestResponse response = new TestResponse(message.openOrGetStreamInput()); - actualData = new MessageData(version, requestId, isRequest, isCompressed, null, response.value); + actualData = new MessageData(version, requestId, isRequest, isCompressed, null, response.getValue()); } actual.add(new Tuple<>(actualData, message.getException())); } catch (IOException e) { @@ -128,49 +137,23 @@ public void testPipelineHandling() throws IOException { final MessageData messageData; Exception expectedExceptionClass = null; - OutboundMessage message; + // NativeOutboundMessage message; + final BytesReference reference; if (isRequest) { if (rarely()) { messageData = new MessageData(version, requestId, true, isCompressed, breakThisAction, null); - message = new OutboundMessage.Request( - threadContext, - new String[0], - new TestRequest(value), - version, - breakThisAction, - requestId, - false, - isCompressed - ); + reference = serialize(true, version, false, isCompressed, breakThisAction, requestId, value); expectedExceptionClass = new CircuitBreakingException("", CircuitBreaker.Durability.PERMANENT); } else { messageData = new MessageData(version, requestId, true, isCompressed, actionName, value); - message = new OutboundMessage.Request( - threadContext, - new String[0], - new TestRequest(value), - version, - actionName, - requestId, - false, - isCompressed - ); + reference = serialize(true, version, false, isCompressed, actionName, requestId, value); } } else { messageData = new MessageData(version, requestId, false, isCompressed, null, value); - message = new OutboundMessage.Response( - threadContext, - Collections.emptySet(), - new TestResponse(value), - version, - requestId, - false, - isCompressed - ); + reference = serialize(false, version, false, isCompressed, actionName, requestId, value); } expected.add(new Tuple<>(messageData, expectedExceptionClass)); - final BytesReference reference = message.serialize(new BytesStreamOutput()); Streams.copy(reference.streamInput(), streamOutput); } @@ -231,31 +214,7 @@ public void testDecodeExceptionIsPropagated() throws IOException { final boolean isRequest = randomBoolean(); final long requestId = randomNonNegativeLong(); - OutboundMessage message; - if (isRequest) { - message = new OutboundMessage.Request( - threadContext, - new String[0], - new TestRequest(value), - invalidVersion, - actionName, - requestId, - false, - false - ); - } else { - message = new OutboundMessage.Response( - threadContext, - Collections.emptySet(), - new TestResponse(value), - invalidVersion, - requestId, - false, - false - ); - } - - final BytesReference reference = message.serialize(streamOutput); + final BytesReference reference = serialize(isRequest, invalidVersion, false, false, actionName, requestId, value); try (ReleasableBytesReference releasable = ReleasableBytesReference.wrap(reference)) { expectThrows(IllegalStateException.class, () -> pipeline.handleBytes(new FakeTcpChannel(), releasable)); } @@ -285,31 +244,7 @@ public void testEnsureBodyIsNotPrematurelyReleased() throws IOException { final boolean isRequest = randomBoolean(); final long requestId = randomNonNegativeLong(); - OutboundMessage message; - if (isRequest) { - message = new OutboundMessage.Request( - threadContext, - new String[0], - new TestRequest(value), - version, - actionName, - requestId, - false, - false - ); - } else { - message = new OutboundMessage.Response( - threadContext, - Collections.emptySet(), - new TestResponse(value), - version, - requestId, - false, - false - ); - } - - final BytesReference reference = message.serialize(streamOutput); + final BytesReference reference = serialize(isRequest, version, false, false, actionName, requestId, value); final int fixedHeaderSize = TcpHeader.headerSize(Version.CURRENT); final int variableHeaderSize = reference.getInt(fixedHeaderSize - 4); final int totalHeaderSize = fixedHeaderSize + variableHeaderSize; diff --git a/server/src/test/java/org/opensearch/transport/NativeOutboundHandlerTests.java b/server/src/test/java/org/opensearch/transport/NativeOutboundHandlerTests.java new file mode 100644 index 0000000000000..01f19bea7a37f --- /dev/null +++ b/server/src/test/java/org/opensearch/transport/NativeOutboundHandlerTests.java @@ -0,0 +1,302 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.transport; + +import org.opensearch.OpenSearchException; +import org.opensearch.Version; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.bytes.ReleasableBytesReference; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.PageCacheRecycler; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.common.util.io.Streams; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.breaker.CircuitBreaker; +import org.opensearch.core.common.breaker.NoopCircuitBreaker; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.transport.TransportResponse; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.nativeprotocol.NativeInboundMessage; +import org.opensearch.transport.nativeprotocol.NativeOutboundHandler; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.Collections; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.LongSupplier; +import java.util.function.Predicate; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.instanceOf; + +public class NativeOutboundHandlerTests extends OpenSearchTestCase { + + private final String feature1 = "feature1"; + private final String feature2 = "feature2"; + private final TestThreadPool threadPool = new TestThreadPool(getClass().getName()); + private final TransportRequestOptions options = TransportRequestOptions.EMPTY; + private final AtomicReference> message = new AtomicReference<>(); + private InboundPipeline pipeline; + private OutboundHandler handler; + private NativeOutboundHandler nativeOutboundHandler; + private FakeTcpChannel channel; + private DiscoveryNode node; + + @Before + public void setUp() throws Exception { + super.setUp(); + channel = new FakeTcpChannel(randomBoolean(), buildNewFakeTransportAddress().address(), buildNewFakeTransportAddress().address()); + TransportAddress transportAddress = buildNewFakeTransportAddress(); + node = new DiscoveryNode("", transportAddress, Version.CURRENT); + String[] features = { feature1, feature2 }; + StatsTracker statsTracker = new StatsTracker(); + handler = new OutboundHandler(statsTracker, threadPool); + nativeOutboundHandler = new NativeOutboundHandler( + "node", + Version.CURRENT, + features, + statsTracker, + threadPool, + BigArrays.NON_RECYCLING_INSTANCE, + handler + ); + + final LongSupplier millisSupplier = () -> TimeValue.nsecToMSec(System.nanoTime()); + final InboundDecoder decoder = new InboundDecoder(Version.CURRENT, PageCacheRecycler.NON_RECYCLING_INSTANCE); + final Supplier breaker = () -> new NoopCircuitBreaker("test"); + final InboundAggregator aggregator = new InboundAggregator(breaker, (Predicate) action -> true); + pipeline = new InboundPipeline(statsTracker, millisSupplier, decoder, aggregator, (c, m) -> { + try (BytesStreamOutput streamOutput = new BytesStreamOutput()) { + NativeInboundMessage m1 = (NativeInboundMessage) m; + Streams.copy(m1.openOrGetStreamInput(), streamOutput); + message.set(new Tuple<>(m1.getHeader(), streamOutput.bytes())); + } catch (IOException e) { + throw new AssertionError(e); + } + }); + } + + @After + public void tearDown() throws Exception { + ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); + super.tearDown(); + } + + public void testSendRequest() throws IOException { + ThreadContext threadContext = threadPool.getThreadContext(); + Version version = randomFrom(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()); + String action = "handshake"; + long requestId = randomLongBetween(0, 300); + boolean isHandshake = randomBoolean(); + boolean compress = randomBoolean(); + String value = "message"; + threadContext.putHeader("header", "header_value"); + TestRequest request = new TestRequest(value); + + AtomicReference nodeRef = new AtomicReference<>(); + AtomicLong requestIdRef = new AtomicLong(); + AtomicReference actionRef = new AtomicReference<>(); + AtomicReference requestRef = new AtomicReference<>(); + nativeOutboundHandler.setMessageListener(new TransportMessageListener() { + @Override + public void onRequestSent( + DiscoveryNode node, + long requestId, + String action, + TransportRequest request, + TransportRequestOptions options + ) { + nodeRef.set(node); + requestIdRef.set(requestId); + actionRef.set(action); + requestRef.set(request); + } + }); + nativeOutboundHandler.sendRequest(node, channel, requestId, action, request, options, version, compress, isHandshake); + + BytesReference reference = channel.getMessageCaptor().get(); + ActionListener sendListener = channel.getListenerCaptor().get(); + if (randomBoolean()) { + sendListener.onResponse(null); + } else { + sendListener.onFailure(new IOException("failed")); + } + assertEquals(node, nodeRef.get()); + assertEquals(requestId, requestIdRef.get()); + assertEquals(action, actionRef.get()); + assertEquals(request, requestRef.get()); + + pipeline.handleBytes(channel, new ReleasableBytesReference(reference, () -> {})); + final Tuple tuple = message.get(); + final Header header = tuple.v1(); + final TestRequest message = new TestRequest(tuple.v2().streamInput()); + assertEquals(version, header.getVersion()); + assertEquals(requestId, header.getRequestId()); + assertTrue(header.isRequest()); + assertFalse(header.isResponse()); + if (isHandshake) { + assertTrue(header.isHandshake()); + } else { + assertFalse(header.isHandshake()); + } + if (compress) { + assertTrue(header.isCompressed()); + } else { + assertFalse(header.isCompressed()); + } + + assertEquals(value, message.getValue()); + assertEquals("header_value", header.getHeaders().v1().get("header")); + } + + public void testSendResponse() throws IOException { + ThreadContext threadContext = threadPool.getThreadContext(); + Version version = randomFrom(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()); + String action = "handshake"; + long requestId = randomLongBetween(0, 300); + boolean isHandshake = randomBoolean(); + boolean compress = randomBoolean(); + String value = "message"; + threadContext.putHeader("header", "header_value"); + TestResponse response = new TestResponse(value); + + AtomicLong requestIdRef = new AtomicLong(); + AtomicReference actionRef = new AtomicReference<>(); + AtomicReference responseRef = new AtomicReference<>(); + nativeOutboundHandler.setMessageListener(new TransportMessageListener() { + @Override + public void onResponseSent(long requestId, String action, TransportResponse response) { + requestIdRef.set(requestId); + actionRef.set(action); + responseRef.set(response); + } + }); + nativeOutboundHandler.sendResponse(version, Collections.emptySet(), channel, requestId, action, response, compress, isHandshake); + + BytesReference reference = channel.getMessageCaptor().get(); + ActionListener sendListener = channel.getListenerCaptor().get(); + if (randomBoolean()) { + sendListener.onResponse(null); + } else { + sendListener.onFailure(new IOException("failed")); + } + assertEquals(requestId, requestIdRef.get()); + assertEquals(action, actionRef.get()); + assertEquals(response, responseRef.get()); + + pipeline.handleBytes(channel, new ReleasableBytesReference(reference, () -> {})); + final Tuple tuple = message.get(); + final Header header = tuple.v1(); + final TestResponse message = new TestResponse(tuple.v2().streamInput()); + assertEquals(version, header.getVersion()); + assertEquals(requestId, header.getRequestId()); + assertFalse(header.isRequest()); + assertTrue(header.isResponse()); + if (isHandshake) { + assertTrue(header.isHandshake()); + } else { + assertFalse(header.isHandshake()); + } + if (compress) { + assertTrue(header.isCompressed()); + } else { + assertFalse(header.isCompressed()); + } + + assertFalse(header.isError()); + + assertEquals(value, message.getValue()); + assertEquals("header_value", header.getHeaders().v1().get("header")); + } + + public void testErrorResponse() throws IOException { + ThreadContext threadContext = threadPool.getThreadContext(); + Version version = randomFrom(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()); + String action = "handshake"; + long requestId = randomLongBetween(0, 300); + threadContext.putHeader("header", "header_value"); + OpenSearchException error = new OpenSearchException("boom"); + + AtomicLong requestIdRef = new AtomicLong(); + AtomicReference actionRef = new AtomicReference<>(); + AtomicReference responseRef = new AtomicReference<>(); + nativeOutboundHandler.setMessageListener(new TransportMessageListener() { + @Override + public void onResponseSent(long requestId, String action, Exception error) { + requestIdRef.set(requestId); + actionRef.set(action); + responseRef.set(error); + } + }); + nativeOutboundHandler.sendErrorResponse(version, Collections.emptySet(), channel, requestId, action, error); + + BytesReference reference = channel.getMessageCaptor().get(); + ActionListener sendListener = channel.getListenerCaptor().get(); + if (randomBoolean()) { + sendListener.onResponse(null); + } else { + sendListener.onFailure(new IOException("failed")); + } + assertEquals(requestId, requestIdRef.get()); + assertEquals(action, actionRef.get()); + assertEquals(error, responseRef.get()); + + pipeline.handleBytes(channel, new ReleasableBytesReference(reference, () -> {})); + final Tuple tuple = message.get(); + final Header header = tuple.v1(); + assertEquals(version, header.getVersion()); + assertEquals(requestId, header.getRequestId()); + assertFalse(header.isRequest()); + assertTrue(header.isResponse()); + assertFalse(header.isCompressed()); + assertFalse(header.isHandshake()); + assertTrue(header.isError()); + + RemoteTransportException remoteException = tuple.v2().streamInput().readException(); + assertThat(remoteException.getCause(), instanceOf(OpenSearchException.class)); + assertEquals(remoteException.getCause().getMessage(), "boom"); + assertEquals(action, remoteException.action()); + assertEquals(channel.getLocalAddress(), remoteException.address().address()); + + assertEquals("header_value", header.getHeaders().v1().get("header")); + } +} diff --git a/server/src/test/java/org/opensearch/transport/OutboundHandlerTests.java b/server/src/test/java/org/opensearch/transport/OutboundHandlerTests.java index ad7d4401af13c..7e7c60e2d3d29 100644 --- a/server/src/test/java/org/opensearch/transport/OutboundHandlerTests.java +++ b/server/src/test/java/org/opensearch/transport/OutboundHandlerTests.java @@ -32,79 +32,33 @@ package org.opensearch.transport; -import org.opensearch.OpenSearchException; -import org.opensearch.Version; -import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.bytes.ReleasableBytesReference; -import org.opensearch.common.collect.Tuple; -import org.opensearch.common.io.stream.BytesStreamOutput; -import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.BigArrays; -import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.common.util.io.Streams; import org.opensearch.core.action.ActionListener; -import org.opensearch.core.common.breaker.CircuitBreaker; -import org.opensearch.core.common.breaker.NoopCircuitBreaker; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; -import org.opensearch.core.common.transport.TransportAddress; -import org.opensearch.core.transport.TransportResponse; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; -import org.opensearch.transport.nativeprotocol.NativeInboundMessage; import org.junit.After; import org.junit.Before; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.Collections; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.LongSupplier; -import java.util.function.Predicate; -import java.util.function.Supplier; - -import static org.hamcrest.Matchers.instanceOf; public class OutboundHandlerTests extends OpenSearchTestCase { - private final String feature1 = "feature1"; - private final String feature2 = "feature2"; private final TestThreadPool threadPool = new TestThreadPool(getClass().getName()); - private final TransportRequestOptions options = TransportRequestOptions.EMPTY; - private final AtomicReference> message = new AtomicReference<>(); - private InboundPipeline pipeline; private OutboundHandler handler; private FakeTcpChannel channel; - private DiscoveryNode node; @Before public void setUp() throws Exception { super.setUp(); channel = new FakeTcpChannel(randomBoolean(), buildNewFakeTransportAddress().address(), buildNewFakeTransportAddress().address()); - TransportAddress transportAddress = buildNewFakeTransportAddress(); - node = new DiscoveryNode("", transportAddress, Version.CURRENT); - String[] features = { feature1, feature2 }; StatsTracker statsTracker = new StatsTracker(); - handler = new OutboundHandler("node", Version.CURRENT, features, statsTracker, threadPool, BigArrays.NON_RECYCLING_INSTANCE); - - final LongSupplier millisSupplier = () -> TimeValue.nsecToMSec(System.nanoTime()); - final InboundDecoder decoder = new InboundDecoder(Version.CURRENT, PageCacheRecycler.NON_RECYCLING_INSTANCE); - final Supplier breaker = () -> new NoopCircuitBreaker("test"); - final InboundAggregator aggregator = new InboundAggregator(breaker, (Predicate) action -> true); - pipeline = new InboundPipeline(statsTracker, millisSupplier, decoder, aggregator, (c, m) -> { - try (BytesStreamOutput streamOutput = new BytesStreamOutput()) { - NativeInboundMessage m1 = (NativeInboundMessage) m; - Streams.copy(m1.openOrGetStreamInput(), streamOutput); - message.set(new Tuple<>(m1.getHeader(), streamOutput.bytes())); - } catch (IOException e) { - throw new AssertionError(e); - } - }); + handler = new OutboundHandler(statsTracker, threadPool); } @After @@ -137,182 +91,4 @@ public void testSendRawBytes() { assertEquals(bytesArray, reference); } - public void testSendRequest() throws IOException { - ThreadContext threadContext = threadPool.getThreadContext(); - Version version = randomFrom(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()); - String action = "handshake"; - long requestId = randomLongBetween(0, 300); - boolean isHandshake = randomBoolean(); - boolean compress = randomBoolean(); - String value = "message"; - threadContext.putHeader("header", "header_value"); - TestRequest request = new TestRequest(value); - - AtomicReference nodeRef = new AtomicReference<>(); - AtomicLong requestIdRef = new AtomicLong(); - AtomicReference actionRef = new AtomicReference<>(); - AtomicReference requestRef = new AtomicReference<>(); - handler.setMessageListener(new TransportMessageListener() { - @Override - public void onRequestSent( - DiscoveryNode node, - long requestId, - String action, - TransportRequest request, - TransportRequestOptions options - ) { - nodeRef.set(node); - requestIdRef.set(requestId); - actionRef.set(action); - requestRef.set(request); - } - }); - handler.sendRequest(node, channel, requestId, action, request, options, version, compress, isHandshake); - - BytesReference reference = channel.getMessageCaptor().get(); - ActionListener sendListener = channel.getListenerCaptor().get(); - if (randomBoolean()) { - sendListener.onResponse(null); - } else { - sendListener.onFailure(new IOException("failed")); - } - assertEquals(node, nodeRef.get()); - assertEquals(requestId, requestIdRef.get()); - assertEquals(action, actionRef.get()); - assertEquals(request, requestRef.get()); - - pipeline.handleBytes(channel, new ReleasableBytesReference(reference, () -> {})); - final Tuple tuple = message.get(); - final Header header = tuple.v1(); - final TestRequest message = new TestRequest(tuple.v2().streamInput()); - assertEquals(version, header.getVersion()); - assertEquals(requestId, header.getRequestId()); - assertTrue(header.isRequest()); - assertFalse(header.isResponse()); - if (isHandshake) { - assertTrue(header.isHandshake()); - } else { - assertFalse(header.isHandshake()); - } - if (compress) { - assertTrue(header.isCompressed()); - } else { - assertFalse(header.isCompressed()); - } - - assertEquals(value, message.value); - assertEquals("header_value", header.getHeaders().v1().get("header")); - } - - public void testSendResponse() throws IOException { - ThreadContext threadContext = threadPool.getThreadContext(); - Version version = randomFrom(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()); - String action = "handshake"; - long requestId = randomLongBetween(0, 300); - boolean isHandshake = randomBoolean(); - boolean compress = randomBoolean(); - String value = "message"; - threadContext.putHeader("header", "header_value"); - TestResponse response = new TestResponse(value); - - AtomicLong requestIdRef = new AtomicLong(); - AtomicReference actionRef = new AtomicReference<>(); - AtomicReference responseRef = new AtomicReference<>(); - handler.setMessageListener(new TransportMessageListener() { - @Override - public void onResponseSent(long requestId, String action, TransportResponse response) { - requestIdRef.set(requestId); - actionRef.set(action); - responseRef.set(response); - } - }); - handler.sendResponse(version, Collections.emptySet(), channel, requestId, action, response, compress, isHandshake); - - BytesReference reference = channel.getMessageCaptor().get(); - ActionListener sendListener = channel.getListenerCaptor().get(); - if (randomBoolean()) { - sendListener.onResponse(null); - } else { - sendListener.onFailure(new IOException("failed")); - } - assertEquals(requestId, requestIdRef.get()); - assertEquals(action, actionRef.get()); - assertEquals(response, responseRef.get()); - - pipeline.handleBytes(channel, new ReleasableBytesReference(reference, () -> {})); - final Tuple tuple = message.get(); - final Header header = tuple.v1(); - final TestResponse message = new TestResponse(tuple.v2().streamInput()); - assertEquals(version, header.getVersion()); - assertEquals(requestId, header.getRequestId()); - assertFalse(header.isRequest()); - assertTrue(header.isResponse()); - if (isHandshake) { - assertTrue(header.isHandshake()); - } else { - assertFalse(header.isHandshake()); - } - if (compress) { - assertTrue(header.isCompressed()); - } else { - assertFalse(header.isCompressed()); - } - - assertFalse(header.isError()); - - assertEquals(value, message.value); - assertEquals("header_value", header.getHeaders().v1().get("header")); - } - - public void testErrorResponse() throws IOException { - ThreadContext threadContext = threadPool.getThreadContext(); - Version version = randomFrom(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()); - String action = "handshake"; - long requestId = randomLongBetween(0, 300); - threadContext.putHeader("header", "header_value"); - OpenSearchException error = new OpenSearchException("boom"); - - AtomicLong requestIdRef = new AtomicLong(); - AtomicReference actionRef = new AtomicReference<>(); - AtomicReference responseRef = new AtomicReference<>(); - handler.setMessageListener(new TransportMessageListener() { - @Override - public void onResponseSent(long requestId, String action, Exception error) { - requestIdRef.set(requestId); - actionRef.set(action); - responseRef.set(error); - } - }); - handler.sendErrorResponse(version, Collections.emptySet(), channel, requestId, action, error); - - BytesReference reference = channel.getMessageCaptor().get(); - ActionListener sendListener = channel.getListenerCaptor().get(); - if (randomBoolean()) { - sendListener.onResponse(null); - } else { - sendListener.onFailure(new IOException("failed")); - } - assertEquals(requestId, requestIdRef.get()); - assertEquals(action, actionRef.get()); - assertEquals(error, responseRef.get()); - - pipeline.handleBytes(channel, new ReleasableBytesReference(reference, () -> {})); - final Tuple tuple = message.get(); - final Header header = tuple.v1(); - assertEquals(version, header.getVersion()); - assertEquals(requestId, header.getRequestId()); - assertFalse(header.isRequest()); - assertTrue(header.isResponse()); - assertFalse(header.isCompressed()); - assertFalse(header.isHandshake()); - assertTrue(header.isError()); - - RemoteTransportException remoteException = tuple.v2().streamInput().readException(); - assertThat(remoteException.getCause(), instanceOf(OpenSearchException.class)); - assertEquals(remoteException.getCause().getMessage(), "boom"); - assertEquals(action, remoteException.action()); - assertEquals(channel.getLocalAddress(), remoteException.address().address()); - - assertEquals("header_value", header.getHeaders().v1().get("header")); - } } diff --git a/server/src/test/java/org/opensearch/transport/TcpTransportTests.java b/server/src/test/java/org/opensearch/transport/TcpTransportTests.java index 7ab78cca7d615..7c5c9ec12360d 100644 --- a/server/src/test/java/org/opensearch/transport/TcpTransportTests.java +++ b/server/src/test/java/org/opensearch/transport/TcpTransportTests.java @@ -43,7 +43,6 @@ import org.opensearch.common.network.NetworkService; import org.opensearch.common.network.NetworkUtils; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.MockPageCacheRecycler; import org.opensearch.core.common.transport.TransportAddress; import org.opensearch.core.indices.breaker.NoneCircuitBreakerService; @@ -540,19 +539,7 @@ private void testExceptionHandling( final PlainActionFuture listener = new PlainActionFuture<>(); channel.addCloseListener(listener); - TcpTransport.handleException( - channel, - exception, - lifecycle, - new OutboundHandler( - randomAlphaOfLength(10), - Version.CURRENT, - new String[0], - new StatsTracker(), - testThreadPool, - BigArrays.NON_RECYCLING_INSTANCE - ) - ); + TcpTransport.handleException(channel, exception, lifecycle, new OutboundHandler(new StatsTracker(), testThreadPool)); if (expectClosed) { assertTrue(listener.isDone()); diff --git a/server/src/test/java/org/opensearch/transport/TransportLoggerTests.java b/server/src/test/java/org/opensearch/transport/TransportLoggerTests.java index 05296e9308657..cbd1b959f5f16 100644 --- a/server/src/test/java/org/opensearch/transport/TransportLoggerTests.java +++ b/server/src/test/java/org/opensearch/transport/TransportLoggerTests.java @@ -33,12 +33,6 @@ import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; -import org.opensearch.Version; -import org.opensearch.action.admin.cluster.stats.ClusterStatsAction; -import org.opensearch.action.admin.cluster.stats.ClusterStatsRequest; -import org.opensearch.common.io.stream.BytesStreamOutput; -import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.test.MockLogAppender; import org.opensearch.test.OpenSearchTestCase; @@ -49,7 +43,7 @@ import static org.mockito.Mockito.mock; @TestLogging(value = "org.opensearch.transport.TransportLogger:trace", reason = "to ensure we log network events on TRACE level") -public class TransportLoggerTests extends OpenSearchTestCase { +public abstract class TransportLoggerTests extends OpenSearchTestCase { public void testLoggingHandler() throws Exception { try (MockLogAppender appender = MockLogAppender.createForLoggers(LogManager.getLogger(TransportLogger.class))) { final String writePattern = ".*\\[length: \\d+" @@ -90,20 +84,5 @@ public void testLoggingHandler() throws Exception { } } - private BytesReference buildRequest() throws IOException { - boolean compress = randomBoolean(); - try (BytesStreamOutput bytesStreamOutput = new BytesStreamOutput()) { - OutboundMessage.Request request = new OutboundMessage.Request( - new ThreadContext(Settings.EMPTY), - new String[0], - new ClusterStatsRequest(), - Version.CURRENT, - ClusterStatsAction.NAME, - randomInt(30), - false, - compress - ); - return request.serialize(bytesStreamOutput); - } - } + public abstract BytesReference buildRequest() throws IOException; } diff --git a/server/src/test/java/org/opensearch/transport/CompressibleBytesOutputStreamTests.java b/server/src/test/java/org/opensearch/transport/nativeprotocol/CompressibleBytesOutputStreamTests.java similarity index 99% rename from server/src/test/java/org/opensearch/transport/CompressibleBytesOutputStreamTests.java rename to server/src/test/java/org/opensearch/transport/nativeprotocol/CompressibleBytesOutputStreamTests.java index 89018b7353e7c..eaa35469b9ec0 100644 --- a/server/src/test/java/org/opensearch/transport/CompressibleBytesOutputStreamTests.java +++ b/server/src/test/java/org/opensearch/transport/nativeprotocol/CompressibleBytesOutputStreamTests.java @@ -30,7 +30,7 @@ * GitHub history for details. */ -package org.opensearch.transport; +package org.opensearch.transport.nativeprotocol; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.core.common.bytes.BytesReference; diff --git a/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundDecoderTests.java b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundDecoderTests.java new file mode 100644 index 0000000000000..bd85939c753fa --- /dev/null +++ b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundDecoderTests.java @@ -0,0 +1,59 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.transport.nativeprotocol; + +import org.opensearch.Version; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.transport.InboundDecoderTests; + +import java.io.IOException; +import java.util.Collections; + +public class NativeInboundDecoderTests extends InboundDecoderTests { + + @Override + protected BytesReference serialize( + boolean isRequest, + Version version, + boolean handshake, + boolean compress, + String action, + long requestId, + Writeable transportMessage + ) throws IOException { + NativeOutboundMessage message; + if (isRequest) { + message = new NativeOutboundMessage.Request( + threadContext, + new String[0], + transportMessage, + version, + action, + requestId, + handshake, + compress + ); + } else { + message = new NativeOutboundMessage.Response( + threadContext, + Collections.emptySet(), + transportMessage, + version, + requestId, + handshake, + compress + ); + } + + return message.serialize(new BytesStreamOutput()); + } + +} diff --git a/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundHandlerTests.java b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundHandlerTests.java new file mode 100644 index 0000000000000..ec0c1a50d5560 --- /dev/null +++ b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundHandlerTests.java @@ -0,0 +1,45 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.transport.nativeprotocol; + +import org.opensearch.Version; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.transport.InboundHandlerTests; + +import java.io.IOException; + +public class NativeInboundHandlerTests extends InboundHandlerTests { + + @Override + public BytesReference serializeOutboundRequest( + ThreadContext threadContext, + Writeable message, + Version version, + String action, + long requestId, + boolean compress, + boolean handshake + ) throws IOException { + NativeOutboundMessage.Request request = new NativeOutboundMessage.Request( + threadContext, + new String[0], + message, + version, + action, + requestId, + handshake, + compress + ); + return request.serialize(new BytesStreamOutput()); + } + +} diff --git a/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundPipelineTests.java b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundPipelineTests.java new file mode 100644 index 0000000000000..6b5bf46eee3ae --- /dev/null +++ b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeInboundPipelineTests.java @@ -0,0 +1,60 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.transport.nativeprotocol; + +import org.opensearch.Version; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.transport.InboundPipelineTests; +import org.opensearch.transport.TestRequest; +import org.opensearch.transport.TestResponse; + +import java.io.IOException; +import java.util.Collections; + +public class NativeInboundPipelineTests extends InboundPipelineTests { + + @Override + protected BytesReference serialize( + boolean isRequest, + Version version, + boolean handshake, + boolean compress, + String action, + long requestId, + String value + ) throws IOException { + NativeOutboundMessage message; + if (isRequest) { + message = new NativeOutboundMessage.Request( + threadContext, + new String[0], + new TestRequest(value), + version, + action, + requestId, + handshake, + compress + ); + } else { + message = new NativeOutboundMessage.Response( + threadContext, + Collections.emptySet(), + new TestResponse(value), + version, + requestId, + handshake, + compress + ); + } + + return message.serialize(new BytesStreamOutput()); + } + +} diff --git a/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeOutboundMessageTests.java b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeOutboundMessageTests.java new file mode 100644 index 0000000000000..75c4e84b4456e --- /dev/null +++ b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeOutboundMessageTests.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.transport.nativeprotocol; + +import org.opensearch.Version; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.transport.TestRequest; + +import java.io.IOException; + +public class NativeOutboundMessageTests extends OpenSearchTestCase { + + public void testNativeOutboundMessageRequestSerialization() throws IOException { + NativeOutboundMessage.Request message = new NativeOutboundMessage.Request( + new ThreadContext(Settings.EMPTY), + new String[0], + new TestRequest("content"), + Version.CURRENT, + "action", + 1, + false, + false + ); + BytesStreamOutput output = new BytesStreamOutput(); + message.serialize(output); + + BytesStreamInput input = new BytesStreamInput(output.bytes().toBytesRef().bytes); + assertEquals(Version.CURRENT, input.getVersion()); + // reading header details + assertEquals((byte) 'E', input.readByte()); + assertEquals((byte) 'S', input.readByte()); + assertNotEquals(0, input.readInt()); + assertEquals(1, input.readLong()); + assertEquals(0, input.readByte()); + assertEquals(Version.CURRENT.id, input.readInt()); + int variableHeaderSize = input.readInt(); + assertNotEquals(-1, variableHeaderSize); + } + +} diff --git a/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeTransportLoggerTests.java b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeTransportLoggerTests.java new file mode 100644 index 0000000000000..db75d6ff45556 --- /dev/null +++ b/server/src/test/java/org/opensearch/transport/nativeprotocol/NativeTransportLoggerTests.java @@ -0,0 +1,42 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.transport.nativeprotocol; + +import org.opensearch.Version; +import org.opensearch.action.admin.cluster.stats.ClusterStatsAction; +import org.opensearch.action.admin.cluster.stats.ClusterStatsRequest; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.test.junit.annotations.TestLogging; +import org.opensearch.transport.TransportLoggerTests; + +import java.io.IOException; + +@TestLogging(value = "org.opensearch.transport.TransportLogger:trace", reason = "to ensure we log network events on TRACE level") +public class NativeTransportLoggerTests extends TransportLoggerTests { + + public BytesReference buildRequest() throws IOException { + boolean compress = randomBoolean(); + try (BytesStreamOutput bytesStreamOutput = new BytesStreamOutput()) { + NativeOutboundMessage.Request request = new NativeOutboundMessage.Request( + new ThreadContext(Settings.EMPTY), + new String[0], + new ClusterStatsRequest(), + Version.CURRENT, + ClusterStatsAction.NAME, + randomInt(30), + false, + compress + ); + return request.serialize(new BytesStreamOutput()); + } + } +} diff --git a/test/framework/src/main/java/org/opensearch/transport/TestRequest.java b/test/framework/src/main/java/org/opensearch/transport/TestRequest.java index 2fe917235e948..0bb5a6e16fff1 100644 --- a/test/framework/src/main/java/org/opensearch/transport/TestRequest.java +++ b/test/framework/src/main/java/org/opensearch/transport/TestRequest.java @@ -54,4 +54,8 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(value); } + + public String getValue() { + return value; + } } diff --git a/test/framework/src/main/java/org/opensearch/transport/TestResponse.java b/test/framework/src/main/java/org/opensearch/transport/TestResponse.java index 14db8b3372bf2..7fd7c760c9cf6 100644 --- a/test/framework/src/main/java/org/opensearch/transport/TestResponse.java +++ b/test/framework/src/main/java/org/opensearch/transport/TestResponse.java @@ -54,4 +54,8 @@ public TestResponse(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeString(value); } + + public String getValue() { + return value; + } } From f8baa4ae79aa167004302eb09d39ad1c71cb64c8 Mon Sep 17 00:00:00 2001 From: Bhumika Saini Date: Tue, 14 May 2024 15:10:20 +0530 Subject: [PATCH 6/7] Allow clearing remote_store.compatibility_mode setting (#13646) Signed-off-by: Bhumika Saini --- CHANGELOG.md | 1 + .../RemoteStoreMigrationSettingsUpdateIT.java | 47 +++++++++++++------ ...eMigrationShardAllocationBaseTestCase.java | 5 ++ .../TransportClusterUpdateSettingsAction.java | 5 +- 4 files changed, 41 insertions(+), 17 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b858ede4b78cf..3471cf545d4e2 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/), ### 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)) ### Security diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationSettingsUpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationSettingsUpdateIT.java index b71f7d7cf7e4a..377bd9529ca7a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationSettingsUpdateIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationSettingsUpdateIT.java @@ -14,6 +14,7 @@ import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; +import java.io.IOException; import java.nio.file.Path; import java.util.Optional; @@ -26,6 +27,8 @@ public class RemoteStoreMigrationSettingsUpdateIT extends RemoteStoreMigrationShardAllocationBaseTestCase { private Client client; + private String nonRemoteNodeName; + private String remoteNodeName; // remote store backed index setting tests @@ -120,18 +123,7 @@ public void testNewRestoredIndexIsRemoteStoreBackedForRemoteStoreDirectionAndMix // compatibility mode setting test public void testSwitchToStrictMode() throws Exception { - logger.info("Initialize cluster"); - initializeCluster(false); - - logger.info("Create a mixed mode cluster"); - setClusterMode(MIXED.mode); - addRemote = true; - String remoteNodeName = internalCluster().startNode(); - addRemote = false; - String nonRemoteNodeName = internalCluster().startNode(); - internalCluster().validateClusterFormed(); - assertNodeInCluster(remoteNodeName); - assertNodeInCluster(nonRemoteNodeName); + createMixedModeCluster(); logger.info("Attempt switching to strict mode"); SettingsException exception = assertThrows(SettingsException.class, () -> setClusterMode(STRICT.mode)); @@ -140,12 +132,39 @@ public void testSwitchToStrictMode() throws Exception { exception.getMessage() ); + stopRemoteNode(); + + logger.info("Attempt switching to strict mode"); + setClusterMode(STRICT.mode); + } + + public void testClearCompatibilityModeSetting() throws Exception { + createMixedModeCluster(); + stopRemoteNode(); + + logger.info("Attempt clearing compatibility mode"); + clearClusterMode(); + } + + private void stopRemoteNode() throws IOException { logger.info("Stop remote node so that cluster had only non-remote nodes"); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(remoteNodeName)); ensureStableCluster(2); + } - logger.info("Attempt switching to strict mode"); - setClusterMode(STRICT.mode); + private void createMixedModeCluster() { + logger.info("Initialize cluster"); + initializeCluster(false); + + logger.info("Create a mixed mode cluster"); + setClusterMode(MIXED.mode); + addRemote = true; + remoteNodeName = internalCluster().startNode(); + addRemote = false; + nonRemoteNodeName = internalCluster().startNode(); + internalCluster().validateClusterFormed(); + assertNodeInCluster(remoteNodeName); + assertNodeInCluster(nonRemoteNodeName); } // bootstrap a cluster diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationShardAllocationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationShardAllocationBaseTestCase.java index cf689aa554c8b..3591846a93bc8 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationShardAllocationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationShardAllocationBaseTestCase.java @@ -55,6 +55,11 @@ protected void setClusterMode(String mode) { assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); } + protected void clearClusterMode() { + updateSettingsRequest.persistentSettings(Settings.builder().putNull(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey())); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + } + // set the migration direction for cluster [remote_store, docrep, none] protected void setDirection(String direction) { updateSettingsRequest.persistentSettings(Settings.builder().put(MIGRATION_DIRECTION_SETTING.getKey(), direction)); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index 6292d32fee26d..216e1fb2ed1cc 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -66,7 +66,6 @@ import java.io.IOException; import java.util.Collection; -import java.util.Locale; import java.util.Set; import java.util.stream.Collectors; @@ -285,9 +284,9 @@ public ClusterState execute(final ClusterState currentState) { public void validateCompatibilityModeSettingRequest(ClusterUpdateSettingsRequest request, ClusterState clusterState) { Settings settings = Settings.builder().put(request.persistentSettings()).put(request.transientSettings()).build(); if (RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.exists(settings)) { - String value = settings.get(RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey()).toLowerCase(Locale.ROOT); + String value = RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get(settings).mode; validateAllNodesOfSameVersion(clusterState.nodes()); - if (value.equals(RemoteStoreNodeService.CompatibilityMode.STRICT.mode)) { + if (RemoteStoreNodeService.CompatibilityMode.STRICT.mode.equals(value)) { validateAllNodesOfSameType(clusterState.nodes()); validateIndexSettings(clusterState); } From a03db0dcd304a600463f665a2bd9e43364985153 Mon Sep 17 00:00:00 2001 From: chengwushi-netapp <153049940+chengwushi-netapp@users.noreply.github.com> Date: Wed, 15 May 2024 00:21:47 +1000 Subject: [PATCH 7/7] Main repository azure managed identity support (#12559) * Added support for Azure Managed Identity in repository-azure Signed-off-by: Chengwu Shi * Refactor tokenCredentialType as an enum when constructing AzureStorageSetting Signed-off-by: Chengwu Shi * fixed indentation Signed-off-by: Chengwu Shi * fixed syntax Signed-off-by: Chengwu Shi * removed unused imports Signed-off-by: Chengwu Shi * applied changes after running :plugins:repository-azure:spotlessApply Signed-off-by: Chengwu Shi * added transitive dependencies Signed-off-by: Chengwu Shi * changed getStorageBlobEndpoint to private and using asm version from buildSrc/version.properties Signed-off-by: Chengwu Shi * run spotlessApply Signed-off-by: Chengwu Shi * update shas for asm 9.7 Signed-off-by: Chengwu Shi * use version.jna for jna-platform Signed-off-by: Chengwu Shi * change string 'core.windows.net' to be a constant, use uri.create instead of new uri, and added a few comments for clarity Signed-off-by: Chengwu Shi * added one more comment line Signed-off-by: Chengwu Shi * refactor TokenCredentialType to not have NOT_APPLICABLE Signed-off-by: Chengwu Shi * refactored code based on recommended changes from Andriy Redko Signed-off-by: Chengwu Shi * added a jvm security policy for reactor-core jar Signed-off-by: Chengwu Shi * fixed failing forbidden api fix Signed-off-by: Chengwu Shi * removed the jvm security policy for reactor-core which was added Signed-off-by: Chengwu Shi * refactored code, such that storage endpoint is not evaluated at compiled time Signed-off-by: Chengwu Shi * refactored token credential types checks Signed-off-by: Chengwu Shi * changed clientlogger in azurestoragesetting to 'AzureStorageService' Signed-off-by: Chengwu Shi * added a nullable argument to getStorageEndpoint Signed-off-by: Chengwu Shi * Fix IdentityClient security permissions, get rid of connection string (since it is not applicable to managed identity configuration) Signed-off-by: Andriy Redko * Responded to feedback from Andrew Ross, fixed typo, spelling, added shudownNow() and Thread.currentThread().interrupt() Signed-off-by: Chengwu Shi --------- Signed-off-by: Chengwu Shi Signed-off-by: Andriy Redko Co-authored-by: Andriy Redko --- CHANGELOG.md | 1 + .../gradle/precommit/LicenseAnalyzer.java | 2 +- plugins/repository-azure/build.gradle | 86 ++- .../licenses/accessors-smart-2.5.0.jar.sha1 | 1 + .../licenses/accessors-smart-LICENSE.txt | 202 ++++++ .../licenses/accessors-smart-NOTICE.txt | 0 .../licenses/asm-9.7.jar.sha1 | 1 + .../repository-azure/licenses/asm-LICENSE.txt | 27 + .../repository-azure/licenses/asm-NOTICE.txt | 0 .../licenses/azure-identity-1.11.4.jar.sha1 | 1 + .../licenses/content-type-2.3.jar.sha1 | 1 + .../licenses/content-type-LICENSE.txt | 202 ++++++ .../licenses/content-type-NOTICE.txt | 0 .../licenses/jna-platform-5.13.0.jar.sha1 | 1 + .../licenses/jna-platform-LICENSE.txt | 26 + .../licenses/jna-platform-NOTICE.txt | 0 .../licenses/json-smart-2.5.0.jar.sha1 | 1 + .../licenses/json-smart-LICENSE.txt | 202 ++++++ .../licenses/json-smart-NOTICE.txt | 0 .../licenses/lang-tag-1.7.jar.sha1 | 1 + .../licenses/lang-tag-LICENSE.txt | 202 ++++++ .../licenses/lang-tag-NOTICE.txt | 0 .../licenses/msal4j-1.14.3.jar.sha1 | 1 + .../licenses/msal4j-LICENSE.txt | 21 + .../licenses/msal4j-NOTICE.txt | 0 ...sal4j-persistence-extension-1.2.0.jar.sha1 | 1 + .../msal4j-persistence-extension-LICENSE.txt | 21 + .../msal4j-persistence-extension-NOTICE.txt | 0 .../licenses/nimbus-jose-jwt-9.37.3.jar.sha1 | 1 + .../licenses/nimbus-jose-jwt-LICENSE.txt | 202 ++++++ .../licenses/nimbus-jose-jwt-NOTICE.txt | 0 .../licenses/oauth2-oidc-sdk-11.9.1.jar.sha1 | 1 + .../licenses/oauth2-oidc-sdk-LICENSE.txt | 202 ++++++ .../licenses/oauth2-oidc-sdk-NOTICE.txt | 0 .../repositories/azure/AzureBlobStore.java | 2 +- .../azure/AzureRepositoryPlugin.java | 1 + .../azure/AzureStorageService.java | 61 +- .../azure/AzureStorageSettings.java | 91 ++- .../azure/TokenCredentialType.java | 40 ++ .../plugin-metadata/plugin-security.policy | 3 + .../azure/AzureStorageServiceTests.java | 573 +++++++++++++----- .../org/opensearch/bootstrap/security.policy | 5 + 42 files changed, 2028 insertions(+), 155 deletions(-) create mode 100644 plugins/repository-azure/licenses/accessors-smart-2.5.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/accessors-smart-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/accessors-smart-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/asm-9.7.jar.sha1 create mode 100644 plugins/repository-azure/licenses/asm-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/asm-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/azure-identity-1.11.4.jar.sha1 create mode 100644 plugins/repository-azure/licenses/content-type-2.3.jar.sha1 create mode 100644 plugins/repository-azure/licenses/content-type-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/content-type-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/jna-platform-5.13.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/jna-platform-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/jna-platform-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/json-smart-2.5.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/json-smart-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/json-smart-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/lang-tag-1.7.jar.sha1 create mode 100644 plugins/repository-azure/licenses/lang-tag-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/lang-tag-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/msal4j-1.14.3.jar.sha1 create mode 100644 plugins/repository-azure/licenses/msal4j-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/msal4j-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/msal4j-persistence-extension-1.2.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/msal4j-persistence-extension-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/msal4j-persistence-extension-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/nimbus-jose-jwt-9.37.3.jar.sha1 create mode 100644 plugins/repository-azure/licenses/nimbus-jose-jwt-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/nimbus-jose-jwt-NOTICE.txt create mode 100644 plugins/repository-azure/licenses/oauth2-oidc-sdk-11.9.1.jar.sha1 create mode 100644 plugins/repository-azure/licenses/oauth2-oidc-sdk-LICENSE.txt create mode 100644 plugins/repository-azure/licenses/oauth2-oidc-sdk-NOTICE.txt create mode 100644 plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/TokenCredentialType.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 3471cf545d4e2..65c5fa1223cc6 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 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)) diff --git a/buildSrc/src/main/java/org/opensearch/gradle/precommit/LicenseAnalyzer.java b/buildSrc/src/main/java/org/opensearch/gradle/precommit/LicenseAnalyzer.java index 4c63516126566..c3acd12e5a1cf 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/precommit/LicenseAnalyzer.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/precommit/LicenseAnalyzer.java @@ -145,7 +145,7 @@ public class LicenseAnalyzer { + "AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER\n" + "LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,\n" + "OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE\n" - + "SOFTWARE\\.\n").replaceAll("\\s+", "\\\\s*"), + + "SOFTWARE\\.?\n").replaceAll("\\s+", "\\\\s*"), Pattern.DOTALL ) ), diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index c7836170d658f..ff62c328c7e74 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -56,6 +56,21 @@ dependencies { api "io.netty:netty-transport-native-unix-common:${versions.netty}" implementation project(':modules:transport-netty4') api 'com.azure:azure-storage-blob:12.23.0' + api 'com.azure:azure-identity:1.11.4' + // Start of transitive dependencies for azure-identity + api 'com.microsoft.azure:msal4j-persistence-extension:1.2.0' + api "net.java.dev.jna:jna-platform:${versions.jna}" + api 'com.microsoft.azure:msal4j:1.14.3' + api 'com.nimbusds:oauth2-oidc-sdk:11.9.1' + api 'com.nimbusds:nimbus-jose-jwt:9.37.3' + api 'com.nimbusds:content-type:2.3' + api 'com.nimbusds:lang-tag:1.7' + // Both msal4j:1.14.3 and oauth2-oidc-sdk:11.9.1 has compile dependency on different versions of json-smart, + // selected the higher version which is 2.5.0 + api 'net.minidev:json-smart:2.5.0' + api 'net.minidev:accessors-smart:2.5.0' + api "org.ow2.asm:asm:${versions.asm}" + // End of transitive dependencies for azure-identity api "io.projectreactor.netty:reactor-netty-core:${versions.reactor_netty}" api "io.projectreactor.netty:reactor-netty-http:${versions.reactor_netty}" api "org.slf4j:slf4j-api:${versions.slf4j}" @@ -180,7 +195,76 @@ thirdPartyAudit { 'io.micrometer.observation.ObservationHandler', 'io.micrometer.observation.ObservationRegistry', 'io.micrometer.observation.ObservationRegistry$ObservationConfig', - 'io.micrometer.tracing.handler.DefaultTracingObservationHandler' + 'io.micrometer.tracing.handler.DefaultTracingObservationHandler', + // Start of the list of classes from the optional compile/provided dependencies used in "com.nimbusds:oauth2-oidc-sdk". + 'com.google.crypto.tink.subtle.Ed25519Sign', + 'com.google.crypto.tink.subtle.Ed25519Sign$KeyPair', + 'com.google.crypto.tink.subtle.Ed25519Verify', + 'com.google.crypto.tink.subtle.X25519', + 'com.google.crypto.tink.subtle.XChaCha20Poly1305', + 'jakarta.servlet.ServletRequest', + 'jakarta.servlet.http.HttpServletRequest', + 'jakarta.servlet.http.HttpServletResponse', + 'javax.servlet.ServletRequest', + 'javax.servlet.http.HttpServletRequest', + 'javax.servlet.http.HttpServletResponse', + // net.shibboleth.utilities:java-support.* is declared as optional in the plugin `bnd-maven-plugin` used in "com.nimbusds:oauth2-oidc-sdk" + // Worth nothing that, the latest dependency "net.shibboleth.utilities:java-support:8.0.0" has many vulnerabilities. + // Hence ignored. + 'net.shibboleth.utilities.java.support.xml.SerializeSupport', + 'org.bouncycastle.asn1.pkcs.PrivateKeyInfo', + 'org.bouncycastle.asn1.x509.AlgorithmIdentifier', + 'org.bouncycastle.asn1.x509.SubjectPublicKeyInfo', + 'org.bouncycastle.cert.X509CertificateHolder', + 'org.bouncycastle.cert.jcajce.JcaX509CertificateHolder', + 'org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder', + 'org.bouncycastle.crypto.InvalidCipherTextException', + 'org.bouncycastle.crypto.engines.AESEngine', + 'org.bouncycastle.crypto.modes.GCMBlockCipher', + 'org.bouncycastle.jcajce.provider.BouncyCastleFipsProvider', + 'org.bouncycastle.jce.provider.BouncyCastleProvider', + 'org.bouncycastle.openssl.PEMKeyPair', + 'org.bouncycastle.openssl.PEMParser', + 'org.bouncycastle.openssl.jcajce.JcaPEMKeyConverter', + 'org.bouncycastle.operator.jcajce.JcaContentSignerBuilder', + 'org.cryptomator.siv.SivMode', + 'org.opensaml.core.config.InitializationException', + 'org.opensaml.core.config.InitializationService', + 'org.opensaml.core.xml.XMLObject', + 'org.opensaml.core.xml.XMLObjectBuilder', + 'org.opensaml.core.xml.XMLObjectBuilderFactory', + 'org.opensaml.core.xml.config.XMLObjectProviderRegistrySupport', + 'org.opensaml.core.xml.io.Marshaller', + 'org.opensaml.core.xml.io.MarshallerFactory', + 'org.opensaml.core.xml.io.MarshallingException', + 'org.opensaml.core.xml.io.Unmarshaller', + 'org.opensaml.core.xml.io.UnmarshallerFactory', + 'org.opensaml.core.xml.schema.XSString', + 'org.opensaml.core.xml.schema.impl.XSStringBuilder', + 'org.opensaml.saml.saml2.core.Assertion', + 'org.opensaml.saml.saml2.core.Attribute', + 'org.opensaml.saml.saml2.core.AttributeStatement', + 'org.opensaml.saml.saml2.core.AttributeValue', + 'org.opensaml.saml.saml2.core.Audience', + 'org.opensaml.saml.saml2.core.AudienceRestriction', + 'org.opensaml.saml.saml2.core.AuthnContext', + 'org.opensaml.saml.saml2.core.AuthnContextClassRef', + 'org.opensaml.saml.saml2.core.AuthnStatement', + 'org.opensaml.saml.saml2.core.Conditions', + 'org.opensaml.saml.saml2.core.Issuer', + 'org.opensaml.saml.saml2.core.NameID', + 'org.opensaml.saml.saml2.core.Subject', + 'org.opensaml.saml.saml2.core.SubjectConfirmation', + 'org.opensaml.saml.saml2.core.SubjectConfirmationData', + 'org.opensaml.saml.security.impl.SAMLSignatureProfileValidator', + 'org.opensaml.security.credential.BasicCredential', + 'org.opensaml.security.credential.Credential', + 'org.opensaml.security.credential.UsageType', + 'org.opensaml.xmlsec.signature.Signature', + 'org.opensaml.xmlsec.signature.support.SignatureException', + 'org.opensaml.xmlsec.signature.support.SignatureValidator', + 'org.opensaml.xmlsec.signature.support.Signer', + // End of the list of classes from the optional compile/provided dependencies used in "com.nimbusds:oauth2-oidc-sdk". ) ignoreViolations( diff --git a/plugins/repository-azure/licenses/accessors-smart-2.5.0.jar.sha1 b/plugins/repository-azure/licenses/accessors-smart-2.5.0.jar.sha1 new file mode 100644 index 0000000000000..1578c94fcdc7b --- /dev/null +++ b/plugins/repository-azure/licenses/accessors-smart-2.5.0.jar.sha1 @@ -0,0 +1 @@ +aca011492dfe9c26f4e0659028a4fe0970829dd8 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/accessors-smart-LICENSE.txt b/plugins/repository-azure/licenses/accessors-smart-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/repository-azure/licenses/accessors-smart-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/plugins/repository-azure/licenses/accessors-smart-NOTICE.txt b/plugins/repository-azure/licenses/accessors-smart-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/asm-9.7.jar.sha1 b/plugins/repository-azure/licenses/asm-9.7.jar.sha1 new file mode 100644 index 0000000000000..84c9a9703af6d --- /dev/null +++ b/plugins/repository-azure/licenses/asm-9.7.jar.sha1 @@ -0,0 +1 @@ +073d7b3086e14beb604ced229c302feff6449723 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/asm-LICENSE.txt b/plugins/repository-azure/licenses/asm-LICENSE.txt new file mode 100644 index 0000000000000..c71bb7bac5d4d --- /dev/null +++ b/plugins/repository-azure/licenses/asm-LICENSE.txt @@ -0,0 +1,27 @@ +ASM: a very small and fast Java bytecode manipulation framework +Copyright (c) 2000-2011 INRIA, France Telecom +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. +3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. diff --git a/plugins/repository-azure/licenses/asm-NOTICE.txt b/plugins/repository-azure/licenses/asm-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/azure-identity-1.11.4.jar.sha1 b/plugins/repository-azure/licenses/azure-identity-1.11.4.jar.sha1 new file mode 100644 index 0000000000000..c8d98ba9c8ad2 --- /dev/null +++ b/plugins/repository-azure/licenses/azure-identity-1.11.4.jar.sha1 @@ -0,0 +1 @@ +59b5ce48888f638b80d85ef5aa0e22a265d3dc89 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/content-type-2.3.jar.sha1 b/plugins/repository-azure/licenses/content-type-2.3.jar.sha1 new file mode 100644 index 0000000000000..e18bbaec9a89c --- /dev/null +++ b/plugins/repository-azure/licenses/content-type-2.3.jar.sha1 @@ -0,0 +1 @@ +e3aa0be212d7a42839a8f3f506f5b990bcce0222 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/content-type-LICENSE.txt b/plugins/repository-azure/licenses/content-type-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/repository-azure/licenses/content-type-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/plugins/repository-azure/licenses/content-type-NOTICE.txt b/plugins/repository-azure/licenses/content-type-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/jna-platform-5.13.0.jar.sha1 b/plugins/repository-azure/licenses/jna-platform-5.13.0.jar.sha1 new file mode 100644 index 0000000000000..e2a8ba1c1bbd3 --- /dev/null +++ b/plugins/repository-azure/licenses/jna-platform-5.13.0.jar.sha1 @@ -0,0 +1 @@ +88e9a306715e9379f3122415ef4ae759a352640d \ No newline at end of file diff --git a/plugins/repository-azure/licenses/jna-platform-LICENSE.txt b/plugins/repository-azure/licenses/jna-platform-LICENSE.txt new file mode 100644 index 0000000000000..c5a025f0c3e6d --- /dev/null +++ b/plugins/repository-azure/licenses/jna-platform-LICENSE.txt @@ -0,0 +1,26 @@ +SPDX-License-Identifier: Apache-2.0 OR LGPL-2.1 + +Java Native Access (JNA) is licensed under the LGPL, version 2.1 +or later, or (from version 4.0 onward) the Apache License, +version 2.0. + +You can freely decide which license you want to apply to the project. + +You may obtain a copy of the LGPL License at: + +http://www.gnu.org/licenses/licenses.html + +A copy is also included in the downloadable source code package +containing JNA, in file "LGPL2.1", under the same directory +as this file. + +You may obtain a copy of the Apache License at: + +http://www.apache.org/licenses/ + +A copy is also included in the downloadable source code package +containing JNA, in file "AL2.0", under the same directory +as this file. + +Commercial support may be available, please e-mail +twall[at]users[dot]sf[dot]net. diff --git a/plugins/repository-azure/licenses/jna-platform-NOTICE.txt b/plugins/repository-azure/licenses/jna-platform-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/json-smart-2.5.0.jar.sha1 b/plugins/repository-azure/licenses/json-smart-2.5.0.jar.sha1 new file mode 100644 index 0000000000000..3ec055efa1255 --- /dev/null +++ b/plugins/repository-azure/licenses/json-smart-2.5.0.jar.sha1 @@ -0,0 +1 @@ +57a64f421b472849c40e77d2e7cce3a141b41e99 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/json-smart-LICENSE.txt b/plugins/repository-azure/licenses/json-smart-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/repository-azure/licenses/json-smart-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/plugins/repository-azure/licenses/json-smart-NOTICE.txt b/plugins/repository-azure/licenses/json-smart-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/lang-tag-1.7.jar.sha1 b/plugins/repository-azure/licenses/lang-tag-1.7.jar.sha1 new file mode 100644 index 0000000000000..9cd79d1dba715 --- /dev/null +++ b/plugins/repository-azure/licenses/lang-tag-1.7.jar.sha1 @@ -0,0 +1 @@ +97c73ecd70bc7e8eefb26c5eea84f251a63f1031 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/lang-tag-LICENSE.txt b/plugins/repository-azure/licenses/lang-tag-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/repository-azure/licenses/lang-tag-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/plugins/repository-azure/licenses/lang-tag-NOTICE.txt b/plugins/repository-azure/licenses/lang-tag-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/msal4j-1.14.3.jar.sha1 b/plugins/repository-azure/licenses/msal4j-1.14.3.jar.sha1 new file mode 100644 index 0000000000000..2a6e42e3f2b48 --- /dev/null +++ b/plugins/repository-azure/licenses/msal4j-1.14.3.jar.sha1 @@ -0,0 +1 @@ +117b28c41bd760f979ed1b6467c5ec491f0d4d60 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/msal4j-LICENSE.txt b/plugins/repository-azure/licenses/msal4j-LICENSE.txt new file mode 100644 index 0000000000000..21071075c2459 --- /dev/null +++ b/plugins/repository-azure/licenses/msal4j-LICENSE.txt @@ -0,0 +1,21 @@ + MIT License + + Copyright (c) Microsoft Corporation. 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/repository-azure/licenses/msal4j-NOTICE.txt b/plugins/repository-azure/licenses/msal4j-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/msal4j-persistence-extension-1.2.0.jar.sha1 b/plugins/repository-azure/licenses/msal4j-persistence-extension-1.2.0.jar.sha1 new file mode 100644 index 0000000000000..cfcf7548b7694 --- /dev/null +++ b/plugins/repository-azure/licenses/msal4j-persistence-extension-1.2.0.jar.sha1 @@ -0,0 +1 @@ +1111a95878de8745ddc9de132df18ebd9ca7024d \ No newline at end of file diff --git a/plugins/repository-azure/licenses/msal4j-persistence-extension-LICENSE.txt b/plugins/repository-azure/licenses/msal4j-persistence-extension-LICENSE.txt new file mode 100644 index 0000000000000..21071075c2459 --- /dev/null +++ b/plugins/repository-azure/licenses/msal4j-persistence-extension-LICENSE.txt @@ -0,0 +1,21 @@ + MIT License + + Copyright (c) Microsoft Corporation. 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/repository-azure/licenses/msal4j-persistence-extension-NOTICE.txt b/plugins/repository-azure/licenses/msal4j-persistence-extension-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/nimbus-jose-jwt-9.37.3.jar.sha1 b/plugins/repository-azure/licenses/nimbus-jose-jwt-9.37.3.jar.sha1 new file mode 100644 index 0000000000000..7278cd8994f71 --- /dev/null +++ b/plugins/repository-azure/licenses/nimbus-jose-jwt-9.37.3.jar.sha1 @@ -0,0 +1 @@ +700f71ffefd60c16bd8ce711a956967ea9071cec \ No newline at end of file diff --git a/plugins/repository-azure/licenses/nimbus-jose-jwt-LICENSE.txt b/plugins/repository-azure/licenses/nimbus-jose-jwt-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/repository-azure/licenses/nimbus-jose-jwt-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/plugins/repository-azure/licenses/nimbus-jose-jwt-NOTICE.txt b/plugins/repository-azure/licenses/nimbus-jose-jwt-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/licenses/oauth2-oidc-sdk-11.9.1.jar.sha1 b/plugins/repository-azure/licenses/oauth2-oidc-sdk-11.9.1.jar.sha1 new file mode 100644 index 0000000000000..96d9a196a172a --- /dev/null +++ b/plugins/repository-azure/licenses/oauth2-oidc-sdk-11.9.1.jar.sha1 @@ -0,0 +1 @@ +fa9a2e447e2cef4dfda40a854dd7ec35624a7799 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/oauth2-oidc-sdk-LICENSE.txt b/plugins/repository-azure/licenses/oauth2-oidc-sdk-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/repository-azure/licenses/oauth2-oidc-sdk-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/plugins/repository-azure/licenses/oauth2-oidc-sdk-NOTICE.txt b/plugins/repository-azure/licenses/oauth2-oidc-sdk-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureBlobStore.java b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureBlobStore.java index e76a6bdd16764..acaaa043df3ac 100644 --- a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureBlobStore.java +++ b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureBlobStore.java @@ -175,7 +175,7 @@ public BlobContainer blobContainer(BlobPath path) { } @Override - public void close() { + public void close() throws IOException { service.close(); } diff --git a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureRepositoryPlugin.java b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureRepositoryPlugin.java index 78db7cb2d0ea7..aca213f9fed79 100644 --- a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureRepositoryPlugin.java +++ b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureRepositoryPlugin.java @@ -91,6 +91,7 @@ public List> getSettings() { AzureStorageSettings.ACCOUNT_SETTING, AzureStorageSettings.KEY_SETTING, AzureStorageSettings.SAS_TOKEN_SETTING, + AzureStorageSettings.TOKEN_CREDENTIAL_TYPE_SETTING, AzureStorageSettings.ENDPOINT_SUFFIX_SETTING, AzureStorageSettings.TIMEOUT_SETTING, AzureStorageSettings.MAX_RETRIES_SETTING, diff --git a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureStorageService.java b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureStorageService.java index 74edd4f3eb23c..f39ed185d8b35 100644 --- a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureStorageService.java +++ b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureStorageService.java @@ -47,7 +47,6 @@ import com.azure.storage.blob.BlobServiceClientBuilder; import com.azure.storage.blob.models.ParallelTransferOptions; import com.azure.storage.blob.specialized.BlockBlobAsyncClient; -import com.azure.storage.common.implementation.connectionstring.StorageConnectionString; import com.azure.storage.common.implementation.connectionstring.StorageEndpoint; import com.azure.storage.common.policy.RequestRetryOptions; import com.azure.storage.common.policy.RetryPolicyType; @@ -59,14 +58,19 @@ import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.common.unit.ByteSizeValue; +import java.io.IOException; import java.net.Authenticator; import java.net.PasswordAuthentication; import java.net.URISyntaxException; +import java.security.AccessController; import java.security.InvalidKeyException; +import java.security.PrivilegedAction; import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -99,6 +103,37 @@ public class AzureStorageService implements AutoCloseable { // 'package' for testing volatile Map storageSettings = emptyMap(); private final Map clients = new ConcurrentHashMap<>(); + private final ExecutorService executor; + + private static final class IdentityClientThreadFactory implements ThreadFactory { + final ThreadGroup group; + final AtomicInteger threadNumber = new AtomicInteger(1); + final String namePrefix; + + @SuppressWarnings("removal") + IdentityClientThreadFactory(String namePrefix) { + this.namePrefix = namePrefix; + SecurityManager s = System.getSecurityManager(); + group = (s != null) ? s.getThreadGroup() : Thread.currentThread().getThreadGroup(); + } + + @Override + public Thread newThread(Runnable r) { + Thread t = new Thread(group, new Runnable() { + @SuppressWarnings("removal") + public void run() { + AccessController.doPrivileged(new PrivilegedAction<>() { + public Void run() { + r.run(); + return null; + } + }); + } + }, namePrefix + "[T#" + threadNumber.getAndIncrement() + "]", 0); + t.setDaemon(true); + return t; + } + } static { // See please: @@ -112,6 +147,9 @@ public AzureStorageService(Settings settings) { // eagerly load client settings so that secure settings are read final Map clientsSettings = AzureStorageSettings.load(settings); refreshAndClearCache(clientsSettings); + executor = SocketAccess.doPrivilegedException( + () -> Executors.newCachedThreadPool(new IdentityClientThreadFactory("azure-identity-client")) + ); } /** @@ -162,7 +200,6 @@ public Tuple> client(String clientName, BiC private ClientState buildClient(AzureStorageSettings azureStorageSettings, BiConsumer statsCollector) throws InvalidKeyException, URISyntaxException { final BlobServiceClientBuilder builder = createClientBuilder(azureStorageSettings); - final NioEventLoopGroup eventLoopGroup = new NioEventLoopGroup(new NioThreadFactory()); final NettyAsyncHttpClientBuilder clientBuilder = new NettyAsyncHttpClientBuilder().eventLoopGroup(eventLoopGroup); @@ -216,8 +253,7 @@ protected PasswordAuthentication getPasswordAuthentication() { * migration guide for mode details: */ private BlobServiceClientBuilder applyLocationMode(final BlobServiceClientBuilder builder, final AzureStorageSettings settings) { - final StorageConnectionString storageConnectionString = StorageConnectionString.create(settings.getConnectString(), logger); - final StorageEndpoint endpoint = storageConnectionString.getBlobEndpoint(); + final StorageEndpoint endpoint = settings.getStorageEndpoint(logger); if (endpoint == null || endpoint.getPrimaryUri() == null) { throw new IllegalArgumentException("connectionString missing required settings to derive blob service primary endpoint."); @@ -247,9 +283,8 @@ private BlobServiceClientBuilder applyLocationMode(final BlobServiceClientBuilde return builder; } - private static BlobServiceClientBuilder createClientBuilder(AzureStorageSettings settings) throws InvalidKeyException, - URISyntaxException { - return SocketAccess.doPrivilegedException(() -> new BlobServiceClientBuilder().connectionString(settings.getConnectString())); + private BlobServiceClientBuilder createClientBuilder(AzureStorageSettings settings) throws InvalidKeyException, URISyntaxException { + return SocketAccess.doPrivilegedException(() -> settings.configure(new BlobServiceClientBuilder(), executor, logger)); } /** @@ -295,9 +330,19 @@ public Map refreshAndClearCache(Map SecureSetting.secureString(key, null) ); + /** Azure token credentials such as Managed Identity */ + public static final AffixSetting TOKEN_CREDENTIAL_TYPE_SETTING = Setting.affixKeySetting( + AZURE_CLIENT_PREFIX_KEY, + "token_credential_type", + key -> Setting.simpleString(key, value -> { + if (Strings.hasText(value) == true) { + TokenCredentialType.valueOfType(value); + } + }, Property.NodeScope), + () -> ACCOUNT_SETTING + ); + /** max_retries: Number of retries in case of Azure errors. Defaults to 3 (RetryPolicy.DEFAULT_CLIENT_RETRY_COUNT). */ public static final AffixSetting MAX_RETRIES_SETTING = Setting.affixKeySetting( AZURE_CLIENT_PREFIX_KEY, @@ -194,7 +218,9 @@ final class AzureStorageSettings { ); private final String account; - private final String connectString; + private final String tokenCredentialType; + private final TriFunction clientBuilder; + private final Function endpointBuilder; private final String endpointSuffix; private final TimeValue timeout; private final int maxRetries; @@ -208,7 +234,9 @@ final class AzureStorageSettings { // copy-constructor private AzureStorageSettings( String account, - String connectString, + String tokenCredentialType, + TriFunction clientBuilder, + Function endpointBuilder, String endpointSuffix, TimeValue timeout, int maxRetries, @@ -220,7 +248,9 @@ private AzureStorageSettings( ProxySettings proxySettings ) { this.account = account; - this.connectString = connectString; + this.tokenCredentialType = tokenCredentialType; + this.clientBuilder = clientBuilder; + this.endpointBuilder = endpointBuilder; this.endpointSuffix = endpointSuffix; this.timeout = timeout; this.maxRetries = maxRetries; @@ -236,6 +266,7 @@ private AzureStorageSettings( String account, String key, String sasToken, + String tokenCredentialType, String endpointSuffix, TimeValue timeout, int maxRetries, @@ -246,7 +277,37 @@ private AzureStorageSettings( ProxySettings proxySettings ) { this.account = account; - this.connectString = buildConnectString(account, key, sasToken, endpointSuffix); + this.tokenCredentialType = tokenCredentialType; + if (Strings.hasText(tokenCredentialType) == true) { + this.endpointBuilder = (logger) -> { + String tokenCredentialEndpointSuffix = endpointSuffix; + if (Strings.hasText(tokenCredentialEndpointSuffix) == false) { + // Default to "core.windows.net". + tokenCredentialEndpointSuffix = Constants.ConnectionStringConstants.DEFAULT_DNS; + } + final URI primaryBlobEndpoint = URI.create("https://" + account + ".blob." + tokenCredentialEndpointSuffix); + final URI secondaryBlobEndpoint = URI.create("https://" + account + "-secondary.blob." + tokenCredentialEndpointSuffix); + return new StorageEndpoint(primaryBlobEndpoint, secondaryBlobEndpoint); + }; + + this.clientBuilder = (builder, executor, logger) -> builder.credential(new ManagedIdentityCredentialBuilder() { + @Override + public ManagedIdentityCredential build() { + // Use the privileged executor with IdentityClient instance + CredentialBuilderBaseHelper.getClientOptions(this).setExecutorService(executor); + return super.build(); + } + }.build()).endpoint(endpointBuilder.apply(logger).getPrimaryUri()); + } else { + final String connectString = buildConnectString(account, key, sasToken, endpointSuffix); + + this.endpointBuilder = (logger) -> { + final StorageConnectionString storageConnectionString = StorageConnectionString.create(connectString, logger); + return storageConnectionString.getBlobEndpoint(); + }; + + this.clientBuilder = (builder, executor, logger) -> builder.connectionString(connectString); + } this.endpointSuffix = endpointSuffix; this.timeout = timeout; this.maxRetries = maxRetries; @@ -258,6 +319,14 @@ private AzureStorageSettings( this.proxySettings = proxySettings; } + public String getTokenCredentialType() { + return tokenCredentialType; + } + + public StorageEndpoint getStorageEndpoint(ClientLogger logger) { + return endpointBuilder.apply(logger); + } + public String getEndpointSuffix() { return endpointSuffix; } @@ -274,10 +343,6 @@ public ProxySettings getProxySettings() { return proxySettings; } - public String getConnectString() { - return connectString; - } - private static String buildConnectString(String account, @Nullable String key, @Nullable String sasToken, String endpointSuffix) { final boolean hasSasToken = Strings.hasText(sasToken); final boolean hasKey = Strings.hasText(key); @@ -325,6 +390,7 @@ public String toString() { final StringBuilder sb = new StringBuilder("AzureStorageSettings{"); sb.append("account='").append(account).append('\''); sb.append(", timeout=").append(timeout); + sb.append(", tokenCredentialType=").append(tokenCredentialType).append('\''); sb.append(", endpointSuffix='").append(endpointSuffix).append('\''); sb.append(", maxRetries=").append(maxRetries); sb.append(", proxySettings=").append(proxySettings != ProxySettings.NO_PROXY_SETTINGS ? "PROXY_SET" : "PROXY_NOT_SET"); @@ -370,6 +436,7 @@ private static AzureStorageSettings getClientSettings(Settings settings, String account.toString(), key.toString(), sasToken.toString(), + getValue(settings, clientName, TOKEN_CREDENTIAL_TYPE_SETTING), getValue(settings, clientName, ENDPOINT_SUFFIX_SETTING), getValue(settings, clientName, TIMEOUT_SETTING), getValue(settings, clientName, MAX_RETRIES_SETTING), @@ -430,7 +497,9 @@ static Map overrideLocationMode( entry.getKey(), new AzureStorageSettings( entry.getValue().account, - entry.getValue().connectString, + entry.getValue().tokenCredentialType, + entry.getValue().clientBuilder, + entry.getValue().endpointBuilder, entry.getValue().endpointSuffix, entry.getValue().timeout, entry.getValue().maxRetries, @@ -445,4 +514,8 @@ static Map overrideLocationMode( } return mapBuilder.immutableMap(); } + + public BlobServiceClientBuilder configure(BlobServiceClientBuilder builder, ExecutorService executor, ClientLogger logger) { + return clientBuilder.apply(builder, executor, logger); + } } diff --git a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/TokenCredentialType.java b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/TokenCredentialType.java new file mode 100644 index 0000000000000..1f78f73934231 --- /dev/null +++ b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/TokenCredentialType.java @@ -0,0 +1,40 @@ +/* + * 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.repositories.azure; + +import java.util.Arrays; + +// Type of token credentials that the plugin supports +public enum TokenCredentialType { + MANAGED_IDENTITY("managed"); + + private final String type; + + TokenCredentialType(String type) { + this.type = type; + } + + public static String[] getTokenCredentialTypes() { + return Arrays.stream(TokenCredentialType.values()).map(tokenCredentialType -> tokenCredentialType.type).toArray(String[]::new); + } + + static TokenCredentialType valueOfType(String type) { + for (TokenCredentialType value : values()) { + if (value.type.equalsIgnoreCase(type) || value.name().equalsIgnoreCase(type)) { + return value; + } + } + throw new IllegalArgumentException( + "The token credential type '" + + type + + "' is unsupported, please use one of the following values: " + + String.join(", ", getTokenCredentialTypes()) + ); + } +} diff --git a/plugins/repository-azure/src/main/plugin-metadata/plugin-security.policy b/plugins/repository-azure/src/main/plugin-metadata/plugin-security.policy index f3bf52ea46505..e8fbe35ebab1d 100644 --- a/plugins/repository-azure/src/main/plugin-metadata/plugin-security.policy +++ b/plugins/repository-azure/src/main/plugin-metadata/plugin-security.policy @@ -41,4 +41,7 @@ grant { // azure client set Authenticator for proxy username/password permission java.net.NetPermission "setDefaultAuthenticator"; + + // azure identity + permission java.util.PropertyPermission "os.name", "read"; }; diff --git a/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureStorageServiceTests.java b/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureStorageServiceTests.java index bb0eafc7d1d4a..ea74a49e593cf 100644 --- a/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureStorageServiceTests.java +++ b/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureStorageServiceTests.java @@ -33,6 +33,7 @@ package org.opensearch.repositories.azure; import com.azure.core.http.policy.HttpPipelinePolicy; +import com.azure.identity.CredentialUnavailableException; import com.azure.storage.blob.BlobServiceClient; import com.azure.storage.common.policy.RequestRetryPolicy; import org.opensearch.common.settings.MockSecureSettings; @@ -50,7 +51,6 @@ import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; -import java.net.UnknownHostException; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.Base64; @@ -106,11 +106,105 @@ public void testCreateClientWithEndpointSuffix() throws IOException { .put("azure.client.azure1.endpoint_suffix", "my_endpoint_suffix") .build(); try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings)) { - final AzureStorageService azureStorageService = plugin.azureStoreService; - final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); - assertThat(client1.getAccountUrl(), equalTo("https://myaccount1.blob.my_endpoint_suffix")); - final BlobServiceClient client2 = azureStorageService.client("azure2").v1(); - assertThat(client2.getAccountUrl(), equalTo("https://myaccount2.blob.core.windows.net")); + try (final AzureStorageService azureStorageService = plugin.azureStoreService) { + final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); + assertThat(client1.getAccountUrl(), equalTo("https://myaccount1.blob.my_endpoint_suffix")); + final BlobServiceClient client2 = azureStorageService.client("azure2").v1(); + assertThat(client2.getAccountUrl(), equalTo("https://myaccount2.blob.core.windows.net")); + } + } + } + + public void testCreateClientWithEndpointSuffixWhenManagedIdentityIsEnabled() throws IOException { + final MockSecureSettings secureSettings = new MockSecureSettings(); + // Azure clients without account key and sas token. + secureSettings.setString("azure.client.azure1.account", "myaccount1"); + secureSettings.setString("azure.client.azure2.account", "myaccount2"); + + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + // Enabled managed identity for all clients + .put("azure.client.azure1.token_credential_type", TokenCredentialType.MANAGED_IDENTITY.name()) + .put("azure.client.azure2.token_credential_type", TokenCredentialType.MANAGED_IDENTITY.name()) + // Defined an endpoint suffix for azure client 1 only. + .put("azure.client.azure1.endpoint_suffix", "my_endpoint_suffix") + .build(); + try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings)) { + try (final AzureStorageService azureStorageService = plugin.azureStoreService) { + // Expect azure client 1 to use the custom endpoint suffix + final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); + assertThat(client1.getAccountUrl(), equalTo("https://myaccount1.blob.my_endpoint_suffix")); + // Expect azure client 2 to use the default endpoint suffix + final BlobServiceClient client2 = azureStorageService.client("azure2").v1(); + assertThat(client2.getAccountUrl(), equalTo("https://myaccount2.blob.core.windows.net")); + } + } + } + + public void testCreateClientWithInvalidEndpointSuffix() throws IOException { + final MockSecureSettings secureSettings = new MockSecureSettings(); + secureSettings.setString("azure.client.azure1.account", "myaccount1"); + secureSettings.setString("azure.client.azure2.account", "myaccount2"); + secureSettings.setString("azure.client.azure2.key", encodeKey("mykey12")); + secureSettings.setString("azure.client.azure3.account", "myaccount1"); + secureSettings.setString("azure.client.azure3.sas_token", encodeKey("mysastoken")); + + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + .put("azure.client.azure1.token_credential_type", TokenCredentialType.MANAGED_IDENTITY.name()) + .put("azure.client.azure1.endpoint_suffix", "invalid endpoint suffix") + .put("azure.client.azure2.endpoint_suffix", "invalid endpoint suffix") + .put("azure.client.azure3.endpoint_suffix", "invalid endpoint suffix") + .build(); + + try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings)) { + try (final AzureStorageService azureStorageService = plugin.azureStoreService) { + // Expect all clients 1 to fail due to invalid endpoint suffix + expectThrows(SettingsException.class, () -> azureStorageService.client("azure1").v1()); + expectThrows(RuntimeException.class, () -> azureStorageService.client("azure2").v1()); + expectThrows(RuntimeException.class, () -> azureStorageService.client("azure3").v1()); + } + } + } + + public void testGettingSecondaryStorageBlobEndpoint() throws IOException { + final MockSecureSettings secureSettings = new MockSecureSettings(); + secureSettings.setString("azure.client.azure1.account", "myaccount1"); + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + // Enabled managed identity + .put("azure.client.azure1.token_credential_type", TokenCredentialType.MANAGED_IDENTITY.name()) + .build(); + try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings)) { + try (final AzureStorageService azureStorageService = plugin.azureStoreService) { + final Map prevSettings = azureStorageService.refreshAndClearCache(Collections.emptyMap()); + final Map newSettings = AzureStorageSettings.overrideLocationMode( + prevSettings, + LocationMode.SECONDARY_ONLY + ); + azureStorageService.refreshAndClearCache(newSettings); + final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); + assertThat(client1.getAccountUrl(), equalTo("https://myaccount1-secondary.blob.core.windows.net")); + } + } + } + + public void testClientUsingManagedIdentity() throws IOException { + // Enabled managed identity + final Settings settings = Settings.builder() + .setSecureSettings(buildSecureSettings()) + .put("azure.client.azure1.token_credential_type", TokenCredentialType.MANAGED_IDENTITY.name()) + .build(); + try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings)) { + try (final AzureStorageService azureStorageService = plugin.azureStoreService) { + final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); + + // Expect the client to use managed identity for authentication, and it should fail because managed identity environment is + // not + // setup in the test + final CredentialUnavailableException e = expectThrows(CredentialUnavailableException.class, () -> client1.getAccountInfo()); + assertThat(e.getMessage(), is("Managed Identity authentication is not available.")); + } } } @@ -128,29 +222,30 @@ public void testReinitClientSettings() throws IOException { secureSettings2.setString("azure.client.azure3.key", encodeKey("mykey23")); final Settings settings2 = Settings.builder().setSecureSettings(secureSettings2).build(); try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings1)) { - final AzureStorageService azureStorageService = plugin.azureStoreService; - final BlobServiceClient client11 = azureStorageService.client("azure1").v1(); - assertThat(client11.getAccountUrl(), equalTo("https://myaccount11.blob.core.windows.net")); - final BlobServiceClient client12 = azureStorageService.client("azure2").v1(); - assertThat(client12.getAccountUrl(), equalTo("https://myaccount12.blob.core.windows.net")); - // client 3 is missing - final SettingsException e1 = expectThrows(SettingsException.class, () -> azureStorageService.client("azure3")); - assertThat(e1.getMessage(), is("Unable to find client with name [azure3]")); - // update client settings - plugin.reload(settings2); - // old client 1 not changed - assertThat(client11.getAccountUrl(), equalTo("https://myaccount11.blob.core.windows.net")); - // new client 1 is changed - final BlobServiceClient client21 = azureStorageService.client("azure1").v1(); - assertThat(client21.getAccountUrl(), equalTo("https://myaccount21.blob.core.windows.net")); - // old client 2 not changed - assertThat(client12.getAccountUrl(), equalTo("https://myaccount12.blob.core.windows.net")); - // new client2 is gone - final SettingsException e2 = expectThrows(SettingsException.class, () -> azureStorageService.client("azure2")); - assertThat(e2.getMessage(), is("Unable to find client with name [azure2]")); - // client 3 emerged - final BlobServiceClient client23 = azureStorageService.client("azure3").v1(); - assertThat(client23.getAccountUrl(), equalTo("https://myaccount23.blob.core.windows.net")); + try (final AzureStorageService azureStorageService = plugin.azureStoreService) { + final BlobServiceClient client11 = azureStorageService.client("azure1").v1(); + assertThat(client11.getAccountUrl(), equalTo("https://myaccount11.blob.core.windows.net")); + final BlobServiceClient client12 = azureStorageService.client("azure2").v1(); + assertThat(client12.getAccountUrl(), equalTo("https://myaccount12.blob.core.windows.net")); + // client 3 is missing + final SettingsException e1 = expectThrows(SettingsException.class, () -> azureStorageService.client("azure3")); + assertThat(e1.getMessage(), is("Unable to find client with name [azure3]")); + // update client settings + plugin.reload(settings2); + // old client 1 not changed + assertThat(client11.getAccountUrl(), equalTo("https://myaccount11.blob.core.windows.net")); + // new client 1 is changed + final BlobServiceClient client21 = azureStorageService.client("azure1").v1(); + assertThat(client21.getAccountUrl(), equalTo("https://myaccount21.blob.core.windows.net")); + // old client 2 not changed + assertThat(client12.getAccountUrl(), equalTo("https://myaccount12.blob.core.windows.net")); + // new client2 is gone + final SettingsException e2 = expectThrows(SettingsException.class, () -> azureStorageService.client("azure2")); + assertThat(e2.getMessage(), is("Unable to find client with name [azure2]")); + // client 3 emerged + final BlobServiceClient client23 = azureStorageService.client("azure3").v1(); + assertThat(client23.getAccountUrl(), equalTo("https://myaccount23.blob.core.windows.net")); + } } } @@ -160,17 +255,18 @@ public void testReinitClientEmptySettings() throws IOException { secureSettings.setString("azure.client.azure1.key", encodeKey("mykey11")); final Settings settings = Settings.builder().setSecureSettings(secureSettings).build(); try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings)) { - final AzureStorageService azureStorageService = plugin.azureStoreService; - final BlobServiceClient client11 = azureStorageService.client("azure1").v1(); - assertThat(client11.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); - // reinit with empty settings - final SettingsException e = expectThrows(SettingsException.class, () -> plugin.reload(Settings.EMPTY)); - assertThat(e.getMessage(), is("If you want to use an azure repository, you need to define a client configuration.")); - // existing client untouched - assertThat(client11.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); - // new client also untouched - final BlobServiceClient client21 = azureStorageService.client("azure1").v1(); - assertThat(client21.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); + try (final AzureStorageService azureStorageService = plugin.azureStoreService) { + final BlobServiceClient client11 = azureStorageService.client("azure1").v1(); + assertThat(client11.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); + // reinit with empty settings + final SettingsException e = expectThrows(SettingsException.class, () -> plugin.reload(Settings.EMPTY)); + assertThat(e.getMessage(), is("If you want to use an azure repository, you need to define a client configuration.")); + // existing client untouched + assertThat(client11.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); + // new client also untouched + final BlobServiceClient client21 = azureStorageService.client("azure1").v1(); + assertThat(client21.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); + } } } @@ -189,138 +285,150 @@ public void testReinitClientWrongSettings() throws IOException { secureSettings3.setString("azure.client.azure1.sas_token", encodeKey("mysasToken33")); final Settings settings3 = Settings.builder().setSecureSettings(secureSettings3).build(); try (AzureRepositoryPlugin plugin = pluginWithSettingsValidation(settings1)) { - final AzureStorageService azureStorageService = plugin.azureStoreService; - final BlobServiceClient client11 = azureStorageService.client("azure1").v1(); - assertThat(client11.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); - final SettingsException e1 = expectThrows(SettingsException.class, () -> plugin.reload(settings2)); - assertThat(e1.getMessage(), is("Neither a secret key nor a shared access token was set.")); - final SettingsException e2 = expectThrows(SettingsException.class, () -> plugin.reload(settings3)); - assertThat(e2.getMessage(), is("Both a secret as well as a shared access token were set.")); - // existing client untouched - assertThat(client11.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); + try (final AzureStorageService azureStorageService = plugin.azureStoreService) { + final BlobServiceClient client11 = azureStorageService.client("azure1").v1(); + assertThat(client11.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); + final SettingsException e1 = expectThrows(SettingsException.class, () -> plugin.reload(settings2)); + assertThat(e1.getMessage(), is("Neither a secret key nor a shared access token was set.")); + final SettingsException e2 = expectThrows(SettingsException.class, () -> plugin.reload(settings3)); + assertThat(e2.getMessage(), is("Both a secret as well as a shared access token were set.")); + // existing client untouched + assertThat(client11.getAccountUrl(), equalTo("https://myaccount1.blob.core.windows.net")); + } } } - public void testGetSelectedClientNonExisting() { - final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(buildSettings()); - final SettingsException e = expectThrows(SettingsException.class, () -> azureStorageService.client("azure4")); - assertThat(e.getMessage(), is("Unable to find client with name [azure4]")); + public void testGetSelectedClientNonExisting() throws IOException { + try (final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(buildSettings())) { + final SettingsException e = expectThrows(SettingsException.class, () -> azureStorageService.client("azure4")); + assertThat(e.getMessage(), is("Unable to find client with name [azure4]")); + } } - public void testGetSelectedClientDefaultTimeout() { + public void testGetSelectedClientDefaultTimeout() throws IOException { final Settings timeoutSettings = Settings.builder() .setSecureSettings(buildSecureSettings()) .put("azure.client.azure3.timeout", "30s") .build(); - final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(timeoutSettings); - assertThat(azureStorageService.getBlobRequestTimeout("azure1"), nullValue()); - assertThat(azureStorageService.getBlobRequestTimeout("azure3"), is(Duration.ofSeconds(30))); + try (final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(timeoutSettings)) { + assertThat(azureStorageService.getBlobRequestTimeout("azure1"), nullValue()); + assertThat(azureStorageService.getBlobRequestTimeout("azure3"), is(Duration.ofSeconds(30))); + } } - public void testClientDefaultConnectTimeout() { + public void testClientDefaultConnectTimeout() throws IOException { final Settings settings = Settings.builder() .setSecureSettings(buildSecureSettings()) .put("azure.client.azure3.connect.timeout", "25s") .build(); - final AzureStorageService mock = storageServiceWithSettingsValidation(settings); - final TimeValue timeout = mock.storageSettings.get("azure3").getConnectTimeout(); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + final TimeValue timeout = mock.storageSettings.get("azure3").getConnectTimeout(); - assertThat(timeout, notNullValue()); - assertThat(timeout, equalTo(TimeValue.timeValueSeconds(25))); - assertThat(mock.storageSettings.get("azure2").getConnectTimeout(), notNullValue()); - assertThat(mock.storageSettings.get("azure2").getConnectTimeout(), equalTo(TimeValue.timeValueSeconds(10))); + assertThat(timeout, notNullValue()); + assertThat(timeout, equalTo(TimeValue.timeValueSeconds(25))); + assertThat(mock.storageSettings.get("azure2").getConnectTimeout(), notNullValue()); + assertThat(mock.storageSettings.get("azure2").getConnectTimeout(), equalTo(TimeValue.timeValueSeconds(10))); + } } - public void testClientDefaultWriteTimeout() { + public void testClientDefaultWriteTimeout() throws IOException { final Settings settings = Settings.builder() .setSecureSettings(buildSecureSettings()) .put("azure.client.azure3.write.timeout", "85s") .build(); - final AzureStorageService mock = storageServiceWithSettingsValidation(settings); - final TimeValue timeout = mock.storageSettings.get("azure3").getWriteTimeout(); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + final TimeValue timeout = mock.storageSettings.get("azure3").getWriteTimeout(); - assertThat(timeout, notNullValue()); - assertThat(timeout, equalTo(TimeValue.timeValueSeconds(85))); - assertThat(mock.storageSettings.get("azure2").getWriteTimeout(), notNullValue()); - assertThat(mock.storageSettings.get("azure2").getWriteTimeout(), equalTo(TimeValue.timeValueSeconds(60))); + assertThat(timeout, notNullValue()); + assertThat(timeout, equalTo(TimeValue.timeValueSeconds(85))); + assertThat(mock.storageSettings.get("azure2").getWriteTimeout(), notNullValue()); + assertThat(mock.storageSettings.get("azure2").getWriteTimeout(), equalTo(TimeValue.timeValueSeconds(60))); + } } - public void testClientDefaultReadTimeout() { + public void testClientDefaultReadTimeout() throws IOException { final Settings settings = Settings.builder() .setSecureSettings(buildSecureSettings()) .put("azure.client.azure3.read.timeout", "120s") .build(); - final AzureStorageService mock = storageServiceWithSettingsValidation(settings); - final TimeValue timeout = mock.storageSettings.get("azure3").getReadTimeout(); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + final TimeValue timeout = mock.storageSettings.get("azure3").getReadTimeout(); - assertThat(timeout, notNullValue()); - assertThat(timeout, equalTo(TimeValue.timeValueSeconds(120))); - assertThat(mock.storageSettings.get("azure2").getReadTimeout(), notNullValue()); - assertThat(mock.storageSettings.get("azure2").getReadTimeout(), equalTo(TimeValue.timeValueSeconds(60))); + assertThat(timeout, notNullValue()); + assertThat(timeout, equalTo(TimeValue.timeValueSeconds(120))); + assertThat(mock.storageSettings.get("azure2").getReadTimeout(), notNullValue()); + assertThat(mock.storageSettings.get("azure2").getReadTimeout(), equalTo(TimeValue.timeValueSeconds(60))); + } } - public void testClientDefaultResponseTimeout() { + public void testClientDefaultResponseTimeout() throws IOException { final Settings settings = Settings.builder() .setSecureSettings(buildSecureSettings()) .put("azure.client.azure3.response.timeout", "1ms") .build(); - final AzureStorageService mock = storageServiceWithSettingsValidation(settings); - final TimeValue timeout = mock.storageSettings.get("azure3").getResponseTimeout(); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + final TimeValue timeout = mock.storageSettings.get("azure3").getResponseTimeout(); - assertThat(timeout, notNullValue()); - assertThat(timeout, equalTo(TimeValue.timeValueMillis(1))); - assertThat(mock.storageSettings.get("azure2").getResponseTimeout(), notNullValue()); - assertThat(mock.storageSettings.get("azure2").getResponseTimeout(), equalTo(TimeValue.timeValueSeconds(60))); + assertThat(timeout, notNullValue()); + assertThat(timeout, equalTo(TimeValue.timeValueMillis(1))); + assertThat(mock.storageSettings.get("azure2").getResponseTimeout(), notNullValue()); + assertThat(mock.storageSettings.get("azure2").getResponseTimeout(), equalTo(TimeValue.timeValueSeconds(60))); + } } - public void testGetSelectedClientNoTimeout() { - final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(buildSettings()); - assertThat(azureStorageService.getBlobRequestTimeout("azure1"), nullValue()); + public void testGetSelectedClientNoTimeout() throws IOException { + try (final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(buildSettings())) { + assertThat(azureStorageService.getBlobRequestTimeout("azure1"), nullValue()); + } } - public void testGetSelectedClientBackoffPolicy() { - final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(buildSettings()); - final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); - assertThat(requestRetryOptions(client1), is(notNullValue())); + public void testGetSelectedClientBackoffPolicy() throws IOException { + try (final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(buildSettings())) { + final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); + assertThat(requestRetryOptions(client1), is(notNullValue())); + } } - public void testGetSelectedClientBackoffPolicyNbRetries() { + public void testGetSelectedClientBackoffPolicyNbRetries() throws IOException { final Settings timeoutSettings = Settings.builder() .setSecureSettings(buildSecureSettings()) .put("azure.client.azure1.max_retries", 7) .build(); - final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(timeoutSettings); - final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); - assertThat(requestRetryOptions(client1), is(notNullValue())); + try (final AzureStorageService azureStorageService = storageServiceWithSettingsValidation(timeoutSettings)) { + final BlobServiceClient client1 = azureStorageService.client("azure1").v1(); + assertThat(requestRetryOptions(client1), is(notNullValue())); + } } - public void testNoProxy() { + public void testNoProxy() throws IOException { final Settings settings = Settings.builder().setSecureSettings(buildSecureSettings()).build(); - final AzureStorageService mock = storageServiceWithSettingsValidation(settings); - assertEquals(mock.storageSettings.get("azure1").getProxySettings(), ProxySettings.NO_PROXY_SETTINGS); - assertEquals(mock.storageSettings.get("azure2").getProxySettings(), ProxySettings.NO_PROXY_SETTINGS); - assertEquals(mock.storageSettings.get("azure3").getProxySettings(), ProxySettings.NO_PROXY_SETTINGS); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + assertEquals(mock.storageSettings.get("azure1").getProxySettings(), ProxySettings.NO_PROXY_SETTINGS); + assertEquals(mock.storageSettings.get("azure2").getProxySettings(), ProxySettings.NO_PROXY_SETTINGS); + assertEquals(mock.storageSettings.get("azure3").getProxySettings(), ProxySettings.NO_PROXY_SETTINGS); + } } - public void testProxyHttp() throws UnknownHostException { + public void testProxyHttp() throws IOException { final Settings settings = Settings.builder() .setSecureSettings(buildSecureSettings()) .put("azure.client.azure1.proxy.host", "127.0.0.1") .put("azure.client.azure1.proxy.port", 8080) .put("azure.client.azure1.proxy.type", "http") .build(); - final AzureStorageService mock = storageServiceWithSettingsValidation(settings); - final ProxySettings azure1Proxy = mock.storageSettings.get("azure1").getProxySettings(); - - assertThat(azure1Proxy, notNullValue()); - assertThat(azure1Proxy.getType(), is(ProxySettings.ProxyType.HTTP)); - assertThat(azure1Proxy.getAddress(), is(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 8080))); - assertEquals(ProxySettings.NO_PROXY_SETTINGS, mock.storageSettings.get("azure2").getProxySettings()); - assertEquals(ProxySettings.NO_PROXY_SETTINGS, mock.storageSettings.get("azure3").getProxySettings()); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + final ProxySettings azure1Proxy = mock.storageSettings.get("azure1").getProxySettings(); + + assertThat(azure1Proxy, notNullValue()); + assertThat(azure1Proxy.getType(), is(ProxySettings.ProxyType.HTTP)); + assertThat(azure1Proxy.getAddress(), is(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 8080))); + assertEquals(ProxySettings.NO_PROXY_SETTINGS, mock.storageSettings.get("azure2").getProxySettings()); + assertEquals(ProxySettings.NO_PROXY_SETTINGS, mock.storageSettings.get("azure3").getProxySettings()); + } } - public void testMultipleProxies() throws UnknownHostException { + public void testMultipleProxies() throws IOException { final Settings settings = Settings.builder() .setSecureSettings(buildSecureSettings()) .put("azure.client.azure1.proxy.host", "127.0.0.1") @@ -330,21 +438,22 @@ public void testMultipleProxies() throws UnknownHostException { .put("azure.client.azure2.proxy.port", 8081) .put("azure.client.azure2.proxy.type", "http") .build(); - final AzureStorageService mock = storageServiceWithSettingsValidation(settings); - final ProxySettings azure1Proxy = mock.storageSettings.get("azure1").getProxySettings(); - assertThat(azure1Proxy, notNullValue()); - assertThat(azure1Proxy.getType(), is(ProxySettings.ProxyType.HTTP)); - assertThat(azure1Proxy.getAddress(), is(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 8080))); - final ProxySettings azure2Proxy = mock.storageSettings.get("azure2").getProxySettings(); - assertThat(azure2Proxy, notNullValue()); - assertThat(azure2Proxy.getType(), is(ProxySettings.ProxyType.HTTP)); - assertThat(azure2Proxy.getAddress(), is(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 8081))); - assertTrue(Strings.isNullOrEmpty(azure2Proxy.getUsername())); - assertTrue(Strings.isNullOrEmpty(azure2Proxy.getPassword())); - assertEquals(mock.storageSettings.get("azure3").getProxySettings(), ProxySettings.NO_PROXY_SETTINGS); - } - - public void testProxySocks() throws UnknownHostException { + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + final ProxySettings azure1Proxy = mock.storageSettings.get("azure1").getProxySettings(); + assertThat(azure1Proxy, notNullValue()); + assertThat(azure1Proxy.getType(), is(ProxySettings.ProxyType.HTTP)); + assertThat(azure1Proxy.getAddress(), is(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 8080))); + final ProxySettings azure2Proxy = mock.storageSettings.get("azure2").getProxySettings(); + assertThat(azure2Proxy, notNullValue()); + assertThat(azure2Proxy.getType(), is(ProxySettings.ProxyType.HTTP)); + assertThat(azure2Proxy.getAddress(), is(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 8081))); + assertTrue(Strings.isNullOrEmpty(azure2Proxy.getUsername())); + assertTrue(Strings.isNullOrEmpty(azure2Proxy.getPassword())); + assertEquals(mock.storageSettings.get("azure3").getProxySettings(), ProxySettings.NO_PROXY_SETTINGS); + } + } + + public void testProxySocks() throws IOException { final MockSecureSettings secureSettings = buildSecureSettings(); secureSettings.setString("azure.client.azure1.proxy.username", "user"); secureSettings.setString("azure.client.azure1.proxy.password", "pwd"); @@ -354,15 +463,16 @@ public void testProxySocks() throws UnknownHostException { .put("azure.client.azure1.proxy.type", "socks5") .setSecureSettings(secureSettings) .build(); - final AzureStorageService mock = storageServiceWithSettingsValidation(settings); - final ProxySettings azure1Proxy = mock.storageSettings.get("azure1").getProxySettings(); - assertThat(azure1Proxy, notNullValue()); - assertThat(azure1Proxy.getType(), is(ProxySettings.ProxyType.SOCKS5)); - assertThat(azure1Proxy.getAddress(), is(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 8080))); - assertEquals("user", azure1Proxy.getUsername()); - assertEquals("pwd", azure1Proxy.getPassword()); - assertEquals(ProxySettings.NO_PROXY_SETTINGS, mock.storageSettings.get("azure2").getProxySettings()); - assertEquals(ProxySettings.NO_PROXY_SETTINGS, mock.storageSettings.get("azure3").getProxySettings()); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + final ProxySettings azure1Proxy = mock.storageSettings.get("azure1").getProxySettings(); + assertThat(azure1Proxy, notNullValue()); + assertThat(azure1Proxy.getType(), is(ProxySettings.ProxyType.SOCKS5)); + assertThat(azure1Proxy.getAddress(), is(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 8080))); + assertEquals("user", azure1Proxy.getUsername()); + assertEquals("pwd", azure1Proxy.getPassword()); + assertEquals(ProxySettings.NO_PROXY_SETTINGS, mock.storageSettings.get("azure2").getProxySettings()); + assertEquals(ProxySettings.NO_PROXY_SETTINGS, mock.storageSettings.get("azure3").getProxySettings()); + } } public void testProxyNoHost() { @@ -420,6 +530,199 @@ public void testBlobNameFromUri() throws URISyntaxException { assertThat(name, is("path/to/myfile")); } + public void testSettingTokenCredentialForAuthenticationIsCaseInsensitive() throws IOException { + final MockSecureSettings secureSettings = new MockSecureSettings(); + // Azure client without account key or sas token. + secureSettings.setString("azure.client.azure.account", "myaccount"); + secureSettings.setString("azure.client.azure2.account", "myaccount"); + + // Enabled Managed Identity in the settings using lower case and mixed case + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + .put("azure.client.azure.token_credential_type", "managed_identity") + .put("azure.client.azure2.token_credential_type", "managed_IDENTITY") + .build(); + + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + assertEquals(mock.storageSettings.get("azure").getTokenCredentialType(), "managed_identity"); + assertEquals(mock.storageSettings.get("azure2").getTokenCredentialType(), "managed_IDENTITY"); + } + } + + public void testSettingTokenCredentialForAuthenticationWithAlternativeEnumValue() throws IOException { + final MockSecureSettings secureSettings = new MockSecureSettings(); + // Azure client without account key or sas token. + secureSettings.setString("azure.client.azure.account", "myaccount"); + + // Enabled Managed Identity in the settings using lower case + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + .put("azure.client.azure.token_credential_type", "managed") + .build(); + + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + assertEquals(mock.storageSettings.get("azure").getTokenCredentialType(), "managed"); + } + } + + public void testSettingUnsupportedTokenCredentialForAuthentication() { + final String unsupported_token_credential_type = "TOKEN_CREDENTIAL_TYPE_THAT_DOES_NOT_EXIST"; + final MockSecureSettings secureSettings = new MockSecureSettings(); + + // Azure client without account key or sas token. + secureSettings.setString("azure.client.azure.account", "myaccount"); + + // Enable the unsupported token credential type + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + .put("azure.client.azure.token_credential_type", unsupported_token_credential_type) + .build(); + + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> storageServiceWithSettingsValidation(settings) + ); + assertEquals( + "The token credential type '" + + unsupported_token_credential_type + + "' is unsupported, please use one of the following values: " + + String.join(", ", TokenCredentialType.getTokenCredentialTypes()), + e.getMessage() + ); + } + + public void testTokenCredentialAuthenticationOverridesOtherFormOfAuthentications() throws IOException { + final String token_credential_type = TokenCredentialType.MANAGED_IDENTITY.name(); + final MockSecureSettings secureSettings = new MockSecureSettings(); + // Azure1 with account key + secureSettings.setString("azure.client.azure1.account", "myaccount1"); + secureSettings.setString("azure.client.azure1.key", encodeKey("mykey")); + + // Azure 2 with sas token + secureSettings.setString("azure.client.azure2.account", "myaccount2"); + secureSettings.setString("azure.client.azure2.sas_token", encodeKey("mysastoken")); + + // Azure 3 with account key and sas token + secureSettings.setString("azure.client.azure3.account", "myaccount3"); + secureSettings.setString("azure.client.azure3.key", encodeKey("mykey")); + secureSettings.setString("azure.client.azure3.sas_token", encodeKey("mysastoken")); + + // Azure 4 without sas token and account key + secureSettings.setString("azure.client.azure4.account", "myaccount4"); + + // Enable Managed Identity in all azure clients + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + .put("azure.client.azure1.token_credential_type", token_credential_type) + .put("azure.client.azure2.token_credential_type", token_credential_type) + .put("azure.client.azure3.token_credential_type", token_credential_type) + .put("azure.client.azure4.token_credential_type", token_credential_type) + .build(); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + // Expect token credential authentication is selected over account key or sas token. + assertEquals(token_credential_type, mock.storageSettings.get("azure1").getTokenCredentialType()); + assertEquals(token_credential_type, mock.storageSettings.get("azure2").getTokenCredentialType()); + assertEquals(token_credential_type, mock.storageSettings.get("azure3").getTokenCredentialType()); + assertEquals(token_credential_type, mock.storageSettings.get("azure4").getTokenCredentialType()); + } + } + + public void testTokenCredentialWhenAccountIsNotProvided() { + // Setting with an account specified + final MockSecureSettings secureSettings = new MockSecureSettings(); + + // Enabled Managed Identity in the settings + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + .put("azure.client.azure.token_credential_type", TokenCredentialType.MANAGED_IDENTITY.name()) + .build(); + final Exception e = expectThrows(Exception.class, () -> storageServiceWithSettingsValidation(settings)); + + // Expect failure due to missing account name + assertEquals( + "missing required setting [azure.client.azure.account] for setting [azure.client.azure.token_credential_type]", + e.getMessage() + ); + } + + public void testAuthenticationMethodNotProvided() { + final MockSecureSettings secureSettings = new MockSecureSettings(); + // Azure client without account key and sas token. + secureSettings.setString("azure.client.azure.account", "myaccount"); + + // Disabled Managed Identity in the settings by default + final Settings settings = Settings.builder().setSecureSettings(secureSettings).build(); + final SettingsException e = expectThrows(SettingsException.class, () -> storageServiceWithSettingsValidation(settings)); + + // Expect fall back to authentication via sas token or account key when token credential is not specified. + assertEquals("Neither a secret key nor a shared access token was set.", e.getMessage()); + } + + public void testSettingTokenCredentialTypeToBeEmpty() { + // Azure clients without account key and sas token. + final MockSecureSettings secureSettings = new MockSecureSettings(); + secureSettings.setString("azure.client.azure1.account", "myaccount"); + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + .put("azure.client.azure1.token_credential_type", "") + .build(); + // Expect fall back to authentication via sas token or account key when token credential is not specified. + final SettingsException e = expectThrows(SettingsException.class, () -> storageServiceWithSettingsValidation(settings)); + assertEquals("Neither a secret key nor a shared access token was set.", e.getMessage()); + + // Azure clients without account key and sas token. + final MockSecureSettings secureSettings2 = new MockSecureSettings(); + secureSettings2.setString("azure.client.azure2.account", "myaccount"); + final Settings settings2 = Settings.builder() + .setSecureSettings(secureSettings2) + .put("azure.client.azure2.token_credential_type", "x") + .build(); + // Expect failing token credential type checks + final IllegalArgumentException e2 = expectThrows( + IllegalArgumentException.class, + () -> storageServiceWithSettingsValidation(settings2) + ); + assertEquals( + "The token credential type 'x' is unsupported, please use one of the following values: " + + String.join(", ", TokenCredentialType.getTokenCredentialTypes()), + e2.getMessage() + ); + } + + public void testManagedIdentityIsEnabled() throws IOException { + final MockSecureSettings secureSettings = new MockSecureSettings(); + // Azure client without account key or sas token. + secureSettings.setString("azure.client.azure.account", "myaccount"); + + // Enabled Managed Identity in the settings. + final Settings settings = Settings.builder() + .setSecureSettings(secureSettings) + .put("azure.client.azure.token_credential_type", TokenCredentialType.MANAGED_IDENTITY.name()) + .build(); + + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + assertEquals(mock.storageSettings.get("azure").getTokenCredentialType(), TokenCredentialType.MANAGED_IDENTITY.name()); + } + } + + public void testNonTokenCredentialAuthenticationEnabled() throws IOException { + final MockSecureSettings secureSettings = new MockSecureSettings(); + // Azure client account key. + secureSettings.setString("azure.client.azure1.account", "myaccount1"); + secureSettings.setString("azure.client.azure1.sas_token", encodeKey("mysastoken")); + + // Azure client with sas token + secureSettings.setString("azure.client.azure2.account", "myaccount2"); + secureSettings.setString("azure.client.azure2.key", encodeKey("mykey")); + + final Settings settings = Settings.builder().setSecureSettings(secureSettings).build(); + try (final AzureStorageService mock = storageServiceWithSettingsValidation(settings)) { + // Expect token credential is not enabled + assertEquals(mock.storageSettings.get("azure1").getTokenCredentialType(), ""); + assertEquals(mock.storageSettings.get("azure2").getTokenCredentialType(), ""); + } + } + private static MockSecureSettings buildSecureSettings() { final MockSecureSettings secureSettings = new MockSecureSettings(); secureSettings.setString("azure.client.azure1.account", "myaccount1"); diff --git a/server/src/main/resources/org/opensearch/bootstrap/security.policy b/server/src/main/resources/org/opensearch/bootstrap/security.policy index e1226345ef961..55e8db0d9c6a3 100644 --- a/server/src/main/resources/org/opensearch/bootstrap/security.policy +++ b/server/src/main/resources/org/opensearch/bootstrap/security.policy @@ -85,6 +85,11 @@ grant codeBase "${codebase.zstd-jni}" { permission java.lang.RuntimePermission "loadLibrary.*"; }; +// repository-azure plugin and server side streaming +grant codeBase "${codebase.reactor-core}" { + permission java.net.SocketPermission "*", "connect,resolve"; +}; + //// Everything else: grant {