Uploaded image for project: 'Hadoop Common'
  1. Hadoop Common
  2. HADOOP-11446

S3AOutputStream should use shared thread pool to avoid OutOfMemoryError

    Details

    • Type: Bug
    • Status: Closed
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 2.6.0
    • Fix Version/s: 2.7.0
    • Component/s: fs/s3
    • Labels:
      None
    • Target Version/s:
    • Release Note:
      Hide
      The following parameters are introduced in this JIRA:
      fs.s3a.threads.max: the maximum number of threads to allow in the pool used by TransferManager
      fs.s3a.threads.core: the number of threads to keep in the pool used by TransferManager
      fs.s3a.threads.keepalivetime: when the number of threads is greater than the core, this is the maximum time that excess idle threads will wait for new tasks before terminating
      fs.s3a.max.total.tasks: the maximum number of tasks that the LinkedBlockingQueue can hold
      Show
      The following parameters are introduced in this JIRA: fs.s3a.threads.max: the maximum number of threads to allow in the pool used by TransferManager fs.s3a.threads.core: the number of threads to keep in the pool used by TransferManager fs.s3a.threads.keepalivetime: when the number of threads is greater than the core, this is the maximum time that excess idle threads will wait for new tasks before terminating fs.s3a.max.total.tasks: the maximum number of tasks that the LinkedBlockingQueue can hold

      Description

      When working with Terry Padgett who used s3a for hbase snapshot, the following issue was uncovered.
      Here is part of the output including the OOME when hbase snapshot is exported to s3a (nofile ulimit was increased to 102400):

      2014-12-19 13:15:03,895 INFO  [main] s3a.S3AFileSystem: OutputStream for key 'FastQueryPOC/2014-12-11/EVENT1-IDX-snapshot/.hbase-snapshot/.tmp/EVENT1_IDX_snapshot_2012_12_11/    650a5678810fbdaa91809668d11ccf09/.regioninfo' closed. Now beginning upload
      2014-12-19 13:15:03,895 INFO  [main] s3a.S3AFileSystem: Minimum upload part size: 16777216 threshold2147483647
      Exception in thread "main" java.lang.OutOfMemoryError: unable to create new native thread
              at java.lang.Thread.start0(Native Method)
              at java.lang.Thread.start(Thread.java:713)
              at java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:949)
              at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1360)
              at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:132)
              at com.amazonaws.services.s3.transfer.internal.UploadMonitor.<init>(UploadMonitor.java:129)
              at com.amazonaws.services.s3.transfer.TransferManager.upload(TransferManager.java:449)
              at com.amazonaws.services.s3.transfer.TransferManager.upload(TransferManager.java:382)
              at org.apache.hadoop.fs.s3a.S3AOutputStream.close(S3AOutputStream.java:127)
              at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
              at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
              at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:54)
              at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:112)
              at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:366)
              at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:356)
              at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:356)
              at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:338)
              at org.apache.hadoop.hbase.snapshot.ExportSnapshot.run(ExportSnapshot.java:791)
              at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70)
              at org.apache.hadoop.hbase.snapshot.ExportSnapshot.innerMain(ExportSnapshot.java:882)
              at org.apache.hadoop.hbase.snapshot.ExportSnapshot.main(ExportSnapshot.java:886)
      

      In S3AOutputStream#close():

            TransferManager transfers = new TransferManager(client);
      

      This results in each TransferManager creating its own thread pool, leading to the OOME.
      One solution is to pass shared thread pool to TransferManager.

      1. hadoop-11446-001.patch
        7 kB
        Ted Yu
      2. hadoop-11446-002.patch
        9 kB
        Ted Yu
      3. hadoop-11446-003.patch
        10 kB
        Ted Yu
      4. hadoop-11446.addendum
        2 kB
        Ted Yu

        Issue Links

          Activity

          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Hadoop-Mapreduce-trunk #2016 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/2016/)
          HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c)

          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          • hadoop-common-project/hadoop-common/CHANGES.txt
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Hadoop-Mapreduce-trunk #2016 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk/2016/ ) HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java hadoop-common-project/hadoop-common/CHANGES.txt hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Hadoop-Mapreduce-trunk-Java8 #66 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk-Java8/66/)
          HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c)

          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          • hadoop-common-project/hadoop-common/CHANGES.txt
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Hadoop-Mapreduce-trunk-Java8 #66 (See https://builds.apache.org/job/Hadoop-Mapreduce-trunk-Java8/66/ ) HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java hadoop-common-project/hadoop-common/CHANGES.txt
          Hide
          hudson Hudson added a comment -

          SUCCESS: Integrated in Hadoop-Hdfs-trunk-Java8 #62 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk-Java8/62/)
          HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c)

          • hadoop-common-project/hadoop-common/CHANGES.txt
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          Show
          hudson Hudson added a comment - SUCCESS: Integrated in Hadoop-Hdfs-trunk-Java8 #62 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk-Java8/62/ ) HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c) hadoop-common-project/hadoop-common/CHANGES.txt hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Hadoop-Hdfs-trunk #1997 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/1997/)
          HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c)

          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          • hadoop-common-project/hadoop-common/CHANGES.txt
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Hadoop-Hdfs-trunk #1997 (See https://builds.apache.org/job/Hadoop-Hdfs-trunk/1997/ ) HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java hadoop-common-project/hadoop-common/CHANGES.txt hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          Hide
          thodemoor Thomas Demoor added a comment -

          @ Ted Yu:

          • I fear we have overlooked this: copyFile and copyLocalFile still use a method-local TransferManager transfers object (instead of the class member object aka this.transfers). The addendum removes the shutDown() calls to the local transfermanager there but we should entirely abolish the local objects and only use this.transfers.
          • concerning close(): I guess calling close is left to the end user. However, I think we do not leak memory as long as fs.s3a.threads.keepalivetime > 0. Because you set tpe.allowCoreThreadTimeOut(true), the TransferManager will be garbage collected after it goes out of scope AND all (core) threads have timed out. Correct?

          @Steve Loughran: I fear we should. Without the addendum: if the purge code is hit, the next fs command will throw an error as the TransferManager has been shut down. Furthermore, my first remark above hints at an addendum-002

          Show
          thodemoor Thomas Demoor added a comment - @ Ted Yu : I fear we have overlooked this: copyFile and copyLocalFile still use a method-local TransferManager transfers object (instead of the class member object aka this.transfers). The addendum removes the shutDown() calls to the local transfermanager there but we should entirely abolish the local objects and only use this.transfers. concerning close(): I guess calling close is left to the end user. However, I think we do not leak memory as long as fs.s3a.threads.keepalivetime > 0. Because you set tpe.allowCoreThreadTimeOut(true), the TransferManager will be garbage collected after it goes out of scope AND all (core) threads have timed out. Correct? @ Steve Loughran : I fear we should. Without the addendum: if the purge code is hit, the next fs command will throw an error as the TransferManager has been shut down. Furthermore, my first remark above hints at an addendum-002
          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Hadoop-Yarn-trunk-Java8 #65 (See https://builds.apache.org/job/Hadoop-Yarn-trunk-Java8/65/)
          HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c)

          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          • hadoop-common-project/hadoop-common/CHANGES.txt
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Hadoop-Yarn-trunk-Java8 #65 (See https://builds.apache.org/job/Hadoop-Yarn-trunk-Java8/65/ ) HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java hadoop-common-project/hadoop-common/CHANGES.txt
          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Hadoop-Yarn-trunk #799 (See https://builds.apache.org/job/Hadoop-Yarn-trunk/799/)
          HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c)

          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          • hadoop-common-project/hadoop-common/CHANGES.txt
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Hadoop-Yarn-trunk #799 (See https://builds.apache.org/job/Hadoop-Yarn-trunk/799/ ) HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java hadoop-common-project/hadoop-common/CHANGES.txt hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          Hide
          stevel@apache.org Steve Loughran added a comment -

          is this addendum something you want to get in? If so, it's going to have to be another patch I'm afraid

          Show
          stevel@apache.org Steve Loughran added a comment - is this addendum something you want to get in? If so, it's going to have to be another patch I'm afraid
          Hide
          yuzhihong@gmail.com Ted Yu added a comment -

          Relying on FileSystem.close() to be called would not always work - ExportSnapshot from hbase doesn't call close().

          FYI

          Show
          yuzhihong@gmail.com Ted Yu added a comment - Relying on FileSystem.close() to be called would not always work - ExportSnapshot from hbase doesn't call close(). FYI
          Hide
          yuzhihong@gmail.com Ted Yu added a comment -

          Thomas:
          Addendum addresses your comment.

          Show
          yuzhihong@gmail.com Ted Yu added a comment - Thomas: Addendum addresses your comment.
          Hide
          thodemoor Thomas Demoor added a comment -

          Some remarks:

          • I guess that at the end of "purging" the TransferManager should not be shut down?
          • Which got me thinking: Should we override FileSystem.close() in S3AFileSystem to call TransferManager.shutDownNow(true)? This will shut down the thread pool and AmazonS3Client. Otherwise, these resources may still leak?

          Regarding my previous comments: I see no harm in giving users full control, although probably a linear relation between fs.s3a.max.connections and fs.s3a.threads.core (etc.) is suitable (at least that's what I derive from the aws-sdk source code, I haven't tested this yet). However, we should at least document their interplay as f.i. increasing threads while keeping connections fixed will likely not yield the (linear) performance improvement users might expect.

          Show
          thodemoor Thomas Demoor added a comment - Some remarks: I guess that at the end of "purging" the TransferManager should not be shut down? Which got me thinking: Should we override FileSystem.close() in S3AFileSystem to call TransferManager.shutDownNow(true)? This will shut down the thread pool and AmazonS3Client. Otherwise, these resources may still leak? Regarding my previous comments: I see no harm in giving users full control, although probably a linear relation between fs.s3a.max.connections and fs.s3a.threads.core (etc.) is suitable (at least that's what I derive from the aws-sdk source code, I haven't tested this yet). However, we should at least document their interplay as f.i. increasing threads while keeping connections fixed will likely not yield the (linear) performance improvement users might expect.
          Hide
          hudson Hudson added a comment -

          FAILURE: Integrated in Hadoop-trunk-Commit #6805 (See https://builds.apache.org/job/Hadoop-trunk-Commit/6805/)
          HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c)

          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
          • hadoop-common-project/hadoop-common/CHANGES.txt
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java
          • hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          Show
          hudson Hudson added a comment - FAILURE: Integrated in Hadoop-trunk-Commit #6805 (See https://builds.apache.org/job/Hadoop-trunk-Commit/6805/ ) HADOOP-11446 S3AOutputStream should use shared thread pool to avoid OutOfMemoryError (stevel: rev 27d8395867f665fea1360087325cda5ed70efd0c) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java hadoop-common-project/hadoop-common/CHANGES.txt hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
          Hide
          stevel@apache.org Steve Loughran added a comment -

          +1

          Show
          stevel@apache.org Steve Loughran added a comment - +1
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12689537/hadoop-11446-003.patch
          against trunk revision 6621c35.

          +1 @author. The patch does not contain any @author tags.

          -1 tests included. The patch doesn't appear to include any new or modified tests.
          Please justify why no new tests are needed for this patch.
          Also please list what manual steps were performed to verify this patch.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. There were no new javadoc warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 2.0.3) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-tools/hadoop-aws.

          Test results: https://builds.apache.org/job/PreCommit-HADOOP-Build/5352//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HADOOP-Build/5352//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12689537/hadoop-11446-003.patch against trunk revision 6621c35. +1 @author . The patch does not contain any @author tags. -1 tests included . The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . There were no new javadoc warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 2.0.3) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-tools/hadoop-aws. Test results: https://builds.apache.org/job/PreCommit-HADOOP-Build/5352//testReport/ Console output: https://builds.apache.org/job/PreCommit-HADOOP-Build/5352//console This message is automatically generated.
          Hide
          yuzhihong@gmail.com Ted Yu added a comment -

          Patch v3 addresses Steve's comment.

          Once the set of new parameters has been agreed upon, I will update release note.

          Show
          yuzhihong@gmail.com Ted Yu added a comment - Patch v3 addresses Steve's comment. Once the set of new parameters has been agreed upon, I will update release note.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Ted, could you pull "fs.s3a.max.total.tasks" & its default value into the constants class.

          I think we'll need the doc updated with the config options too.

          test-wise, I can't see an easy way to test that this works, except by you repeating your experiment

          Show
          stevel@apache.org Steve Loughran added a comment - Ted, could you pull "fs.s3a.max.total.tasks" & its default value into the constants class. I think we'll need the doc updated with the config options too. test-wise, I can't see an easy way to test that this works, except by you repeating your experiment
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12689269/hadoop-11446-002.patch
          against trunk revision 1454efe.

          +1 @author. The patch does not contain any @author tags.

          -1 tests included. The patch doesn't appear to include any new or modified tests.
          Please justify why no new tests are needed for this patch.
          Also please list what manual steps were performed to verify this patch.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. There were no new javadoc warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 2.0.3) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-tools/hadoop-aws.

          Test results: https://builds.apache.org/job/PreCommit-HADOOP-Build/5351//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HADOOP-Build/5351//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12689269/hadoop-11446-002.patch against trunk revision 1454efe. +1 @author . The patch does not contain any @author tags. -1 tests included . The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . There were no new javadoc warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 2.0.3) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-tools/hadoop-aws. Test results: https://builds.apache.org/job/PreCommit-HADOOP-Build/5351//testReport/ Console output: https://builds.apache.org/job/PreCommit-HADOOP-Build/5351//console This message is automatically generated.
          Hide
          yuzhihong@gmail.com Ted Yu added a comment -

          w.r.t. comment #1 above:

              awsConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS, 
                DEFAULT_MAXIMUM_CONNECTIONS));
          

          The default value is from current setting.
          Do you think setting max connections as value for fs.s3a.threads.core makes sense ?

          w.r.t. last comment, having different values for core.threads and max.threads would get the full potential out of ThreadPoolExecutor.

          Next patch would address remaining comments.

          Show
          yuzhihong@gmail.com Ted Yu added a comment - w.r.t. comment #1 above: awsConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS, DEFAULT_MAXIMUM_CONNECTIONS)); The default value is from current setting. Do you think setting max connections as value for fs.s3a.threads.core makes sense ? w.r.t. last comment, having different values for core.threads and max.threads would get the full potential out of ThreadPoolExecutor. Next patch would address remaining comments.
          Hide
          thodemoor Thomas Demoor added a comment -

          Hi Ted,
          Looks good to me. Some minor remarks:

          • The parameters should be defined (and documented) in Constants.java. Your default of fs.s3a.threads.core=256 means up to 256 parallel (part)Uploads. That should fill up your network pipe . However, the number of concurrent http connections opened by the underlying AmazonS3Client (fs.s3a.max.connections) is set to a much lower value by default (too low?). Could you elaborate on the default values? I think we should tweak these a bit to give a good "out of the box" experience and/or document some tuning guidelines for different network conditions (use cases).
          • Also use the shiny new single TransferManager for purging at the end of initialize() in S3AFileSystem, replacing the following code path
            if (purgeExistingMultipart) {
                  TransferManager transferManager = new TransferManager(s3);
            
          • I like that you went for a low-level implementation for the Executor instead of using Executors.newFixedThreadPool. The ability to block submitting threads by setting fs.s3a.max.total.tasks is nice tool for limiting memory consumption. Out of curiosiity: can you envision use cases where setting different values for core.threads and max.threads would be important?
          Show
          thodemoor Thomas Demoor added a comment - Hi Ted, Looks good to me. Some minor remarks: The parameters should be defined (and documented) in Constants.java. Your default of fs.s3a.threads.core=256 means up to 256 parallel (part)Uploads. That should fill up your network pipe . However, the number of concurrent http connections opened by the underlying AmazonS3Client (fs.s3a.max.connections) is set to a much lower value by default (too low?). Could you elaborate on the default values? I think we should tweak these a bit to give a good "out of the box" experience and/or document some tuning guidelines for different network conditions (use cases). Also use the shiny new single TransferManager for purging at the end of initialize() in S3AFileSystem, replacing the following code path if (purgeExistingMultipart) { TransferManager transferManager = new TransferManager(s3); I like that you went for a low-level implementation for the Executor instead of using Executors.newFixedThreadPool. The ability to block submitting threads by setting fs.s3a.max.total.tasks is nice tool for limiting memory consumption. Out of curiosiity: can you envision use cases where setting different values for core.threads and max.threads would be important?
          Hide
          hadoopqa Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12689078/hadoop-11446-001.patch
          against trunk revision 4f18018.

          +1 @author. The patch does not contain any @author tags.

          -1 tests included. The patch doesn't appear to include any new or modified tests.
          Please justify why no new tests are needed for this patch.
          Also please list what manual steps were performed to verify this patch.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. There were no new javadoc warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 2.0.3) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-tools/hadoop-aws.

          Test results: https://builds.apache.org/job/PreCommit-HADOOP-Build/5341//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HADOOP-Build/5341//console

          This message is automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12689078/hadoop-11446-001.patch against trunk revision 4f18018. +1 @author . The patch does not contain any @author tags. -1 tests included . The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . There were no new javadoc warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 2.0.3) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-tools/hadoop-aws. Test results: https://builds.apache.org/job/PreCommit-HADOOP-Build/5341//testReport/ Console output: https://builds.apache.org/job/PreCommit-HADOOP-Build/5341//console This message is automatically generated.
          Hide
          yuzhihong@gmail.com Ted Yu added a comment -

          Thanks for the comment, Thomas.
          AmazonS3Client is used to construct TransferManager. I replaced it with TransferManager in S3AOutputStream ctor.

          Introduced the following parameters which control threadpool used by TransferManager:
          fs.s3a.threads.max
          fs.s3a.threads.core
          fs.s3a.max.total.tasks
          fs.s3a.threads.keepalivetime

          Show
          yuzhihong@gmail.com Ted Yu added a comment - Thanks for the comment, Thomas. AmazonS3Client is used to construct TransferManager. I replaced it with TransferManager in S3AOutputStream ctor. Introduced the following parameters which control threadpool used by TransferManager: fs.s3a.threads.max fs.s3a.threads.core fs.s3a.max.total.tasks fs.s3a.threads.keepalivetime
          Hide
          thodemoor Thomas Demoor added a comment -

          Maybe we can take it even further and share a single TransferManager (as advised by AWS) with a custom threadpool of tunable size (default is FixedThreadPool(10), which is probably too low). Thus, construct a final TransferManager in S3AFileSystem.initialize() and pass it around (to S3AOutputStream)?

          Show
          thodemoor Thomas Demoor added a comment - Maybe we can take it even further and share a single TransferManager (as advised by AWS) with a custom threadpool of tunable size (default is FixedThreadPool(10), which is probably too low). Thus, construct a final TransferManager in S3AFileSystem.initialize() and pass it around (to S3AOutputStream)?

            People

            • Assignee:
              yuzhihong@gmail.com Ted Yu
              Reporter:
              yuzhihong@gmail.com Ted Yu
            • Votes:
              0 Vote for this issue
              Watchers:
              10 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development