zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Vishal Khandelwal (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ZOOKEEPER-2447) Zookeeper adds good delay when one of the quorum host is not reachable
Date Wed, 03 Aug 2016 17:13:20 GMT

    [ https://issues.apache.org/jira/browse/ZOOKEEPER-2447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15406241#comment-15406241
] 

Vishal Khandelwal commented on ZOOKEEPER-2447:
----------------------------------------------

Yeah I also looked into patch. The problem i described in the JIRA and fix in patch are totally
different. The way i would like to solve the problem is reducing the connectTimeout to a lower
Value as compared to sessionTimeout/quorum size. That is place when I was debugging I found
the delay. I agree with folks that "taddr.isReachable" might not work in every situation.
We observe that in our environment as well. Where server in not pingable but zookeeper connection
can work. I tried to fix by reducing the connecttiemout to a lower or fixed value.so bad host
can be caught quickly and also it is not removed part of the list. In net iteration it will
checked for conenction again. I would submit the patch tomorrow on this if people agree. Tried
it already on our clusters.

> Zookeeper adds  good delay when one of the quorum host is not reachable
> -----------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-2447
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2447
>             Project: ZooKeeper
>          Issue Type: Bug
>    Affects Versions: 3.4.6, 3.5.0
>            Reporter: Vishal Khandelwal
>            Assignee: Edward Ribeiro
>             Fix For: 3.5.3, 3.6.0
>
>         Attachments: ZOOKEEPER-2447-MinConnectTimeoutOnly.patch, ZOOKEEPER-2447.3.5.patch,
withfix.txt, withoutFix.txt
>
>
> StaticHostProvider --> resolveAndShuffle method adds all of the address which are
valid in the quorum to the list, shuffles them and sends back to client connection class.
If after shuffling if first node appear to be the one which is not reachable, Clientcnx.SendThread.run
will keep on connecting to the failure till a timeout and the moves to a different node. This
adds up random delay in zookeeper connection in case a host is down. Rather we could check
if host is reachable in StaticHostProvider and ignore isReachable is false. Same as we do
for UnknownHostException Exception.
> This can tested using following test code by providing a valid host which is not reachable.
for quick test comment Collections.shuffle(tmpList, sourceOfRandomness); in StaticHostProvider.resolveAndShuffle
> {code}
>  @Test
>   public void test() throws Exception {
>     EventsWatcher watcher = new EventsWatcher();
>     QuorumUtil qu = new QuorumUtil(1);
>     qu.startAll();
>     
>     ZooKeeper zk =
>         new ZooKeeper("<hostnamet:2181," + qu.getConnString(), 180 * 1000, watcher);
>     
>     watcher.waitForConnected(CONNECTION_TIMEOUT * 5);
>     Assert.assertTrue("connection Established", watcher.isConnected());
>     zk.close();    
>   }
> {code}
> Following fix can be added to StaticHostProvider.resolveAndShuffle
> {code}
>  if(taddr.isReachable(4000 // can be some value)) {
>                       tmpList.add(new InetSocketAddress(taddr, address.getPort()));
>                     } 
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message