[GitHub] [hadoop] steveloughran commented on pull request #2179: HADOOP-17166. ABFS: making max concurrent requests and max requests that can be que…
steveloughran commented on pull request #2179: URL: https://github.com/apache/hadoop/pull/2179#issuecomment-686411091 LGTM, +1 *pending you make clear in the option names and docs that this is experimental* > These configs are tested on prod environments. > The same can remain as a means to control the resource usage. With the internal discussions we had we would like to keep the same this way. We all have this problem, we all want to get a fix in. My point of view is that a shared thread pool with queue managed to stop one single output stream using up all the capacity is that the correct solution. I base this on S3ABlockOutputStream, whose pool class is in hadoop-common. I also understand, why a simple "let's do this right now" fix can address a situation will rapidly before the ABFS streams switch to `BlockingThreadPoolExecutorService`. However, it is precisely because we know it is an interim fix that I want all the options to have 'experimental' in their name. That way, when they get removed, people won't get upset that the options they were using have gone away. I recognise that you are shipping with this fix, and that you have cluster configurations which use them. However, it is long-standing policy in the project which is "the ASF project must not have its decisions determined by the fact that someone has already shipped a feature in their own branch". That's important: we have all shipped fixes early, and then had to deal with catching up with production releases. I believe the HDFS IPC wire format change between Hadoop 2.0.205 (used in CDH) and Hadoop 2.2.0 was the most controversial here as it was actual protocol incompatibility. The situation here is minor in comparision. _Anyone is free to ship a hadoop build with their own changes, but that cannot be used as a veto on changes in the open source codebase itself_ This makes sense, when you think of it. The good news, `Configuration.addDeprecations()` lets you provide an automated way to map from deprecated values (your current set of options) to ones with "experimental" in the name, such as here (S3ATestUtils) ``` private static void addDeprecatedKeys() { Configuration.DeprecationDelta[] deltas = { // STS endpoint configuration option new Configuration.DeprecationDelta( S3ATestConstants.TEST_STS_ENDPOINT, ASSUMED_ROLE_STS_ENDPOINT) }; Configuration.addDeprecations(deltas); Configuration.reloadExistingConfigurations(); } static { addDeprecatedKeys(); } ``` If the old option is used, .the user will see a message logged @ info (unless they have turned off the deprecation log), and the value is remapped to the new one. ``` log4j.logger.org.apache.hadoop.conf.Configuration.deprecation=WARN ``` So, please use experimental in the name, it gives us freedom to come up with a good design for how to do block upload buffering/pooling without having any future design decisions constrained by this intermediate patch. thanks. 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
[GitHub] [hadoop] steveloughran commented on pull request #2179: HADOOP-17166. ABFS: making max concurrent requests and max requests that can be que…
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
[GitHub] [hadoop] steveloughran commented on pull request #2179: HADOOP-17166. ABFS: making max concurrent requests and max requests that can be que…
steveloughran commented on pull request #2179: URL: https://github.com/apache/hadoop/pull/2179#issuecomment-680270262 > We are working on a similar approach. The same needs extensive tests to ensure the perf side is fine. Till the fix is available this can be used to control the memory consumption. This has been verified with a customer as well The s3a queue design is pretty well tested 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
[GitHub] [hadoop] steveloughran commented on pull request #2179: HADOOP-17166. ABFS: making max concurrent requests and max requests that can be que…
steveloughran commented on pull request #2179: URL: https://github.com/apache/hadoop/pull/2179#issuecomment-678634936 Prefer you use the strategy outlined in HADOOP-17195. Rather than pool settings for each output stream, the store has a single shared pool with semaphores to limit #of active entries per output stream. org.apache.hadoop.util.SemaphoredDelegatingExecutor does this for you. This ensures that when there is a low #of output streams, they good upload performance, but in heavy use then it throttles back. 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