ZOOKEEPER-4766 - Avoid snapshots during leader election
Hello ZK Community! Tagging some friends in the community for starting the conversation. * @Enrico Olivelli from bookkeeper and pulsar who might have more context. * Yan Zhao who brought up very good concerns. Yan Zhao thinks it's a good idea to implement this, however has concerns <https://github.com/apache/zookeeper/pull/2153#discussion_r1581839692> that if we give up taking a snapshot, the current session info may not persist and hence the session data may not be accurate during the next restart. Rishabh (the developer of this feature request) asks: > is this session info used for something upon restart that is necessary for correctness? Is it mainly important for telemetry? If it is not needed for correctness could skipping this snapshotting step during leader election be made a configurable option? As per my knowledge, If session info is not persisted, sessions will be lost upon restart. This means all ephemeral nodes associated with those sessions will be lost, leading to potential inconsistencies. Hoping to open this up for consideration with the community. We can use advice on how to make this change happen as it will greatly improve the leader election times for some of our large scale clusters. Thanks, Anup
[jira] [Created] (ZOOKEEPER-4771) Fast leader election taking too long
Ivo Vrdoljak created ZOOKEEPER-4771: --- Summary: Fast leader election taking too long Key: ZOOKEEPER-4771 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4771 Project: ZooKeeper Issue Type: Bug Components: leaderElection Affects Versions: 3.4.10 Reporter: Ivo Vrdoljak Attachments: zookeeper10.log, zookeeper11.log, zookeeper12.log, zookeeper20.log, zookeeper21.log Hello ZooKeeper Community, Background: We are using ZooKeeper version 3.4.10. in our system and we have 5 Zookeeper servers running, that are distributed across 2 clusters of servers. In the first cluster, we have 3 Zookeeper servers, each deployed on its own machine, and in the second cluster we have 2 Zookeeper servers, also each on its own machine. Zookeeper servers that are distributed on the same cluster communicate through the local network, and with the servers on the remote cluster through an external network. The situation is the following: {code:java} Cluster 1 Zookeeper server 10 Zookeeper server 11 Zookeeper server 12 -> Leader Cluster 2 Zookeeper server 20 Zookeeper server 21 {code} Problem: We have an issue with Fast Leader Election when we kill the ZooKeeper leader process. After the leader (server 12) is killed and leader election starts, we can see in Zookeeper logs that voting notifications are exchanged from each Zookeeper server that remained alive towards all the others. Notification on Zookeeper servers located in the same cluster (communicating over the local network) are successfully exchanged. The problem seems to be for Zookeeper server sending votes over external network as according to the logs they are only sent in one direction. Logs from zookeeper server 10: {code:java} Nov 22 10:31:13 sc_2_1 BC[myid: 10]: INFO LOOKING Nov 22 10:31:13 sc_2_1 BC[myid: 10]: DEBUG Initializing leader election protocol... Nov 22 10:31:13 sc_2_1 BC[myid: 10]: DEBUG Updating proposal: 10 (newleader), 0xe9c97 (newzxid), 12 (oldleader), 0xd1380 (oldzxid) Nov 22 10:31:13 sc_2_1 BC[myid: 10]: INFO New election. My id = 10, proposed zxid=0xe9c97 Nov 22 10:31:13 sc_2_1 BC[myid: 10]: DEBUG Sending Notification: 10 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round), 20 (recipient), 10 (myid), 0xe (n.peerEpoch) Nov 22 10:31:13 sc_2_1 BC[myid: 10]: DEBUG Sending Notification: 10 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round), 21 (recipient), 10 (myid), 0xe (n.peerEpoch) Nov 22 10:31:13 sc_2_1 BC[myid: 10]: DEBUG Sending Notification: 10 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round), 10 (recipient), 10 (myid), 0xe (n.peerEpoch) Nov 22 10:31:13 sc_2_1 BC[myid: 10]: DEBUG Sending Notification: 10 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round), 11 (recipient), 10 (myid), 0xe (n.peerEpoch) Nov 22 10:31:13 sc_2_1 BC[myid: 10]: DEBUG Sending Notification: 10 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round), 12 (recipient), 10 (myid), 0xe (n.peerEpoch) Nov 22 10:31:13 sc_2_1 BC[myid: 10]: DEBUG Adding vote: from=10, proposed leader=10, proposed zxid=0xe9c97, proposed election epoch=0x11 Nov 22 10:31:14 sc_2_1 BC[myid: 10]: DEBUG Adding vote: from=11, proposed leader=11, proposed zxid=0xe9c97, proposed election epoch=0x11 Nov 22 10:31:14 sc_2_1 BC[myid: 10]: DEBUG Adding vote: from=10, proposed leader=10, proposed zxid=0xe9c97, proposed election epoch=0x11 Nov 22 10:31:14 sc_2_1 BC[myid: 10]: DEBUG Adding vote: from=10, proposed leader=11, proposed zxid=0xe9c97, proposed election epoch=0x11 Nov 22 10:31:14 sc_2_1 BC[myid: 10]: DEBUG Adding vote: from=10, proposed leader=11, proposed zxid=0xe9c97, proposed election epoch=0x11 Nov 22 10:31:14 sc_2_1 BC[myid: 10]: DEBUG Adding vote: from=11, proposed leader=11, proposed zxid=0xe9c97, proposed election epoch=0x11 Nov 22 10:31:15 sc_2_1 BC[myid: 10]: DEBUG Adding vote: from=10, proposed leader=11, proposed zxid=0xe9c97, proposed election epoch=0x11 Nov 22 10:31:15 sc_2_1 BC[myid: 10]: DEBUG Adding vote: from=11, proposed leader=11, proposed zxid=0xe9c97, proposed election epoch=0x11{code} Logs from zookeeper server 20: {code:java} Nov 22 10:31:13 sc_2_1 BC[myid: 20]: INFO LOOKING Nov 22 10:31:13 sc_2_1 BC[myid: 20]: DEBUG Initializing leader election protocol... Nov 22 10:31:13 sc_2_1 BC[myid: 20]: DEBUG Sending Notification: 20 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round), 20 (recipient), 20 (myid), 0xe (n.peerEpoch) Nov 22 10:31:13 sc_2_1 BC[myid: 20]: DEBUG Sending Notification: 20 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round), 21 (recipient), 20 (myid), 0xe (n.peerEpoch) Nov 22 10:31:13 sc_2_1 BC[myid: 20]: DEBUG Sending Notification: 20 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round), 10 (recipient), 20 (myid), 0xe (n.peerEpoch) Nov 22 10:31:13 sc_2_1 BC[myid: 20]: DEBUG Sending Notification: 20 (n.leader), 0xe9c97 (n.zxid), 0x11 (n.round),
[jira] [Created] (ZOOKEEPER-4766) Ensure leader election time does not unnecessarily scale with tree size due to snapshotting
Rishabh Rai created ZOOKEEPER-4766: -- Summary: Ensure leader election time does not unnecessarily scale with tree size due to snapshotting Key: ZOOKEEPER-4766 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4766 Project: ZooKeeper Issue Type: Improvement Components: leaderElection Affects Versions: 3.8.3, 3.5.9 Environment: General behavior, should occur in all environments Reporter: Rishabh Rai Fix For: 3.8.3, 3.5.9 Hi ZK community, this is regarding a fix for a behavior that is causing the leader election time to unnecessarily scale with the amount of data in the ZK data tree. *tl;dr:* During leader election, the leader always saves a snapshot when loading its data tree. This snapshot seems unnecessary, even in the case where the leader needs to send an updated SNAP to a learner, since it serializes the tree before sending anyway. Snapshotting slows down leader election and increases ZK downtime significantly as more data is stored in the tree. This improvement is to avoid taking a snapshot so that this unnecessary downtime is avoided. During leader election, when the [data is loaded|https://github.com/apache/zookeeper/blob/79f1f71a9a76689065c14d0846a69d0d71d3586e/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java#L601] by the tentatively elected (i.e. pre-finalized quorum) leader server, a [snapshot of the tree is always taken|https://github.com/apache/zookeeper/blob/79f1f71a9a76689065c14d0846a69d0d71d3586e/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java#L540]. The loadData method is called from multiple places, but specifically in the context of leader election, it seems like the snapshotting step is unnecessary for the leader when loading data: * Because it has loaded the tree at this point, we know that if the leader were to go down again, it would still be able to recover back to the current state at which we are snapshotting without using the snapshot that we are taking in loadData() * There are no ongoing transactions until leader election is completed and the ZK ensemble is back up, so no data would be lost after the point at which the data tree is loaded * Once the ensemble is healthy and the leader is handling transactions again, any new transactions are being logged and when needed the log is being rolled over when needed anyway, so if the leader is recovering from a failure, the snapshot taken during loadData() does not afford us any additional benefits over the initial snapshot (if it existed) and transaction log that the leader used to load its data from in loadData() * When the leader is deciding to send a SNAP or a DIFF to a learner, a [SNAP is serialized|https://github.com/apache/zookeeper/blob/79f1f71a9a76689065c14d0846a69d0d71d3586e/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java#L582] and sent [if and only if it is needed|https://github.com/apache/zookeeper/blob/79f1f71a9a76689065c14d0846a69d0d71d3586e/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java#L562]. The snapshot taken in loadData() again does not seem to be beneficial here. The PR for this fix only skips this snapshotting step in loadData() during leader election. The behavior of the function remains the same for other usages. With this change, during leader election the data tree would only be serialized when sending a SNAP to a learner. In other scenarios, no data tree serialization would be needed at all. In both cases, there is a significant in the time spent in leader election. If my understanding of any of this is incorrect, or if I'm failing to consider some other aspect of the process, please let me know. The PR for the change can also be changed to enable/disable this behavior via a java property. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (ZOOKEEPER-4620) zookeeper leader election time metric not reported correctly
Benton Liang created ZOOKEEPER-4620: --- Summary: zookeeper leader election time metric not reported correctly Key: ZOOKEEPER-4620 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4620 Project: ZooKeeper Issue Type: Bug Components: leaderElection, metric system Reporter: Benton Liang We added support to pro-actively give up leadership when a node is going down. With this the leader nominates the next leader before giving up leader ship. This avoids the leadership election. There is a bug where they calculate the election time. Normally when the leadership election starts, the start time is noted, and after the election the delta between the current time and startup time is used to determine the leadership time. But now since we are avoiding the leadership election, the start time is not updated this shows up as a big delta. example: `FOLLOWING - LEADER ELECTION TOOK - 1206674579 MS` -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (ZOOKEEPER-4502) SyncRequestProcessor leak when leader election occurred
xiongjianbo created ZOOKEEPER-4502: -- Summary: SyncRequestProcessor leak when leader election occurred Key: ZOOKEEPER-4502 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4502 Project: ZooKeeper Issue Type: Bug Components: quorum Affects Versions: 3.6.3 Reporter: xiongjianbo After a three-node ZooKeeper cluster runs for a period of time, OutOfMemory occurs. Check the memory stack and find that the number of SyncRequestProcessor objects is over 400. !image-2022-03-28-16-18-43-147.png! Check the previous modification records. It is found that the Learner.shutdown logic is modified this time. [https://github.com/apache/zookeeper/pull/1619] After the modification, the original invoking logic is changed. Before modification: Learner.shutdown() -> LearnerZooKeeperServer.shutdown() -> ZooKeeperServer.shutdown() After modification: Learner.shutdown() -> ZooKeeperServer.shutdown(boolean) Finally, LearnerZooKeeperServer.syncProcessor.shutdown() was never called. syncProcessor -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Created] (ZOOKEEPER-4316) Leader election fails due to SocketTimeoutException in QuorumCnxManager
Arun Subramanian R created ZOOKEEPER-4316: - Summary: Leader election fails due to SocketTimeoutException in QuorumCnxManager Key: ZOOKEEPER-4316 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4316 Project: ZooKeeper Issue Type: Bug Components: quorum Affects Versions: 3.5.7, 3.4.12 Environment: cat /etc/os-release NAME="SLES" VERSION="12-SP5" VERSION_ID="12.5" PRETTY_NAME="SUSE Linux Enterprise Server 12 SP5" ID="sles" ANSI_COLOR="0;32" CPE_NAME="cpe:/o:suse:sles:12:sp5" docker version Client: Version: 20.10.6-ce API version: 1.41 Go version: go1.13.15 Git commit: 8728dd246c3a Built: Tue Apr 27 09:45:18 2021 OS/Arch: linux/amd64 Context: default Experimental: true Server: Engine: Version: 20.10.6-ce API version: 1.41 (minimum version 1.12) Go version: go1.13.15 Git commit: 8728dd246c3a Built: Fri Apr 9 22:06:18 2021 OS/Arch: linux/amd64 Experimental: false containerd: Version: v1.4.4 GitCommit: 05f951a3781f4f2c1911b05e61c160e9c30eaa8e runc: Version: 1.0.0-rc93 GitCommit: 12644e614e25b05da6fd08a38ffa0cfe1903fdec docker-init: Version: 0.1.3_catatonit GitCommit: zookeeper version - 3.5.7 Reporter: Arun Subramanian R Attachments: docker-entrypoint.sh, zoo.cfg, zoo_3.5.7.yml I have a 3 node zookeeper cluster deployed as a stack using docker swarm. Deploying this stack causes zookeeper to fail with a SocketTimeoutException during leader election with the following log {noformat} 2021-06-11 03:59:34,607 [myid:2] - WARN [QuorumPeer[myid=2]/0.0.0.0:2181:QuorumCnxManager@584] - Cannot open channel to 3 at election address zoo3/10.0.11.5:3888 java.net.SocketTimeoutException: connect timed out 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.QuorumCnxManager.connectOne(QuorumCnxManager.java:558) at org.apache.zookeeper.server.quorum.QuorumCnxManager.connectAll(QuorumCnxManager.java:610) at org.apache.zookeeper.server.quorum.FastLeaderElection.lookForLeader(FastLeaderElection.java:838) at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:957){noformat} The docker overlay network itself appears to be sound. A netstat on one of the nodes outputs {noformat} bash-4.4# netstat -tuln Active Internet connections (only servers) Proto Recv-Q Send-Q Local Address Foreign Address State tcp0 0 0.0.0.0:21810.0.0.0:* LISTEN tcp0 0 0.0.0.0:38880.0.0.0:* LISTEN tcp0 0 0.0.0.0:42941 0.0.0.0:* LISTEN tcp0 0 127.0.0.11:354530.0.0.0:* LISTEN udp0 0 127.0.0.11:550090.0.0.0:*{noformat} showing the 3888 port is open. but a tcpdump only shows send and re-transmissions and there are no responses in port 3888. Suspecting the issue maybe due to a short timeout or small number of retries, I have tried increasing the cnxTimeout to 30 and electionPortBindRetry to 0 (infinite), but even after 13 hrs of continuous running and retrying election the same error persists I have attached the stack.yml, the custom docker-entrypoint.sh that we override on top of the official container to enable running from a root host user, and the zoo.cfg file from inside the container. Any help in identifying the underlying issue or mis-configuration, or any configuration parameter that may help solve the issue is deeply appreciated. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Created] (ZOOKEEPER-4220) Redundant connection attempts during leader election
Alex Mirgorodskiy created ZOOKEEPER-4220: Summary: Redundant connection attempts during leader election Key: ZOOKEEPER-4220 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4220 Project: ZooKeeper Issue Type: Bug Components: server Affects Versions: 3.5.5 Reporter: Alex Mirgorodskiy We've seen a few failures or long delays in electing a new leader when the previous one has a hard host reset (as opposed to just the service process down, since connections don't need to wait for timeout there). Symptoms are similar to https://issues.apache.org/jira/browse/ZOOKEEPER-2164. Reducing cnxTimeout from 5 to 1.5 seconds makes the problem much less frequent, but doesn't fix it completely. We are still using an old ZooKeeper version (3.5.5), and the new async connect feature will presumably avoid it. But we noticed a pattern of twice the expected number of connection attempts to the same downed instance in the log, and it appears to be due to a code glitch in QuorumCnxManager.java: {code:java} synchronized void connectOne(long sid) { ... if (lastCommittedView.containsKey(sid)) { knownId = true; if (connectOne(sid, lastCommittedView.get(sid).electionAddr)) return; } if (lastSeenQV != null && lastProposedView.containsKey(sid) && (!knownId || (lastProposedView.get(sid).electionAddr != < lastCommittedView.get(sid).electionAddr))) { knownId = true; if (connectOne(sid, lastProposedView.get(sid).electionAddr)) return; } {code} Comparing electionAddrs should be done with !equals presumably, otherwise connectOne will be invoked an extra time even in the common case when the addresses do match. The code around it has changed recently, but the check itself still exists at the top of master. It might not matter as much with the async connects, but perhaps it helps even then. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Created] (ZOOKEEPER-4185) Addition of new ZooTrace Mask LEADER_ELECTION_MASK and related log lines for better leader election telemetry
Julian Chow created ZOOKEEPER-4185: -- Summary: Addition of new ZooTrace Mask LEADER_ELECTION_MASK and related log lines for better leader election telemetry Key: ZOOKEEPER-4185 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4185 Project: ZooKeeper Issue Type: New Feature Components: leaderElection, metric system, server Affects Versions: 3.5.5 Reporter: Julian Chow Recently we have been seeing longer leader election abnormalities and we don't have visibility into "why" leader election took this long and "where exactly" it took long. As a result we cannot explain the long Zk service outages which we're seeing. To handle this I created a change that adds: -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Created] (ZOOKEEPER-4183) Leader election not working when using hostname in server config and hostname resolves to an internal IP addresses
Harald Musum created ZOOKEEPER-4183: --- Summary: Leader election not working when using hostname in server config and hostname resolves to an internal IP addresses Key: ZOOKEEPER-4183 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4183 Project: ZooKeeper Issue Type: Bug Components: server Reporter: Harald Musum We have a working setup with ZooKeeper 3.5.6 with 3 servers and server config similar to: server.1=1.foo.com:2182:2183:participant server.2=2.foo.com:2182:2183:participant server.3=3.foo.com:2182:2183:participan ZooKeeper servers are running in Docker containers and have IP addresses in the 10.x.x.x range, but also IP addresses from the default Docker network (172.17.x.x addresses) that are only usable inside the Docker container, /etc/hosts has e.g.: 172.17.2.192 1.foo.com When upgrading to 3.5.7 leader election failed with the following error in zookeeper log: .org.apache.zookeeper.server.quorum.QuorumCnxManager Received connection request 10.2.2.192:37028 .org.apache.zookeeper.server.quorum.UnifiedServerSocket Accepted TLS connection from /10.2.2.192:37028 - TLSv1.2 - TLS_ECDHE_RSA_WITH_AES_128_GCM_SH A256 .org.apache.zookeeper.server.quorum.QuorumCnxManager Cannot open channel to 0 at election address /172.17.2.192:2183 exception= java.net.NoRouteToHostException: No route to host (Host unreachable) at java.base/java.net.PlainSocketImpl.socketConnect(Native Method) at java.base/java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:399) at java.base/java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:242) at java.base/java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:224) at java.base/java.net.SocksSocketImpl.connect(SocksSocketImpl.java:403) at java.base/java.net.Socket.connect(Socket.java:609) at java.base/sun.security.ssl.SSLSocketImpl.connect(SSLSocketImpl.java:285) We tried using version 3.6.2, but had the same issue there. After reading the code and looking at changes between 3.5.6 and 3.5.7 I found https://issues.apache.org/jira/browse/ZOOKEEPER-3057 and the corresponding PR https://github.com/apache/zookeeper/pull/548 It seems like this PR changed the way election addresses are sent in the message when doing leader election, from using hostnames (when hostnames are specified in server config) to always using IP addresses. Patching 3.6.2 with the following change makes this work again for us: {code:java} diff --git a/zookeeper-server/zookeeper-server-3.6.2/src/main/java/org/apache/zookeeper/common/NetUtils.java b/zookeeper-server/zookeeper-server-3.6.2/src/main/java/org/apache/zookeeper/common/NetUtils.java index be8cb9a638..f32f1da7c8 100644 --- a/zookeeper-server/zookeeper-server-3.6.2/src/main/java/org/apache/zookeeper/common/NetUtils.java +++ b/zookeeper-server/zookeeper-server-3.6.2/src/main/java/org/apache/zookeeper/common/NetUtils.java @@ -27,13 +27,18 @@ import java.net.InetSocketAddress; */ public class NetUtils {+// Note: Changed from original to use hostname from InetSocketAddress if there exists one public static String formatInetAddr(InetSocketAddress addr) { +String hostName = addr.getHostName(); +if (hostName != null) { +return String.format("%s:%s", hostName, addr.getPort()); +} + InetAddress ia = addr.getAddress(); if (ia == null) { return String.format("%s:%s", addr.getHostString(), addr.getPort()); } if (ia instanceof Inet6Address) { return String.format("[%s]:%s", ia.getHostAddress(), addr.getPort()); } else {{code} -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Created] (ZOOKEEPER-3923) Leader election issues with Istio
Matteo Merli created ZOOKEEPER-3923: --- Summary: Leader election issues with Istio Key: ZOOKEEPER-3923 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3923 Project: ZooKeeper Issue Type: Bug Affects Versions: 3.5.8, 3.6.1 Reporter: Matteo Merli We have seen repeated occasion in which restarting the ZK leader node can lead to ~2min of quorum unavailability. This is easily reproducible on a cluster when leader election happens through a TCP proxying layer (such as Istio) in a Kubernetes environment. This happens > ~80% of the times on a 3 nodes cluster. After investigation, this is sequence of events: # zk-1 is the leader and gets restarted # zk-2 is voted as leader # zk-0 tries to connect to zk-2:2888 to sync-up. The TCP connection is established, though it's remotely closed immediately after # zk-2 starts listening to port 2888 but never receives any connection # zk-0, after the read error on the connection goes back into LOOKING mode, ready for a new leader election # zk-2 is still waiting for follower to sync-up, at waits until the timeout expires (eg: 30sec) after which it goes back into looking state. This sequence might get repeated several times until finally one leader election round can get through. h3. Logs excerpt: ZK-2 becomes leader: {noformat} 2020-08-27 16:40:07.216000+00:00 [INFO ] [che.zookeeper.server.quorum.Leader] LEADING - LEADER ELECTION TOOK - 214 MS 2020-08-27 16:40:07.218000+00:00 [INFO ] [zookeeper.server.quorum.QuorumPeer] Peer state changed: leading - discovery 2020-08-27 16:40:07.218000+00:00 [INFO ] [.server.persistence.FileTxnSnapLog] Snapshotting: 0xf0650 to /streamlio/zookeeper/data/version-2/snapshot.f0650 2020-08-27 16:40:07.249000+00:00 [INFO ] [e.zookeeper.server.ZooKeeperServer] Snapshot taken in 30 ms {noformat} ZK-0 is following but immediately goes back into LOOKING state: {noformat} 2020-08-27 16:40:07.207000+00:00 [INFO ] [he.zookeeper.server.quorum.Learner] FOLLOWING - LEADER ELECTION TOOK - 211 MS 2020-08-27 16:40:07.208000+00:00 [INFO ] [zookeeper.server.quorum.QuorumPeer] Peer state changed: following - discovery 2020-08-27 16:40:07.208000+00:00 [INFO ] [he.zookeeper.server.quorum.Learner] Successfully connected to leader, using address: zookeeper-zk35-2.zookeeper-zk35.pulsar-developers.svc.cluster.local/100.101.166.47:2888 2020-08-27 16:40:07.214000+00:00 [WARN ] [he.zookeeper.server.quorum.Learner] Exception when following the leader java.net.SocketException: Connection reset at java.net.SocketInputStream.read(SocketInputStream.java:186) ~[?:?] at java.net.SocketInputStream.read(SocketInputStream.java:140) ~[?:?] at java.io.BufferedInputStream.fill(BufferedInputStream.java:252) ~[?:?] at java.io.BufferedInputStream.read(BufferedInputStream.java:271) ~[?:?] at java.io.DataInputStream.readInt(DataInputStream.java:392) ~[?:?] at org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:96) at org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:86) at org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:134) at org.apache.zookeeper.server.quorum.Learner.readPacket(Learner.java:182) at org.apache.zookeeper.server.quorum.Learner.registerWithLeader(Learner.java:451) at org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:89) at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1458) 2020-08-27 16:40:07.215000+00:00 [INFO ] [he.zookeeper.server.quorum.Learner] Disconnected from leader (with address: zookeeper-zk35-2.zookeeper-zk35.pulsar-developers.svc.cluster.local/100.101.166.47:2888). Was connected for 6ms. Sync state: false 2020-08-27 16:40:07.215000+00:00 [INFO ] [he.zookeeper.server.quorum.Learner] shutdown Follower 2020-08-27 16:40:07.215000+00:00 [INFO ] [zookeeper.server.quorum.QuorumPeer] Peer state changed: looking 2020-08-27 16:40:07.215000+00:00 [WARN ] [zookeeper.server.quorum.QuorumPeer] PeerState set to LOOKING 2020-08-27 16:40:07.215000+00:00 [INFO ] [zookeeper.server.quorum.QuorumPeer] LOOKING 2020-08-27 16:40:07.215000+00:00 [INFO ] [r.server.quorum.FastLeaderElection] New election. My id = 1, proposed zxid=0xf0650 {noformat} After timeout, ZK-2 goes back into looking and a new leader election takes place: {noformat} 2020-08-27 16:40:27.251000+00:00 [WARN ] [zookeeper.server.quorum.QuorumPeer] Unexpected exception java.lang.InterruptedException: Timeout while waiting for epoch from quorum at org.apache.zookeeper.server.quorum.Leader.getEpochToPropose(Leader.java:1428) at org.apache.zookeeper.server.quorum.Leader.lead(Leader.java:599) at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1471) 2020-08-27
[jira] [Created] (ZOOKEEPER-3769) fast leader election does not end if leader is taken down
Lasaro Camargos created ZOOKEEPER-3769: -- Summary: fast leader election does not end if leader is taken down Key: ZOOKEEPER-3769 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3769 Project: ZooKeeper Issue Type: Bug Components: leaderElection Affects Versions: 3.5.7 Reporter: Lasaro Camargos In a cluster with three nodes, node3 is the leader and the other nodes are followers. If I stop node3, the other two nodes do not finish the leader election. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Created] (ZOOKEEPER-3757) Transaction log sync can take 20+ seconds after leader election when there is a large snapCount
Alex Kaiser created ZOOKEEPER-3757: -- Summary: Transaction log sync can take 20+ seconds after leader election when there is a large snapCount Key: ZOOKEEPER-3757 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3757 Project: ZooKeeper Issue Type: Bug Components: leaderElection Affects Versions: 3.5.6 Reporter: Alex Kaiser Short overview: If you have a large snapCount (we are using 10,000,000) you can end up with a very large transaction log (ours are between 1GB - 1.5 GB), which can cause the sync between a newly elected leader and it's followers to take 20+ seconds. This stems from the code (FileTxnIterator.getStorageSize()) in most cases returning 0 even if the transaction log is 1GB. Long Explanation: A few years ago we had some trouble with our zookeeper cluster having many shortish (100-500ms) pauses during our peak traffic times. These ended up resulting from the master taking a snap shot. To solve this we upped the snapCount to 10,000,000 so that we weren't taking snapshots nearly as often. We also made changes to reduce the size of our snapshots (from around 2.5 GB to ~500 MB). I don't remember what version of zookeeper we were using originally, but this was all working fine using 3.4.10, but we started to have problems when we upgraded to 3.5.6 around 3 months ago. We have a fairly high transaction rate and thus end up hitting the zxid overflow about once a month, which will cause a leader election. When we were on 3.4.10, this was fine because leader election and syncing would happen within 2-4 seconds, which was low enough for us to be able to basically ignore it. However after we upgraded to 3.5.6 the pauses we saw took between 15 - 30 seconds which were unacceptable for us. For now to solve this I set zookeeper.forceSnapshotSync=true (yes, I know the comments say this is only supposed to be used for testing), which causes syncing using snapshots (only 10-50 MB) instead of the transaction log (1-1.5 GB). Technical details: I tried taking a look at the code and I think I know why this happens. From what I learned, it looks like when a follower needs to sync with a leader on the leader LearnerHandler.syncFollower() gets called. It goes through a big if statement, but at one point it will call db.getProposalsFromTxnLog(peerLastZxid, sizeLimit). That peerLastZxid could be some very old zxid if the follower hadn't taken a snapshot in a long time (i.e. has a large snapCount) and the sizeLimit will generally be 0.33 * snapshot size (in my case around 10 MB). Inside of that getProposalsFromTxnLog it will create a TxnIterator and then call getStorageSize() on it. The problem comes from the fact that this call to getStorageSize() will usually return with 0. The reason that happens is because the FileTxnIterator class has a "current" log file that it is reading, this.logFile, and a list of files that it would still have to iterate through, this.storedFiles. The getStroageSize() function though only looks at the storedFiles list, so if the iterator has one large transaction log as the "current" log file and nothing in the storedFiles list, then this method will return 0 even though there is a huge transaction log to sync. One other side affect of this problem is that even bouncing a follower can cause long (5-10 second) pauses as the leader will hold a read lock on the transaction log while it syncs up with the follower. While I know what the problem is I don't know what the best solution is. I'm willing to work on the solution, but I would appreciate suggestions. One possible solution would be to include the this.logFile in the getStorageSize() calculation, however this could cause the estimate to over estimate the amount of data that is in the iterator (possibly by a lot) and I don't know what the consequences of doing that is. I'm not quite sure what is a good way to get an accurate estimate. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Created] (ZOOKEEPER-3537) Leader election - Use of out of election messages
Karolos Antoniadis created ZOOKEEPER-3537: - Summary: Leader election - Use of out of election messages Key: ZOOKEEPER-3537 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3537 Project: ZooKeeper Issue Type: Improvement Reporter: Karolos Antoniadis Assignee: Karolos Antoniadis Hello ZooKeeper developers, in {{lookForLeader}} in {{FastLeaderElection}} there is the following switch block in case a notification message {{n}} is received where {{n.state}} is either {{FOLLOWING}} or {{LEADING}} ([https://github.com/apache/zookeeper/blob/master/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L1029]). {code:java} case FOLLOWING: case LEADING: /* * Consider all notifications from the same epoch * together. */ if (n.electionEpoch == logicalclock.get()) { recvset.put(n.sid, new Vote(n.leader, n.zxid, n.electionEpoch, n.peerEpoch)); voteSet = getVoteTracker(recvset, new Vote(n.version, n.leader, n.zxid, n.electionEpoch, n.peerEpoch, n.state)); if (voteSet.hasAllQuorums() && checkLeader(outofelection, n.leader, n.electionEpoch)) { setPeerState(n.leader, voteSet); Vote endVote = new Vote(n.leader, n.zxid, n.electionEpoch, n.peerEpoch); leaveInstance(endVote); return endVote; } } /* * Before joining an established ensemble, verify that * a majority are following the same leader. */ outofelection.put(n.sid, new Vote(n.version, n.leader, n.zxid, n.electionEpoch, n.peerEpoch, n.state)); voteSet = getVoteTracker(outofelection, new Vote(n.version, n.leader, n.zxid, n.electionEpoch, n.peerEpoch, n.state)); if (voteSet.hasAllQuorums() && checkLeader(outofelection, n.leader, n.electionEpoch)) { synchronized (this) { logicalclock.set(n.electionEpoch); setPeerState(n.leader, voteSet); } Vote endVote = new Vote(n.leader, n.zxid, n.electionEpoch, n.peerEpoch); leaveInstance(endVote); return endVote; } break;{code} We notice that when {{n.electionEpoch == logicalclock.get()}}, votes are being added into {{recvset}}, however {{checkLeader}} is called immediately afterwards with the votes in {{outofelection}} as can be seen here ([https://github.com/apache/zookeeper/blob/master/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java#L1037]). Checking {{outofelection}} instead of {{recvset}} does not cause any problems. If {{checkLeader}} on {{outofelection}} fails, although it would have succeeded in {{recvset}}, {{checkLeader}} succeeds immediately afterwards when the vote is added in {{outofelection}}. Still, it seems natural to check for a leader in {{recvSet}} and not in {{outofelection}}. Cheers, Karolos -- This message was sent by Atlassian Jira (v8.3.2#803003)
[jira] [Created] (ZOOKEEPER-3479) Logging false leader election times
Karolos Antoniadis created ZOOKEEPER-3479: - Summary: Logging false leader election times Key: ZOOKEEPER-3479 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3479 Project: ZooKeeper Issue Type: Bug Components: leaderElection Affects Versions: 3.5.5 Reporter: Karolos Antoniadis Attachments: server1.txt, server2.txt There seems to be a problem with the logging of leader election times: the logged times are much smaller than the actual time it took for the leader election to complete. This bug can be easily reproduced by following these steps: 1) Run a ZK cluster of 3 servers 2) Kill the server that is currently the leader 3) The log files of the remaining 2 servers contain false leader election times In the attached files you can see the log files of the remaining 2 serve. For brevity, I removed the parts before and after the leader election from the log files. For example, in {{server1.txt}} we can see that: {code:java} 2019-07-31 00:57:31,852 [myid:1] - WARN [QuorumPeer[myid=1](plain=/0.0.0.0:2791)(secure=disabled):QuorumPeer@1318] - PeerState set to LOOKING 2019-07-31 00:57:31,853 [myid:1] - INFO [QuorumPeer[myid=1](plain=/0.0.0.0:2791)(secure=disabled):QuorumPeer@1193] - LOOKING 2019-07-31 00:57:31,853 [myid:1] - INFO [QuorumPeer[myid=1](plain=/0.0.0.0:2791)(secure=disabled):FastLeaderElection@885] - New election. My id = 1, proposed zxid=0x10001 [...] 2019-07-31 00:57:32,272 [myid:1] - INFO [QuorumPeer[myid=1](plain=/0.0.0.0:2791)(secure=disabled):Follower@69] - FOLLOWING - LEADER ELECTION TOOK - 1 MS{code} Leader election supposedly took only 1ms, but in reality it took (32,272 - 31,853) = 419ms! The reason for this bug seems to be the introduction of this line {code:java} start_fle = Time.currentElapsedTime();{code} (seen here [https://github.com/apache/zookeeper/blob/master/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java#L1402]) back in this commit [https://github.com/apache/zookeeper/commit/5428cd4bc963c2e653a260c458a8a8edf3fa08ef]. -- This message was sent by Atlassian JIRA (v7.6.14#76016)
Re: Leader election logging during reconfiguration
>> we should measure the total time more accurately +1 - it would be good to have a new metric to measure reconfiguration time, and leaving existing LE time metric dedicated to measure the conventional FLE time. Mixing both (as of today) will provide some confusing insights on how long the conventional FLE actually took. On Mon, Jul 29, 2019 at 7:13 PM Alexander Shraer wrote: > Please see comments inline. > > Thanks, > Alex > > On Mon, Jul 29, 2019 at 5:29 PM Karolos Antoniadis > wrote: > > > Hi ZooKeeper developers, > > > > ZooKeeper seems to be logging a "*LEADER ELECTION TOOK*" message even > > though no leader election takes place during a reconfiguration. > > > > This can be reproduced by following these steps: > > 1) start a ZooKeeper cluster (e.g., 3 participants) > > 2) start a client that connects to some follower > > 3) perform a *reconfig* operation that removes the leader from the > cluster > > > > After the reconfiguration takes place, we can see that the log files of > the > > remaining participants contain a "*LEADER ELECTION TOOK*" message. For > > example, a line that contains > > > > *2019-07-29 23:07:38,518 [myid:2] - INFO > > [QuorumPeer[myid=2](plain=0.0.0.0:2792)(secure=disabled):Follower@75] - > > FOLLOWING - LEADER ELECTION TOOK - 57 MS* > > > > However, no leader election took place. With that, I mean that no server > > went *LOOKING *and then started voting and sending notifications to other > > participants as would be in a normal leader election. > > It seems, that before the *reconfig *is committed, the participant that > is > > going to be the next leader is already decided (see here: > > > > > https://github.com/apache/zookeeper/blob/master/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java#L865 > > ). > > > > I think the above issue raises the following questions: > > - Should we avoid logging LEADER ELECTION messages altogether in such > > cases? > > > > In the specific scenario you described, the leader has changed, but our > heuristic for choosing the leader apparently worked and a new leader could > be elected without running the full election. > Notice that we could be unlucky and the designated leader could be offline, > and then we'll fall back on election. It's useful to know how much time it > takes to start following the new leader. > > > > - Or, should there be some logging for the time it took for the > > reconfiguration (e.g., the time between a participant gets a *reconfig* > > operation till the operation is committed)? Would such a time value be > > useful? > > > > IIRC the LEADER ELECTION message is used for this purpose. if you just look > on the time to commit the reconfig operation, you won't > account for the work that happens when the commit message is received, such > as leader re-election, role-change (follower->observer conversion and such) > etc which is what takes most of the time. > Committing a reconfig operation is usually not much more expensive than > committing a normal operation. But perhaps you're right that we should > measure the total time more accurately. Would you > like to open a Jira and perhaps take a stab at improving this ? > > > > > Best, > > Karolos > > >
Re: Leader election logging during reconfiguration
Please see comments inline. Thanks, Alex On Mon, Jul 29, 2019 at 5:29 PM Karolos Antoniadis wrote: > Hi ZooKeeper developers, > > ZooKeeper seems to be logging a "*LEADER ELECTION TOOK*" message even > though no leader election takes place during a reconfiguration. > > This can be reproduced by following these steps: > 1) start a ZooKeeper cluster (e.g., 3 participants) > 2) start a client that connects to some follower > 3) perform a *reconfig* operation that removes the leader from the cluster > > After the reconfiguration takes place, we can see that the log files of the > remaining participants contain a "*LEADER ELECTION TOOK*" message. For > example, a line that contains > > *2019-07-29 23:07:38,518 [myid:2] - INFO > [QuorumPeer[myid=2](plain=0.0.0.0:2792)(secure=disabled):Follower@75] - > FOLLOWING - LEADER ELECTION TOOK - 57 MS* > > However, no leader election took place. With that, I mean that no server > went *LOOKING *and then started voting and sending notifications to other > participants as would be in a normal leader election. > It seems, that before the *reconfig *is committed, the participant that is > going to be the next leader is already decided (see here: > > https://github.com/apache/zookeeper/blob/master/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java#L865 > ). > > I think the above issue raises the following questions: > - Should we avoid logging LEADER ELECTION messages altogether in such > cases? > In the specific scenario you described, the leader has changed, but our heuristic for choosing the leader apparently worked and a new leader could be elected without running the full election. Notice that we could be unlucky and the designated leader could be offline, and then we'll fall back on election. It's useful to know how much time it takes to start following the new leader. > - Or, should there be some logging for the time it took for the > reconfiguration (e.g., the time between a participant gets a *reconfig* > operation till the operation is committed)? Would such a time value be > useful? > IIRC the LEADER ELECTION message is used for this purpose. if you just look on the time to commit the reconfig operation, you won't account for the work that happens when the commit message is received, such as leader re-election, role-change (follower->observer conversion and such) etc which is what takes most of the time. Committing a reconfig operation is usually not much more expensive than committing a normal operation. But perhaps you're right that we should measure the total time more accurately. Would you like to open a Jira and perhaps take a stab at improving this ? > > Best, > Karolos >
Leader election logging during reconfiguration
Hi ZooKeeper developers, ZooKeeper seems to be logging a "*LEADER ELECTION TOOK*" message even though no leader election takes place during a reconfiguration. This can be reproduced by following these steps: 1) start a ZooKeeper cluster (e.g., 3 participants) 2) start a client that connects to some follower 3) perform a *reconfig* operation that removes the leader from the cluster After the reconfiguration takes place, we can see that the log files of the remaining participants contain a "*LEADER ELECTION TOOK*" message. For example, a line that contains *2019-07-29 23:07:38,518 [myid:2] - INFO [QuorumPeer[myid=2](plain=0.0.0.0:2792)(secure=disabled):Follower@75] - FOLLOWING - LEADER ELECTION TOOK - 57 MS* However, no leader election took place. With that, I mean that no server went *LOOKING *and then started voting and sending notifications to other participants as would be in a normal leader election. It seems, that before the *reconfig *is committed, the participant that is going to be the next leader is already decided (see here: https://github.com/apache/zookeeper/blob/master/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java#L865 ). I think the above issue raises the following questions: - Should we avoid logging LEADER ELECTION messages altogether in such cases? - Or, should there be some logging for the time it took for the reconfiguration (e.g., the time between a participant gets a *reconfig* operation till the operation is committed)? Would such a time value be useful? Best, Karolos
[jira] [Created] (ZOOKEEPER-3456) Service temporarily unavailable due to an ongoing leader election. Please refresh
Marzieh created ZOOKEEPER-3456: -- Summary: Service temporarily unavailable due to an ongoing leader election. Please refresh Key: ZOOKEEPER-3456 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3456 Project: ZooKeeper Issue Type: Bug Components: server Environment: docker container with Ubuntu 16.04 Reporter: Marzieh Fix For: 3.4.14 Hi I configured Zookeeper with four nodes for my Mesos cluster with Marathon. When I ran Flink Json file on Marathon, it was run without problem. But, when I entered IP of my two slaves, just one slave shew Flink UI and another slave shew this error: Service temporarily unavailable due to an ongoing leader election. Please refresh I checked "zookeeper.out" file and it said that : 019-07-07 11:48:43,412 [myid:] - INFO [main:QuorumPeerConfig@136] - Reading configuration from: /home/zookeeper-3.4.14/bin/../conf/zoo.cfg 2019-07-07 11:48:43,421 [myid:] - INFO [main:QuorumPeer$QuorumServer@185] - Resolved hostname: 0.0.0.0 to address: /0.0.0.0 2019-07-07 11:48:43,421 [myid:] - INFO [main:QuorumPeer$QuorumServer@185] - Resolved hostname: 10.32.0.3 to address: /10.32.0.3 2019-07-07 11:48:43,422 [myid:] - INFO [main:QuorumPeer$QuorumServer@185] - Resolved hostname: 10.32.0.2 to address: /10.32.0.2 2019-07-07 11:48:43,422 [myid:] - INFO [main:QuorumPeer$QuorumServer@185] - Resolved hostname: 10.32.0.5 to address: /10.32.0.5 2019-07-07 11:48:43,422 [myid:] - WARN [main:QuorumPeerConfig@354] - Non-optimial configuration, consider an odd number of servers. 2019-07-07 11:48:43,422 [myid:] - INFO [main:QuorumPeerConfig@398] - Defaulting to majority quorums 2019-07-07 11:48:43,425 [myid:3] - INFO [main:DatadirCleanupManager@78] - autopurge.snapRetainCount set to 3 2019-07-07 11:48:43,425 [myid:3] - INFO [main:DatadirCleanupManager@79] - autopurge.purgeInterval set to 0 2019-07-07 11:48:43,425 [myid:3] - INFO [main:DatadirCleanupManager@101] - Purge task is not scheduled. 2019-07-07 11:48:43,432 [myid:3] - INFO [main:QuorumPeerMain@130] - Starting quorum peer 2019-07-07 11:48:43,437 [myid:3] - INFO [main:ServerCnxnFactory@117] - Using org.apache.zookeeper.server.NIOServerCnxnFactory as server connect$ 2019-07-07 11:48:43,439 [myid:3] - INFO [main:NIOServerCnxnFactory@89] - binding to port 0.0.0.0/0.0.0.0:2181 2019-07-07 11:48:43,440 [myid:3] - ERROR [main:QuorumPeerMain@92] - Unexpected exception, exiting abnormally java.net.BindException: Address already in use at sun.nio.ch.Net.bind0(Native Method) at sun.nio.ch.Net.bind(Net.java:433) at sun.nio.ch.Net.bind(Net.java:425) at sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:223) at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:74) at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:67) at org.apache.zookeeper.server.NIOServerCnxnFactory.configure(NIOServerCnxnFactory.java:90) at org.apache.zookeeper.server.quorum.QuorumPeerMain.runFromConfig(QuorumPeerMain.java:133) at org.apache.zookeeper.server.quorum.QuorumPeerMain.initializeAndRun(QuorumPeerMain.java:114) at org.apache.zookeeper.server.quorum.QuorumPeerMain.main(QuorumPeerMain.java:81) I searched a lot and could not find the solution. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3336) Leader election terminated, two leaders or not following leader or not having state
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16837129#comment-16837129 ] Simin Oraee commented on ZOOKEEPER-3336: Well, in our testing we started each of the leader election from the beginning. I mean it is not an infinite sequence of message passing and I just provide 3 slices of this infinite sequence. Therefore each case is just one leader election protocol which starts with 3 nodes and finishes when there was no enabled message. Therefore in case 2, there is no previous leader. > Leader election terminated, two leaders or not following leader or not having > state > --- > > Key: ZOOKEEPER-3336 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3336 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.13 > Environment: Debian, Java 8 >Reporter: Simin Oraee >Priority: Major > Attachments: conf, zookeeper.log > > > I am working on a testing tool for distributed systems. I tested Zookeeper, > enforcing different possible orderings of events. I encountered some > inconsistencies in the election of the leader. Here are the logs of 3 > completed executions. > I am wondering if these behaviors are expected or not. > 1) More than one node consider themselves leaders: > NodeCrashEvent\{id=1, nodeId=0} > NodeStartEvent\{id=7, nodeId=0} > MessageEvent\{id=8, predecessors=[7], from=0, to=0, leader=0, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=9, predecessors=[8, 7], from=0, to=1, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=10, predecessors=[9, 7], from=0, to=2, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=5, predecessors=[], from=1, to=0, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=12, predecessors=[5, 10, 7], from=0, to=0, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=13, predecessors=[12, 5, 7], from=0, to=1, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=14, predecessors=[5, 13, 7], from=0, to=2, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=11, predecessors=[5], from=1, to=1, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=15, predecessors=[11], from=1, to=2, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=6, predecessors=[], from=2, to=0, leader=2, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > Node 1 state: LEADING > Node 1 final vote: Vote\{leader=1, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=17, predecessors=[6, 14, 7], from=0, to=0, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=18, predecessors=[17, 6, 7], from=0, to=1, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=19, predecessors=[18, 6, 7], from=0, to=2, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=20, predecessors=[18], from=1, to=0, leader=1, > state=LEADING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=16, predecessors=[6], from=2, to=1, leader=2, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=22, predecessors=[16, 20], from=1, to=2, leader=1, > state=LEADING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=21, predecessors=[16], from=2, to=2, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > Node 0 state: FOLLOWING > Node 0 final vote: Vote\{leader=2, zxid=0, electionEpoch=1, peerEpoch=0} > Node 2 state: LEADING > Node 2 final vote: Vote\{leader=2, zxid=0, electionEpoch=1, peerEpoch=0} > 2) There are some nodes that follow nodes other than the leaders: > NodeCrashEvent\{id=1, nodeId=0} > NodeStartEvent\{id=7, nodeId=0} > MessageEvent\{id=8, predecessors=[7], from=0, to=0, leader=0, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=9, predecessors=[8, 7], from=0, to=1, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=10, predecessors=[9, 7], from=0, to=2, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=5, predecessors=[], from=1, to=0, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=12, predecessors=[5, 10, 7], from=0, to=0, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peer
[jira] [Commented] (ZOOKEEPER-3336) Leader election terminated, two leaders or not following leader or not having state
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16836921#comment-16836921 ] Fangmin Lv commented on ZOOKEEPER-3336: --- [~NIWIS] for 2) is it following the node previously it was a leader? We saw some cases that if the follower takes sometime to shutdown, before it goes to LOOKING it will report it's still following the previous leader. For 3) I can think of scenarios like 0 partitioned away, 1, 2, 3 votes for 3, then 3 goes to leading, 1 following it, but before 2 changed it's state to following, it received vote from 4, and changed mind to vote for 4, so 3 and 4 will keep voting 4 and in LOOKING state. > Leader election terminated, two leaders or not following leader or not having > state > --- > > Key: ZOOKEEPER-3336 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3336 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.13 > Environment: Debian, Java 8 >Reporter: Simin Oraee >Priority: Major > Attachments: conf, zookeeper.log > > > I am working on a testing tool for distributed systems. I tested Zookeeper, > enforcing different possible orderings of events. I encountered some > inconsistencies in the election of the leader. Here are the logs of 3 > completed executions. > I am wondering if these behaviors are expected or not. > 1) More than one node consider themselves leaders: > NodeCrashEvent\{id=1, nodeId=0} > NodeStartEvent\{id=7, nodeId=0} > MessageEvent\{id=8, predecessors=[7], from=0, to=0, leader=0, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=9, predecessors=[8, 7], from=0, to=1, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=10, predecessors=[9, 7], from=0, to=2, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=5, predecessors=[], from=1, to=0, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=12, predecessors=[5, 10, 7], from=0, to=0, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=13, predecessors=[12, 5, 7], from=0, to=1, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=14, predecessors=[5, 13, 7], from=0, to=2, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=11, predecessors=[5], from=1, to=1, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=15, predecessors=[11], from=1, to=2, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=6, predecessors=[], from=2, to=0, leader=2, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > Node 1 state: LEADING > Node 1 final vote: Vote\{leader=1, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=17, predecessors=[6, 14, 7], from=0, to=0, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=18, predecessors=[17, 6, 7], from=0, to=1, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=19, predecessors=[18, 6, 7], from=0, to=2, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=20, predecessors=[18], from=1, to=0, leader=1, > state=LEADING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=16, predecessors=[6], from=2, to=1, leader=2, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=22, predecessors=[16, 20], from=1, to=2, leader=1, > state=LEADING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=21, predecessors=[16], from=2, to=2, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > Node 0 state: FOLLOWING > Node 0 final vote: Vote\{leader=2, zxid=0, electionEpoch=1, peerEpoch=0} > Node 2 state: LEADING > Node 2 final vote: Vote\{leader=2, zxid=0, electionEpoch=1, peerEpoch=0} > 2) There are some nodes that follow nodes other than the leaders: > NodeCrashEvent\{id=1, nodeId=0} > NodeStartEvent\{id=7, nodeId=0} > MessageEvent\{id=8, predecessors=[7], from=0, to=0, leader=0, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=9, predecessors=[8, 7], from=0, to=1, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=10, predecessors=[9, 7], from=0, to=2, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=5, predecessors=[], from=1, to=0, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0}
[GitHub] [zookeeper] Lagrang edited a comment on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang edited a comment on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477946006 Zookeeper binds to LE port on start, if you can't bind to it, I think fail-fast is more reliable behavior than "silently" catch error. From monitoring point of view: process alive, port is bind(but by someone else). Also, not so very good:) In case of DNS issue in Kubernetes: I start a 3 ZK nodes, all containers in cluster alive, but 2 nodes stop listen LE port. Quorum not reached. But, I found another case when `QuorumCnxManager.Listener` can stop process: we can fail while accepting new connections. I agree that this case should not stop a process. But errors while connection accepting can be handled separately. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477946006 Zookeeper binds to LE port on start, if you can't bind to it, I think fail-fast is more reliable behavior than "silently" catch error. From monitoring point of view: process alive, port is bind(but by someone else). Also, not so very good:) In case of DNS issue in Kubernetes: I start a 3 ZK nodes, all containers in cluster alive, but 2 nodes stop listen LE port. Quorum not reached. But, I found another case when `QuorumCnxManager.Listener` can stop process: we can fail while accepting new connections. I agree that this case should not stop a process. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[jira] [Commented] (ZOOKEEPER-3336) Leader election terminated, two leaders or not following leader or not having state
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16804683#comment-16804683 ] Simin Oraee commented on ZOOKEEPER-3336: [~lvfangmin] Thank you so much for your fast answer and I appreciate your time. Then what about the other two conditions I mentioned? 2) There are some nodes that follow nodes other than the leaders. 3) There are some nodes that neither following nor leading > Leader election terminated, two leaders or not following leader or not having > state > --- > > Key: ZOOKEEPER-3336 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3336 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.13 > Environment: Debian, Java 8 >Reporter: Simin Oraee >Priority: Major > Attachments: conf, zookeeper.log > > > I am working on a testing tool for distributed systems. I tested Zookeeper, > enforcing different possible orderings of events. I encountered some > inconsistencies in the election of the leader. Here are the logs of 3 > completed executions. > I am wondering if these behaviors are expected or not. > 1) More than one node consider themselves leaders: > NodeCrashEvent\{id=1, nodeId=0} > NodeStartEvent\{id=7, nodeId=0} > MessageEvent\{id=8, predecessors=[7], from=0, to=0, leader=0, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=9, predecessors=[8, 7], from=0, to=1, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=10, predecessors=[9, 7], from=0, to=2, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=5, predecessors=[], from=1, to=0, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=12, predecessors=[5, 10, 7], from=0, to=0, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=13, predecessors=[12, 5, 7], from=0, to=1, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=14, predecessors=[5, 13, 7], from=0, to=2, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=11, predecessors=[5], from=1, to=1, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=15, predecessors=[11], from=1, to=2, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=6, predecessors=[], from=2, to=0, leader=2, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > Node 1 state: LEADING > Node 1 final vote: Vote\{leader=1, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=17, predecessors=[6, 14, 7], from=0, to=0, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=18, predecessors=[17, 6, 7], from=0, to=1, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=19, predecessors=[18, 6, 7], from=0, to=2, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=20, predecessors=[18], from=1, to=0, leader=1, > state=LEADING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=16, predecessors=[6], from=2, to=1, leader=2, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=22, predecessors=[16, 20], from=1, to=2, leader=1, > state=LEADING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=21, predecessors=[16], from=2, to=2, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > Node 0 state: FOLLOWING > Node 0 final vote: Vote\{leader=2, zxid=0, electionEpoch=1, peerEpoch=0} > Node 2 state: LEADING > Node 2 final vote: Vote\{leader=2, zxid=0, electionEpoch=1, peerEpoch=0} > 2) There are some nodes that follow nodes other than the leaders: > NodeCrashEvent\{id=1, nodeId=0} > NodeStartEvent\{id=7, nodeId=0} > MessageEvent\{id=8, predecessors=[7], from=0, to=0, leader=0, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=9, predecessors=[8, 7], from=0, to=1, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=10, predecessors=[9, 7], from=0, to=2, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=5, predecessors=[], from=1, to=0, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=12, predecessors=[5, 10, 7], from=0, to=0, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=13, predecessors=[12, 5, 7], from=0, to=1, leader=1, > state=LOOKING, zxid=0, elect
[GitHub] [zookeeper] maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477893575 After extending the `ZooKeeperCriticalThread`, When `QuorumCnxManager.Listener `had failed in listening to the LE port(e.g that port had already been used),then ZooKeeperServer will shutdown and exit. this suitation which `QuorumCnxManager.Listener` had exited is not a fatal one,even though she cannot take part in the new leader election), the zkServer can still take server for read/write request. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477882906 @maoling Can you please describe your concerns? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477839636 I have reservations about extending `QuorumCnxManager.Listener `to the `ZooKeeperCriticalThread` This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477804567 Good find @maoling ! I don't see any discussion in ZOOKEEPER-602 about criteria for what should and should not be a critical thread - I'm curious on how those decisions were made. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[jira] [Commented] (ZOOKEEPER-3336) Leader election terminated, two leaders or not following leader or not having state
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16804129#comment-16804129 ] Fangmin Lv commented on ZOOKEEPER-3336: --- [~NIWIS] the FastLeaderElection used currently allow to elect two leaders, but only one of them will be activated with majority, so it won't cause the split brain. Here is the simple scenario which could cause multiple leader: Let's say we have 5 nodes ensemble: # node 1, 2 were stopped at the beginning # node 3, 4, 5 started new round of election, and 5 was elected as leader and goes to waiting epoch from followers # node 3 is stopped before it following 5, so 4 is following 5, 5 is waiting for another node to join before it's activating the leadership and goes to broadcast # then node 1, 2, 3 restarted, and started a new round of leader election # node 4 and 5 mentioned 5 is leader, but 1, 2, 3 only following 5 when there is another majority confirmed it, which is not # node 1, 2, 3 voted for 3, 3 gets majority so it's start leading, meanwhile 5 is still waiting for another peer to join before it's timed out > Leader election terminated, two leaders or not following leader or not having > state > --- > > Key: ZOOKEEPER-3336 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3336 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.13 > Environment: Debian, Java 8 >Reporter: Simin Oraee >Priority: Major > Attachments: conf, zookeeper.log > > > I am working on a testing tool for distributed systems. I tested Zookeeper, > enforcing different possible orderings of events. I encountered some > inconsistencies in the election of the leader. Here are the logs of 3 > completed executions. > I am wondering if these behaviors are expected or not. > 1) More than one node consider themselves leaders: > NodeCrashEvent\{id=1, nodeId=0} > NodeStartEvent\{id=7, nodeId=0} > MessageEvent\{id=8, predecessors=[7], from=0, to=0, leader=0, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=9, predecessors=[8, 7], from=0, to=1, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=10, predecessors=[9, 7], from=0, to=2, leader=0, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=5, predecessors=[], from=1, to=0, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=12, predecessors=[5, 10, 7], from=0, to=0, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=13, predecessors=[12, 5, 7], from=0, to=1, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=14, predecessors=[5, 13, 7], from=0, to=2, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=11, predecessors=[5], from=1, to=1, leader=1, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=15, predecessors=[11], from=1, to=2, leader=1, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=6, predecessors=[], from=2, to=0, leader=2, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > Node 1 state: LEADING > Node 1 final vote: Vote\{leader=1, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=17, predecessors=[6, 14, 7], from=0, to=0, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=18, predecessors=[17, 6, 7], from=0, to=1, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=19, predecessors=[18, 6, 7], from=0, to=2, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=20, predecessors=[18], from=1, to=0, leader=1, > state=LEADING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=16, predecessors=[6], from=2, to=1, leader=2, state=LOOKING, > zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=22, predecessors=[16, 20], from=1, to=2, leader=1, > state=LEADING, zxid=0, electionEpoch=1, peerEpoch=0} > MessageEvent\{id=21, predecessors=[16], from=2, to=2, leader=2, > state=LOOKING, zxid=0, electionEpoch=1, peerEpoch=0} > Node 0 state: FOLLOWING > Node 0 final vote: Vote\{leader=2, zxid=0, electionEpoch=1, peerEpoch=0} > Node 2 state: LEADING > Node 2 final vote: Vote\{leader=2, zxid=0, electionEpoch=1, peerEpoch=0} > 2) There are some nodes that follow nodes other than the leaders: > NodeCrashEvent\{id=1, nodeId=0} > NodeStartEvent\{id=7, nodeId=0} > MessageEvent\{id=8, predecessors=[7], from=0, to=0, leader=0, state=LOOK
[GitHub] [zookeeper] maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477471802 - There was a [sum-up](https://issues.apache.org/jira/browse/ZOOKEEPER-602?focusedCommentId=13117887=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-13117887) about which class should extend the ZooKeeperCriticalThread - After extending the `ZooKeeperCriticalThread`, When Listen-Thread had failed in listening to the LE port(a fatal one?), then ZooKeeperServer will shutdown and exit. Is it the behaviour what we want? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477303890 Thanks for the new test. :) This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang edited a comment on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang edited a comment on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477172489 Push changes which extend `QuorumCnxManager.Listener` from `ZookeeperCriticalThread`. Add test into CnxManagerTest to check configurable retries and shutdown of quorum peer after max retries. Also add call to `shutdown` method in finally block of `QuorumPeer.run()`, because I didn't find place where resources closed(connections, ZK DB, etc) on quorum peer exit. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-477172489 Push changes to extends QuorumCnxManager.Listener from ZookeeperCriticalThread. Plus test into CnxManagerTest to check configurable retries and shutdown of quorum peer after max retries. Also add call to `shutdown` method in finally block of QuorumPeer.run(), because I didn't find place where resources closed(connections, ZK DB, etc) on quorum peer exit. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-476324387 Thanks for adding docs! I'm wary of adding an infinite loop for the same reasons that I'd agree with this becoming a ZooKeeperCriticalThread. `QuorumCnxManager.Listener` is only instantiated once in the lifetime of a `QuorumPeer` and it doesn't seem good to allow it to fail silently - cutting the peer off from the election proceeds for the rest of its life. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-476284089 Add doc for introduced property to `zookeeperAdmin.md` file, `Cluster Options` section. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-475969622 > An infinite loop is a better option, especially for the availability? because the container will be self-healing just a few minutes after its DNS is unresolvable, we cannot have a good evaluation of the retry times. I proposed two solutions in [ZOOKEEPER-3320](https://issues.apache.org/jira/projects/ZOOKEEPER/issues/ZOOKEEPER-3320), one is configurable retry, other is to remove retries at all. @enixon noted that such kind of error happens mostly in container environment. I agree with him, in non-container environment port binding can fail, for instance, when port already in use and indefinitely retry has no meaning. But from another point of view, today QuorumCnxManager.Listener extends `ZooKeeperThread`and if ZK server can't bind election port, it stop trying and process remains alive(and mostly useless at this point). Maybe `QuorumCnxManager.Listener` should extends `ZookeeperCriticalThread`? This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-475965281 @Lagrang - you also need to doc this property in the `zookeeperAdmin.md` - Look back, this retry logic was firstly introduced by [ZOOKEEPER-512 ](https://issues.apache.org/jira/browse/ZOOKEEPER-512) > This is for retrying. If there is a problem while listening or trying to bind to the socket, it tries again and gives up after 3 consecutive attempts. - An infinite loop is a better option, especially for the availability? because the container will be self-healing just a few minutes after its DNS is unresolvable, we cannot have a good evaluation of the retry times. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-475885134 retest maven build This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] eolivelli commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
eolivelli commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-475788393 retest maven build This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] eolivelli commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
eolivelli commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-475767323 Even Travis This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-475741434 Some thing is wrong with JenkinsMaven here. :( This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang opened a new pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang opened a new pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863 This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang closed pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang closed pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863 This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang closed pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang closed pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863 This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang opened a new pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang opened a new pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863 This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
enixon commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-475707431 LGTM! The test failure (NonRecoverableErrorTest::testZooKeeperServiceAvailableOnLeader) looks unrelated to me - if you close and reopen the pull request then it will be re-run. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
maoling commented on issue #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#issuecomment-475644114 @Lagrang if you're interested in the same issue, you can step into [this ](https://github.com/etcd-io/etcd/issues/10575) This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] Lagrang commented on a change in pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang commented on a change in pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#discussion_r268053495 ## File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java ## @@ -839,12 +839,17 @@ private void resetConnectionThreadCount() { */ public class Listener extends ZooKeeperThread { +private final int portBindMaxRetry; volatile ServerSocket ss = null; public Listener() { // During startup of thread, thread name will be overridden to // specific election address super("ListenerThread"); + +// maximum retry count while trying to bind to election port +// see ZOOKEEPER-3320 for more details +portBindMaxRetry = Integer.getInteger("zookeeper.electionPortBindRetry", 3); Review comment: Add logs and validation This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [zookeeper] enixon commented on a change in pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
enixon commented on a change in pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863#discussion_r267946154 ## File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java ## @@ -839,12 +839,17 @@ private void resetConnectionThreadCount() { */ public class Listener extends ZooKeeperThread { +private final int portBindMaxRetry; volatile ServerSocket ss = null; public Listener() { // During startup of thread, thread name will be overridden to // specific election address super("ListenerThread"); + +// maximum retry count while trying to bind to election port +// see ZOOKEEPER-3320 for more details +portBindMaxRetry = Integer.getInteger("zookeeper.electionPortBindRetry", 3); Review comment: I'd recommend adding an info log that says what this property is set to when it reads and some validation in place to make sure the property is not negative. Overall change seems reasonable - let's see what the others think. This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[jira] [Commented] (ZOOKEEPER-3320) Leader election port stop listen when hostname unresolvable for some time
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16797880#comment-16797880 ] Igor Skokov commented on ZOOKEEPER-3320: I attach link to pull request. It based on branch-3.5, but as far as a know, this error can happen on master branch. If needed, I can create another pull request for master. > Leader election port stop listen when hostname unresolvable for some time > -- > > Key: ZOOKEEPER-3320 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3320 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.10, 3.5.4 >Reporter: Igor Skokov >Priority: Major > Labels: pull-request-available > Time Spent: 10m > Remaining Estimate: 0h > > When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in > some circumstances Zookeeper node stop listening on leader election port. > This cause unavailability of ZK cluster. > Zookeeper deployed as StatefulSet in term of Kubernetes and has following > dynamic configuration: > {code:java} > zookeeper-0.zookeeper:2182:2183:participant;2181 > zookeeper-1.zookeeper:2182:2183:participant;2181 > zookeeper-2.zookeeper:2182:2183:participant;2181 > {code} > Bind address contains DNS name which generated by Kubernetes for each > StatefulSet pod. > These DNS names will become resolvable after container start, but with some > delay. That delay cause stopping of leader election port listener in > QuorumCnxManager.Listener class. > Error happens in QuorumCnxManager.Listener "run" method, it tries to bind > leader election port to hostname which not resolvable at this moment. Retry > count is hard-coded and equals to 3(with backoff of 1 sec). > Zookeeper server log contains following errors: > {code:java} > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - > Unexpected exception > java.net.SocketException: Unresolved address > at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) > at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) > at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) > at > org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) > at > org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1261] - > PeerState set to LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1136] - > LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):FastLeaderElection@893] > - New election. My id = 1, proposed zxid=0x0 > 2019-03-17 07:56:04,846 [myid:1] - INFO > [WorkerReceiver[myid=1]:FastLeaderElection@687] - Notification: 2 (message > format version), 1 (n.leader), 0x0 (n.zxid), 0xf (n.round), LOOKING > (n.state), 1 (n.sid), 0x0 (n.peerEPoch), LOOKING (my state)0 (n.config > version) > 2019-03-17 07:56:04,979 [myid:1] - INFO > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@892] - Leaving listener > 2019-03-17 07:56:04,979 [myid:1] - ERROR > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@894] - As I'm leaving > the listener thread, I won't be able to participate in leader election any > longer: zookeeper-0.zookeeper:2183 > {code} > This error happens on most nodes on cluster start and Zookeeper is unable to > form quorum. This will leave cluster in unusable state. > As I can see, error present on branches 3.4 and 3.5. > I think, this error can be fixed by configurable number of retries(instead of > hard-coded value of 3). > Other way to fix this is removing of max retries at all. Currently, ZK server > only stop leader election listener and continue to serve on other ports. > Maybe, if leader election halts, we should abort process. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (ZOOKEEPER-3320) Leader election port stop listen when hostname unresolvable for some time
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated ZOOKEEPER-3320: -- Labels: pull-request-available (was: ) > Leader election port stop listen when hostname unresolvable for some time > -- > > Key: ZOOKEEPER-3320 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3320 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.10, 3.5.4 >Reporter: Igor Skokov >Priority: Major > Labels: pull-request-available > > When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in > some circumstances Zookeeper node stop listening on leader election port. > This cause unavailability of ZK cluster. > Zookeeper deployed as StatefulSet in term of Kubernetes and has following > dynamic configuration: > {code:java} > zookeeper-0.zookeeper:2182:2183:participant;2181 > zookeeper-1.zookeeper:2182:2183:participant;2181 > zookeeper-2.zookeeper:2182:2183:participant;2181 > {code} > Bind address contains DNS name which generated by Kubernetes for each > StatefulSet pod. > These DNS names will become resolvable after container start, but with some > delay. That delay cause stopping of leader election port listener in > QuorumCnxManager.Listener class. > Error happens in QuorumCnxManager.Listener "run" method, it tries to bind > leader election port to hostname which not resolvable at this moment. Retry > count is hard-coded and equals to 3(with backoff of 1 sec). > Zookeeper server log contains following errors: > {code:java} > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - > Unexpected exception > java.net.SocketException: Unresolved address > at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) > at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) > at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) > at > org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) > at > org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1261] - > PeerState set to LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1136] - > LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):FastLeaderElection@893] > - New election. My id = 1, proposed zxid=0x0 > 2019-03-17 07:56:04,846 [myid:1] - INFO > [WorkerReceiver[myid=1]:FastLeaderElection@687] - Notification: 2 (message > format version), 1 (n.leader), 0x0 (n.zxid), 0xf (n.round), LOOKING > (n.state), 1 (n.sid), 0x0 (n.peerEPoch), LOOKING (my state)0 (n.config > version) > 2019-03-17 07:56:04,979 [myid:1] - INFO > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@892] - Leaving listener > 2019-03-17 07:56:04,979 [myid:1] - ERROR > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@894] - As I'm leaving > the listener thread, I won't be able to participate in leader election any > longer: zookeeper-0.zookeeper:2183 > {code} > This error happens on most nodes on cluster start and Zookeeper is unable to > form quorum. This will leave cluster in unusable state. > As I can see, error present on branches 3.4 and 3.5. > I think, this error can be fixed by configurable number of retries(instead of > hard-coded value of 3). > Other way to fix this is removing of max retries at all. Currently, ZK server > only stop leader election listener and continue to serve on other ports. > Maybe, if leader election halts, we should abort process. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[GitHub] [zookeeper] Lagrang opened a new pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time
Lagrang opened a new pull request #863: ZOOKEEPER-3320: Leader election port stop listen when hostname unresolvable for some time URL: https://github.com/apache/zookeeper/pull/863 This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[jira] [Commented] (ZOOKEEPER-3320) Leader election port stop listen when hostname unresolvable for some time
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16797657#comment-16797657 ] Brian Nixon commented on ZOOKEEPER-3320: A configurable retry seems like a good idea to me. Either something like "election port bind time" or "dns unavailable time" if we want to be more general. Do you want to contribute a short diff? This may also be related to ZOOKEEPER-2982 (or may not, making a note to check later). > Leader election port stop listen when hostname unresolvable for some time > -- > > Key: ZOOKEEPER-3320 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3320 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.10, 3.5.4 >Reporter: Igor Skokov >Priority: Major > > When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in > some circumstances Zookeeper node stop listening on leader election port. > This cause unavailability of ZK cluster. > Zookeeper deployed as StatefulSet in term of Kubernetes and has following > dynamic configuration: > {code:java} > zookeeper-0.zookeeper:2182:2183:participant;2181 > zookeeper-1.zookeeper:2182:2183:participant;2181 > zookeeper-2.zookeeper:2182:2183:participant;2181 > {code} > Bind address contains DNS name which generated by Kubernetes for each > StatefulSet pod. > These DNS names will become resolvable after container start, but with some > delay. That delay cause stopping of leader election port listener in > QuorumCnxManager.Listener class. > Error happens in QuorumCnxManager.Listener "run" method, it tries to bind > leader election port to hostname which not resolvable at this moment. Retry > count is hard-coded and equals to 3(with backoff of 1 sec). > Zookeeper server log contains following errors: > {code:java} > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - > Unexpected exception > java.net.SocketException: Unresolved address > at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) > at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) > at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) > at > org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) > at > org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1261] - > PeerState set to LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1136] - > LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):FastLeaderElection@893] > - New election. My id = 1, proposed zxid=0x0 > 2019-03-17 07:56:04,846 [myid:1] - INFO > [WorkerReceiver[myid=1]:FastLeaderElection@687] - Notification: 2 (message > format version), 1 (n.leader), 0x0 (n.zxid), 0xf (n.round), LOOKING > (n.state), 1 (n.sid), 0x0 (n.peerEPoch), LOOKING (my state)0 (n.config > version) > 2019-03-17 07:56:04,979 [myid:1] - INFO > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@892] - Leaving listener > 2019-03-17 07:56:04,979 [myid:1] - ERROR > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@894] - As I'm leaving > the listener thread, I won't be able to participate in leader election any > longer: zookeeper-0.zookeeper:2183 > {code} > This error happens on most nodes on cluster start and Zookeeper is unable to > form quorum. This will leave cluster in unusable state. > As I can see, error present on branches 3.4 and 3.5. > I think, this error can be fixed by configurable number of retries(instead of > hard-coded value of 3). > Other way to fix this is removing of max retries at all. Currently, ZK server > only stop leader election listener and continue to serve on other ports. > Maybe, if leader election halts, we should abort process. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3320) Leader election port stop listen when hostname unresolvable for some time
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16796844#comment-16796844 ] Igor Skokov commented on ZOOKEEPER-3320: ??I would expect an issue in QuorumCnxManager to bring the peer down if it cannot create the socket but it seems this only occurs with a BindException and not a generic SocketException. At the least, I think we ought to fix that.?? As I can see, QuorumCnxManager.Listener.run() catches IOException, and consequently SocketException In any case, ZK server will continue to run without leader election participation:) ??Looking at this from the opposite direction, can you add the desired delay in the startup sequence of your Kubernetes container? My concern is that the pattern of "DNS is currently unreliable but will be reliable soon" seems specific to the container management and may result in strange behavior when applied to other environments.?? Sure, I can integrate artificial delay to my container(or some DNS name check loop). But, it will be great if ZK can handle such temporary errors by itself. I agree, that "retry forever" is not best solution, but as I mention earlier, maybe we can make retry count configurable? > Leader election port stop listen when hostname unresolvable for some time > -- > > Key: ZOOKEEPER-3320 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3320 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.10, 3.5.4 >Reporter: Igor Skokov >Priority: Major > > When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in > some circumstances Zookeeper node stop listening on leader election port. > This cause unavailability of ZK cluster. > Zookeeper deployed as StatefulSet in term of Kubernetes and has following > dynamic configuration: > {code:java} > zookeeper-0.zookeeper:2182:2183:participant;2181 > zookeeper-1.zookeeper:2182:2183:participant;2181 > zookeeper-2.zookeeper:2182:2183:participant;2181 > {code} > Bind address contains DNS name which generated by Kubernetes for each > StatefulSet pod. > These DNS names will become resolvable after container start, but with some > delay. That delay cause stopping of leader election port listener in > QuorumCnxManager.Listener class. > Error happens in QuorumCnxManager.Listener "run" method, it tries to bind > leader election port to hostname which not resolvable at this moment. Retry > count is hard-coded and equals to 3(with backoff of 1 sec). > Zookeeper server log contains following errors: > {code:java} > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - > Unexpected exception > java.net.SocketException: Unresolved address > at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) > at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) > at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) > at > org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) > at > org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1261] - > PeerState set to LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1136] - > LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):FastLeaderElection@893] > - New election. My id = 1, proposed zxid=0x0 > 2019-03-17 07:56:04,846 [myid:1] - INFO > [WorkerReceiver[myid=1]:FastLeaderElection@687] - Notification: 2 (message > format version), 1 (n.leader), 0x0 (n.zxid), 0xf (n.round), LOOKING > (n.state), 1 (n.sid), 0x0 (n.peerEPoch), LOOKING (my state)0 (n.config > version) > 2019-03-17 07:56:04,979 [myid:1] - INFO > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@892] - Leaving listener > 2019-03-17 07:56:04,979 [myid:1] - ERROR > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@894] - As I'm leaving > the listener thread, I won't be able to participate in leader election any > longer: zookeeper-0.zookeeper:2183 > {code} > This error happens on most nodes on cluster start and Zookeeper is unable to > form quorum. This will leave cluster in unusable state. > As I can see, error present on branches 3.4 and 3.5. > I think, this error can be fixed by configurable number of ret
[jira] [Commented] (ZOOKEEPER-3320) Leader election port stop listen when hostname unresolvable for some time
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16796341#comment-16796341 ] Brian Nixon commented on ZOOKEEPER-3320: This is an interesting error case! I would expect an issue in QuorumCnxManager to bring the peer down if it cannot create the socket but it seems this only occurs with a BindException and not a generic SocketException. At the least, I think we ought to fix that. Looking at this from the opposite direction, can you add the desired delay in the startup sequence of your Kubernetes container? My concern is that the pattern of "DNS is currently unreliable but will be reliable soon" seems specific to the container management and may result in strange behavior when applied to other environments. > Leader election port stop listen when hostname unresolvable for some time > -- > > Key: ZOOKEEPER-3320 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3320 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.10, 3.5.4 >Reporter: Igor Skokov >Priority: Major > > When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in > some circumstances Zookeeper node stop listening on leader election port. > This cause unavailability of ZK cluster. > Zookeeper deployed as StatefulSet in term of Kubernetes and has following > dynamic configuration: > {code:java} > zookeeper-0.zookeeper:2182:2183:participant;2181 > zookeeper-1.zookeeper:2182:2183:participant;2181 > zookeeper-2.zookeeper:2182:2183:participant;2181 > {code} > Bind address contains DNS name which generated by Kubernetes for each > StatefulSet pod. > These DNS names will become resolvable after container start, but with some > delay. That delay cause stopping of leader election port listener in > QuorumCnxManager.Listener class. > Error happens in QuorumCnxManager.Listener "run" method, it tries to bind > leader election port to hostname which not resolvable at this moment. Retry > count is hard-coded and equals to 3(with backoff of 1 sec). > Zookeeper server log contains following errors: > {code:java} > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - > Unexpected exception > java.net.SocketException: Unresolved address > at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) > at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) > at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) > at > org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) > at > org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1261] - > PeerState set to LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1136] - > LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):FastLeaderElection@893] > - New election. My id = 1, proposed zxid=0x0 > 2019-03-17 07:56:04,846 [myid:1] - INFO > [WorkerReceiver[myid=1]:FastLeaderElection@687] - Notification: 2 (message > format version), 1 (n.leader), 0x0 (n.zxid), 0xf (n.round), LOOKING > (n.state), 1 (n.sid), 0x0 (n.peerEPoch), LOOKING (my state)0 (n.config > version) > 2019-03-17 07:56:04,979 [myid:1] - INFO > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@892] - Leaving listener > 2019-03-17 07:56:04,979 [myid:1] - ERROR > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@894] - As I'm leaving > the listener thread, I won't be able to participate in leader election any > longer: zookeeper-0.zookeeper:2183 > {code} > This error happens on most nodes on cluster start and Zookeeper is unable to > form quorum. This will leave cluster in unusable state. > As I can see, error present on branches 3.4 and 3.5. > I think, this error can be fixed by configurable number of retries(instead of > hard-coded value of 3). > Other way to fix this is removing of max retries at all. Currently, ZK server > only stop leader election listener and continue to serve on other ports. > Maybe, if leader election halts, we should abort process. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (ZOOKEEPER-3320) Leader election port stop listen when hostname unresolvable for some time
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Skokov updated ZOOKEEPER-3320: --- Description: When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in some circumstances Zookeeper node stop listening on leader election port. This cause unavailability of ZK cluster. Zookeeper deployed as StatefulSet in term of Kubernetes and has following dynamic configuration: {code:java} zookeeper-0.zookeeper:2182:2183:participant;2181 zookeeper-1.zookeeper:2182:2183:participant;2181 zookeeper-2.zookeeper:2182:2183:participant;2181 {code} Bind address contains DNS name which generated by Kubernetes for each StatefulSet pod. These DNS names will become resolvable after container start, but with some delay. That delay cause stopping of leader election port listener in QuorumCnxManager.Listener class. Error happens in QuorumCnxManager.Listener "run" method, it tries to bind leader election port to hostname which not resolvable at this moment. Retry count is hard-coded and equals to 3(with backoff of 1 sec). Zookeeper server log contains following errors: {code:java} 2019-03-17 07:56:04,844 [myid:1] - WARN [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - Unexpected exception java.net.SocketException: Unresolved address at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) at org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) 2019-03-17 07:56:04,844 [myid:1] - WARN [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1261] - PeerState set to LOOKING 2019-03-17 07:56:04,845 [myid:1] - INFO [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1136] - LOOKING 2019-03-17 07:56:04,845 [myid:1] - INFO [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):FastLeaderElection@893] - New election. My id = 1, proposed zxid=0x0 2019-03-17 07:56:04,846 [myid:1] - INFO [WorkerReceiver[myid=1]:FastLeaderElection@687] - Notification: 2 (message format version), 1 (n.leader), 0x0 (n.zxid), 0xf (n.round), LOOKING (n.state), 1 (n.sid), 0x0 (n.peerEPoch), LOOKING (my state)0 (n.config version) 2019-03-17 07:56:04,979 [myid:1] - INFO [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@892] - Leaving listener 2019-03-17 07:56:04,979 [myid:1] - ERROR [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@894] - As I'm leaving the listener thread, I won't be able to participate in leader election any longer: zookeeper-0.zookeeper:2183 {code} This error happens on most nodes on cluster start and Zookeeper is unable to form quorum. This will leave cluster in unusable state. As I can see, error present on branches 3.4 and 3.5. I think, this error can be fixed by configurable number of retries(instead of hard-coded value of 3). Other way to fix this is removing of max retries at all. Currently, ZK server only stop leader election listener and continue to serve on other ports. Maybe, if leader election halts, we should abort process. was: When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in some circumstances Zookeeper node stop listening on leader election port. This cause unavailability of ZK cluster. Zookeeper deployed as StatefulSet in term of Kubernetes and has following dynamic configuration: {code:java} zookeeper-0.zookeeper:2182:2183:participant;2181 zookeeper-1.zookeeper:2182:2183:participant;2181 zookeeper-2.zookeeper:2182:2183:participant;2181 {code} Bind address contains DNS name which generated by Kubernetes for each StatefulSet pod. These DNS names will become resolvable after container start, but with some delay. That delay cause stopping of leader election port listener in QuorumCnxManager.Listener class. Error happens in QuorumCnxManager.Listener "run" method, it tries to bind leader election port to hostname which not resolvable at this moment. Retry count is hard-coded and equals to 3(with backoff of 1 sec). Zookeeper server log contains following errors: {code:java} 2019-03-17 07:56:04,844 [myid:1] - WARN [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - Unexpected exception java.net.SocketException: Unresolved address at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) at org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) 2019-03-17
[jira] [Updated] (ZOOKEEPER-3320) Leader election port stop listen when hostname unresolvable for some time
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3320?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Skokov updated ZOOKEEPER-3320: --- Summary: Leader election port stop listen when hostname unresolvable for some time (was: Don't give up on bind of leader election port) > Leader election port stop listen when hostname unresolvable for some time > -- > > Key: ZOOKEEPER-3320 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3320 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.10, 3.5.4 >Reporter: Igor Skokov >Priority: Major > > When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in > some circumstances Zookeeper node stop listening on leader election port. > This cause unavailability of ZK cluster. > Zookeeper deployed as StatefulSet in term of Kubernetes and has following > dynamic configuration: > {code:java} > zookeeper-0.zookeeper:2182:2183:participant;2181 > zookeeper-1.zookeeper:2182:2183:participant;2181 > zookeeper-2.zookeeper:2182:2183:participant;2181 > {code} > Bind address contains DNS name which generated by Kubernetes for each > StatefulSet pod. > These DNS names will become resolvable after container start, but with some > delay. That delay cause stopping of leader election port listener in > QuorumCnxManager.Listener class. > Error happens in QuorumCnxManager.Listener "run" method, it tries to bind > leader election port to hostname which not resolvable at this moment. Retry > count is hard-coded and equals to 3(with backoff of 1 sec). > Zookeeper server log contains following errors: > {code:java} > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - > Unexpected exception > java.net.SocketException: Unresolved address > at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) > at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) > at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) > at > org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) > at > org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) > 2019-03-17 07:56:04,844 [myid:1] - WARN > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1261] - > PeerState set to LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1136] - > LOOKING > 2019-03-17 07:56:04,845 [myid:1] - INFO > [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):FastLeaderElection@893] > - New election. My id = 1, proposed zxid=0x0 > 2019-03-17 07:56:04,846 [myid:1] - INFO > [WorkerReceiver[myid=1]:FastLeaderElection@687] - Notification: 2 (message > format version), 1 (n.leader), 0x0 (n.zxid), 0xf (n.round), LOOKING > (n.state), 1 (n.sid), 0x0 (n.peerEPoch), LOOKING (my state)0 (n.config > version) > 2019-03-17 07:56:04,979 [myid:1] - INFO > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@892] - Leaving listener > 2019-03-17 07:56:04,979 [myid:1] - ERROR > [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@894] - As I'm leaving > the listener thread, I won't be able to participate in leader election any > longer: zookeeper-0.zookeeper:2183 > {code} > This error happens on most nodes on cluster start and Zookeeper is unable to > form quorum. This will leave cluster in unusable state. > As I can see, error present on branches 3.4 and 3.5. > I think, this error can be fixed by configurable number of retries(instead of > hard-coded value of 3). > Other way to fix this is removing of max retries at all. Currently, ZK server > only stop leader election listener and continue to serve on other ports. May > be, if leader election halts, we should abort process. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (ZOOKEEPER-3320) Don't give up on bind of leader election port
Igor Skokov created ZOOKEEPER-3320: -- Summary: Don't give up on bind of leader election port Key: ZOOKEEPER-3320 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3320 Project: ZooKeeper Issue Type: Bug Components: leaderElection Affects Versions: 3.5.4, 3.4.10 Reporter: Igor Skokov When trying to run Zookeeper 3.5.4 cluster on Kubernetes, I found out that in some circumstances Zookeeper node stop listening on leader election port. This cause unavailability of ZK cluster. Zookeeper deployed as StatefulSet in term of Kubernetes and has following dynamic configuration: {code:java} zookeeper-0.zookeeper:2182:2183:participant;2181 zookeeper-1.zookeeper:2182:2183:participant;2181 zookeeper-2.zookeeper:2182:2183:participant;2181 {code} Bind address contains DNS name which generated by Kubernetes for each StatefulSet pod. These DNS names will become resolvable after container start, but with some delay. That delay cause stopping of leader election port listener in QuorumCnxManager.Listener class. Error happens in QuorumCnxManager.Listener "run" method, it tries to bind leader election port to hostname which not resolvable at this moment. Retry count is hard-coded and equals to 3(with backoff of 1 sec). Zookeeper server log contains following errors: {code:java} 2019-03-17 07:56:04,844 [myid:1] - WARN [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1230] - Unexpected exception java.net.SocketException: Unresolved address at java.base/java.net.ServerSocket.bind(ServerSocket.java:374) at java.base/java.net.ServerSocket.bind(ServerSocket.java:335) at org.apache.zookeeper.server.quorum.Leader.(Leader.java:241) at org.apache.zookeeper.server.quorum.QuorumPeer.makeLeader(QuorumPeer.java:1023) at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1226) 2019-03-17 07:56:04,844 [myid:1] - WARN [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1261] - PeerState set to LOOKING 2019-03-17 07:56:04,845 [myid:1] - INFO [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):QuorumPeer@1136] - LOOKING 2019-03-17 07:56:04,845 [myid:1] - INFO [QuorumPeer[myid=1](plain=/0.0.0.0:2181)(secure=disabled):FastLeaderElection@893] - New election. My id = 1, proposed zxid=0x0 2019-03-17 07:56:04,846 [myid:1] - INFO [WorkerReceiver[myid=1]:FastLeaderElection@687] - Notification: 2 (message format version), 1 (n.leader), 0x0 (n.zxid), 0xf (n.round), LOOKING (n.state), 1 (n.sid), 0x0 (n.peerEPoch), LOOKING (my state)0 (n.config version) 2019-03-17 07:56:04,979 [myid:1] - INFO [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@892] - Leaving listener 2019-03-17 07:56:04,979 [myid:1] - ERROR [zookeeper-0.zookeeper:2183:QuorumCnxManager$Listener@894] - As I'm leaving the listener thread, I won't be able to participate in leader election any longer: zookeeper-0.zookeeper:2183 {code} This error happens on most nodes on cluster start and Zookeeper is unable to form quorum. This will leave cluster in unusable state. As I can see, error present on branches 3.4 and 3.5. I think, this error can be fixed by configurable number of retries(instead of hard-coded value of 3). Other way to fix this is removing of max retries at all. Currently, ZK server only stop leader election listener and continue to serve on other ports. May be, if leader election halts, we should abort process. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-2164) fast leader election keeps failing
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2164?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16751546#comment-16751546 ] Ian Spence commented on ZOOKEEPER-2164: --- We can reproduce this issue with 3.4.6. 5 node ZK cluster, we restarted one node and after an hour it still has not joined the quorum. stat and mntr show "This ZooKeeper instance is not currently serving requests". > fast leader election keeps failing > -- > > Key: ZOOKEEPER-2164 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2164 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.5 >Reporter: Michi Mutsuzaki >Priority: Major > Fix For: 3.6.0, 3.5.5 > > > I have a 3-node cluster with sids 1, 2 and 3. Originally 2 is the leader. > When I shut down 2, 1 and 3 keep going back to leader election. Here is what > seems to be happening. > - Both 1 and 3 elect 3 as the leader. > - 1 receives votes from 3 and itself, and starts trying to connect to 3 as a > follower. > - 3 doesn't receive votes for 5 seconds because connectOne() to 2 doesn't > timeout for 5 seconds: > https://github.com/apache/zookeeper/blob/41c9fcb3ca09cd3d05e59fe47f08ecf0b85532c8/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java#L346 > - By the time 3 receives votes, 1 has given up trying to connect to 3: > https://github.com/apache/zookeeper/blob/41c9fcb3ca09cd3d05e59fe47f08ecf0b85532c8/src/java/main/org/apache/zookeeper/server/quorum/Learner.java#L247 > I'm using 3.4.5, but it looks like this part of the code hasn't changed for a > while, so I'm guessing later versions have the same issue. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (ZOOKEEPER-3247) New lest admin command to get leader election time
Dinesh Appavoo created ZOOKEEPER-3247: - Summary: New lest admin command to get leader election time Key: ZOOKEEPER-3247 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3247 Project: ZooKeeper Issue Type: New Feature Components: leaderElection Reporter: Dinesh Appavoo Add lest admin command to get the last leader election time. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
Re: Leader election
>> Can we reduce this time by configuring "syncLimit" and "tickTime" to let's say 5 seconds? Can we have a strong guarantee on this time bound? It's not possible to guarantee the time bound, because of FLP impossibility (reliable failure detection is not possible in async environment). Though it's certainly possible to tune the parameters to some reasonable value that fits your environment (which would be the SLA of your service). >> As describe above - you might use 'sync'+'read' to avoid this problem. I am afraid sync + read would not be correct 100% in all cases here. The state of the world (e.g. leaders) could change between sync and read operation. What we need here is linearizable read, which means we need have read operations also go through the quorum consensus, which might be a nice feature to have for ZooKeeper (for reference, etcd implements linearizable read). Also, note ZooKeeper sync has bugs (sync should be a quorum operation itself, but it's not implemented that way). On Fri, Dec 7, 2018 at 2:11 AM Maciej Smoleński wrote: > On Fri, Dec 7, 2018 at 3:03 AM Michael Borokhovich > wrote: > > > We are planning to run Zookeeper nodes embedded with the client nodes. > > I.e., each client runs also a ZK node. So, network partition will > > disconnect a ZK node and not only the client. > > My concern is about the following statement from the ZK documentation: > > > > "Timeliness: The clients view of the system is guaranteed to be > up-to-date > > within a certain time bound. (*On the order of tens of seconds.*) Either > > system changes will be seen by a client within this bound, or the client > > will detect a service outage." > > > > This is related to the fact that ZooKeeper server handles reads from its > local state - without communicating with other ZooKeeper servers. > This design ensures scalability for read dominated workloads. > In this approach client might receive data which is not up to date (it > might not contain updates from other ZooKeeper servers (quorum)). > Parameter 'syncLimit' describes how often ZooKeeper server > synchronizes/updates its local state to global state. > Client read operation will retrieve data from state not older then > described by 'syncLimit'. > > However ZooKeeper client can always force to retrieve data which is up to > date. > It needs to issue 'sync' command to ZooKeeper server before issueing > 'read'. > With 'sync' ZooKeeper server with synchronize its local state with global > state. > Later 'read' will be handled from updated state. > Client should be careful here - so that it communicates with the same > ZooKeeper server for both 'sync' and 'read'. > > > > What are these "*tens of seconds*"? Can we reduce this time by > configuring > > "syncLimit" and "tickTime" to let's say 5 seconds? Can we have a strong > > guarantee on this time bound? > > > > As describe above - you might use 'sync'+'read' to avoid this problem. > > > > > > > > On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman < > > jor...@jordanzimmerman.com> > > wrote: > > > > > > Old service leader will detect network partition max 15 seconds after > > it > > > > happened. > > > > > > If the old service leader is in a very long GC it will not detect the > > > partition. In the face of VM pauses, etc. it's not possible to avoid 2 > > > leaders for a short period of time. > > > > > > -JZ > > >
Re: Leader election
Thanks a lot for sharing the design, Ted. It is very helpful. Will check what is applicable to our case and let you know in case of questions. On Mon, Dec 10, 2018 at 23:37 Ted Dunning wrote: > One very useful way to deal with this is the method used in MapR FS. The > idea is that ZK should only be used rarely and short periods of two leaders > must be tolerated, but other data has to be written with absolute > consistency. > > The method that we chose was to associate an epoch number with every write, > require all writes to to all replicas and require that all replicas only > acknowledge writes with their idea of the current epoch for an object. > > What happens in the even of partition is that we have a few possible cases, > but in any case where data replicas are split by a partition, writes will > fail triggering a new leader election. Only replicas on the side of the new > ZK quorum (which may be the old quorum) have a chance of succeeding here. > If the replicas are split away from the ZK quorum, writes may not be > possible until the partition heals. If a new leader is elected, it will > increment the epoch and form a replication chain out of the replicas it can > find telling them about the new epoch. Writes can then proceed. During > partition healing, any pending writes from the old epoch will be ignored by > the current replicas. None of the writes to the new epoch will be directed > to the old replicas after partition healing, but such writes should be > ignored as well. > > In a side process, replicas that have come back after a partition may be > updated with writes from the new replicas. If the partition lasts long > enough, a new replica should be formed from the members of the current > epoch. If a new replica is formed and an old one is resurrected, then the > old one should probably be deprecated, although data balancing > considerations may come into play. > > In the actual implementation of MapR FS, there is a lot of sophistication > that does into the details, of course, and there is actually one more level > of delegation that happens, but this outline is good enough for a lot of > systems. > > The virtues of this system are multiple: > > 1) partition is detected exactly as soon as it affects a write. Detecting a > partition sooner than that doesn't serve a lot of purpose, especially since > the time to recover from a failed write is comparable to the duration of a > fair number of partitions. > > 2) having an old master continue under false pretenses does no harm since > it cannot write to a more recent replica chain. This is more important than > it might seem since there can be situations where clocks don't necessarily > advance at the expected rate so what seems like a short time can actually > be much longer (Rip van Winkle failures). > > 3) forcing writes to all live replicas while allowing reorganization is > actually very fast and as long as we can retain one live replica we can > continue writing. This is in contrast to quorum systems where dropping > below the quorum stops writes. This is important because the replicas of > different objects can be arranged so that the portion of the cluster with a > ZK quorum might not have a majority of replicas for some objects. > > 4) electing a new master of a replica chain can be done quite quickly so > the duration of any degradation can be quite short (because you can set > write timeouts fairly short because an unnecessary election takes less time > than a long timeout) > > Anyway, you probably already have a design in mind. If this helps anyway, > that's great. > > On Mon, Dec 10, 2018 at 10:32 PM Michael Borokhovich > > wrote: > > > Makes sense. Thanks, Ted. We will design our system to cope with the > short > > periods where we might have two leaders. > > > > On Thu, Dec 6, 2018 at 11:03 PM Ted Dunning > wrote: > > > > > ZK is able to guarantee that there is only one leader for the purposes > of > > > updating ZK data. That is because all commits have to originate with > the > > > current quorum leader and then be acknowledged by a quorum of the > current > > > cluster. IF the leader can't get enough acks, then it has de facto lost > > > leadership. > > > > > > The problem comes when there is another system that depends on ZK data. > > > Such data might record which node is the leader for some other > purposes. > > > That leader will only assume that they have become leader if they > succeed > > > in writing to ZK, but if there is a partition, then the old leader may > > not > > > be notified that another leader has established themselves until some > > time > >
[jira] [Commented] (ZOOKEEPER-2461) There is no difference between the observer and the participants in the leader election algorithm
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16718439#comment-16718439 ] John Kim commented on ZOOKEEPER-2461: - I think this and ZOOKEEPER-2555 are describing the same issue. Anyway, we ran into what looks close to what [~nerdyyatrice] described (running 3.4.12), but with slight difference in order: Observers took about 10-15ms less time to complete their shutdown than Followers. The difference seems to be the times between when CommitProcessor is shut down and when SyncRequestProcessor is shut down. So around step 2, Followers have not yet completed shutdown and were sending back election notification with n.zxid and n.leader that was used/set in the previous election. One part I'm not sure of is that observer logs do not mention any of the notifications participants sent after they started their own elections, as they should've been within the 200ms window. Perhaps this is where bug lies. So observers try to follow the old leader, eventually timing out and completing a new set of election. > There is no difference between the observer and the participants in the > leader election algorithm > - > > Key: ZOOKEEPER-2461 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2461 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum >Affects Versions: 3.5.0 >Reporter: Ryan Zhang >Assignee: Ryan Zhang >Priority: Major > Fix For: 3.6.0, 3.5.5 > > > We have observed a case that when a leader machine crashes hard, non-voting > learners take a long time to detect the new leader. After looking at the > details more carefully, we identified one potential improvement (and one bug > fixed in the 3.5). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
Re: Leader election
One very useful way to deal with this is the method used in MapR FS. The idea is that ZK should only be used rarely and short periods of two leaders must be tolerated, but other data has to be written with absolute consistency. The method that we chose was to associate an epoch number with every write, require all writes to to all replicas and require that all replicas only acknowledge writes with their idea of the current epoch for an object. What happens in the even of partition is that we have a few possible cases, but in any case where data replicas are split by a partition, writes will fail triggering a new leader election. Only replicas on the side of the new ZK quorum (which may be the old quorum) have a chance of succeeding here. If the replicas are split away from the ZK quorum, writes may not be possible until the partition heals. If a new leader is elected, it will increment the epoch and form a replication chain out of the replicas it can find telling them about the new epoch. Writes can then proceed. During partition healing, any pending writes from the old epoch will be ignored by the current replicas. None of the writes to the new epoch will be directed to the old replicas after partition healing, but such writes should be ignored as well. In a side process, replicas that have come back after a partition may be updated with writes from the new replicas. If the partition lasts long enough, a new replica should be formed from the members of the current epoch. If a new replica is formed and an old one is resurrected, then the old one should probably be deprecated, although data balancing considerations may come into play. In the actual implementation of MapR FS, there is a lot of sophistication that does into the details, of course, and there is actually one more level of delegation that happens, but this outline is good enough for a lot of systems. The virtues of this system are multiple: 1) partition is detected exactly as soon as it affects a write. Detecting a partition sooner than that doesn't serve a lot of purpose, especially since the time to recover from a failed write is comparable to the duration of a fair number of partitions. 2) having an old master continue under false pretenses does no harm since it cannot write to a more recent replica chain. This is more important than it might seem since there can be situations where clocks don't necessarily advance at the expected rate so what seems like a short time can actually be much longer (Rip van Winkle failures). 3) forcing writes to all live replicas while allowing reorganization is actually very fast and as long as we can retain one live replica we can continue writing. This is in contrast to quorum systems where dropping below the quorum stops writes. This is important because the replicas of different objects can be arranged so that the portion of the cluster with a ZK quorum might not have a majority of replicas for some objects. 4) electing a new master of a replica chain can be done quite quickly so the duration of any degradation can be quite short (because you can set write timeouts fairly short because an unnecessary election takes less time than a long timeout) Anyway, you probably already have a design in mind. If this helps anyway, that's great. On Mon, Dec 10, 2018 at 10:32 PM Michael Borokhovich wrote: > Makes sense. Thanks, Ted. We will design our system to cope with the short > periods where we might have two leaders. > > On Thu, Dec 6, 2018 at 11:03 PM Ted Dunning wrote: > > > ZK is able to guarantee that there is only one leader for the purposes of > > updating ZK data. That is because all commits have to originate with the > > current quorum leader and then be acknowledged by a quorum of the current > > cluster. IF the leader can't get enough acks, then it has de facto lost > > leadership. > > > > The problem comes when there is another system that depends on ZK data. > > Such data might record which node is the leader for some other purposes. > > That leader will only assume that they have become leader if they succeed > > in writing to ZK, but if there is a partition, then the old leader may > not > > be notified that another leader has established themselves until some > time > > after it has happened. Of course, if the erstwhile leader tried to > validate > > its position with a write to ZK, that write would fail, but you can't > spend > > 100% of your time doing that. > > > > it all comes down to the fact that a ZK client determines that it is > > connected to a ZK cluster member by pinging and that cluster member sees > > heartbeats from the leader. The fact is, though, that you can't tune > these > > pings to be faster than some level because you start to see lots of false > > positives for loss of connection. Remember that it isn't jus
Re: Leader election
Thanks, Maciej. That sounds good. We will try playing with the parameters and have at least a known upper limit on the inconsistency interval. On Fri, Dec 7, 2018 at 2:11 AM Maciej Smoleński wrote: > On Fri, Dec 7, 2018 at 3:03 AM Michael Borokhovich > wrote: > > > We are planning to run Zookeeper nodes embedded with the client nodes. > > I.e., each client runs also a ZK node. So, network partition will > > disconnect a ZK node and not only the client. > > My concern is about the following statement from the ZK documentation: > > > > "Timeliness: The clients view of the system is guaranteed to be > up-to-date > > within a certain time bound. (*On the order of tens of seconds.*) Either > > system changes will be seen by a client within this bound, or the client > > will detect a service outage." > > > > This is related to the fact that ZooKeeper server handles reads from its > local state - without communicating with other ZooKeeper servers. > This design ensures scalability for read dominated workloads. > In this approach client might receive data which is not up to date (it > might not contain updates from other ZooKeeper servers (quorum)). > Parameter 'syncLimit' describes how often ZooKeeper server > synchronizes/updates its local state to global state. > Client read operation will retrieve data from state not older then > described by 'syncLimit'. > > However ZooKeeper client can always force to retrieve data which is up to > date. > It needs to issue 'sync' command to ZooKeeper server before issueing > 'read'. > With 'sync' ZooKeeper server with synchronize its local state with global > state. > Later 'read' will be handled from updated state. > Client should be careful here - so that it communicates with the same > ZooKeeper server for both 'sync' and 'read'. > > > > What are these "*tens of seconds*"? Can we reduce this time by > configuring > > "syncLimit" and "tickTime" to let's say 5 seconds? Can we have a strong > > guarantee on this time bound? > > > > As describe above - you might use 'sync'+'read' to avoid this problem. > > > > > > > > On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman < > > jor...@jordanzimmerman.com> > > wrote: > > > > > > Old service leader will detect network partition max 15 seconds after > > it > > > > happened. > > > > > > If the old service leader is in a very long GC it will not detect the > > > partition. In the face of VM pauses, etc. it's not possible to avoid 2 > > > leaders for a short period of time. > > > > > > -JZ > > >
Re: Leader election
Yes, I agree, our system should be able to tolerate two leaders for a short and bounded period of time. Thank you for the help! On Thu, Dec 6, 2018 at 11:09 AM Jordan Zimmerman wrote: > > it seems like the > > inconsistency may be caused by the partition of the Zookeeper cluster > > itself > > Yes - there are many ways in which you can end up with 2 leaders. However, > if properly tuned and configured, it will be for a few seconds at most. > During a GC pause no work is being done anyway. But, this stuff is very > tricky. Requiring an atomically unique leader is actually a design smell > and you should reconsider your architecture. > > > Maybe we can use a more > > lightweight Hazelcast for example? > > There is no distributed system that can guarantee a single leader. Instead > you need to adjust your design and algorithms to deal with this (using > optimistic locking, etc.). > > -Jordan > > > On Dec 6, 2018, at 1:52 PM, Michael Borokhovich > wrote: > > > > Thanks Jordan, > > > > Yes, I will try Curator. > > Also, beyond the problem described in the Tech Note, it seems like the > > inconsistency may be caused by the partition of the Zookeeper cluster > > itself. E.g., if a "leader" client is connected to the partitioned ZK > node, > > it may be notified not at the same time as the other clients connected to > > the other ZK nodes. So, another client may take leadership while the > > current leader still unaware of the change. Is it true? > > > > Another follow up question. If Zookeeper can guarantee a single leader, > is > > it worth using it just for leader election? Maybe we can use a more > > lightweight Hazelcast for example? > > > > Michael. > > > > > > On Thu, Dec 6, 2018 at 4:50 AM Jordan Zimmerman < > jor...@jordanzimmerman.com> > > wrote: > > > >> It is not possible to achieve the level of consistency you're after in > an > >> eventually consistent system such as ZooKeeper. There will always be an > >> edge case where two ZooKeeper clients will believe they are leaders > (though > >> for a short period of time). In terms of how it affects Apache Curator, > we > >> have this Tech Note on the subject: > >> https://cwiki.apache.org/confluence/display/CURATOR/TN10 < > >> https://cwiki.apache.org/confluence/display/CURATOR/TN10> (the > >> description is true for any ZooKeeper client, not just Curator > clients). If > >> you do still intend to use a ZooKeeper lock/leader I suggest you try > Apache > >> Curator as writing these "recipes" is not trivial and have many gotchas > >> that aren't obvious. > >> > >> -Jordan > >> > >> http://curator.apache.org <http://curator.apache.org/> > >> > >> > >>> On Dec 5, 2018, at 6:20 PM, Michael Borokhovich > >> wrote: > >>> > >>> Hello, > >>> > >>> We have a service that runs on 3 hosts for high availability. However, > at > >>> any given time, exactly one instance must be active. So, we are > thinking > >> to > >>> use Leader election using Zookeeper. > >>> To this goal, on each service host we also start a ZK server, so we > have > >> a > >>> 3-nodes ZK cluster and each service instance is a client to its > dedicated > >>> ZK server. > >>> Then, we implement a leader election on top of Zookeeper using a basic > >>> recipe: > >>> https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection > . > >>> > >>> I have the following questions doubts regarding the approach: > >>> > >>> 1. It seems like we can run into inconsistency issues when network > >>> partition occurs. Zookeeper documentation says that the inconsistency > >>> period may last “tens of seconds”. Am I understanding correctly that > >> during > >>> this time we may have 0 or 2 leaders? > >>> 2. Is it possible to reduce this inconsistency time (let's say to 3 > >>> seconds) by tweaking tickTime and syncLimit parameters? > >>> 3. Is there a way to guarantee exactly one leader all the time? Should > we > >>> implement a more complex leader election algorithm than the one > suggested > >>> in the recipe (using ephemeral_sequential nodes)? > >>> > >>> Thanks, > >>> Michael. > >> > >> > >
Re: Leader election
Makes sense. Thanks, Ted. We will design our system to cope with the short periods where we might have two leaders. On Thu, Dec 6, 2018 at 11:03 PM Ted Dunning wrote: > ZK is able to guarantee that there is only one leader for the purposes of > updating ZK data. That is because all commits have to originate with the > current quorum leader and then be acknowledged by a quorum of the current > cluster. IF the leader can't get enough acks, then it has de facto lost > leadership. > > The problem comes when there is another system that depends on ZK data. > Such data might record which node is the leader for some other purposes. > That leader will only assume that they have become leader if they succeed > in writing to ZK, but if there is a partition, then the old leader may not > be notified that another leader has established themselves until some time > after it has happened. Of course, if the erstwhile leader tried to validate > its position with a write to ZK, that write would fail, but you can't spend > 100% of your time doing that. > > it all comes down to the fact that a ZK client determines that it is > connected to a ZK cluster member by pinging and that cluster member sees > heartbeats from the leader. The fact is, though, that you can't tune these > pings to be faster than some level because you start to see lots of false > positives for loss of connection. Remember that it isn't just loss of > connection here that is the point. Any kind of delay would have the same > effect. Getting these ping intervals below one second makes for a very > twitchy system. > > > > On Fri, Dec 7, 2018 at 11:03 AM Michael Borokhovich > wrote: > > > We are planning to run Zookeeper nodes embedded with the client nodes. > > I.e., each client runs also a ZK node. So, network partition will > > disconnect a ZK node and not only the client. > > My concern is about the following statement from the ZK documentation: > > > > "Timeliness: The clients view of the system is guaranteed to be > up-to-date > > within a certain time bound. (*On the order of tens of seconds.*) Either > > system changes will be seen by a client within this bound, or the client > > will detect a service outage." > > > > What are these "*tens of seconds*"? Can we reduce this time by > configuring > > "syncLimit" and "tickTime" to let's say 5 seconds? Can we have a strong > > guarantee on this time bound? > > > > > > On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman < > > jor...@jordanzimmerman.com> > > wrote: > > > > > > Old service leader will detect network partition max 15 seconds after > > it > > > > happened. > > > > > > If the old service leader is in a very long GC it will not detect the > > > partition. In the face of VM pauses, etc. it's not possible to avoid 2 > > > leaders for a short period of time. > > > > > > -JZ > > >
Re: Leader election
On Fri, Dec 7, 2018 at 3:03 AM Michael Borokhovich wrote: > We are planning to run Zookeeper nodes embedded with the client nodes. > I.e., each client runs also a ZK node. So, network partition will > disconnect a ZK node and not only the client. > My concern is about the following statement from the ZK documentation: > > "Timeliness: The clients view of the system is guaranteed to be up-to-date > within a certain time bound. (*On the order of tens of seconds.*) Either > system changes will be seen by a client within this bound, or the client > will detect a service outage." > This is related to the fact that ZooKeeper server handles reads from its local state - without communicating with other ZooKeeper servers. This design ensures scalability for read dominated workloads. In this approach client might receive data which is not up to date (it might not contain updates from other ZooKeeper servers (quorum)). Parameter 'syncLimit' describes how often ZooKeeper server synchronizes/updates its local state to global state. Client read operation will retrieve data from state not older then described by 'syncLimit'. However ZooKeeper client can always force to retrieve data which is up to date. It needs to issue 'sync' command to ZooKeeper server before issueing 'read'. With 'sync' ZooKeeper server with synchronize its local state with global state. Later 'read' will be handled from updated state. Client should be careful here - so that it communicates with the same ZooKeeper server for both 'sync' and 'read'. > What are these "*tens of seconds*"? Can we reduce this time by configuring > "syncLimit" and "tickTime" to let's say 5 seconds? Can we have a strong > guarantee on this time bound? > As describe above - you might use 'sync'+'read' to avoid this problem. > > > On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman < > jor...@jordanzimmerman.com> > wrote: > > > > Old service leader will detect network partition max 15 seconds after > it > > > happened. > > > > If the old service leader is in a very long GC it will not detect the > > partition. In the face of VM pauses, etc. it's not possible to avoid 2 > > leaders for a short period of time. > > > > -JZ >
Re: Leader election
ZK is able to guarantee that there is only one leader for the purposes of updating ZK data. That is because all commits have to originate with the current quorum leader and then be acknowledged by a quorum of the current cluster. IF the leader can't get enough acks, then it has de facto lost leadership. The problem comes when there is another system that depends on ZK data. Such data might record which node is the leader for some other purposes. That leader will only assume that they have become leader if they succeed in writing to ZK, but if there is a partition, then the old leader may not be notified that another leader has established themselves until some time after it has happened. Of course, if the erstwhile leader tried to validate its position with a write to ZK, that write would fail, but you can't spend 100% of your time doing that. it all comes down to the fact that a ZK client determines that it is connected to a ZK cluster member by pinging and that cluster member sees heartbeats from the leader. The fact is, though, that you can't tune these pings to be faster than some level because you start to see lots of false positives for loss of connection. Remember that it isn't just loss of connection here that is the point. Any kind of delay would have the same effect. Getting these ping intervals below one second makes for a very twitchy system. On Fri, Dec 7, 2018 at 11:03 AM Michael Borokhovich wrote: > We are planning to run Zookeeper nodes embedded with the client nodes. > I.e., each client runs also a ZK node. So, network partition will > disconnect a ZK node and not only the client. > My concern is about the following statement from the ZK documentation: > > "Timeliness: The clients view of the system is guaranteed to be up-to-date > within a certain time bound. (*On the order of tens of seconds.*) Either > system changes will be seen by a client within this bound, or the client > will detect a service outage." > > What are these "*tens of seconds*"? Can we reduce this time by configuring > "syncLimit" and "tickTime" to let's say 5 seconds? Can we have a strong > guarantee on this time bound? > > > On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman < > jor...@jordanzimmerman.com> > wrote: > > > > Old service leader will detect network partition max 15 seconds after > it > > > happened. > > > > If the old service leader is in a very long GC it will not detect the > > partition. In the face of VM pauses, etc. it's not possible to avoid 2 > > leaders for a short period of time. > > > > -JZ >
Re: Leader election
We are planning to run Zookeeper nodes embedded with the client nodes. I.e., each client runs also a ZK node. So, network partition will disconnect a ZK node and not only the client. My concern is about the following statement from the ZK documentation: "Timeliness: The clients view of the system is guaranteed to be up-to-date within a certain time bound. (*On the order of tens of seconds.*) Either system changes will be seen by a client within this bound, or the client will detect a service outage." What are these "*tens of seconds*"? Can we reduce this time by configuring "syncLimit" and "tickTime" to let's say 5 seconds? Can we have a strong guarantee on this time bound? On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman wrote: > > Old service leader will detect network partition max 15 seconds after it > > happened. > > If the old service leader is in a very long GC it will not detect the > partition. In the face of VM pauses, etc. it's not possible to avoid 2 > leaders for a short period of time. > > -JZ
Re: Leader election
Tweak timeout is tempting as your solution might work most of the time yet fail in certain cases (which others have pointed out). If the goal is absolute correctness then we should avoid timeout, which does not guarantee correctness as it only makes the problem hard to manifest. Fencing is the right solution here - the zxid and also znode cversion can be used as fencing token if you use ZooKeeper. Fencing will guarantee at any single point in time you will have one active leader in action (it does not guarantee that at a single point of time there are multiple parties *think* they are the leader). An alternative solution, depends on your use case, is to instead of requiring a single active leader in action at any time, make your workload idempotent so multiple active leaders don't do any damage. On Thu, Dec 6, 2018 at 1:05 PM Jordan Zimmerman wrote: > > Old service leader will detect network partition max 15 seconds after it > > happened. > > If the old service leader is in a very long GC it will not detect the > partition. In the face of VM pauses, etc. it's not possible to avoid 2 > leaders for a short period of time. > > -JZ
Re: Leader election
> Old service leader will detect network partition max 15 seconds after it > happened. If the old service leader is in a very long GC it will not detect the partition. In the face of VM pauses, etc. it's not possible to avoid 2 leaders for a short period of time. -JZ
Re: Leader election
Hello, Ensuring reliability requires to use consensus directly in your service or change the service to use distributed log/journal (e.g. bookkeeper). However following idea is simple and in many situation good enough. If you configure session timeout to 15 seconds - then zookeeper client will be disconnected when partitioned - after max 15 seconds. Old service leader will detect network partition max 15 seconds after it happened. The new service leader should be idle for initial 15+ seconds (let's say 30 seconds). In this way you avoid situation with 2 concurrently working leaders. Described solution has time dependencies and in some situations leads to incorrect state e.g.: High load on machine might cause that zookeeper client will detect disconnection after 60 seconds (instead of expected 15 seconds). In such situation there will be 2 concurrent leaders. Maciej On Thu, Dec 6, 2018 at 8:09 PM Jordan Zimmerman wrote: > > it seems like the > > inconsistency may be caused by the partition of the Zookeeper cluster > > itself > > Yes - there are many ways in which you can end up with 2 leaders. However, > if properly tuned and configured, it will be for a few seconds at most. > During a GC pause no work is being done anyway. But, this stuff is very > tricky. Requiring an atomically unique leader is actually a design smell > and you should reconsider your architecture. > > > Maybe we can use a more > > lightweight Hazelcast for example? > > There is no distributed system that can guarantee a single leader. Instead > you need to adjust your design and algorithms to deal with this (using > optimistic locking, etc.). > > -Jordan > > > On Dec 6, 2018, at 1:52 PM, Michael Borokhovich > wrote: > > > > Thanks Jordan, > > > > Yes, I will try Curator. > > Also, beyond the problem described in the Tech Note, it seems like the > > inconsistency may be caused by the partition of the Zookeeper cluster > > itself. E.g., if a "leader" client is connected to the partitioned ZK > node, > > it may be notified not at the same time as the other clients connected to > > the other ZK nodes. So, another client may take leadership while the > > current leader still unaware of the change. Is it true? > > > > Another follow up question. If Zookeeper can guarantee a single leader, > is > > it worth using it just for leader election? Maybe we can use a more > > lightweight Hazelcast for example? > > > > Michael. > > > > > > On Thu, Dec 6, 2018 at 4:50 AM Jordan Zimmerman < > jor...@jordanzimmerman.com> > > wrote: > > > >> It is not possible to achieve the level of consistency you're after in > an > >> eventually consistent system such as ZooKeeper. There will always be an > >> edge case where two ZooKeeper clients will believe they are leaders > (though > >> for a short period of time). In terms of how it affects Apache Curator, > we > >> have this Tech Note on the subject: > >> https://cwiki.apache.org/confluence/display/CURATOR/TN10 < > >> https://cwiki.apache.org/confluence/display/CURATOR/TN10> (the > >> description is true for any ZooKeeper client, not just Curator > clients). If > >> you do still intend to use a ZooKeeper lock/leader I suggest you try > Apache > >> Curator as writing these "recipes" is not trivial and have many gotchas > >> that aren't obvious. > >> > >> -Jordan > >> > >> http://curator.apache.org <http://curator.apache.org/> > >> > >> > >>> On Dec 5, 2018, at 6:20 PM, Michael Borokhovich > >> wrote: > >>> > >>> Hello, > >>> > >>> We have a service that runs on 3 hosts for high availability. However, > at > >>> any given time, exactly one instance must be active. So, we are > thinking > >> to > >>> use Leader election using Zookeeper. > >>> To this goal, on each service host we also start a ZK server, so we > have > >> a > >>> 3-nodes ZK cluster and each service instance is a client to its > dedicated > >>> ZK server. > >>> Then, we implement a leader election on top of Zookeeper using a basic > >>> recipe: > >>> https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection > . > >>> > >>> I have the following questions doubts regarding the approach: > >>> > >>> 1. It seems like we can run into inconsistency issues when network > >>> partition occurs. Zookeeper documentation says that the inconsistency > >>> period may last “tens of seconds”. Am I understanding correctly that > >> during > >>> this time we may have 0 or 2 leaders? > >>> 2. Is it possible to reduce this inconsistency time (let's say to 3 > >>> seconds) by tweaking tickTime and syncLimit parameters? > >>> 3. Is there a way to guarantee exactly one leader all the time? Should > we > >>> implement a more complex leader election algorithm than the one > suggested > >>> in the recipe (using ephemeral_sequential nodes)? > >>> > >>> Thanks, > >>> Michael. > >> > >> > >
Re: Leader election
> it seems like the > inconsistency may be caused by the partition of the Zookeeper cluster > itself Yes - there are many ways in which you can end up with 2 leaders. However, if properly tuned and configured, it will be for a few seconds at most. During a GC pause no work is being done anyway. But, this stuff is very tricky. Requiring an atomically unique leader is actually a design smell and you should reconsider your architecture. > Maybe we can use a more > lightweight Hazelcast for example? There is no distributed system that can guarantee a single leader. Instead you need to adjust your design and algorithms to deal with this (using optimistic locking, etc.). -Jordan > On Dec 6, 2018, at 1:52 PM, Michael Borokhovich wrote: > > Thanks Jordan, > > Yes, I will try Curator. > Also, beyond the problem described in the Tech Note, it seems like the > inconsistency may be caused by the partition of the Zookeeper cluster > itself. E.g., if a "leader" client is connected to the partitioned ZK node, > it may be notified not at the same time as the other clients connected to > the other ZK nodes. So, another client may take leadership while the > current leader still unaware of the change. Is it true? > > Another follow up question. If Zookeeper can guarantee a single leader, is > it worth using it just for leader election? Maybe we can use a more > lightweight Hazelcast for example? > > Michael. > > > On Thu, Dec 6, 2018 at 4:50 AM Jordan Zimmerman > wrote: > >> It is not possible to achieve the level of consistency you're after in an >> eventually consistent system such as ZooKeeper. There will always be an >> edge case where two ZooKeeper clients will believe they are leaders (though >> for a short period of time). In terms of how it affects Apache Curator, we >> have this Tech Note on the subject: >> https://cwiki.apache.org/confluence/display/CURATOR/TN10 < >> https://cwiki.apache.org/confluence/display/CURATOR/TN10> (the >> description is true for any ZooKeeper client, not just Curator clients). If >> you do still intend to use a ZooKeeper lock/leader I suggest you try Apache >> Curator as writing these "recipes" is not trivial and have many gotchas >> that aren't obvious. >> >> -Jordan >> >> http://curator.apache.org <http://curator.apache.org/> >> >> >>> On Dec 5, 2018, at 6:20 PM, Michael Borokhovich >> wrote: >>> >>> Hello, >>> >>> We have a service that runs on 3 hosts for high availability. However, at >>> any given time, exactly one instance must be active. So, we are thinking >> to >>> use Leader election using Zookeeper. >>> To this goal, on each service host we also start a ZK server, so we have >> a >>> 3-nodes ZK cluster and each service instance is a client to its dedicated >>> ZK server. >>> Then, we implement a leader election on top of Zookeeper using a basic >>> recipe: >>> https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection. >>> >>> I have the following questions doubts regarding the approach: >>> >>> 1. It seems like we can run into inconsistency issues when network >>> partition occurs. Zookeeper documentation says that the inconsistency >>> period may last “tens of seconds”. Am I understanding correctly that >> during >>> this time we may have 0 or 2 leaders? >>> 2. Is it possible to reduce this inconsistency time (let's say to 3 >>> seconds) by tweaking tickTime and syncLimit parameters? >>> 3. Is there a way to guarantee exactly one leader all the time? Should we >>> implement a more complex leader election algorithm than the one suggested >>> in the recipe (using ephemeral_sequential nodes)? >>> >>> Thanks, >>> Michael. >> >>
Re: Leader election
Thanks Jordan, Yes, I will try Curator. Also, beyond the problem described in the Tech Note, it seems like the inconsistency may be caused by the partition of the Zookeeper cluster itself. E.g., if a "leader" client is connected to the partitioned ZK node, it may be notified not at the same time as the other clients connected to the other ZK nodes. So, another client may take leadership while the current leader still unaware of the change. Is it true? Another follow up question. If Zookeeper can guarantee a single leader, is it worth using it just for leader election? Maybe we can use a more lightweight Hazelcast for example? Michael. On Thu, Dec 6, 2018 at 4:50 AM Jordan Zimmerman wrote: > It is not possible to achieve the level of consistency you're after in an > eventually consistent system such as ZooKeeper. There will always be an > edge case where two ZooKeeper clients will believe they are leaders (though > for a short period of time). In terms of how it affects Apache Curator, we > have this Tech Note on the subject: > https://cwiki.apache.org/confluence/display/CURATOR/TN10 < > https://cwiki.apache.org/confluence/display/CURATOR/TN10> (the > description is true for any ZooKeeper client, not just Curator clients). If > you do still intend to use a ZooKeeper lock/leader I suggest you try Apache > Curator as writing these "recipes" is not trivial and have many gotchas > that aren't obvious. > > -Jordan > > http://curator.apache.org <http://curator.apache.org/> > > > > On Dec 5, 2018, at 6:20 PM, Michael Borokhovich > wrote: > > > > Hello, > > > > We have a service that runs on 3 hosts for high availability. However, at > > any given time, exactly one instance must be active. So, we are thinking > to > > use Leader election using Zookeeper. > > To this goal, on each service host we also start a ZK server, so we have > a > > 3-nodes ZK cluster and each service instance is a client to its dedicated > > ZK server. > > Then, we implement a leader election on top of Zookeeper using a basic > > recipe: > > https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection. > > > > I have the following questions doubts regarding the approach: > > > > 1. It seems like we can run into inconsistency issues when network > > partition occurs. Zookeeper documentation says that the inconsistency > > period may last “tens of seconds”. Am I understanding correctly that > during > > this time we may have 0 or 2 leaders? > > 2. Is it possible to reduce this inconsistency time (let's say to 3 > > seconds) by tweaking tickTime and syncLimit parameters? > > 3. Is there a way to guarantee exactly one leader all the time? Should we > > implement a more complex leader election algorithm than the one suggested > > in the recipe (using ephemeral_sequential nodes)? > > > > Thanks, > > Michael. > >
Re: Leader election
It is not possible to achieve the level of consistency you're after in an eventually consistent system such as ZooKeeper. There will always be an edge case where two ZooKeeper clients will believe they are leaders (though for a short period of time). In terms of how it affects Apache Curator, we have this Tech Note on the subject: https://cwiki.apache.org/confluence/display/CURATOR/TN10 <https://cwiki.apache.org/confluence/display/CURATOR/TN10> (the description is true for any ZooKeeper client, not just Curator clients). If you do still intend to use a ZooKeeper lock/leader I suggest you try Apache Curator as writing these "recipes" is not trivial and have many gotchas that aren't obvious. -Jordan http://curator.apache.org <http://curator.apache.org/> > On Dec 5, 2018, at 6:20 PM, Michael Borokhovich wrote: > > Hello, > > We have a service that runs on 3 hosts for high availability. However, at > any given time, exactly one instance must be active. So, we are thinking to > use Leader election using Zookeeper. > To this goal, on each service host we also start a ZK server, so we have a > 3-nodes ZK cluster and each service instance is a client to its dedicated > ZK server. > Then, we implement a leader election on top of Zookeeper using a basic > recipe: > https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection. > > I have the following questions doubts regarding the approach: > > 1. It seems like we can run into inconsistency issues when network > partition occurs. Zookeeper documentation says that the inconsistency > period may last “tens of seconds”. Am I understanding correctly that during > this time we may have 0 or 2 leaders? > 2. Is it possible to reduce this inconsistency time (let's say to 3 > seconds) by tweaking tickTime and syncLimit parameters? > 3. Is there a way to guarantee exactly one leader all the time? Should we > implement a more complex leader election algorithm than the one suggested > in the recipe (using ephemeral_sequential nodes)? > > Thanks, > Michael.
回复:Re: Leader election
---> "Can it happen that we end up with 2 leaders or 0 leader for some period of time (for example, during network delays/partitions)?" look at the code: https://github.com/apache/curator/blob/master/curator-recipes/src/main/java/org/apache/curator/framework/recipes/leader/LeaderSelector.java#L340 it can guarantee exactly one leader all the time(EPHEMERAL_SEQUENTIAL zk-node) which has not too much correlations with the network partitions of zk ensembles itself. I guess,haha! - 原始邮件 - 发件人:Michael Borokhovich 收件人:dev@zookeeper.apache.org, maoling199210...@sina.com 主题:Re: Leader election 日期:2018年12月06日 15点18分 Thanks, I will check it out. However, do you know if it gives any better guarantees? Can it happen that we end up with 2 leaders or 0 leader for some period of time (for example, during network delays/partitions)? On Wed, Dec 5, 2018 at 10:54 PM 毛蛤丝 wrote: > suggest you use the ready-made implements of curator: > http://curator.apache.org/curator-recipes/leader-election.html > - 原始邮件 - > 发件人:Michael Borokhovich > 收件人:"dev@zookeeper.apache.org" > 主题:Leader election > 日期:2018年12月06日 07点29分 > > Hello, > We have a service that runs on 3 hosts for high availability. However, at > any given time, exactly one instance must be active. So, we are thinking to > use Leader election using Zookeeper. > To this goal, on each service host we also start a ZK server, so we have a > 3-nodes ZK cluster and each service instance is a client to its dedicated > ZK server. > Then, we implement a leader election on top of Zookeeper using a basic > recipe: > https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection. > I have the following questions doubts regarding the approach: > 1. It seems like we can run into inconsistency issues when network > partition occurs. Zookeeper documentation says that the inconsistency > period may last “tens of seconds”. Am I understanding correctly that during > this time we may have 0 or 2 leaders? > 2. Is it possible to reduce this inconsistency time (let's say to 3 > seconds) by tweaking tickTime and syncLimit parameters? > 3. Is there a way to guarantee exactly one leader all the time? Should we > implement a more complex leader election algorithm than the one suggested > in the recipe (using ephemeral_sequential nodes)? > Thanks, > Michael. >
Re: Leader election
Michael, Leader election is not enough. You must have some mechanism to fence off the partitioned leader. If you are building a replicated state machine Apache Zookeeper + Apache Bookkeeper can be a good choice See this just an example: https://github.com/ivankelly/bookkeeper-tutorial This is the "bible" for ZooKeepers and it describes how to build such systems and the importance of "fencing" https://www.amazon.com/ZooKeeper-Distributed-Coordination-Flavio-Junqueira-ebook/dp/B00GRCODKS If you are interested in BookKeeper ping on user@ Apache BookKeeper mailing list Enrico Il gio 6 dic 2018, 08:18 Michael Borokhovich ha scritto: > Thanks, I will check it out. > However, do you know if it gives any better guarantees? > Can it happen that we end up with 2 leaders or 0 leader for some period of > time (for example, during network delays/partitions)? > > > > On Wed, Dec 5, 2018 at 10:54 PM 毛蛤丝 wrote: > > > suggest you use the ready-made implements of curator: > > http://curator.apache.org/curator-recipes/leader-election.html > > - 原始邮件 - > > 发件人:Michael Borokhovich > > 收件人:"dev@zookeeper.apache.org" > > 主题:Leader election > > 日期:2018年12月06日 07点29分 > > > > Hello, > > We have a service that runs on 3 hosts for high availability. However, at > > any given time, exactly one instance must be active. So, we are thinking > to > > use Leader election using Zookeeper. > > To this goal, on each service host we also start a ZK server, so we have > a > > 3-nodes ZK cluster and each service instance is a client to its dedicated > > ZK server. > > Then, we implement a leader election on top of Zookeeper using a basic > > recipe: > > https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection. > > I have the following questions doubts regarding the approach: > > 1. It seems like we can run into inconsistency issues when network > > partition occurs. Zookeeper documentation says that the inconsistency > > period may last “tens of seconds”. Am I understanding correctly that > during > > this time we may have 0 or 2 leaders? > > 2. Is it possible to reduce this inconsistency time (let's say to 3 > > seconds) by tweaking tickTime and syncLimit parameters? > > 3. Is there a way to guarantee exactly one leader all the time? Should we > > implement a more complex leader election algorithm than the one suggested > > in the recipe (using ephemeral_sequential nodes)? > > Thanks, > > Michael. > > >
Re: Leader election
Thanks, I will check it out. However, do you know if it gives any better guarantees? Can it happen that we end up with 2 leaders or 0 leader for some period of time (for example, during network delays/partitions)? On Wed, Dec 5, 2018 at 10:54 PM 毛蛤丝 wrote: > suggest you use the ready-made implements of curator: > http://curator.apache.org/curator-recipes/leader-election.html > - 原始邮件 - > 发件人:Michael Borokhovich > 收件人:"dev@zookeeper.apache.org" > 主题:Leader election > 日期:2018年12月06日 07点29分 > > Hello, > We have a service that runs on 3 hosts for high availability. However, at > any given time, exactly one instance must be active. So, we are thinking to > use Leader election using Zookeeper. > To this goal, on each service host we also start a ZK server, so we have a > 3-nodes ZK cluster and each service instance is a client to its dedicated > ZK server. > Then, we implement a leader election on top of Zookeeper using a basic > recipe: > https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection. > I have the following questions doubts regarding the approach: > 1. It seems like we can run into inconsistency issues when network > partition occurs. Zookeeper documentation says that the inconsistency > period may last “tens of seconds”. Am I understanding correctly that during > this time we may have 0 or 2 leaders? > 2. Is it possible to reduce this inconsistency time (let's say to 3 > seconds) by tweaking tickTime and syncLimit parameters? > 3. Is there a way to guarantee exactly one leader all the time? Should we > implement a more complex leader election algorithm than the one suggested > in the recipe (using ephemeral_sequential nodes)? > Thanks, > Michael. >
回复:Leader election
suggest you use the ready-made implements of curator: http://curator.apache.org/curator-recipes/leader-election.html - 原始邮件 - 发件人:Michael Borokhovich 收件人:"dev@zookeeper.apache.org" 主题:Leader election 日期:2018年12月06日 07点29分 Hello, We have a service that runs on 3 hosts for high availability. However, at any given time, exactly one instance must be active. So, we are thinking to use Leader election using Zookeeper. To this goal, on each service host we also start a ZK server, so we have a 3-nodes ZK cluster and each service instance is a client to its dedicated ZK server. Then, we implement a leader election on top of Zookeeper using a basic recipe: https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection. I have the following questions doubts regarding the approach: 1. It seems like we can run into inconsistency issues when network partition occurs. Zookeeper documentation says that the inconsistency period may last “tens of seconds”. Am I understanding correctly that during this time we may have 0 or 2 leaders? 2. Is it possible to reduce this inconsistency time (let's say to 3 seconds) by tweaking tickTime and syncLimit parameters? 3. Is there a way to guarantee exactly one leader all the time? Should we implement a more complex leader election algorithm than the one suggested in the recipe (using ephemeral_sequential nodes)? Thanks, Michael.
Leader election
Hello, We have a service that runs on 3 hosts for high availability. However, at any given time, exactly one instance must be active. So, we are thinking to use Leader election using Zookeeper. To this goal, on each service host we also start a ZK server, so we have a 3-nodes ZK cluster and each service instance is a client to its dedicated ZK server. Then, we implement a leader election on top of Zookeeper using a basic recipe: https://zookeeper.apache.org/doc/r3.1.2/recipes.html#sc_leaderElection. I have the following questions doubts regarding the approach: 1. It seems like we can run into inconsistency issues when network partition occurs. Zookeeper documentation says that the inconsistency period may last “tens of seconds”. Am I understanding correctly that during this time we may have 0 or 2 leaders? 2. Is it possible to reduce this inconsistency time (let's say to 3 seconds) by tweaking tickTime and syncLimit parameters? 3. Is there a way to guarantee exactly one leader all the time? Should we implement a more complex leader election algorithm than the one suggested in the recipe (using ephemeral_sequential nodes)? Thanks, Michael.
[jira] [Commented] (ZOOKEEPER-1814) Reduction of waiting time during Fast Leader Election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16695632#comment-16695632 ] Daniel Peon commented on ZOOKEEPER-1814: Hi Michael, I'm afraid of that the patch is not applicable, long time without receiving approval. It was design for 3.4.5 and it will need adaptation. I'll try to grab some time to adapt it to new code, but if someone else is interested in it and will be able to make it faster, don't hesitate to handle it. BR, Dani. > Reduction of waiting time during Fast Leader Election > - > > Key: ZOOKEEPER-1814 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1814 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.5, 3.5.0 >Reporter: Daniel Peon >Assignee: Daniel Peon >Priority: Major > Fix For: 3.6.0, 3.5.5 > > Attachments: ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814_release_3_5_0.patch, > ZOOKEEPER-1814_trunk.patch > > Original Estimate: 24h > Remaining Estimate: 24h > > FastLeader election takes long time because of the exponential backoff. > Currently the time is 60 seconds. > It would be interesting to give the possibility to configure this parameter, > like for example for a Server shutdown. > Otherwise, it sometimes takes so long and it has been detected a test failure > when executing: org.apache.zookeeper.server.quorum.QuorumPeerMainTest. > This test case waits until 30 seconds and this is smaller than the 60 seconds > where the leader election can be waiting for at the moment of shutting down. > Considering the failure during the test case, this issue was considered a > possible bug. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-1814) Reduction of waiting time during Fast Leader Election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16695374#comment-16695374 ] Hadoop QA commented on ZOOKEEPER-1814: -- -1 overall. Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12678142/ZOOKEEPER-1814_release_3_5_0.patch against trunk revision 1507f67a06175155003722297daeb60bc912af1d. +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 9 new or modified tests. -1 patch. The patch command could not apply the patch. Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-Build/3715//console This message is automatically generated. > Reduction of waiting time during Fast Leader Election > - > > Key: ZOOKEEPER-1814 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1814 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.5, 3.5.0 >Reporter: Daniel Peon >Assignee: Daniel Peon >Priority: Major > Fix For: 3.6.0, 3.5.5 > > Attachments: ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814_release_3_5_0.patch, > ZOOKEEPER-1814_trunk.patch > > Original Estimate: 24h > Remaining Estimate: 24h > > FastLeader election takes long time because of the exponential backoff. > Currently the time is 60 seconds. > It would be interesting to give the possibility to configure this parameter, > like for example for a Server shutdown. > Otherwise, it sometimes takes so long and it has been detected a test failure > when executing: org.apache.zookeeper.server.quorum.QuorumPeerMainTest. > This test case waits until 30 seconds and this is smaller than the 60 seconds > where the leader election can be waiting for at the moment of shutting down. > Considering the failure during the test case, this issue was considered a > possible bug. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-1814) Reduction of waiting time during Fast Leader Election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-1814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16695369#comment-16695369 ] Michael K. Edwards commented on ZOOKEEPER-1814: --- Is this applicable to the current branch-3.5? > Reduction of waiting time during Fast Leader Election > - > > Key: ZOOKEEPER-1814 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1814 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.5, 3.5.0 >Reporter: Daniel Peon >Assignee: Daniel Peon >Priority: Major > Fix For: 3.6.0, 3.5.5 > > Attachments: ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, ZOOKEEPER-1814.patch, > ZOOKEEPER-1814.patch, ZOOKEEPER-1814_release_3_5_0.patch, > ZOOKEEPER-1814_trunk.patch > > Original Estimate: 24h > Remaining Estimate: 24h > > FastLeader election takes long time because of the exponential backoff. > Currently the time is 60 seconds. > It would be interesting to give the possibility to configure this parameter, > like for example for a Server shutdown. > Otherwise, it sometimes takes so long and it has been detected a test failure > when executing: org.apache.zookeeper.server.quorum.QuorumPeerMainTest. > This test case waits until 30 seconds and this is smaller than the 60 seconds > where the leader election can be waiting for at the moment of shutting down. > Considering the failure during the test case, this issue was considered a > possible bug. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-2164) fast leader election keeps failing
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2164?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16695360#comment-16695360 ] Michael K. Edwards commented on ZOOKEEPER-2164: --- Is this reproducible with the current branch-3.5 code? > fast leader election keeps failing > -- > > Key: ZOOKEEPER-2164 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2164 > Project: ZooKeeper > Issue Type: Bug > Components: leaderElection >Affects Versions: 3.4.5 >Reporter: Michi Mutsuzaki >Priority: Major > Fix For: 3.6.0, 3.5.5 > > > I have a 3-node cluster with sids 1, 2 and 3. Originally 2 is the leader. > When I shut down 2, 1 and 3 keep going back to leader election. Here is what > seems to be happening. > - Both 1 and 3 elect 3 as the leader. > - 1 receives votes from 3 and itself, and starts trying to connect to 3 as a > follower. > - 3 doesn't receive votes for 5 seconds because connectOne() to 2 doesn't > timeout for 5 seconds: > https://github.com/apache/zookeeper/blob/41c9fcb3ca09cd3d05e59fe47f08ecf0b85532c8/src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java#L346 > - By the time 3 receives votes, 1 has given up trying to connect to 3: > https://github.com/apache/zookeeper/blob/41c9fcb3ca09cd3d05e59fe47f08ecf0b85532c8/src/java/main/org/apache/zookeeper/server/quorum/Learner.java#L247 > I'm using 3.4.5, but it looks like this part of the code hasn't changed for a > while, so I'm guessing later versions have the same issue. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (ZOOKEEPER-2845) Data inconsistency issue due to retain database in leader election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated ZOOKEEPER-2845: -- Labels: pull-request-available (was: ) > Data inconsistency issue due to retain database in leader election > -- > > Key: ZOOKEEPER-2845 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2845 > Project: ZooKeeper > Issue Type: Bug > Components: quorum >Affects Versions: 3.4.10, 3.5.3, 3.6.0 >Reporter: Fangmin Lv >Assignee: Robert Joseph Evans >Priority: Critical > Labels: pull-request-available > Fix For: 3.5.4, 3.6.0, 3.4.12 > > > In ZOOKEEPER-2678, the ZKDatabase is retained to reduce the unavailable time > during leader election. In ZooKeeper ensemble, it's possible that the > snapshot is ahead of txn file (due to slow disk on the server, etc), or the > txn file is ahead of snapshot due to no commit message being received yet. > If snapshot is ahead of txn file, since the SyncRequestProcessor queue will > be drained during shutdown, the snapshot and txn file will keep consistent > before leader election happening, so this is not an issue. > But if txn is ahead of snapshot, it's possible that the ensemble will have > data inconsistent issue, here is the simplified scenario to show the issue: > Let's say we have a 3 servers in the ensemble, server A and B are followers, > and C is leader, and all the snapshot and txn are up to T0: > 1. A new request reached to leader C to create Node N, and it's converted to > txn T1 > 2. Txn T1 was synced to disk in C, but just before the proposal reaching out > to the followers, A and B restarted, so the T1 didn't exist in A and B > 3. A and B formed a new quorum after restart, let's say B is the leader > 4. C changed to looking state due to no enough followers, it will sync with > leader B with last Zxid T0, which will have an empty diff sync > 5. Before C take snapshot it restarted, it replayed the txns on disk which > includes T1, now it will have Node N, but A and B doesn't have it. > Also I included the a test case to reproduce this issue consistently. > We have a totally different RetainDB version which will avoid this issue by > doing consensus between snapshot and txn files before leader election, will > submit for review. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Comment Edited] (ZOOKEEPER-2845) Data inconsistency issue due to retain database in leader election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16614183#comment-16614183 ] Fangmin Lv edited comment on ZOOKEEPER-2845 at 9/14/18 12:18 AM: - [~revans2] sorry to get back to this lately, I was in parental leave and totally missed this thread (my girl was born on Jan 25, so was busy dealing with the new challenges there :) ) I'm revisiting my opening PR today and came across this one. Checked your fix, looks nice and simple! There was one thing I thought which might be a problem but actually it won't be a problem anymore with ZOOKEEPER-2678 you made last time. The thing I was thinking is in [ZooKeeperServer.processTxn|https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java#L1213] it didn't add itself to commit log in ZKDatabase, which will leave a hole in commit logs if we apply txns directly to DataTree during DIFF sync, which in turn could cause data inconsistency if it became leader. But we're not doing this anymore with ZOOKEEPER-2678, so it's fine. Our internal patch is a little bit heavier and complexity, we may change to use this simpler solution as well. Thanks again for moving this forward! was (Author: lvfangmin): [~revans2] sorry to get back to this lately, I was in parental leave and totally missed this thread (my girl was born on Jan 25, so was busy dealing with the new challenges there :) ) I'm revisiting my opening PR today and came across this one. Checked your fix, looks nice and simple! There was one thing I thought which might be a problem but actually it won't be a problem anymore with ZOOKEEPER-2678 you made last time. The thing I was thinking is in [ZooKeeperServer.processTxn(TxnHeader, Record)](https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java#L1213) it didn't add itself to commit log in ZKDatabase, which will leave a hole in commit logs if we apply txns directly to DataTree during DIFF sync, which in turn could cause data inconsistency if it became leader. But we're not doing this anymore with ZOOKEEPER-2678, so it's fine. Our internal patch is a little bit heavier and complexity, we may change to use this simpler solution as well. Thanks again for moving this forward! > Data inconsistency issue due to retain database in leader election > -- > > Key: ZOOKEEPER-2845 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2845 > Project: ZooKeeper > Issue Type: Bug > Components: quorum >Affects Versions: 3.4.10, 3.5.3, 3.6.0 >Reporter: Fangmin Lv >Assignee: Robert Joseph Evans >Priority: Critical > Fix For: 3.5.4, 3.6.0, 3.4.12 > > > In ZOOKEEPER-2678, the ZKDatabase is retained to reduce the unavailable time > during leader election. In ZooKeeper ensemble, it's possible that the > snapshot is ahead of txn file (due to slow disk on the server, etc), or the > txn file is ahead of snapshot due to no commit message being received yet. > If snapshot is ahead of txn file, since the SyncRequestProcessor queue will > be drained during shutdown, the snapshot and txn file will keep consistent > before leader election happening, so this is not an issue. > But if txn is ahead of snapshot, it's possible that the ensemble will have > data inconsistent issue, here is the simplified scenario to show the issue: > Let's say we have a 3 servers in the ensemble, server A and B are followers, > and C is leader, and all the snapshot and txn are up to T0: > 1. A new request reached to leader C to create Node N, and it's converted to > txn T1 > 2. Txn T1 was synced to disk in C, but just before the proposal reaching out > to the followers, A and B restarted, so the T1 didn't exist in A and B > 3. A and B formed a new quorum after restart, let's say B is the leader > 4. C changed to looking state due to no enough followers, it will sync with > leader B with last Zxid T0, which will have an empty diff sync > 5. Before C take snapshot it restarted, it replayed the txns on disk which > includes T1, now it will have Node N, but A and B doesn't have it. > Also I included the a test case to reproduce this issue consistently. > We have a totally different RetainDB version which will avoid this issue by > doing consensus between snapshot and txn files before leader election, will > submit for review. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-2845) Data inconsistency issue due to retain database in leader election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16614183#comment-16614183 ] Fangmin Lv commented on ZOOKEEPER-2845: --- [~revans2] sorry to get back to this lately, I was in parental leave and totally missed this thread (my girl was born on Jan 25, so was busy dealing with the new challenges there :) ) I'm revisiting my opening PR today and came across this one. Checked your fix, looks nice and simple! There was one thing I thought which might be a problem but actually it won't be a problem anymore with ZOOKEEPER-2678 you made last time. The thing I was thinking is in [ZooKeeperServer.processTxn(TxnHeader, Record)](https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java#L1213) it didn't add itself to commit log in ZKDatabase, which will leave a hole in commit logs if we apply txns directly to DataTree during DIFF sync, which in turn could cause data inconsistency if it became leader. But we're not doing this anymore with ZOOKEEPER-2678, so it's fine. Our internal patch is a little bit heavier and complexity, we may change to use this simpler solution as well. Thanks again for moving this forward! > Data inconsistency issue due to retain database in leader election > -- > > Key: ZOOKEEPER-2845 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2845 > Project: ZooKeeper > Issue Type: Bug > Components: quorum >Affects Versions: 3.4.10, 3.5.3, 3.6.0 >Reporter: Fangmin Lv >Assignee: Robert Joseph Evans >Priority: Critical > Fix For: 3.5.4, 3.6.0, 3.4.12 > > > In ZOOKEEPER-2678, the ZKDatabase is retained to reduce the unavailable time > during leader election. In ZooKeeper ensemble, it's possible that the > snapshot is ahead of txn file (due to slow disk on the server, etc), or the > txn file is ahead of snapshot due to no commit message being received yet. > If snapshot is ahead of txn file, since the SyncRequestProcessor queue will > be drained during shutdown, the snapshot and txn file will keep consistent > before leader election happening, so this is not an issue. > But if txn is ahead of snapshot, it's possible that the ensemble will have > data inconsistent issue, here is the simplified scenario to show the issue: > Let's say we have a 3 servers in the ensemble, server A and B are followers, > and C is leader, and all the snapshot and txn are up to T0: > 1. A new request reached to leader C to create Node N, and it's converted to > txn T1 > 2. Txn T1 was synced to disk in C, but just before the proposal reaching out > to the followers, A and B restarted, so the T1 didn't exist in A and B > 3. A and B formed a new quorum after restart, let's say B is the leader > 4. C changed to looking state due to no enough followers, it will sync with > leader B with last Zxid T0, which will have an empty diff sync > 5. Before C take snapshot it restarted, it replayed the txns on disk which > includes T1, now it will have Node N, but A and B doesn't have it. > Also I included the a test case to reproduce this issue consistently. > We have a totally different RetainDB version which will avoid this issue by > doing consensus between snapshot and txn files before leader election, will > submit for review. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3084) Exit when ZooKeeper cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16542507#comment-16542507 ] Hadoop QA commented on ZOOKEEPER-3084: -- -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 3 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs (version 3.0.1) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1932//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1932//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1932//console This message is automatically generated. > Exit when ZooKeeper cannot bind to the leader election port > --- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3084) Exit when ZooKeeper cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16539817#comment-16539817 ] Andor Molnar commented on ZOOKEEPER-3084: - [~lvfangmin] Would you please kindly create a separate pull request for the 3.5 branch? I think that would be the easiest to get this done. > Exit when ZooKeeper cannot bind to the leader election port > --- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3084) Exit when ZooKeeper cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16539554#comment-16539554 ] Hudson commented on ZOOKEEPER-3084: --- SUCCESS: Integrated in Jenkins build ZooKeeper-trunk #95 (See [https://builds.apache.org/job/ZooKeeper-trunk/95/]) ZOOKEEPER-3084: Exit when ZooKeeper cannot bind to the leader election (hanm: rev c2e7ed1e6f8f2de48778db7f3d63f9629c086ea8) * (edit) src/java/main/org/apache/zookeeper/server/quorum/QuorumCnxManager.java > Exit when ZooKeeper cannot bind to the leader election port > --- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3084) Exit when ZooKeeper cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16539534#comment-16539534 ] Michael Han commented on ZOOKEEPER-3084: committed to master: [https://github.com/apache/zookeeper/commit/c2e7ed1e6f8f2de48778db7f3d63f9629c086ea8] tried to merge to 3.5 but there are merge conflicts. have this on record so we can get this into 3.5 at some point before next release. > Exit when ZooKeeper cannot bind to the leader election port > --- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Issue Comment Deleted] (ZOOKEEPER-3084) Exit when ZooKeeper cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Han updated ZOOKEEPER-3084: --- Comment: was deleted (was: FormatedSqoop (SQOOP).bat ) > Exit when ZooKeeper cannot bind to the leader election port > --- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Resolved] (ZOOKEEPER-3084) Exit when ZooKeeper cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Han resolved ZOOKEEPER-3084. Resolution: Fixed > Exit when ZooKeeper cannot bind to the leader election port > --- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 1h 40m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3084) Exit when ZooKeeper cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16537741#comment-16537741 ] Mohamedvolt commented on ZOOKEEPER-3084: - FormatedSqoop (SQOOP).bat > Exit when ZooKeeper cannot bind to the leader election port > --- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 1h 10m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (ZOOKEEPER-3084) Exit when ZooKeeper cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Fangmin Lv updated ZOOKEEPER-3084: -- Summary: Exit when ZooKeeper cannot bind to the leader election port (was: Exit when zeus cannot bind to the leader election port) > Exit when ZooKeeper cannot bind to the leader election port > --- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 40m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3084) Exit when zeus cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16534462#comment-16534462 ] Hadoop QA commented on ZOOKEEPER-3084: -- -1 overall. GitHub Pull Request Build +1 @author. The patch does not contain any @author tags. +0 tests included. The patch appears to be a documentation patch that doesn't require tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 1 new Findbugs (version 3.0.1) warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. +1 core tests. The patch passed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1898//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1898//artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Console output: https://builds.apache.org/job/PreCommit-ZOOKEEPER-github-pr-build/1898//console This message is automatically generated. > Exit when zeus cannot bind to the leader election port > -- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > Time Spent: 10m > Remaining Estimate: 0h > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Updated] (ZOOKEEPER-3084) Exit when zeus cannot bind to the leader election port
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] ASF GitHub Bot updated ZOOKEEPER-3084: -- Labels: easyfix pull-request-available (was: easyfix) > Exit when zeus cannot bind to the leader election port > -- > > Key: ZOOKEEPER-3084 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 > Project: ZooKeeper > Issue Type: Improvement > Components: quorum, server >Reporter: Fangmin Lv >Assignee: Fangmin Lv >Priority: Minor > Labels: easyfix, pull-request-available > Fix For: 3.6.0 > > > In QuorumCnxManager, the listener thread will exit if it cannot bind to the > election port after trying 3 times. Which will keep the server running but > unable to join a quorum, the process will be dangling there and only > rejecting requests. It seems it's better to exit instead of sitting there > doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (ZOOKEEPER-3084) Exit when zeus cannot bind to the leader election port
Fangmin Lv created ZOOKEEPER-3084: - Summary: Exit when zeus cannot bind to the leader election port Key: ZOOKEEPER-3084 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3084 Project: ZooKeeper Issue Type: Improvement Components: quorum, server Reporter: Fangmin Lv Assignee: Fangmin Lv Fix For: 3.6.0 In QuorumCnxManager, the listener thread will exit if it cannot bind to the election port after trying 3 times. Which will keep the server running but unable to join a quorum, the process will be dangling there and only rejecting requests. It seems it's better to exit instead of sitting there doing nothing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-2845) Data inconsistency issue due to retain database in leader election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16378693#comment-16378693 ] ASF GitHub Bot commented on ZOOKEEPER-2845: --- Github user revans2 commented on the issue: https://github.com/apache/zookeeper/pull/453 Thanks @afine I closed them. > Data inconsistency issue due to retain database in leader election > -- > > Key: ZOOKEEPER-2845 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2845 > Project: ZooKeeper > Issue Type: Bug > Components: quorum >Affects Versions: 3.4.10, 3.5.3, 3.6.0 >Reporter: Fangmin Lv >Assignee: Robert Joseph Evans >Priority: Critical > Fix For: 3.5.4, 3.6.0, 3.4.12 > > > In ZOOKEEPER-2678, the ZKDatabase is retained to reduce the unavailable time > during leader election. In ZooKeeper ensemble, it's possible that the > snapshot is ahead of txn file (due to slow disk on the server, etc), or the > txn file is ahead of snapshot due to no commit message being received yet. > If snapshot is ahead of txn file, since the SyncRequestProcessor queue will > be drained during shutdown, the snapshot and txn file will keep consistent > before leader election happening, so this is not an issue. > But if txn is ahead of snapshot, it's possible that the ensemble will have > data inconsistent issue, here is the simplified scenario to show the issue: > Let's say we have a 3 servers in the ensemble, server A and B are followers, > and C is leader, and all the snapshot and txn are up to T0: > 1. A new request reached to leader C to create Node N, and it's converted to > txn T1 > 2. Txn T1 was synced to disk in C, but just before the proposal reaching out > to the followers, A and B restarted, so the T1 didn't exist in A and B > 3. A and B formed a new quorum after restart, let's say B is the leader > 4. C changed to looking state due to no enough followers, it will sync with > leader B with last Zxid T0, which will have an empty diff sync > 5. Before C take snapshot it restarted, it replayed the txns on disk which > includes T1, now it will have Node N, but A and B doesn't have it. > Also I included the a test case to reproduce this issue consistently. > We have a totally different RetainDB version which will avoid this issue by > doing consensus between snapshot and txn files before leader election, will > submit for review. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-2845) Data inconsistency issue due to retain database in leader election
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16378692#comment-16378692 ] ASF GitHub Bot commented on ZOOKEEPER-2845: --- Github user revans2 closed the pull request at: https://github.com/apache/zookeeper/pull/455 > Data inconsistency issue due to retain database in leader election > -- > > Key: ZOOKEEPER-2845 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2845 > Project: ZooKeeper > Issue Type: Bug > Components: quorum >Affects Versions: 3.4.10, 3.5.3, 3.6.0 >Reporter: Fangmin Lv >Assignee: Robert Joseph Evans >Priority: Critical > Fix For: 3.5.4, 3.6.0, 3.4.12 > > > In ZOOKEEPER-2678, the ZKDatabase is retained to reduce the unavailable time > during leader election. In ZooKeeper ensemble, it's possible that the > snapshot is ahead of txn file (due to slow disk on the server, etc), or the > txn file is ahead of snapshot due to no commit message being received yet. > If snapshot is ahead of txn file, since the SyncRequestProcessor queue will > be drained during shutdown, the snapshot and txn file will keep consistent > before leader election happening, so this is not an issue. > But if txn is ahead of snapshot, it's possible that the ensemble will have > data inconsistent issue, here is the simplified scenario to show the issue: > Let's say we have a 3 servers in the ensemble, server A and B are followers, > and C is leader, and all the snapshot and txn are up to T0: > 1. A new request reached to leader C to create Node N, and it's converted to > txn T1 > 2. Txn T1 was synced to disk in C, but just before the proposal reaching out > to the followers, A and B restarted, so the T1 didn't exist in A and B > 3. A and B formed a new quorum after restart, let's say B is the leader > 4. C changed to looking state due to no enough followers, it will sync with > leader B with last Zxid T0, which will have an empty diff sync > 5. Before C take snapshot it restarted, it replayed the txns on disk which > includes T1, now it will have Node N, but A and B doesn't have it. > Also I included the a test case to reproduce this issue consistently. > We have a totally different RetainDB version which will avoid this issue by > doing consensus between snapshot and txn files before leader election, will > submit for review. -- This message was sent by Atlassian JIRA (v7.6.3#76005)