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

[#80][Part-1] feat: Add decommisson logic to shuffle server #606

Merged
merged 27 commits into from
Feb 21, 2023

Conversation

xianjingfeng
Copy link
Member

What changes were proposed in this pull request?

Add decommisson logic to shuffle server

Why are the changes needed?

Support shuffle server decommission. It is a part of #80

Does this PR introduce any user-facing change?

No

How was this patch tested?

UT

@codecov-commenter
Copy link

codecov-commenter commented Feb 16, 2023

Codecov Report

Merging #606 (470cd8c) into master (ffa26b4) will increase coverage by 1.96%.
The diff coverage is 63.29%.

@@             Coverage Diff              @@
##             master     #606      +/-   ##
============================================
+ Coverage     60.90%   62.87%   +1.96%     
+ Complexity     1799     1798       -1     
============================================
  Files           214      202      -12     
  Lines         12381    10489    -1892     
  Branches       1042     1051       +9     
============================================
- Hits           7541     6595     -946     
+ Misses         4437     3547     -890     
+ Partials        403      347      -56     
Impacted Files Coverage Δ
...n/java/org/apache/uniffle/common/ServerStatus.java 0.00% <0.00%> (ø)
...ffle/common/exception/InvalidRequestException.java 0.00% <0.00%> (ø)
.../java/org/apache/uniffle/server/ShuffleServer.java 61.69% <70.90%> (-2.70%) ⬇️
...a/org/apache/uniffle/server/ShuffleServerConf.java 99.33% <100.00%> (+0.02%) ⬆️
...a/org/apache/uniffle/server/RegisterHeartBeat.java 43.85% <0.00%> (-43.86%) ⬇️
...he/uniffle/server/buffer/ShuffleBufferManager.java 82.74% <0.00%> (-0.36%) ⬇️
...rnetes/operator/pkg/webhook/inspector/inspector.go
...tor/pkg/controller/sync/coordinator/coordinator.go
...oy/kubernetes/operator/pkg/utils/shufflerserver.go
... and 12 more

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

}

