zookeeper-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Flavio Junqueira <...@apache.org>
Subject Re: Zookeeper exception: Timeout while waiting for epoch from quorum
Date Tue, 25 Oct 2016 18:15:18 GMT
There are two potential reasons I can think of:

1- The configuration is incorrect and it is pointing to the wrong port. Keep in mind that
the election port is different from the quorum server communication port.
2- There is something block traffic on the configured port between those two servers.

Could it be any of these?

-Flavio

> On 25 Oct 2016, at 14:33, Krzysztof Rybak <krzysztof.rybak@motorolasolutions.com>
wrote:
> 
> Hi,
> first log is a little misleading as epoch exception is observed later in a
> log and may be a consequence of lack of connection.
> The first exception observed in failing scenario is:
> 
> [2016-10-25 07:48:41,107] WARN Unexpected exception, tries=0, connecting to
> /10.54.1.53:12001 (org.apache.zookeeper.server.quorum.Learner)
> java.net.ConnectException: Connection refused
>        at java.net.PlainSocketImpl.socketConnect(Native Method)
>        at
> java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:350)
>        at
> java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:206)
>        at
> java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188)
>        at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392)
>        at java.net.Socket.connect(Socket.java:589)
>        at
> org.apache.zookeeper.server.quorum.Learner.connectToLeader(Learner.java:225)
>        at
> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:71)
>        at
> org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:786)
> 
> I think that instances are able to elect a Leader(zookeeper instance on
> address 10.54.1.53, leader port 12001-2888 in zookeeper examples) as such
> information is logged before exception:
> 
> [2016-10-25 07:48:41,103] INFO FOLLOWING - LEADER ELECTION TOOK - 10249
> (org.apache.zookeeper.server.quorum.Learner)
> Then follower tries to connect to leader but this fails.
> 
> - If a leader is being elected but the leader isn't being able to sync and
> establish itself, then you need to determine the reason. Is the follower
> being able to connect to the leader? Is the initialization taking too long
> such that it times out? Is the socket read timing out?
> 
> This seems to be an issue that follower cannot connect to a leader on
> leader port (12001 in my case).
> Exception is related to connect() method:
> 
> at
> org.apache.zookeeper.server.quorum.Learner.connectToLeader(Learner.java:225)
> 
> and the exception is
> 
> java.net.ConnectException: Connection refused
> 
> Follower tries to connect just after the leader stops listening on a leader
> port (12001 in my case, 2888 in zookeeper examples).
> I just don't know why follower starts trying to connect after around 10
> seconds after setting a new leader(during this 10 seconds leader listens on
> a leader port). There is no timeout in socket connect as leader has already
> stopped listening.
> This seems to be a reason, but why this happens ?
> 
> I assume this scenario is supported and server should be set.
> 
>> [2016-10-20 04:04:19,996] WARN Unexpected exception, tries=0, connecting
> to
>> /10.54.1.53:12001 (org.apache.zookeeper.server.quorum.Learner)
>> java.net.ConnectException: Connection refused
>> at java.net.PlainSocketImpl.socketConnect(Native Method)
>> at
>> java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:
> 350)
>> at
>> java.net.AbstractPlainSocketImpl.connectToAddress(
> AbstractPlainSocketImpl.java:206)
>> at
>> java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188)
>> at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392)
>> at java.net.Socket.connect(Socket.java:589)
>> at
>> org.apache.zookeeper.server.quorum.Learner.connectToLeader(Learner.java:
> 225)
>> at
>> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:71)
>> at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:786)
> 
> 
> 
> On Mon, Oct 24, 2016 at 6:57 PM, Flavio Junqueira <fpj@apache.org> wrote:
> 
>> Hi Krzysztof,
>> 
>> Your first message seems to indicate that a leader is being elected, but
>> it is not being able to sync with enough followers (in your case 1
>> follower). I'm saying this because of this line:
>> 
>>> at org.apache.zookeeper.server.quorum.Leader.lead(Leader.java:377)
>> 
>> You say that the issue is timing in your second message, but it isn't
>> entirely clear to me why you think so. I think you need to determine the
>> following:
>> 
>> - Are the two remaining servers being able to elect a leader among them?
>> If not, is the problem that they can't connect to each other?
>> - If a leader is being elected but the leader isn't being able to sync and
>> establish itself, then you need to determine the reason. Is the follower
>> being able to connect to the leader? Is the initialization taking too long
>> such that it times out? Is the socket read timing out?
>> 
>> -Flavio
>> 
>> 
>>> On 24 Oct 2016, at 11:37, Krzysztof Rybak <krzysztof.rybak@
>> motorolasolutions.com> wrote:
>>> 
>>> Hi,
>>> I've logged some more information and the problem seems to be a timing
>>> issue.
>>> Long story short, the scenario is:
>>> zk1(follower)
>>> zk2(leader)
>>> zk3(follower)
>>> 
>>> step 1. stop zk1
>>> step 2. stop zk2
>>> step 3. start zk1
>>> 
>>> result:
>>> zk1 and zk3 cannot create a cluster of 2 zk instances(is such scenario
>>> supported by zookeeper?).
>>> When zk2 is started again instead of zk1, cluster is created.
>>> 
>>> Observation:
>>> After step 3., zk3 is elected to be a leader and starts listening on port
>>> 2888
>>> ( example port from example
>>> http://zookeeper.apache.org/doc/trunk/zookeeperStarted.html ). Listening
>>> takes around 10 seconds but during this time zk1 is not trying to
>> connect.
>>> zk1 tries to connect just after zk3 stops listening.
>>> 
>>> a part of a log is:
>>> zk3 stops listening on 04:04:17(testing with netstat) and zk1 starts
>> trying
>>> to connect at 04:04:19,996 as below.
>>> 
>>> [2016-10-20 04:04:19,996] WARN Unexpected exception, tries=0, connecting
>> to
>>> /10.54.1.53:12001 (org.apache.zookeeper.server.quorum.Learner)
>>> java.net.ConnectException: Connection refused
>>> at java.net.PlainSocketImpl.socketConnect(Native Method)
>>> at
>>> java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:
>> 350)
>>> at
>>> java.net.AbstractPlainSocketImpl.connectToAddress(
>> AbstractPlainSocketImpl.java:206)
>>> at
>>> java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:
>> 188)
>>> at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392)
>>> at java.net.Socket.connect(Socket.java:589)
>>> at
>>> org.apache.zookeeper.server.quorum.Learner.connectToLeader(Learner.java:
>> 225)
>>> at
>>> org.apache.zookeeper.server.quorum.Follower.followLeader(
>> Follower.java:71)
>>> at org.apache.zookeeper.server.quorum.QuorumPeer.run(
>> QuorumPeer.java:786)
>>> 
>>> This is strange as I couldn't reproduce this on local VM VirtualBox but
>> on
>>> VMware reproduction is 100%.
>>> 
>>> thanks,
>>> Krzysztof
>>> 
>>> ---------- Forwarded message ----------
>>> From: Krzysztof Rybak <krzysztof.rybak@motorolasolutions.com>
>>> Date: Thu, Oct 20, 2016 at 4:50 PM
>>> Subject: Zookeeper exception: Timeout while waiting for epoch from quorum
>>> To: user@zookeeper.apache.org
>>> 
>>> 
>>> Hi All,
>>> first mail in the group so sorry for possible inconsistency in advance.
>>> Zookeeper version is zookeeper-3.4.6.
>>> 
>>> I'm facing a problem when zookeeper is reconfiguring a cluster.
>>> 
>>> Initial state:
>>> machine A:
>>> zk1(follower)
>>> zk2(leader)
>>> machine B:
>>> zk3(follower)
>>> 
>>> zk1 and zk2 are stopped (in that order).
>>> zk1 is started on machine B.
>>> zk1 and zk3 are not creating a cluster, status is (using srvr word)
>>> 'This ZooKeeper instance is not currently serving requests'
>>> 
>>> A part of a log is:
>>> [2016-10-20 04:03:10,053] WARN Unexpected exception
>>> (org.apache.zookeeper.server.quorum.QuorumPeer)
>>> java.lang.InterruptedException: Timeout while waiting for epoch from
>> quorum
>>> at org.apache.zookeeper.server.quorum.Leader.
>> getEpochToPropose(Leader.java:
>>> 878)
>>> at org.apache.zookeeper.server.quorum.Leader.lead(Leader.java:377)
>>> at org.apache.zookeeper.server.quorum.QuorumPeer.run(
>> QuorumPeer.java:799)
>>> [2016-10-20 04:03:10,054] INFO Shutting down
>> (org.apache.zookeeper.server.
>>> quorum.Leader)
>>> [2016-10-20 04:03:10,054] INFO Shutdown called
>> (org.apache.zookeeper.server.
>>> quorum.Leader)
>>> java.lang.Exception: shutdown Leader! reason: Forcing shutdown
>>> at org.apache.zookeeper.server.quorum.Leader.shutdown(Leader.java:499)
>>> at org.apache.zookeeper.server.quorum.QuorumPeer.run(
>> QuorumPeer.java:805)
>>> [2016-10-20 04:03:10,054] INFO shutting down
>> (org.apache.zookeeper.server.
>>> ZooKeeperServer)
>>> 
>>> What is interesting: when zk2(previous leader) is started on machine B
>>> (instead of zk1) cluster is configured correctly.
>>> The same situation happens when all happen on the single machine.
>>> 
>>> Issue is similar to this, but algorithm used by me is 3 (by default and
>>> confirmed with electionAlg=3 in .cfg files)
>>> https://issues.apache.org/jira/browse/ZOOKEEPER-2400
>>> 
>>> thanks,
>>> Krzysztof
>> 
>> 


Mime
View raw message