deepakkarki opened a new issue, #4049: URL: https://github.com/apache/bookkeeper/issues/4049
I have a Bookkeeper setup that uses EBS on Kubernetes. To reduce write latencies it was suggested to me on the bookkeeper slack to use the "ack before sync" mode, or the `writeFlags.DEFERRED_SYNC` flag for the ledger. But when I enabled the the `DEFERRED_SYNC` locally, some of my tests started failing. One test in particular that did the following - ``` 1. Create a new ledger 2. Write 100 entries to it 3. Wait for 100 acks 4. Close the ledger 5. Make sure we can read back 100 entires on a (recovery) read ``` The test was failing the read path, claiming the "ledger was empty". I started debugging, and upon further reading the source code I found out (to the best of my knowledge) it was because the following was happening - ``` 1. The ledgers were being written to and the acks were getting sent - as expected. 2. Once we collected the acks, we closed the ledger. At this point the local state of the ledgerHandle still had the LAC as `-1` (and not 99) 3. Closing makes a metadata update, marking the ledger status as closed and setting LAC = -1 4. When we try to read, reader gets the metadata from ZK, sees that LAC=-1 and assumes the ledger is empty. ``` Now step 2 was the surprising bit because without the `DEFERRED_SYNC` flag, the LAC does get updated locally with every ack, so digging further I found it this was happening because of [this piece of code](https://github.com/apache/bookkeeper/blob/f30ff4f2ad4778f1f73b29872e2a95adb22ca116/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L1834-L1836) - ```java // In Bookkeeper client - LedgerHandle.java (sendAddSuccessCallbacks) if (!writeFlags.contains(WriteFlag.DEFERRED_SYNC)) { this.lastAddConfirmed = pendingAddsSequenceHead; } ``` Looks like the LAC is consciously not being updated for some reason. I tried to [look back at the PR](https://github.com/apache/bookkeeper/pull/853/) that introduced this change and could not find a hint as to why this could be. But getting some more context I realized doing a `ledgerHandle.force()` before a `close()` does fix this issue. But this whole experience did leave a few questions unanswered, I would be grateful if someone here can answer them - 1. Why does BK not update LAC when `DEFERRED_SYNC` is set? I guess it is to have some reliability guarantees behind the LAC id? (Even then I think the force() should be implicit when closing a ledger which has `DEFERRED_SYNC` set) 2. If the above is for reliability / data sanity reasons - why is there no similar behavior when [syncData flag](https://github.com/apache/bookkeeper/blob/f30ff4f2ad4778f1f73b29872e2a95adb22ca116/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java#L617-L618) is false? It has pretty similar semantics, and also when `syncData` is set to false it triggers the same code path as `DEFERRED_SYNC`. 3. Also this breaks our tailing (non recovery) reads. Nothing can be tailed until the ledger is closed and updated with a LAC. ---- I apologize if I have some fundamental misunderstanding of the system - I'm still pretty new to Bookkeeper! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
