-
Notifications
You must be signed in to change notification settings - Fork 380
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
[CELEBORN-1792][FOLLOWUP] Keep resume for a while after resumeByPinnedMemory #3099
base: main
Are you sure you want to change the base?
Conversation
@TheodoreLx, please use |
@SteNicholas The code has been reformatted |
Could you fix the ut |
common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala
Outdated
Show resolved
Hide resolved
worker/src/main/java/org/apache/celeborn/service/deploy/worker/memory/MemoryManager.java
Outdated
Show resolved
Hide resolved
worker/src/test/scala/org/apache/celeborn/service/deploy/memory/MemoryManagerSuite.scala
Outdated
Show resolved
Hide resolved
common/src/main/scala/org/apache/celeborn/common/CelebornConf.scala
Outdated
Show resolved
Hide resolved
…umeThreshold or timeout
Here are some test results about #3018 I want to share: After triggering resumeByPinnedMemory, it will enter pause immediately. The duration of resumeByPinnedMemory is less than 10ms, and the worker will soon fall into the pause deadlock state again until resumeByPinnedMemory is triggered again in 10 seconds. In other words, worker wastes about 10 seconds staying in the pause state. 2025-02-19 16:53:11,053 resumeByPinnedMemory |
worker/src/main/java/org/apache/celeborn/service/deploy/worker/memory/MemoryManager.java
Outdated
Show resolved
Hide resolved
worker/src/main/java/org/apache/celeborn/service/deploy/worker/memory/MemoryManager.java
Outdated
Show resolved
Hide resolved
worker/src/main/java/org/apache/celeborn/service/deploy/worker/memory/MemoryManager.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
return false; | ||
} | ||
if (System.currentTimeMillis() - pinnedMemoryLastCheckTime >= pinnedMemoryCheckInterval) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
introduce a variable for System.currentTimeMillis()
?
Codecov ReportAll modified and coverable lines are covered by tests ✅
Additional details and impacted files@@ Coverage Diff @@
## main #3099 +/- ##
==========================================
+ Coverage 32.62% 32.68% +0.06%
==========================================
Files 336 338 +2
Lines 20083 20273 +190
Branches 1798 1812 +14
==========================================
+ Hits 6551 6625 +74
- Misses 13165 13279 +114
- Partials 367 369 +2 ☔ View full report in Codecov by Sentry. |
resumeReplicate(); | ||
} else { | ||
logger.info("Trigger action: PAUSE PUSH"); | ||
pausePushDataStartTime = System.currentTimeMillis(); | ||
resumingByPinnedMemory = false; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
resumingByPinnedMemory need also change to false when servingState changes to NONE_PAUSED state
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
@TheodoreLx There is an issue with this PR. The trimCounter should be moved into the block of the The current implementation will cause the metric of PAUSE_PUSH_DATA_TIME and PAUSE_PUSH_DATA_AND_REPLICATE_TIME to be larger than the time spent. |
What changes were proposed in this pull request?
In the switchServingState after resumeByPinnedMemory, keep the resume channel to prevent the channel from frequently resuming and pausing before memoryUsage decreases to pausePushDataThreshold.
Why are the changes needed?
Frequent channel resume and pause will result in slow data reception and failure to quickly reduce memoryUsage to below pausePushDataThreshold.
Does this PR introduce any user-facing change?
no
How was this patch tested?
ut