zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Andor Molnar (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ZOOKEEPER-2938) Server is unable to join quorum after connection broken to other peers
Date Fri, 11 May 2018 18:48:00 GMT

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

Andor Molnar commented on ZOOKEEPER-2938:
-----------------------------------------

[~bothra90]

Thanks for reporting this issue, I'm taking a look at it. You mentioned that's it might be
related to ZOOKEEPER-1732, are you sure about that? That would mean that it had never been
fixed.

Because the log snippet you provided and the one I can see from [~adamjshook] doesn't seem
to me related. In the original issue there wasn't any connection problem between the participants.
The node was unable to join the ensemble even though it had all the necessary information
of the running ensembe. (e.g. all notifications from other servers)

[~sumitagrawal] This applies to your report as well, it looks like you all had similar issues,
but I doubt it's anyhow related to 1732.

[~fpj] Would you please take a very quick look at this ticket?

You worked on the original issues and might be able to confirm that this is not related. Thanks.

> Server is unable to join quorum after connection broken to other peers
> ----------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-2938
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2938
>             Project: ZooKeeper
>          Issue Type: Bug
>    Affects Versions: 3.4.6
>            Reporter: Abhay Bothra
>            Priority: Major
>
> We see the following logs in the node with {{myid: 1}}
> {code}
> 2017-11-08 15:06:28,375 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (2, 1)
> 2017-11-08 15:06:28,375 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (3, 1)
> 2017-11-08 15:07:28,375 [myid:1] - INFO  [WorkerReceiver[myid=1]:FastLeaderElection@597]
- Notification: 1 (message format version), 1 (n.leader), 0x28e000a8750 (n.zxid), 0x1 (n.round),
LOOKING (n.state), 1 (n.sid), 0x28e (n.peerEpoch) LOOKING (my state)
> 2017-11-08 15:07:28,375 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (2, 1)
> 2017-11-08 15:07:28,376 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (3, 1)
> 2017-11-08 15:08:28,375 [myid:1] - INFO  [WorkerReceiver[myid=1]:FastLeaderElection@597]
- Notification: 1 (message format version), 1 (n.leader), 0x28e000a8750 (n.zxid), 0x1 (n.round),
LOOKING (n.state), 1 (n.sid), 0x28e (n.peerEpoch) LOOKING (my state)
> 2017-11-08 15:08:28,376 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (2, 1)
> 2017-11-08 15:08:28,376 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (3, 1)
> 2017-11-08 15:09:28,376 [myid:1] - INFO  [WorkerReceiver[myid=1]:FastLeaderElection@597]
- Notification: 1 (message format version), 1 (n.leader), 0x28e000a8750 (n.zxid), 0x1 (n.round),
LOOKING (n.state), 1 (n.sid), 0x28e (n.peerEpoch) LOOKING (my state)
> 2017-11-08 15:09:28,376 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (2, 1)
> 2017-11-08 15:09:28,376 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (3, 1)
> 2017-11-08 15:10:28,376 [myid:1] - INFO  [WorkerReceiver[myid=1]:FastLeaderElection@597]
- Notification: 1 (message format version), 1 (n.leader), 0x28e000a8750 (n.zxid), 0x1 (n.round),
LOOKING (n.state), 1 (n.sid), 0x28e (n.peerEpoch) LOOKING (my state)
> 2017-11-08 15:10:28,376 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (2, 1)
> 2017-11-08 15:10:28,377 [myid:1] - INFO  [WorkerSender[myid=1]:QuorumCnxManager@193]
- Have smaller server identifier, so dropping the connection: (3, 1)
> {code}
> On the nodes with {{myid: 2}} and {{myid: 3}}, we see connection broken events for {{myid:
1}}
> {code}
> 2017-11-07 02:54:32,135 [myid:2] - WARN  [RecvWorker:1:QuorumCnxManager$RecvWorker@780]
- Connection broken for id 1, my id = 2, error =
> java.net.SocketException: Connection reset
>         at java.net.SocketInputStream.read(SocketInputStream.java:209)
>         at java.net.SocketInputStream.read(SocketInputStream.java:141)
>         at java.net.SocketInputStream.read(SocketInputStream.java:223)
>         at java.io.DataInputStream.readInt(DataInputStream.java:387)
>         at org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:765)
> 2017-11-07 02:54:32,135 [myid:2] - WARN  [RecvWorker:1:QuorumCnxManager$RecvWorker@783]
- Interrupting SendWorker
> 2017-11-07 02:54:32,135 [myid:2] - WARN  [SendWorker:1:QuorumCnxManager$SendWorker@697]
- Interrupted while waiting for message on queue
> java.lang.InterruptedException
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:2014)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2088)
>         at java.util.concurrent.ArrayBlockingQueue.poll(ArrayBlockingQueue.java:418)
>         at org.apache.zookeeper.server.quorum.QuorumCnxManager.pollSendQueue(QuorumCnxManager.java:849)
>         at org.apache.zookeeper.server.quorum.QuorumCnxManager.access$500(QuorumCnxManager.java:64)
>         at org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:685)
> 2017-11-07 02:54:32,135 [myid:2] - WARN  [SendWorker:1:QuorumCnxManager$SendWorker@706]
- Send worker leaving thread
> {code}
> From the reported occurrences, it looks like this is a problem only when the node with
the smallest {{myid}} loses connection.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message