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

David Bros commented on ZOOKEEPER-4444:
---------------------------------------

Hi,

I found the exact issue by enabling the following property in my 
*log4j.properties*
 * log4j.logger.org.apache.zookeeper=DEBUG

After setting that and restarting my zookeeper which was not synchronizing my 
brokers information I found this in *server.log*

 
{noformat}
[2022-02-01 11:22:00,346] DEBUG Created new input archive: 
/var/log/zookeeper/version-2/log.4a000084d6 
(org.apache.zookeeper.server.persistence.FileTxnLog)
[2022-02-01 11:22:00,349] DEBUG EOF exception 
(org.apache.zookeeper.server.persistence.FileTxnLog)
java.io.EOFException: Failed to read /var/log/zookeeper/version-2/log.4a000084d6
    at 
org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.next(FileTxnLog.java:771)
    at 
org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.<init>(FileTxnLog.java:650)
    at 
org.apache.zookeeper.server.persistence.FileTxnLog.read(FileTxnLog.java:462)
    at 
org.apache.zookeeper.server.persistence.FileTxnLog.read(FileTxnLog.java:449)
    at 
org.apache.zookeeper.server.persistence.FileTxnLog.getLastLoggedZxid(FileTxnLog.java:360)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:277)
    at org.apache.zookeeper.server.ZKDatabase.loadDataBase(ZKDatabase.java:285)
    at 
org.apache.zookeeper.server.quorum.QuorumPeer.loadDataBase(QuorumPeer.java:1094)
    at org.apache.zookeeper.server.quorum.QuorumPeer.start(QuorumPeer.java:1079)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.runFromConfig(QuorumPeerMain.java:227)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.initializeAndRun(QuorumPeerMain.java:136)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.main(QuorumPeerMain.java:90)
[2022-02-01 11:22:00,351] DEBUG Created new input stream: 
/var/log/zookeeper/version-2/log.4a000084dc 
(org.apache.zookeeper.server.persistence.FileTxnLog)
[2022-02-01 11:22:00,351] DEBUG Created new input archive: 
/var/log/zookeeper/version-2/log.4a000084dc 
(org.apache.zookeeper.server.persistence.FileTxnLog)
[2022-02-01 11:22:00,351] DEBUG EOF exception 
(org.apache.zookeeper.server.persistence.FileTxnLog)
java.io.EOFException: Failed to read /var/log/zookeeper/version-2/log.4a000084dc
    at 
org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.next(FileTxnLog.java:771)
    at 
org.apache.zookeeper.server.persistence.FileTxnLog.getLastLoggedZxid(FileTxnLog.java:362)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:277)
    at org.apache.zookeeper.server.ZKDatabase.loadDataBase(ZKDatabase.java:285)
    at 
org.apache.zookeeper.server.quorum.QuorumPeer.loadDataBase(QuorumPeer.java:1094)
    at org.apache.zookeeper.server.quorum.QuorumPeer.start(QuorumPeer.java:1079)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.runFromConfig(QuorumPeerMain.java:227)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.initializeAndRun(QuorumPeerMain.java:136)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.main(QuorumPeerMain.java:90)
