From 4ba39a5d914a22e021a6c51066ac31fdecb6e9d2 Mon Sep 17 00:00:00 2001 From: vikasvb90 Date: Mon, 5 Feb 2024 08:50:04 +0530 Subject: [PATCH 1/5] Permit backed futures to prevent timeouts during upload bursts Signed-off-by: vikasvb90 --- .../repositories/s3/S3Repository.java | 16 + .../repositories/s3/S3RepositoryPlugin.java | 31 +- .../s3/async/AsyncPartsHandler.java | 76 +-- .../s3/async/AsyncTransferManager.java | 79 +-- .../PermitBackedRetryableFutureUtils.java | 344 ++++++++++++ .../s3/S3BlobContainerMockClientTests.java | 21 +- .../s3/S3BlobContainerRetriesTests.java | 18 +- .../s3/async/AsyncTransferManagerTests.java | 9 +- ...PermitBackedRetryableFutureUtilsTests.java | 505 ++++++++++++++++++ .../blobstore/stream/write/WritePriority.java | 3 +- .../shard/RemoteStoreRefreshListener.java | 6 +- .../index/store/RemoteDirectory.java | 10 +- .../store/RemoteSegmentStoreDirectory.java | 4 +- .../index/store/RemoteDirectoryTests.java | 6 +- .../RemoteSegmentStoreDirectoryTests.java | 4 +- 15 files changed, 1054 insertions(+), 78 deletions(-) create mode 100644 plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtils.java create mode 100644 plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtilsTests.java 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..f8481e3a9402c 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 @@ -194,6 +194,22 @@ class S3Repository extends MeteredBlobStoreRepository { Setting.Property.NodeScope ); + /** + * Number of retries in case of a transfer failure. + */ + public static Setting S3_MAX_TRANSFER_RETRIES = Setting.intSetting("s3_max_transfer_retries", 3, 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 + ); + /** * Big files can be broken down into chunks during snapshotting if needed. Defaults to 1g. */ 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..63872f0b98cd8 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 @@ -53,6 +53,7 @@ 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.PermitBackedRetryableFutureUtils; import org.opensearch.script.ScriptService; import org.opensearch.threadpool.ExecutorBuilder; import org.opensearch.threadpool.FixedExecutorBuilder; @@ -69,6 +70,8 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; /** @@ -81,6 +84,7 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private static final String PRIORITY_FUTURE_COMPLETION = "priority_future_completion"; private static final String PRIORITY_STREAM_READER = "priority_stream_reader"; private static final String FUTURE_COMPLETION = "future_completion"; + private static final String REMOTE_TRANSFER_RETRY = "remote_transfer_retry"; private static final String STREAM_READER = "stream_reader"; protected final S3Service service; @@ -91,6 +95,8 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private AsyncExecutorContainer urgentExecutorBuilder; private AsyncExecutorContainer priorityExecutorBuilder; private AsyncExecutorContainer normalExecutorBuilder; + private ExecutorService remoteTransferRetryPool; + private ScheduledExecutorService scheduler; public S3RepositoryPlugin(final Settings settings, final Path configPath) { this(settings, configPath, new S3Service(configPath), new S3AsyncService(configPath)); @@ -120,6 +126,14 @@ public List> getExecutorBuilders(Settings settings) { TimeValue.timeValueMinutes(5) ) ); + executorBuilders.add( + new ScalingExecutorBuilder( + REMOTE_TRANSFER_RETRY, + allocatedProcessors(settings), + allocatedProcessors(settings) * 2, + TimeValue.timeValueMinutes(5) + ) + ); return executorBuilders; } @@ -189,6 +203,8 @@ public Collection createComponents( threadPool.executor(STREAM_READER), new AsyncTransferEventLoopGroup(normalEventLoopThreads) ); + this.remoteTransferRetryPool = threadPool.executor(REMOTE_TRANSFER_RETRY); + this.scheduler = threadPool.scheduler(); return Collections.emptyList(); } @@ -204,7 +220,16 @@ protected S3Repository createRepository( S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.get(clusterService.getSettings()).getBytes(), normalExecutorBuilder.getStreamReader(), priorityExecutorBuilder.getStreamReader(), - urgentExecutorBuilder.getStreamReader() + urgentExecutorBuilder.getStreamReader(), + new PermitBackedRetryableFutureUtils<>( + S3Repository.S3_MAX_TRANSFER_RETRIES.get(clusterService.getSettings()), + // High permit allocation because each op acquiring permit performs disk IO, computation and network IO. + Math.max(allocatedProcessors(clusterService.getSettings()) * 5, 10), + ((double) S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT.get(clusterService.getSettings())) / 100, + remoteTransferRetryPool, + scheduler + ) + ); return new S3Repository( metadata, @@ -263,7 +288,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_MAX_TRANSFER_RETRIES, + S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT ); } 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..f3955c9d69186 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 @@ -35,6 +35,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicReferenceArray; +import java.util.function.Supplier; /** * Responsible for handling parts of the original multipart request @@ -56,8 +57,8 @@ public class AsyncPartsHandler { * @param inputStreamContainers Checksum containers * @param statsMetricPublisher sdk metric publisher * @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 +70,51 @@ public static List> uploadParts( AtomicReferenceArray completedParts, AtomicReferenceArray inputStreamContainers, StatsMetricPublisher statsMetricPublisher, - boolean uploadRetryEnabled - ) throws IOException { + boolean uploadRetryEnabled, + PermitBackedRetryableFutureUtils permitBackedRetryableFutureUtils + ) throws InterruptedException { List> futures = new ArrayList<>(); + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBackedRetryableFutureUtils.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 + int finalPartIdx = partIdx; + Supplier> partFutureSupplier = () -> { + InputStreamContainer inputStreamContainer; + try { + inputStreamContainer = streamContext.provideStream(finalPartIdx); + } catch (IOException e) { + return CompletableFuture.failedFuture(e); + } + inputStreamContainers.set(finalPartIdx, new CheckedContainer(inputStreamContainer.getContentLength())); + UploadPartRequest.Builder uploadPartRequestBuilder = UploadPartRequest.builder() + .bucket(uploadRequest.getBucket()) + .partNumber(finalPartIdx + 1) + .key(uploadRequest.getKey()) + .uploadId(uploadId) + .overrideConfiguration(o -> o.addMetricPublisher(statsMetricPublisher.multipartUploadMetricCollector)) + .contentLength(inputStreamContainer.getContentLength()); + if (uploadRequest.doRemoteDataIntegrityCheck()) { + uploadPartRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32); + } + return uploadPart( + s3AsyncClient, + executorService, + priorityExecutorService, + urgentExecutorService, + completedParts, + inputStreamContainers, + uploadPartRequestBuilder.build(), + inputStreamContainer, + uploadRequest, + uploadRetryEnabled + ); + }; + + CompletableFuture retryableFuture = permitBackedRetryableFutureUtils.createPermitBackedRetryableFuture( + partFutureSupplier, + uploadRequest.getWritePriority(), + requestContext ); + futures.add(retryableFuture); } return futures; @@ -145,14 +162,13 @@ public static InputStream maybeRetryInputStream( return inputStream; } - private static void uploadPart( + private static CompletableFuture uploadPart( S3AsyncClient s3AsyncClient, ExecutorService executorService, ExecutorService priorityExecutorService, ExecutorService urgentExecutorService, AtomicReferenceArray completedParts, AtomicReferenceArray inputStreamContainers, - List> futures, UploadPartRequest uploadPartRequest, InputStreamContainer inputStreamContainer, UploadRequest uploadRequest, @@ -205,9 +221,9 @@ private static void uploadPart( uploadRequest.doRemoteDataIntegrityCheck() ) ); - futures.add(convertFuture); CompletableFutureUtils.forwardExceptionTo(convertFuture, uploadPartResponseFuture); + return convertFuture; } private static CompletedPart convertUploadPartResponse( 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..ad9ab884b0fe0 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 @@ -65,6 +65,9 @@ public final class AsyncTransferManager { private final ExecutorService urgentExecutorService; private final long minimumPartSize; + @SuppressWarnings("rawtypes") + private final PermitBackedRetryableFutureUtils permitBackedRetryableFutureUtils; + /** * The max number of parts on S3 side is 10,000 */ @@ -74,19 +77,20 @@ 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, + PermitBackedRetryableFutureUtils permitBackedRetryableFutureUtils ) { this.executorService = executorService; this.priorityExecutorService = priorityExecutorService; this.minimumPartSize = minimumPartSize; this.urgentExecutorService = urgentExecutorService; + this.permitBackedRetryableFutureUtils = permitBackedRetryableFutureUtils; } /** @@ -108,7 +112,7 @@ 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); + uploadInOneChunk(s3AsyncClient, uploadRequest, streamContext, returnFuture, statsMetricPublisher); } else { log.debug(() -> "Starting the upload as multipart upload request"); uploadInParts(s3AsyncClient, uploadRequest, streamContext, returnFuture, statsMetricPublisher); @@ -146,21 +150,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 +191,8 @@ private void doUploadInParts( completedParts, inputStreamContainers, statsMetricPublisher, - uploadRequest.isUploadRetryEnabled() + uploadRequest.isUploadRetryEnabled(), + permitBackedRetryableFutureUtils ); } catch (Exception ex) { try { @@ -320,13 +323,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 - ) { + ) throws InterruptedException { PutObjectRequest.Builder putObjectRequestBuilder = PutObjectRequest.builder() .bucket(uploadRequest.getBucket()) .key(uploadRequest.getKey()) @@ -349,14 +353,20 @@ private void uploadInOneChunk( streamReadExecutor = executorService; } - InputStream inputStream = AsyncPartsHandler.maybeRetryInputStream( - inputStreamContainer.getInputStream(), - uploadRequest.getWritePriority(), - uploadRequest.isUploadRetryEnabled(), - uploadRequest.getContentLength() - ); - CompletableFuture putObjectFuture = SocketAccess.doPrivileged( - () -> s3AsyncClient.putObject( + Supplier> putObjectFutureSupplier = () -> SocketAccess.doPrivileged(() -> { + InputStreamContainer inputStreamContainer; + try { + inputStreamContainer = streamContext.provideStream(0); + } catch (IOException e) { + return CompletableFuture.failedFuture(e); + } + InputStream inputStream = AsyncPartsHandler.maybeRetryInputStream( + inputStreamContainer.getInputStream(), + uploadRequest.getWritePriority(), + uploadRequest.isUploadRetryEnabled(), + uploadRequest.getContentLength() + ); + return s3AsyncClient.putObject( putObjectRequestBuilder.build(), AsyncRequestBody.fromInputStream(inputStream, inputStreamContainer.getContentLength(), streamReadExecutor) ).handle((resp, throwable) -> { @@ -395,7 +405,14 @@ private void uploadInOneChunk( } return null; - }) + }); + }); + + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBackedRetryableFutureUtils.createRequestContext(); + CompletableFuture putObjectFuture = permitBackedRetryableFutureUtils.createPermitBackedRetryableFuture( + putObjectFutureSupplier, + uploadRequest.getWritePriority(), + requestContext ); CompletableFutureUtils.forwardExceptionTo(returnFuture, putObjectFuture); diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtils.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtils.java new file mode 100644 index 0000000000000..25a45fb28c23f --- /dev/null +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtils.java @@ -0,0 +1,344 @@ +/* + * 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 software.amazon.awssdk.core.exception.SdkException; + +import org.opensearch.ExceptionsHelper; +import org.opensearch.action.bulk.BackoffPolicy; +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.common.unit.TimeValue; + +import java.util.Iterator; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import java.util.function.Supplier; + +/** + * Async wrapper over a completable future backed by transfer permits which provides natural backpressure in case + * of transfer bursts. Additionally, it retries futures (with exp backoff) which fail due to S3 exception or + * when permit couldn't be acquired within timeout period. + * + * @param Type of future response + */ +public class PermitBackedRetryableFutureUtils { + + // Package access for testing. + Semaphore lowPrioritySemaphore; + Semaphore highPrioritySemaphore; + private final int lowPriorityPermits; + private final int highPriorityPermits; + private final int maxRetryAttempts; + private static final int RETRY_BASE_INTERVAL_MILLIS = 1_000; + private final AtomicBoolean lowPriorityTransferProgress; + + private final ExecutorService remoteTransferRetryPool; + private final ScheduledExecutorService scheduler; + + /** + * + * @param maxRetryAttempts max number of retries + * @param availablePermits Total available permits for transfer + * @param priorityPermitAllocation Allocation bandwidth of priority permits. Rest will be allocated to low + * priority permits. + */ + public PermitBackedRetryableFutureUtils( + int maxRetryAttempts, + int availablePermits, + double priorityPermitAllocation, + ExecutorService remoteTransferRetryPool, + ScheduledExecutorService scheduler + ) { + this.maxRetryAttempts = maxRetryAttempts; + this.highPriorityPermits = (int) (priorityPermitAllocation * availablePermits); + this.highPrioritySemaphore = new TypeSemaphore(highPriorityPermits, "high"); + this.lowPriorityPermits = availablePermits - highPriorityPermits; + this.lowPrioritySemaphore = new TypeSemaphore(lowPriorityPermits, "low"); + this.lowPriorityTransferProgress = new AtomicBoolean(); + this.remoteTransferRetryPool = remoteTransferRetryPool; + this.scheduler = scheduler; + } + + /** + * Available low priority permits + * @return available low priority permits + */ + public int getAvailableLowPriorityPermits() { + return lowPrioritySemaphore.availablePermits(); + } + + /** + * Available high priority permits + * @return available high priority permits + */ + public int getAvailableHighPriorityPermits() { + return highPrioritySemaphore.availablePermits(); + } + + /** + * Named semaphore for debugging purpose + */ + static class TypeSemaphore extends Semaphore { + private final String type; + + public TypeSemaphore(int permits, String type) { + super(permits); + this.type = type; + } + + @Override + public String toString() { + String toStr = super.toString(); + return toStr + " , type = " + type; + } + + public String getType() { + return type; + } + + } + + /** + * 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); + } + + /** + * Custom exception to distinguish retryable futures. + */ + static class RetryableException extends CompletionException { + private final Iterator retryBackoffDelayIterator; + + public RetryableException(Iterator retryBackoffDelayIterator, String message, Throwable cause) { + super(message, cause); + this.retryBackoffDelayIterator = retryBackoffDelayIterator; + } + + public RetryableException(Iterator retryBackoffDelayIterator) { + this.retryBackoffDelayIterator = retryBackoffDelayIterator; + } + } + + /** + * DelayedExecutor and TaskSubmitter are copied from CompletableFuture. Duplicate classes are needed because + * scheduler used by these cannot be overriden and we need a way to manage it from outside. + */ + private static final class DelayedExecutor implements Executor { + private final long delay; + private final TimeUnit unit; + private final Executor executor; + private final ScheduledExecutorService scheduler; + + DelayedExecutor(long delay, TimeUnit unit, Executor executor, ScheduledExecutorService scheduler) { + this.delay = delay; + this.unit = unit; + this.executor = executor; + this.scheduler = scheduler; + } + + public void execute(Runnable r) { + scheduler.schedule(new TaskSubmitter(executor, r), delay, unit); + } + } + + private static final class TaskSubmitter implements Runnable { + final Executor executor; + final Runnable action; + + TaskSubmitter(Executor executor, Runnable action) { + this.executor = executor; + this.action = action; + } + + public void run() { + executor.execute(action); + } + } + + /** + * + * @param futureSupplier Supplier of the completable future + * @param writePriority Priority of transfer + * @param requestContext Request context object to set the decisions pertaining to transfer before transfers are + * initiated. + * + * @return completable future backed by permits and retryable future. + */ + public CompletableFuture createPermitBackedRetryableFuture( + Supplier> futureSupplier, + WritePriority writePriority, + RequestContext requestContext + ) { + Iterator retryBackoffDelayIterator = BackoffPolicy.exponentialBackoff( + TimeValue.timeValueMillis(RETRY_BASE_INTERVAL_MILLIS), + maxRetryAttempts + ).iterator(); + Supplier> permitBackedFutureSupplier = createPermitBackedFutureSupplier( + retryBackoffDelayIterator, + requestContext.lowPriorityPermitsConsumable, + futureSupplier, + writePriority + ); + + CompletableFuture permitBackedFuture; + try { + permitBackedFuture = permitBackedFutureSupplier.get(); + } catch (RetryableException re) { + // We need special handling when an exception occurs during first future creation itself. + permitBackedFuture = retry(re, permitBackedFutureSupplier, retryBackoffDelayIterator); + } catch (Exception ex) { + return CompletableFuture.failedFuture(ex); + } + + return flatten( + permitBackedFuture.thenApply(CompletableFuture::completedFuture) + .exceptionally(t -> retry(t, permitBackedFutureSupplier, retryBackoffDelayIterator)) + ); + } + + private static CompletableFuture flatten( + CompletableFuture> completableCompletable + ) { + return completableCompletable.thenCompose(Function.identity()); + } + + private CompletableFuture retry( + Throwable ex, + Supplier> futureSupplier, + Iterator retryBackoffDelayIterator + ) { + if (!(ex instanceof RetryableException)) { + return CompletableFuture.failedFuture(ex); + } + + RetryableException retryableException = (RetryableException) ex; + if (!retryBackoffDelayIterator.hasNext()) { + return CompletableFuture.failedFuture(ex); + } + + return flatten( + flatten( + CompletableFuture.supplyAsync( + futureSupplier, + new DelayedExecutor( + retryableException.retryBackoffDelayIterator.next().millis(), + TimeUnit.MILLISECONDS, + remoteTransferRetryPool, + scheduler + ) + ) + ).thenApply(CompletableFuture::completedFuture).exceptionally(t -> { + if (t instanceof RetryableException) { + ex.addSuppressed(t); + return retry(ex, futureSupplier, retryBackoffDelayIterator); + } else { + ex.addSuppressed(t); + return CompletableFuture.failedFuture(ex); + } + }) + ); + } + + // Package access for testing + Semaphore acquirePermit(WritePriority writePriority, boolean isLowPriorityPermitsConsumable) throws InterruptedException { + // 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 (highPrioritySemaphore.availablePermits() > 0.4 * highPriorityPermits && highPrioritySemaphore.tryAcquire()) { + return highPrioritySemaphore; + } else if (lowPrioritySemaphore.tryAcquire(5, TimeUnit.MINUTES)) { + return lowPrioritySemaphore; + } + return null; + } + + // Try acquiring high priority permit or low priority permit immediately if available. + // Otherwise, we wait for high priority permit. + if (highPrioritySemaphore.tryAcquire()) { + return highPrioritySemaphore; + } else if (isLowPriorityPermitsConsumable && lowPrioritySemaphore.tryAcquire()) { + return lowPrioritySemaphore; + } else if (highPrioritySemaphore.tryAcquire(5, TimeUnit.MINUTES)) { + return highPrioritySemaphore; + } + return null; + } + + private Supplier> createPermitBackedFutureSupplier( + Iterator retryBackoffDelayIterator, + boolean lowPriorityPermitsConsumable, + Supplier> futureSupplier, + WritePriority writePriority + ) { + return () -> { + Semaphore semaphore; + try { + semaphore = acquirePermit(writePriority, lowPriorityPermitsConsumable); + if (semaphore == null) { + throw new RetryableException(retryBackoffDelayIterator); + } + } catch (InterruptedException e) { + throw new CompletionException(e); + } + + CompletableFuture future; + try { + future = futureSupplier.get(); + } catch (Exception ex) { + // Exception in future creation. Can't retry this. + semaphore.release(); + throw new RuntimeException(ex); + } + + return future.handle((resp, t) -> { + try { + if (t != null) { + Throwable ex = ExceptionsHelper.unwrap(t, SdkException.class); + if (ex != null) { + throw new RetryableException(retryBackoffDelayIterator, t.getMessage(), t); + } + throw new CompletionException(t); + } + return resp; + } finally { + semaphore.release(); + } + }); + }; + } + +} 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..fee00fd9e6ec7 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,9 @@ 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.PermitBackedRetryableFutureUtils; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.Scheduler; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -65,6 +67,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,6 +94,8 @@ public class S3BlobContainerMockClientTests extends OpenSearchTestCase implement private MockS3AsyncService asyncService; private ExecutorService futureCompletionService; private ExecutorService streamReaderService; + private ExecutorService remoteTransferRetry; + private ScheduledExecutorService scheduler; private AsyncTransferEventLoopGroup transferNIOGroup; private S3BlobContainer blobContainer; @@ -364,6 +369,8 @@ public void setUp() throws Exception { asyncService = new MockS3AsyncService(configPath(), 1000); futureCompletionService = Executors.newSingleThreadExecutor(); streamReaderService = Executors.newSingleThreadExecutor(); + remoteTransferRetry = Executors.newFixedThreadPool(20); + scheduler = new Scheduler.SafeScheduledThreadPoolExecutor(1); transferNIOGroup = new AsyncTransferEventLoopGroup(1); blobContainer = createBlobContainer(); super.setUp(); @@ -373,6 +380,11 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { IOUtils.close(asyncService); + futureCompletionService.shutdown(); + streamReaderService.shutdown(); + remoteTransferRetry.shutdown(); + scheduler.shutdown(); + transferNIOGroup.close(); super.tearDown(); } @@ -410,7 +422,14 @@ private S3BlobStore createBlobStore() { S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.getDefault(Settings.EMPTY).getBytes(), asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), - asyncExecutorContainer.getStreamReader() + asyncExecutorContainer.getStreamReader(), + new PermitBackedRetryableFutureUtils<>( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 0.7, + remoteTransferRetry, + scheduler + ) ), asyncExecutorContainer, asyncExecutorContainer, 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..14691c89c202a 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,7 @@ 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.PermitBackedRetryableFutureUtils; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -87,6 +88,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,6 +117,8 @@ public class S3BlobContainerRetriesTests extends AbstractBlobContainerRetriesTes private S3AsyncService asyncService; private ExecutorService futureCompletionService; private ExecutorService streamReaderService; + private ExecutorService remoteTransferRetry; + private ScheduledExecutorService scheduler; private AsyncTransferEventLoopGroup transferNIOGroup; @Before @@ -125,6 +130,8 @@ public void setUp() throws Exception { futureCompletionService = Executors.newSingleThreadExecutor(); streamReaderService = Executors.newSingleThreadExecutor(); transferNIOGroup = new AsyncTransferEventLoopGroup(1); + remoteTransferRetry = Executors.newFixedThreadPool(20); + scheduler = new ScheduledThreadPoolExecutor(1); // needed by S3AsyncService SocketAccess.doPrivileged(() -> System.setProperty("opensearch.path.conf", configPath().toString())); @@ -137,6 +144,8 @@ public void tearDown() throws Exception { streamReaderService.shutdown(); futureCompletionService.shutdown(); + remoteTransferRetry.shutdown(); + scheduler.shutdown(); IOUtils.close(transferNIOGroup); if (previousOpenSearchPathConf != null) { @@ -223,7 +232,14 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( S3Repository.PARALLEL_MULTIPART_UPLOAD_MINIMUM_PART_SIZE_SETTING.getDefault(Settings.EMPTY).getBytes(), asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), - asyncExecutorContainer.getStreamReader() + asyncExecutorContainer.getStreamReader(), + new PermitBackedRetryableFutureUtils<>( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 0.7, + remoteTransferRetry, + scheduler + ) ), asyncExecutorContainer, asyncExecutorContainer, 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..e967e2b023465 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 @@ -67,7 +67,14 @@ public void setUp() throws Exception { ByteSizeUnit.MB.toBytes(5), Executors.newSingleThreadExecutor(), Executors.newSingleThreadExecutor(), - Executors.newSingleThreadExecutor() + Executors.newSingleThreadExecutor(), + new PermitBackedRetryableFutureUtils<>( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 0.7, + Executors.newSingleThreadExecutor(), + Executors.newSingleThreadScheduledExecutor() + ) ); super.setUp(); } diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtilsTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtilsTests.java new file mode 100644 index 0000000000000..b07bf0630f969 --- /dev/null +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtilsTests.java @@ -0,0 +1,505 @@ +/* + * 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 software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException; +import software.amazon.awssdk.utils.CompletableFutureUtils; + +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.repositories.s3.SocketAccess; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.After; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +@SuppressWarnings({ "unchecked", "rawtypes" }) +public class PermitBackedRetryableFutureUtilsTests extends OpenSearchTestCase { + private ExecutorService testExecutor; + private ScheduledExecutorService scheduler; + + @Before + public void setup() { + this.testExecutor = Executors.newFixedThreadPool(30); + this.scheduler = Executors.newSingleThreadScheduledExecutor(); + } + + @After + public void cleanUp() { + testExecutor.shutdown(); + scheduler.shutdown(); + } + + public void testFutureExecAndPermitRelease() throws InterruptedException, ExecutionException { + PermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new PermitBackedRetryableFutureUtils( + 3, + Runtime.getRuntime().availableProcessors(), + 0.7, + testExecutor, + scheduler + ); + int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); + int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; + assertEquals((int) (0.7 * Runtime.getRuntime().availableProcessors()), maxHighPermits); + assertEquals( + Runtime.getRuntime().availableProcessors() - maxHighPermits, + permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits() + ); + + Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { + assertEquals(maxHighPermits - 1, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + return "success"; + }, testExecutor)); + + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.HIGH, + requestContext + ); + resultFuture.get(); + + assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + } + + static class TestPermitBackedRetryableFutureUtils extends PermitBackedRetryableFutureUtils { + + private final AtomicBoolean highPermitsFullyConsumed = new AtomicBoolean(); + private final AtomicBoolean lowPermitsFullyConsumed = new AtomicBoolean(); + private final AtomicBoolean lowPermitConsumedForHighPriority = new AtomicBoolean(); + private final AtomicBoolean highPermitsConsumedForLowPriority = new AtomicBoolean(); + private final AtomicBoolean waitedForPermit = new AtomicBoolean(); + private final Runnable onWaitForPermit; + + /** + * @param maxRetryAttempts max number of retries + * @param availablePermits Total available permits for transfer + * @param priorityPermitAllocation Allocation bandwidth of priority permits. Rest will be allocated to low + * priority permits. + */ + public TestPermitBackedRetryableFutureUtils( + int maxRetryAttempts, + int availablePermits, + double priorityPermitAllocation, + ExecutorService remoteTransferRetry, + ScheduledExecutorService scheduler + ) { + super(maxRetryAttempts, availablePermits, priorityPermitAllocation, remoteTransferRetry, scheduler); + this.onWaitForPermit = null; + + } + + static class TestSemaphore extends TypeSemaphore { + private final Runnable onWaitForPermit; + private final Supplier preAcquireFailure; + + public TestSemaphore(int permits, String type, Runnable onWaitForPermit, Supplier preAcquireFailure) { + super(permits, type); + this.onWaitForPermit = onWaitForPermit; + this.preAcquireFailure = preAcquireFailure; + } + + @Override + public boolean tryAcquire(long timeout, TimeUnit unit) throws InterruptedException { + if (preAcquireFailure != null && preAcquireFailure.get()) { + return false; + } + onWaitForPermit.run(); + return super.tryAcquire(timeout, unit); + } + + @Override + public boolean tryAcquire() { + if (preAcquireFailure != null && preAcquireFailure.get()) { + return false; + } + return super.tryAcquire(); + } + } + + public TestPermitBackedRetryableFutureUtils( + int maxRetryAttempts, + int availablePermits, + double priorityPermitAllocation, + Runnable onWaitForPermit, + Supplier preAcquireFailure, + ExecutorService remoteTransferRetry, + ScheduledExecutorService scheduler + ) { + super(maxRetryAttempts, availablePermits, priorityPermitAllocation, remoteTransferRetry, scheduler); + this.onWaitForPermit = () -> { + waitedForPermit.set(true); + onWaitForPermit.run(); + }; + this.highPrioritySemaphore = new TestSemaphore( + highPrioritySemaphore.availablePermits(), + "high", + this.onWaitForPermit, + preAcquireFailure + ); + this.lowPrioritySemaphore = new TestSemaphore( + lowPrioritySemaphore.availablePermits(), + "low", + this.onWaitForPermit, + preAcquireFailure + ); + } + + Semaphore acquirePermit(WritePriority writePriority, boolean isLowPriorityPermitsConsumable) throws InterruptedException { + TypeSemaphore semaphore = (TypeSemaphore) super.acquirePermit(writePriority, isLowPriorityPermitsConsumable); + if (semaphore == null) { + return null; + } + if (semaphore.getType().equals("high")) { + if (getAvailableHighPriorityPermits() == 0) { + highPermitsFullyConsumed.set(true); + } + if (writePriority == WritePriority.LOW) { + highPermitsConsumedForLowPriority.set(true); + } + } else if (semaphore.getType().equals("low")) { + if (getAvailableLowPriorityPermits() == 0) { + lowPermitsFullyConsumed.set(true); + } + if (writePriority == WritePriority.HIGH) { + lowPermitConsumedForHighPriority.set(true); + } + } + return semaphore; + } + } + + public void testLowPermitConsumptionForHighTask() throws InterruptedException, ExecutionException { + TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( + 3, + Runtime.getRuntime().availableProcessors(), + 0.7, + testExecutor, + scheduler + ); + int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); + int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; + + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); + List> futures = new ArrayList<>(); + CountDownLatch delayedLatch = new CountDownLatch(1); + for (int reqIdx = 0; reqIdx < (maxHighPermits + lowPermits); reqIdx++) { + Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { + try { + // Keep the permit acquired + delayedLatch.await(10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return "success"; + }, testExecutor)); + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.HIGH, + requestContext + ); + futures.add(resultFuture); + } + // Now release all permits + delayedLatch.countDown(); + + CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); + assertTrue(permitBasedRetryableFutureUtils.lowPermitsFullyConsumed.get()); + assertTrue(permitBasedRetryableFutureUtils.highPermitsFullyConsumed.get()); + assertTrue(permitBasedRetryableFutureUtils.lowPermitConsumedForHighPriority.get()); + assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + } + + public void testOnlyHighPermitsAcquiredWhenLowTaskInProgress() throws ExecutionException, InterruptedException { + CountDownLatch delayedLatch = new CountDownLatch(1); + TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( + 3, + Runtime.getRuntime().availableProcessors(), + 0.7, + delayedLatch::countDown, + null, + testExecutor, + scheduler + ); + int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); + int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; + + PermitBackedRetryableFutureUtils.RequestContext lowRequestContext = permitBasedRetryableFutureUtils.createRequestContext(); + List> futures = new ArrayList<>(); + + Supplier> lowSupplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { + try { + // Keep the permit acquired + delayedLatch.await(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return "success"; + }, testExecutor)); + CompletableFuture lowResultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + lowSupplier, + WritePriority.LOW, + lowRequestContext + ); + futures.add(lowResultFuture); + + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); + for (int reqIdx = 0; reqIdx < maxHighPermits; reqIdx++) { + Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { + try { + // Keep the permit acquired + delayedLatch.await(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return "success"; + }, testExecutor)); + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.HIGH, + requestContext + ); + futures.add(resultFuture); + } + + Thread t = new Thread(() -> { + Supplier> supplier = () -> SocketAccess.doPrivileged( + () -> CompletableFuture.supplyAsync(() -> "success", testExecutor) + ); + + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.HIGH, + requestContext + ); + futures.add(resultFuture); + }); + t.start(); + t.join(); + + CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); + assertTrue(permitBasedRetryableFutureUtils.waitedForPermit.get()); + assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + } + + public void testHighPermitsConsumedForLowTasks() throws ExecutionException, InterruptedException { + CountDownLatch delayedLatch = new CountDownLatch(1); + TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( + 3, + Runtime.getRuntime().availableProcessors(), + 0.7, + delayedLatch::countDown, + null, + testExecutor, + scheduler + ); + int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); + int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; + + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); + List> futures = new ArrayList<>(); + for (int reqIdx = 0; reqIdx < lowPermits; reqIdx++) { + Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { + try { + // Keep the permit acquired + delayedLatch.await(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return "success"; + }, testExecutor)); + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.LOW, + requestContext + ); + futures.add(resultFuture); + } + + Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { + try { + // Keep the permit acquired + delayedLatch.await(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return "success"; + }, testExecutor)); + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.LOW, + requestContext + ); + futures.add(resultFuture); + delayedLatch.countDown(); + + CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); + assertTrue(permitBasedRetryableFutureUtils.highPermitsConsumedForLowPriority.get()); + assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + } + + public void testFutureRetryOnSemaphoreFailure() throws ExecutionException, InterruptedException { + int retryCount = 3; + // Multiply by 3 as there are 3 ways in which permit can be acquired. + AtomicInteger failureCount = new AtomicInteger((retryCount - 1) * 3); + AtomicBoolean exhaustRetries = new AtomicBoolean(); + TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( + retryCount, + Runtime.getRuntime().availableProcessors(), + 0.7, + null, + () -> { + if (failureCount.get() > 0 || exhaustRetries.get()) { + failureCount.decrementAndGet(); + return true; + } + return false; + }, + testExecutor, + scheduler + ); + int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); + int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; + + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); + List> futures = new ArrayList<>(); + Supplier> supplier = () -> SocketAccess.doPrivileged( + () -> CompletableFuture.supplyAsync(() -> "success", testExecutor) + ); + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.HIGH, + requestContext + ); + futures.add(resultFuture); + + CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); + + assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + // Reached here so future executed successfully after retries. + + exhaustRetries.set(true); + resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture(supplier, WritePriority.HIGH, requestContext); + resultFuture.whenComplete((r, t) -> { + assertNotNull(t); + assertTrue(t instanceof PermitBackedRetryableFutureUtils.RetryableException); + }); + CompletableFuture finalResultFuture = resultFuture; + assertThrows(Exception.class, finalResultFuture::get); + } + + public void testFutureRetryOnExecFailure() throws ExecutionException, InterruptedException { + int retryCount = 3; + AtomicInteger failureCount = new AtomicInteger(retryCount); + AtomicBoolean exhaustRetries = new AtomicBoolean(); + TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( + retryCount, + Runtime.getRuntime().availableProcessors(), + 0.7, + null, + null, + testExecutor, + scheduler + ); + int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); + int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; + + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); + List> futures = new ArrayList<>(); + Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { + if (failureCount.get() > 0 || exhaustRetries.get()) { + failureCount.decrementAndGet(); + throw ApiCallAttemptTimeoutException.builder().build(); + } + return "success"; + }, testExecutor)); + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.HIGH, + requestContext + ); + futures.add(resultFuture); + + CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); + + assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + // Reached here so future executed successfully after retries. + + exhaustRetries.set(true); + resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture(supplier, WritePriority.HIGH, requestContext); + resultFuture.whenComplete((r, t) -> { + assertNotNull(t); + assertTrue(t instanceof PermitBackedRetryableFutureUtils.RetryableException); + }); + CompletableFuture finalResultFuture = resultFuture; + assertThrows(Exception.class, finalResultFuture::get); + + assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + } + + public void testNonRetryableFailure() throws ExecutionException, InterruptedException { + // Throw only once to ensure no retries for unknown exception. + AtomicInteger failureCount = new AtomicInteger(1); + AtomicBoolean exhaustRetries = new AtomicBoolean(); + TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( + 3, + Runtime.getRuntime().availableProcessors(), + 0.7, + null, + null, + testExecutor, + scheduler + ); + int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); + int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; + + PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); + Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { + if (failureCount.get() > 0) { + failureCount.decrementAndGet(); + throw new RuntimeException("Generic exception"); + } + return "success"; + }, testExecutor)); + + CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( + supplier, + WritePriority.HIGH, + requestContext + ); + resultFuture.whenComplete((r, t) -> { + assertNotNull(t); + assertFalse(t instanceof PermitBackedRetryableFutureUtils.RetryableException); + }); + assertThrows(Exception.class, resultFuture::get); + + assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); + assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); + } +} 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..9888612b444bc 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 @@ -16,5 +16,6 @@ public enum WritePriority { NORMAL, HIGH, - URGENT + URGENT, + 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..4fef8c6179c8e 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -323,11 +323,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 +343,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; @@ -358,7 +360,7 @@ private void uploadBlob( 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 a613eb52b4910543eef7bee0be6d1758d438056c Mon Sep 17 00:00:00 2001 From: vikasvb90 Date: Sun, 3 Mar 2024 18:59:14 +0530 Subject: [PATCH 2/5] Producer-consumer queue to track in-queue transfer events Signed-off-by: vikasvb90 --- .../s3/S3BlobStoreRepositoryTests.java | 16 +- .../repositories/s3/S3BlobContainer.java | 57 +- .../repositories/s3/S3BlobStore.java | 17 +- .../repositories/s3/S3Repository.java | 49 +- .../repositories/s3/S3RepositoryPlugin.java | 97 +++- .../s3/S3TransferRejectedException.java | 20 + .../s3/async/AsyncPartsHandler.java | 115 ++-- .../s3/async/AsyncTransferManager.java | 103 ++-- .../PermitBackedRetryableFutureUtils.java | 344 ------------ .../s3/async/SizeBasedBlockingQ.java | 190 +++++++ .../s3/async/TransferSemaphoresHolder.java | 143 +++++ .../repositories/s3/async/UploadRequest.java | 2 +- .../s3/RepositoryCredentialsTests.java | 16 +- .../s3/S3BlobContainerMockClientTests.java | 62 ++- .../s3/S3BlobContainerRetriesTests.java | 35 +- .../repositories/s3/S3RepositoryTests.java | 5 +- .../s3/async/AsyncTransferManagerTests.java | 9 +- ...PermitBackedRetryableFutureUtilsTests.java | 505 ------------------ .../s3/async/SizeBasedBlockingQTests.java | 94 ++++ .../async/TransferSemaphoresHolderTests.java | 253 +++++++++ .../index/store/RemoteDirectory.java | 2 + 21 files changed, 1132 insertions(+), 1002 deletions(-) create mode 100644 plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3TransferRejectedException.java delete mode 100644 plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtils.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 delete mode 100644 plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtilsTests.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..b001ac878f5df 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,21 @@ 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 + ) { @Override public BlobStore blobStore() { 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..1b9b022b09847 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,13 @@ 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().canProduce(uploadRequest.getContentLength()) == false) + || (uploadRequest.getWritePriority() != WritePriority.HIGH + && uploadRequest.getWritePriority() != WritePriority.URGENT + && blobStore.getOtherPrioritySizeBasedBlockingQ().canProduce(uploadRequest.getContentLength()) == false)) { StreamContext streamContext = SocketAccess.doPrivileged( () -> writeContext.getStreamProvider(uploadRequest.getContentLength()) ); @@ -258,16 +265,26 @@ 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) { + 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 { + blobStore.getOtherPrioritySizeBasedBlockingQ() + .produce( + new SizeBasedBlockingQ.Item( + writeContext.getFileSize(), + () -> createFileCompletableFuture(s3AsyncClient, uploadRequest, streamContext, completionListener) + ) + ); + } } } catch (Exception e) { logger.info("exception error from blob container for file {}", writeContext.getFileName()); @@ -275,6 +292,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..e1a8fff87bc3e 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; @@ -94,6 +95,8 @@ class S3BlobStore implements BlobStore { private final AsyncExecutorContainer priorityExecutorBuilder; private final AsyncExecutorContainer normalExecutorBuilder; private final boolean multipartUploadEnabled; + private final SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; S3BlobStore( S3Service service, @@ -109,7 +112,9 @@ class S3BlobStore implements BlobStore { AsyncTransferManager asyncTransferManager, AsyncExecutorContainer urgentExecutorBuilder, AsyncExecutorContainer priorityExecutorBuilder, - AsyncExecutorContainer normalExecutorBuilder + AsyncExecutorContainer normalExecutorBuilder, + SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ, + SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ ) { this.service = service; this.s3AsyncService = s3AsyncService; @@ -128,6 +133,8 @@ 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.otherPrioritySizeBasedBlockingQ = otherPrioritySizeBasedBlockingQ; + this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; } @Override @@ -184,6 +191,14 @@ public int getBulkDeletesSize() { return bulkDeletesSize; } + public SizeBasedBlockingQ getOtherPrioritySizeBasedBlockingQ() { + return otherPrioritySizeBasedBlockingQ; + } + + public SizeBasedBlockingQ getLowPrioritySizeBasedBlockingQ() { + return lowPrioritySizeBasedBlockingQ; + } + @Override public BlobContainer blobContainer(BlobPath path) { return new S3BlobContainer(path, this); 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 f8481e3a9402c..3306062c653cd 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; @@ -193,12 +195,6 @@ class S3Repository extends MeteredBlobStoreRepository { true, Setting.Property.NodeScope ); - - /** - * Number of retries in case of a transfer failure. - */ - public static Setting S3_MAX_TRANSFER_RETRIES = Setting.intSetting("s3_max_transfer_retries", 3, Setting.Property.NodeScope); - /** * Percentage of total available permits to be available for priority transfers. */ @@ -210,6 +206,27 @@ class S3Repository extends MeteredBlobStoreRepository { 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. */ @@ -268,6 +285,8 @@ class S3Repository extends MeteredBlobStoreRepository { private final AsyncExecutorContainer priorityExecutorBuilder; private final AsyncExecutorContainer normalExecutorBuilder; private final Path pluginConfigPath; + private final SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; private volatile int bulkDeletesSize; @@ -283,7 +302,9 @@ class S3Repository extends MeteredBlobStoreRepository { final AsyncExecutorContainer priorityExecutorBuilder, final AsyncExecutorContainer normalExecutorBuilder, final S3AsyncService s3AsyncService, - final boolean multipartUploadEnabled + final boolean multipartUploadEnabled, + final SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ, + final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ ) { this( metadata, @@ -297,7 +318,9 @@ class S3Repository extends MeteredBlobStoreRepository { normalExecutorBuilder, s3AsyncService, multipartUploadEnabled, - Path.of("") + Path.of(""), + otherPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ ); } @@ -316,7 +339,9 @@ class S3Repository extends MeteredBlobStoreRepository { final AsyncExecutorContainer normalExecutorBuilder, final S3AsyncService s3AsyncService, final boolean multipartUploadEnabled, - Path pluginConfigPath + Path pluginConfigPath, + final SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ, + final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ ) { super(metadata, namedXContentRegistry, clusterService, recoverySettings, buildLocation(metadata)); this.service = service; @@ -327,6 +352,8 @@ class S3Repository extends MeteredBlobStoreRepository { this.urgentExecutorBuilder = urgentExecutorBuilder; this.priorityExecutorBuilder = priorityExecutorBuilder; this.normalExecutorBuilder = normalExecutorBuilder; + this.otherPrioritySizeBasedBlockingQ = otherPrioritySizeBasedBlockingQ; + this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; validateRepositoryMetadata(metadata); readRepositoryMetadata(); @@ -389,7 +416,9 @@ protected S3BlobStore createBlobStore() { asyncUploadUtils, urgentExecutorBuilder, priorityExecutorBuilder, - normalExecutorBuilder + normalExecutorBuilder, + otherPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ ); } 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 63872f0b98cd8..c4acb9ed67e22 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,7 +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.PermitBackedRetryableFutureUtils; +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; @@ -71,7 +75,7 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.function.Supplier; /** @@ -84,8 +88,9 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private static final String PRIORITY_FUTURE_COMPLETION = "priority_future_completion"; private static final String PRIORITY_STREAM_READER = "priority_stream_reader"; private static final String FUTURE_COMPLETION = "future_completion"; - private static final String REMOTE_TRANSFER_RETRY = "remote_transfer_retry"; private static final String STREAM_READER = "stream_reader"; + private static final String LOW_TRANSFER_QUEUE_CONSUMER = "low_transfer_queue_consumer"; + private static final String OTHER_TRANSFER_QUEUE_CONSUMER = "other_transfer_queue_consumer"; protected final S3Service service; private final S3AsyncService s3AsyncService; @@ -95,8 +100,11 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private AsyncExecutorContainer urgentExecutorBuilder; private AsyncExecutorContainer priorityExecutorBuilder; private AsyncExecutorContainer normalExecutorBuilder; - private ExecutorService remoteTransferRetryPool; - private ScheduledExecutorService scheduler; + private ExecutorService lowTransferQConsumerService; + private ExecutorService otherTransferQConsumerService; + private SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; + private TransferSemaphoresHolder transferSemaphoresHolder; public S3RepositoryPlugin(final Settings settings, final Path configPath) { this(settings, configPath, new S3Service(configPath), new S3AsyncService(configPath)); @@ -127,16 +135,35 @@ public List> getExecutorBuilders(Settings settings) { ) ); executorBuilders.add( - new ScalingExecutorBuilder( - REMOTE_TRANSFER_RETRY, - allocatedProcessors(settings), - allocatedProcessors(settings) * 2, - TimeValue.timeValueMinutes(5) + new FixedExecutorBuilder( + settings, + LOW_TRANSFER_QUEUE_CONSUMER, + lowPriorityTransferQConsumers(settings), + 10, + "thread_pool." + LOW_TRANSFER_QUEUE_CONSUMER + ) + ); + executorBuilders.add( + new FixedExecutorBuilder( + settings, + OTHER_TRANSFER_QUEUE_CONSUMER, + otherPriorityTransferQConsumers(settings), + 10, + "thread_pool." + OTHER_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 otherPriorityTransferQConsumers(Settings settings) { + return S3Repository.S3_TRANSFER_QUEUE_CONSUMERS.get(settings); + } + static int halfNumberOfProcessors(int numberOfProcessors) { return (numberOfProcessors + 1) / 2; } @@ -203,9 +230,38 @@ public Collection createComponents( threadPool.executor(STREAM_READER), new AsyncTransferEventLoopGroup(normalEventLoopThreads) ); - this.remoteTransferRetryPool = threadPool.executor(REMOTE_TRANSFER_RETRY); - this.scheduler = threadPool.scheduler(); - return Collections.emptyList(); + this.lowTransferQConsumerService = threadPool.executor(LOW_TRANSFER_QUEUE_CONSUMER); + this.otherTransferQConsumerService = threadPool.executor(OTHER_TRANSFER_QUEUE_CONSUMER); + int otherPriorityConsumers = otherPriorityTransferQConsumers(clusterService.getSettings()); + this.otherPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(otherPriorityConsumers * 10L, ByteSizeUnit.GB), + otherTransferQConsumerService, + otherPriorityConsumers + ); + int lowPriorityConsumers = lowPriorityTransferQConsumers(clusterService.getSettings()); + LowPrioritySizeBasedBlockingQ lowPrioritySizeBasedBlockingQ = new LowPrioritySizeBasedBlockingQ( + new ByteSizeValue(lowPriorityConsumers * 20L, ByteSizeUnit.GB), + lowTransferQConsumerService, + lowPriorityConsumers + ); + this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; + this.transferSemaphoresHolder = new TransferSemaphoresHolder( + // High number of permit allocation because each op acquiring permit performs disk IO, computation and network IO. + Math.max(allocatedProcessors(clusterService.getSettings()) * 4, 10), + ((double) S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT.get(clusterService.getSettings())) / 100, + S3Repository.S3_PERMIT_WAIT_DURATION_MIN.get(clusterService.getSettings()), + TimeUnit.MINUTES + ); + + return CollectionUtils.arrayAsArrayList(this.otherPrioritySizeBasedBlockingQ, 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) { + super(capacity, executorService, consumers); + } } // proxy method for testing @@ -221,15 +277,7 @@ protected S3Repository createRepository( normalExecutorBuilder.getStreamReader(), priorityExecutorBuilder.getStreamReader(), urgentExecutorBuilder.getStreamReader(), - new PermitBackedRetryableFutureUtils<>( - S3Repository.S3_MAX_TRANSFER_RETRIES.get(clusterService.getSettings()), - // High permit allocation because each op acquiring permit performs disk IO, computation and network IO. - Math.max(allocatedProcessors(clusterService.getSettings()) * 5, 10), - ((double) S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT.get(clusterService.getSettings())) / 100, - remoteTransferRetryPool, - scheduler - ) - + transferSemaphoresHolder ); return new S3Repository( metadata, @@ -243,7 +291,9 @@ protected S3Repository createRepository( normalExecutorBuilder, s3AsyncService, S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING.get(clusterService.getSettings()), - configPath + configPath, + otherPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ ); } @@ -289,7 +339,6 @@ public List> getSettings() { S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING, S3Repository.REDIRECT_LARGE_S3_UPLOAD, S3Repository.UPLOAD_RETRY_ENABLED, - S3Repository.S3_MAX_TRANSFER_RETRIES, S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT ); } 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 f3955c9d69186..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,15 +35,16 @@ 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; -import java.util.function.Supplier; /** * Responsible for handling parts of the original multipart request */ 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* @@ -56,6 +58,7 @@ 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 */ @SuppressWarnings({ "rawtypes", "unchecked" }) @@ -71,23 +74,24 @@ public static List> uploadParts( AtomicReferenceArray inputStreamContainers, StatsMetricPublisher statsMetricPublisher, boolean uploadRetryEnabled, - PermitBackedRetryableFutureUtils permitBackedRetryableFutureUtils + TransferSemaphoresHolder transferSemaphoresHolder, + long maxRetryablePartSize ) throws InterruptedException { List> futures = new ArrayList<>(); - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBackedRetryableFutureUtils.createRequestContext(); + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); for (int partIdx = 0; partIdx < streamContext.getNumberOfParts(); partIdx++) { - int finalPartIdx = partIdx; - Supplier> partFutureSupplier = () -> { - InputStreamContainer inputStreamContainer; - try { - inputStreamContainer = streamContext.provideStream(finalPartIdx); - } catch (IOException e) { - return CompletableFuture.failedFuture(e); - } - inputStreamContainers.set(finalPartIdx, new CheckedContainer(inputStreamContainer.getContentLength())); + 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(finalPartIdx + 1) + .partNumber(partIdx + 1) .key(uploadRequest.getKey()) .uploadId(uploadId) .overrideConfiguration(o -> o.addMetricPublisher(statsMetricPublisher.multipartUploadMetricCollector)) @@ -95,26 +99,26 @@ public static List> uploadParts( if (uploadRequest.doRemoteDataIntegrityCheck()) { uploadPartRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32); } - return uploadPart( + uploadPart( s3AsyncClient, executorService, priorityExecutorService, urgentExecutorService, completedParts, inputStreamContainers, + futures, uploadPartRequestBuilder.build(), inputStreamContainer, uploadRequest, - uploadRetryEnabled + uploadRetryEnabled, + maxRetryablePartSize, + semaphore ); - }; - - CompletableFuture retryableFuture = permitBackedRetryableFutureUtils.createPermitBackedRetryableFuture( - partFutureSupplier, - uploadRequest.getWritePriority(), - requestContext - ); - futures.add(retryableFuture); + } catch (Exception ex) { + if (semaphore != null) { + semaphore.release(); + } + } } return futures; @@ -154,26 +158,70 @@ 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; } - private static CompletableFuture uploadPart( + 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, ExecutorService priorityExecutorService, ExecutorService urgentExecutorService, AtomicReferenceArray completedParts, AtomicReferenceArray inputStreamContainers, + List> futures, UploadPartRequest uploadPartRequest, InputStreamContainer inputStreamContainer, UploadRequest uploadRequest, - boolean uploadRetryEnabled + boolean uploadRetryEnabled, + long maxRetryablePartSize, + Semaphore semaphore ) { + Integer partNumber = uploadPartRequest.partNumber(); ExecutorService streamReadExecutor; @@ -189,7 +237,8 @@ private static CompletableFuture uploadPart( inputStreamContainer.getInputStream(), uploadRequest.getWritePriority(), uploadRetryEnabled, - uploadPartRequest.contentLength() + uploadPartRequest.contentLength(), + maxRetryablePartSize ); CompletableFuture uploadPartResponseFuture = SocketAccess.doPrivileged( () -> s3AsyncClient.uploadPart( @@ -199,6 +248,10 @@ private static CompletableFuture uploadPart( ); CompletableFuture convertFuture = uploadPartResponseFuture.whenComplete((resp, throwable) -> { + if (semaphore != null) { + semaphore.release(); + } + try { inputStream.close(); } catch (IOException ex) { @@ -221,9 +274,9 @@ private static CompletableFuture uploadPart( uploadRequest.doRemoteDataIntegrityCheck() ) ); + futures.add(convertFuture); CompletableFutureUtils.forwardExceptionTo(convertFuture, uploadPartResponseFuture); - return convertFuture; } private static CompletedPart convertUploadPartResponse( 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 ad9ab884b0fe0..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,9 +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 PermitBackedRetryableFutureUtils permitBackedRetryableFutureUtils; + private final TransferSemaphoresHolder transferSemaphoresHolder; /** * The max number of parts on S3 side is 10,000 @@ -84,13 +87,15 @@ public AsyncTransferManager( ExecutorService executorService, ExecutorService priorityExecutorService, ExecutorService urgentExecutorService, - PermitBackedRetryableFutureUtils permitBackedRetryableFutureUtils + 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.permitBackedRetryableFutureUtils = permitBackedRetryableFutureUtils; + this.transferSemaphoresHolder = transferSemaphoresHolder; } /** @@ -112,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, 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); @@ -192,7 +211,8 @@ private void doUploadInParts( inputStreamContainers, statsMetricPublisher, uploadRequest.isUploadRetryEnabled(), - permitBackedRetryableFutureUtils + transferSemaphoresHolder, + maxRetryablePartSize ); } catch (Exception ex) { try { @@ -329,8 +349,9 @@ private void uploadInOneChunk( UploadRequest uploadRequest, StreamContext streamContext, CompletableFuture returnFuture, - StatsMetricPublisher statsMetricPublisher - ) throws InterruptedException { + StatsMetricPublisher statsMetricPublisher, + Semaphore semaphore + ) { PutObjectRequest.Builder putObjectRequestBuilder = PutObjectRequest.builder() .bucket(uploadRequest.getBucket()) .key(uploadRequest.getKey()) @@ -344,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; @@ -353,31 +375,33 @@ private void uploadInOneChunk( streamReadExecutor = executorService; } - Supplier> putObjectFutureSupplier = () -> SocketAccess.doPrivileged(() -> { - InputStreamContainer inputStreamContainer; + CompletableFuture putObjectFuture = SocketAccess.doPrivileged(() -> { + InputStream inputStream = null; + CompletableFuture putObjectRespFuture; try { - inputStreamContainer = streamContext.provideStream(0); - } catch (IOException e) { + 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 inputStream = AsyncPartsHandler.maybeRetryInputStream( - inputStreamContainer.getInputStream(), - uploadRequest.getWritePriority(), - uploadRequest.isUploadRetryEnabled(), - uploadRequest.getContentLength() - ); - return 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 - ); - } + + 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) { @@ -408,17 +432,24 @@ private void uploadInOneChunk( }); }); - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBackedRetryableFutureUtils.createRequestContext(); - CompletableFuture putObjectFuture = permitBackedRetryableFutureUtils.createPermitBackedRetryableFuture( - putObjectFutureSupplier, - uploadRequest.getWritePriority(), - requestContext - ); - 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/PermitBackedRetryableFutureUtils.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtils.java deleted file mode 100644 index 25a45fb28c23f..0000000000000 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtils.java +++ /dev/null @@ -1,344 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.repositories.s3.async; - -import software.amazon.awssdk.core.exception.SdkException; - -import org.opensearch.ExceptionsHelper; -import org.opensearch.action.bulk.BackoffPolicy; -import org.opensearch.common.blobstore.stream.write.WritePriority; -import org.opensearch.common.unit.TimeValue; - -import java.util.Iterator; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; -import java.util.function.Supplier; - -/** - * Async wrapper over a completable future backed by transfer permits which provides natural backpressure in case - * of transfer bursts. Additionally, it retries futures (with exp backoff) which fail due to S3 exception or - * when permit couldn't be acquired within timeout period. - * - * @param Type of future response - */ -public class PermitBackedRetryableFutureUtils { - - // Package access for testing. - Semaphore lowPrioritySemaphore; - Semaphore highPrioritySemaphore; - private final int lowPriorityPermits; - private final int highPriorityPermits; - private final int maxRetryAttempts; - private static final int RETRY_BASE_INTERVAL_MILLIS = 1_000; - private final AtomicBoolean lowPriorityTransferProgress; - - private final ExecutorService remoteTransferRetryPool; - private final ScheduledExecutorService scheduler; - - /** - * - * @param maxRetryAttempts max number of retries - * @param availablePermits Total available permits for transfer - * @param priorityPermitAllocation Allocation bandwidth of priority permits. Rest will be allocated to low - * priority permits. - */ - public PermitBackedRetryableFutureUtils( - int maxRetryAttempts, - int availablePermits, - double priorityPermitAllocation, - ExecutorService remoteTransferRetryPool, - ScheduledExecutorService scheduler - ) { - this.maxRetryAttempts = maxRetryAttempts; - this.highPriorityPermits = (int) (priorityPermitAllocation * availablePermits); - this.highPrioritySemaphore = new TypeSemaphore(highPriorityPermits, "high"); - this.lowPriorityPermits = availablePermits - highPriorityPermits; - this.lowPrioritySemaphore = new TypeSemaphore(lowPriorityPermits, "low"); - this.lowPriorityTransferProgress = new AtomicBoolean(); - this.remoteTransferRetryPool = remoteTransferRetryPool; - this.scheduler = scheduler; - } - - /** - * Available low priority permits - * @return available low priority permits - */ - public int getAvailableLowPriorityPermits() { - return lowPrioritySemaphore.availablePermits(); - } - - /** - * Available high priority permits - * @return available high priority permits - */ - public int getAvailableHighPriorityPermits() { - return highPrioritySemaphore.availablePermits(); - } - - /** - * Named semaphore for debugging purpose - */ - static class TypeSemaphore extends Semaphore { - private final String type; - - public TypeSemaphore(int permits, String type) { - super(permits); - this.type = type; - } - - @Override - public String toString() { - String toStr = super.toString(); - return toStr + " , type = " + type; - } - - public String getType() { - return type; - } - - } - - /** - * 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); - } - - /** - * Custom exception to distinguish retryable futures. - */ - static class RetryableException extends CompletionException { - private final Iterator retryBackoffDelayIterator; - - public RetryableException(Iterator retryBackoffDelayIterator, String message, Throwable cause) { - super(message, cause); - this.retryBackoffDelayIterator = retryBackoffDelayIterator; - } - - public RetryableException(Iterator retryBackoffDelayIterator) { - this.retryBackoffDelayIterator = retryBackoffDelayIterator; - } - } - - /** - * DelayedExecutor and TaskSubmitter are copied from CompletableFuture. Duplicate classes are needed because - * scheduler used by these cannot be overriden and we need a way to manage it from outside. - */ - private static final class DelayedExecutor implements Executor { - private final long delay; - private final TimeUnit unit; - private final Executor executor; - private final ScheduledExecutorService scheduler; - - DelayedExecutor(long delay, TimeUnit unit, Executor executor, ScheduledExecutorService scheduler) { - this.delay = delay; - this.unit = unit; - this.executor = executor; - this.scheduler = scheduler; - } - - public void execute(Runnable r) { - scheduler.schedule(new TaskSubmitter(executor, r), delay, unit); - } - } - - private static final class TaskSubmitter implements Runnable { - final Executor executor; - final Runnable action; - - TaskSubmitter(Executor executor, Runnable action) { - this.executor = executor; - this.action = action; - } - - public void run() { - executor.execute(action); - } - } - - /** - * - * @param futureSupplier Supplier of the completable future - * @param writePriority Priority of transfer - * @param requestContext Request context object to set the decisions pertaining to transfer before transfers are - * initiated. - * - * @return completable future backed by permits and retryable future. - */ - public CompletableFuture createPermitBackedRetryableFuture( - Supplier> futureSupplier, - WritePriority writePriority, - RequestContext requestContext - ) { - Iterator retryBackoffDelayIterator = BackoffPolicy.exponentialBackoff( - TimeValue.timeValueMillis(RETRY_BASE_INTERVAL_MILLIS), - maxRetryAttempts - ).iterator(); - Supplier> permitBackedFutureSupplier = createPermitBackedFutureSupplier( - retryBackoffDelayIterator, - requestContext.lowPriorityPermitsConsumable, - futureSupplier, - writePriority - ); - - CompletableFuture permitBackedFuture; - try { - permitBackedFuture = permitBackedFutureSupplier.get(); - } catch (RetryableException re) { - // We need special handling when an exception occurs during first future creation itself. - permitBackedFuture = retry(re, permitBackedFutureSupplier, retryBackoffDelayIterator); - } catch (Exception ex) { - return CompletableFuture.failedFuture(ex); - } - - return flatten( - permitBackedFuture.thenApply(CompletableFuture::completedFuture) - .exceptionally(t -> retry(t, permitBackedFutureSupplier, retryBackoffDelayIterator)) - ); - } - - private static CompletableFuture flatten( - CompletableFuture> completableCompletable - ) { - return completableCompletable.thenCompose(Function.identity()); - } - - private CompletableFuture retry( - Throwable ex, - Supplier> futureSupplier, - Iterator retryBackoffDelayIterator - ) { - if (!(ex instanceof RetryableException)) { - return CompletableFuture.failedFuture(ex); - } - - RetryableException retryableException = (RetryableException) ex; - if (!retryBackoffDelayIterator.hasNext()) { - return CompletableFuture.failedFuture(ex); - } - - return flatten( - flatten( - CompletableFuture.supplyAsync( - futureSupplier, - new DelayedExecutor( - retryableException.retryBackoffDelayIterator.next().millis(), - TimeUnit.MILLISECONDS, - remoteTransferRetryPool, - scheduler - ) - ) - ).thenApply(CompletableFuture::completedFuture).exceptionally(t -> { - if (t instanceof RetryableException) { - ex.addSuppressed(t); - return retry(ex, futureSupplier, retryBackoffDelayIterator); - } else { - ex.addSuppressed(t); - return CompletableFuture.failedFuture(ex); - } - }) - ); - } - - // Package access for testing - Semaphore acquirePermit(WritePriority writePriority, boolean isLowPriorityPermitsConsumable) throws InterruptedException { - // 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 (highPrioritySemaphore.availablePermits() > 0.4 * highPriorityPermits && highPrioritySemaphore.tryAcquire()) { - return highPrioritySemaphore; - } else if (lowPrioritySemaphore.tryAcquire(5, TimeUnit.MINUTES)) { - return lowPrioritySemaphore; - } - return null; - } - - // Try acquiring high priority permit or low priority permit immediately if available. - // Otherwise, we wait for high priority permit. - if (highPrioritySemaphore.tryAcquire()) { - return highPrioritySemaphore; - } else if (isLowPriorityPermitsConsumable && lowPrioritySemaphore.tryAcquire()) { - return lowPrioritySemaphore; - } else if (highPrioritySemaphore.tryAcquire(5, TimeUnit.MINUTES)) { - return highPrioritySemaphore; - } - return null; - } - - private Supplier> createPermitBackedFutureSupplier( - Iterator retryBackoffDelayIterator, - boolean lowPriorityPermitsConsumable, - Supplier> futureSupplier, - WritePriority writePriority - ) { - return () -> { - Semaphore semaphore; - try { - semaphore = acquirePermit(writePriority, lowPriorityPermitsConsumable); - if (semaphore == null) { - throw new RetryableException(retryBackoffDelayIterator); - } - } catch (InterruptedException e) { - throw new CompletionException(e); - } - - CompletableFuture future; - try { - future = futureSupplier.get(); - } catch (Exception ex) { - // Exception in future creation. Can't retry this. - semaphore.release(); - throw new RuntimeException(ex); - } - - return future.handle((resp, t) -> { - try { - if (t != null) { - Throwable ex = ExceptionsHelper.unwrap(t, SdkException.class); - if (ex != null) { - throw new RetryableException(retryBackoffDelayIterator, t.getMessage(), t); - } - throw new CompletionException(t); - } - return resp; - } finally { - semaphore.release(); - } - }); - }; - } - -} 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..289c92c14a8b9 --- /dev/null +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQ.java @@ -0,0 +1,190 @@ +/* + * 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.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; + + /** + * Constructor to create sized based blocking queue. + */ + public SizeBasedBlockingQ(ByteSizeValue capacity, ExecutorService executorService, int consumers) { + 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; + } + + @Override + protected void doStart() { + for (int worker = 0; worker < consumers; worker++) { + Thread consumer = new Consumer(queue, currentSize, lock, notEmpty, closed); + 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."); + } + + 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(); + } finally { + lock.unlock(); + } + } + + public int getSize() { + return queue.size(); + } + + public boolean canProduce(long contentLength) { + return (currentSize.get() + 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; + + public Consumer(LinkedBlockingQueue queue, AtomicLong currentSize, Lock lock, Condition notEmpty, AtomicBoolean closed) { + this.queue = queue; + this.lock = lock; + this.notEmpty = notEmpty; + this.currentSize = currentSize; + this.closed = closed; + } + + @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); + } 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..aa3b6a78e2c4a --- /dev/null +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolder.java @@ -0,0 +1,143 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.repositories.s3.async; + +import org.opensearch.common.blobstore.stream.write.WritePriority; + +import java.util.Objects; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +/** + * Transfer semaphore holder for controlled transfer of data to remote. + */ +public class TransferSemaphoresHolder { + + // For tests + protected TypeSemaphore lowPrioritySemaphore; + protected TypeSemaphore highPrioritySemaphore; + private final int highPriorityPermits; + private final int lowPriorityPermits; + private final int acquireWaitDuration; + private final TimeUnit acquireWaitDurationUnit; + + /** + * Constructor to create semaphores holder. + */ + public TransferSemaphoresHolder(int availablePermits, double priorityPermitAllocation, int acquireWaitDuration, TimeUnit timeUnit) { + + this.highPriorityPermits = (int) (priorityPermitAllocation * availablePermits); + this.highPrioritySemaphore = new TypeSemaphore(highPriorityPermits, "high"); + this.lowPriorityPermits = availablePermits - highPriorityPermits; + this.lowPrioritySemaphore = new TypeSemaphore(lowPriorityPermits, "low"); + 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 String type; + + public TypeSemaphore(int permits, String type) { + super(permits); + this.type = type; + } + + public String getType() { + return type; + } + } + + /** + * 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 { + // 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 (highPrioritySemaphore.availablePermits() > 0.4 * highPriorityPermits && highPrioritySemaphore.tryAcquire()) { + return highPrioritySemaphore; + } 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 (highPrioritySemaphore.tryAcquire()) { + return highPrioritySemaphore; + } else if (requestContext.lowPriorityPermitsConsumable && lowPrioritySemaphore.tryAcquire()) { + return lowPrioritySemaphore; + } else if (highPrioritySemaphore.tryAcquire(acquireWaitDuration, acquireWaitDurationUnit)) { + return highPrioritySemaphore; + } + return null; + } + + /** + * Used in tests. + */ + public int getHighPriorityPermits() { + return highPriorityPermits; + } + + /** + * Used in tests. + */ + public int getLowPriorityPermits() { + return lowPriorityPermits; + } + + /** + * Used in tests. + */ + public int getAvailableHighPriorityPermits() { + return highPrioritySemaphore.availablePermits(); + } + + /** + * Used in tests. + */ + public int getAvailableLowPriorityPermits() { + return lowPrioritySemaphore.availablePermits(); + } +} 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..16dbe05dcaee8 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,21 @@ 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 + ) { @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 fee00fd9e6ec7..4ec1baf634f52 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,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.PermitBackedRetryableFutureUtils; +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; @@ -95,9 +96,12 @@ public class S3BlobContainerMockClientTests extends OpenSearchTestCase implement private ExecutorService futureCompletionService; private ExecutorService streamReaderService; private ExecutorService remoteTransferRetry; + private ExecutorService transferQueueConsumerService; private ScheduledExecutorService scheduler; private AsyncTransferEventLoopGroup transferNIOGroup; private S3BlobContainer blobContainer; + private SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; static class MockS3AsyncService extends S3AsyncService { @@ -370,8 +374,21 @@ public void setUp() throws Exception { futureCompletionService = Executors.newSingleThreadExecutor(); streamReaderService = Executors.newSingleThreadExecutor(); remoteTransferRetry = Executors.newFixedThreadPool(20); + transferQueueConsumerService = Executors.newFixedThreadPool(20); scheduler = new Scheduler.SafeScheduledThreadPoolExecutor(1); transferNIOGroup = new AsyncTransferEventLoopGroup(1); + otherPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 10L, ByteSizeUnit.GB), + transferQueueConsumerService, + 10 + ); + lowPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 20L, ByteSizeUnit.GB), + transferQueueConsumerService, + 5 + ); + otherPrioritySizeBasedBlockingQ.start(); + lowPrioritySizeBasedBlockingQ.start(); blobContainer = createBlobContainer(); super.setUp(); } @@ -383,6 +400,9 @@ public void tearDown() throws Exception { futureCompletionService.shutdown(); streamReaderService.shutdown(); remoteTransferRetry.shutdown(); + transferQueueConsumerService.shutdown(); + otherPrioritySizeBasedBlockingQ.close(); + lowPrioritySizeBasedBlockingQ.close(); scheduler.shutdown(); transferNIOGroup.close(); super.tearDown(); @@ -423,17 +443,13 @@ private S3BlobStore createBlobStore() { asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), - new PermitBackedRetryableFutureUtils<>( - 3, - Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), - 0.7, - remoteTransferRetry, - scheduler - ) + new TransferSemaphoresHolder(3, Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES) ), asyncExecutorContainer, asyncExecutorContainer, - asyncExecutorContainer + asyncExecutorContainer, + otherPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ ); } @@ -593,19 +609,25 @@ 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); + + SizeBasedBlockingQ sizeBasedBlockingQ = new SizeBasedBlockingQ(capacity, transferQueueConsumerService, 10); + + 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<>(); @@ -628,6 +650,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.getOtherPrioritySizeBasedBlockingQ()).thenReturn(sizeBasedBlockingQ); + final boolean serverSideEncryption = randomBoolean(); when(blobStore.serverSideEncryption()).thenReturn(serverSideEncryption); @@ -677,7 +702,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<>()) @@ -712,5 +737,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 14691c89c202a..95d807efa92cc 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,7 +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.PermitBackedRetryableFutureUtils; +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; @@ -118,8 +119,11 @@ public class S3BlobContainerRetriesTests extends AbstractBlobContainerRetriesTes private ExecutorService futureCompletionService; private ExecutorService streamReaderService; private ExecutorService remoteTransferRetry; + private ExecutorService transferQueueConsumerService; private ScheduledExecutorService scheduler; private AsyncTransferEventLoopGroup transferNIOGroup; + private SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; @Before public void setUp() throws Exception { @@ -131,8 +135,20 @@ public void setUp() throws Exception { streamReaderService = Executors.newSingleThreadExecutor(); transferNIOGroup = new AsyncTransferEventLoopGroup(1); remoteTransferRetry = Executors.newFixedThreadPool(20); + transferQueueConsumerService = Executors.newFixedThreadPool(2); scheduler = new ScheduledThreadPoolExecutor(1); - + otherPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 5L, ByteSizeUnit.GB), + transferQueueConsumerService, + 2 + ); + lowPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 5L, ByteSizeUnit.GB), + transferQueueConsumerService, + 2 + ); + otherPrioritySizeBasedBlockingQ.start(); + lowPrioritySizeBasedBlockingQ.start(); // needed by S3AsyncService SocketAccess.doPrivileged(() -> System.setProperty("opensearch.path.conf", configPath().toString())); super.setUp(); @@ -145,7 +161,10 @@ public void tearDown() throws Exception { streamReaderService.shutdown(); futureCompletionService.shutdown(); remoteTransferRetry.shutdown(); + transferQueueConsumerService.shutdown(); scheduler.shutdown(); + otherPrioritySizeBasedBlockingQ.close(); + lowPrioritySizeBasedBlockingQ.close(); IOUtils.close(transferNIOGroup); if (previousOpenSearchPathConf != null) { @@ -233,17 +252,13 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), - new PermitBackedRetryableFutureUtils<>( - 3, - Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), - 0.7, - remoteTransferRetry, - scheduler - ) + new TransferSemaphoresHolder(3, Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES) ), asyncExecutorContainer, asyncExecutorContainer, - asyncExecutorContainer + asyncExecutorContainer, + otherPrioritySizeBasedBlockingQ, + lowPrioritySizeBasedBlockingQ ) ) { @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..cc7aad76fcd0c 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,10 @@ private S3Repository createS3Repo(RepositoryMetadata metadata) { null, null, null, - false + false, + 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 e967e2b023465..3a2712c0feaaf 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 @@ -46,6 +46,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; @@ -68,13 +69,7 @@ public void setUp() throws Exception { Executors.newSingleThreadExecutor(), Executors.newSingleThreadExecutor(), Executors.newSingleThreadExecutor(), - new PermitBackedRetryableFutureUtils<>( - 3, - Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), - 0.7, - Executors.newSingleThreadExecutor(), - Executors.newSingleThreadScheduledExecutor() - ) + new TransferSemaphoresHolder(3, Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES) ); super.setUp(); } diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtilsTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtilsTests.java deleted file mode 100644 index b07bf0630f969..0000000000000 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/PermitBackedRetryableFutureUtilsTests.java +++ /dev/null @@ -1,505 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.repositories.s3.async; - -import software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException; -import software.amazon.awssdk.utils.CompletableFutureUtils; - -import org.opensearch.common.blobstore.stream.write.WritePriority; -import org.opensearch.repositories.s3.SocketAccess; -import org.opensearch.test.OpenSearchTestCase; -import org.junit.After; -import org.junit.Before; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; - -@SuppressWarnings({ "unchecked", "rawtypes" }) -public class PermitBackedRetryableFutureUtilsTests extends OpenSearchTestCase { - private ExecutorService testExecutor; - private ScheduledExecutorService scheduler; - - @Before - public void setup() { - this.testExecutor = Executors.newFixedThreadPool(30); - this.scheduler = Executors.newSingleThreadScheduledExecutor(); - } - - @After - public void cleanUp() { - testExecutor.shutdown(); - scheduler.shutdown(); - } - - public void testFutureExecAndPermitRelease() throws InterruptedException, ExecutionException { - PermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new PermitBackedRetryableFutureUtils( - 3, - Runtime.getRuntime().availableProcessors(), - 0.7, - testExecutor, - scheduler - ); - int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); - int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; - assertEquals((int) (0.7 * Runtime.getRuntime().availableProcessors()), maxHighPermits); - assertEquals( - Runtime.getRuntime().availableProcessors() - maxHighPermits, - permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits() - ); - - Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { - assertEquals(maxHighPermits - 1, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - return "success"; - }, testExecutor)); - - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.HIGH, - requestContext - ); - resultFuture.get(); - - assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - } - - static class TestPermitBackedRetryableFutureUtils extends PermitBackedRetryableFutureUtils { - - private final AtomicBoolean highPermitsFullyConsumed = new AtomicBoolean(); - private final AtomicBoolean lowPermitsFullyConsumed = new AtomicBoolean(); - private final AtomicBoolean lowPermitConsumedForHighPriority = new AtomicBoolean(); - private final AtomicBoolean highPermitsConsumedForLowPriority = new AtomicBoolean(); - private final AtomicBoolean waitedForPermit = new AtomicBoolean(); - private final Runnable onWaitForPermit; - - /** - * @param maxRetryAttempts max number of retries - * @param availablePermits Total available permits for transfer - * @param priorityPermitAllocation Allocation bandwidth of priority permits. Rest will be allocated to low - * priority permits. - */ - public TestPermitBackedRetryableFutureUtils( - int maxRetryAttempts, - int availablePermits, - double priorityPermitAllocation, - ExecutorService remoteTransferRetry, - ScheduledExecutorService scheduler - ) { - super(maxRetryAttempts, availablePermits, priorityPermitAllocation, remoteTransferRetry, scheduler); - this.onWaitForPermit = null; - - } - - static class TestSemaphore extends TypeSemaphore { - private final Runnable onWaitForPermit; - private final Supplier preAcquireFailure; - - public TestSemaphore(int permits, String type, Runnable onWaitForPermit, Supplier preAcquireFailure) { - super(permits, type); - this.onWaitForPermit = onWaitForPermit; - this.preAcquireFailure = preAcquireFailure; - } - - @Override - public boolean tryAcquire(long timeout, TimeUnit unit) throws InterruptedException { - if (preAcquireFailure != null && preAcquireFailure.get()) { - return false; - } - onWaitForPermit.run(); - return super.tryAcquire(timeout, unit); - } - - @Override - public boolean tryAcquire() { - if (preAcquireFailure != null && preAcquireFailure.get()) { - return false; - } - return super.tryAcquire(); - } - } - - public TestPermitBackedRetryableFutureUtils( - int maxRetryAttempts, - int availablePermits, - double priorityPermitAllocation, - Runnable onWaitForPermit, - Supplier preAcquireFailure, - ExecutorService remoteTransferRetry, - ScheduledExecutorService scheduler - ) { - super(maxRetryAttempts, availablePermits, priorityPermitAllocation, remoteTransferRetry, scheduler); - this.onWaitForPermit = () -> { - waitedForPermit.set(true); - onWaitForPermit.run(); - }; - this.highPrioritySemaphore = new TestSemaphore( - highPrioritySemaphore.availablePermits(), - "high", - this.onWaitForPermit, - preAcquireFailure - ); - this.lowPrioritySemaphore = new TestSemaphore( - lowPrioritySemaphore.availablePermits(), - "low", - this.onWaitForPermit, - preAcquireFailure - ); - } - - Semaphore acquirePermit(WritePriority writePriority, boolean isLowPriorityPermitsConsumable) throws InterruptedException { - TypeSemaphore semaphore = (TypeSemaphore) super.acquirePermit(writePriority, isLowPriorityPermitsConsumable); - if (semaphore == null) { - return null; - } - if (semaphore.getType().equals("high")) { - if (getAvailableHighPriorityPermits() == 0) { - highPermitsFullyConsumed.set(true); - } - if (writePriority == WritePriority.LOW) { - highPermitsConsumedForLowPriority.set(true); - } - } else if (semaphore.getType().equals("low")) { - if (getAvailableLowPriorityPermits() == 0) { - lowPermitsFullyConsumed.set(true); - } - if (writePriority == WritePriority.HIGH) { - lowPermitConsumedForHighPriority.set(true); - } - } - return semaphore; - } - } - - public void testLowPermitConsumptionForHighTask() throws InterruptedException, ExecutionException { - TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( - 3, - Runtime.getRuntime().availableProcessors(), - 0.7, - testExecutor, - scheduler - ); - int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); - int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; - - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); - List> futures = new ArrayList<>(); - CountDownLatch delayedLatch = new CountDownLatch(1); - for (int reqIdx = 0; reqIdx < (maxHighPermits + lowPermits); reqIdx++) { - Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { - try { - // Keep the permit acquired - delayedLatch.await(10, TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - return "success"; - }, testExecutor)); - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.HIGH, - requestContext - ); - futures.add(resultFuture); - } - // Now release all permits - delayedLatch.countDown(); - - CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); - assertTrue(permitBasedRetryableFutureUtils.lowPermitsFullyConsumed.get()); - assertTrue(permitBasedRetryableFutureUtils.highPermitsFullyConsumed.get()); - assertTrue(permitBasedRetryableFutureUtils.lowPermitConsumedForHighPriority.get()); - assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - } - - public void testOnlyHighPermitsAcquiredWhenLowTaskInProgress() throws ExecutionException, InterruptedException { - CountDownLatch delayedLatch = new CountDownLatch(1); - TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( - 3, - Runtime.getRuntime().availableProcessors(), - 0.7, - delayedLatch::countDown, - null, - testExecutor, - scheduler - ); - int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); - int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; - - PermitBackedRetryableFutureUtils.RequestContext lowRequestContext = permitBasedRetryableFutureUtils.createRequestContext(); - List> futures = new ArrayList<>(); - - Supplier> lowSupplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { - try { - // Keep the permit acquired - delayedLatch.await(30, TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - return "success"; - }, testExecutor)); - CompletableFuture lowResultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - lowSupplier, - WritePriority.LOW, - lowRequestContext - ); - futures.add(lowResultFuture); - - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); - for (int reqIdx = 0; reqIdx < maxHighPermits; reqIdx++) { - Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { - try { - // Keep the permit acquired - delayedLatch.await(30, TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - return "success"; - }, testExecutor)); - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.HIGH, - requestContext - ); - futures.add(resultFuture); - } - - Thread t = new Thread(() -> { - Supplier> supplier = () -> SocketAccess.doPrivileged( - () -> CompletableFuture.supplyAsync(() -> "success", testExecutor) - ); - - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.HIGH, - requestContext - ); - futures.add(resultFuture); - }); - t.start(); - t.join(); - - CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); - assertTrue(permitBasedRetryableFutureUtils.waitedForPermit.get()); - assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - } - - public void testHighPermitsConsumedForLowTasks() throws ExecutionException, InterruptedException { - CountDownLatch delayedLatch = new CountDownLatch(1); - TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( - 3, - Runtime.getRuntime().availableProcessors(), - 0.7, - delayedLatch::countDown, - null, - testExecutor, - scheduler - ); - int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); - int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; - - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); - List> futures = new ArrayList<>(); - for (int reqIdx = 0; reqIdx < lowPermits; reqIdx++) { - Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { - try { - // Keep the permit acquired - delayedLatch.await(30, TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - return "success"; - }, testExecutor)); - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.LOW, - requestContext - ); - futures.add(resultFuture); - } - - Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { - try { - // Keep the permit acquired - delayedLatch.await(30, TimeUnit.SECONDS); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - return "success"; - }, testExecutor)); - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.LOW, - requestContext - ); - futures.add(resultFuture); - delayedLatch.countDown(); - - CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); - assertTrue(permitBasedRetryableFutureUtils.highPermitsConsumedForLowPriority.get()); - assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - } - - public void testFutureRetryOnSemaphoreFailure() throws ExecutionException, InterruptedException { - int retryCount = 3; - // Multiply by 3 as there are 3 ways in which permit can be acquired. - AtomicInteger failureCount = new AtomicInteger((retryCount - 1) * 3); - AtomicBoolean exhaustRetries = new AtomicBoolean(); - TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( - retryCount, - Runtime.getRuntime().availableProcessors(), - 0.7, - null, - () -> { - if (failureCount.get() > 0 || exhaustRetries.get()) { - failureCount.decrementAndGet(); - return true; - } - return false; - }, - testExecutor, - scheduler - ); - int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); - int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; - - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); - List> futures = new ArrayList<>(); - Supplier> supplier = () -> SocketAccess.doPrivileged( - () -> CompletableFuture.supplyAsync(() -> "success", testExecutor) - ); - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.HIGH, - requestContext - ); - futures.add(resultFuture); - - CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); - - assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - // Reached here so future executed successfully after retries. - - exhaustRetries.set(true); - resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture(supplier, WritePriority.HIGH, requestContext); - resultFuture.whenComplete((r, t) -> { - assertNotNull(t); - assertTrue(t instanceof PermitBackedRetryableFutureUtils.RetryableException); - }); - CompletableFuture finalResultFuture = resultFuture; - assertThrows(Exception.class, finalResultFuture::get); - } - - public void testFutureRetryOnExecFailure() throws ExecutionException, InterruptedException { - int retryCount = 3; - AtomicInteger failureCount = new AtomicInteger(retryCount); - AtomicBoolean exhaustRetries = new AtomicBoolean(); - TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( - retryCount, - Runtime.getRuntime().availableProcessors(), - 0.7, - null, - null, - testExecutor, - scheduler - ); - int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); - int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; - - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); - List> futures = new ArrayList<>(); - Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { - if (failureCount.get() > 0 || exhaustRetries.get()) { - failureCount.decrementAndGet(); - throw ApiCallAttemptTimeoutException.builder().build(); - } - return "success"; - }, testExecutor)); - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.HIGH, - requestContext - ); - futures.add(resultFuture); - - CompletableFutureUtils.allOfExceptionForwarded(futures.toArray(CompletableFuture[]::new)).get(); - - assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - // Reached here so future executed successfully after retries. - - exhaustRetries.set(true); - resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture(supplier, WritePriority.HIGH, requestContext); - resultFuture.whenComplete((r, t) -> { - assertNotNull(t); - assertTrue(t instanceof PermitBackedRetryableFutureUtils.RetryableException); - }); - CompletableFuture finalResultFuture = resultFuture; - assertThrows(Exception.class, finalResultFuture::get); - - assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - } - - public void testNonRetryableFailure() throws ExecutionException, InterruptedException { - // Throw only once to ensure no retries for unknown exception. - AtomicInteger failureCount = new AtomicInteger(1); - AtomicBoolean exhaustRetries = new AtomicBoolean(); - TestPermitBackedRetryableFutureUtils permitBasedRetryableFutureUtils = new TestPermitBackedRetryableFutureUtils( - 3, - Runtime.getRuntime().availableProcessors(), - 0.7, - null, - null, - testExecutor, - scheduler - ); - int maxHighPermits = permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits(); - int lowPermits = Runtime.getRuntime().availableProcessors() - maxHighPermits; - - PermitBackedRetryableFutureUtils.RequestContext requestContext = permitBasedRetryableFutureUtils.createRequestContext(); - Supplier> supplier = () -> SocketAccess.doPrivileged(() -> CompletableFuture.supplyAsync(() -> { - if (failureCount.get() > 0) { - failureCount.decrementAndGet(); - throw new RuntimeException("Generic exception"); - } - return "success"; - }, testExecutor)); - - CompletableFuture resultFuture = permitBasedRetryableFutureUtils.createPermitBackedRetryableFuture( - supplier, - WritePriority.HIGH, - requestContext - ); - resultFuture.whenComplete((r, t) -> { - assertNotNull(t); - assertFalse(t instanceof PermitBackedRetryableFutureUtils.RetryableException); - }); - assertThrows(Exception.class, resultFuture::get); - - assertEquals(maxHighPermits, permitBasedRetryableFutureUtils.getAvailableHighPriorityPermits()); - assertEquals(lowPermits, permitBasedRetryableFutureUtils.getAvailableLowPriorityPermits()); - } -} 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..53f507c72784a --- /dev/null +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/SizeBasedBlockingQTests.java @@ -0,0 +1,94 @@ +/* + * 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.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 { + + SizeBasedBlockingQ sizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(ByteSizeUnit.BYTES.toBytes(10)), + consumerService, + 10 + ); + 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()); + } + + 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..bcec8ece062cf --- /dev/null +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/TransferSemaphoresHolderTests.java @@ -0,0 +1,253 @@ +/* + * 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.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; + +public class TransferSemaphoresHolderTests extends OpenSearchTestCase { + + public void testAllocation() { + int availablePermits = randomIntBetween(5, 20); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int highPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - highPermits; + TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( + availablePermits, + priorityAllocation, + 1, + TimeUnit.NANOSECONDS + ); + assertEquals(highPermits, transferSemaphoresHolder.getHighPriorityPermits()); + assertEquals(lowPermits, transferSemaphoresHolder.getLowPriorityPermits()); + } + + public void testLowPriorityEventPermitAcquisition() throws InterruptedException { + int availablePermits = randomIntBetween(5, 50); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int highPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - highPermits; + TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( + availablePermits, + priorityAllocation, + 1, + TimeUnit.NANOSECONDS + ); + + List semaphores = new ArrayList<>(); + int highPermitsEligibleForLowEvents = highPermits - (int) (highPermits * 0.4); + + int lowAcquisitionsExpected = (highPermitsEligibleForLowEvents + 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("high", acquiredSemaphore.getType()); + } else { + assertEquals("low", acquiredSemaphore.getType()); + } + } + + for (int i = 0; i < highPermits - highPermitsEligibleForLowEvents; i++) { + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.HIGH, + requestContext + ); + assertEquals("high", 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(highPermits, transferSemaphoresHolder.getHighPriorityPermits()); + assertEquals(lowPermits, transferSemaphoresHolder.getLowPriorityPermits()); + + } + + public void testHighPermitEventAcquisition() throws InterruptedException { + int availablePermits = randomIntBetween(5, 50); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int highPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - highPermits; + TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( + availablePermits, + priorityAllocation, + 1, + TimeUnit.NANOSECONDS + ); + + List semaphores = new ArrayList<>(); + List lowSemaphores = new ArrayList<>(); + int highAcquisitionsExpected = highPermits + lowPermits; + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + for (int i = 0; i < highAcquisitionsExpected; i++) { + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.HIGH, + requestContext + ); + semaphores.add(acquiredSemaphore); + if (i >= highPermits) { + assertEquals("low", acquiredSemaphore.getType()); + lowSemaphores.add(acquiredSemaphore); + } else { + assertEquals("high", acquiredSemaphore.getType()); + } + } + assertEquals(availablePermits, semaphores.size()); + + int lowAcquired = lowPermits; + lowSemaphores.get(0).release(); + + 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.HIGH, requestContext); + assertNull(acquiredSemaphore); + lowAcquired--; + } + + semaphores.forEach(Semaphore::release); + assertEquals(highPermits, transferSemaphoresHolder.getHighPriorityPermits()); + assertEquals(lowPermits, transferSemaphoresHolder.getLowPriorityPermits()); + } + + private static class TestTransferSemaphoresHolder extends TransferSemaphoresHolder { + AtomicInteger highWaitCount = new AtomicInteger(); + AtomicInteger lowWaitCount = new AtomicInteger(); + + /** + * Constructor to create semaphores holder. + */ + public TestTransferSemaphoresHolder( + int availablePermits, + double priorityPermitAllocation, + int acquireWaitDuration, + TimeUnit timeUnit + ) throws InterruptedException { + super(availablePermits, priorityPermitAllocation, acquireWaitDuration, timeUnit); + TypeSemaphore executingHighSemaphore = highPrioritySemaphore; + TypeSemaphore executingLowSemaphore = lowPrioritySemaphore; + + this.highPrioritySemaphore = Mockito.spy(highPrioritySemaphore); + this.lowPrioritySemaphore = Mockito.spy(lowPrioritySemaphore); + Mockito.doAnswer(invocation -> { + highWaitCount.incrementAndGet(); + return false; + }).when(highPrioritySemaphore).tryAcquire(Mockito.anyLong(), Mockito.any(TimeUnit.class)); + Mockito.doAnswer(invocation -> executingHighSemaphore.availablePermits()).when(highPrioritySemaphore).availablePermits(); + Mockito.doAnswer(invocation -> executingHighSemaphore.tryAcquire()).when(highPrioritySemaphore).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 testHighSemaphoreAcquiredWait() throws InterruptedException { + int availablePermits = randomIntBetween(10, 50); + double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); + int highPermits = (int) (availablePermits * priorityAllocation); + TestTransferSemaphoresHolder transferSemaphoresHolder = new TestTransferSemaphoresHolder( + availablePermits, + priorityAllocation, + 5, + TimeUnit.MINUTES + ); + + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore lowSemaphore = transferSemaphoresHolder.acquirePermit(WritePriority.LOW, requestContext); + assertEquals("low", lowSemaphore.getType()); + for (int i = 0; i < highPermits; i++) { + requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.HIGH, + requestContext + ); + assertEquals("high", acquiredSemaphore.getType()); + } + + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.HIGH, + requestContext + ); + assertNull(acquiredSemaphore); + assertEquals(1, transferSemaphoresHolder.highWaitCount.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 highPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - highPermits; + TestTransferSemaphoresHolder transferSemaphoresHolder = new TestTransferSemaphoresHolder( + availablePermits, + priorityAllocation, + 5, + TimeUnit.MINUTES + ); + + TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); + int highPermitsEligibleForLowEvents = highPermits - (int) (highPermits * 0.4); + for (int i = 0; i < highPermitsEligibleForLowEvents; i++) { + TransferSemaphoresHolder.TypeSemaphore lowSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.HIGH, + requestContext + ); + assertEquals("high", lowSemaphore.getType()); + } + + for (int i = 0; i < lowPermits; i++) { + requestContext = transferSemaphoresHolder.createRequestContext(); + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.LOW, + requestContext + ); + assertEquals("low", acquiredSemaphore.getType()); + } + + TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( + WritePriority.LOW, + requestContext + ); + assertNull(acquiredSemaphore); + assertEquals(1, transferSemaphoresHolder.lowWaitCount.get()); + assertEquals(0, transferSemaphoresHolder.highWaitCount.get()); + } + +} 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 4fef8c6179c8e..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; @@ -355,6 +356,7 @@ private void uploadBlob( if (getBlobContainer() instanceof AsyncMultiStreamBlobContainer) { remoteIntegrityEnabled = ((AsyncMultiStreamBlobContainer) getBlobContainer()).remoteIntegrityCheckSupported(); } + lowPriorityUpload = lowPriorityUpload || contentLength > ByteSizeUnit.GB.toBytes(15); RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( src, remoteFileName, From a518a0881e5483736a931476879aa8ca09f97f6c Mon Sep 17 00:00:00 2001 From: vikasvb90 Date: Wed, 24 Apr 2024 08:52:39 +0530 Subject: [PATCH 3/5] Renamed other priority resources to normal priority resources Signed-off-by: vikasvb90 --- .../repositories/s3/S3BlobContainer.java | 10 ++++--- .../repositories/s3/S3BlobStore.java | 10 +++---- .../repositories/s3/S3Repository.java | 12 ++++---- .../repositories/s3/S3RepositoryPlugin.java | 30 +++++++++---------- .../s3/async/SizeBasedBlockingQ.java | 5 ++-- .../s3/async/TransferSemaphoresHolder.java | 12 +++++++- .../s3/S3BlobContainerMockClientTests.java | 12 ++++---- .../s3/S3BlobContainerRetriesTests.java | 10 +++---- .../blobstore/stream/write/WritePriority.java | 4 +++ 9 files changed, 61 insertions(+), 44 deletions(-) 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 1b9b022b09847..3c8a3f127a295 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 @@ -222,10 +222,10 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp // 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().canProduce(uploadRequest.getContentLength()) == false) + && blobStore.getLowPrioritySizeBasedBlockingQ().isBelowCapacity(uploadRequest.getContentLength()) == false) || (uploadRequest.getWritePriority() != WritePriority.HIGH && uploadRequest.getWritePriority() != WritePriority.URGENT - && blobStore.getOtherPrioritySizeBasedBlockingQ().canProduce(uploadRequest.getContentLength()) == false)) { + && blobStore.getNormalPrioritySizeBasedBlockingQ().isBelowCapacity(uploadRequest.getContentLength()) == false)) { StreamContext streamContext = SocketAccess.doPrivileged( () -> writeContext.getStreamProvider(uploadRequest.getContentLength()) ); @@ -276,14 +276,16 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp () -> createFileCompletableFuture(s3AsyncClient, uploadRequest, streamContext, completionListener) ) ); - } else { - blobStore.getOtherPrioritySizeBasedBlockingQ() + } 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) { 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 e1a8fff87bc3e..414e6eeb9369b 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 @@ -95,7 +95,7 @@ class S3BlobStore implements BlobStore { private final AsyncExecutorContainer priorityExecutorBuilder; private final AsyncExecutorContainer normalExecutorBuilder; private final boolean multipartUploadEnabled; - private final SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; S3BlobStore( @@ -113,7 +113,7 @@ class S3BlobStore implements BlobStore { AsyncExecutorContainer urgentExecutorBuilder, AsyncExecutorContainer priorityExecutorBuilder, AsyncExecutorContainer normalExecutorBuilder, - SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ, + SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ ) { this.service = service; @@ -133,7 +133,7 @@ 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.otherPrioritySizeBasedBlockingQ = otherPrioritySizeBasedBlockingQ; + this.normalPrioritySizeBasedBlockingQ = normalPrioritySizeBasedBlockingQ; this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; } @@ -191,8 +191,8 @@ public int getBulkDeletesSize() { return bulkDeletesSize; } - public SizeBasedBlockingQ getOtherPrioritySizeBasedBlockingQ() { - return otherPrioritySizeBasedBlockingQ; + public SizeBasedBlockingQ getNormalPrioritySizeBasedBlockingQ() { + return normalPrioritySizeBasedBlockingQ; } public SizeBasedBlockingQ getLowPrioritySizeBasedBlockingQ() { 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 3306062c653cd..269135a15d411 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 @@ -285,7 +285,7 @@ class S3Repository extends MeteredBlobStoreRepository { private final AsyncExecutorContainer priorityExecutorBuilder; private final AsyncExecutorContainer normalExecutorBuilder; private final Path pluginConfigPath; - private final SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; private volatile int bulkDeletesSize; @@ -303,7 +303,7 @@ class S3Repository extends MeteredBlobStoreRepository { final AsyncExecutorContainer normalExecutorBuilder, final S3AsyncService s3AsyncService, final boolean multipartUploadEnabled, - final SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ, + final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ ) { this( @@ -319,7 +319,7 @@ class S3Repository extends MeteredBlobStoreRepository { s3AsyncService, multipartUploadEnabled, Path.of(""), - otherPrioritySizeBasedBlockingQ, + normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ ); } @@ -340,7 +340,7 @@ class S3Repository extends MeteredBlobStoreRepository { final S3AsyncService s3AsyncService, final boolean multipartUploadEnabled, Path pluginConfigPath, - final SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ, + final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ ) { super(metadata, namedXContentRegistry, clusterService, recoverySettings, buildLocation(metadata)); @@ -352,7 +352,7 @@ class S3Repository extends MeteredBlobStoreRepository { this.urgentExecutorBuilder = urgentExecutorBuilder; this.priorityExecutorBuilder = priorityExecutorBuilder; this.normalExecutorBuilder = normalExecutorBuilder; - this.otherPrioritySizeBasedBlockingQ = otherPrioritySizeBasedBlockingQ; + this.normalPrioritySizeBasedBlockingQ = normalPrioritySizeBasedBlockingQ; this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; validateRepositoryMetadata(metadata); @@ -417,7 +417,7 @@ protected S3BlobStore createBlobStore() { urgentExecutorBuilder, priorityExecutorBuilder, normalExecutorBuilder, - otherPrioritySizeBasedBlockingQ, + normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ ); } 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 c4acb9ed67e22..b9065a52601c6 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 @@ -90,7 +90,7 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo 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 OTHER_TRANSFER_QUEUE_CONSUMER = "other_transfer_queue_consumer"; + private static final String NORMAL_TRANSFER_QUEUE_CONSUMER = "normal_transfer_queue_consumer"; protected final S3Service service; private final S3AsyncService s3AsyncService; @@ -101,8 +101,8 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private AsyncExecutorContainer priorityExecutorBuilder; private AsyncExecutorContainer normalExecutorBuilder; private ExecutorService lowTransferQConsumerService; - private ExecutorService otherTransferQConsumerService; - private SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private ExecutorService normalTransferQConsumerService; + private SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; private TransferSemaphoresHolder transferSemaphoresHolder; @@ -146,10 +146,10 @@ public List> getExecutorBuilders(Settings settings) { executorBuilders.add( new FixedExecutorBuilder( settings, - OTHER_TRANSFER_QUEUE_CONSUMER, - otherPriorityTransferQConsumers(settings), + NORMAL_TRANSFER_QUEUE_CONSUMER, + normalPriorityTransferQConsumers(settings), 10, - "thread_pool." + OTHER_TRANSFER_QUEUE_CONSUMER + "thread_pool." + NORMAL_TRANSFER_QUEUE_CONSUMER ) ); return executorBuilders; @@ -160,7 +160,7 @@ private int lowPriorityTransferQConsumers(Settings settings) { return Math.max(2, (int) (lowPriorityAllocation * S3Repository.S3_TRANSFER_QUEUE_CONSUMERS.get(settings))); } - private int otherPriorityTransferQConsumers(Settings settings) { + private int normalPriorityTransferQConsumers(Settings settings) { return S3Repository.S3_TRANSFER_QUEUE_CONSUMERS.get(settings); } @@ -231,12 +231,12 @@ public Collection createComponents( new AsyncTransferEventLoopGroup(normalEventLoopThreads) ); this.lowTransferQConsumerService = threadPool.executor(LOW_TRANSFER_QUEUE_CONSUMER); - this.otherTransferQConsumerService = threadPool.executor(OTHER_TRANSFER_QUEUE_CONSUMER); - int otherPriorityConsumers = otherPriorityTransferQConsumers(clusterService.getSettings()); - this.otherPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( - new ByteSizeValue(otherPriorityConsumers * 10L, ByteSizeUnit.GB), - otherTransferQConsumerService, - otherPriorityConsumers + this.normalTransferQConsumerService = threadPool.executor(NORMAL_TRANSFER_QUEUE_CONSUMER); + int normalPriorityConsumers = normalPriorityTransferQConsumers(clusterService.getSettings()); + this.normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + new ByteSizeValue(normalPriorityConsumers * 10L, ByteSizeUnit.GB), + normalTransferQConsumerService, + normalPriorityConsumers ); int lowPriorityConsumers = lowPriorityTransferQConsumers(clusterService.getSettings()); LowPrioritySizeBasedBlockingQ lowPrioritySizeBasedBlockingQ = new LowPrioritySizeBasedBlockingQ( @@ -253,7 +253,7 @@ public Collection createComponents( TimeUnit.MINUTES ); - return CollectionUtils.arrayAsArrayList(this.otherPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ); + return CollectionUtils.arrayAsArrayList(this.normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ); } // New class because in core, components are injected via guice only by instance creation due to which @@ -292,7 +292,7 @@ protected S3Repository createRepository( s3AsyncService, S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING.get(clusterService.getSettings()), configPath, - otherPrioritySizeBasedBlockingQ, + normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ ); } 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 index 289c92c14a8b9..ace8003d0fe55 100644 --- 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 @@ -69,6 +69,7 @@ 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"); @@ -96,8 +97,8 @@ public int getSize() { return queue.size(); } - public boolean canProduce(long contentLength) { - return (currentSize.get() + contentLength) < capacity.getBytes(); + public boolean isBelowCapacity(long contentLength) { + return contentLength < capacity.getBytes(); } protected static class Consumer extends Thread { 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 index aa3b6a78e2c4a..77c2a77fb5c12 100644 --- 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 @@ -8,6 +8,8 @@ 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 java.util.Objects; @@ -18,7 +20,7 @@ * 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 highPrioritySemaphore; @@ -88,6 +90,14 @@ public RequestContext createRequestContext() { * transfers. */ public TypeSemaphore acquirePermit(WritePriority writePriority, RequestContext requestContext) throws InterruptedException { + log.debug( + () -> "Acquire permit request for transfer type: " + + writePriority + + ". Available high priority permits: " + + highPrioritySemaphore.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) { 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 4ec1baf634f52..d5f664b1f0608 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 @@ -100,7 +100,7 @@ public class S3BlobContainerMockClientTests extends OpenSearchTestCase implement private ScheduledExecutorService scheduler; private AsyncTransferEventLoopGroup transferNIOGroup; private S3BlobContainer blobContainer; - private SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; static class MockS3AsyncService extends S3AsyncService { @@ -377,7 +377,7 @@ public void setUp() throws Exception { transferQueueConsumerService = Executors.newFixedThreadPool(20); scheduler = new Scheduler.SafeScheduledThreadPoolExecutor(1); transferNIOGroup = new AsyncTransferEventLoopGroup(1); - otherPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 10L, ByteSizeUnit.GB), transferQueueConsumerService, 10 @@ -387,7 +387,7 @@ public void setUp() throws Exception { transferQueueConsumerService, 5 ); - otherPrioritySizeBasedBlockingQ.start(); + normalPrioritySizeBasedBlockingQ.start(); lowPrioritySizeBasedBlockingQ.start(); blobContainer = createBlobContainer(); super.setUp(); @@ -401,7 +401,7 @@ public void tearDown() throws Exception { streamReaderService.shutdown(); remoteTransferRetry.shutdown(); transferQueueConsumerService.shutdown(); - otherPrioritySizeBasedBlockingQ.close(); + normalPrioritySizeBasedBlockingQ.close(); lowPrioritySizeBasedBlockingQ.close(); scheduler.shutdown(); transferNIOGroup.close(); @@ -448,7 +448,7 @@ private S3BlobStore createBlobStore() { asyncExecutorContainer, asyncExecutorContainer, asyncExecutorContainer, - otherPrioritySizeBasedBlockingQ, + normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ ); } @@ -651,7 +651,7 @@ private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException, W when(blobStore.bufferSizeInBytes()).thenReturn(bufferSize); when(blobStore.getLowPrioritySizeBasedBlockingQ()).thenReturn(sizeBasedBlockingQ); - when(blobStore.getOtherPrioritySizeBasedBlockingQ()).thenReturn(sizeBasedBlockingQ); + when(blobStore.getNormalPrioritySizeBasedBlockingQ()).thenReturn(sizeBasedBlockingQ); final boolean serverSideEncryption = randomBoolean(); when(blobStore.serverSideEncryption()).thenReturn(serverSideEncryption); 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 95d807efa92cc..f2f87231e9620 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 @@ -122,7 +122,7 @@ public class S3BlobContainerRetriesTests extends AbstractBlobContainerRetriesTes private ExecutorService transferQueueConsumerService; private ScheduledExecutorService scheduler; private AsyncTransferEventLoopGroup transferNIOGroup; - private SizeBasedBlockingQ otherPrioritySizeBasedBlockingQ; + private SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; @Before @@ -137,7 +137,7 @@ public void setUp() throws Exception { remoteTransferRetry = Executors.newFixedThreadPool(20); transferQueueConsumerService = Executors.newFixedThreadPool(2); scheduler = new ScheduledThreadPoolExecutor(1); - otherPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( + normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 5L, ByteSizeUnit.GB), transferQueueConsumerService, 2 @@ -147,7 +147,7 @@ public void setUp() throws Exception { transferQueueConsumerService, 2 ); - otherPrioritySizeBasedBlockingQ.start(); + normalPrioritySizeBasedBlockingQ.start(); lowPrioritySizeBasedBlockingQ.start(); // needed by S3AsyncService SocketAccess.doPrivileged(() -> System.setProperty("opensearch.path.conf", configPath().toString())); @@ -163,7 +163,7 @@ public void tearDown() throws Exception { remoteTransferRetry.shutdown(); transferQueueConsumerService.shutdown(); scheduler.shutdown(); - otherPrioritySizeBasedBlockingQ.close(); + normalPrioritySizeBasedBlockingQ.close(); lowPrioritySizeBasedBlockingQ.close(); IOUtils.close(transferNIOGroup); @@ -257,7 +257,7 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( asyncExecutorContainer, asyncExecutorContainer, asyncExecutorContainer, - otherPrioritySizeBasedBlockingQ, + normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ ) ) { 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 9888612b444bc..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,8 +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, + // 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 } From 58d730befde68cab529ac7cf2e5fc2e0745ab1d4 Mon Sep 17 00:00:00 2001 From: vikasvb90 Date: Fri, 3 May 2024 12:09:52 +0530 Subject: [PATCH 4/5] Added stats for permits and queue in repository-s3 for permit backed transfers Signed-off-by: vikasvb90 --- .../s3/GenericStatsMetricPublisher.java | 74 ++++++++++ .../repositories/s3/S3BlobContainer.java | 9 +- .../repositories/s3/S3BlobStore.java | 19 ++- .../repositories/s3/S3Repository.java | 20 ++- .../repositories/s3/S3RepositoryPlugin.java | 26 +++- .../s3/async/SizeBasedBlockingQ.java | 49 ++++++- .../s3/async/TransferSemaphoresHolder.java | 105 +++++++++----- .../s3/S3BlobContainerMockClientTests.java | 29 +++- .../s3/S3BlobContainerRetriesTests.java | 20 ++- .../repositories/s3/S3RepositoryTests.java | 3 +- .../s3/async/AsyncTransferManagerTests.java | 9 +- .../s3/async/SizeBasedBlockingQTests.java | 12 +- .../async/TransferSemaphoresHolderTests.java | 131 ++++++++++-------- .../common/blobstore/BlobStore.java | 3 +- 14 files changed, 384 insertions(+), 125 deletions(-) create mode 100644 plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/GenericStatsMetricPublisher.java 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..8a37dc06033fa --- /dev/null +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/GenericStatsMetricPublisher.java @@ -0,0 +1,74 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.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(); + + 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("NormalPriorityQSize", normalPriorityQSize.get()); + results.put("LowPriorityQSize", lowPriorityQSize.get()); + results.put("AcquiredNormalPriorityPermits", (long) normalPriorityPermits.get()); + results.put("AcquiredLowPriorityPermits", (long) lowPriorityPermits.get()); + 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 3c8a3f127a295..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 @@ -222,10 +222,11 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp // 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().isBelowCapacity(uploadRequest.getContentLength()) == false) + && blobStore.getLowPrioritySizeBasedBlockingQ().isMaxCapacityBelowContentLength(uploadRequest.getContentLength()) == false) || (uploadRequest.getWritePriority() != WritePriority.HIGH && uploadRequest.getWritePriority() != WritePriority.URGENT - && blobStore.getNormalPrioritySizeBasedBlockingQ().isBelowCapacity(uploadRequest.getContentLength()) == false)) { + && blobStore.getNormalPrioritySizeBasedBlockingQ() + .isMaxCapacityBelowContentLength(uploadRequest.getContentLength()) == false)) { StreamContext streamContext = SocketAccess.doPrivileged( () -> writeContext.getStreamProvider(uploadRequest.getContentLength()) ); @@ -266,7 +267,9 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp s3AsyncClient = amazonS3Reference.get().client(); } - if (writeContext.getWritePriority() == WritePriority.URGENT || writeContext.getWritePriority() == WritePriority.HIGH) { + 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() 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 414e6eeb9369b..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 @@ -57,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; @@ -78,6 +79,8 @@ class S3BlobStore implements BlobStore { private volatile boolean uploadRetryEnabled; + private volatile boolean permitBackedTransferEnabled; + private volatile boolean serverSideEncryption; private volatile ObjectCannedACL cannedACL; @@ -97,6 +100,7 @@ class S3BlobStore implements BlobStore { private final boolean multipartUploadEnabled; private final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; + private final GenericStatsMetricPublisher genericStatsMetricPublisher; S3BlobStore( S3Service service, @@ -114,7 +118,8 @@ class S3BlobStore implements BlobStore { AsyncExecutorContainer priorityExecutorBuilder, AsyncExecutorContainer normalExecutorBuilder, SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, - SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ + SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ, + GenericStatsMetricPublisher genericStatsMetricPublisher ) { this.service = service; this.s3AsyncService = s3AsyncService; @@ -135,6 +140,8 @@ class S3BlobStore implements BlobStore { 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 @@ -148,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 @@ -175,6 +183,10 @@ public boolean isUploadRetryEnabled() { return uploadRetryEnabled; } + public boolean isPermitBackedTransferEnabled() { + return permitBackedTransferEnabled; + } + public String bucket() { return bucket; } @@ -216,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 @@ -226,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 269135a15d411..80be255b33e9b 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 @@ -158,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. */ @@ -287,6 +296,7 @@ class S3Repository extends MeteredBlobStoreRepository { private final Path pluginConfigPath; private final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; + private final GenericStatsMetricPublisher genericStatsMetricPublisher; private volatile int bulkDeletesSize; @@ -320,7 +330,8 @@ class S3Repository extends MeteredBlobStoreRepository { multipartUploadEnabled, Path.of(""), normalPrioritySizeBasedBlockingQ, - lowPrioritySizeBasedBlockingQ + lowPrioritySizeBasedBlockingQ, + new GenericStatsMetricPublisher() ); } @@ -341,7 +352,8 @@ class S3Repository extends MeteredBlobStoreRepository { final boolean multipartUploadEnabled, Path pluginConfigPath, final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, - final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ + final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ, + final GenericStatsMetricPublisher genericStatsMetricPublisher ) { super(metadata, namedXContentRegistry, clusterService, recoverySettings, buildLocation(metadata)); this.service = service; @@ -354,6 +366,7 @@ class S3Repository extends MeteredBlobStoreRepository { this.normalExecutorBuilder = normalExecutorBuilder; this.normalPrioritySizeBasedBlockingQ = normalPrioritySizeBasedBlockingQ; this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; + this.genericStatsMetricPublisher = genericStatsMetricPublisher; validateRepositoryMetadata(metadata); readRepositoryMetadata(); @@ -418,7 +431,8 @@ protected S3BlobStore createBlobStore() { priorityExecutorBuilder, normalExecutorBuilder, normalPrioritySizeBasedBlockingQ, - lowPrioritySizeBasedBlockingQ + 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 b9065a52601c6..cbb1488b520c2 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 @@ -105,6 +105,7 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; private TransferSemaphoresHolder transferSemaphoresHolder; + private GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); public S3RepositoryPlugin(final Settings settings, final Path configPath) { this(settings, configPath, new S3Service(configPath), new S3AsyncService(configPath)); @@ -236,13 +237,16 @@ public Collection createComponents( this.normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(normalPriorityConsumers * 10L, ByteSizeUnit.GB), normalTransferQConsumerService, - normalPriorityConsumers + normalPriorityConsumers, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL ); int lowPriorityConsumers = lowPriorityTransferQConsumers(clusterService.getSettings()); LowPrioritySizeBasedBlockingQ lowPrioritySizeBasedBlockingQ = new LowPrioritySizeBasedBlockingQ( new ByteSizeValue(lowPriorityConsumers * 20L, ByteSizeUnit.GB), lowTransferQConsumerService, - lowPriorityConsumers + lowPriorityConsumers, + genericStatsMetricPublisher ); this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; this.transferSemaphoresHolder = new TransferSemaphoresHolder( @@ -250,7 +254,8 @@ public Collection createComponents( Math.max(allocatedProcessors(clusterService.getSettings()) * 4, 10), ((double) S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT.get(clusterService.getSettings())) / 100, S3Repository.S3_PERMIT_WAIT_DURATION_MIN.get(clusterService.getSettings()), - TimeUnit.MINUTES + TimeUnit.MINUTES, + genericStatsMetricPublisher ); return CollectionUtils.arrayAsArrayList(this.normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ); @@ -259,8 +264,13 @@ public Collection createComponents( // 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) { - super(capacity, executorService, consumers); + public LowPrioritySizeBasedBlockingQ( + ByteSizeValue capacity, + ExecutorService executorService, + int consumers, + GenericStatsMetricPublisher genericStatsMetricPublisher + ) { + super(capacity, executorService, consumers, genericStatsMetricPublisher, QueueEventType.LOW); } } @@ -293,7 +303,8 @@ protected S3Repository createRepository( S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING.get(clusterService.getSettings()), configPath, normalPrioritySizeBasedBlockingQ, - lowPrioritySizeBasedBlockingQ + lowPrioritySizeBasedBlockingQ, + genericStatsMetricPublisher ); } @@ -339,7 +350,8 @@ public List> getSettings() { S3Repository.PARALLEL_MULTIPART_UPLOAD_ENABLED_SETTING, S3Repository.REDIRECT_LARGE_S3_UPLOAD, S3Repository.UPLOAD_RETRY_ENABLED, - S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT + S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT, + S3Repository.PERMIT_BACKED_TRANSFER_ENABLED ); } 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 index ace8003d0fe55..170c80f5d4db6 100644 --- 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 @@ -13,6 +13,7 @@ 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; @@ -39,11 +40,19 @@ public class SizeBasedBlockingQ extends AbstractLifecycleComponent { 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) { + public SizeBasedBlockingQ( + ByteSizeValue capacity, + ExecutorService executorService, + int consumers, + GenericStatsMetricPublisher genericStatsMetricPublisher, + QueueEventType queueEventType + ) { this.queue = new LinkedBlockingQueue<>(); this.lock = new ReentrantLock(); this.notEmpty = lock.newCondition(); @@ -52,12 +61,19 @@ public SizeBasedBlockingQ(ByteSizeValue capacity, ExecutorService executorServic 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); + Thread consumer = new Consumer(queue, currentSize, lock, notEmpty, closed, genericStatsMetricPublisher, queueEventType); executorService.submit(consumer); } } @@ -88,16 +104,26 @@ public void produce(Item item) throws InterruptedException { 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 isBelowCapacity(long contentLength) { + public boolean isMaxCapacityBelowContentLength(long contentLength) { return contentLength < capacity.getBytes(); } @@ -107,13 +133,25 @@ protected static class Consumer extends Thread { private final Condition notEmpty; private final AtomicLong currentSize; private final AtomicBoolean closed; - - public Consumer(LinkedBlockingQueue queue, AtomicLong currentSize, Lock lock, Condition notEmpty, 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 @@ -147,6 +185,7 @@ private void consume() throws InterruptedException { item = queue.take(); currentSize.addAndGet(-item.size); + updateStats(-item.size, queueEventType, genericStatsMetricPublisher); } 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 index 77c2a77fb5c12..ce3b8929c35f4 100644 --- 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 @@ -11,10 +11,12 @@ 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. @@ -23,8 +25,8 @@ public class TransferSemaphoresHolder { private static final Logger log = LogManager.getLogger(TransferSemaphoresHolder.class); // For tests protected TypeSemaphore lowPrioritySemaphore; - protected TypeSemaphore highPrioritySemaphore; - private final int highPriorityPermits; + protected TypeSemaphore normalPrioritySemaphore; + private final int normalPriorityPermits; private final int lowPriorityPermits; private final int acquireWaitDuration; private final TimeUnit acquireWaitDurationUnit; @@ -32,12 +34,26 @@ public class TransferSemaphoresHolder { /** * Constructor to create semaphores holder. */ - public TransferSemaphoresHolder(int availablePermits, double priorityPermitAllocation, int acquireWaitDuration, TimeUnit timeUnit) { - - this.highPriorityPermits = (int) (priorityPermitAllocation * availablePermits); - this.highPrioritySemaphore = new TypeSemaphore(highPriorityPermits, "high"); - this.lowPriorityPermits = availablePermits - highPriorityPermits; - this.lowPrioritySemaphore = new TypeSemaphore(lowPriorityPermits, "low"); + public TransferSemaphoresHolder( + int availablePermits, + double priorityPermitAllocation, + int acquireWaitDuration, + TimeUnit timeUnit, + GenericStatsMetricPublisher genericStatsPublisher + ) { + + this.normalPriorityPermits = (int) (priorityPermitAllocation * availablePermits); + this.normalPrioritySemaphore = new TypeSemaphore( + normalPriorityPermits, + TypeSemaphore.PermitType.NORMAL, + genericStatsPublisher::updateNormalPermits + ); + this.lowPriorityPermits = availablePermits - normalPriorityPermits; + this.lowPrioritySemaphore = new TypeSemaphore( + lowPriorityPermits, + TypeSemaphore.PermitType.LOW, + genericStatsPublisher::updateLowPermits + ); this.acquireWaitDuration = acquireWaitDuration; this.acquireWaitDurationUnit = timeUnit; } @@ -46,15 +62,46 @@ public TransferSemaphoresHolder(int availablePermits, double priorityPermitAlloc * Overridden semaphore to identify transfer semaphores among all other semaphores for triaging. */ public static class TypeSemaphore extends Semaphore { - private final String type; + private final PermitType permitType; + private final Consumer permitChangeConsumer; - public TypeSemaphore(int permits, String type) { + public enum PermitType { + NORMAL, + LOW; + } + + public TypeSemaphore(int permits, PermitType permitType, Consumer permitChangeConsumer) { super(permits); - this.type = type; + 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; } - public String getType() { - return type; + @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); } } @@ -94,7 +141,7 @@ public TypeSemaphore acquirePermit(WritePriority writePriority, RequestContext r () -> "Acquire permit request for transfer type: " + writePriority + ". Available high priority permits: " - + highPrioritySemaphore.availablePermits() + + normalPrioritySemaphore.availablePermits() + " and low priority permits: " + lowPrioritySemaphore.availablePermits() ); @@ -103,8 +150,8 @@ public TypeSemaphore acquirePermit(WritePriority writePriority, RequestContext r if (Objects.requireNonNull(writePriority) == WritePriority.LOW) { if (lowPrioritySemaphore.tryAcquire()) { return lowPrioritySemaphore; - } else if (highPrioritySemaphore.availablePermits() > 0.4 * highPriorityPermits && highPrioritySemaphore.tryAcquire()) { - return highPrioritySemaphore; + } else if (normalPrioritySemaphore.availablePermits() > 0.4 * normalPriorityPermits && normalPrioritySemaphore.tryAcquire()) { + return normalPrioritySemaphore; } else if (lowPrioritySemaphore.tryAcquire(acquireWaitDuration, acquireWaitDurationUnit)) { return lowPrioritySemaphore; } @@ -113,12 +160,12 @@ public TypeSemaphore acquirePermit(WritePriority writePriority, RequestContext r // Try acquiring high priority permit or low priority permit immediately if available. // Otherwise, we wait for high priority permit. - if (highPrioritySemaphore.tryAcquire()) { - return highPrioritySemaphore; + if (normalPrioritySemaphore.tryAcquire()) { + return normalPrioritySemaphore; } else if (requestContext.lowPriorityPermitsConsumable && lowPrioritySemaphore.tryAcquire()) { return lowPrioritySemaphore; - } else if (highPrioritySemaphore.tryAcquire(acquireWaitDuration, acquireWaitDurationUnit)) { - return highPrioritySemaphore; + } else if (normalPrioritySemaphore.tryAcquire(acquireWaitDuration, acquireWaitDurationUnit)) { + return normalPrioritySemaphore; } return null; } @@ -126,8 +173,8 @@ public TypeSemaphore acquirePermit(WritePriority writePriority, RequestContext r /** * Used in tests. */ - public int getHighPriorityPermits() { - return highPriorityPermits; + public int getNormalPriorityPermits() { + return normalPriorityPermits; } /** @@ -136,18 +183,4 @@ public int getHighPriorityPermits() { public int getLowPriorityPermits() { return lowPriorityPermits; } - - /** - * Used in tests. - */ - public int getAvailableHighPriorityPermits() { - return highPrioritySemaphore.availablePermits(); - } - - /** - * Used in tests. - */ - public int getAvailableLowPriorityPermits() { - return lowPrioritySemaphore.availablePermits(); - } } 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 d5f664b1f0608..9b15db358dbb9 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 @@ -377,15 +377,20 @@ public void setUp() throws Exception { transferQueueConsumerService = Executors.newFixedThreadPool(20); scheduler = new Scheduler.SafeScheduledThreadPoolExecutor(1); transferNIOGroup = new AsyncTransferEventLoopGroup(1); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 10L, ByteSizeUnit.GB), transferQueueConsumerService, - 10 + 10, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL ); lowPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 20L, ByteSizeUnit.GB), transferQueueConsumerService, - 5 + 5, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL ); normalPrioritySizeBasedBlockingQ.start(); lowPrioritySizeBasedBlockingQ.start(); @@ -443,13 +448,20 @@ private S3BlobStore createBlobStore() { asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), - new TransferSemaphoresHolder(3, Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES) + new TransferSemaphoresHolder( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 5, + TimeUnit.MINUTES, + new GenericStatsMetricPublisher() + ) ), asyncExecutorContainer, asyncExecutorContainer, asyncExecutorContainer, normalPrioritySizeBasedBlockingQ, - lowPrioritySizeBasedBlockingQ + lowPrioritySizeBasedBlockingQ, + new GenericStatsMetricPublisher() ); } @@ -625,7 +637,14 @@ private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException, W int numberOfParts = 20; final ByteSizeValue partSize = new ByteSizeValue(capacity.getBytes() / numberOfParts + 1, ByteSizeUnit.BYTES); - SizeBasedBlockingQ sizeBasedBlockingQ = new SizeBasedBlockingQ(capacity, transferQueueConsumerService, 10); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); + 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; 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 f2f87231e9620..b65aee5fb8028 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 @@ -137,15 +137,20 @@ public void setUp() throws Exception { remoteTransferRetry = Executors.newFixedThreadPool(20); transferQueueConsumerService = Executors.newFixedThreadPool(2); scheduler = new ScheduledThreadPoolExecutor(1); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 5L, ByteSizeUnit.GB), transferQueueConsumerService, - 2 + 2, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.NORMAL ); lowPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 5L, ByteSizeUnit.GB), transferQueueConsumerService, - 2 + 2, + genericStatsMetricPublisher, + SizeBasedBlockingQ.QueueEventType.LOW ); normalPrioritySizeBasedBlockingQ.start(); lowPrioritySizeBasedBlockingQ.start(); @@ -252,13 +257,20 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), asyncExecutorContainer.getStreamReader(), - new TransferSemaphoresHolder(3, Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES) + new TransferSemaphoresHolder( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 5, + TimeUnit.MINUTES, + new GenericStatsMetricPublisher() + ) ), asyncExecutorContainer, asyncExecutorContainer, asyncExecutorContainer, normalPrioritySizeBasedBlockingQ, - lowPrioritySizeBasedBlockingQ + lowPrioritySizeBasedBlockingQ, + new 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 cc7aad76fcd0c..26ce3a29f36fa 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 @@ -172,7 +172,8 @@ private S3Repository createS3Repo(RepositoryMetadata metadata) { false, null, null, - null + null, + new GenericStatsMetricPublisher() ) { @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 3a2712c0feaaf..f874f1b43849a 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; @@ -69,7 +70,13 @@ public void setUp() throws Exception { Executors.newSingleThreadExecutor(), Executors.newSingleThreadExecutor(), Executors.newSingleThreadExecutor(), - new TransferSemaphoresHolder(3, Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES) + new TransferSemaphoresHolder( + 3, + Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), + 5, + TimeUnit.MINUTES, + new 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 index 53f507c72784a..6dc86b6d5b227 100644 --- 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 @@ -10,6 +10,7 @@ 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; @@ -40,11 +41,16 @@ public void tearDown() throws Exception { } public void testProducerConsumerOfBulkItems() throws InterruptedException { - + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); + SizeBasedBlockingQ.QueueEventType queueEventType = randomBoolean() + ? SizeBasedBlockingQ.QueueEventType.NORMAL + : SizeBasedBlockingQ.QueueEventType.LOW; SizeBasedBlockingQ sizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(ByteSizeUnit.BYTES.toBytes(10)), consumerService, - 10 + 10, + genericStatsMetricPublisher, + queueEventType ); sizeBasedBlockingQ.start(); int numOfItems = randomIntBetween(100, 1000); @@ -76,6 +82,8 @@ public void testProducerConsumerOfBulkItems() throws InterruptedException { latch.await(); sizeBasedBlockingQ.close(); assertFalse(unknownError.get()); + assertEquals(0L, genericStatsMetricPublisher.getNormalPriorityQSize()); + assertEquals(0L, genericStatsMetricPublisher.getLowPriorityQSize()); } static class TestItemToStr extends SizeBasedBlockingQ.Item { 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 index bcec8ece062cf..01510aac86330 100644 --- 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 @@ -9,6 +9,7 @@ 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; @@ -19,39 +20,47 @@ 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 highPermits = (int) (availablePermits * priorityAllocation); - int lowPermits = availablePermits - highPermits; + int normalPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - normalPermits; + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(); TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( availablePermits, priorityAllocation, 1, - TimeUnit.NANOSECONDS + TimeUnit.NANOSECONDS, + genericStatsPublisher ); - assertEquals(highPermits, transferSemaphoresHolder.getHighPriorityPermits()); + 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 highPermits = (int) (availablePermits * priorityAllocation); - int lowPermits = availablePermits - highPermits; + int normalPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - normalPermits; + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(); TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( availablePermits, priorityAllocation, 1, - TimeUnit.NANOSECONDS + TimeUnit.NANOSECONDS, + genericStatsPublisher ); List semaphores = new ArrayList<>(); - int highPermitsEligibleForLowEvents = highPermits - (int) (highPermits * 0.4); + int normalPermitsEligibleForLowEvents = normalPermits - (int) (normalPermits * 0.4); - int lowAcquisitionsExpected = (highPermitsEligibleForLowEvents + lowPermits); + int lowAcquisitionsExpected = (normalPermitsEligibleForLowEvents + lowPermits); for (int i = 0; i < lowAcquisitionsExpected; i++) { TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( @@ -60,19 +69,19 @@ public void testLowPriorityEventPermitAcquisition() throws InterruptedException ); semaphores.add(acquiredSemaphore); if (i >= lowPermits) { - assertEquals("high", acquiredSemaphore.getType()); + assertEquals(PermitType.NORMAL, acquiredSemaphore.getType()); } else { - assertEquals("low", acquiredSemaphore.getType()); + assertEquals(PermitType.LOW, acquiredSemaphore.getType()); } } - for (int i = 0; i < highPermits - highPermitsEligibleForLowEvents; i++) { + for (int i = 0; i < normalPermits - normalPermitsEligibleForLowEvents; i++) { TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( - WritePriority.HIGH, + WritePriority.NORMAL, requestContext ); - assertEquals("high", acquiredSemaphore.getType()); + assertEquals(PermitType.NORMAL, acquiredSemaphore.getType()); semaphores.add(acquiredSemaphore); } @@ -85,44 +94,51 @@ public void testLowPriorityEventPermitAcquisition() throws InterruptedException assertEquals(availablePermits, semaphores.size()); semaphores.forEach(Semaphore::release); - assertEquals(highPermits, transferSemaphoresHolder.getHighPriorityPermits()); + assertEquals(normalPermits, transferSemaphoresHolder.getNormalPriorityPermits()); assertEquals(lowPermits, transferSemaphoresHolder.getLowPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredNormalPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredLowPriorityPermits()); } - public void testHighPermitEventAcquisition() throws InterruptedException { + public void testNormalPermitEventAcquisition() throws InterruptedException { int availablePermits = randomIntBetween(5, 50); double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); - int highPermits = (int) (availablePermits * priorityAllocation); - int lowPermits = availablePermits - highPermits; + int normalPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - normalPermits; + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(); TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( availablePermits, priorityAllocation, 1, - TimeUnit.NANOSECONDS + TimeUnit.NANOSECONDS, + genericStatsPublisher ); List semaphores = new ArrayList<>(); List lowSemaphores = new ArrayList<>(); - int highAcquisitionsExpected = highPermits + lowPermits; + int normalAcquisitionsExpected = normalPermits + lowPermits; TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); - for (int i = 0; i < highAcquisitionsExpected; i++) { + for (int i = 0; i < normalAcquisitionsExpected; i++) { TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( - WritePriority.HIGH, + WritePriority.NORMAL, requestContext ); semaphores.add(acquiredSemaphore); - if (i >= highPermits) { - assertEquals("low", acquiredSemaphore.getType()); + if (i >= normalPermits) { + assertEquals(PermitType.LOW, acquiredSemaphore.getType()); lowSemaphores.add(acquiredSemaphore); } else { - assertEquals("high", acquiredSemaphore.getType()); + assertEquals(PermitType.NORMAL, acquiredSemaphore.getType()); } } assertEquals(availablePermits, semaphores.size()); int lowAcquired = lowPermits; - lowSemaphores.get(0).release(); + + Semaphore removedLowSemaphore = lowSemaphores.remove(0); + removedLowSemaphore.release(); + semaphores.remove(removedLowSemaphore); requestContext = transferSemaphoresHolder.createRequestContext(); TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( @@ -133,18 +149,20 @@ public void testHighPermitEventAcquisition() throws InterruptedException { lowSemaphores.add(acquiredSemaphore); while (lowAcquired > 1) { requestContext = transferSemaphoresHolder.createRequestContext(); - acquiredSemaphore = transferSemaphoresHolder.acquirePermit(WritePriority.HIGH, requestContext); + acquiredSemaphore = transferSemaphoresHolder.acquirePermit(WritePriority.NORMAL, requestContext); assertNull(acquiredSemaphore); lowAcquired--; } semaphores.forEach(Semaphore::release); - assertEquals(highPermits, transferSemaphoresHolder.getHighPriorityPermits()); + assertEquals(normalPermits, transferSemaphoresHolder.getNormalPriorityPermits()); assertEquals(lowPermits, transferSemaphoresHolder.getLowPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredNormalPriorityPermits()); + assertEquals(0, genericStatsPublisher.getAcquiredLowPriorityPermits()); } private static class TestTransferSemaphoresHolder extends TransferSemaphoresHolder { - AtomicInteger highWaitCount = new AtomicInteger(); + AtomicInteger normalWaitCount = new AtomicInteger(); AtomicInteger lowWaitCount = new AtomicInteger(); /** @@ -154,20 +172,21 @@ public TestTransferSemaphoresHolder( int availablePermits, double priorityPermitAllocation, int acquireWaitDuration, - TimeUnit timeUnit + TimeUnit timeUnit, + GenericStatsMetricPublisher genericStatsMetricPublisher ) throws InterruptedException { - super(availablePermits, priorityPermitAllocation, acquireWaitDuration, timeUnit); - TypeSemaphore executingHighSemaphore = highPrioritySemaphore; + super(availablePermits, priorityPermitAllocation, acquireWaitDuration, timeUnit, genericStatsMetricPublisher); + TypeSemaphore executingNormalSemaphore = normalPrioritySemaphore; TypeSemaphore executingLowSemaphore = lowPrioritySemaphore; - this.highPrioritySemaphore = Mockito.spy(highPrioritySemaphore); + this.normalPrioritySemaphore = Mockito.spy(normalPrioritySemaphore); this.lowPrioritySemaphore = Mockito.spy(lowPrioritySemaphore); Mockito.doAnswer(invocation -> { - highWaitCount.incrementAndGet(); + normalWaitCount.incrementAndGet(); return false; - }).when(highPrioritySemaphore).tryAcquire(Mockito.anyLong(), Mockito.any(TimeUnit.class)); - Mockito.doAnswer(invocation -> executingHighSemaphore.availablePermits()).when(highPrioritySemaphore).availablePermits(); - Mockito.doAnswer(invocation -> executingHighSemaphore.tryAcquire()).when(highPrioritySemaphore).tryAcquire(); + }).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(); @@ -178,58 +197,60 @@ public TestTransferSemaphoresHolder( } } - public void testHighSemaphoreAcquiredWait() throws InterruptedException { + public void testNormalSemaphoreAcquiredWait() throws InterruptedException { int availablePermits = randomIntBetween(10, 50); double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); - int highPermits = (int) (availablePermits * priorityAllocation); + int normalPermits = (int) (availablePermits * priorityAllocation); TestTransferSemaphoresHolder transferSemaphoresHolder = new TestTransferSemaphoresHolder( availablePermits, priorityAllocation, 5, - TimeUnit.MINUTES + TimeUnit.MINUTES, + new GenericStatsMetricPublisher() ); TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); TransferSemaphoresHolder.TypeSemaphore lowSemaphore = transferSemaphoresHolder.acquirePermit(WritePriority.LOW, requestContext); - assertEquals("low", lowSemaphore.getType()); - for (int i = 0; i < highPermits; i++) { + assertEquals(PermitType.LOW, lowSemaphore.getType()); + for (int i = 0; i < normalPermits; i++) { requestContext = transferSemaphoresHolder.createRequestContext(); TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( - WritePriority.HIGH, + WritePriority.NORMAL, requestContext ); - assertEquals("high", acquiredSemaphore.getType()); + assertEquals(PermitType.NORMAL, acquiredSemaphore.getType()); } TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( - WritePriority.HIGH, + WritePriority.NORMAL, requestContext ); assertNull(acquiredSemaphore); - assertEquals(1, transferSemaphoresHolder.highWaitCount.get()); + 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 highPermits = (int) (availablePermits * priorityAllocation); - int lowPermits = availablePermits - highPermits; + int normalPermits = (int) (availablePermits * priorityAllocation); + int lowPermits = availablePermits - normalPermits; TestTransferSemaphoresHolder transferSemaphoresHolder = new TestTransferSemaphoresHolder( availablePermits, priorityAllocation, 5, - TimeUnit.MINUTES + TimeUnit.MINUTES, + new GenericStatsMetricPublisher() ); TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); - int highPermitsEligibleForLowEvents = highPermits - (int) (highPermits * 0.4); - for (int i = 0; i < highPermitsEligibleForLowEvents; i++) { + int normalPermitsEligibleForLowEvents = normalPermits - (int) (normalPermits * 0.4); + for (int i = 0; i < normalPermitsEligibleForLowEvents; i++) { TransferSemaphoresHolder.TypeSemaphore lowSemaphore = transferSemaphoresHolder.acquirePermit( - WritePriority.HIGH, + WritePriority.NORMAL, requestContext ); - assertEquals("high", lowSemaphore.getType()); + assertEquals(PermitType.NORMAL, lowSemaphore.getType()); } for (int i = 0; i < lowPermits; i++) { @@ -238,7 +259,7 @@ public void testLowSemaphoreAcquiredWait() throws InterruptedException { WritePriority.LOW, requestContext ); - assertEquals("low", acquiredSemaphore.getType()); + assertEquals(PermitType.LOW, acquiredSemaphore.getType()); } TransferSemaphoresHolder.TypeSemaphore acquiredSemaphore = transferSemaphoresHolder.acquirePermit( @@ -247,7 +268,7 @@ public void testLowSemaphoreAcquiredWait() throws InterruptedException { ); assertNull(acquiredSemaphore); assertEquals(1, transferSemaphoresHolder.lowWaitCount.get()); - assertEquals(0, transferSemaphoresHolder.highWaitCount.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..d702b2776c3e7 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobStore.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobStore.java @@ -78,7 +78,8 @@ enum Metric { REQUEST_SUCCESS("request_success_total"), REQUEST_FAILURE("request_failures_total"), REQUEST_LATENCY("request_time_in_millis"), - RETRY_COUNT("request_retry_count_total"); + RETRY_COUNT("request_retry_count_total"), + GENERIC_STATS("generic_stats"); private String metricName; From 8c54c68bfb6019611e2477bcde2b0a6183dc4cd0 Mon Sep 17 00:00:00 2001 From: vikasvb90 Date: Sat, 11 May 2024 17:26:52 +0530 Subject: [PATCH 5/5] Converted generic repository stats from absolute to percent utilization in repository-s3 Signed-off-by: vikasvb90 --- .../s3/S3BlobStoreRepositoryTests.java | 1 + .../s3/GenericStatsMetricPublisher.java | 24 ++++++++++-- .../repositories/s3/S3Repository.java | 5 ++- .../repositories/s3/S3RepositoryPlugin.java | 34 +++++++++++++---- .../s3/async/TransferSemaphoresHolder.java | 8 ++-- .../s3/RepositoryCredentialsTests.java | 1 + .../s3/S3BlobContainerMockClientTests.java | 10 ++--- .../s3/S3BlobContainerRetriesTests.java | 8 ++-- .../repositories/s3/S3RepositoryTests.java | 2 +- .../s3/async/AsyncTransferManagerTests.java | 3 +- .../s3/async/SizeBasedBlockingQTests.java | 2 +- .../async/TransferSemaphoresHolderTests.java | 38 ++++++++++--------- .../common/blobstore/BlobStore.java | 4 +- 13 files changed, 91 insertions(+), 49 deletions(-) diff --git a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java index b001ac878f5df..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 @@ -262,6 +262,7 @@ protected S3Repository createRepository( null, false, null, + null, null ) { 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 index 8a37dc06033fa..136fd68223354 100644 --- 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 @@ -22,6 +22,22 @@ public class GenericStatsMetricPublisher { 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); @@ -65,10 +81,10 @@ public int getAcquiredLowPriorityPermits() { Map stats() { final Map results = new HashMap<>(); - results.put("NormalPriorityQSize", normalPriorityQSize.get()); - results.put("LowPriorityQSize", lowPriorityQSize.get()); - results.put("AcquiredNormalPriorityPermits", (long) normalPriorityPermits.get()); - results.put("AcquiredLowPriorityPermits", (long) lowPriorityPermits.get()); + 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/S3Repository.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java index 80be255b33e9b..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 @@ -314,7 +314,8 @@ class S3Repository extends MeteredBlobStoreRepository { final S3AsyncService s3AsyncService, final boolean multipartUploadEnabled, final SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ, - final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ + final SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ, + final GenericStatsMetricPublisher genericStatsMetricPublisher ) { this( metadata, @@ -331,7 +332,7 @@ class S3Repository extends MeteredBlobStoreRepository { Path.of(""), normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ, - new GenericStatsMetricPublisher() + 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 cbb1488b520c2..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 @@ -105,7 +105,7 @@ public class S3RepositoryPlugin extends Plugin implements RepositoryPlugin, Relo private SizeBasedBlockingQ normalPrioritySizeBasedBlockingQ; private SizeBasedBlockingQ lowPrioritySizeBasedBlockingQ; private TransferSemaphoresHolder transferSemaphoresHolder; - private GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); + private GenericStatsMetricPublisher genericStatsMetricPublisher; public S3RepositoryPlugin(final Settings settings, final Path configPath) { this(settings, configPath, new S3Service(configPath), new S3AsyncService(configPath)); @@ -231,28 +231,48 @@ public Collection createComponents( threadPool.executor(STREAM_READER), new AsyncTransferEventLoopGroup(normalEventLoopThreads) ); + 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( - new ByteSizeValue(normalPriorityConsumers * 10L, ByteSizeUnit.GB), + normalPriorityQCapacity, normalTransferQConsumerService, normalPriorityConsumers, genericStatsMetricPublisher, SizeBasedBlockingQ.QueueEventType.NORMAL ); - int lowPriorityConsumers = lowPriorityTransferQConsumers(clusterService.getSettings()); + LowPrioritySizeBasedBlockingQ lowPrioritySizeBasedBlockingQ = new LowPrioritySizeBasedBlockingQ( - new ByteSizeValue(lowPriorityConsumers * 20L, ByteSizeUnit.GB), + lowPriorityQCapacity, lowTransferQConsumerService, lowPriorityConsumers, genericStatsMetricPublisher ); this.lowPrioritySizeBasedBlockingQ = lowPrioritySizeBasedBlockingQ; this.transferSemaphoresHolder = new TransferSemaphoresHolder( - // High number of permit allocation because each op acquiring permit performs disk IO, computation and network IO. - Math.max(allocatedProcessors(clusterService.getSettings()) * 4, 10), - ((double) S3Repository.S3_PRIORITY_PERMIT_ALLOCATION_PERCENT.get(clusterService.getSettings())) / 100, + normalPriorityPermits, + lowPriorityPermits, S3Repository.S3_PERMIT_WAIT_DURATION_MIN.get(clusterService.getSettings()), TimeUnit.MINUTES, genericStatsMetricPublisher 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 index ce3b8929c35f4..7dccedb8d5278 100644 --- 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 @@ -35,20 +35,20 @@ public class TransferSemaphoresHolder { * Constructor to create semaphores holder. */ public TransferSemaphoresHolder( - int availablePermits, - double priorityPermitAllocation, + int normalPriorityPermits, + int lowPriorityPermits, int acquireWaitDuration, TimeUnit timeUnit, GenericStatsMetricPublisher genericStatsPublisher ) { - this.normalPriorityPermits = (int) (priorityPermitAllocation * availablePermits); + this.normalPriorityPermits = normalPriorityPermits; + this.lowPriorityPermits = lowPriorityPermits; this.normalPrioritySemaphore = new TypeSemaphore( normalPriorityPermits, TypeSemaphore.PermitType.NORMAL, genericStatsPublisher::updateNormalPermits ); - this.lowPriorityPermits = availablePermits - normalPriorityPermits; this.lowPrioritySemaphore = new TypeSemaphore( lowPriorityPermits, TypeSemaphore.PermitType.LOW, 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 16dbe05dcaee8..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 @@ -316,6 +316,7 @@ protected S3Repository createRepository( null, false, null, + null, null ) { @Override 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 9b15db358dbb9..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 @@ -377,7 +377,7 @@ public void setUp() throws Exception { transferQueueConsumerService = Executors.newFixedThreadPool(20); scheduler = new Scheduler.SafeScheduledThreadPoolExecutor(1); transferNIOGroup = new AsyncTransferEventLoopGroup(1); - GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 10L, ByteSizeUnit.GB), transferQueueConsumerService, @@ -431,7 +431,7 @@ private S3BlobStore createBlobStore() { streamReaderService, transferNIOGroup ); - + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); return new S3BlobStore( null, asyncService, @@ -453,7 +453,7 @@ private S3BlobStore createBlobStore() { Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES, - new GenericStatsMetricPublisher() + genericStatsMetricPublisher ) ), asyncExecutorContainer, @@ -461,7 +461,7 @@ private S3BlobStore createBlobStore() { asyncExecutorContainer, normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ, - new GenericStatsMetricPublisher() + genericStatsMetricPublisher ); } @@ -637,7 +637,7 @@ private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException, W int numberOfParts = 20; final ByteSizeValue partSize = new ByteSizeValue(capacity.getBytes() / numberOfParts + 1, ByteSizeUnit.BYTES); - GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); SizeBasedBlockingQ sizeBasedBlockingQ = new SizeBasedBlockingQ( capacity, transferQueueConsumerService, 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 b65aee5fb8028..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 @@ -137,7 +137,7 @@ public void setUp() throws Exception { remoteTransferRetry = Executors.newFixedThreadPool(20); transferQueueConsumerService = Executors.newFixedThreadPool(2); scheduler = new ScheduledThreadPoolExecutor(1); - GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); normalPrioritySizeBasedBlockingQ = new SizeBasedBlockingQ( new ByteSizeValue(Runtime.getRuntime().availableProcessors() * 5L, ByteSizeUnit.GB), transferQueueConsumerService, @@ -238,7 +238,7 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( streamReaderService, transferNIOGroup ); - + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); return new S3BlobContainer( BlobPath.cleanPath(), new S3BlobStore( @@ -262,7 +262,7 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES, - new GenericStatsMetricPublisher() + genericStatsMetricPublisher ) ), asyncExecutorContainer, @@ -270,7 +270,7 @@ protected AsyncMultiStreamBlobContainer createBlobContainer( asyncExecutorContainer, normalPrioritySizeBasedBlockingQ, lowPrioritySizeBasedBlockingQ, - new GenericStatsMetricPublisher() + 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 26ce3a29f36fa..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 @@ -173,7 +173,7 @@ private S3Repository createS3Repo(RepositoryMetadata metadata) { null, null, null, - new GenericStatsMetricPublisher() + 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 f874f1b43849a..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 @@ -65,6 +65,7 @@ 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(), @@ -75,7 +76,7 @@ public void setUp() throws Exception { Math.max(Runtime.getRuntime().availableProcessors() * 5, 10), 5, TimeUnit.MINUTES, - new GenericStatsMetricPublisher() + 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 index 6dc86b6d5b227..5be4037407d23 100644 --- 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 @@ -41,7 +41,7 @@ public void tearDown() throws Exception { } public void testProducerConsumerOfBulkItems() throws InterruptedException { - GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(); + GenericStatsMetricPublisher genericStatsMetricPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); SizeBasedBlockingQ.QueueEventType queueEventType = randomBoolean() ? SizeBasedBlockingQ.QueueEventType.NORMAL : SizeBasedBlockingQ.QueueEventType.LOW; 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 index 01510aac86330..236f02c5eb1f7 100644 --- 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 @@ -29,10 +29,10 @@ public void testAllocation() { double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); int normalPermits = (int) (availablePermits * priorityAllocation); int lowPermits = availablePermits - normalPermits; - GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(); + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( - availablePermits, - priorityAllocation, + normalPermits, + lowPermits, 1, TimeUnit.NANOSECONDS, genericStatsPublisher @@ -48,10 +48,10 @@ public void testLowPriorityEventPermitAcquisition() throws InterruptedException double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); int normalPermits = (int) (availablePermits * priorityAllocation); int lowPermits = availablePermits - normalPermits; - GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(); + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( - availablePermits, - priorityAllocation, + normalPermits, + lowPermits, 1, TimeUnit.NANOSECONDS, genericStatsPublisher @@ -106,10 +106,10 @@ public void testNormalPermitEventAcquisition() throws InterruptedException { double priorityAllocation = randomDoubleBetween(0.1, 0.9, true); int normalPermits = (int) (availablePermits * priorityAllocation); int lowPermits = availablePermits - normalPermits; - GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(); + GenericStatsMetricPublisher genericStatsPublisher = new GenericStatsMetricPublisher(10000L, 10, 10000L, 10); TransferSemaphoresHolder transferSemaphoresHolder = new TransferSemaphoresHolder( - availablePermits, - priorityAllocation, + normalPermits, + lowPermits, 1, TimeUnit.NANOSECONDS, genericStatsPublisher @@ -169,13 +169,13 @@ private static class TestTransferSemaphoresHolder extends TransferSemaphoresHold * Constructor to create semaphores holder. */ public TestTransferSemaphoresHolder( - int availablePermits, - double priorityPermitAllocation, + int normalPermits, + int lowPermits, int acquireWaitDuration, TimeUnit timeUnit, GenericStatsMetricPublisher genericStatsMetricPublisher ) throws InterruptedException { - super(availablePermits, priorityPermitAllocation, acquireWaitDuration, timeUnit, genericStatsMetricPublisher); + super(normalPermits, lowPermits, acquireWaitDuration, timeUnit, genericStatsMetricPublisher); TypeSemaphore executingNormalSemaphore = normalPrioritySemaphore; TypeSemaphore executingLowSemaphore = lowPrioritySemaphore; @@ -201,12 +201,13 @@ 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( - availablePermits, - priorityAllocation, + normalPermits, + availablePermits - normalPermits, 5, TimeUnit.MINUTES, - new GenericStatsMetricPublisher() + genericStatsPublisher ); TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); @@ -235,12 +236,13 @@ public void testLowSemaphoreAcquiredWait() throws InterruptedException { 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( - availablePermits, - priorityAllocation, + normalPermits, + lowPermits, 5, TimeUnit.MINUTES, - new GenericStatsMetricPublisher() + genericStatsPublisher ); TransferSemaphoresHolder.RequestContext requestContext = transferSemaphoresHolder.createRequestContext(); 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 d702b2776c3e7..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,11 +75,11 @@ 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"), - RETRY_COUNT("request_retry_count_total"), - GENERIC_STATS("generic_stats"); + RETRY_COUNT("request_retry_count_total"); private String metricName;