kfaraz commented on code in PR #16481: URL: https://github.com/apache/druid/pull/16481#discussion_r1618124099
########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java: ########## @@ -0,0 +1,133 @@ +/* + * 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.HumanReadableBytes; +import org.apache.druid.java.util.common.logger.Logger; + +import java.util.concurrent.ExecutorService; +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. + */ +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(); + + /** + * An atomic counter to track the current number of chunks saved to local disk. + */ + private final AtomicInteger currentNumChunks = 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; + + /** + * Threadpool used for uploading the chunks asynchronously. + */ Review Comment: Not needed. ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java: ########## @@ -0,0 +1,133 @@ +/* + * 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.HumanReadableBytes; +import org.apache.druid.java.util.common.logger.Logger; + +import java.util.concurrent.ExecutorService; +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. + */ +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(); + + /** + * An atomic counter to track the current number of chunks saved to local disk. + */ Review Comment: Not really needed, just rename the field `currentNumChunksOnDisk`. ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadManager.java: ########## @@ -0,0 +1,133 @@ +/* + * 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.HumanReadableBytes; +import org.apache.druid.java.util.common.logger.Logger; + +import java.util.concurrent.ExecutorService; +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. Review Comment: ```suggestion * It tracks the number of chunks currently present on local disk and ensures that * it does not exceed a specified limit. ``` ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java: ########## @@ -182,4 +186,16 @@ public Supplier<ServerSideEncryptingAmazonS3> 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); + } Review Comment: Instead of this, I guess it would be better to just have an `@Inject` constructor for `S3UploadManager`. That way, the executor service would be created inside the `S3UploadManager`, giving it complete control over its lifecycle. ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java: ########## @@ -103,27 +104,38 @@ public class RetryableS3OutputStream extends OutputStream private boolean error; private boolean closed; - public RetryableS3OutputStream( - S3OutputConfig config, - ServerSideEncryptingAmazonS3 s3, - String s3Key - ) throws IOException - { + /** + * An atomic counter to store number of files pending to be uploaded for the particular uploadId. + */ + private final AtomicInteger pendingFiles = new AtomicInteger(0); - this(config, s3, s3Key, true); - } + /** + * 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; - @VisibleForTesting - protected RetryableS3OutputStream( + /** + * A lock to restrict the maximum number of chunks on disk at any given point in time. + */ + private final Object maxChunksLock = new Object(); Review Comment: Shouldn't this restriction be enforced by `S3UploadManager` instead? ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java: ########## @@ -199,15 +221,44 @@ private void pushCurrentChunk() throws IOException { currentChunk.close(); final Chunk chunk = currentChunk; - try { - if (chunk.length() > 0) { - resultsSize += chunk.length(); + if (chunk.length() > 0) { + uploadManager.incrementCurrentNumChunks(); + pendingFiles.incrementAndGet(); - pushStopwatch.start(); - pushResults.add(push(chunk)); - pushStopwatch.stop(); - numChunksPushed++; + uploadManager.submitTask(() -> { + try { + uploadChunk(chunk); + } + catch (Exception e) { + error = true; + LOG.error(e, e.getMessage()); + throw new RuntimeException(e); + } + finally { + synchronized (maxChunksLock) { + uploadManager.decrementCurrentNumChunks(); + maxChunksLock.notifyAll(); + } + if (pendingFiles.decrementAndGet() == 0) { + synchronized (fileLock) { + fileLock.notifyAll(); + } + } + } + }); + } + } Review Comment: All the upload related logic should live in `S3UploadManager` itself, otherwise it is just an `ExecutorService`. Let me know if you need any assistance with this. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
