Apologies for sending this on the mail list instead of through a comment on
the JIRA - I can't seem to get https://issues.apache.org/jira/ to load for
me.

Given that the goal is to open a socket, implementing an isReachable()
method which is just opening a socket with a timeout seems wasteful. It
will add unnecessary additional latency (the RTT latency for TCP handshake)
and overhead to every connection, even successful ones where the ensemble
isn't suffering due to a down host; it'll also put a little extra load on
the servers, not just clients. Would it make sense to make
ClientCnxnSocket.open() implementations apply the timeout directly?

As far as choosing the connection timeout to use, I would suggest just
using the time-sliced session timeout (I thought this was already done, but
perhaps it's only applied later in the connection handshake?) but applying
some reasonable lower bound (say, default of 1 second and overridable by a
Java system property). We actually have a fork of the ZK code which has
that minimum bound for connection timeout logic in the Java client, and I'd
be happy to prepare a patch for that part. It's pretty trivial code, but
it's important if you start using a combination of large ZK clusters and
very small session timeouts, which we have in some of our ensembles.

Thanks,
Dan

On Wed, Jun 29, 2016 at 12:27 PM, Andrew Purtell (JIRA) <[email protected]>
wrote:

>
>     [
> https://issues.apache.org/jira/browse/ZOOKEEPER-2447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15355708#comment-15355708
> ]
>
> Andrew Purtell commented on ZOOKEEPER-2447:
> -------------------------------------------
>
> bq. An alternative would be to roll out our own equivalent of
> {{InetAddress#isReachable()}} as I scribbled below?
> Makes sense.
>
> > 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: Vishal Khandelwal
> >             Fix For: 3.5.3, 3.6.0
> >
> >         Attachments: 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)
>

Reply via email to