boaz-gold commented on issue #15898:
URL: https://github.com/apache/iceberg/issues/15898#issuecomment-4302745801

   Hi @xiaoxuandev 
   
   We're using GlueCatalog wrapped by CachingCatalog.                           
                                                                                
                    
                                                       
   On FileIOTracker: it closes FileIO instances when catalog.close() is called. 
In a long-running Spark Thrift Server, catalog.close() is never invoked — the 
process runs for days until it crashes. So FileIOTracker accumulates every 
S3Client opened since startup without releasing any of them, regardless of what 
the TTL cache evicts.                       
      
   On removing softValues(): the assumption is that a GC-eligible Table will 
pull the S3Client with it. It won't. Each S3Client owns a 
ScheduledExecutorService, and live threads are GC roots — they pin the 
executor, which pins the client. The GC cannot collect them. This is exactly 
why the crash is os::commit_memory (errno=12) (thread stack exhaustion) and not 
a heap OOM.                                                                     
                                                                                
         
                     
   The only thing that actually closes the S3Client is an explicit 
fileIO.close() call. 
   The leak is that nothing calls it on cache eviction.  


-- 
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