LakshSingla commented on code in PR #16481: URL: https://github.com/apache/druid/pull/16481#discussion_r1616590921
########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.guice.LazySingleton; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.logger.Logger; + +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. + */ +@LazySingleton Review Comment: I don't think it's required here. ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java: ########## @@ -166,13 +184,25 @@ 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."); Review Comment: ```suggestion LOG.debug("Waiting for lock for writing further chunks to local disk."); ``` ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java: ########## @@ -182,4 +187,20 @@ public Supplier<ServerSideEncryptingAmazonS3> getAmazonS3ClientSupplier( { return Suppliers.memoize(serverSideEncryptingAmazonS3Builder::build); } + + @Provides + @LazySingleton + @Named("S3UploadThreadPool") + public ExecutorService getUploadPoolExecutorService(ScheduledExecutorFactory scheduledExecutorFactory, RuntimeInfo runtimeInfo) + { + int poolSize = Math.max(4, 3 * runtimeInfo.getAvailableProcessors()); Review Comment: What's the rationale behind this? It seems super weird that we are allocating more threads to uploading than the number of processors. Check the basic cluster tuning section: https://druid.apache.org/docs/latest/operations/basic-cluster-tuning/#processing-threads-and-buffers. The number of processing threads is limited to the number of processors to avoid over utilisation. In any case, this value should be extracted somewhere and a Javadoc should be present behind this logic. ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java: ########## @@ -199,15 +229,44 @@ private void pushCurrentChunk() throws IOException { currentChunk.close(); final Chunk chunk = currentChunk; - try { - if (chunk.length() > 0) { - resultsSize += chunk.length(); + if (chunk.length() > 0) { + s3UploadConfig.incrementCurrentNumChunks(); Review Comment: It really doesn't sit well with me that we are able to modify a "config" class. Perhaps the nomenclature is off, however configs aren't supposed to be modified during the execution. They can be dynamically updated, however I'd still expect it to be stateless. ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageDruidModule.java: ########## @@ -182,4 +187,20 @@ public Supplier<ServerSideEncryptingAmazonS3> getAmazonS3ClientSupplier( { return Suppliers.memoize(serverSideEncryptingAmazonS3Builder::build); } + + @Provides + @LazySingleton + @Named("S3UploadThreadPool") Review Comment: "S3UploadThreadPool" should be extracted out somewhere. ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.guice.LazySingleton; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.logger.Logger; + +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. + */ +@LazySingleton +public class S3UploadConfig +{ + /** + * 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; Review Comment: Same comment as above. Also, why is it not an atomic integer? ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/RetryableS3OutputStream.java: ########## @@ -271,50 +328,72 @@ 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() + { + synchronized (fileLock) { + while (pendingFiles.get() > 0) { + try { + LOG.info("Waiting for lock for completing multipart task for uploadId [%s].", uploadId); Review Comment: Would this not pollute the logs, given that we can have one log line per upload? ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.guice.LazySingleton; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.logger.Logger; + +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. + */ +@LazySingleton +public class S3UploadConfig +{ + /** + * 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; + + private static final Logger log = new Logger(S3UploadConfig.class); + + /** + * Increments the counter for the current number of chunks saved on disk. + */ + public void incrementCurrentNumChunks() + { + currentNumChunks.incrementAndGet(); + } + + /** + * Decrements the counter for the current number of chunks saved on disk. + */ + public void decrementCurrentNumChunks() + { + currentNumChunks.decrementAndGet(); + } + + /** + * Gets the current number of chunks saved to local disk. + * + * @return the current number of chunks saved to local disk. + */ + public int getCurrentNumChunks() + { + return currentNumChunks.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 void updateChunkSizeIfGreater(long chunkSize) + { + if (this.chunkSize < chunkSize) { + this.chunkSize = chunkSize; + recomputeMaxConcurrentNumChunks(); + } + } + + /** + * 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() + { + maxConcurrentNumChunks = (int) (S3OutputConfig.S3_MULTIPART_UPLOAD_MAX_PART_SIZE_BYTES / chunkSize); + log.info("Recomputed maxConcurrentNumChunks: %d", maxConcurrentNumChunks); Review Comment: How often do we expect this to be called? ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.guice.LazySingleton; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.logger.Logger; + +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. + */ +@LazySingleton +public class S3UploadConfig +{ + /** + * 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); Review Comment: Why are we tracking something in a config? This doesn't seem very cool for a singleton. I would have expected it to be stateless. If this needs to be updated dynamically, it should mention that somewhere, `DynamicS3UploadConfig`, and a global instance of this should be avoided. All the callers that need to share this must receive it explicitly instead of being injected via Guice. This seems error-prone if someone decides to use the storage connector down the line for some other task and they both share the same config. ########## extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3UploadConfig.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.guice.LazySingleton; +import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.logger.Logger; + +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. + */ +@LazySingleton +public class S3UploadConfig +{ + /** + * 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; Review Comment: The multithreading model for this class is unclear to me. It isn't thread-safe, however, `currentNumChunks` is marked as an atomic integer. -- 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]
