[ https://issues.apache.org/jira/browse/ZOOKEEPER-1560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13483592#comment-13483592 ]
Nikita Vetoshkin commented on ZOOKEEPER-1560: --------------------------------------------- If no one can prepend {{outgoingQueue}} with packet, straightforward implementation like this should work: {noformat} diff --git a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java index 70d8538..457c8cc 100644 --- a/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java +++ b/src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java @@ -111,17 +111,20 @@ public class ClientCnxnSocketNIO extends ClientCnxnSocket { cnxn.sendThread.clientTunneledAuthenticationInProgress()); if (p != null) { - outgoingQueue.removeFirstOccurrence(p); updateLastSend(); - if ((p.requestHeader != null) && - (p.requestHeader.getType() != OpCode.ping) && - (p.requestHeader.getType() != OpCode.auth)) { - p.requestHeader.setXid(cnxn.getXid()); + if (p.bb != null) { + if ((p.requestHeader != null) && + (p.requestHeader.getType() != OpCode.ping) && + (p.requestHeader.getType() != OpCode.auth)) { + p.requestHeader.setXid(cnxn.getXid()); + } + p.createBB(); + // otherwise we're in the middle of sending packet } - p.createBB(); ByteBuffer pbb = p.bb; sock.write(pbb); if (!pbb.hasRemaining()) { + outgoingQueue.removeFirstOccurrence(p); sentCount++; if (p.requestHeader != null && p.requestHeader.getType() != OpCode.ping {noformat} > Zookeeper client hangs on creation of large nodes > ------------------------------------------------- > > Key: ZOOKEEPER-1560 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1560 > Project: ZooKeeper > Issue Type: Bug > Components: java client > Affects Versions: 3.4.4, 3.5.0 > Reporter: Igor Motov > Assignee: Ted Yu > Fix For: 3.5.0, 3.4.5 > > Attachments: ZOOKEEPER-1560.patch, zookeeper-1560-v1.txt, > zookeeper-1560-v2.txt, zookeeper-1560-v3.txt, zookeeper-1560-v4.txt, > zookeeper-1560-v5.txt, zookeeper-1560-v6.txt, zookeeper-1560-v7.txt > > > To reproduce, try creating a node with 0.5M of data using java client. The > test will hang waiting for a response from the server. See the attached patch > for the test that reproduces the issue. > It seems that ZOOKEEPER-1437 introduced a few issues to > {{ClientCnxnSocketNIO.doIO}} that prevent {{ClientCnxnSocketNIO}} from > sending large packets that require several invocations of > {{SocketChannel.write}} to complete. The first issue is that the call to > {{outgoingQueue.removeFirstOccurrence(p);}} removes the packet from the queue > even if the packet wasn't completely sent yet. It looks to me that this call > should be moved under {{if (!pbb.hasRemaining())}} The second issue is that > {{p.createBB()}} is reinitializing {{ByteBuffer}} on every iteration, which > confuses {{SocketChannel.write}}. And the third issue is caused by extra > calls to {{cnxn.getXid()}} that increment xid on every iteration and confuse > the server. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira