[ https://issues.apache.org/jira/browse/HADOOP-13826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744762#comment-15744762 ]
Steve Loughran commented on HADOOP-13826: ----------------------------------------- Ok, I've sat down for a review. Like I said before, the test needs some work. BTW, {{S3AFastOutputStream}} is gone. Are you referring to {{S3ABlockOutputStream}}? h2. {[S3AFileSystem]} Aren't there limits on the size of the AWS httpclient pool, or is there something I'm missing? h2. {[ITestConcurrentOps]} This must be a subclass of {{AbstractS3ATestBase}}, so only run when {{-Dscale}} is set. This will automatically give it extended test timeout, and allow that timeout to be overridden in the config or on the maven command line. * Must use {{path(subpath)}} to get a unique path which works in parallel execution, rather than {{new Path("/ITestS3AManyFiles");}} * {[getRestrictedFileSystem]} can just go "5M" and "10M" when setting sizes * {{teardown()}} must check auxFS for being null, just in case something went wrong in setup. * Note you can use {{ContractTestUtils.dataset()}} to create a 1MB file; it writes in a range of values so it's easier to detect problems on reads. Not really needed here, but you should get into the habit of using those methods where possible. * {{testParallelRename}} must give the threads useful names. * {{testParallelRename}} should use Callables, so any exceptions raised in threads can be raised by test runner. We don't want tests to go wrong and us not to notice. * If {{testParallelRename}} logs exceptions, it must use {{LOG.error()}} * {{testParallelRename}} must check that the dest files exist, and that the source ones don't. Otherwise, it's not actually verifying that the rename worked, only that a parallel series of operations completed. > S3A Deadlock in multipart copy due to thread pool limits. > --------------------------------------------------------- > > Key: HADOOP-13826 > URL: https://issues.apache.org/jira/browse/HADOOP-13826 > Project: Hadoop Common > Issue Type: Bug > Components: fs/s3 > Affects Versions: 2.7.3 > Reporter: Sean Mackrory > Assignee: Sean Mackrory > Priority: Critical > Attachments: HADOOP-13826.001.patch, HADOOP-13826.002.patch, > HADOOP-13826.003.patch > > > In testing HIVE-15093 we have encountered deadlocks in the s3a connector. The > TransferManager javadocs > (http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/s3/transfer/TransferManager.html) > explain how this is possible: > {quote}It is not recommended to use a single threaded executor or a thread > pool with a bounded work queue as control tasks may submit subtasks that > can't complete until all sub tasks complete. Using an incorrectly configured > thread pool may cause a deadlock (I.E. the work queue is filled with control > tasks that can't finish until subtasks complete but subtasks can't execute > because the queue is filled).{quote} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org