[ https://issues.apache.org/jira/browse/HADOOP-19624?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18011707#comment-18011707 ]
ASF GitHub Bot commented on HADOOP-19624: ----------------------------------------- mattkduran opened a new pull request, #7852: URL: https://github.com/apache/hadoop/pull/7852 <!-- Thanks for sending a pull request! 1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute 2. Make sure your PR title starts with JIRA issue id, e.g., 'HADOOP-17799. Your PR title ...'. --> ### Description of PR The ABFS driver's auto-throttling feature (`fs.azure.enable.autothrottling=true`) creates Timer threads in AbfsClientThrottlingAnalyzer that are never properly cleaned up, leading to a memory leak that eventually causes OutOfMemoryError in long-running applications like Hive Metastore. #### Impact: - Thread count grows indefinitely (observed >100,000 timer threads) - Affects any long-running service that creates multiple ABFS filesystem instances #### Root Cause: AbfsClientThrottlingAnalyzer creates Timer objects in its constructor but provides no mechanism to cancel them. When AbfsClient instances are closed, the associated timer threads continue running indefinitely. #### Solution Implement proper resource cleanup by making the throttling components implement Closeable and ensuring timers are cancelled when ABFS clients are closed. #### Changes Made 1. AbfsClientThrottlingAnalyzer.java - Added: implements Closeable - Added: close() method that calls timer.cancel() and timer.purge() - Purpose: Ensures timer threads are properly terminated when analyzer is no longer needed 2. AbfsThrottlingIntercept.java (Interface) - Added: extends Closeable - Added: close() method signature - Purpose: Establishes cleanup contract for all throttling intercept implementations 3. AbfsClientThrottlingIntercept.java - Added: close() method that closes both readThrottler and writeThrottler - Purpose: Coordinates cleanup of both read and write throttling analyzers 4. AbfsNoOpThrottlingIntercept.java - Added: No-op close() method - Purpose: Satisfies interface contract for no-op implementation 5. AbfsClient.java - Added: IOUtils.cleanupWithLogger(LOG, intercept) in existing close() method - Purpose: Integrates throttling cleanup into existing client resource management https://github.com/mattkduran/ABFSleaktest https://www.mail-archive.com/common-dev@hadoop.apache.org/msg43483.html ### How was this patch tested? #### Standalone Validation Tool This fix was validated using a standalone reproduction and testing tool that directly exercises the ABFS auto-throttling components outside of a full Hadoop deployment. Repository: [ABFSLeakTest](https://github.com/mattkduran/ABFSleaktest) #### Testing Scope - Problem reproduction confirmed - demonstrates the timer thread leak - Fix validation confirmed - proves close() method resolves the leak - Resource cleanup verified - shows proper timer cancellation - Limited integration testing - standalone tool, not full Hadoop test suite #### Test Results Leak Reproduction Evidence ``` # Without fix: Timer threads accumulate over filesystem creation cycles Cycle Total Threads ABFS Timer Threads Status 1 50->52 0->2 LEAK DETECTED 50 150->152 98->100 LEAK GROWING 200 250->252 398->400 LEAK CONFIRMED Final Analysis: 400 leaked timer threads named "abfs-timer-client-throttling-analyzer-*" Memory Impact: ~90MB additional heap usage # Direct analyzer testing: 🔴 Without close(): +3 timer threads (LEAKED) ✅ With close(): +0 timer threads (NO LEAK) ``` #### Test Environment - Java Version: OpenJDK 11.0.x - Hadoop Version: 3.3.6/3.4.1 (both affected) - Test Duration: 200 filesystem creation/destruction cycles - Thread Monitoring: JMX ThreadMXBean # Fix effectiveness: 100% - no threads leaked when close() called ### For code changes: - [ X ] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')? - [ ] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation? - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, `NOTICE-binary` files? > [Bug Report] Thread leak in ABFS AbfsClientThrottlingAnalyzer > ------------------------------------------------------------- > > Key: HADOOP-19624 > URL: https://issues.apache.org/jira/browse/HADOOP-19624 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/azure > Affects Versions: 3.5.0, 3.4.1 > Reporter: Anuj Modi > Priority: Major > > Bug reported by Matt over common-dev discussion. > > What seems to be the issue is that the timer tasks are cleaned up but > > the timer threads themselves are never actually cleaned up. This will > > eventually lead to an OOM since nothing is collecting these. I was > > able to reproduce this locally in 3.3.6 and 3.4.1 but I believe that > > it would affect any version that relies on autothrottling for ABFS. > > > > I was also able to make a quick fix as well as confirm a workaround -- > > the long term fix would be to include `timer.cancel()` and > > `timer.purge()` in a method for AbfsClientThrottlingAnalyzer.java. The > > short term workaround is to disable autothrottling and rely on Azure > > to throttle the connections as needed with the below configuration. -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org