Uploaded image for project: 'ZooKeeper'
  1. ZooKeeper
  2. ZOOKEEPER-3923

Leader election issues with Istio

    XMLWordPrintableJSON

    Details

    • Type: Bug
    • Status: Open
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: 3.6.1, 3.5.8
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      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:

      1. zk-1 is the leader and gets restarted
      2.  zk-2 is voted as leader
      3. zk-0 tries to connect to zk-2:2888 to sync-up. The TCP connection is established, though it's remotely closed immediately after
      4. zk-2 starts listening to port 2888 but never receives any connection
      5. zk-0, after the read error on the connection goes back into LOOKING mode, ready for a new leader election
      6. 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.

      Logs excerpt:

      ZK-2 becomes leader: 

      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: 0xf00000650 to /streamlio/zookeeper/data/version-2/snapshot.f00000650
      2020-08-27 16:40:07.249000+00:00 [INFO ] [e.zookeeper.server.ZooKeeperServer]  Snapshot taken in 30 ms
      

       

      ZK-0 is following but immediately goes back into LOOKING state:

      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=0xf00000650
      

      After timeout, ZK-2 goes back into looking and a new leader election takes place:

      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:27.251000+00:00 [INFO ] [che.zookeeper.server.quorum.Leader]  Shutting down
      2020-08-27 16:40:27.251000+00:00 [INFO ] [che.zookeeper.server.quorum.Leader]  Shutdown called. For the reason Forcing shutdown
      2020-08-27 16:40:27.251000+00:00 [INFO ] [zookeeper.server.quorum.QuorumPeer]  Peer state changed: looking
      2020-08-27 16:40:27.252000+00:00 [WARN ] [zookeeper.server.quorum.QuorumPeer]  PeerState set to LOOKING
      2020-08-27 16:40:27.252000+00:00 [INFO ] [zookeeper.server.quorum.QuorumPeer]  LOOKING
      

       

      The main issue that triggers this is that there is an intrinsic race condition between the leader to start listening on port 2888 and the followers trying to connect to it. In normal deployment scenario, the followers will get a connection refused error, on which they will be retrying up to 5 times.

      Instead, with a TCP proxy in between, the follower is able to establish the TCP connection (to the proxy) but this connection will be terminated immediately as we try to read or write on the socket, since when the proxy connects to the leader that will be failing.

      One solution here, would be to apply the same retry logic, within the boundaries of the timeout also when we get a read error and not only on connection error. 

        Attachments

          Activity

            People

            • Assignee:
              Unassigned
              Reporter:
              mmerli Matteo Merli
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:

                Time Tracking

                Estimated:
                Original Estimate - Not Specified
                Not Specified
                Remaining:
                Remaining Estimate - 0h
                0h
                Logged:
                Time Spent - 20m
                20m