From 38f81ebfeebba2ff2dd2d8b3979327ca45058699 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Fri, 17 May 2024 18:43:18 +0530 Subject: [PATCH 01/33] Optimise S3 storage writing for MSQ durable storage --- .../s3/output/RetryableS3OutputStream.java | 175 +++++++++++++----- .../druid/storage/StorageConnector.java | 2 +- 2 files changed, 130 insertions(+), 47 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index c71bb4e788b7..fe2a678bd21f 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -24,7 +24,6 @@ import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; -import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; @@ -49,7 +48,13 @@ import java.util.List; import java.util.Objects; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; /** * A retryable output stream for s3. How it works is: @@ -89,12 +94,13 @@ public class RetryableS3OutputStream extends OutputStream private Chunk currentChunk; private int nextChunkId = 1; // multipart upload requires partNumber to be in the range between 1 and 10000 - private int numChunksPushed; + private final AtomicInteger numChunksPushed = new AtomicInteger(0); + /** * Total size of all chunks. This size is updated whenever the chunk is ready for push, * not when {@link #write(byte[], int, int)} is called. */ - private long resultsSize; + private final AtomicLong resultsSize = new AtomicLong(0); /** * A flag indicating whether there was an upload error. @@ -103,13 +109,35 @@ public class RetryableS3OutputStream extends OutputStream private boolean error; private boolean closed; + /** + * A map to store number of files pending to be uploaded for a given uploadId. + */ + private static final ConcurrentHashMap PENDING_FILES = new ConcurrentHashMap<>(); + + /** + * A map containing the lock for a given uploadId used for notifying the main thread about the completion of + * s3.uploadPart() for all chunks for the uploadId, and hence starting the s3.completeMultipartUpload() for + * the uploadId. + */ + private static final ConcurrentHashMap FILE_LOCK_MAP = new ConcurrentHashMap<>(); + + /** + * Semaphore to restrict the maximum number of simultaneous chunks on disk. + */ + private static final int MAX_CONCURRENT_CHUNKS = 10; + private static final Semaphore SEMAPHORE = new Semaphore(MAX_CONCURRENT_CHUNKS); + + /** + * Threadpool used for uploading the chunks asynchronously. + */ + private static final ExecutorService UPLOAD_EXECUTOR = Executors.newFixedThreadPool(10); + public RetryableS3OutputStream( S3OutputConfig config, ServerSideEncryptingAmazonS3 s3, String s3Key ) throws IOException { - this(config, s3, s3Key, true); } @@ -138,9 +166,7 @@ protected RetryableS3OutputStream( this.chunkStorePath = new File(config.getTempDir(), uploadId + UUID.randomUUID()); FileUtils.mkdirp(this.chunkStorePath); this.chunkSize = config.getChunkSize(); - this.pushStopwatch = Stopwatch.createUnstarted(); - this.pushStopwatch.reset(); - + this.pushStopwatch = Stopwatch.createStarted(); this.currentChunk = new Chunk(nextChunkId, new File(chunkStorePath, String.valueOf(nextChunkId++))); } @@ -199,15 +225,49 @@ private void pushCurrentChunk() throws IOException { currentChunk.close(); final Chunk chunk = currentChunk; - try { - if (chunk.length() > 0) { - resultsSize += chunk.length(); + if (chunk.length() > 0) { + try { + SEMAPHORE.acquire(); // Acquire a permit from the semaphore + PENDING_FILES.computeIfAbsent(uploadId, f -> new AtomicInteger(0)).incrementAndGet(); + + UPLOAD_EXECUTOR.submit(() -> { + try { + uploadChunk(chunk); + } + catch (Exception e) { + error = true; + LOG.error(e, e.getMessage()); + throw new RuntimeException(e); + } + finally { + SEMAPHORE.release(); // Release the permit after upload is completed + AtomicInteger counter = PENDING_FILES.get(uploadId); + int remaining = counter.decrementAndGet(); + if (remaining == 0) { + synchronized (FILE_LOCK_MAP.computeIfAbsent(uploadId, f -> new Object())) { + FILE_LOCK_MAP.get(uploadId).notifyAll(); + } + } + } + }); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } + } + } - pushStopwatch.start(); - pushResults.add(push(chunk)); - pushStopwatch.stop(); - numChunksPushed++; + private void uploadChunk(Chunk chunk) throws IOException + { + try { + LOG.info("Uploading chunk [%d] for uploadId [%s].", chunk.id, uploadId); + resultsSize.addAndGet(chunk.length()); + final PartETag partETag = push(chunk); + synchronized (pushResults) { + pushResults.add(partETag); } + numChunksPushed.incrementAndGet(); } finally { if (!chunk.delete()) { @@ -240,8 +300,6 @@ private PartETag uploadPartIfPossible( Chunk chunk ) { - final ObjectMetadata objectMetadata = new ObjectMetadata(); - objectMetadata.setContentLength(resultsSize); final UploadPartRequest uploadPartRequest = new UploadPartRequest() .withUploadId(uploadId) .withBucketName(bucket) @@ -271,50 +329,75 @@ public void close() throws IOException // This should be emitted as a metric LOG.info( "Pushed total [%d] parts containing [%d] bytes in [%d]ms.", - numChunksPushed, - resultsSize, + numChunksPushed.get(), + resultsSize.get(), pushStopwatch.elapsed(TimeUnit.MILLISECONDS) ); }); - closer.register(() -> org.apache.commons.io.FileUtils.forceDelete(chunkStorePath)); + try (Closer ignored = closer) { + if (!error) { + pushCurrentChunk(); + completeMultipartUpload(); + } + } + } - closer.register(() -> { - try { - if (resultsSize > 0 && isAllPushSucceeded()) { - RetryUtils.retry( - () -> s3.completeMultipartUpload( - new CompleteMultipartUploadRequest(config.getBucket(), s3Key, uploadId, pushResults) - ), - S3Utils.S3RETRY, - config.getMaxRetry() - ); - } else { - RetryUtils.retry( - () -> { - s3.cancelMultiPartUpload(new AbortMultipartUploadRequest(config.getBucket(), s3Key, uploadId)); - return null; - }, - S3Utils.S3RETRY, - config.getMaxRetry() - ); + private void completeMultipartUpload() + { + Object fileLock = FILE_LOCK_MAP.computeIfAbsent(uploadId, f -> new Object()); + synchronized (fileLock) { + while (PENDING_FILES.getOrDefault(uploadId, new AtomicInteger(0)).get() > 0) { + try { + LOG.info("Waiting for lock for completing multipart task for uploadId [%s].", uploadId); + fileLock.wait(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); } } - catch (Exception e) { - throw new IOException(e); - } - }); + } - try (Closer ignored = closer) { - if (!error) { - pushCurrentChunk(); + try { + if (resultsSize.get() > 0 && isAllPushSucceeded()) { + RetryUtils.retry( + () -> s3.completeMultipartUpload( + new CompleteMultipartUploadRequest(config.getBucket(), s3Key, uploadId, pushResults) + ), + S3Utils.S3RETRY, + config.getMaxRetry() + ); + } else { + RetryUtils.retry( + () -> { + s3.cancelMultiPartUpload(new AbortMultipartUploadRequest(config.getBucket(), s3Key, uploadId)); + return null; + }, + S3Utils.S3RETRY, + config.getMaxRetry() + ); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + try { + org.apache.commons.io.FileUtils.forceDelete(chunkStorePath); + pushStopwatch.stop(); + } + catch (IOException e) { + throw new RuntimeException(e); } + PENDING_FILES.remove(uploadId); + FILE_LOCK_MAP.remove(uploadId); } } private boolean isAllPushSucceeded() { - return !error && !pushResults.isEmpty() && numChunksPushed == pushResults.size(); + return !error && !pushResults.isEmpty() && numChunksPushed.get() == pushResults.size(); } private static class Chunk implements Closeable diff --git a/processing/src/main/java/org/apache/druid/storage/StorageConnector.java b/processing/src/main/java/org/apache/druid/storage/StorageConnector.java index d99d9469f08f..65e5c626d959 100644 --- a/processing/src/main/java/org/apache/druid/storage/StorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/StorageConnector.java @@ -42,7 +42,7 @@ *
  • {@code druid.extension.custom.type="s3"} *
  • {@code druid.extension.custom.bucket="myBucket"} * - * The final state of this inteface would have + * The final state of this interface would have *
      *
    1. Future Non blocking API's
    2. *
    3. Offset based fetch
    4. From bf518d8af15994570085b4b5dae6be1cc15c97b3 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 22 May 2024 08:37:00 +0530 Subject: [PATCH 02/33] Get rid of static ConcurrentHashMap --- .../s3/output/RetryableS3OutputStream.java | 27 +++++++------------ 1 file changed, 10 insertions(+), 17 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index fe2a678bd21f..4d6bb329c2b4 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -48,7 +48,6 @@ import java.util.List; import java.util.Objects; import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Semaphore; @@ -110,16 +109,15 @@ public class RetryableS3OutputStream extends OutputStream private boolean closed; /** - * A map to store number of files pending to be uploaded for a given uploadId. + * An atomic counter to store number of files pending to be uploaded for the particular uploadId. */ - private static final ConcurrentHashMap PENDING_FILES = new ConcurrentHashMap<>(); + private final AtomicInteger pendingFiles = new AtomicInteger(0); /** - * A map containing the lock for a given uploadId used for notifying the main thread about the completion of - * s3.uploadPart() for all chunks for the uploadId, and hence starting the s3.completeMultipartUpload() for - * the uploadId. + * A lock used for notifying the main thread about the completion of s3.uploadPart() for all chunks + * and hence starting the s3.completeMultipartUpload() for the uploadId. */ - private static final ConcurrentHashMap FILE_LOCK_MAP = new ConcurrentHashMap<>(); + private final Object fileLock = new Object(); /** * Semaphore to restrict the maximum number of simultaneous chunks on disk. @@ -228,7 +226,7 @@ private void pushCurrentChunk() throws IOException if (chunk.length() > 0) { try { SEMAPHORE.acquire(); // Acquire a permit from the semaphore - PENDING_FILES.computeIfAbsent(uploadId, f -> new AtomicInteger(0)).incrementAndGet(); + pendingFiles.incrementAndGet(); UPLOAD_EXECUTOR.submit(() -> { try { @@ -241,11 +239,9 @@ private void pushCurrentChunk() throws IOException } finally { SEMAPHORE.release(); // Release the permit after upload is completed - AtomicInteger counter = PENDING_FILES.get(uploadId); - int remaining = counter.decrementAndGet(); - if (remaining == 0) { - synchronized (FILE_LOCK_MAP.computeIfAbsent(uploadId, f -> new Object())) { - FILE_LOCK_MAP.get(uploadId).notifyAll(); + if (pendingFiles.decrementAndGet() == 0) { + synchronized (fileLock) { + fileLock.notifyAll(); } } } @@ -345,9 +341,8 @@ public void close() throws IOException private void completeMultipartUpload() { - Object fileLock = FILE_LOCK_MAP.computeIfAbsent(uploadId, f -> new Object()); synchronized (fileLock) { - while (PENDING_FILES.getOrDefault(uploadId, new AtomicInteger(0)).get() > 0) { + while (pendingFiles.get() > 0) { try { LOG.info("Waiting for lock for completing multipart task for uploadId [%s].", uploadId); fileLock.wait(); @@ -390,8 +385,6 @@ private void completeMultipartUpload() catch (IOException e) { throw new RuntimeException(e); } - PENDING_FILES.remove(uploadId); - FILE_LOCK_MAP.remove(uploadId); } } From a920f747306d5c76ce9d7a44f174a60e71af5d56 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 22 May 2024 09:15:07 +0530 Subject: [PATCH 03/33] Fix static checks --- .../druid/storage/s3/output/RetryableS3OutputStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index 4d6bb329c2b4..d2d026867355 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -33,6 +33,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.storage.s3.S3Utils; @@ -49,7 +50,6 @@ import java.util.Objects; import java.util.UUID; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -128,7 +128,7 @@ public class RetryableS3OutputStream extends OutputStream /** * Threadpool used for uploading the chunks asynchronously. */ - private static final ExecutorService UPLOAD_EXECUTOR = Executors.newFixedThreadPool(10); + private static final ExecutorService UPLOAD_EXECUTOR = Execs.multiThreaded(10, "UploadThreadPool-%d"); public RetryableS3OutputStream( S3OutputConfig config, From 38dc51c439f755e3b8dcd1902dd586f88b076e91 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 22 May 2024 11:15:25 +0530 Subject: [PATCH 04/33] Fix tests --- .../s3/output/RetryableS3OutputStreamTest.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java index 1f8eac3bbae0..5acf2e11dc40 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java @@ -36,7 +36,6 @@ import org.apache.druid.storage.s3.NoopServerSideEncryption; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; import org.easymock.EasyMock; -import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -50,6 +49,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; public class RetryableS3OutputStreamTest @@ -206,9 +206,7 @@ public void testFailToUploadAfterRetries() throws IOException out.write(bb.array()); } - expectedException.expect(RuntimeException.class); - expectedException.expectCause(CoreMatchers.instanceOf(AmazonClientException.class)); - expectedException.expectMessage("Upload failure test. Remaining failures [1]"); + // No exception is thrown in the main thread, since the upload now happens in a separate threadpool. bb.clear(); bb.putInt(3); out.write(bb.array()); @@ -275,8 +273,10 @@ private void assertCompleted(long chunkSize, long expectedFileSize) Assert.assertNotNull(completeRequest); Assert.assertFalse(cancelled); + Set partNumbersFromRequest = partRequests.stream().map(UploadPartRequest::getPartNumber).collect(Collectors.toSet()); + Assert.assertEquals(partRequests.size(), partNumbersFromRequest.size()); + for (int i = 0; i < partRequests.size(); i++) { - Assert.assertEquals(i + 1, partRequests.get(i).getPartNumber()); if (i < partRequests.size() - 1) { Assert.assertEquals(chunkSize, partRequests.get(i).getPartSize()); } else { @@ -286,12 +286,12 @@ private void assertCompleted(long chunkSize, long expectedFileSize) final List eTags = completeRequest.getPartETags(); Assert.assertEquals(partRequests.size(), eTags.size()); Assert.assertEquals( - partRequests.stream().map(UploadPartRequest::getPartNumber).collect(Collectors.toList()), - eTags.stream().map(PartETag::getPartNumber).collect(Collectors.toList()) + partNumbersFromRequest, + eTags.stream().map(PartETag::getPartNumber).collect(Collectors.toSet()) ); Assert.assertEquals( - partRequests.stream().map(UploadPartRequest::getPartNumber).collect(Collectors.toList()), - eTags.stream().map(tag -> Integer.parseInt(tag.getETag())).collect(Collectors.toList()) + partNumbersFromRequest, + eTags.stream().map(tag -> Integer.parseInt(tag.getETag())).collect(Collectors.toSet()) ); Assert.assertEquals( expectedFileSize, From 934ecd125d1ec41fef6032fa3466ce329b42eb83 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 22 May 2024 11:51:57 +0530 Subject: [PATCH 05/33] Remove unused constructor parameter chunkValidation + relevant cleanup --- .../s3/output/RetryableS3OutputStream.java | 12 ------------ .../s3/output/RetryableS3OutputStreamTest.java | 15 +++++---------- 2 files changed, 5 insertions(+), 22 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index d2d026867355..eb2a9963826d 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -27,7 +27,6 @@ import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import com.google.common.io.CountingOutputStream; import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; @@ -135,17 +134,6 @@ public RetryableS3OutputStream( ServerSideEncryptingAmazonS3 s3, String s3Key ) throws IOException - { - this(config, s3, s3Key, true); - } - - @VisibleForTesting - protected RetryableS3OutputStream( - S3OutputConfig config, - ServerSideEncryptingAmazonS3 s3, - String s3Key, - boolean chunkValidation - ) throws IOException { this.config = config; this.s3 = s3; diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java index 5acf2e11dc40..5186243692b2 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java @@ -109,8 +109,7 @@ public void testWriteAndHappy() throws IOException try (RetryableS3OutputStream out = new RetryableS3OutputStream( config, s3, - path, - false + path )) { for (int i = 0; i < 25; i++) { bb.clear(); @@ -131,8 +130,7 @@ public void testWriteSizeLargerThanConfiguredMaxChunkSizeShouldSucceed() throws try (RetryableS3OutputStream out = new RetryableS3OutputStream( config, s3, - path, - false + path )) { bb.clear(); bb.putInt(1); @@ -152,8 +150,7 @@ public void testWriteSmallBufferShouldSucceed() throws IOException try (RetryableS3OutputStream out = new RetryableS3OutputStream( config, s3, - path, - false + path )) { for (int i = 0; i < 600; i++) { out.write(i); @@ -174,8 +171,7 @@ public void testSuccessToUploadAfterRetry() throws IOException try (RetryableS3OutputStream out = new RetryableS3OutputStream( config, s3, - path, - false + path )) { for (int i = 0; i < 25; i++) { bb.clear(); @@ -197,8 +193,7 @@ public void testFailToUploadAfterRetries() throws IOException try (RetryableS3OutputStream out = new RetryableS3OutputStream( config, s3, - path, - false + path )) { for (int i = 0; i < 2; i++) { bb.clear(); From 82fa8785ca35e90bb4edb76fc021f6b4f6881cbb Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 22 May 2024 11:57:26 +0530 Subject: [PATCH 06/33] Assert etags as String instead of Integer --- .../storage/s3/output/RetryableS3OutputStreamTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java index 5186243692b2..1e66c3ade6a6 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java @@ -244,7 +244,7 @@ public UploadPartResult uploadPart(UploadPartRequest request) throws SdkClientEx } partRequests.add(request); UploadPartResult result = new UploadPartResult(); - result.setETag(StringUtils.format("%s", request.getPartNumber())); + result.setETag(StringUtils.format("etag-%s", request.getPartNumber())); result.setPartNumber(request.getPartNumber()); return result; } @@ -285,8 +285,8 @@ private void assertCompleted(long chunkSize, long expectedFileSize) eTags.stream().map(PartETag::getPartNumber).collect(Collectors.toSet()) ); Assert.assertEquals( - partNumbersFromRequest, - eTags.stream().map(tag -> Integer.parseInt(tag.getETag())).collect(Collectors.toSet()) + partNumbersFromRequest.stream().map(partNumber -> "etag-" + partNumber).collect(Collectors.toSet()), + eTags.stream().map(PartETag::getETag).collect(Collectors.toSet()) ); Assert.assertEquals( expectedFileSize, From bc55f0a8b044187a51285fcbfc18e2edd4222f79 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 22 May 2024 12:31:37 +0530 Subject: [PATCH 07/33] Fix flaky test --- .../druid/storage/s3/output/RetryableS3OutputStreamTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java index 1e66c3ade6a6..cac4367b013e 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java @@ -242,7 +242,9 @@ public UploadPartResult uploadPart(UploadPartRequest request) throws SdkClientEx new IOE("Upload failure test. Remaining failures [%s]", --uploadFailuresLeft) ); } - partRequests.add(request); + synchronized (partRequests) { + partRequests.add(request); + } UploadPartResult result = new UploadPartResult(); result.setETag(StringUtils.format("etag-%s", request.getPartNumber())); result.setPartNumber(request.getPartNumber()); From 438008ee0de68baf0b23a87eb5631e12a633b863 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 27 May 2024 16:46:28 +0530 Subject: [PATCH 08/33] Inject executor service --- .../storage/s3/S3StorageDruidModule.java | 11 ++++++++++ .../s3/output/RetryableS3OutputStream.java | 10 +++++----- .../s3/output/S3ExportStorageProvider.java | 8 +++++++- .../storage/s3/output/S3StorageConnector.java | 7 +++++-- .../s3/output/S3StorageConnectorProvider.java | 8 +++++++- .../output/RetryableS3OutputStreamTest.java | 20 +++++++++++++------ .../s3/output/S3StorageConnectorTest.java | 2 +- 7 files changed, 50 insertions(+), 16 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java index 3747088aeb6e..ae872f9db64d 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java @@ -34,6 +34,7 @@ import com.google.inject.Binder; import com.google.inject.Provides; import com.google.inject.multibindings.MapBinder; +import com.google.inject.name.Named; import org.apache.commons.lang.StringUtils; import org.apache.druid.common.aws.AWSClientConfig; import org.apache.druid.common.aws.AWSEndpointConfig; @@ -43,9 +44,11 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; import java.util.List; +import java.util.concurrent.ExecutorService; /** * @@ -182,4 +185,12 @@ public Supplier getAmazonS3ClientSupplier( { return Suppliers.memoize(serverSideEncryptingAmazonS3Builder::build); } + + @Provides + @LazySingleton + @Named("S3UploadThreadPool") + public ExecutorService getUploadPoolExecutorService(ScheduledExecutorFactory scheduledExecutorFactory) + { + return scheduledExecutorFactory.create(20, "UploadThreadPool-%d"); + } } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index eb2a9963826d..0550678c1a3b 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -32,7 +32,6 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.RetryUtils; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.storage.s3.S3Utils; @@ -127,17 +126,19 @@ public class RetryableS3OutputStream extends OutputStream /** * Threadpool used for uploading the chunks asynchronously. */ - private static final ExecutorService UPLOAD_EXECUTOR = Execs.multiThreaded(10, "UploadThreadPool-%d"); + private final ExecutorService uploadExecutor; public RetryableS3OutputStream( S3OutputConfig config, ServerSideEncryptingAmazonS3 s3, - String s3Key + String s3Key, + ExecutorService executorService ) throws IOException { this.config = config; this.s3 = s3; this.s3Key = s3Key; + this.uploadExecutor = executorService; final InitiateMultipartUploadResult result; try { @@ -184,7 +185,6 @@ public void write(byte[] b, int off, int len) throws IOException while (remainingBytesToWrite > 0) { final int writtenBytes = writeToCurrentChunk(b, offsetToWrite, remainingBytesToWrite); - if (currentChunk.length() >= chunkSize) { pushCurrentChunk(); currentChunk = new Chunk(nextChunkId, new File(chunkStorePath, String.valueOf(nextChunkId++))); @@ -216,7 +216,7 @@ private void pushCurrentChunk() throws IOException SEMAPHORE.acquire(); // Acquire a permit from the semaphore pendingFiles.incrementAndGet(); - UPLOAD_EXECUTOR.submit(() -> { + uploadExecutor.submit(() -> { try { uploadChunk(chunk); } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java index 9b03a4f07c74..00fe73ef6598 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; +import com.google.inject.name.Named; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.s3.S3InputSource; import org.apache.druid.error.DruidException; @@ -39,6 +40,7 @@ import java.io.File; import java.net.URI; import java.util.List; +import java.util.concurrent.ExecutorService; @JsonTypeName(S3ExportStorageProvider.TYPE_NAME) public class S3ExportStorageProvider implements ExportStorageProvider @@ -56,6 +58,10 @@ public class S3ExportStorageProvider implements ExportStorageProvider @JacksonInject ServerSideEncryptingAmazonS3 s3; + @JacksonInject + @Named("S3UploadThreadPool") + ExecutorService executorService; + @JsonCreator public S3ExportStorageProvider( @JsonProperty(value = "bucket", required = true) String bucket, @@ -90,7 +96,7 @@ public StorageConnector get() s3ExportConfig.getChunkSize(), s3ExportConfig.getMaxRetry() ); - return new S3StorageConnector(s3OutputConfig, s3); + return new S3StorageConnector(s3OutputConfig, s3, executorService); } @VisibleForTesting diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index a68ed9c1c00c..c426737bd7d7 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -45,6 +45,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.concurrent.ExecutorService; /** * In this implementation, all remote calls to aws s3 are retried {@link S3OutputConfig#getMaxRetry()} times. @@ -55,15 +56,17 @@ public class S3StorageConnector extends ChunkingStorageConnector Date: Mon, 27 May 2024 16:58:46 +0530 Subject: [PATCH 09/33] Make threadpool size dynamic based on number of cores --- .../org/apache/druid/storage/s3/S3StorageDruidModule.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java index ae872f9db64d..124422dc55d4 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java @@ -32,6 +32,7 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; +import com.google.inject.Injector; import com.google.inject.Provides; import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; @@ -40,6 +41,7 @@ import org.apache.druid.common.aws.AWSEndpointConfig; import org.apache.druid.common.aws.AWSProxyConfig; import org.apache.druid.data.SearchableVersionedDataFinder; +import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.Binders; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; @@ -189,8 +191,9 @@ public Supplier getAmazonS3ClientSupplier( @Provides @LazySingleton @Named("S3UploadThreadPool") - public ExecutorService getUploadPoolExecutorService(ScheduledExecutorFactory scheduledExecutorFactory) + public ExecutorService getUploadPoolExecutorService(ScheduledExecutorFactory scheduledExecutorFactory, final Injector injector) { - return scheduledExecutorFactory.create(20, "UploadThreadPool-%d"); + int poolSize = Math.max(4, 3 * injector.getInstance(Bouncer.class).getMaxCount()); + return scheduledExecutorFactory.create(poolSize, "UploadThreadPool-%d"); } } From c266a9e902e4d327b8bfa8b72dbd84401dad28c3 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 27 May 2024 17:34:41 +0530 Subject: [PATCH 10/33] Fix S3StorageDruidModuleTest --- .../org/apache/druid/storage/s3/S3StorageDruidModuleTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageDruidModuleTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageDruidModuleTest.java index 4d9b4f2d6ee6..3932c147695b 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageDruidModuleTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageDruidModuleTest.java @@ -23,6 +23,7 @@ import com.google.inject.Injector; import org.apache.druid.common.aws.AWSModule; import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.guice.ServerModule; import org.apache.druid.segment.loading.OmniDataSegmentArchiver; import org.apache.druid.segment.loading.OmniDataSegmentKiller; import org.apache.druid.segment.loading.OmniDataSegmentMover; @@ -72,7 +73,8 @@ private static Injector createInjector() return GuiceInjectors.makeStartupInjectorWithModules( ImmutableList.of( new AWSModule(), - new S3StorageDruidModule() + new S3StorageDruidModule(), + new ServerModule() ) ); } From ce60f7e7ed4b88182c6ad84e8f95b28be877c38f Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 27 May 2024 19:17:50 +0530 Subject: [PATCH 11/33] Fix S3StorageConnectorProviderTest --- .../druid/storage/s3/S3StorageConnectorProviderTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java index 9f9d632f6181..54738e92ddf7 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java @@ -31,6 +31,7 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.StartupInjectorBuilder; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorModule; import org.apache.druid.storage.StorageConnectorProvider; @@ -42,6 +43,7 @@ import java.io.File; import java.util.Properties; +import java.util.concurrent.ExecutorService; public class S3StorageConnectorProviderTest { @@ -146,6 +148,10 @@ public void configure(Binder binder) .addValue( ServerSideEncryptingAmazonS3.class, new ServerSideEncryptingAmazonS3(null, new NoopServerSideEncryption()) + ) + .addValue( + ExecutorService.class, + Execs.multiThreaded(10, "UploadThreadPool-%d") )); From c382daf69ddefde892a9ee85b0c8a1d9e5d7affa Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 27 May 2024 20:10:17 +0530 Subject: [PATCH 12/33] Fix injection issues --- .../org/apache/druid/storage/s3/S3StorageDruidModule.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java index 124422dc55d4..81a7ca00dc83 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java @@ -32,7 +32,6 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; -import com.google.inject.Injector; import com.google.inject.Provides; import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; @@ -41,13 +40,13 @@ import org.apache.druid.common.aws.AWSEndpointConfig; import org.apache.druid.common.aws.AWSProxyConfig; import org.apache.druid.data.SearchableVersionedDataFinder; -import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.Binders; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.utils.RuntimeInfo; import java.util.List; import java.util.concurrent.ExecutorService; @@ -191,9 +190,9 @@ public Supplier getAmazonS3ClientSupplier( @Provides @LazySingleton @Named("S3UploadThreadPool") - public ExecutorService getUploadPoolExecutorService(ScheduledExecutorFactory scheduledExecutorFactory, final Injector injector) + public ExecutorService getUploadPoolExecutorService(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) { - int poolSize = Math.max(4, 3 * injector.getInstance(Bouncer.class).getMaxCount()); + int poolSize = Math.max(4, 3 * runtimeInfo.getAvailableProcessors()); return scheduledExecutorFactory.create(poolSize, "UploadThreadPool-%d"); } } From 2224d1109fa07918fae829740ec08dffb2c0991a Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Tue, 28 May 2024 10:10:38 +0530 Subject: [PATCH 13/33] Add S3UploadConfig to manage maximum number of concurrent chunks dynamically based on chunk size --- .../storage/s3/S3StorageDruidModule.java | 8 ++ .../s3/output/RetryableS3OutputStream.java | 75 +++++++----- .../s3/output/S3ExportStorageProvider.java | 6 +- .../storage/s3/output/S3StorageConnector.java | 6 +- .../s3/output/S3StorageConnectorProvider.java | 6 +- .../storage/s3/output/S3UploadConfig.java | 115 ++++++++++++++++++ .../s3/S3StorageConnectorProviderTest.java | 5 + .../output/RetryableS3OutputStreamTest.java | 17 ++- .../s3/output/S3StorageConnectorTest.java | 2 +- .../storage/s3/output/S3UploadConfigTest.java | 50 ++++++++ 10 files changed, 250 insertions(+), 40 deletions(-) create mode 100644 extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java create mode 100644 extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadConfigTest.java diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java index 81a7ca00dc83..aaeeda35f97b 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java @@ -46,6 +46,7 @@ import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.storage.s3.output.S3UploadConfig; import org.apache.druid.utils.RuntimeInfo; import java.util.List; @@ -195,4 +196,11 @@ public ExecutorService getUploadPoolExecutorService(ScheduledExecutorFactory sch int poolSize = Math.max(4, 3 * runtimeInfo.getAvailableProcessors()); return scheduledExecutorFactory.create(poolSize, "UploadThreadPool-%d"); } + + @Provides + @LazySingleton + public S3UploadConfig getS3UploadConfig() + { + return new S3UploadConfig(); + } } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index 0550678c1a3b..d5752b5a8ddf 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -48,7 +48,6 @@ import java.util.Objects; import java.util.UUID; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -118,27 +117,33 @@ public class RetryableS3OutputStream extends OutputStream private final Object fileLock = new Object(); /** - * Semaphore to restrict the maximum number of simultaneous chunks on disk. + * Threadpool used for uploading the chunks asynchronously. */ - private static final int MAX_CONCURRENT_CHUNKS = 10; - private static final Semaphore SEMAPHORE = new Semaphore(MAX_CONCURRENT_CHUNKS); + private final ExecutorService uploadExecutor; /** - * Threadpool used for uploading the chunks asynchronously. + * Helper class for calculating maximum number of simultaneous chunks allowed on local disk. */ - private final ExecutorService uploadExecutor; + private final S3UploadConfig s3UploadConfig; + + /** + * A lock to restrict the maximum number of chunks on disk at any given point in time. + */ + private final Object maxChunksLock = new Object(); public RetryableS3OutputStream( S3OutputConfig config, ServerSideEncryptingAmazonS3 s3, String s3Key, - ExecutorService executorService + ExecutorService executorService, + S3UploadConfig s3UploadConfig ) throws IOException { this.config = config; this.s3 = s3; this.s3Key = s3Key; this.uploadExecutor = executorService; + this.s3UploadConfig = s3UploadConfig; final InitiateMultipartUploadResult result; try { @@ -179,6 +184,19 @@ public void write(byte[] b, int off, int len) throws IOException return; } + synchronized (maxChunksLock) { + while (s3UploadConfig.getCurrentNumChunks() > s3UploadConfig.getMaxConcurrentNumChunks()) { + try { + LOG.info("Waiting for lock for writing further chunks to local disk."); + maxChunksLock.wait(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + } + try { int offsetToWrite = off; int remainingBytesToWrite = len; @@ -212,33 +230,30 @@ private void pushCurrentChunk() throws IOException currentChunk.close(); final Chunk chunk = currentChunk; if (chunk.length() > 0) { - try { - SEMAPHORE.acquire(); // Acquire a permit from the semaphore - pendingFiles.incrementAndGet(); + s3UploadConfig.incrementCurrentNumChunks(); + pendingFiles.incrementAndGet(); - uploadExecutor.submit(() -> { - try { - uploadChunk(chunk); - } - catch (Exception e) { - error = true; - LOG.error(e, e.getMessage()); - throw new RuntimeException(e); + uploadExecutor.submit(() -> { + try { + uploadChunk(chunk); + } + catch (Exception e) { + error = true; + LOG.error(e, e.getMessage()); + throw new RuntimeException(e); + } + finally { + synchronized (maxChunksLock) { + s3UploadConfig.decrementCurrentNumChunks(); + maxChunksLock.notifyAll(); } - finally { - SEMAPHORE.release(); // Release the permit after upload is completed - if (pendingFiles.decrementAndGet() == 0) { - synchronized (fileLock) { - fileLock.notifyAll(); - } + if (pendingFiles.decrementAndGet() == 0) { + synchronized (fileLock) { + fileLock.notifyAll(); } } - }); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IOException(e); - } + } + }); } } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java index 00fe73ef6598..7a15f142c5dd 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java @@ -62,6 +62,9 @@ public class S3ExportStorageProvider implements ExportStorageProvider @Named("S3UploadThreadPool") ExecutorService executorService; + @JacksonInject + S3UploadConfig s3UploadConfig; + @JsonCreator public S3ExportStorageProvider( @JsonProperty(value = "bucket", required = true) String bucket, @@ -96,7 +99,8 @@ public StorageConnector get() s3ExportConfig.getChunkSize(), s3ExportConfig.getMaxRetry() ); - return new S3StorageConnector(s3OutputConfig, s3, executorService); + s3UploadConfig.updateChunkSizeIfGreater(s3ExportConfig.getChunkSize().getBytes()); + return new S3StorageConnector(s3OutputConfig, s3, executorService, s3UploadConfig); } @VisibleForTesting diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index c426737bd7d7..51ef96241667 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -57,16 +57,18 @@ public class S3StorageConnector extends ChunkingStorageConnector Date: Tue, 28 May 2024 12:54:31 +0530 Subject: [PATCH 14/33] Address the minor review comments --- .../org/apache/druid/storage/s3/S3StorageDruidModule.java | 6 +++--- .../druid/storage/s3/output/RetryableS3OutputStream.java | 2 +- .../druid/storage/s3/output/S3ExportStorageProvider.java | 2 +- .../druid/storage/s3/output/S3StorageConnectorProvider.java | 2 +- .../org/apache/druid/storage/s3/output/S3UploadConfig.java | 4 ++-- .../druid/storage/s3/S3StorageConnectorProviderTest.java | 2 +- .../storage/s3/output/RetryableS3OutputStreamTest.java | 2 +- 7 files changed, 10 insertions(+), 10 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java index aaeeda35f97b..445b3c9ec24d 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java @@ -190,10 +190,10 @@ public Supplier getAmazonS3ClientSupplier( @Provides @LazySingleton - @Named("S3UploadThreadPool") - public ExecutorService getUploadPoolExecutorService(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) + @Named(S3UploadConfig.UPLOAD_THREADPOOL_NAMED_VALUE) + public ExecutorService getUploadExecutorService(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) { - int poolSize = Math.max(4, 3 * runtimeInfo.getAvailableProcessors()); + int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); return scheduledExecutorFactory.create(poolSize, "UploadThreadPool-%d"); } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index d5752b5a8ddf..e4446c128457 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -187,7 +187,7 @@ public void write(byte[] b, int off, int len) throws IOException synchronized (maxChunksLock) { while (s3UploadConfig.getCurrentNumChunks() > s3UploadConfig.getMaxConcurrentNumChunks()) { try { - LOG.info("Waiting for lock for writing further chunks to local disk."); + LOG.debug("Waiting for lock for writing further chunks to local disk."); maxChunksLock.wait(); } catch (InterruptedException e) { diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java index 7a15f142c5dd..069a06d4e39b 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java @@ -59,7 +59,7 @@ public class S3ExportStorageProvider implements ExportStorageProvider ServerSideEncryptingAmazonS3 s3; @JacksonInject - @Named("S3UploadThreadPool") + @Named(S3UploadConfig.UPLOAD_THREADPOOL_NAMED_VALUE) ExecutorService executorService; @JacksonInject diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java index 50dbd086116d..399d55cc75c6 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java @@ -41,7 +41,7 @@ public class S3StorageConnectorProvider extends S3OutputConfig implements Storag ServerSideEncryptingAmazonS3 s3; @JacksonInject - @Named("S3UploadThreadPool") + @Named(S3UploadConfig.UPLOAD_THREADPOOL_NAMED_VALUE) ExecutorService executorService; @JacksonInject diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java index 3c4cde547e61..cd8524cdf85f 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java @@ -19,7 +19,6 @@ package org.apache.druid.storage.s3.output; -import org.apache.druid.guice.LazySingleton; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.logger.Logger; @@ -30,9 +29,10 @@ * It tracks the current number of chunks written to local disk concurrently and ensures that the * maximum number of chunks on disk does not exceed a specified limit at any given point in time. */ -@LazySingleton public class S3UploadConfig { + public static final String UPLOAD_THREADPOOL_NAMED_VALUE = "S3UploadThreadPool"; + /** * The maximum chunk size based on injected values for {@link S3OutputConfig} and {@link S3ExportConfig} used for computing maximum number of chunks to save on disk at any given point in time. * It is initialized to 5 MiB which is the minimum chunk size possible, denoted by {@link S3OutputConfig#S3_MULTIPART_UPLOAD_MIN_PART_SIZE_BYTES}. diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java index 7d336761e0c1..669ca1c746fb 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java @@ -152,7 +152,7 @@ public void configure(Binder binder) ) .addValue( ExecutorService.class, - Execs.multiThreaded(10, "UploadThreadPool-%d") + Execs.singleThreaded("UploadThreadPool-%d") ) .addValue( S3UploadConfig.class, diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java index ea3ea5419c77..e10762bda139 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java @@ -68,7 +68,7 @@ public class RetryableS3OutputStreamTest private S3OutputConfig config; private long chunkSize; - private final ExecutorService executorService = Execs.multiThreaded(10, "UploadThreadPool-%d"); + private final ExecutorService executorService = Execs.singleThreaded("UploadThreadPool-%d"); private final S3UploadConfig s3UploadConfig = new S3UploadConfig(); From 6d62bf96b61a93e0dbd756651ad2d2da1946f0cf Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Tue, 28 May 2024 15:44:00 +0530 Subject: [PATCH 15/33] Refactor S3UploadConfig + ExecutorService into S3UploadManager --- .../storage/s3/S3StorageDruidModule.java | 21 +++++------- .../s3/output/RetryableS3OutputStream.java | 22 ++++-------- .../s3/output/S3ExportStorageProvider.java | 12 ++----- .../storage/s3/output/S3StorageConnector.java | 11 +++--- .../s3/output/S3StorageConnectorProvider.java | 12 ++----- ...UploadConfig.java => S3UploadManager.java} | 34 ++++++++++++++----- .../s3/S3StorageConnectorProviderTest.java | 11 ++---- .../output/RetryableS3OutputStreamTest.java | 20 ++++------- .../s3/output/S3StorageConnectorTest.java | 3 +- ...nfigTest.java => S3UploadManagerTest.java} | 21 ++++++------ 10 files changed, 73 insertions(+), 94 deletions(-) rename extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/{S3UploadConfig.java => S3UploadManager.java} (83%) rename extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/{S3UploadConfigTest.java => S3UploadManagerTest.java} (61%) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java index 445b3c9ec24d..11c16708b331 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java @@ -34,7 +34,6 @@ import com.google.inject.Binder; import com.google.inject.Provides; import com.google.inject.multibindings.MapBinder; -import com.google.inject.name.Named; import org.apache.commons.lang.StringUtils; import org.apache.druid.common.aws.AWSClientConfig; import org.apache.druid.common.aws.AWSEndpointConfig; @@ -46,11 +45,11 @@ import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.storage.s3.output.S3UploadConfig; +import org.apache.druid.storage.s3.output.S3UploadManager; import org.apache.druid.utils.RuntimeInfo; import java.util.List; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; /** * @@ -190,17 +189,13 @@ public Supplier getAmazonS3ClientSupplier( @Provides @LazySingleton - @Named(S3UploadConfig.UPLOAD_THREADPOOL_NAMED_VALUE) - public ExecutorService getUploadExecutorService(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) + public S3UploadManager getS3UploadManager(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) { int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); - return scheduledExecutorFactory.create(poolSize, "UploadThreadPool-%d"); - } - - @Provides - @LazySingleton - public S3UploadConfig getS3UploadConfig() - { - return new S3UploadConfig(); + ScheduledExecutorService executorService = scheduledExecutorFactory.create( + poolSize, + "UploadThreadPool-%d" + ); + return new S3UploadManager(executorService); } } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index e4446c128457..97ea1e99c7b9 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -47,7 +47,6 @@ import java.util.List; import java.util.Objects; import java.util.UUID; -import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -116,15 +115,10 @@ public class RetryableS3OutputStream extends OutputStream */ private final Object fileLock = new Object(); - /** - * Threadpool used for uploading the chunks asynchronously. - */ - private final ExecutorService uploadExecutor; - /** * Helper class for calculating maximum number of simultaneous chunks allowed on local disk. */ - private final S3UploadConfig s3UploadConfig; + private final S3UploadManager uploadManager; /** * A lock to restrict the maximum number of chunks on disk at any given point in time. @@ -135,15 +129,13 @@ public RetryableS3OutputStream( S3OutputConfig config, ServerSideEncryptingAmazonS3 s3, String s3Key, - ExecutorService executorService, - S3UploadConfig s3UploadConfig + S3UploadManager uploadManager ) throws IOException { this.config = config; this.s3 = s3; this.s3Key = s3Key; - this.uploadExecutor = executorService; - this.s3UploadConfig = s3UploadConfig; + this.uploadManager = uploadManager; final InitiateMultipartUploadResult result; try { @@ -185,7 +177,7 @@ public void write(byte[] b, int off, int len) throws IOException } synchronized (maxChunksLock) { - while (s3UploadConfig.getCurrentNumChunks() > s3UploadConfig.getMaxConcurrentNumChunks()) { + while (uploadManager.getCurrentNumChunks() > uploadManager.getMaxConcurrentNumChunks()) { try { LOG.debug("Waiting for lock for writing further chunks to local disk."); maxChunksLock.wait(); @@ -230,10 +222,10 @@ private void pushCurrentChunk() throws IOException currentChunk.close(); final Chunk chunk = currentChunk; if (chunk.length() > 0) { - s3UploadConfig.incrementCurrentNumChunks(); + uploadManager.incrementCurrentNumChunks(); pendingFiles.incrementAndGet(); - uploadExecutor.submit(() -> { + uploadManager.submitTask(() -> { try { uploadChunk(chunk); } @@ -244,7 +236,7 @@ private void pushCurrentChunk() throws IOException } finally { synchronized (maxChunksLock) { - s3UploadConfig.decrementCurrentNumChunks(); + uploadManager.decrementCurrentNumChunks(); maxChunksLock.notifyAll(); } if (pendingFiles.decrementAndGet() == 0) { diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java index 069a06d4e39b..39ac4a38943e 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java @@ -26,7 +26,6 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; -import com.google.inject.name.Named; import org.apache.druid.data.input.impl.CloudObjectLocation; import org.apache.druid.data.input.s3.S3InputSource; import org.apache.druid.error.DruidException; @@ -40,7 +39,6 @@ import java.io.File; import java.net.URI; import java.util.List; -import java.util.concurrent.ExecutorService; @JsonTypeName(S3ExportStorageProvider.TYPE_NAME) public class S3ExportStorageProvider implements ExportStorageProvider @@ -59,11 +57,7 @@ public class S3ExportStorageProvider implements ExportStorageProvider ServerSideEncryptingAmazonS3 s3; @JacksonInject - @Named(S3UploadConfig.UPLOAD_THREADPOOL_NAMED_VALUE) - ExecutorService executorService; - - @JacksonInject - S3UploadConfig s3UploadConfig; + S3UploadManager s3UploadManager; @JsonCreator public S3ExportStorageProvider( @@ -99,8 +93,8 @@ public StorageConnector get() s3ExportConfig.getChunkSize(), s3ExportConfig.getMaxRetry() ); - s3UploadConfig.updateChunkSizeIfGreater(s3ExportConfig.getChunkSize().getBytes()); - return new S3StorageConnector(s3OutputConfig, s3, executorService, s3UploadConfig); + s3UploadManager.updateChunkSizeIfGreater(s3ExportConfig.getChunkSize().getBytes()); + return new S3StorageConnector(s3OutputConfig, s3, s3UploadManager); } @VisibleForTesting diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index 51ef96241667..8eb391a24d39 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -45,7 +45,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.concurrent.ExecutorService; /** * In this implementation, all remote calls to aws s3 are retried {@link S3OutputConfig#getMaxRetry()} times. @@ -56,19 +55,17 @@ public class S3StorageConnector extends ChunkingStorageConnector Date: Tue, 28 May 2024 18:38:01 +0530 Subject: [PATCH 16/33] Address review comments --- .../storage/s3/output/RetryableS3OutputStream.java | 2 +- .../druid/storage/s3/output/S3UploadManager.java | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index 97ea1e99c7b9..13591c365743 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -339,7 +339,7 @@ private void completeMultipartUpload() synchronized (fileLock) { while (pendingFiles.get() > 0) { try { - LOG.info("Waiting for lock for completing multipart task for uploadId [%s].", uploadId); + LOG.debug("Waiting for lock for completing multipart task for uploadId [%s].", uploadId); fileLock.wait(); } catch (InterruptedException e) { diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index dd3fcd001d62..b52cda95a92c 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -47,7 +47,7 @@ public class S3UploadManager * The maximum number of chunks that can be saved to local disk concurrently. * This value is recalculated when the chunk size is updated in {@link #updateChunkSizeIfGreater(long)}. */ - private final AtomicInteger maxConcurrentNumChunks = new AtomicInteger(100); + private int maxConcurrentNumChunks = 100; /** * Threadpool used for uploading the chunks asynchronously. @@ -94,7 +94,7 @@ public int getCurrentNumChunks() */ public int getMaxConcurrentNumChunks() { - return maxConcurrentNumChunks.get(); + return maxConcurrentNumChunks; } /** @@ -116,10 +116,10 @@ public void updateChunkSizeIfGreater(long chunkSize) * The maximum allowed chunk size is {@link S3OutputConfig#S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES} which is quite big, * so we restrict the maximum disk space used to the same, at any given point in time. */ - private void recomputeMaxConcurrentNumChunks() + private synchronized void recomputeMaxConcurrentNumChunks() { - maxConcurrentNumChunks.set((int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / chunkSize)); - log.info("Recomputed maxConcurrentNumChunks: %d", maxConcurrentNumChunks.get()); + maxConcurrentNumChunks = (int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / chunkSize); + log.info("Recomputed maxConcurrentNumChunks: %d", maxConcurrentNumChunks); } /** From 1c0bfe52a5610543499988dc811c7a332d90f19c Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Tue, 28 May 2024 18:40:10 +0530 Subject: [PATCH 17/33] Make updateChunkSizeIfGreater() synchronized instead of recomputeMaxConcurrentNumChunks() --- .../org/apache/druid/storage/s3/output/S3UploadManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index b52cda95a92c..1b58a650bc02 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -103,7 +103,7 @@ public int getMaxConcurrentNumChunks() * * @param chunkSize the new chunk size to be set if it is greater than the current chunk size. */ - public void updateChunkSizeIfGreater(long chunkSize) + public synchronized void updateChunkSizeIfGreater(long chunkSize) { if (this.chunkSize < chunkSize) { this.chunkSize = chunkSize; @@ -116,7 +116,7 @@ public void updateChunkSizeIfGreater(long chunkSize) * The maximum allowed chunk size is {@link S3OutputConfig#S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES} which is quite big, * so we restrict the maximum disk space used to the same, at any given point in time. */ - private synchronized void recomputeMaxConcurrentNumChunks() + private void recomputeMaxConcurrentNumChunks() { maxConcurrentNumChunks = (int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / chunkSize); log.info("Recomputed maxConcurrentNumChunks: %d", maxConcurrentNumChunks); From 74187900c6eeba422149d8249ed7b486584817e4 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 29 May 2024 10:12:56 +0530 Subject: [PATCH 18/33] Address the minor review comments --- .../storage/s3/S3StorageDruidModule.java | 16 -------- .../s3/output/RetryableS3OutputStream.java | 6 +-- .../storage/s3/output/S3UploadManager.java | 40 ++++++++++--------- .../s3/S3StorageConnectorProviderTest.java | 9 +++-- .../output/RetryableS3OutputStreamTest.java | 10 ++++- .../s3/output/S3StorageConnectorTest.java | 5 ++- .../s3/output/S3UploadManagerTest.java | 23 +++++++++-- 7 files changed, 61 insertions(+), 48 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java index 11c16708b331..3747088aeb6e 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java @@ -43,13 +43,9 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.initialization.DruidModule; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.storage.s3.output.S3UploadManager; -import org.apache.druid.utils.RuntimeInfo; import java.util.List; -import java.util.concurrent.ScheduledExecutorService; /** * @@ -186,16 +182,4 @@ public Supplier getAmazonS3ClientSupplier( { return Suppliers.memoize(serverSideEncryptingAmazonS3Builder::build); } - - @Provides - @LazySingleton - public S3UploadManager getS3UploadManager(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) - { - int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); - ScheduledExecutorService executorService = scheduledExecutorFactory.create( - poolSize, - "UploadThreadPool-%d" - ); - return new S3UploadManager(executorService); - } } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index 13591c365743..40a3a0858bc5 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -177,7 +177,7 @@ public void write(byte[] b, int off, int len) throws IOException } synchronized (maxChunksLock) { - while (uploadManager.getCurrentNumChunks() > uploadManager.getMaxConcurrentNumChunks()) { + while (uploadManager.getCurrentNumChunksOnDisk() > uploadManager.getMaxConcurrentNumChunks()) { try { LOG.debug("Waiting for lock for writing further chunks to local disk."); maxChunksLock.wait(); @@ -222,7 +222,7 @@ private void pushCurrentChunk() throws IOException currentChunk.close(); final Chunk chunk = currentChunk; if (chunk.length() > 0) { - uploadManager.incrementCurrentNumChunks(); + uploadManager.incrementCurrentNumChunksOnDisk(); pendingFiles.incrementAndGet(); uploadManager.submitTask(() -> { @@ -236,7 +236,7 @@ private void pushCurrentChunk() throws IOException } finally { synchronized (maxChunksLock) { - uploadManager.decrementCurrentNumChunks(); + uploadManager.decrementCurrentNumChunksOnDisk(); maxChunksLock.notifyAll(); } if (pendingFiles.decrementAndGet() == 0) { diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index 1b58a650bc02..a0f936364cdd 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -19,16 +19,19 @@ package org.apache.druid.storage.s3.output; +import com.google.inject.Inject; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.utils.RuntimeInfo; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; /** * This class manages the configuration for uploading files to S3 in chunks. - * It tracks the current number of chunks written to local disk concurrently and ensures that the - * maximum number of chunks on disk does not exceed a specified limit at any given point in time. + * It tracks the number of chunks currently present on local disk and ensures that + * it does not exceed a specified limit. */ public class S3UploadManager { @@ -38,10 +41,7 @@ public class S3UploadManager */ private long chunkSize = new HumanReadableBytes("5MiB").getBytes(); - /** - * An atomic counter to track the current number of chunks saved to local disk. - */ - private final AtomicInteger currentNumChunks = new AtomicInteger(0); + private final AtomicInteger currentNumChunksOnDisk = new AtomicInteger(0); /** * The maximum number of chunks that can be saved to local disk concurrently. @@ -49,32 +49,34 @@ public class S3UploadManager */ private int maxConcurrentNumChunks = 100; - /** - * Threadpool used for uploading the chunks asynchronously. - */ - private final ExecutorService uploadExecutor; + private final ScheduledExecutorService uploadExecutor; private static final Logger log = new Logger(S3UploadManager.class); - public S3UploadManager(ExecutorService executorService) + @Inject + public S3UploadManager(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) { - this.uploadExecutor = executorService; + int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); + this.uploadExecutor = scheduledExecutorFactory.create( + poolSize, + "UploadThreadPool-%d" + ); } /** * Increments the counter for the current number of chunks saved on disk. */ - public void incrementCurrentNumChunks() + public void incrementCurrentNumChunksOnDisk() { - currentNumChunks.incrementAndGet(); + currentNumChunksOnDisk.incrementAndGet(); } /** * Decrements the counter for the current number of chunks saved on disk. */ - public void decrementCurrentNumChunks() + public void decrementCurrentNumChunksOnDisk() { - currentNumChunks.decrementAndGet(); + currentNumChunksOnDisk.decrementAndGet(); } /** @@ -82,9 +84,9 @@ public void decrementCurrentNumChunks() * * @return the current number of chunks saved to local disk. */ - public int getCurrentNumChunks() + public int getCurrentNumChunksOnDisk() { - return currentNumChunks.get(); + return currentNumChunksOnDisk.get(); } /** diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java index 95ac547c78ba..7c5619272afc 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java @@ -31,7 +31,8 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.StartupInjectorBuilder; -import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.query.DruidProcessingConfigTest; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorModule; import org.apache.druid.storage.StorageConnectorProvider; @@ -39,6 +40,7 @@ import org.apache.druid.storage.s3.output.S3StorageConnectorModule; import org.apache.druid.storage.s3.output.S3StorageConnectorProvider; import org.apache.druid.storage.s3.output.S3UploadManager; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -151,8 +153,9 @@ public void configure(Binder binder) ) .addValue( S3UploadManager.class, - new S3UploadManager(Execs.singleThreaded("UploadThreadPool-%d")) - )); + new S3UploadManager(EasyMock.mock(ScheduledExecutorFactory.class), new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)) + ) + ); StorageConnectorProvider storageConnectorProvider = injector.getInstance(Key.get( diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java index 53fe58ae4124..6f24b46e7308 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java @@ -34,6 +34,8 @@ import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.query.DruidProcessingConfigTest; import org.apache.druid.storage.s3.NoopServerSideEncryption; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; import org.easymock.EasyMock; @@ -67,7 +69,7 @@ public class RetryableS3OutputStreamTest private S3OutputConfig config; private long chunkSize; - private final S3UploadManager s3UploadManager = new S3UploadManager(Execs.singleThreaded("UploadThreadPool-%d")); + private S3UploadManager s3UploadManager; @Before public void setup() throws IOException @@ -101,6 +103,12 @@ public int getMaxRetry() return 2; } }; + + ScheduledExecutorFactory executorFactory = EasyMock.mock(ScheduledExecutorFactory.class); + EasyMock.expect(executorFactory.create(EasyMock.anyInt(), EasyMock.anyString())) + .andReturn(Execs.scheduledSingleThreaded("UploadThreadPool-%d")); + EasyMock.replay(executorFactory); + s3UploadManager = new S3UploadManager(executorFactory, new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)); } @Test diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java index 2810494745da..da6d7702542b 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java @@ -31,7 +31,8 @@ import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.query.DruidProcessingConfigTest; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.s3.NoopServerSideEncryption; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; @@ -85,7 +86,7 @@ public void setup() null, null, true - ), service, new S3UploadManager(Execs.singleThreaded("UploadThreadPool-%d"))); + ), service, new S3UploadManager(EasyMock.mock(ScheduledExecutorFactory.class), new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0))); } catch (IOException e) { throw new RuntimeException(e); diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java index 583885235a95..bdaf0f144e5e 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java @@ -20,19 +20,34 @@ package org.apache.druid.storage.s3.output; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.query.DruidProcessingConfigTest; +import org.easymock.EasyMock; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import java.io.IOException; + public class S3UploadManagerTest { + private S3UploadManager s3UploadManager; - S3UploadManager s3UploadManager = new S3UploadManager(Execs.singleThreaded("UploadThreadPool-%d")); + @Before + public void setup() throws IOException + { + ScheduledExecutorFactory executorFactory = EasyMock.mock(ScheduledExecutorFactory.class); + EasyMock.expect(executorFactory.create(EasyMock.anyInt(), EasyMock.anyString())) + .andReturn(Execs.scheduledSingleThreaded("UploadThreadPool-%d")); + EasyMock.replay(executorFactory); + s3UploadManager = new S3UploadManager(executorFactory, new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)); + } @Test public void testDefault() { Assert.assertEquals(100, s3UploadManager.getMaxConcurrentNumChunks()); - Assert.assertEquals(0, s3UploadManager.getCurrentNumChunks()); + Assert.assertEquals(0, s3UploadManager.getCurrentNumChunksOnDisk()); } @Test @@ -41,11 +56,11 @@ public void testUpdateChunkSize() // Verify that updating chunk size to a smaller or equal value doesn't change the maxConcurrentNumChunks. s3UploadManager.updateChunkSizeIfGreater(5L * 1024 * 1024L); Assert.assertEquals(100, s3UploadManager.getMaxConcurrentNumChunks()); - Assert.assertEquals(0, s3UploadManager.getCurrentNumChunks()); + Assert.assertEquals(0, s3UploadManager.getCurrentNumChunksOnDisk()); // Verify that updating chunk size to a greater value changes the maxConcurrentNumChunks. s3UploadManager.updateChunkSizeIfGreater(5L * 1024 * 1024 * 1024L); Assert.assertEquals(1, s3UploadManager.getMaxConcurrentNumChunks()); - Assert.assertEquals(0, s3UploadManager.getCurrentNumChunks()); + Assert.assertEquals(0, s3UploadManager.getCurrentNumChunksOnDisk()); } } From fbe71e38b5de5eb6304a04af92de8a6a5af488a9 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 29 May 2024 13:46:04 +0530 Subject: [PATCH 19/33] Fix intellij-inspections check --- .../apache/druid/storage/s3/output/S3UploadManagerTest.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java index bdaf0f144e5e..4bd18c1cd25e 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java @@ -27,14 +27,12 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; - public class S3UploadManagerTest { private S3UploadManager s3UploadManager; @Before - public void setup() throws IOException + public void setup() { ScheduledExecutorFactory executorFactory = EasyMock.mock(ScheduledExecutorFactory.class); EasyMock.expect(executorFactory.create(EasyMock.anyInt(), EasyMock.anyString())) From 5c83514955a7e2ee727fa764e99caa91a798c56a Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Fri, 31 May 2024 13:21:14 +0530 Subject: [PATCH 20/33] Refactor code to use futures for maxNumConcurrentChunks. Also use executor service with blocking queue for backpressure semantics. --- .../s3/output/RetryableS3OutputStream.java | 160 +++--------------- .../s3/output/S3ExportStorageProvider.java | 1 - .../storage/s3/output/S3OutputConfig.java | 4 + .../s3/output/S3StorageConnectorProvider.java | 1 - .../storage/s3/output/S3UploadManager.java | 146 +++++++--------- .../s3/S3StorageConnectorProviderTest.java | 9 +- .../output/RetryableS3OutputStreamTest.java | 11 +- .../s3/output/S3StorageConnectorTest.java | 8 +- .../s3/output/S3UploadManagerTest.java | 64 ------- 9 files changed, 110 insertions(+), 294 deletions(-) delete mode 100644 extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index 40a3a0858bc5..5755bad32ed4 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -19,13 +19,11 @@ package org.apache.druid.storage.s3.output; -import com.amazonaws.AmazonServiceException; import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.InitiateMultipartUploadResult; import com.amazonaws.services.s3.model.PartETag; -import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; import com.google.common.base.Stopwatch; import com.google.common.io.CountingOutputStream; @@ -47,9 +45,9 @@ import java.util.List; import java.util.Objects; import java.util.UUID; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; /** * A retryable output stream for s3. How it works is: @@ -91,12 +89,6 @@ public class RetryableS3OutputStream extends OutputStream private int nextChunkId = 1; // multipart upload requires partNumber to be in the range between 1 and 10000 private final AtomicInteger numChunksPushed = new AtomicInteger(0); - /** - * Total size of all chunks. This size is updated whenever the chunk is ready for push, - * not when {@link #write(byte[], int, int)} is called. - */ - private final AtomicLong resultsSize = new AtomicLong(0); - /** * A flag indicating whether there was an upload error. * This flag is tested in {@link #close()} to determine whether it needs to upload the current chunk or not. @@ -104,26 +96,16 @@ public class RetryableS3OutputStream extends OutputStream private boolean error; private boolean closed; - /** - * An atomic counter to store number of files pending to be uploaded for the particular uploadId. - */ - private final AtomicInteger pendingFiles = new AtomicInteger(0); - - /** - * A lock used for notifying the main thread about the completion of s3.uploadPart() for all chunks - * and hence starting the s3.completeMultipartUpload() for the uploadId. - */ - private final Object fileLock = new Object(); - /** * Helper class for calculating maximum number of simultaneous chunks allowed on local disk. */ private final S3UploadManager uploadManager; /** - * A lock to restrict the maximum number of chunks on disk at any given point in time. + * A list of futures to allow us to wait for completion of all uploadPart() calls + * before hitting {@link #completeMultipartUpload()}. */ - private final Object maxChunksLock = new Object(); + private final List> futures = new ArrayList<>(); public RetryableS3OutputStream( S3OutputConfig config, @@ -176,19 +158,6 @@ public void write(byte[] b, int off, int len) throws IOException return; } - synchronized (maxChunksLock) { - while (uploadManager.getCurrentNumChunksOnDisk() > uploadManager.getMaxConcurrentNumChunks()) { - try { - LOG.debug("Waiting for lock for writing further chunks to local disk."); - maxChunksLock.wait(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - } - try { int offsetToWrite = off; int remainingBytesToWrite = len; @@ -222,88 +191,16 @@ private void pushCurrentChunk() throws IOException currentChunk.close(); final Chunk chunk = currentChunk; if (chunk.length() > 0) { - uploadManager.incrementCurrentNumChunksOnDisk(); - pendingFiles.incrementAndGet(); - - uploadManager.submitTask(() -> { - try { - uploadChunk(chunk); - } - catch (Exception e) { - error = true; - LOG.error(e, e.getMessage()); - throw new RuntimeException(e); - } - finally { - synchronized (maxChunksLock) { - uploadManager.decrementCurrentNumChunksOnDisk(); - maxChunksLock.notifyAll(); - } - if (pendingFiles.decrementAndGet() == 0) { - synchronized (fileLock) { - fileLock.notifyAll(); - } - } - } - }); - } - } - - private void uploadChunk(Chunk chunk) throws IOException - { - try { - LOG.info("Uploading chunk [%d] for uploadId [%s].", chunk.id, uploadId); - resultsSize.addAndGet(chunk.length()); - final PartETag partETag = push(chunk); - synchronized (pushResults) { - pushResults.add(partETag); - } - numChunksPushed.incrementAndGet(); - } - finally { - if (!chunk.delete()) { - LOG.warn("Failed to delete chunk [%s]", chunk.getAbsolutePath()); - } - } - } - - private PartETag push(Chunk chunk) throws IOException - { - try { - return RetryUtils.retry( - () -> uploadPartIfPossible(uploadId, config.getBucket(), s3Key, chunk), - S3Utils.S3RETRY, - config.getMaxRetry() + Future uploadPartResultFuture = uploadManager.queueChunkForUpload( + s3, + s3Key, + chunk.id, + chunk.file, + uploadId, + config ); + futures.add(uploadPartResultFuture); } - catch (AmazonServiceException e) { - throw new IOException(e); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private PartETag uploadPartIfPossible( - String uploadId, - String bucket, - String key, - Chunk chunk - ) - { - final UploadPartRequest uploadPartRequest = new UploadPartRequest() - .withUploadId(uploadId) - .withBucketName(bucket) - .withKey(key) - .withFile(chunk.file) - .withPartNumber(chunk.id) - .withPartSize(chunk.length()); - - if (LOG.isDebugEnabled()) { - LOG.debug("Pushing chunk [%s] to bucket[%s] and key[%s].", chunk, bucket, key); - } - UploadPartResult uploadResult = s3.uploadPart(uploadPartRequest); - return uploadResult.getPartETag(); } @Override @@ -318,10 +215,11 @@ public void close() throws IOException // Closeables are closed in LIFO order closer.register(() -> { // This should be emitted as a metric + long totalChunkSize = (currentChunk.id - 1) * chunkSize + currentChunk.length(); LOG.info( "Pushed total [%d] parts containing [%d] bytes in [%d]ms.", numChunksPushed.get(), - resultsSize.get(), + totalChunkSize, pushStopwatch.elapsed(TimeUnit.MILLISECONDS) ); }); @@ -336,21 +234,21 @@ public void close() throws IOException private void completeMultipartUpload() { - synchronized (fileLock) { - while (pendingFiles.get() > 0) { - try { - LOG.debug("Waiting for lock for completing multipart task for uploadId [%s].", uploadId); - fileLock.wait(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); + for (Future future : futures) { + try { + UploadPartResult result = future.get(); + synchronized (pushResults) { + pushResults.add(result.getPartETag()); } + numChunksPushed.incrementAndGet(); + } + catch (Exception e) { + throw new RuntimeException("Error while waiting for upload to complete", e); } } try { - if (resultsSize.get() > 0 && isAllPushSucceeded()) { + if (isAllPushSucceeded()) { RetryUtils.retry( () -> s3.completeMultipartUpload( new CompleteMultipartUploadRequest(config.getBucket(), s3Key, uploadId, pushResults) @@ -407,16 +305,6 @@ private long length() return outputStream.getCount(); } - private boolean delete() - { - return file.delete(); - } - - private String getAbsolutePath() - { - return file.getAbsolutePath(); - } - @Override public boolean equals(Object o) { diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java index 39ac4a38943e..ca599fc9d495 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java @@ -93,7 +93,6 @@ public StorageConnector get() s3ExportConfig.getChunkSize(), s3ExportConfig.getMaxRetry() ); - s3UploadManager.updateChunkSizeIfGreater(s3ExportConfig.getChunkSize().getBytes()); return new S3StorageConnector(s3OutputConfig, s3, s3UploadManager); } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java index 35e228f7ef3e..159ec094aa35 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java @@ -65,6 +65,10 @@ public S3OutputConfig( this(bucket, prefix, tempDir, chunkSize, maxRetry, true); } + public S3OutputConfig() + { + } + @VisibleForTesting protected S3OutputConfig( String bucket, diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java index 72c9a3534ecf..f86aee9a1aaf 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java @@ -56,7 +56,6 @@ public S3StorageConnectorProvider( @Override public StorageConnector get() { - s3UploadManager.updateChunkSizeIfGreater(this.getChunkSize()); return new S3StorageConnector(this, s3, s3UploadManager); } } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index a0f936364cdd..25c02c49d90a 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -19,117 +19,101 @@ package org.apache.druid.storage.s3.output; +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.amazonaws.services.s3.model.UploadPartResult; import com.google.inject.Inject; -import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.storage.s3.S3Utils; +import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; import org.apache.druid.utils.RuntimeInfo; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicInteger; +import java.io.File; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; /** * This class manages the configuration for uploading files to S3 in chunks. * It tracks the number of chunks currently present on local disk and ensures that * it does not exceed a specified limit. */ +@ManageLifecycle public class S3UploadManager { - /** - * The maximum chunk size based on injected values for {@link S3OutputConfig} and {@link S3ExportConfig} used for computing maximum number of chunks to save on disk at any given point in time. - * It is initialized to 5 MiB which is the minimum chunk size possible, denoted by {@link S3OutputConfig#S3_MULTIPART_UPLOAD_MIN_PART_SIZE_BYTES}. - */ - private long chunkSize = new HumanReadableBytes("5MiB").getBytes(); - - private final AtomicInteger currentNumChunksOnDisk = new AtomicInteger(0); - - /** - * The maximum number of chunks that can be saved to local disk concurrently. - * This value is recalculated when the chunk size is updated in {@link #updateChunkSizeIfGreater(long)}. - */ - private int maxConcurrentNumChunks = 100; - - private final ScheduledExecutorService uploadExecutor; + private final ExecutorService uploadExecutor; private static final Logger log = new Logger(S3UploadManager.class); @Inject - public S3UploadManager(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) + public S3UploadManager(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig, RuntimeInfo runtimeInfo) { int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); - this.uploadExecutor = scheduledExecutorFactory.create( - poolSize, - "UploadThreadPool-%d" - ); + int maxNumConcurrentChunks = 10; + this.uploadExecutor = Execs.newBlockingThreaded("UploadThreadPool-%d", poolSize, maxNumConcurrentChunks); } - /** - * Increments the counter for the current number of chunks saved on disk. - */ - public void incrementCurrentNumChunksOnDisk() + public Future queueChunkForUpload(ServerSideEncryptingAmazonS3 s3Client, String key, int chunkNumber, File chunkFile, String uploadId, S3OutputConfig config) { - currentNumChunksOnDisk.incrementAndGet(); + return uploadExecutor.submit(() -> RetryUtils.retry( + () -> { + log.info("Uploading chunk [%d] for uploadId [%s].", chunkNumber, uploadId); + UploadPartResult uploadPartResult = uploadPartIfPossible( + s3Client, + uploadId, + config.getBucket(), + key, + chunkNumber, + chunkFile + ); + if (!chunkFile.delete()) { + log.warn("Failed to delete chunk [%s]", chunkFile.getAbsolutePath()); + } + return uploadPartResult; + }, + S3Utils.S3RETRY, + config.getMaxRetry() + )); } - /** - * Decrements the counter for the current number of chunks saved on disk. - */ - public void decrementCurrentNumChunksOnDisk() + private UploadPartResult uploadPartIfPossible( + ServerSideEncryptingAmazonS3 s3Client, + String uploadId, + String bucket, + String key, + int chunkNumber, + File chunkFile + ) { - currentNumChunksOnDisk.decrementAndGet(); - } - - /** - * Gets the current number of chunks saved to local disk. - * - * @return the current number of chunks saved to local disk. - */ - public int getCurrentNumChunksOnDisk() - { - return currentNumChunksOnDisk.get(); - } - - /** - * Gets the maximum number of concurrent chunks that can be saved to local disk. - * - * @return the maximum number of concurrent chunks that can be saved to local disk. - */ - public int getMaxConcurrentNumChunks() - { - return maxConcurrentNumChunks; - } - - /** - * Updates the chunk size if the provided size is greater than the current chunk size. - * Recomputes the maximum number of concurrent chunks that can be saved to local disk based on the new chunk size. - * - * @param chunkSize the new chunk size to be set if it is greater than the current chunk size. - */ - public synchronized void updateChunkSizeIfGreater(long chunkSize) - { - if (this.chunkSize < chunkSize) { - this.chunkSize = chunkSize; - recomputeMaxConcurrentNumChunks(); + final UploadPartRequest uploadPartRequest = new UploadPartRequest() + .withUploadId(uploadId) + .withBucketName(bucket) + .withKey(key) + .withFile(chunkFile) + .withPartNumber(chunkNumber) + .withPartSize(chunkFile.length()); + + if (log.isDebugEnabled()) { + log.debug("Pushing chunk [%s] to bucket[%s] and key[%s].", chunkNumber, bucket, key); } + return s3Client.uploadPart(uploadPartRequest); } - /** - * Recomputes the maximum number of concurrent chunks that can be saved to local disk based on the current chunk size. - * The maximum allowed chunk size is {@link S3OutputConfig#S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES} which is quite big, - * so we restrict the maximum disk space used to the same, at any given point in time. - */ - private void recomputeMaxConcurrentNumChunks() + @LifecycleStart + public void start() { - maxConcurrentNumChunks = (int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / chunkSize); - log.info("Recomputed maxConcurrentNumChunks: %d", maxConcurrentNumChunks); + // No state startup required } - /** - * Submit the runnable to the {@link #uploadExecutor}. - * @param task - */ - public void submitTask(Runnable task) + @LifecycleStop + public void stop() { - uploadExecutor.submit(task); + log.debug("Stopping S3UploadManager"); + uploadExecutor.shutdown(); + log.debug("Stopped S3UploadManager"); } + } diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java index 7c5619272afc..676352daf4f5 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java @@ -31,11 +31,13 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.StartupInjectorBuilder; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.query.DruidProcessingConfigTest; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorModule; import org.apache.druid.storage.StorageConnectorProvider; +import org.apache.druid.storage.s3.output.S3ExportConfig; +import org.apache.druid.storage.s3.output.S3OutputConfig; import org.apache.druid.storage.s3.output.S3StorageConnector; import org.apache.druid.storage.s3.output.S3StorageConnectorModule; import org.apache.druid.storage.s3.output.S3StorageConnectorProvider; @@ -153,7 +155,10 @@ public void configure(Binder binder) ) .addValue( S3UploadManager.class, - new S3UploadManager(EasyMock.mock(ScheduledExecutorFactory.class), new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)) + new S3UploadManager( + new S3OutputConfig("bucket", "prefix", EasyMock.mock(File.class), new HumanReadableBytes("5MiB"), 1), + new S3ExportConfig("tempDir", new HumanReadableBytes("5MiB"), 1, null), + new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)) ) ); diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java index 6f24b46e7308..259acec449fb 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java @@ -33,8 +33,6 @@ import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.query.DruidProcessingConfigTest; import org.apache.druid.storage.s3.NoopServerSideEncryption; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; @@ -104,11 +102,10 @@ public int getMaxRetry() } }; - ScheduledExecutorFactory executorFactory = EasyMock.mock(ScheduledExecutorFactory.class); - EasyMock.expect(executorFactory.create(EasyMock.anyInt(), EasyMock.anyString())) - .andReturn(Execs.scheduledSingleThreaded("UploadThreadPool-%d")); - EasyMock.replay(executorFactory); - s3UploadManager = new S3UploadManager(executorFactory, new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)); + s3UploadManager = new S3UploadManager( + new S3OutputConfig("bucket", "prefix", EasyMock.mock(File.class), new HumanReadableBytes("5MiB"), 1), + new S3ExportConfig("tempDir", new HumanReadableBytes("5MiB"), 1, null), + new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)); } @Test diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java index da6d7702542b..a114ac2c8723 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java @@ -31,7 +31,7 @@ import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.query.DruidProcessingConfigTest; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.s3.NoopServerSideEncryption; @@ -49,6 +49,7 @@ import java.io.BufferedReader; import java.io.ByteArrayInputStream; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -86,7 +87,10 @@ public void setup() null, null, true - ), service, new S3UploadManager(EasyMock.mock(ScheduledExecutorFactory.class), new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0))); + ), service, new S3UploadManager( + new S3OutputConfig("bucket", "prefix", EasyMock.mock(File.class), new HumanReadableBytes("5MiB"), 1), + new S3ExportConfig("tempDir", new HumanReadableBytes("5MiB"), 1, null), + new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0))); } catch (IOException e) { throw new RuntimeException(e); diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java deleted file mode 100644 index 4bd18c1cd25e..000000000000 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.storage.s3.output; - -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; -import org.apache.druid.query.DruidProcessingConfigTest; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class S3UploadManagerTest -{ - private S3UploadManager s3UploadManager; - - @Before - public void setup() - { - ScheduledExecutorFactory executorFactory = EasyMock.mock(ScheduledExecutorFactory.class); - EasyMock.expect(executorFactory.create(EasyMock.anyInt(), EasyMock.anyString())) - .andReturn(Execs.scheduledSingleThreaded("UploadThreadPool-%d")); - EasyMock.replay(executorFactory); - s3UploadManager = new S3UploadManager(executorFactory, new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)); - } - - @Test - public void testDefault() - { - Assert.assertEquals(100, s3UploadManager.getMaxConcurrentNumChunks()); - Assert.assertEquals(0, s3UploadManager.getCurrentNumChunksOnDisk()); - } - - @Test - public void testUpdateChunkSize() - { - // Verify that updating chunk size to a smaller or equal value doesn't change the maxConcurrentNumChunks. - s3UploadManager.updateChunkSizeIfGreater(5L * 1024 * 1024L); - Assert.assertEquals(100, s3UploadManager.getMaxConcurrentNumChunks()); - Assert.assertEquals(0, s3UploadManager.getCurrentNumChunksOnDisk()); - - // Verify that updating chunk size to a greater value changes the maxConcurrentNumChunks. - s3UploadManager.updateChunkSizeIfGreater(5L * 1024 * 1024 * 1024L); - Assert.assertEquals(1, s3UploadManager.getMaxConcurrentNumChunks()); - Assert.assertEquals(0, s3UploadManager.getCurrentNumChunksOnDisk()); - } -} From 8d6442bfbc068714d0d579d8571a9cb6648104ce Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Fri, 31 May 2024 18:51:51 +0530 Subject: [PATCH 21/33] Update javadoc --- .../org/apache/druid/storage/s3/output/S3UploadManager.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index 25c02c49d90a..1fcbb507a285 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -37,9 +37,8 @@ import java.util.concurrent.Future; /** - * This class manages the configuration for uploading files to S3 in chunks. - * It tracks the number of chunks currently present on local disk and ensures that - * it does not exceed a specified limit. + * This class manages uploading files to S3 in chunks, while ensuring that the + * number of chunks currently present on local disk does not exceed a specific limit. */ @ManageLifecycle public class S3UploadManager From 295689f5915d55372531510a3b493a5ef6116d18 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 3 Jun 2024 15:48:36 +0530 Subject: [PATCH 22/33] Get rid of cyclic dependency injection between S3UploadManager and S3OutputConfig --- .../AzureStorageConnectorProviderTest.java | 13 ++- .../storage/s3/output/S3OutputConfig.java | 4 - .../s3/output/S3StorageConnectorModule.java | 1 + .../s3/output/S3StorageConnectorProvider.java | 24 ++--- .../storage/s3/output/S3UploadManager.java | 16 +++- .../s3/S3StorageConnectorProviderTest.java | 87 ++----------------- 6 files changed, 34 insertions(+), 111 deletions(-) diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java index f03287e7310e..6f233d6e9628 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java @@ -56,15 +56,14 @@ public void createAzureStorageFactoryWithRequiredProperties() properties.setProperty(CUSTOM_NAMESPACE + ".container", "container"); properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); - StorageConnectorProvider s3StorageConnectorProvider = getStorageConnectorProvider(properties); + StorageConnectorProvider storageConnectorProvider = getStorageConnectorProvider(properties); - assertInstanceOf(AzureStorageConnectorProvider.class, s3StorageConnectorProvider); - assertInstanceOf(AzureStorageConnector.class, s3StorageConnectorProvider.get()); - assertEquals("container", ((AzureStorageConnectorProvider) s3StorageConnectorProvider).getContainer()); - assertEquals("prefix", ((AzureStorageConnectorProvider) s3StorageConnectorProvider).getPrefix()); + assertInstanceOf(AzureStorageConnectorProvider.class, storageConnectorProvider); + assertInstanceOf(AzureStorageConnector.class, storageConnectorProvider.get()); + assertEquals("container", ((AzureStorageConnectorProvider) storageConnectorProvider).getContainer()); + assertEquals("prefix", ((AzureStorageConnectorProvider) storageConnectorProvider).getPrefix()); assertEquals(new File("/tmp"), - ((AzureStorageConnectorProvider) s3StorageConnectorProvider).getTempDir()); - + ((AzureStorageConnectorProvider) storageConnectorProvider).getTempDir()); } @Test diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java index 159ec094aa35..35e228f7ef3e 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3OutputConfig.java @@ -65,10 +65,6 @@ public S3OutputConfig( this(bucket, prefix, tempDir, chunkSize, maxRetry, true); } - public S3OutputConfig() - { - } - @VisibleForTesting protected S3OutputConfig( String bucket, diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java index a57a93a525fb..ea33b18265b2 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java @@ -44,5 +44,6 @@ public List getJacksonModules() public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.export.storage.s3", S3ExportConfig.class); + JsonConfigProvider.bind(binder, "druid.msq.intermediate.storage", S3OutputConfig.class); } } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java index f86aee9a1aaf..4224b461db03 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java @@ -21,41 +21,27 @@ import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorProvider; import org.apache.druid.storage.s3.S3StorageDruidModule; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; -import java.io.File; - @JsonTypeName(S3StorageDruidModule.SCHEME) -public class S3StorageConnectorProvider extends S3OutputConfig implements StorageConnectorProvider +public class S3StorageConnectorProvider implements StorageConnectorProvider { @JacksonInject ServerSideEncryptingAmazonS3 s3; @JacksonInject - S3UploadManager s3UploadManager; + S3OutputConfig s3OutputConfig; - @JsonCreator - public S3StorageConnectorProvider( - @JsonProperty(value = "bucket", required = true) String bucket, - @JsonProperty(value = "prefix", required = true) String prefix, - @JsonProperty(value = "tempDir", required = true) File tempDir, - @JsonProperty("chunkSize") HumanReadableBytes chunkSize, - @JsonProperty("maxRetry") Integer maxRetry - ) - { - super(bucket, prefix, tempDir, chunkSize, maxRetry); - } + @JacksonInject + S3UploadManager s3UploadManager; @Override public StorageConnector get() { - return new S3StorageConnector(this, s3, s3UploadManager); + return new S3StorageConnector(s3OutputConfig, s3, s3UploadManager); } } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index 1fcbb507a285..810a57d92c14 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -51,8 +51,22 @@ public class S3UploadManager public S3UploadManager(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig, RuntimeInfo runtimeInfo) { int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); - int maxNumConcurrentChunks = 10; + int maxNumConcurrentChunks = computeMaxNumConcurrentChunks(s3OutputConfig, s3ExportConfig); this.uploadExecutor = Execs.newBlockingThreaded("UploadThreadPool-%d", poolSize, maxNumConcurrentChunks); + log.info("Creating executor service for S3 multipart upload with pool size [%d] and work queue capacity [%d]", poolSize, maxNumConcurrentChunks); + } + + private int computeMaxNumConcurrentChunks(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig) + { + long chunkSize = S3OutputConfig.S3_MULTIPART_UPLOAD_MIN_PART_SIZE_BYTES; + if (s3OutputConfig != null && s3OutputConfig.getChunkSize() != null) { + chunkSize = Math.max(chunkSize, s3OutputConfig.getChunkSize()); + } + if (s3ExportConfig != null && s3ExportConfig.getChunkSize() != null) { + chunkSize = Math.max(chunkSize, s3ExportConfig.getChunkSize().getBytes()); + } + + return (int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / chunkSize); } public Future queueChunkForUpload(ServerSideEncryptingAmazonS3 s3Client, String key, int chunkNumber, File chunkFile, String uploadId, S3OutputConfig config) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java index 676352daf4f5..ed3250ca52f8 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java @@ -25,18 +25,13 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.ProvisionException; import com.google.inject.name.Names; import org.apache.druid.common.aws.AWSModule; import org.apache.druid.guice.JsonConfigProvider; -import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.StartupInjectorBuilder; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.query.DruidProcessingConfigTest; -import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorModule; import org.apache.druid.storage.StorageConnectorProvider; -import org.apache.druid.storage.s3.output.S3ExportConfig; import org.apache.druid.storage.s3.output.S3OutputConfig; import org.apache.druid.storage.s3.output.S3StorageConnector; import org.apache.druid.storage.s3.output.S3StorageConnectorModule; @@ -57,71 +52,16 @@ public class S3StorageConnectorProviderTest @Test public void createS3StorageFactoryWithRequiredProperties() { - - final Properties properties = new Properties(); - properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); - properties.setProperty(CUSTOM_NAMESPACE + ".bucket", "bucket"); - properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); - properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); - StorageConnectorProvider s3StorageConnectorProvider = getStorageConnectorProvider(properties); + StorageConnectorProvider s3StorageConnectorProvider = getStorageConnectorProvider(); Assert.assertTrue(s3StorageConnectorProvider instanceof S3StorageConnectorProvider); Assert.assertTrue(s3StorageConnectorProvider.get() instanceof S3StorageConnector); - Assert.assertEquals("bucket", ((S3StorageConnectorProvider) s3StorageConnectorProvider).getBucket()); - Assert.assertEquals("prefix", ((S3StorageConnectorProvider) s3StorageConnectorProvider).getPrefix()); - Assert.assertEquals(new File("/tmp"), ((S3StorageConnectorProvider) s3StorageConnectorProvider).getTempDir()); - - } - - @Test - public void createS3StorageFactoryWithMissingPrefix() - { - - final Properties properties = new Properties(); - properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); - properties.setProperty(CUSTOM_NAMESPACE + ".bucket", "bucket"); - properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); - Assert.assertThrows( - "Missing required creator property 'prefix'", - ProvisionException.class, - () -> getStorageConnectorProvider(properties) - ); } - - @Test - public void createS3StorageFactoryWithMissingBucket() + private StorageConnectorProvider getStorageConnectorProvider() { - final Properties properties = new Properties(); properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); - properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); - properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); - Assert.assertThrows( - "Missing required creator property 'bucket'", - ProvisionException.class, - () -> getStorageConnectorProvider(properties) - ); - } - - @Test - public void createS3StorageFactoryWithMissingTempDir() - { - - final Properties properties = new Properties(); - properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); - properties.setProperty(CUSTOM_NAMESPACE + ".bucket", "bucket"); - properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); - - Assert.assertThrows( - "Missing required creator property 'tempDir'", - ProvisionException.class, - () -> getStorageConnectorProvider(properties) - ); - } - - private StorageConnectorProvider getStorageConnectorProvider(Properties properties) - { StartupInjectorBuilder startupInjectorBuilder = new StartupInjectorBuilder().add( new AWSModule(), new StorageConnectorModule(), @@ -137,36 +77,23 @@ public void configure(Binder binder) StorageConnectorProvider.class, Names.named(CUSTOM_NAMESPACE) ); - - binder.bind(Key.get(StorageConnector.class, Names.named(CUSTOM_NAMESPACE))) - .toProvider(Key.get(StorageConnectorProvider.class, Names.named(CUSTOM_NAMESPACE))) - .in(LazySingleton.class); } } ).withProperties(properties); Injector injector = startupInjectorBuilder.build(); injector.getInstance(ObjectMapper.class).registerModules(new S3StorageConnectorModule().getJacksonModules()); + injector.getInstance(ObjectMapper.class).setInjectableValues( new InjectableValues.Std() - .addValue( - ServerSideEncryptingAmazonS3.class, - new ServerSideEncryptingAmazonS3(null, new NoopServerSideEncryption()) - ) - .addValue( - S3UploadManager.class, - new S3UploadManager( - new S3OutputConfig("bucket", "prefix", EasyMock.mock(File.class), new HumanReadableBytes("5MiB"), 1), - new S3ExportConfig("tempDir", new HumanReadableBytes("5MiB"), 1, null), - new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)) - ) + .addValue(ServerSideEncryptingAmazonS3.class, EasyMock.mock(ServerSideEncryptingAmazonS3.class)) + .addValue(S3OutputConfig.class, new S3OutputConfig("bucket", "prefix", new File("/tmp"), new HumanReadableBytes("5MiB"), 1)) + .addValue(S3UploadManager.class, EasyMock.mock(S3UploadManager.class)) ); - - StorageConnectorProvider storageConnectorProvider = injector.getInstance(Key.get( + return injector.getInstance(Key.get( StorageConnectorProvider.class, Names.named(CUSTOM_NAMESPACE) )); - return storageConnectorProvider; } } From 6ec023fcf48170b53893ecef68f1bb23b5c4b08e Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 3 Jun 2024 16:38:27 +0530 Subject: [PATCH 23/33] Fix RetryableS3OutputStreamTest --- .../druid/storage/s3/output/RetryableS3OutputStream.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index 5755bad32ed4..73e7f5be133c 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -243,7 +243,9 @@ private void completeMultipartUpload() numChunksPushed.incrementAndGet(); } catch (Exception e) { - throw new RuntimeException("Error while waiting for upload to complete", e); + error = true; + LOG.error(e, "Error in uploading part for upload ID [%s]", uploadId); + break; } } From f43a0c91d375696af08b7557071f27499cff2025 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 3 Jun 2024 17:03:10 +0530 Subject: [PATCH 24/33] Remove unnecessary synchronization parts from RetryableS3OutputStream --- .../storage/s3/output/RetryableS3OutputStream.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index 73e7f5be133c..5fe0f3b30d93 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -47,7 +47,6 @@ import java.util.UUID; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; /** * A retryable output stream for s3. How it works is: @@ -87,7 +86,6 @@ public class RetryableS3OutputStream extends OutputStream private Chunk currentChunk; private int nextChunkId = 1; // multipart upload requires partNumber to be in the range between 1 and 10000 - private final AtomicInteger numChunksPushed = new AtomicInteger(0); /** * A flag indicating whether there was an upload error. @@ -103,7 +101,7 @@ public class RetryableS3OutputStream extends OutputStream /** * A list of futures to allow us to wait for completion of all uploadPart() calls - * before hitting {@link #completeMultipartUpload()}. + * before hitting {@link ServerSideEncryptingAmazonS3#completeMultipartUpload(CompleteMultipartUploadRequest)}. */ private final List> futures = new ArrayList<>(); @@ -218,7 +216,7 @@ public void close() throws IOException long totalChunkSize = (currentChunk.id - 1) * chunkSize + currentChunk.length(); LOG.info( "Pushed total [%d] parts containing [%d] bytes in [%d]ms.", - numChunksPushed.get(), + futures.size(), totalChunkSize, pushStopwatch.elapsed(TimeUnit.MILLISECONDS) ); @@ -237,10 +235,7 @@ private void completeMultipartUpload() for (Future future : futures) { try { UploadPartResult result = future.get(); - synchronized (pushResults) { - pushResults.add(result.getPartETag()); - } - numChunksPushed.incrementAndGet(); + pushResults.add(result.getPartETag()); } catch (Exception e) { error = true; @@ -285,7 +280,7 @@ private void completeMultipartUpload() private boolean isAllPushSucceeded() { - return !error && !pushResults.isEmpty() && numChunksPushed.get() == pushResults.size(); + return !error && !pushResults.isEmpty() && futures.size() == pushResults.size(); } private static class Chunk implements Closeable From 79a22edf27efb5ee1b3cfce7d6311baa07685f51 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Mon, 3 Jun 2024 17:09:14 +0530 Subject: [PATCH 25/33] Update javadoc --- .../druid/storage/s3/output/S3UploadManager.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index 810a57d92c14..a5ef7169af44 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -53,9 +53,19 @@ public S3UploadManager(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportCon int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); int maxNumConcurrentChunks = computeMaxNumConcurrentChunks(s3OutputConfig, s3ExportConfig); this.uploadExecutor = Execs.newBlockingThreaded("UploadThreadPool-%d", poolSize, maxNumConcurrentChunks); - log.info("Creating executor service for S3 multipart upload with pool size [%d] and work queue capacity [%d]", poolSize, maxNumConcurrentChunks); + log.info("Initialized executor service for S3 multipart upload with pool size [%d] and work queue capacity [%d]", + poolSize, maxNumConcurrentChunks); } + /** + * Computes the maximum number of concurrent chunks for an S3 multipart upload. + * We want to determine the maximum number of concurrent chunks on disk based on the maximum value of chunkSize + * between the 2 configs: S3OutputConfig and S3ExportConfig. + * + * @param s3OutputConfig The S3 output configuration, which may specify a custom chunk size. + * @param s3ExportConfig The S3 export configuration, which may also specify a custom chunk size. + * @return The maximum number of concurrent chunks. + */ private int computeMaxNumConcurrentChunks(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig) { long chunkSize = S3OutputConfig.S3_MULTIPART_UPLOAD_MIN_PART_SIZE_BYTES; @@ -69,6 +79,9 @@ private int computeMaxNumConcurrentChunks(S3OutputConfig s3OutputConfig, S3Expor return (int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / chunkSize); } + /** + * Queues a chunk of a file for upload to S3 as part of a multipart upload. + */ public Future queueChunkForUpload(ServerSideEncryptingAmazonS3 s3Client, String key, int chunkNumber, File chunkFile, String uploadId, S3OutputConfig config) { return uploadExecutor.submit(() -> RetryUtils.retry( From da5e57af03fa6abc97a4cf66a895c926c44e48f0 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Tue, 4 Jun 2024 11:24:07 +0530 Subject: [PATCH 26/33] Add S3UploadManagerTest --- .../AzureStorageConnectorProviderTest.java | 2 +- .../storage/s3/output/S3UploadManager.java | 15 +- .../s3/output/S3StorageConnectorTest.java | 8 +- .../s3/output/S3UploadManagerTest.java | 174 ++++++++++++++++++ 4 files changed, 191 insertions(+), 8 deletions(-) create mode 100644 extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java index 6f233d6e9628..0b76f02af29e 100644 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java +++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java @@ -71,7 +71,7 @@ public void createAzureStorageFactoryWithMissingPrefix() { final Properties properties = new Properties(); - properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); + properties.setProperty(CUSTOM_NAMESPACE + ".type", "azure"); properties.setProperty(CUSTOM_NAMESPACE + ".container", "container"); properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); assertThrows( diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index a5ef7169af44..08e900904229 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -21,6 +21,7 @@ import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; +import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.RetryUtils; @@ -52,7 +53,7 @@ public S3UploadManager(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportCon { int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); int maxNumConcurrentChunks = computeMaxNumConcurrentChunks(s3OutputConfig, s3ExportConfig); - this.uploadExecutor = Execs.newBlockingThreaded("UploadThreadPool-%d", poolSize, maxNumConcurrentChunks); + this.uploadExecutor = createExecutorService(poolSize, maxNumConcurrentChunks); log.info("Initialized executor service for S3 multipart upload with pool size [%d] and work queue capacity [%d]", poolSize, maxNumConcurrentChunks); } @@ -66,7 +67,8 @@ public S3UploadManager(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportCon * @param s3ExportConfig The S3 export configuration, which may also specify a custom chunk size. * @return The maximum number of concurrent chunks. */ - private int computeMaxNumConcurrentChunks(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig) + @VisibleForTesting + int computeMaxNumConcurrentChunks(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig) { long chunkSize = S3OutputConfig.S3_MULTIPART_UPLOAD_MIN_PART_SIZE_BYTES; if (s3OutputConfig != null && s3OutputConfig.getChunkSize() != null) { @@ -105,7 +107,8 @@ public Future queueChunkForUpload(ServerSideEncryptingAmazonS3 )); } - private UploadPartResult uploadPartIfPossible( + @VisibleForTesting + UploadPartResult uploadPartIfPossible( ServerSideEncryptingAmazonS3 s3Client, String uploadId, String bucket, @@ -128,6 +131,12 @@ private UploadPartResult uploadPartIfPossible( return s3Client.uploadPart(uploadPartRequest); } + @VisibleForTesting + ExecutorService createExecutorService(int poolSize, int maxNumConcurrentChunks) + { + return Execs.newBlockingThreaded("UploadThreadPool-%d", poolSize, maxNumConcurrentChunks); + } + @LifecycleStart public void start() { diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java index a114ac2c8723..67dcb3b6db6c 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java @@ -49,7 +49,6 @@ import java.io.BufferedReader; import java.io.ByteArrayInputStream; -import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -80,15 +79,16 @@ public class S3StorageConnectorTest public void setup() { try { - storageConnector = new S3StorageConnector(new S3OutputConfig( + S3OutputConfig s3OutputConfig = new S3OutputConfig( BUCKET, PREFIX, temporaryFolder.newFolder(), null, null, true - ), service, new S3UploadManager( - new S3OutputConfig("bucket", "prefix", EasyMock.mock(File.class), new HumanReadableBytes("5MiB"), 1), + ); + storageConnector = new S3StorageConnector(s3OutputConfig, service, new S3UploadManager( + s3OutputConfig, new S3ExportConfig("tempDir", new HumanReadableBytes("5MiB"), 1, null), new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0))); } diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java new file mode 100644 index 000000000000..126afa9fed56 --- /dev/null +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.storage.s3.output; + +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.amazonaws.services.s3.model.UploadPartResult; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; +import org.apache.druid.utils.RuntimeInfo; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class S3UploadManagerTest +{ + + private S3UploadManager s3UploadManager; + private S3OutputConfig s3OutputConfig; + private S3ExportConfig s3ExportConfig; + private static ExecutorService uploadExecutor; + + @Before + public void setUp() + { + s3OutputConfig = EasyMock.mock(S3OutputConfig.class); + s3ExportConfig = EasyMock.mock(S3ExportConfig.class); + final RuntimeInfo runtimeInfo = EasyMock.mock(RuntimeInfo.class); + uploadExecutor = Execs.singleThreaded("UploadThreadPool-%d"); + + EasyMock.expect(runtimeInfo.getAvailableProcessors()).andReturn(8).anyTimes(); + EasyMock.expect(s3OutputConfig.getChunkSize()).andReturn(100L * 1024 * 1024).anyTimes(); // 100 MB + EasyMock.expect(s3OutputConfig.getMaxRetry()).andReturn(1).anyTimes(); + EasyMock.expect(s3OutputConfig.getBucket()).andReturn("bucket").anyTimes(); + EasyMock.expect(s3ExportConfig.getChunkSize()).andReturn(HumanReadableBytes.valueOf(200L * 1024 * 1024)).anyTimes(); // 200 MB + + EasyMock.replay(runtimeInfo, s3OutputConfig, s3ExportConfig); + + s3UploadManager = new TestS3UploadManager(s3OutputConfig, s3ExportConfig, runtimeInfo); + } + + @Test + public void testQueueChunkForUpload() throws Exception + { + ServerSideEncryptingAmazonS3 s3Client = EasyMock.mock(ServerSideEncryptingAmazonS3.class); + + File chunkFile = EasyMock.mock(File.class); + EasyMock.expect(chunkFile.length()).andReturn(1024L).anyTimes(); + EasyMock.expect(chunkFile.delete()).andReturn(true).anyTimes(); + + int chunkId = 42; + UploadPartResult uploadPartResult = new UploadPartResult(); + uploadPartResult.setPartNumber(chunkId); + uploadPartResult.setETag("etag"); + EasyMock.expect(s3Client.uploadPart(EasyMock.anyObject(UploadPartRequest.class))).andReturn(uploadPartResult); + + EasyMock.replay(chunkFile, s3Client); + + Future result = s3UploadManager.queueChunkForUpload(s3Client, "test-key", chunkId, chunkFile, "upload-id", s3OutputConfig); + + UploadPartResult futureResult = result.get(); + Assert.assertEquals(chunkId, futureResult.getPartNumber()); + Assert.assertEquals("etag", futureResult.getETag()); + } + + @Test + public void testComputeMaxNumConcurrentChunks() + { + int maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(s3OutputConfig, s3ExportConfig); + int expectedMaxNumConcurrentChunks = 25; // maxChunkSizePossible/200 MB + assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); + } + + @Test + public void testComputeMaxNumConcurrentChunksWithNullOutputConfig() + { + // Null S3OutputConfig + int maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(null, s3ExportConfig); + int expectedMaxNumConcurrentChunks = 25; // maxChunkSizePossible / s3ExportConfig's chunk size + assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); + + // Null S3OutputConfig#getChunkSize() + maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(EasyMock.mock(S3OutputConfig.class), s3ExportConfig); + assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); + } + + @Test + public void testComputeMaxNumConcurrentChunksWithNullExportConfig() + { + // Null S3ExportConfig + int maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(s3OutputConfig, null); + int expectedMaxNumConcurrentChunks = 51; // maxChunkSizePossible / s3OutputConfig's chunk size + assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); + + // Null S3ExportConfig#getChunkSize() + maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(s3OutputConfig, EasyMock.mock(S3ExportConfig.class)); + assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); + } + + @Test + public void testUploadPartIfPossible() + { + ServerSideEncryptingAmazonS3 s3Client = EasyMock.mock(ServerSideEncryptingAmazonS3.class); + + File chunkFile = EasyMock.mock(File.class); + EasyMock.expect(chunkFile.length()).andReturn(1024L).anyTimes(); + + UploadPartResult uploadPartResult = new UploadPartResult(); + Capture partRequestCapture = EasyMock.newCapture(); + EasyMock.expect(s3Client.uploadPart(EasyMock.capture(partRequestCapture))).andReturn(uploadPartResult); + EasyMock.replay(s3Client, chunkFile); + + UploadPartResult result = s3UploadManager.uploadPartIfPossible(s3Client, "upload-id", "bucket", "key", 1, chunkFile); + + UploadPartRequest capturedRequest = partRequestCapture.getValue(); + assertEquals("upload-id", capturedRequest.getUploadId()); + assertEquals("bucket", capturedRequest.getBucketName()); + assertEquals("key", capturedRequest.getKey()); + assertEquals(1, capturedRequest.getPartNumber()); + assertEquals(chunkFile, capturedRequest.getFile()); + assertEquals(1024L, capturedRequest.getPartSize()); + + assertEquals(uploadPartResult, result); + } + + @Test + public void testStartAndStop() + { + s3UploadManager.start(); + s3UploadManager.stop(); + + assertTrue(uploadExecutor.isShutdown()); + } + + private static class TestS3UploadManager extends S3UploadManager + { + public TestS3UploadManager(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig, RuntimeInfo runtimeInfo) + { + super(s3OutputConfig, s3ExportConfig, runtimeInfo); + } + + @Override + ExecutorService createExecutorService(int poolSize, int maxNumConcurrentChunks) + { + return uploadExecutor; + } + } +} From 059a5e9359fed30f00907cf2a40e13aeb317bd75 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Tue, 4 Jun 2024 20:56:37 +0530 Subject: [PATCH 27/33] Revert back to S3StorageConnectorProvider extends S3OutputConfig --- .../s3/output/S3StorageConnectorProvider.java | 24 +++-- .../s3/S3StorageConnectorProviderTest.java | 87 +++++++++++++++++-- 2 files changed, 99 insertions(+), 12 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java index 4224b461db03..f86aee9a1aaf 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorProvider.java @@ -21,27 +21,41 @@ import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorProvider; import org.apache.druid.storage.s3.S3StorageDruidModule; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; +import java.io.File; + @JsonTypeName(S3StorageDruidModule.SCHEME) -public class S3StorageConnectorProvider implements StorageConnectorProvider +public class S3StorageConnectorProvider extends S3OutputConfig implements StorageConnectorProvider { @JacksonInject ServerSideEncryptingAmazonS3 s3; - @JacksonInject - S3OutputConfig s3OutputConfig; - @JacksonInject S3UploadManager s3UploadManager; + @JsonCreator + public S3StorageConnectorProvider( + @JsonProperty(value = "bucket", required = true) String bucket, + @JsonProperty(value = "prefix", required = true) String prefix, + @JsonProperty(value = "tempDir", required = true) File tempDir, + @JsonProperty("chunkSize") HumanReadableBytes chunkSize, + @JsonProperty("maxRetry") Integer maxRetry + ) + { + super(bucket, prefix, tempDir, chunkSize, maxRetry); + } + @Override public StorageConnector get() { - return new S3StorageConnector(s3OutputConfig, s3, s3UploadManager); + return new S3StorageConnector(this, s3, s3UploadManager); } } diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java index ed3250ca52f8..676352daf4f5 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java @@ -25,13 +25,18 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; +import com.google.inject.ProvisionException; import com.google.inject.name.Names; import org.apache.druid.common.aws.AWSModule; import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.StartupInjectorBuilder; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.query.DruidProcessingConfigTest; +import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.StorageConnectorModule; import org.apache.druid.storage.StorageConnectorProvider; +import org.apache.druid.storage.s3.output.S3ExportConfig; import org.apache.druid.storage.s3.output.S3OutputConfig; import org.apache.druid.storage.s3.output.S3StorageConnector; import org.apache.druid.storage.s3.output.S3StorageConnectorModule; @@ -52,16 +57,71 @@ public class S3StorageConnectorProviderTest @Test public void createS3StorageFactoryWithRequiredProperties() { - StorageConnectorProvider s3StorageConnectorProvider = getStorageConnectorProvider(); + + final Properties properties = new Properties(); + properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); + properties.setProperty(CUSTOM_NAMESPACE + ".bucket", "bucket"); + properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); + properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); + StorageConnectorProvider s3StorageConnectorProvider = getStorageConnectorProvider(properties); Assert.assertTrue(s3StorageConnectorProvider instanceof S3StorageConnectorProvider); Assert.assertTrue(s3StorageConnectorProvider.get() instanceof S3StorageConnector); + Assert.assertEquals("bucket", ((S3StorageConnectorProvider) s3StorageConnectorProvider).getBucket()); + Assert.assertEquals("prefix", ((S3StorageConnectorProvider) s3StorageConnectorProvider).getPrefix()); + Assert.assertEquals(new File("/tmp"), ((S3StorageConnectorProvider) s3StorageConnectorProvider).getTempDir()); + + } + + @Test + public void createS3StorageFactoryWithMissingPrefix() + { + + final Properties properties = new Properties(); + properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); + properties.setProperty(CUSTOM_NAMESPACE + ".bucket", "bucket"); + properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); + Assert.assertThrows( + "Missing required creator property 'prefix'", + ProvisionException.class, + () -> getStorageConnectorProvider(properties) + ); } - private StorageConnectorProvider getStorageConnectorProvider() + + @Test + public void createS3StorageFactoryWithMissingBucket() { + final Properties properties = new Properties(); properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); + properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); + properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp"); + Assert.assertThrows( + "Missing required creator property 'bucket'", + ProvisionException.class, + () -> getStorageConnectorProvider(properties) + ); + } + + @Test + public void createS3StorageFactoryWithMissingTempDir() + { + + final Properties properties = new Properties(); + properties.setProperty(CUSTOM_NAMESPACE + ".type", "s3"); + properties.setProperty(CUSTOM_NAMESPACE + ".bucket", "bucket"); + properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix"); + + Assert.assertThrows( + "Missing required creator property 'tempDir'", + ProvisionException.class, + () -> getStorageConnectorProvider(properties) + ); + } + + private StorageConnectorProvider getStorageConnectorProvider(Properties properties) + { StartupInjectorBuilder startupInjectorBuilder = new StartupInjectorBuilder().add( new AWSModule(), new StorageConnectorModule(), @@ -77,23 +137,36 @@ public void configure(Binder binder) StorageConnectorProvider.class, Names.named(CUSTOM_NAMESPACE) ); + + binder.bind(Key.get(StorageConnector.class, Names.named(CUSTOM_NAMESPACE))) + .toProvider(Key.get(StorageConnectorProvider.class, Names.named(CUSTOM_NAMESPACE))) + .in(LazySingleton.class); } } ).withProperties(properties); Injector injector = startupInjectorBuilder.build(); injector.getInstance(ObjectMapper.class).registerModules(new S3StorageConnectorModule().getJacksonModules()); - injector.getInstance(ObjectMapper.class).setInjectableValues( new InjectableValues.Std() - .addValue(ServerSideEncryptingAmazonS3.class, EasyMock.mock(ServerSideEncryptingAmazonS3.class)) - .addValue(S3OutputConfig.class, new S3OutputConfig("bucket", "prefix", new File("/tmp"), new HumanReadableBytes("5MiB"), 1)) - .addValue(S3UploadManager.class, EasyMock.mock(S3UploadManager.class)) + .addValue( + ServerSideEncryptingAmazonS3.class, + new ServerSideEncryptingAmazonS3(null, new NoopServerSideEncryption()) + ) + .addValue( + S3UploadManager.class, + new S3UploadManager( + new S3OutputConfig("bucket", "prefix", EasyMock.mock(File.class), new HumanReadableBytes("5MiB"), 1), + new S3ExportConfig("tempDir", new HumanReadableBytes("5MiB"), 1, null), + new DruidProcessingConfigTest.MockRuntimeInfo(10, 0, 0)) + ) ); - return injector.getInstance(Key.get( + + StorageConnectorProvider storageConnectorProvider = injector.getInstance(Key.get( StorageConnectorProvider.class, Names.named(CUSTOM_NAMESPACE) )); + return storageConnectorProvider; } } From 70c73525886779abb47c9e95b067b65995f08bea Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Wed, 5 Jun 2024 11:51:04 +0530 Subject: [PATCH 28/33] Address Karan's review comments --- .../org/apache/druid/storage/s3/output/S3UploadManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index 08e900904229..0b8aced860d8 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -134,7 +134,7 @@ UploadPartResult uploadPartIfPossible( @VisibleForTesting ExecutorService createExecutorService(int poolSize, int maxNumConcurrentChunks) { - return Execs.newBlockingThreaded("UploadThreadPool-%d", poolSize, maxNumConcurrentChunks); + return Execs.newBlockingThreaded("S3UploadThreadPool-%d", poolSize, maxNumConcurrentChunks); } @LifecycleStart @@ -146,9 +146,9 @@ public void start() @LifecycleStop public void stop() { - log.debug("Stopping S3UploadManager"); + log.info("Stopping S3UploadManager"); uploadExecutor.shutdown(); - log.debug("Stopped S3UploadManager"); + log.info("Stopped S3UploadManager"); } } From 749b20dc2a216e07763c7d3fb836cb9381688659 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 6 Jun 2024 09:39:13 +0530 Subject: [PATCH 29/33] Address Kashif's review comments --- .../s3/output/RetryableS3OutputStream.java | 38 ++++++------------ .../storage/s3/output/S3UploadManager.java | 39 +++++++++---------- .../output/RetryableS3OutputStreamTest.java | 1 - .../s3/output/S3UploadManagerTest.java | 16 ++++---- 4 files changed, 39 insertions(+), 55 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java index 5fe0f3b30d93..d0e5d0ee3ff6 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java @@ -78,7 +78,6 @@ public class RetryableS3OutputStream extends OutputStream private final File chunkStorePath; private final long chunkSize; - private final List pushResults = new ArrayList<>(); private final byte[] singularBuffer = new byte[1]; // metric @@ -189,15 +188,9 @@ private void pushCurrentChunk() throws IOException currentChunk.close(); final Chunk chunk = currentChunk; if (chunk.length() > 0) { - Future uploadPartResultFuture = uploadManager.queueChunkForUpload( - s3, - s3Key, - chunk.id, - chunk.file, - uploadId, - config + futures.add( + uploadManager.queueChunkForUpload(s3, s3Key, chunk.id, chunk.file, uploadId, config) ); - futures.add(uploadPartResultFuture); } } @@ -212,6 +205,9 @@ public void close() throws IOException // Closeables are closed in LIFO order closer.register(() -> { + org.apache.commons.io.FileUtils.forceDelete(chunkStorePath); + LOG.info("Deleted chunkStorePath[%s]", chunkStorePath); + // This should be emitted as a metric long totalChunkSize = (currentChunk.id - 1) * chunkSize + currentChunk.length(); LOG.info( @@ -232,20 +228,24 @@ public void close() throws IOException private void completeMultipartUpload() { + final List pushResults = new ArrayList<>(); for (Future future : futures) { + if (error) { + future.cancel(true); + } try { - UploadPartResult result = future.get(); + UploadPartResult result = future.get(1, TimeUnit.HOURS); pushResults.add(result.getPartETag()); } catch (Exception e) { error = true; LOG.error(e, "Error in uploading part for upload ID [%s]", uploadId); - break; } } try { - if (isAllPushSucceeded()) { + boolean isAllPushSucceeded = !error && !pushResults.isEmpty() && futures.size() == pushResults.size(); + if (isAllPushSucceeded) { RetryUtils.retry( () -> s3.completeMultipartUpload( new CompleteMultipartUploadRequest(config.getBucket(), s3Key, uploadId, pushResults) @@ -267,20 +267,6 @@ private void completeMultipartUpload() catch (Exception e) { throw new RuntimeException(e); } - finally { - try { - org.apache.commons.io.FileUtils.forceDelete(chunkStorePath); - pushStopwatch.stop(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - } - - private boolean isAllPushSucceeded() - { - return !error && !pushResults.isEmpty() && futures.size() == pushResults.size(); } private static class Chunk implements Closeable diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index 0b8aced860d8..8c90c58e0b04 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -52,39 +52,40 @@ public class S3UploadManager public S3UploadManager(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig, RuntimeInfo runtimeInfo) { int poolSize = Math.max(4, runtimeInfo.getAvailableProcessors()); - int maxNumConcurrentChunks = computeMaxNumConcurrentChunks(s3OutputConfig, s3ExportConfig); - this.uploadExecutor = createExecutorService(poolSize, maxNumConcurrentChunks); + int maxNumChunksOnDisk = computeMaxNumChunksOnDisk(s3OutputConfig, s3ExportConfig); + this.uploadExecutor = createExecutorService(poolSize, maxNumChunksOnDisk); log.info("Initialized executor service for S3 multipart upload with pool size [%d] and work queue capacity [%d]", - poolSize, maxNumConcurrentChunks); + poolSize, maxNumChunksOnDisk); } /** - * Computes the maximum number of concurrent chunks for an S3 multipart upload. - * We want to determine the maximum number of concurrent chunks on disk based on the maximum value of chunkSize - * between the 2 configs: S3OutputConfig and S3ExportConfig. - * - * @param s3OutputConfig The S3 output configuration, which may specify a custom chunk size. - * @param s3ExportConfig The S3 export configuration, which may also specify a custom chunk size. - * @return The maximum number of concurrent chunks. + * Computes the maximum number of S3 upload chunks that can be kept on disk using the + * maximum chunk size specified in {@link S3OutputConfig} and {@link S3ExportConfig}. */ - @VisibleForTesting - int computeMaxNumConcurrentChunks(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig) + public static int computeMaxNumChunksOnDisk(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig) { - long chunkSize = S3OutputConfig.S3_MULTIPART_UPLOAD_MIN_PART_SIZE_BYTES; + long maxChunkSize = S3OutputConfig.S3_MULTIPART_UPLOAD_MIN_PART_SIZE_BYTES; if (s3OutputConfig != null && s3OutputConfig.getChunkSize() != null) { - chunkSize = Math.max(chunkSize, s3OutputConfig.getChunkSize()); + maxChunkSize = Math.max(maxChunkSize, s3OutputConfig.getChunkSize()); } if (s3ExportConfig != null && s3ExportConfig.getChunkSize() != null) { - chunkSize = Math.max(chunkSize, s3ExportConfig.getChunkSize().getBytes()); + maxChunkSize = Math.max(maxChunkSize, s3ExportConfig.getChunkSize().getBytes()); } - return (int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / chunkSize); + return (int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / maxChunkSize); } /** * Queues a chunk of a file for upload to S3 as part of a multipart upload. */ - public Future queueChunkForUpload(ServerSideEncryptingAmazonS3 s3Client, String key, int chunkNumber, File chunkFile, String uploadId, S3OutputConfig config) + public Future queueChunkForUpload( + ServerSideEncryptingAmazonS3 s3Client, + String key, + int chunkNumber, + File chunkFile, + String uploadId, + S3OutputConfig config + ) { return uploadExecutor.submit(() -> RetryUtils.retry( () -> { @@ -126,7 +127,7 @@ UploadPartResult uploadPartIfPossible( .withPartSize(chunkFile.length()); if (log.isDebugEnabled()) { - log.debug("Pushing chunk [%s] to bucket[%s] and key[%s].", chunkNumber, bucket, key); + log.debug("Pushing chunk[%s] to bucket[%s] and key[%s].", chunkNumber, bucket, key); } return s3Client.uploadPart(uploadPartRequest); } @@ -146,9 +147,7 @@ public void start() @LifecycleStop public void stop() { - log.info("Stopping S3UploadManager"); uploadExecutor.shutdown(); - log.info("Stopped S3UploadManager"); } } diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java index 259acec449fb..8e7a81eb48dd 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java @@ -213,7 +213,6 @@ public void testFailToUploadAfterRetries() throws IOException out.write(bb.array()); } - // No exception is thrown in the main thread, since the upload now happens in a separate threadpool. bb.clear(); bb.putInt(3); out.write(bb.array()); diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java index 126afa9fed56..b5846fac3952 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java @@ -90,36 +90,36 @@ public void testQueueChunkForUpload() throws Exception } @Test - public void testComputeMaxNumConcurrentChunks() + public void testComputeMaxNumChunksOnDisk() { - int maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(s3OutputConfig, s3ExportConfig); + int maxNumConcurrentChunks = S3UploadManager.computeMaxNumChunksOnDisk(s3OutputConfig, s3ExportConfig); int expectedMaxNumConcurrentChunks = 25; // maxChunkSizePossible/200 MB assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); } @Test - public void testComputeMaxNumConcurrentChunksWithNullOutputConfig() + public void testComputeMaxNumChunksOnDiskWithNullOutputConfig() { // Null S3OutputConfig - int maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(null, s3ExportConfig); + int maxNumConcurrentChunks = S3UploadManager.computeMaxNumChunksOnDisk(null, s3ExportConfig); int expectedMaxNumConcurrentChunks = 25; // maxChunkSizePossible / s3ExportConfig's chunk size assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); // Null S3OutputConfig#getChunkSize() - maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(EasyMock.mock(S3OutputConfig.class), s3ExportConfig); + maxNumConcurrentChunks = S3UploadManager.computeMaxNumChunksOnDisk(EasyMock.mock(S3OutputConfig.class), s3ExportConfig); assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); } @Test - public void testComputeMaxNumConcurrentChunksWithNullExportConfig() + public void testComputeMaxNumChunksOnDiskWithNullExportConfig() { // Null S3ExportConfig - int maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(s3OutputConfig, null); + int maxNumConcurrentChunks = S3UploadManager.computeMaxNumChunksOnDisk(s3OutputConfig, null); int expectedMaxNumConcurrentChunks = 51; // maxChunkSizePossible / s3OutputConfig's chunk size assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); // Null S3ExportConfig#getChunkSize() - maxNumConcurrentChunks = s3UploadManager.computeMaxNumConcurrentChunks(s3OutputConfig, EasyMock.mock(S3ExportConfig.class)); + maxNumConcurrentChunks = S3UploadManager.computeMaxNumChunksOnDisk(s3OutputConfig, EasyMock.mock(S3ExportConfig.class)); assertEquals(expectedMaxNumConcurrentChunks, maxNumConcurrentChunks); } From 258376e4bc4119379771789b25305c9afe8fc803 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 6 Jun 2024 16:16:07 +0530 Subject: [PATCH 30/33] Change a log message to debug --- .../org/apache/druid/storage/s3/output/S3UploadManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index 8c90c58e0b04..cd635e422084 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -89,7 +89,7 @@ public Future queueChunkForUpload( { return uploadExecutor.submit(() -> RetryUtils.retry( () -> { - log.info("Uploading chunk [%d] for uploadId [%s].", chunkNumber, uploadId); + log.debug("Uploading chunk [%d] for uploadId [%s].", chunkNumber, uploadId); UploadPartResult uploadPartResult = uploadPartIfPossible( s3Client, uploadId, From 8e7882ed8db1c08fa0fa5b3c70c3b4222ef28c2c Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 6 Jun 2024 19:54:27 +0530 Subject: [PATCH 31/33] Address review comments --- .../storage/s3/output/S3UploadManager.java | 5 +- .../s3/output/S3UploadManagerTest.java | 46 ++++--------------- 2 files changed, 11 insertions(+), 40 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java index cd635e422084..9caa2bcb2e31 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java @@ -89,7 +89,7 @@ public Future queueChunkForUpload( { return uploadExecutor.submit(() -> RetryUtils.retry( () -> { - log.debug("Uploading chunk [%d] for uploadId [%s].", chunkNumber, uploadId); + log.debug("Uploading chunk[%d] for uploadId[%s].", chunkNumber, uploadId); UploadPartResult uploadPartResult = uploadPartIfPossible( s3Client, uploadId, @@ -132,8 +132,7 @@ UploadPartResult uploadPartIfPossible( return s3Client.uploadPart(uploadPartRequest); } - @VisibleForTesting - ExecutorService createExecutorService(int poolSize, int maxNumConcurrentChunks) + private ExecutorService createExecutorService(int poolSize, int maxNumConcurrentChunks) { return Execs.newBlockingThreaded("S3UploadThreadPool-%d", poolSize, maxNumConcurrentChunks); } diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java index b5846fac3952..93b8d7cd2e07 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java @@ -22,21 +22,21 @@ import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.query.DruidProcessingConfigTest; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; import org.apache.druid.utils.RuntimeInfo; import org.easymock.Capture; import org.easymock.EasyMock; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.io.File; -import java.util.concurrent.ExecutorService; +import java.io.IOException; import java.util.concurrent.Future; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; public class S3UploadManagerTest { @@ -44,25 +44,14 @@ public class S3UploadManagerTest private S3UploadManager s3UploadManager; private S3OutputConfig s3OutputConfig; private S3ExportConfig s3ExportConfig; - private static ExecutorService uploadExecutor; @Before public void setUp() { - s3OutputConfig = EasyMock.mock(S3OutputConfig.class); - s3ExportConfig = EasyMock.mock(S3ExportConfig.class); - final RuntimeInfo runtimeInfo = EasyMock.mock(RuntimeInfo.class); - uploadExecutor = Execs.singleThreaded("UploadThreadPool-%d"); - - EasyMock.expect(runtimeInfo.getAvailableProcessors()).andReturn(8).anyTimes(); - EasyMock.expect(s3OutputConfig.getChunkSize()).andReturn(100L * 1024 * 1024).anyTimes(); // 100 MB - EasyMock.expect(s3OutputConfig.getMaxRetry()).andReturn(1).anyTimes(); - EasyMock.expect(s3OutputConfig.getBucket()).andReturn("bucket").anyTimes(); - EasyMock.expect(s3ExportConfig.getChunkSize()).andReturn(HumanReadableBytes.valueOf(200L * 1024 * 1024)).anyTimes(); // 200 MB - - EasyMock.replay(runtimeInfo, s3OutputConfig, s3ExportConfig); - - s3UploadManager = new TestS3UploadManager(s3OutputConfig, s3ExportConfig, runtimeInfo); + s3OutputConfig = new S3OutputConfig("bucket", "prefix", EasyMock.mock(File.class), new HumanReadableBytes("100MiB"), 1); + s3ExportConfig = new S3ExportConfig("tempDir", new HumanReadableBytes("200MiB"), 1, null); + final RuntimeInfo runtimeInfo = new DruidProcessingConfigTest.MockRuntimeInfo(8, 0, 0); + s3UploadManager = new S3UploadManager(s3OutputConfig, s3ExportConfig, runtimeInfo); } @Test @@ -149,26 +138,9 @@ public void testUploadPartIfPossible() assertEquals(uploadPartResult, result); } - @Test - public void testStartAndStop() + @After + public void teardown() throws IOException { - s3UploadManager.start(); s3UploadManager.stop(); - - assertTrue(uploadExecutor.isShutdown()); - } - - private static class TestS3UploadManager extends S3UploadManager - { - public TestS3UploadManager(S3OutputConfig s3OutputConfig, S3ExportConfig s3ExportConfig, RuntimeInfo runtimeInfo) - { - super(s3OutputConfig, s3ExportConfig, runtimeInfo); - } - - @Override - ExecutorService createExecutorService(int poolSize, int maxNumConcurrentChunks) - { - return uploadExecutor; - } } } From f3df9c7185512e1f087481c9dc28c3a72e4a5e15 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 6 Jun 2024 20:23:26 +0530 Subject: [PATCH 32/33] Fix intellij-inspections check --- .../org/apache/druid/storage/s3/output/S3UploadManagerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java index 93b8d7cd2e07..a660c294a4d2 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java @@ -139,7 +139,7 @@ public void testUploadPartIfPossible() } @After - public void teardown() throws IOException + public void teardown() { s3UploadManager.stop(); } From a47400870286d8c02f3912a671aa6e57d15afbe2 Mon Sep 17 00:00:00 2001 From: Akshat Jain Date: Thu, 6 Jun 2024 20:53:39 +0530 Subject: [PATCH 33/33] Fix checkstyle --- .../org/apache/druid/storage/s3/output/S3UploadManagerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java index a660c294a4d2..b79c392844d3 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3UploadManagerTest.java @@ -33,7 +33,6 @@ import org.junit.Test; import java.io.File; -import java.io.IOException; import java.util.concurrent.Future; import static org.junit.Assert.assertEquals;