rkhachatryan commented on a change in pull request #15322:
URL: https://github.com/apache/flink/pull/15322#discussion_r670292361



##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/FsStateChangelogOptions.java
##########
@@ -57,4 +87,18 @@
                             "Number of threads to use to perform cleanup in 
case an upload is discarded "
                                     + "(and not cleaned up by JM). "
                                     + "If the cleanup doesn't keep up then 
task might be back-pressured.");
+
+    public static final ConfigOption<Integer> NUM_UPLOAD_THREADS =
+            ConfigOptions.key("dstl.dfs.upload.num-threads")
+                    .intType()
+                    .defaultValue(5)
+                    .withDescription("Number of threads to use for upload.");

Review comment:
       I think the assumption that async checkpoint part uploads are **always** 
using the same shared pool is incorrect. For example RocksDB uses separate 
threads to upload, and the IO executor is only used to wait for the result. 
   
   The advantages are
   - better isolation (e.g. heavy network traffic from the upstream does not 
affect snapshot uploading => checkpointing)
   - fine-grained tuning is possible
   - in some scenarios it's easier to debug (i.e. to see where changelog 
uploader is stuck, not why IO thread isn't uploading)
   
   The disadvantages are flip-sides of the latter two: the additional config 
option and in more difficult to debug in other cases .
   
   Additionally, here **scheduled** executor is needed, while current IO 
executor is just `Executor`.




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to