You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ha...@apache.org on 2018/07/27 03:13:27 UTC
zookeeper git commit: ZOOKEEPER-3097: Using Runnable instead of
thread work items to improve the throughput of CommitProcessor
Repository: zookeeper
Updated Branches:
refs/heads/master ed4689fbf -> 75c652f45
ZOOKEEPER-3097: Using Runnable instead of thread work items to improve the throughput of CommitProcessor
Author: Fangmin Lyu <al...@fb.com>
Reviewers: Michael Han <ha...@apache.org>
Closes #578 from lvfangmin/ZOOKEEPER-3097
Project: http://git-wip-us.apache.org/repos/asf/zookeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/zookeeper/commit/75c652f4
Tree: http://git-wip-us.apache.org/repos/asf/zookeeper/tree/75c652f4
Diff: http://git-wip-us.apache.org/repos/asf/zookeeper/diff/75c652f4
Branch: refs/heads/master
Commit: 75c652f45090f5309060ab7defb37f337d327d2c
Parents: ed4689f
Author: Fangmin Lyu <al...@fb.com>
Authored: Thu Jul 26 20:13:22 2018 -0700
Committer: Michael Han <ha...@apache.org>
Committed: Thu Jul 26 20:13:22 2018 -0700
----------------------------------------------------------------------
.../main/org/apache/zookeeper/server/WorkerService.java | 11 ++---------
1 file changed, 2 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/75c652f4/src/java/main/org/apache/zookeeper/server/WorkerService.java
----------------------------------------------------------------------
diff --git a/src/java/main/org/apache/zookeeper/server/WorkerService.java b/src/java/main/org/apache/zookeeper/server/WorkerService.java
index 416e3af..f6100d5 100644
--- a/src/java/main/org/apache/zookeeper/server/WorkerService.java
+++ b/src/java/main/org/apache/zookeeper/server/WorkerService.java
@@ -133,21 +133,14 @@ public class WorkerService {
} else {
// When there is no worker thread pool, do the work directly
// and wait for its completion
- scheduledWorkRequest.start();
- try {
- scheduledWorkRequest.join();
- } catch (InterruptedException e) {
- LOG.warn("Unexpected exception", e);
- Thread.currentThread().interrupt();
- }
+ scheduledWorkRequest.run();
}
}
- private class ScheduledWorkRequest extends ZooKeeperThread {
+ private class ScheduledWorkRequest implements Runnable {
private final WorkRequest workRequest;
ScheduledWorkRequest(WorkRequest workRequest) {
- super("ScheduledWorkRequest");
this.workRequest = workRequest;
}