steveloughran commented on pull request #2179: URL: https://github.com/apache/hadoop/pull/2179#issuecomment-680982644
Here is my current view of this patch * as a quick workaround for scale-up issues in the existing code -it works * as the long term design for something which delivers great performance when scaled up as well as down -it's not suitable * the code you need is already there If you look at the S3A code, we have a single shared thread pool, which is based on another ASF project: https://github.com/apache/incubator-retired-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java ```java private ListeningExecutorService boundedThreadPool; // and in initialize() int totalTasks = intOption(conf, MAX_TOTAL_TASKS, DEFAULT_MAX_TOTAL_TASKS, 1); long keepAliveTime = longOption(conf, KEEPALIVE_TIME, DEFAULT_KEEPALIVE_TIME, 0); boundedThreadPool = BlockingThreadPoolExecutorService.newInstance( maxThreads, maxThreads + totalTasks, keepAliveTime, TimeUnit.SECONDS, "s3a-transfer-shared"); // default value is 4 blockOutputActiveBlocks = intOption(conf, FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1); ``` When we create the output stream, we create a new output stream which, although it uses the thread pool, limits the #of blocks each worker can queue for upload. ```java ... new S3ABlockOutputStream(this, destKey, new SemaphoredDelegatingExecutor(boundedThreadPool, blockOutputActiveBlocks, true), progress, partSize, blockFactory, statisticsContext.newOutputStreamStatistics(), getWriteOperationHelper(), putTracker), ``` This gives us * low latency for output stream launch (shared pool) * observability into total pool size/load (could make it a gauge...) * semaphore to stop a single worker from overloading the system * A thread pool to use for other operations. Weaknesses * I think we overuse the blocking thread pool in other places (openFile(),..)& should do a review to make sure we are using the unbounded one more. I managed to create a deadlock once. * If you set the active block option "fs.s3a.fast.upload.active.blocks" too big then you can still OOM if you buffer in RAM. Default S3A block buffer is on disk to avoid this problem. Now people only complain about us not cleaning up temp disk space when spark workers are killed * You still need to decide that blocking thread pool size (default = 32). I think the S3A code 1. should lift that * cores option and say "if the pool size < 0 then we use it as a multiplier for cores"", so -4 would mean "4 * cores". Maybe too late to do this in a backwards compatible way though now, unless I add a new option, deprecate the old one, etc. 1. Add pool size as a gauge. If we could also (somehow) include #of pending blocks across all semaphored executors then you'd get a view of how many threads were being held up. 1. And we could add "time to wait for a thread" as a metric too. Actually, I like that...if we can tie to a stream we can tie to a task, and hence to a job. Tricky though. Let me see what I can do there with the IOStatistics PR in front of me. for point #3: just done it for IOStatistics: https://github.com/apache/hadoop/blob/f5efa4b27536a9e266d9dc06cd3a1e11ded3bfd3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java If you pass in a duration factory then the executor will measure the time to acquire a thread before the actual execution. This is nice to know when trying to answer the "why so slow?" question -as it will either show a problem or show where not to look. of course -doesn't just mean 'not enough executors' as 'why are all the submitted operations taking so long?' could be a sign of network problems. To summarise then - just go for the same thread pool in Store and semaphored wrapper around this passed in to each output stream - choose some pool size, with the idea of a negative value == per core making it easy to do a scalable config - plan for wiring up statistics collection in the near future. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org