You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by "Michael Han (JIRA)" <ji...@apache.org> on 2017/03/13 16:22:04 UTC
[jira] [Updated] (ZOOKEEPER-932) Move blocking read/write calls to
SendWorker and RecvWorker Threads
[ https://issues.apache.org/jira/browse/ZOOKEEPER-932?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Michael Han updated ZOOKEEPER-932:
----------------------------------
Fix Version/s: (was: 3.5.3)
3.5.4
> Move blocking read/write calls to SendWorker and RecvWorker Threads
> -------------------------------------------------------------------
>
> Key: ZOOKEEPER-932
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-932
> Project: ZooKeeper
> Issue Type: Sub-task
> Components: leaderElection
> Affects Versions: 3.3.2
> Reporter: Vishal Kher
> Assignee: Vishal Kher
> Fix For: 3.5.4, 3.6.0
>
> Attachments: ZOOKEEPER-932.patch, ZOOKEEPER-932.patch, ZOOKEEPER-932.patch, ZOOKEEPER-932.patchv3, ZOOKEEPER-932.patchv3
>
>
> Copying relevant comments:
> Vishal K added a comment - 02/Nov/10 02:09 PM
> Hi Flavio,
> I have a suggestion for changing the blocking IO code in QuorumCnxManager. It keeps the current code structure and requires a small amount of changes. I am not sure if these comments should go in ZOOKEEPER-901. ZOOKEEPER-901 is probably addressing netty as well. Please feel free to close this JIRA if you intend to make all the changes as a part of ZOOKEEPER-901.
> Basically we jusy need to move parts of initiateConnection and receiveConnection to SenderWorker and ReceiveWorker.
> A. Current flow for receiving connection:
> 1. accept connection in Listener.run()
> 2. receiveConnection()
> * Read remote server's ID
> * Take action based on my ID and remote server's ID (disconnect and reconnect if my ID is > remote server's ID).
> * kill current set of SenderWorker and ReciveWorker threads
> * Start a new pair
> B Current flow for initiating connection:
> 1. In connectOne(), connect if not already connected. else return.
> 2. send my ID to the remote server
> 3. if my ID < remote server disconnect and return
> 4. if my ID > remote server
> * kill current set of SenderWorker and ReceiveWorkter threads for the remote server
> * Start a new pair
> Proposed changes:
> Move the code that performs any blocking IO in SenderWorker and ReceiveWorker.
> A. Proposed flow for receiving connection:
> 1. accept connection in Listener.run()
> 2. receiveConnection()
> * kill current set of SenderWorker and ReciveWorker threads
> * Start a new pair
> Proposed changed to SenderWorker:
> * Read remote server's ID
> * Take action based on my ID and remote server's ID (disconnect and reconnect if my ID is > remote server's ID).
> * Proceed to normal operation
> B Proposed flow for initiating connection:
> 1. in connectOne(), return if already connected
> 2. Start a new SenderWorker and ReceiveWorker pair
> 2. In SenderWorker
> * connect to remote server
> * write my ID
> * if my ID < remote server disconnect and return (shutdown the pair).
> * Proceed to normal operation
> Questions:
> * In QuorumCnxManager, is it necessary to kill the current pair and restart a new one every time we receive a connect request?
> * In receiveConnection we may choose to reject an accepted connection if a thread in
> SenderWorker is in the process of connecting. Otherwise a server with ID <
> remote server may keep sending frequent connect request that will result in the
> remote server closing connections for this peer. But I think we add a delay
> before sending notifications, which might be good enough to prevent this
> problem.
> Let me know what you think about this. I can also help with the implementation.
> Flavio Junqueira added a comment - 03/Nov/10 05:28 PM
> Hi Vishal, I like your proposal, it seems reasonable and not difficult to implement.
> On your questions:
> 1. I don't think it is necessary to kill a pair SenderWorker/RecvWorker every time, and I'd certainly support changing it;
> 2. I'm not sure where you're suggesting to introduce a delay. In the FLE code, a server sends a new batch of notifications if it changes its vote or if it times out waiting for a new notification. This timeout value increases over time. I was actually thinking that we should reset the timeout value upon receiving a notification. I think this is a bug....
> Given that it is your proposal, I'd be happy to let you take a stab at it and help you out if you need a hand. Does it make sense for you?
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)