public void decommission() {
checkStatusForDecommission();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe decommission should be idempotent?

Once the shuffle server is entering decomissioning state, following calls should just do nothing, rather than throw an exception?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There will be other statuses in the future, such as Upgrading.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it doesn't matter that much?

Upgrading could also been decommissoned?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If current status is Upgrading, and than invoke decommisson, the status will be changed to decommissoning?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what do you in mind the Upgrading status mean? when the shuffle server would be upgrading?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I mean if the server is not decomissioning and not in normal status(such as upgrading), decommisson command should be reject. How about if server is decomissioning, we do nothing, but if server is upgrading, throw an exception?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated

Copy link
Contributor

@kaijchen kaijchen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @xianjingfeng for the work, some suggestions here.

Copy link
Contributor

@advancedxy advancedxy left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe this pr is in good shape, and almost ready to go. Left some minor comments.

@kaijchen
Copy link
Contributor

kaijchen commented Feb 17, 2023

Will it better to use a ExecutorService and use future.cancel() and ExecutorService.shutdownNow() for termination?

Hi @xianjingfeng, sorry for not expressing it correctly. Maybe ForkJoinPool.commonPool() is more appropriate here.

Edit: maybe just use a Future for cancel the task.

diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java
index ee8f0775..0c5ecd46 100644
--- a/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java
+++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServer.java
@@ -88,7 +87,7 @@ public class ShuffleServer {
   private MetricReporter metricReporter;
   private volatile ServerStatus serverStatus = ServerStatus.ACTIVE;
   private volatile boolean running;
   private ExecutorService executorService;
+  private Future<?> decommissionFuture;

   public ShuffleServer(ShuffleServerConf shuffleServerConf) throws Exception {
     this.shuffleServerConf = shuffleServerConf;
@@ -287,11 +286,7 @@ public class ShuffleServer {
     }
     serverStatus = ServerStatus.DECOMMISSIONING;
     LOG.info("Shuffle Server is decommissioning.");
     if (executorService == null) {
       executorService = Executors.newSingleThreadExecutor(
           ThreadUtils.getThreadFactory("shuffle-server-decommission-%d"));
     }
-    executorService.submit(this::waitDecommissionFinish);
+    decommissionFuture = executorService.submit(this::waitDecommissionFinish);
   }

   private void waitDecommissionFinish() {
@@ -332,8 +327,12 @@ public class ShuffleServer {
       LOG.info("Shuffle server is not decommissioning. Nothing needs to be done.");
       return;
     }
-    serverStatus = ServerStatus.ACTIVE;
-    LOG.info("Decommission canceled.");
+    if (decommissionFuture.cancel(true)) {
+      serverStatus = ServerStatus.ACTIVE;
+      LOG.info("Decommission canceled.");
+    } else {
+      LOG.warn("Failed to cancel decommission.");
+    }
   }

   public String getIp() {

@xianjingfeng
Copy link
Member Author

Maybe ForkJoinPool.commonPool() is more appropriate here.

What are the benefits?

@kaijchen
Copy link
Contributor

kaijchen commented Feb 18, 2023

What are the benefits?

Keeping a thread pool (singleThreadExecutor) for an uncommon task sounds inefficient.

Edit: it is not a good practice to blocking the thread using ForkJoinPool.commonPool().
But you may consider using Future to cancel the task.

@advancedxy
Copy link
Contributor

Edit: maybe just use a Future for cancel the task.

+1 for this.

@xianjingfeng
Copy link
Member Author

What are the benefits?

Keeping a thread pool (singleThreadExecutor) for an uncommon task sounds inefficient.

Edit: it is not a good practice to blocking the thread using ForkJoinPool.commonPool(). But you may consider using Future to cancel the task.

Done

@advancedxy
Copy link
Contributor

@xianjingfeng Hi, could you rebase you code with the latest master?
The CI is broken which has been addressed in #631

Copy link
Contributor

@kaijchen kaijchen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @xianjingfeng, thanks for updating the PR and docs. Some comments regarding to the "cancel decommission".

Comment on lines +342 to +346
if (decommissionFuture.cancel(true)) {
LOG.info("Decommission canceled.");
} else {
LOG.warn("Failed to cancel decommission.");
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The result of decommissionFuture.cancel() doesn't matter now. Since "cancel" always brings a server back to ACTIVE. Maybe we should call executorService.shutdownNow() here, @advancedxy WDYT?

Suggested change
if (decommissionFuture.cancel(true)) {
LOG.info("Decommission canceled.");
} else {
LOG.warn("Failed to cancel decommission.");
}
if (decommissionFuture != null) {
decommissionFuture.cancel(true);
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No. Cancel is always need. And we cannot call shutdownNow here, otherwise we cannot decommission again.

advancedxy
advancedxy previously approved these changes Feb 20, 2023
Copy link
Contributor

@advancedxy advancedxy left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Approving this since no blocking issues.

advancedxy
advancedxy previously approved these changes Feb 21, 2023
jerqi
jerqi previously approved these changes Feb 21, 2023
Copy link
Contributor

@jerqi jerqi left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@@ -149,6 +163,7 @@ public void stopServer() throws Exception {
}
SecurityContextFactory.get().getSecurityContext().close();
server.stop();
running = false;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shutdown the executorService here?

if (executorService != null) {
  executorService.shutdown()
}

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@xianjingfeng xianjingfeng dismissed stale reviews from jerqi and advancedxy via 6e1ef4a February 21, 2023 12:50
Copy link
Contributor

@kaijchen kaijchen left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks @xianjingfeng.

@xianjingfeng xianjingfeng merged commit 711f089 into apache:master Feb 21, 2023
@xianjingfeng xianjingfeng deleted the issue_80_part1 branch February 21, 2023 14:06
@xianjingfeng
Copy link
Member Author

Thanks @kaijchen @advancedxy @jerqi for the review

xianjingfeng added a commit to xianjingfeng/incubator-uniffle that referenced this pull request Apr 5, 2023
…ache#606)

### What changes were proposed in this pull request?
Add decommisson logic to shuffle server

### Why are the changes needed?
Support shuffle server decommission. It is a part of apache#80

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
UT
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants