Konstantin Bereznyakov created HADOOP-19862:
-----------------------------------------------

             Summary: S3A: Thread leak from AWS SDK v2 ScheduledExecutorService
                 Key: HADOOP-19862
                 URL: https://issues.apache.org/jira/browse/HADOOP-19862
             Project: Hadoop Common
          Issue Type: Bug
            Reporter: Konstantin Bereznyakov


AWS SDK v2 S3 clients create internal ScheduledExecutorService instances that 
accumulate over time, causing unbounded thread growth. Thread dumps show 
thousands of sdk-ScheduledExecutor-* threads in processes that create multiple 
S3AFileSystem instances.

  Environment

  - Hadoop 3.4.x with AWS SDK v2
  - Not observed in Hadoop 3.3.x (AWS SDK v1)

  Observed Behavior

  Thread dump comparison:
  Hadoop 3.3.x (AWS SDK v1):  Normal thread count
  Hadoop 3.4.x (AWS SDK v2):  1600+ "sdk-ScheduledExecutor-*" threads

  Thread pattern:
  "sdk-ScheduledExecutor-0-0" daemon prio=5 waiting
  "sdk-ScheduledExecutor-0-1" daemon prio=5 waiting
  ...
  "sdk-ScheduledExecutor-0-4" daemon prio=5 waiting
  "sdk-ScheduledExecutor-1-0" daemon prio=5 waiting
  ...

  Root Cause

  AWS SDK v2's SdkDefaultClientBuilder creates a ScheduledThreadPoolExecutor 
with 5 threads per client when no executor is explicitly provided 
(https://github.com/aws/aws-sdk-java-v2/issues/1690):

  Executors.newScheduledThreadPool(5,
      new 
ThreadFactoryBuilder().threadNamePrefix("sdk-ScheduledExecutor").build())

  These threads are used for retry scheduling, timeout handling, and credential 
refresh.

  Contributing Factors

  1. AbstractFileSystem has no caching

  Unlike FileSystem.get() which uses CACHE.get(uri, conf), 
AbstractFileSystem.get() always creates new instances:

  // AbstractFileSystem.java:263-266
  public static AbstractFileSystem get(final URI uri, final Configuration conf) 
{
      return createFileSystem(uri, conf);  // NO CACHING
  }

  Each FileContext.getFileContext() call with an S3 URI creates:
  - New AbstractFileSystem (S3A)
  - New S3AFileSystem
  - New S3Client
  - 5 new sdk-ScheduledExecutor threads

  2. S3Client threads not released on close

  As documented in https://github.com/aws/aws-sdk-java-v2/issues/1690:
  "When using cached, ephemeral clients, I can see that the scheduled thread 
pool will at times be leaked when the aws client is evicted"

  3. Multiple client types affected

  S3A creates multiple AWS SDK clients:
  - S3Client (sync)
  - S3AsyncClient
  - STS client (for delegation tokens)
  - KMS client (for encryption)

  Each client instance creates its own 5-thread pool.

  Impact

  - Unbounded thread growth in any process using S3A
  - Resource exhaustion leading to OOM or system instability
  - Particularly affects YARN NodeManager, Spark drivers/executors, and other 
services that create many filesystem instances

  Related

  - https://github.com/aws/aws-sdk-java-v2/issues/1690 - SDK issue documenting 
the problem
  - https://github.com/aws/aws-sdk-java-v2/pull/4002 - SDK fix allowing shared 
executor configuration
  - HADOOP-19624 - Similar thread leak in ABFS (AbfsClientThrottlingAnalyzer)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to