Skip to content

AWS: Fix sdk-ScheduledExecutor thread leak by sharing a single executor across S3 clients#16129

Open
xiaoxuandev wants to merge 1 commit intoapache:mainfrom
xiaoxuandev:fix-sdk-ScheduledExecutor-thread-leak
Open

AWS: Fix sdk-ScheduledExecutor thread leak by sharing a single executor across S3 clients#16129
xiaoxuandev wants to merge 1 commit intoapache:mainfrom
xiaoxuandev:fix-sdk-ScheduledExecutor-thread-leak

Conversation

@xiaoxuandev
Copy link
Copy Markdown
Contributor

@xiaoxuandev xiaoxuandev commented Apr 27, 2026

Summary

When CachingCatalog evicts a table entry and GlueCatalog creates a new S3Client on reload, the old S3Client's sdk-ScheduledExecutor threads are never shut down (threads are GC roots and cannot be reclaimed). In long-running applications like Spark Thrift Server, this causes unbounded thread accumulation leading to native OOM crash.

Root Cause

When ScheduledThreadPoolExecutor creates its core threads, they inherit the parent thread's inheritableThreadLocals. In Spark Thrift Server, the parent thread holds SparkSessionSessionStateCatalogManagerSparkCatalogFileIOTracker → Caffeine cache → S3FileIO. Every sdk-ScheduledExecutor thread copies this chain, and since live threads are GC roots, the entire object graph is rooted — not just the threads, but also the S3FileIO instances, S3Clients, and everything they reference.

Fix

Share a single ScheduledExecutorService (pool size 5, matching AWS SDK v2 default) across all S3Clients created by Iceberg's AWS client factories. The shared executor creates its threads only once at startup. S3FileIO instances are then held only via the Caffeine cache's normal entries — when evicted from the cache, they become unreachable and are collected normally.

This eliminates both the thread leak and the object leak by preventing the creation of new GC-rooting threads that inherit ThreadLocals.

Closes #15898

