This is an automated email from the ASF dual-hosted git repository.
sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/master by this push:
new 30b7beb Allow to send callbacks from journal thread
30b7beb is described below
commit 30b7bebff7c99ac2cf6c5b784b8a122b7b3b6022
Author: Matteo Merli <[email protected]>
AuthorDate: Wed Mar 14 14:30:44 2018 -0700
Allow to send callbacks from journal thread
In BK 4.3 it was possible to configure 0 threads for the dispatching of
journal callback threads.
There might be good reasons to not use a thread pool when dispatching the
callback threads:
* Avoiding one more context switch
* Avoiding contention on the executor enqueuing
* When using multiple journals, the journal thread is not a bottleneck
anymore
We should allow the same convention (threads=0 means direct execution) that
we have in other places.
Author: Matteo Merli <[email protected]>
Reviewers: Ivan Kelly <[email protected]>, Sijie Guo <[email protected]>
This closes #1258 from merlimat/journal-callbacks
---
.../src/main/java/org/apache/bookkeeper/bookie/Journal.java | 11 ++++++++---
1 file changed, 8 insertions(+), 3 deletions(-)
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
index 5a816a0..9e9d055 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java
@@ -23,11 +23,13 @@ package org.apache.bookkeeper.bookie;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Stopwatch;
+import com.google.common.util.concurrent.MoreExecutors;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.util.Recycler;
import io.netty.util.Recycler.Handle;
+import io.netty.util.concurrent.DefaultThreadFactory;
import java.io.File;
import java.io.FileInputStream;
@@ -50,7 +52,6 @@ import org.apache.bookkeeper.stats.Counter;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.DaemonThreadFactory;
import org.apache.bookkeeper.util.IOUtils;
import org.apache.bookkeeper.util.MathUtils;
import org.apache.bookkeeper.util.collections.GrowableArrayBlockingQueue;
@@ -625,8 +626,12 @@ public class Journal extends BookieCriticalThread
implements CheckpointSource {
this.maxGroupWaitInNanos =
TimeUnit.MILLISECONDS.toNanos(conf.getJournalMaxGroupWaitMSec());
this.bufferedWritesThreshold =
conf.getJournalBufferedWritesThreshold();
this.bufferedEntriesThreshold =
conf.getJournalBufferedEntriesThreshold();
- this.cbThreadPool =
Executors.newFixedThreadPool(conf.getNumJournalCallbackThreads(),
- new
DaemonThreadFactory());
+ if (conf.getNumJournalCallbackThreads() > 0) {
+ this.cbThreadPool =
Executors.newFixedThreadPool(conf.getNumJournalCallbackThreads(),
+ new
DefaultThreadFactory("bookie-journal-callback"));
+ } else {
+ this.cbThreadPool = MoreExecutors.newDirectExecutorService();
+ }
// Unless there is a cap on the max wait (which requires group force
writes)
// we cannot skip flushing for queue empty
--
To stop receiving notification emails like this one, please contact
[email protected].