This is an automated email from the ASF dual-hosted git repository.
mmerli 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 956b37d Avoid contention on netty channel promise
956b37d is described below
commit 956b37dad75a98b23cff34e8cfe0179e15e6d98a
Author: Matteo Merli <[email protected]>
AuthorDate: Fri Apr 6 11:57:48 2018 -0700
Avoid contention on netty channel promise
With profiler, I have seen there can be heavy contention between BK threads
and Netty IO thread due the the checking for channel write condition that was
recently added for monitoring purpose.
The problem relies in that there is one BK thread that is doing the
`writeAndFlush()` on the PCBC and getting the `ChannelFuture`, adding a
listener to the future.
The write operation, though, is completed in the Netty IO thread and the
promise gets also triggered from that thread. So, there is contention between
current thread adding the listener and the IO threads completing the promise.
If we add the listener before doing the write on channel, we can avoid the
contention. Another option could be to do the write from the Netty IO thread as
well.
Author: Matteo Merli <[email protected]>
Reviewers: Enrico Olivelli <[email protected]>, Sijie Guo
<[email protected]>
This closes #1321 from merlimat/channel-promise-contention
---
.../apache/bookkeeper/proto/PerChannelBookieClient.java | 16 ++++++++--------
1 file changed, 8 insertions(+), 8 deletions(-)
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
index 20fcdda..b125d97 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
@@ -42,6 +42,7 @@ import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline;
+import io.netty.channel.ChannelPromise;
import io.netty.channel.DefaultEventLoopGroup;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.WriteBufferWaterMark;
@@ -877,21 +878,20 @@ public class PerChannelBookieClient extends
ChannelInboundHandlerAdapter {
try {
final long startTime = MathUtils.nowInNano();
- ChannelFuture future = channel.writeAndFlush(request);
- future.addListener(future1 -> {
- if (future1.isSuccess()) {
-
nettyOpLogger.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime),
- TimeUnit.NANOSECONDS);
+
+ ChannelPromise promise = channel.newPromise().addListener(future
-> {
+ if (future.isSuccess()) {
+
nettyOpLogger.registerSuccessfulEvent(MathUtils.elapsedNanos(startTime),
TimeUnit.NANOSECONDS);
CompletionValue completion = completionObjects.get(key);
if (completion != null) {
completion.setOutstanding();
}
-
} else {
-
nettyOpLogger.registerFailedEvent(MathUtils.elapsedNanos(startTime),
- TimeUnit.NANOSECONDS);
+
nettyOpLogger.registerFailedEvent(MathUtils.elapsedNanos(startTime),
TimeUnit.NANOSECONDS);
}
});
+
+ channel.writeAndFlush(request, promise);
} catch (Throwable e) {
LOG.warn("Operation {} failed", requestToString(request), e);
errorOut(key);
--
To stop receiving notification emails like this one, please contact
[email protected].