[2022-02-01 11:22:00,352] WARN No snapshot found, but there are log entries. 
This should only be allowed during upgrading. 
(org.apache.zookeeper.server.persistence.FileTxnSnapLog)
[2022-02-01 11:22:00,352] DEBUG Created new input stream: 
/var/log/zookeeper/version-2/log.4000000001 
(org.apache.zookeeper.server.persistence.FileTxnLog)
[2022-02-01 11:22:00,352] DEBUG Created new input archive: 
/var/log/zookeeper/version-2/log.4000000001 
(org.apache.zookeeper.server.persistence.FileTxnLog)
[2022-02-01 11:22:00,377] DEBUG Failed: 
218688867411165184,4,274877906946,1607953214895,5
:'/config/topics/netflow_pre_agg,#7b2276657273696f6e223a312c22636f6e666967223a7b22726574656e74696f6e2e6d73223a223836343030303030227d7d,1
 (org.apache.zookeeper.server.DataTree)
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
    at org.apache.zookeeper.server.DataTree.setData(DataTree.java:659)
    at org.apache.zookeeper.server.DataTree.processTxn(DataTree.java:955)
    at org.apache.zookeeper.server.DataTree.processTxn(DataTree.java:873)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.processTransaction(FileTxnSnapLog.java:433)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.fastForwardFromEdits(FileTxnSnapLog.java:341)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.lambda$restore$0(FileTxnSnapLog.java:258)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:284)
    at org.apache.zookeeper.server.ZKDatabase.loadDataBase(ZKDatabase.java:285)
    at 
org.apache.zookeeper.server.quorum.QuorumPeer.loadDataBase(QuorumPeer.java:1094)
    at org.apache.zookeeper.server.quorum.QuorumPeer.start(QuorumPeer.java:1079)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.runFromConfig(QuorumPeerMain.java:227)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.initializeAndRun(QuorumPeerMain.java:136)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.main(QuorumPeerMain.java:90)
[2022-02-01 11:22:00,379] DEBUG Ignoring processTxn failure hdr: 5 : error: 
-101 (org.apache.zookeeper.server.DataTree)
[2022-02-01 11:22:00,379] DEBUG Ignoring processTxn failure hdr: 5, error: 
-101, path: /config/topics/netflow_pre_agg 
(org.apache.zookeeper.server.persistence.FileTxnSnapLog)
[2022-02-01 11:22:00,379] DEBUG Ignoring processTxn failure hdr: -1 : error: 
-110 (org.apache.zookeeper.server.DataTree)
[2022-02-01 11:22:00,379] DEBUG Ignoring processTxn failure hdr: -1, error: 
-110, path: null 
(org.apache.zookeeper.server.persistence.FileTxnSnapLog){noformat}
 

 

After the last exception java runs into the processTxn failure with error: 
-101, there are some more errors below regarding the fact that zookeeper will 
try to access my clusters information to decide quorum (I think, i'm not really 
sure)

 

 
{noformat}
[2022-02-01 11:22:00,384] DEBUG Ignoring processTxn failure hdr: 2, error: 
-101, path: /config/changes/config_change_0000000013 
(org.apache.zookeeper.server.persistence.FileTxnSnapLog)
[2022-02-01 11:22:00,385] DEBUG Failed: 
144115191103946754,731,274877906951,1607953215001,2
:'/config/changes/config_change_0000000014
 (org.apache.zookeeper.server.DataTree)
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
    at org.apache.zookeeper.server.DataTree.deleteNode(DataTree.java:576)
    at org.apache.zookeeper.server.DataTree.processTxn(DataTree.java:949)
    at org.apache.zookeeper.server.DataTree.processTxn(DataTree.java:873)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.processTransaction(FileTxnSnapLog.java:433)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.fastForwardFromEdits(FileTxnSnapLog.java:341)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.lambda$restore$0(FileTxnSnapLog.java:258)
    at 
org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:284)
    at org.apache.zookeeper.server.ZKDatabase.loadDataBase(ZKDatabase.java:285)
    at 
org.apache.zookeeper.server.quorum.QuorumPeer.loadDataBase(QuorumPeer.java:1094)
    at org.apache.zookeeper.server.quorum.QuorumPeer.start(QuorumPeer.java:1079)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.runFromConfig(QuorumPeerMain.java:227)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.initializeAndRun(QuorumPeerMain.java:136)
    at 
org.apache.zookeeper.server.quorum.QuorumPeerMain.main(QuorumPeerMain.java:90){noformat}
The weird thing is that after about 130MB of errors quorum was still reached, 
leader recognised it as a follower and so did itself, as a matter of fact Zxid 
was still being updated, even with my kafka clusters information missing.

