[ https://issues.apache.org/jira/browse/HADOOP-11684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14954431#comment-14954431 ]
Aaron Fabbri commented on HADOOP-11684: --------------------------------------- +1 on the 002 patch. I'm attaching a 004 patch that is identical, except it adds an extra unit test on {{BlockingThreadPoolExecutorService}}. It is optional, but demonstrates some testing I've done. These blobstore tests don't run by default, but my test does depend on timing, so it could theoretically fail on a heavily loaded system. (I detect whether or not the threadpool blocks via elapsed system time across {{submit()}} calls, and these aren't realtime systems.} Reviewers: The idea behind the 002 approach pretty simple. The meat of the change is just a semaphore around {{ExecutorService}} functions {{submit()}} etc.. e.g. {code} + @Override + public ListenableFuture<?> submit(Runnable task) { + try { + queueingPermits.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return Futures.immediateFailedCheckedFuture(e); + } + return super.submit(new RunnableWithPermitRelease(task)); + } {code} > S3a to use thread pool that blocks clients > ------------------------------------------ > > Key: HADOOP-11684 > URL: https://issues.apache.org/jira/browse/HADOOP-11684 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 > Affects Versions: 2.7.0 > Reporter: Thomas Demoor > Assignee: Thomas Demoor > Attachments: HADOOP-11684-001.patch, HADOOP-11684-002.patch, > HADOOP-11684-003.patch, HADOOP-11684-004.patch > > > Currently, if fs.s3a.max.total.tasks are queued and another (part)upload > wants to start, a RejectedExecutionException is thrown. > We should use a threadpool that blocks clients, nicely throtthling them, > rather than throwing an exception. F.i. something similar to > https://github.com/apache/incubator-s4/blob/master/subprojects/s4-comm/src/main/java/org/apache/s4/comm/staging/BlockingThreadPoolExecutorService.java -- This message was sent by Atlassian JIRA (v6.3.4#6332)