There is a new finding about it.

https://github.com/apache/bookkeeper/pull/2805 also introduces deadlock in 
ZkLedgerUnderreplicationManager.


## ReplicationWorker mechanism
The working mechanism of ReplicationWorker is as follows: it retrieves an 
under-replicated ledger, transfers the data of that ledger to a new bookie, and 
then marks the ledger as replicated. It then proceeds to fetch the next 
under-replicated ledger and repeats the process.

When all under-replicated ledgers have been marked as replicated, and no new 
under-replicated ledgers are available, the ReplicationWorker will register a 
`getChildren` watcher for the under-replicated ledger's ZooKeeper (zk) node. It 
then blocks and waits until a new ledger is marked as under-replicated and 
receives a NodeChildrenChanged event from zk. Once the event is received, it 
cancels the block and resumes processing.

According to the thread stack info, it show that the Replication is blocked due 
to waiting zk event notification.
```
"ReplicationWorker" #25 prio=5 os_prio=0 cpu=2166.58ms elapsed=10643.50s 
tid=0x00007f4ead93c720 nid=0xab waiting on condition  [0x00007f4bfbafe000]
   java.lang.Thread.State: WAITING (parking)
        at jdk.internal.misc.Unsafe.park(java.base@17.0.7/Native Method)
        - parking to wait for  <0x000010001c40ad30> (a 
java.util.concurrent.CountDownLatch$Sync)
        at 
java.util.concurrent.locks.LockSupport.park(java.base@17.0.7/LockSupport.java:211)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(java.base@17.0.7/AbstractQueuedSynchronizer.java:715)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(java.base@17.0.7/AbstractQueuedSynchronizer.java:1047)
        at 
java.util.concurrent.CountDownLatch.await(java.base@17.0.7/CountDownLatch.java:230)
        at 
org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager.getLedgerToRereplicate(ZkLedgerUnderreplicationManager.java:611)
        at 
org.apache.bookkeeper.replication.ReplicationWorker.rereplicate(ReplicationWorker.java:296)
        at 
org.apache.bookkeeper.replication.ReplicationWorker.run(ReplicationWorker.java:249)
        at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.run(java.base@17.0.7/Thread.java:833)
```

In theory, if a new ledger is marked as under-replicated, the block will be 
canceled. However, in the user's environment, even if a new ledger has already 
been marked as under-replicated, it is still blocked.

## Root case
After introducing https://github.com/apache/bookkeeper/pull/2805, we found that 
the ReplicationWorker can't received the  NodeChildrenChanged zk event, so the 
ReplicationWorker block forever.

In https://github.com/apache/bookkeeper/pull/2805, it register a Persistent, 
Recursive watcher for under-replicated ledger path, the path is same with 
ReplicationWorker child listener path. When the same node has both persistent 
and recursive watchers, as well as a getChildren watcher, the ZooKeeper server, 
during event notifications, may ignore the NodeChildrenChanged event and only 
push NodeCreated events. As a result, the ReplicationWorker is unable to 
receive the NodeChildrenChanged event, leading to the block not being canceled 
for the user.

The zookeeper docs: 
https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#ch_zkWatches
See `Persistent, Recursive Watches` section.

> Persistent, Recursive Watches
New in 3.6.0: There is now a variation on the standard watch described above 
whereby you can set a watch that does not get removed when triggered. 
Additionally, these watches trigger the event types NodeCreated, NodeDeleted, 
and NodeDataChanged and, optionally, recursively for all znodes starting at the 
znode that the watch is registered for. Note that NodeChildrenChanged events 
are not triggered for persistent recursive watches as it would be redundant.

Reply via email to