Skip to content
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-1215][FOLLOWUP] Improve PausePushDataTime and PausePushDataAndReplicateTime metric calculation logic #3069

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,7 @@ protected void switchServingState() {
if (trimCounter >= forceAppendPauseSpentTimeThreshold) {
logger.debug(
"Trigger action: TRIM for {} times, force to append pause spent time.", trimCounter);
appendPauseSpentTime(servingState);
appendPauseSpentTime(lastState, servingState);
}
trimAllListeners();
}
Expand All @@ -335,14 +335,15 @@ protected void switchServingState() {
logger.info("Serving state transformed from {} to {}", lastState, servingState);
switch (servingState) {
case PUSH_PAUSED:
pausePushDataCounter.increment();
if (lastState == ServingState.PUSH_AND_REPLICATE_PAUSED) {
logger.info("Trigger action: RESUME REPLICATE");
appendPauseSpentTime(lastState, servingState);
memoryPressureListeners.forEach(
memoryPressureListener ->
memoryPressureListener.onResume(TransportModuleConstants.REPLICATE_MODULE));
} else if (lastState == ServingState.NONE_PAUSED) {
logger.info("Trigger action: PAUSE PUSH");
pausePushDataCounter.increment();
pausePushDataStartTime = System.currentTimeMillis();
memoryPressureListeners.forEach(
memoryPressureListener ->
Expand All @@ -352,9 +353,12 @@ protected void switchServingState() {
break;
case PUSH_AND_REPLICATE_PAUSED:
pausePushDataAndReplicateCounter.increment();
long currentTime = System.currentTimeMillis();
pausePushDataAndReplicateStartTime = currentTime;
if (lastState == ServingState.NONE_PAUSED) {
logger.info("Trigger action: PAUSE PUSH");
pausePushDataAndReplicateStartTime = System.currentTimeMillis();
pausePushDataCounter.increment();
pausePushDataStartTime = currentTime;
memoryPressureListeners.forEach(
memoryPressureListener ->
memoryPressureListener.onPause(TransportModuleConstants.PUSH_MODULE));
Expand All @@ -367,7 +371,7 @@ protected void switchServingState() {
break;
case NONE_PAUSED:
// resume from paused mode, append pause spent time
appendPauseSpentTime(lastState);
appendPauseSpentTime(lastState, servingState);
if (lastState == ServingState.PUSH_AND_REPLICATE_PAUSED) {
logger.info("Trigger action: RESUME REPLICATE");
memoryPressureListeners.forEach(
Expand Down Expand Up @@ -489,15 +493,17 @@ public long getPausePushDataAndReplicateTime() {
return pausePushDataAndReplicateTime;
}

private void appendPauseSpentTime(ServingState servingState) {
private void appendPauseSpentTime(ServingState lastState, ServingState currentState) {
long nextPauseStartTime = System.currentTimeMillis();
if (servingState == ServingState.PUSH_PAUSED) {
pausePushDataTime += nextPauseStartTime - pausePushDataStartTime;
pausePushDataStartTime = nextPauseStartTime;
} else {
if (lastState == ServingState.PUSH_AND_REPLICATE_PAUSED) {
pausePushDataAndReplicateTime += nextPauseStartTime - pausePushDataAndReplicateStartTime;
pausePushDataAndReplicateStartTime = nextPauseStartTime;
}
if (currentState == ServingState.NONE_PAUSED
|| (currentState == ServingState.PUSH_PAUSED && lastState == ServingState.PUSH_PAUSED)) {
pausePushDataTime += nextPauseStartTime - pausePushDataStartTime;
pausePushDataStartTime = nextPauseStartTime;
}
// reset
trimCounter = 0;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,9 +133,17 @@ class MemoryManagerSuite extends CelebornFunSuite {
}
// [CELEBORN-882] Test record pause push time
assert(memoryManager.getPausePushDataTime.longValue() > 0)
assert(memoryManager.getPausePushDataAndReplicateTime.longValue() == 0)
val lastPauseTime = memoryManager.getPausePushDataTime.longValue()

// [CELEBORN-1215][FOLLOWUP] Improve PausePushDataTime and PausePushDataAndReplicateTime metric calculation logic
assert(memoryManager.getPausePushDataAndReplicateTime.longValue() > 0)
assert(memoryManager.getPausePushDataTime.longValue()
> memoryManager.getPausePushDataAndReplicateTime.longValue())
assert(memoryManager.getPausePushDataCounter.longValue() == 1)
assert(memoryManager.getPausePushDataAndReplicateCounter.longValue() == 1)

val lastPausePushDataTime = memoryManager.getPausePushDataTime.longValue()
val lastPausePushDataAndReplicateTime =
memoryManager.getPausePushDataAndReplicateTime.longValue()
// NONE PAUSED -> PAUSE PUSH AND REPLICATE
memoryCounter.set(replicateThreshold + 1)
eventually(timeout(30.second), interval(10.milliseconds)) {
Expand All @@ -149,8 +157,14 @@ class MemoryManagerSuite extends CelebornFunSuite {
assert(!pushListener.isPause)
assert(!replicateListener.isPause)
}
assert(memoryManager.getPausePushDataTime.longValue() == lastPauseTime)

assert(memoryManager.getPausePushDataAndReplicateTime.longValue() > 0)

// [CELEBORN-1215][FOLLOWUP] Improve PausePushDataTime and PausePushDataAndReplicateTime metric calculation logic
assert(memoryManager.getPausePushDataTime.longValue() - lastPausePushDataTime
== memoryManager.getPausePushDataAndReplicateTime.longValue() - lastPausePushDataAndReplicateTime)
assert(memoryManager.getPausePushDataCounter.longValue() == 2)
assert(memoryManager.getPausePushDataAndReplicateCounter.longValue() == 2)
}

class MockMemoryPressureListener(
Expand Down