You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@zookeeper.apache.org by "xiongjianbo (Jira)" <ji...@apache.org> on 2022/03/28 08:38:00 UTC

[jira] [Updated] (ZOOKEEPER-4502) SyncRequestProcessor leak when leader election occurred

     [ https://issues.apache.org/jira/browse/ZOOKEEPER-4502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

xiongjianbo updated ZOOKEEPER-4502:
-----------------------------------
    Description: 
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.

 

 

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

  was:
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

    Environment: 
Analysis by MAT:

401 instances of {*}"org.apache.zookeeper.server.SyncRequestProcessor"{*}, loaded by *"sun.misc.Launcher$AppClassLoader @ 0xfc000000"* occupy *48,494,208 (40.02%)* bytes. 

*Keywords*
org.apache.zookeeper.server.SyncRequestProcessor
sun.misc.Launcher$AppClassLoader @ 0xfc000000

 

 

Follower shutdown logs:

03-15 09:20:23,917 WARN (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (Follower:129) Exception when following the leader
java.io.EOFException: null
        at java.io.DataInputStream.readInt(DataInputStream.java:393) ~[?:1.8.0_322]
        at org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:96) ~[zookeeper-jute-3.6.3.jar:3.6.3]
        at org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:86) ~[zookeeper-jute-3.6.3.jar:3.6.3]
        at org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:134) ~[zookeeper-jute-3.6.3.jar:3.6.3]
        at org.apache.zookeeper.server.quorum.Learner.readPacket(Learner.java:221) ~[zookeeper-3.6.3-h0.gdd.pub.r65.jar:3.6.3]
        at org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:125) [zookeeper-3.6.3-h0.gdd.pub.r65.jar:3.6.3]
        at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1480) [zookeeper-3.6.3-h0.gdd.pub.r65.jar:3.6.3]
03-15 09:20:23,918 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (Follower:143) Disconnected from leader (with address: /192.168.18.24:26311). Was connected for 35683ms. Sync state: true
03-15 09:20:23,918 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (Follower:292) shutdown Follower
03-15 09:20:23,919 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (ZooKeeperServer:812) shutting down
03-15 09:20:23,919 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (RequestThrottler:244) Shutting down
03-15 09:20:23,919 INFO (RequestThrottler) (RequestThrottler:205) Draining request throttler queue
03-15 09:20:23,919 INFO (RequestThrottler) (RequestThrottler:181) RequestThrottler shutdown. Dropped 0 requests
03-15 09:20:23,920 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (FollowerRequestProcessor:148) Shutting down
03-15 09:20:23,920 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (CommitProcessor:617) Shutting down
03-15 09:20:23,920 INFO (FollowerRequestProcessor:1) (FollowerRequestProcessor:112) FollowerRequestProcessor exited loop!
03-15 09:20:23,920 INFO (CommitProcessor:1) (CommitProcessor:406) CommitProcessor exited loop!
03-15 09:20:23,928 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (FinalRequestProcessor:661) shutdown of request processor complete
03-15 09:20:24,298 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (QuorumPeer:864) Peer state changed: looking
03-15 09:20:24,298 WARN (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (QuorumPeer:1526) PeerState set to LOOKING
03-15 09:20:24,298 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (QuorumPeer:1396) LOOKING

> 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
>         Environment: Analysis by MAT:
> 401 instances of {*}"org.apache.zookeeper.server.SyncRequestProcessor"{*}, loaded by *"sun.misc.Launcher$AppClassLoader @ 0xfc000000"* occupy *48,494,208 (40.02%)* bytes. 
> *Keywords*
> org.apache.zookeeper.server.SyncRequestProcessor
> sun.misc.Launcher$AppClassLoader @ 0xfc000000
>  
>  
> Follower shutdown logs:
> 03-15 09:20:23,917 WARN (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (Follower:129) Exception when following the leader
> java.io.EOFException: null
>         at java.io.DataInputStream.readInt(DataInputStream.java:393) ~[?:1.8.0_322]
>         at org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:96) ~[zookeeper-jute-3.6.3.jar:3.6.3]
>         at org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:86) ~[zookeeper-jute-3.6.3.jar:3.6.3]
>         at org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:134) ~[zookeeper-jute-3.6.3.jar:3.6.3]
>         at org.apache.zookeeper.server.quorum.Learner.readPacket(Learner.java:221) ~[zookeeper-3.6.3-h0.gdd.pub.r65.jar:3.6.3]
>         at org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:125) [zookeeper-3.6.3-h0.gdd.pub.r65.jar:3.6.3]
>         at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:1480) [zookeeper-3.6.3-h0.gdd.pub.r65.jar:3.6.3]
> 03-15 09:20:23,918 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (Follower:143) Disconnected from leader (with address: /192.168.18.24:26311). Was connected for 35683ms. Sync state: true
> 03-15 09:20:23,918 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (Follower:292) shutdown Follower
> 03-15 09:20:23,919 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (ZooKeeperServer:812) shutting down
> 03-15 09:20:23,919 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (RequestThrottler:244) Shutting down
> 03-15 09:20:23,919 INFO (RequestThrottler) (RequestThrottler:205) Draining request throttler queue
> 03-15 09:20:23,919 INFO (RequestThrottler) (RequestThrottler:181) RequestThrottler shutdown. Dropped 0 requests
> 03-15 09:20:23,920 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (FollowerRequestProcessor:148) Shutting down
> 03-15 09:20:23,920 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (CommitProcessor:617) Shutting down
> 03-15 09:20:23,920 INFO (FollowerRequestProcessor:1) (FollowerRequestProcessor:112) FollowerRequestProcessor exited loop!
> 03-15 09:20:23,920 INFO (CommitProcessor:1) (CommitProcessor:406) CommitProcessor exited loop!
> 03-15 09:20:23,928 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (FinalRequestProcessor:661) shutdown of request processor complete
> 03-15 09:20:24,298 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (QuorumPeer:864) Peer state changed: looking
> 03-15 09:20:24,298 WARN (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (QuorumPeer:1526) PeerState set to LOOKING
> 03-15 09:20:24,298 INFO (QuorumPeer[myid=1](plain=disabled)(secure=192.168.18.23:26310)) (QuorumPeer:1396) LOOKING
>            Reporter: xiongjianbo
>            Priority: Major
>
> 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.
>  
>  
> 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)