Re: Remove UnderReplicatedLedgersChangedCb in Auditor

2023-09-05 Thread Yan Zhao
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=0x7f4ead93c720 nid=0xab waiting on condition  [0x7f4bfbafe000]
   java.lang.Thread.State: WAITING (parking)
at jdk.internal.misc.Unsafe.park(java.base@17.0.7/Native Method)
- parking to wait for  <0x10001c40ad30> (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.



Re: Remove UnderReplicatedLedgersChangedCb in Auditor

2023-08-22 Thread Yan Zhao
> Overall looks good to me.
> 
> Only one thing I want to mention, should we use a new way to fix it,
> not only remove the original method?
> Because the issue indeed exists, and we need to fix it.
> 
> Thanks,
> Yong


Yes, I would like to use a scheduled task to trigger it.


Re: Data loss problem when DefaultEntryLogger switch to DirectEntryLogger, then switch back to DefaultEntryLogger.

2023-07-31 Thread Yan Zhao
> You should never do this in production.

But the user may not know the limitation, they may believe that the switching 
is compatible, not introduce some problems. 

> BTW do you have a possible solution?

see https://github.com/apache/bookkeeper/pull/4041


Re: Data loss problem when DefaultEntryLogger switch to DirectEntryLogger, then switch back to DefaultEntryLogger.

2023-07-30 Thread Yan Zhao
I am concerned about a possible scenario where a user tries to use DirectIO, 
runs it for a period of time, and experiences some problem on the bookie server 
side, but is unsure of the cause and suspects it may be related to DirectIO. As 
a result, the user disables DirectIO and restarts. In this case, data may be 
lost.



On 2023/07/30 17:40:25 Enrico Olivelli wrote:
> I think that you should not do this kind of things.
> You should start a new bookie with the new configuration.
> 
> It is not generally possible to guarantee compatibility between different
> implementations.
> 
> I tend to close the case as 'won't fix'
> 
> 
> Enrico
> 
> 
> Il Dom 30 Lug 2023, 13:00 horizonzy  ha scritto:
> 
> > Hi, the followers.
> > I would need to talk about a serious matter. Especially, if you use
> > DirectEntryLogger in a production environment.
> >
> > Switching from DefaultEntryLogger to DirectEntryLogger and switching back
> > from DirectEntryLogger to DefaultEntryLogger may result in data loss.
> >
> > Here is the detailed issue
> > https://github.com/apache/bookkeeper/issues/4040
> >
> 


Re: Improve DefaultEntryLogger read performance

2023-07-30 Thread Yan Zhao
The PR link: https://github.com/apache/bookkeeper/pull/4038


Re: [ANNOUNCE] New committer Wenbing Shen

2023-07-08 Thread Yan Zhao
Wenbing, Congratulatiton! 

On 2023/07/08 12:39:14 Enrico Olivelli wrote:
> Hello Bookkeepers,
> 
> The Project Management Committee (PMC) for Apache BookKeeper
> has invited Wenbing Shen to become a committer and we are pleased
> to announce that they have accepted.
> 
> Being a committer enables easier contribution to the
> project since there is no need to go via the patch
> submission process. This should enable better productivity.
> 
> For more information about the ASF works you can follow this link
> https://www.apache.org/foundation/how-it-works.html
> 
> Please join me and congratulate Wenbing Shen !
> 
> Enrico Olivelli
> 


Re: The CI tests didn't cover bookkeeper V2 protocol.

2023-02-21 Thread Yan Zhao
I submit https://github.com/apache/bookkeeper/pull/3794 for it. Please help to 
view it when you are in inconvenient.


Re: CI shows successfully but the tests actually has errors

2023-02-20 Thread Yan Zhao
>  Hi Yong,
>  Thanks for finding this critical issue. I think it's dangerous to
> merge new PRs before the CI is fixed, because we can't ensure whether
> the new PRs introduces bugs.

+1


Re: The CI tests didn't cover bookkeeper V2 protocol.

2023-02-20 Thread Yan Zhao
> According to the original PR's motivation
> , we wrapped a Netty
> allocator
> and want to configure something through bookkeeper.
> So the user will use our customized allocator and need to obey the rules
> we introduced. Then the Netty's configuration `io.netty.leakDetection.level`
> seems useless for us. I think they would never have a chance to take it
> differently.
> 
> We defined a new configuration property to configure the
> LeakDetectionPolicy.
> 
> 
> 
> 
> *public LeakDetectionPolicy getAllocatorLeakDetectionPolicy() {
> return LeakDetectionPolicy
> .valueOf(this.getString(ALLOCATOR_LEAK_DETECTION_POLICY,
> LeakDetectionPolicy.Paranoid.toString()));}*
> 
> Some users may don't know the `*ALLOCATOR_LEAK_DETECTION_POLICY`*
> and only configure* `*io.netty.leakDetection.level` to open the leak
> detection.
> I would suggest getting both `*ALLOCATOR_LEAK_DETECTION_POLICY`*
> and `io.netty.leakDetection.level` from the system property, use the highest
> policy of it. And then, no matter how you configured it, the detection
> policy always worked.

Agree.


Re: The CI tests didn't cover bookkeeper V2 protocol.

2023-02-19 Thread Yan Zhao
> Make sense. 
> There are second level config for the memory detection. The first level is 
> netty jvm param `-Dio.netty.leakDetection.level`, the second level is 
> bookkeeper config `AbstractConfiguration#setAllocatorLeakDetectionPolicy`. 
> 
> If the second level be config greater than `Disabled`, the second level will 
> override the first level. 
> If the second level is `Disabled`, we use the first level config.

The default detection level in netty is `Simple`, the default detection level 
in bookkeeper is `Disabled`. If the user didn't config the bookkeeper level 
detection policy, it will use netty default detection level  `Simple`, it's a 
breaking change.

So I suggest that we introduce a new jvm param `preferNettyLeakDetectionPolicy` 
in bookkeeper, the default value is `false`.

If the user config `-DpreferNettyReLeakDetectionPolicy=true`, the bookkeeper 
leak detection policy won't override the netty config. If the ci tests, we 
config `-Dio.netty.leakDetection.level=paranoid 
-DpreferNettyLeakDetectionPolicy=true`, it will use netty leak detection level 
`paranoid`.


Re: The CI tests didn't cover bookkeeper V2 protocol.

2023-02-19 Thread Yan Zhao
> IMO, this is not expected behavior. In our test, both the client and
> server's memory leak detection policy should be gotten from
> `-Dio.netty.leakDetection.level` configuration instead of hard code as
> `LeakDetectionPolicy.Paranoid`

Make sense. 
There are second level config for the memory detection. The first level is 
netty jvm param `-Dio.netty.leakDetection.level`, the second level is 
bookkeeper config `AbstractConfiguration#setAllocatorOutOfMemoryPolicy`. 

If the second level be config greater than `Disabled`, the second level will 
override the first level. 
If the second level is `Disabled`, we use the first level config.



Re: [DISCUSS] BP-61: Revert BP-59 to release ByteBuf using ByteBuf.release() instead of ReferenceCountUtil.safeRelease()

2023-02-02 Thread Yan Zhao
> I think that we must do something, current master branch is not stable.
> 
> My colleague Massimiliano opened this issue
> https://github.com/apache/bookkeeper/issues/3751
> 
> Basically in the current master there is some problem that leads to
> Netty BytBuf corruption.
> 
> The problem is solved by reverting this PR
> https://github.com/apache/bookkeeper/pull/3528 Fix memory leak when
> reading entry but the connection disconnected
> 
> Enrico

Hi, Enrico. We found that the pulsar timeout may not be introduced by 
https://github.com/apache/bookkeeper/pull/3528. Only roll back the pulsar 
version and still maintain https://github.com/apache/bookkeeper/pull/3528, the 
problem is also solved.

I guess the bk thread hangs for some reason in pulsar, the bk server handles 
the request successfully and sends the response to the bk client, but the bk 
thread hangs, it didn't handle the response, so timeout.

Massimiliano is tracing it at pulsar side.


Re: BP-54: Repaired the ledger fragment which ensemble not adhere placement policy.

2022-08-01 Thread Yan Zhao
Could you describe detail.

On 2022/08/01 17:00:47 steven lu wrote:
> I feel that the code is not the same as the discussion:
> https://github.com/apache/bookkeeper/pull/3359


Re: BP-54: Repaired the ledger fragment which ensemble not adhere placement policy.

2022-07-29 Thread Yan Zhao
You means we define api in EnsemblePlacementPolicy, and use SPI to load special 
implementation?
 
On 2022/07/29 15:10:05 Enrico Olivelli wrote:
> I support this feature.
> 
> We should design the changes to the APIs.
> We need to change EnsemblePlacementPolicy, that is a pluggable API:
> users can provide. their own implementation


Re: BP-54: Repaired the ledger fragment which ensemble not adhere placement policy.

2022-07-29 Thread Yan Zhao
@Enrico