> On Dec. 11, 2014, 11:16 p.m., fpj wrote:
> > src/java/main/org/apache/zookeeper/ClientCnxn.java, line 134
> > <https://reviews.apache.org/r/27244/diff/28/?file=789547#file789547line134>
> >
> >     It sounds like making pendingQueue and outgoingQueue are optimizations 
> > that are independent from this issue. Shouldn't we make this change in a 
> > different jira?
> 
> Hongchao Deng wrote:
>     outgoingQueue isn't optimization... My previous patch used a semaphore 
> which will be waken up by WakeupCnxn(). Your suggestion is use a blocking 
> outgoingQueue. What do you think? I am fine since I know both changes well 
> and I can do it in separate JIRAs.
> 
> fpj wrote:
>     My concern here is that it ended up implying more changes than just 
> removing the semaphore. For example, the patch removes this synchronized 
> block (not sure it is going to format it right):
>     
>                 for (Packet p : outgoingQueue) {
>                    conLossPacket(p);
>                 }
>                 outgoingQueue.clear();
>                 
>     and checking the other synchronized blocks removed, it isn't entirely 
> straightforward that you can remove all of them.

I can add those synchronized back. However, findbugs will complain that so I 
will add rules too.

On Netty side I can assure your concern:
1. In ClientCnxn, primeConnection(), cleanup(), queuePacket(). 
primeConnection(), cleanup() is guaranteed to be start and end of life cycles. 
queuePacket() is appending packets to outgoingQueue and waking up the blocking 
queue.


What do you think?


- Hongchao


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/27244/#review64807
-----------------------------------------------------------


On Dec. 12, 2014, 7:16 p.m., Hongchao Deng wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/27244/
> -----------------------------------------------------------
> 
> (Updated Dec. 12, 2014, 7:16 p.m.)
> 
> 
> Review request for zookeeper.
> 
> 
> Repository: zookeeper-git
> 
> 
> Description
> -------
> 
> ZOOKEEPER-2069
> 
> 
> Diffs
> -----
> 
>   build.xml bb5ff4f 
>   src/java/main/org/apache/zookeeper/ClientCnxn.java b4ece07 
>   src/java/main/org/apache/zookeeper/ClientCnxnSocket.java 5ca0ba7 
>   src/java/main/org/apache/zookeeper/ClientCnxnSocketNIO.java adb27ee 
>   src/java/main/org/apache/zookeeper/ClientCnxnSocketNetty.java PRE-CREATION 
>   src/java/main/org/apache/zookeeper/ZooKeeperTestable.java 775d1a2 
>   src/java/main/org/apache/zookeeper/client/ZooKeeperSaslClient.java dbc1080 
>   src/java/test/org/apache/zookeeper/test/ClientTest.java dbe595c 
>   src/java/test/org/apache/zookeeper/test/NettyNettySuiteBase.java 
> PRE-CREATION 
>   src/java/test/org/apache/zookeeper/test/NettyNettySuiteHammerTest.java 
> PRE-CREATION 
>   src/java/test/org/apache/zookeeper/test/NettyNettySuiteTest.java 
> PRE-CREATION 
> 
> Diff: https://reviews.apache.org/r/27244/diff/
> 
> 
> Testing
> -------
> 
> 1. use LinkedBlockingDeque.
> 
> 
> Thanks,
> 
> Hongchao Deng
> 
>

Reply via email to