From dev-return-78187-archive-asf-public=cust-asf.ponee.io@zookeeper.apache.org Wed Feb 6 13:34:21 2019 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id CBC3F18067A for ; Wed, 6 Feb 2019 14:34:20 +0100 (CET) Received: (qmail 34422 invoked by uid 500); 6 Feb 2019 13:34:12 -0000 Mailing-List: contact dev-help@zookeeper.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@zookeeper.apache.org Delivered-To: mailing list dev@zookeeper.apache.org Received: (qmail 33136 invoked by uid 99); 6 Feb 2019 13:34:11 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 06 Feb 2019 13:34:11 +0000 From: GitBox To: dev@zookeeper.apache.org Subject: [GitHub] normanmaurer commented on a change in pull request #753: ZOOKEEPER-3204: Reconfig tests are constantly failing on 3.5 after applying Java 11 fix Message-ID: <154946005101.25688.13781711176288338342.gitbox@gitbox.apache.org> Date: Wed, 06 Feb 2019 13:34:11 -0000 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit normanmaurer commented on a change in pull request #753: ZOOKEEPER-3204: Reconfig tests are constantly failing on 3.5 after applying Java 11 fix URL: https://github.com/apache/zookeeper/pull/753#discussion_r254269570 ########## File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java ########## @@ -307,19 +281,126 @@ private boolean checkFourLetterWord(final Channel channel, } } - public void receiveMessage(ChannelBuffer message) { + /** + * Process incoming message. This should only be called from the event + * loop thread. + * @param buf the message bytes to process. + */ + void processMessage(ByteBuf buf) { + assert channel.eventLoop().inEventLoop(); + if (LOG.isDebugEnabled()) { + LOG.debug("0x{} queuedBuffer: {}", + Long.toHexString(sessionId), + queuedBuffer); + } + + if (LOG.isTraceEnabled()) { + LOG.trace("0x{} buf {}", + Long.toHexString(sessionId), + ByteBufUtil.hexDump(buf)); + } + + if (throttled.get()) { + LOG.debug("Received message while throttled"); + // we are throttled, so we need to queue + if (queuedBuffer == null) { + LOG.debug("allocating queue"); + queuedBuffer = channel.alloc().buffer(buf.readableBytes()); + } + queuedBuffer.writeBytes(buf); + if (LOG.isTraceEnabled()) { + LOG.trace("0x{} queuedBuffer {}", + Long.toHexString(sessionId), + ByteBufUtil.hexDump(queuedBuffer)); + } + } else { + LOG.debug("not throttled"); + if (queuedBuffer != null) { + queuedBuffer.writeBytes(buf); + processQueuedBuffer(); + } else { + receiveMessage(buf); + // Have to check !closingChannel, because an error in + // receiveMessage() could have led to close() being called. + if (!closingChannel && buf.isReadable()) { + if (LOG.isTraceEnabled()) { + LOG.trace("Before copy {}", buf); + } + if (queuedBuffer == null) { + queuedBuffer = channel.alloc().buffer(buf.readableBytes()); + } + queuedBuffer.writeBytes(buf); + if (LOG.isTraceEnabled()) { + LOG.trace("Copy is {}", queuedBuffer); + LOG.trace("0x{} queuedBuffer {}", + Long.toHexString(sessionId), + ByteBufUtil.hexDump(queuedBuffer)); + } + } + } + } + } + + /** + * Try to process previously queued message. This should only be called + * from the event loop thread. + */ + void processQueuedBuffer() { + assert channel.eventLoop().inEventLoop(); + if (queuedBuffer != null) { + if (LOG.isTraceEnabled()) { + LOG.trace("processing queue 0x{} queuedBuffer {}", + Long.toHexString(sessionId), + ByteBufUtil.hexDump(queuedBuffer)); + } + receiveMessage(queuedBuffer); + if (closingChannel) { + // close() could have been called if receiveMessage() failed + LOG.debug("Processed queue - channel closed, dropping remaining bytes"); + } else if (!queuedBuffer.isReadable()) { + LOG.debug("Processed queue - no bytes remaining"); + releaseQueuedBuffer(); + } else { + LOG.debug("Processed queue - bytes remaining"); Review comment: you should add a call to `queuedBuffer.discardSomeReadBytes()` here to free up memory otherwise you may continue growing the buffer until it is completely consumed. ---------------------------------------------------------------- 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: users@infra.apache.org With regards, Apache Git Services