[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 16:40
[jira] [Commented] (ZOOKEEPER-3056) Fails to load database with missing snapshot file but valid transaction log file
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16508448#comment-16508448 ] Matteo Merli commented on ZOOKEEPER-3056: - The main case is with ZK 3.4.x, for example with a single ZK quorum (though I think that would happen in the same way in a multi-node cluster). * Start ZK with 3.4.X * Create few z-nodes (not enough to trigger snapshot) * Stop ZK * Start with ZK 3.5.x --> Fail because snapshot is missing > Fails to load database with missing snapshot file but valid transaction log > file > > > Key: ZOOKEEPER-3056 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3056 > Project: ZooKeeper > Issue Type: Bug > Components: server >Affects Versions: 3.5.3, 3.5.4 >Reporter: Michael Han >Priority: Critical > > [An > issue|https://lists.apache.org/thread.html/cc17af6ef05d42318f74148f1a704f16934d1253f14721a93b4b@%3Cdev.zookeeper.apache.org%3E] > was reported when a user failed to upgrade from 3.4.10 to 3.5.4 with missing > snapshot file. > The code complains about missing snapshot file is > [here|https://github.com/apache/zookeeper/blob/release-3.5.4/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java#L206] > which is introduced as part of ZOOKEEPER-2325. > With this check, ZK will not load the db without a snapshot file, even the > transaction log files are present and valid. This could be a problem for > restoring a ZK instance which does not have a snapshot file but have a sound > state (e.g. it crashes before being able to take the first snap shot with a > large snapCount parameter configured). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-3056) Fails to load database with missing snapshot file but valid transaction log file
[ https://issues.apache.org/jira/browse/ZOOKEEPER-3056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16506443#comment-16506443 ] Matteo Merli commented on ZOOKEEPER-3056: - [~nixon] My concern is that in most cases there might not be an experienced ZK admin available, confortable in doing that operation. I would hope for a way to get a seamless migration path from 3.4 to 3.5 that doesn't require manual intervention. A flag to turn off that validation would be helpful. The risk is understood, but it would be the same behavior as in ZK-3.4.x. > Fails to load database with missing snapshot file but valid transaction log > file > > > Key: ZOOKEEPER-3056 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-3056 > Project: ZooKeeper > Issue Type: Bug > Components: server >Affects Versions: 3.5.3, 3.5.4 >Reporter: Michael Han >Priority: Critical > > [An > issue|https://lists.apache.org/thread.html/cc17af6ef05d42318f74148f1a704f16934d1253f14721a93b4b@%3Cdev.zookeeper.apache.org%3E] > was reported when a user failed to upgrade from 3.4.10 to 3.5.4 with missing > snapshot file. > The code complains about missing snapshot file is > [here|https://github.com/apache/zookeeper/blob/release-3.5.4/src/java/main/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java#L206] > which is introduced as part of ZOOKEEPER-2325. > With this check, ZK will not load the db without a snapshot file, even the > transaction log files are present and valid. This could be a problem for > restoring a ZK instance which does not have a snapshot file but have a sound > state (e.g. it crashes before being able to take the first snap shot with a > large snapCount parameter configured). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (ZOOKEEPER-2619) Client library reconnecting breaks FIFO client order
[ https://issues.apache.org/jira/browse/ZOOKEEPER-2619?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15620604#comment-15620604 ] Matteo Merli commented on ZOOKEEPER-2619: - Although unrelated to the API semantic discussion above, you could use {{multi()}} to have multiple updates to be performed atomically. That would be as good as the async op followed by the sync op, in that you only have to wait for 1 operation to be synced/complete. Eg: {code:java} List operations = new ArrayList<>(); operations.add(Op.create("/data-23857", "...")); operations.add(Op.create("/pointer", "/data-23857")); zk.multi(operations); {code} > Client library reconnecting breaks FIFO client order > > > Key: ZOOKEEPER-2619 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2619 > Project: ZooKeeper > Issue Type: Bug >Reporter: Diego Ongaro > > According to the USENIX ATC 2010 > [paper|https://www.usenix.org/conference/usenix-atc-10/zookeeper-wait-free-coordination-internet-scale-systems], > ZooKeeper provides "FIFO client order: all requests from a given client are > executed in the order that they were sent by the client." I believe > applications written using the Java client library are unable to rely on this > guarantee, and any current application that does so is broken. Other client > libraries are also likely to be affected. > Consider this application, which is simplified from the algorithm described > on Page 4 (right column) of the paper: > {code} > zk = new ZooKeeper(...) > zk.createAsync("/data-23857", "...", callback) > zk.createSync("/pointer", "/data-23857") > {code} > Assume an empty ZooKeeper database to begin with and no other writers. > Applying the above definition, if the ZooKeeper database contains /pointer, > it must also contain /data-23857. > Now consider this series of unfortunate events: > {code} > zk = new ZooKeeper(...) > // The library establishes a TCP connection. > zk.createAsync("/data-23857", "...", callback) > // The library/kernel closes the TCP connection because it times out, and > // the create of /data-23857 is doomed to fail with ConnectionLoss. Suppose > // that it never reaches the server. > // The library establishes a new TCP connection. > zk.createSync("/pointer", "/data-23857") > // The create of /pointer succeeds. > {code} > That's the problem: subsequent operations get assigned to the new connection > and succeed, while earlier operations fail. > In general, I believe it's impossible to have a system with the following > three properties: > # FIFO client order for asynchronous operations, > # Failing operations when connections are lost, AND > # Transparently reconnecting when connections are lost. > To argue this, consider an application that issues a series of pipelined > operations, then upon noticing a connection loss, issues a series of recovery > operations, repeating the recovery procedure as necessary. If a pipelined > operation fails, all subsequent operations in the pipeline must also fail. > Yet the client must also carry on eventually: the recovery operations cannot > be trivially failed forever. Unfortunately, the client library does not know > where the pipelined operations end and the recovery operations begin. At the > time of a connection loss, subsequent pipelined operations may or may not be > queued in the library; others might be upcoming in the application thread. If > the library re-establishes a connection too early, it will send pipelined > operations out of FIFO client order. > I considered a possible workaround of having the client diligently check its > callbacks and watchers for connection loss events, and do its best to stop > the subsequent pipelined operations at the first sign of a connection loss. > In addition to being a large burden for the application, this does not solve > the problem all the time. In particular, if the callback thread is delayed > significantly (as can happen due to excessive computation or scheduling > hiccups), the application may not learn about the connection loss event until > after the connection has been re-established and after dependent pipelined > operations have already been transmitted over the new connection. > I suggest the following API changes to fix the problem: > - Add a method ZooKeeper.getConnection() returning a ZKConnection object. > ZKConnection would wrap a TCP connection. It would include all synchronous > and asynchronous operations currently defined on the ZooKeeper class. Upon a > connection loss on a ZKConnection, all subsequent operations on the same > ZKConnection would return a Connection Loss error. Upon noticing, the client > would need to call ZooKeeper.getConnection() again to get a working > ZKConnection object, and
[jira] [Updated] (BOOKKEEPER-351) asyncAddEntry should not throw an exception
[ https://issues.apache.org/jira/browse/BOOKKEEPER-351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matteo Merli updated BOOKKEEPER-351: Attachment: 0001-BOOKKEEPER-351-asyncAddEntry-should-not-throw-an-exc.patch Attached patch with test case showing the issue > asyncAddEntry should not throw an exception > --- > > Key: BOOKKEEPER-351 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-351 > Project: Bookkeeper > Issue Type: Bug >Affects Versions: 4.2.0 >Reporter: Matteo Merli >Priority: Minor > Fix For: 4.2.0 > > Attachments: > 0001-BOOKKEEPER-351-asyncAddEntry-should-not-throw-an-exc.patch > > > There are cases where LedgerHandle.asyncAddEntry() fails with a > RuntimeException that is thrown by executor.submit(). > It should better invoke the callback with a failure result. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (BOOKKEEPER-351) asyncAddEntry should not throw an exception
Matteo Merli created BOOKKEEPER-351: --- Summary: asyncAddEntry should not throw an exception Key: BOOKKEEPER-351 URL: https://issues.apache.org/jira/browse/BOOKKEEPER-351 Project: Bookkeeper Issue Type: Bug Affects Versions: 4.2.0 Reporter: Matteo Merli Priority: Minor There are cases where LedgerHandle.asyncAddEntry() fails with a RuntimeException that is thrown by executor.submit(). It should better invoke the callback with a failure result. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira