-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HADOOP-19624. [ABFS] Fixing Thread leak in AbfsClientThrottlingAnalyzer #7852
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
+158
−3
Merged
Changes from all commits
Commits
Show all changes
17 commits
Select commit
Hold shift + click to select a range
0492703
Import closable
mattkduran 792a86f
Class implements closable
mattkduran 05ee721
Add in close method
mattkduran f894117
Import in IOException
mattkduran 5cf275c
Add in implementation to call close method
mattkduran 0a0f4e1
Add in cleanup method
mattkduran 25c6d89
Adding in close method
mattkduran fee9861
Added no-op method
mattkduran 566f596
Fix typo
matt-duran-starburst 862e002
Add @throws in the javadoc
matt-duran-starburst 7f1e3e8
Add @throws to javadoc
matt-duran-starburst 5a8b78e
Remove closeable from interface
matt-duran-starburst 37baae3
Add tests for AbfsClientThrottlingAnalyzer cleanup and idempotency
matt-duran-starburst 90abc25
Merge branch 'apache:trunk' into trunk
mattkduran 425eccb
Removing whitespaces
matt-duran-starburst 90b097b
Merge branch 'trunk' of https://github.com/mattkduran/hadoop into trunk
matt-duran-starburst cd88518
Dummy commit to force CI job
matt-duran-starburst File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,6 +20,10 @@ | |
|
|
||
| import java.io.IOException; | ||
|
|
||
| import java.lang.management.ManagementFactory; | ||
| import java.lang.management.ThreadInfo; | ||
| import java.lang.management.ThreadMXBean; | ||
|
|
||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.azurebfs.AbfsConfiguration; | ||
| import org.apache.hadoop.fs.contract.ContractTestUtils; | ||
|
|
@@ -180,4 +184,103 @@ public void testManySuccessAndErrorsAndWaiting() { | |
| sleep(10 * ANALYSIS_PERIOD); | ||
| validate(0, analyzer.getSleepDuration()); | ||
| } | ||
|
|
||
| /** | ||
| * Test that timer thread is properly cleaned up when analyzer is closed. | ||
| * This validates the fix for HADOOP-19624. | ||
| */ | ||
| @Test | ||
| public void testAnalyzerTimerCleanup() throws Exception { | ||
| int initialTimerThreads = countAbfsTimerThreads(); | ||
|
|
||
| // Create analyzer - should create one timer thread | ||
| AbfsClientThrottlingAnalyzer analyzer = | ||
| new AbfsClientThrottlingAnalyzer("test-cleanup", abfsConfiguration); | ||
|
|
||
| // Verify timer thread was created | ||
| assertEquals(initialTimerThreads + 1, countAbfsTimerThreads(), | ||
| "Timer thread should be created"); | ||
|
|
||
| // Close analyzer - should clean up timer thread | ||
| analyzer.close(); | ||
|
|
||
| // Wait for cleanup to complete | ||
| sleep(1000); | ||
|
|
||
| // Verify timer thread was cleaned up | ||
| assertEquals(initialTimerThreads, countAbfsTimerThreads(), | ||
| "Timer thread should be cleaned up after close"); | ||
| } | ||
|
|
||
| /** | ||
| * Test that close() is idempotent and can be called multiple times. | ||
| */ | ||
| @Test | ||
| public void testAnalyzerCloseIdempotent() throws Exception { | ||
| AbfsClientThrottlingAnalyzer analyzer = | ||
| new AbfsClientThrottlingAnalyzer("test-idempotent", abfsConfiguration); | ||
|
|
||
| int beforeClose = countAbfsTimerThreads(); | ||
|
|
||
| // Close multiple times - should not throw exceptions | ||
| analyzer.close(); | ||
| analyzer.close(); | ||
| analyzer.close(); | ||
|
|
||
| sleep(500); | ||
|
|
||
| // Should only clean up once | ||
| assertTrue(countAbfsTimerThreads() < beforeClose, | ||
| "Multiple close() calls should be safe"); | ||
| } | ||
|
|
||
| /** | ||
| * Test cleanup with multiple analyzers to ensure no interference. | ||
| */ | ||
| @Test | ||
| public void testMultipleAnalyzersCleanup() throws Exception { | ||
| int initialTimerThreads = countAbfsTimerThreads(); | ||
|
|
||
| // Create multiple analyzers | ||
| AbfsClientThrottlingAnalyzer analyzer1 = | ||
| new AbfsClientThrottlingAnalyzer("test-multi-1", abfsConfiguration); | ||
| AbfsClientThrottlingAnalyzer analyzer2 = | ||
| new AbfsClientThrottlingAnalyzer("test-multi-2", abfsConfiguration); | ||
| AbfsClientThrottlingAnalyzer analyzer3 = | ||
| new AbfsClientThrottlingAnalyzer("test-multi-3", abfsConfiguration); | ||
|
|
||
| // Should have created 3 timer threads | ||
| assertEquals(initialTimerThreads + 3, countAbfsTimerThreads(), | ||
| "Should create 3 timer threads"); | ||
|
|
||
| // Close all analyzers | ||
| analyzer1.close(); | ||
| analyzer2.close(); | ||
| analyzer3.close(); | ||
|
|
||
| sleep(1000); | ||
|
|
||
| // All timer threads should be cleaned up | ||
| assertEquals(initialTimerThreads, countAbfsTimerThreads(), | ||
| "All timer threads should be cleaned up"); | ||
| } | ||
|
|
||
| /** | ||
| * Helper method to count ABFS timer threads. | ||
| */ | ||
| private int countAbfsTimerThreads() { | ||
| java.lang.management.ThreadMXBean threadBean = | ||
| java.lang.management.ManagementFactory.getThreadMXBean(); | ||
| long[] threadIds = threadBean.getAllThreadIds(); | ||
|
|
||
| int count = 0; | ||
| for (long id : threadIds) { | ||
| java.lang.management.ThreadInfo info = threadBean.getThreadInfo(id); | ||
| if (info != null && | ||
| info.getThreadName().contains("abfs-timer-client-throttling-analyzer")) { | ||
| count++; | ||
| } | ||
| } | ||
| return count; | ||
| } | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Extra line needed at end of file |
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.