Changes

  • Added applySharedScheduledExecutor method to S3FileIOProperties that injects a shared executor into the S3 client builder
  • Applied the method in all sync S3 client factory paths (AwsClientFactories, AssumeRoleAwsClientFactory, LakeFormationAwsClientFactory, DefaultS3FileIOAwsClientFactory)
  • Wrapped the shared executor with NonClosingScheduledExecutorService to prevent the AWS SDK from shutting it down on S3Client.close() (the SDK's AttributeMap.closeIfPossible() calls shutdown() on any ExecutorService)
  • Added configuration properties:
    • s3.shared-scheduled-executor.enabled (default: true) — opt-out switch

Testing

  • manual testing with Spark Thrift Server
  • unit tests

Known Limitations

  • Async S3 client path (S3AsyncClient.builder()) is not covered. The default path uses CRT which has its own thread model and is unaffected. The standard async builder path (when s3.crt.enabled=false) has the same leak but is deferred to a follow-up.

Copy link
Copy Markdown
Contributor

@anoopj anoopj left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this fixes the thread exhaustion problem in #15898 (thanks for working on this fix) by sharing the S3 executors.

But don't we still have a problem with S3 clients getting leaked? That would manifest as Java objects requiring heap space, open TCP connections etc?

So I worry that we are fixing the symptom and not the root cause of the leak.

@xiaoxuandev
Copy link
Copy Markdown
Contributor Author

@anoopj Thanks for the review! Good question.

This also addresses the S3Client object leak as well, not just the thread exhaustion.

Why S3Clients were leaking before: Each S3Client created its own ScheduledThreadPoolExecutor whose core threads block indefinitely on DelayedWorkQueue.take(). These threads are GC roots, and they hold a reference back to the executor via the worker's run loop. This anchors the executor in memory, and since the S3Client is reachable from the executor's task infrastructure, the entire chain (thread → executor → S3Client → S3FileIO) stays strongly reachable. GC cannot collect any of it.

Why the shared executor fixes this: With a shared executor, the S3Client no longer owns any threads. After eviction, there are no GC roots holding the S3Client, it becomes unreachable and GC collects it normally.

Verification on our test cluster (Thrift Server, cache.expiration-interval-ms=10, continuous queries):

Repeated jmap -histo over time shows S3FileIO instance count oscillating between 20-26, never accumulating.

@anoopj
Copy link
Copy Markdown
Contributor

anoopj commented Apr 27, 2026

Thank you for clarifying. But why does AWS SDK's S3Client maintain a reference to Iceberg's S3FileIO? The relationship is actually inverted - is there a callback or something from S3 client to S3 file I/O for some reason?

@xiaoxuandev
Copy link
Copy Markdown
Contributor Author

Good catch on the inverted relationship question, you're right that S3Client doesn't directly reference S3FileIO. I did a heap dump analysis and found the actual root cause is inheritableThreadLocals.

When ScheduledThreadPoolExecutor creates its core threads, they inherit the parent thread's inheritableThreadLocals.

In Spark Thrift Server, the parent thread holds
SparkSession → SessionState → CatalogManager → SparkCatalog → FileIOTracker → Caffeine cache → S3FileIO. Every sdk-ScheduledExecutor thread copies this chain, and since live threads are GC roots, the entire
object graph is rooted — not just the threads, but also the S3FileIO instances, S3Clients, and everything they reference.

Without the fix, each S3Client construction creates a new ScheduledThreadPoolExecutor with 5 core threads → 5 new GC roots each holding a copy of the inherited ThreadLocal chain. This is why both threads and Java objects (S3FileIO, DefaultS3Client, etc.) accumulate without bound.
gc-root-path

With the shared executor fix, the shared executor creates its threads only once at startup. The only remaining GC root through this path is response-input-stream-timeout-scheduler (a single global SDK thread). S3FileIO instances are held only via the Caffeine cache's normal entries — when evicted from the cache, they become unreachable and are collected normally.
gc-root-path_fix
This is confirmed by the heap histogram data, the shared executor fix eliminates both the thread leak and the object leak by preventing the creation of new GC-rooting threads that inherit ThreadLocals.

@anoopj
Copy link
Copy Markdown
Contributor

anoopj commented Apr 27, 2026

Thank you for the explanation @xiaoxuandev. That makes sense. Basically the back reference is coming from Spark's use of InheritableThreadLocal.

I also did a quick cross-check on ADLS and GCS to see if we have a problem: neither creates per-instance thread pools (have static singletons), so seems low risk.

Copy link
Copy Markdown
Contributor

@anoopj anoopj left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will there be a separate PR for Async Client?

public static final boolean S3_SHARED_SCHEDULED_EXECUTOR_ENABLED_DEFAULT = true;

/**
* Pool size for the shared ScheduledExecutorService. Matches the AWS SDK v2 default pool size.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Perhaps clarify that this is only used for book keeping and doesn't do data transfer.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

updated, thanks!

}

@Override
public List<Runnable> shutdownNow() {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Perhaps add a test for shutdown being a no-op?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added, thanks!

Comment on lines +742 to +750
this.isSharedScheduledExecutorEnabled =
PropertyUtil.propertyAsBoolean(
properties,
S3_SHARED_SCHEDULED_EXECUTOR_ENABLED,
S3_SHARED_SCHEDULED_EXECUTOR_ENABLED_DEFAULT);
this.sharedScheduledExecutorPoolSize =
PropertyUtil.propertyAsInt(
properties,
S3_SHARED_SCHEDULED_EXECUTOR_POOL_SIZE,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Two catalogs can configure these with separate values, right? But we have one JVM-wide singleton property. That means that the tunable for the one of the catalog will be silently ignored. Is that acceptable?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point. Since this is a JVM-wide singleton, a per-catalog pool size tunable doesn't make sense. I'll remove the s3.shared-scheduled-executor.pool-size property and hard-code the pool size to 5 (the AWS SDK v2 default). The s3.shared-scheduled-executor.enabled toggle is still useful as an escape hatch.

@xiaoxuandev xiaoxuandev force-pushed the fix-sdk-ScheduledExecutor-thread-leak branch from f7f50ad to e6d956a Compare April 28, 2026 19:42
@xiaoxuandev
Copy link
Copy Markdown
Contributor Author

Will there be a separate PR for Async Client?

@anoopj Yes, I'll address the async client path in a follow-up PR to keep this one focused and reviewable.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

CachingCatalog does not close FileIO on cache eviction, causing S3FileIO / SDK v2 thread leak in long-running applications

2 participants