rdhabalia opened a new pull request #2281: [bookie-ledger-recovery] Fix bookie recovery stuck even with enough ack-quorum response URL: https://github.com/apache/bookkeeper/pull/2281 ### Motivation As discussed at https://github.com/apache/pulsar/issues/6505 Bk-client was not able to recover ledger which has 2 write/ack quorum and one of the bookie went down and recovery was kept failing and bookkeeper client was not able to recover the ledger. **BK-Client log** ``` 20:44:43.721 [BookKeeperClientWorker-OrderedExecutor-1-0] ERROR org.apache.bookkeeper.client.ReadLastConfirmedOp - While readLastConfirmed ledger: 1234567 did not hear success responses from all quorums 20:44:43.721 [bookkeeper-io-12-27] ERROR org.apache.bookkeeper.proto.PerChannelBookieClient - Could not connect to bookie: [id: 0xb8b97441, L:/1.1.1.1:1234]/1.1.1.2:3181, current s tate CONNECTING : io.netty.channel.AbstractChannel$AnnotatedConnectException: finishConnect(..) failed: No route to host: /1.1.1.2:3181 at io.netty.channel.unix.Errors.throwConnectException(Errors.java:112) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.unix.Socket.finishConnect(Socket.java:269) ~[netty-all-4.1.32.Final.jar:4.1.32.Final] at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.doFinishConnect(AbstractEpollChannel.java:665) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:642) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:522) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:423) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:330) [netty-transport-native-epoll-4.1.31.Final.jar:4.1.31.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:897) [netty-common-4.1.31.Final.jar:4.1.31.Final] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [netty-common-4.1.31.Final.jar:4.1.31.Final] at java.lang.Thread.run(Thread.java:834) [?:?] Caused by: java.net.ConnectException: finishConnect(..) failed: No route to host ``` **Ledger metadata** ``` BookieMetadataFormatVersion 2 quorumSize: 2 ensembleSize: 2 length: 0 lastEntryId: -1 state: IN_RECOVERY segment { ensembleMember: "1.1.1.1:3181" ensembleMember: "1.1.1.2:3181" firstEntryId: 0 } digestType: CRC32 ``` **Root cause:** Bookie should be able to recover ledger once it receives the response from total N (`(Qw - Qa)+1`) bookies. But it was waiting for a successful response from both quorums. Reference: https://bookkeeper.apache.org/docs/4.5.0/development/protocol/ ### Modification Bookie should be able to recover ledger once it receives the response from total N (`(Qw - Qa)+1`) bookies.
---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
