xiaoxuandev commented on issue #15898:
URL: https://github.com/apache/iceberg/issues/15898#issuecomment-4328230200

   @boaz-gold Thanks for sharing the additional diagnostics, the heap histogram 
and full GC results are helpful and confirm the root cause.
   
   ### Reproduction
   
   I am able to reproduce the thread leak on EMR 7.12 using Spark Thrift Server 
with cache.expiration-interval-ms=10 and io-impl=S3FileIO. We observed 
sdk-ScheduledExecutor threads growing monotonically (947 → 1067 in ~5 minutes), 
matching your production pattern.
   
   ### Root Cause
   
   Your analysis is correct. The reference chain is:
   
   sdk-ScheduledExecutor thread (GC root)
     → ScheduledExecutorService
       → S3Client internals
         → S3FileIO
   Each time CachingCatalog evicts a table entry (via TTL or max-total-bytes 
pressure) and GlueCatalog reloads it, newTableOps() creates a new S3FileIO → 
new S3Client → new ScheduledExecutorService with its own sdk-ScheduledExecutor 
threads. The old S3Client is never closed, and since its threads are live (GC 
roots), the entire chain is strongly reachable. Full GC cannot help — these are 
not unreachable objects awaiting collection, they are anchored by their own 
threads.
   This issue manifests specifically in long-running JVM processes (Thrift 
Server, Structured Streaming, long-lived notebook sessions) where threads 
accumulate over hours/days. Short-lived spark-submit jobs are unaffected 
because the JVM exits before threads accumulate to problematic levels.
   
   ### Fix
   
   Rather than closing FileIO on eviction (which could break long-running 
queries still using the evicted table's S3Client as 
   @anoopj pointed out), we share a single ScheduledExecutorService (pool size 
5, matching the AWS SDK v2 default) across all S3Clients created by Iceberg's 
AWS client factories. This is done via 
ClientOverrideConfiguration.scheduledExecutorService().
   
   With this fix, no matter how many S3Clients are created or evicted, the 
total sdk-ScheduledExecutor thread count stays fixed at 5. We verified on EMR 
7.12 Thrift Server: after the fix, sdk-ScheduledExecutor count is 0 and 
iceberg-shared-sdk-executor count is stable at 5 (previously 718 and keep 
growing). 
   
   Please take a look at the fix here: 
https://github.com/apache/iceberg/pull/16129


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to