This happened after upgrading from 3.4 to 3.6.3 and I don't know what are the 
conditions for it since I had upgraded 6 znodes before this one failed, all of 
them with the following specs, including the one that failed:
 * Centos7 kernel v3.10.0
 * Scala 2.13
 * Java 17.0.1 (openjdk-headless)
 * Zookeeper 3.6.3

*I solved it by deleting zookeeper's data directory* with the following steps:
 * Stop zookeeper service
 * Remove data directory, in my case under /var/log/zookeeper ``` rm -rf 
/var/log/zookeeper/ ```
 * Start zookeeper service

After that zookeeper quorum was reached again without issues and the sync 
underwent without issues.

Also remember to delete your DEBUG property otherwise logs explode in size.

Hope this helps!

 

 

 

 

 

> Follower doesn't get synchronized after process restart
> -------------------------------------------------------
>
>                 Key: ZOOKEEPER-4444
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4444
>             Project: ZooKeeper
>          Issue Type: Bug
>    Affects Versions: 3.6.3
>            Reporter: Andreas Weber
>            Priority: Major
>
> Hi folks, I've got an issue with 3.6.3.
> I can't provide a simple test, because it seems to depend on timing in a 
> cluster environment, but I tried to reduce the scenario as far as possible:
>  * zookeeper cluster with 5 nodes, all of them Followers (no Observers)
>  * start some parallel threads which do some writes in the cluster (e.g. 
> create/delete znodes)
>  * kill one of the zookeeper processes, let's say on node X (where node X is 
> not the Leader)
>  * restart zookeeper process on node X
>  * wait a few seconds
>  * kill zookeeper process on node X again
>  * restart zookeeper process on node X again
> In some cases (every 3-4 runs) I see the following in the log of node X:
> After first restart of node X:
> {noformat}
>  WARN  persistence.FileTxnLog           - Current zxid 4294968525 is <= 
> 4294969524 for 15
>  WARN  persistence.FileTxnLog           - Current zxid 4294968526 is <= 
> 4294969524 for 15
>  WARN  persistence.FileTxnLog           - Current zxid 4294968527 is <= 
> 4294969524 for 15
>  ... (this kind of WARN is repeated some hundred times)
>  WARN  quorum.SendAckRequestProcessor   - Closing connection to leader, 
> exception during packet send java.net.SocketException: Socket closed ...
>  ... (this kind of WARN is repeated some hundred times)
> {noformat}
> After second restart of node X:
> {noformat}
>  ERROR persistence.FileTxnSnapLog       - 4294970146(highestZxid) > 
> 4294969147(next log) for type 2
>  WARN  server.DataTree                  - Message:Digests are not matching. 
> Value is Zxid. Value:4294969147
>  ERROR server.DataTree                  - First digest mismatch on txn: 
> 360466402305310720,3870,4294969147,1639258399998,2
> , ...
> , expected digest is 2,1365261838770
> , actual digest is 1098406565142, 
>  ERROR persistence.FileTxnSnapLog       - 4294970146(highestZxid) > 
> 4294969148(next log) for type 2
>  ERROR persistence.FileTxnSnapLog       - 4294970146(highestZxid) > 
> 4294969149(next log) for type 5
>  ERROR persistence.FileTxnSnapLog       - 4294970146(highestZxid) > 
> 4294969150(next log) for type 2
>  ... (this kind of ERROR is repeated some hundred times)
> {noformat}
> And afterwards (in the actual application), zookepeer on node X seems to have 
> a different view of the cluster state and doesn't get synchronized, at least 
> for a few hours.
> This e.g. leads to phantom reads of znodes that were deleted a long time ago.
> (The resulting behaviour looks a little bit similar as described in 
> ZOOKEEPER-3911.)
> This does not happen with zookeeper 3.6.2 !
> (at least I can't reproduce it with this version)



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to