dlg99 commented on a change in pull request #1088: ISSUE #1086 (@bug 
W-4146427@) Client-side backpressure in netty (Fixes: 
io.netty.util.internal.OutOfDirectMemoryError under continuous heavy load)
URL: https://github.com/apache/bookkeeper/pull/1088#discussion_r164895511
 
 

 ##########
 File path: 
bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
 ##########
 @@ -849,17 +857,84 @@ private ChannelFuture closeChannel(Channel c) {
         if (LOG.isDebugEnabled()) {
             LOG.debug("Closing channel {}", c);
         }
-        return c.close();
+        return c.close().addListener(x -> channelChanged());
+    }
+
+    private void channelChanged() {
+        synchronized (channelMonitor) {
+            channelMonitor.notifyAll();
+        }
+    }
+
+    @Override
+    public void channelWritabilityChanged(ChannelHandlerContext ctx) throws 
Exception {
+        channelChanged();
+        super.channelWritabilityChanged(ctx);
     }
 
     private void writeAndFlush(final Channel channel,
                                final CompletionKey key,
                                final Object request) {
-        if (channel == null) {
+        writeAndFlush(channel, key, request, -1);
+    }
+
+    private void writeAndFlush(final Channel channel,
+                           final CompletionKey key,
+                           final Object request,
+                           final long timeoutMillis) {
+        if (channel == null || !channel.isActive() || channel != this.channel) 
{
             errorOut(key);
             return;
         }
 
+        // channel.isWritable() check will actually respect netty's behavior 
configured with
+        // clientWriteBufferLowWaterMark and clientWriteBufferHighWaterMark 
config parameters.
+        // It is highly recommended setting them reasonably high for expected 
size of requests and load
+        // Otherwise client may end up doing frequent waits in this loop.
+        if (!channel.isWritable()) {
 
 Review comment:
   channel.isWritable() is netty channel's attribute. It switches to false when 
clientWriteBufferHighWaterMark of unsent data is reached. it switches to true 
when some data is sent and buffer usage drops below 
clientWriteBufferLowWaterMark. 
   This is netty's way to let us know that we are trying to push to much and 
cannot keep up.
   the fact that we did not pay any attention to it was a problem addressed 
here.
   
   It is of course possible to just fail request in this case but in case of 
busy system / maxed out NIC capacity it will result in too many failed requests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to