Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ReplicationWorker not work problem. #4012

Open
horizonzy opened this issue Jul 3, 2023 · 0 comments · May be fixed by #4013
Open

ReplicationWorker not work problem. #4012

horizonzy opened this issue Jul 3, 2023 · 0 comments · May be fixed by #4013
Labels

Comments

@horizonzy
Copy link
Member

horizonzy commented Jul 3, 2023

Problem

In our customer prd, they found there are many repeat logs.

2023-06-30T01:14:01,252+0000 [ReplicationWorker] ERROR org.apache.bookkeeper.replication.ReplicationWorker - ReplicationWorker failed to replicate Ledger : 46095 for 0 number of times, so deferring the ledger lock release by 9375 msecs
2023-06-30T01:14:01,255+0000 [main-EventThread] INFO  org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager - Latch countdown due to ZK event: WatchedEvent state:SyncConnected type:NodeChildrenChanged path:/ledgers/underreplication/locks
2023-06-30T01:14:01,257+0000 [ReplicationWorker] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to find 1 bookies : excludeBookies [<Bookie:bookie-2:3181>, <Bookie:bookie-1:3181>, <Bookie:bookie-0:3181>], allBookies [<Bookie:bookie-0:3181>, <Bookie:bookie-1:3181>, <Bookie:bookie-2:3181>].
2023-06-30T01:14:01,257+0000 [ReplicationWorker] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to choose a bookie: excluded [<Bookie:bookie-2:3181>, <Bookie:bookie-1:3181>, <Bookie:bookie-0:3181>], fallback to choose bookie randomly from the cluster.
2023-06-30T01:14:01,257+0000 [ReplicationWorker] WARN  org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl - Failed to find 1 bookies : excludeBookies [<Bookie:bookie-2:3181>, <Bookie:bookie-1:3181>, <Bookie:bookie-0:3181>], allBookies [<Bookie:bookie-1:3181>, <Bookie:bookie-2:3181>, <Bookie:bookie-0:3181>].
2023-06-30T01:14:01,257+0000 [ReplicationWorker] WARN  org.apache.bookkeeper.replication.ReplicationWorker - BKNotEnoughBookiesException while replicating the fragment
org.apache.bookkeeper.client.BKException$BKNotEnoughBookiesException: Not enough non-faulty bookies available
        at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectRandomInternal(RackawareEnsemblePlacementPolicyImpl.java:781) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectRandom(RackawareEnsemblePlacementPolicyImpl.java:698) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectFromNetworkLocation(RackawareEnsemblePlacementPolicyImpl.java:587) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.selectFromNetworkLocation(RackawareEnsemblePlacementPolicy.java:206) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectFromNetworkLocation(RackawareEnsemblePlacementPolicyImpl.java:547) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.selectFromNetworkLocation(RackawareEnsemblePlacementPolicy.java:227) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.replaceBookie(RackawareEnsemblePlacementPolicyImpl.java:475) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.replaceBookie(RackawareEnsemblePlacementPolicy.java:120) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.BookKeeperAdmin.getReplacementBookiesByIndexes(BookKeeperAdmin.java:1088) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.client.BookKeeperAdmin.replicateLedgerFragment(BookKeeperAdmin.java:1137) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.replication.ReplicationWorker.rereplicate(ReplicationWorker.java:387) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.replication.ReplicationWorker.rereplicate(ReplicationWorker.java:279) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at org.apache.bookkeeper.replication.ReplicationWorker.run(ReplicationWorker.java:230) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.87.Final.jar:4.1.87.Final]
        at java.lang.Thread.run(Thread.java:829) ~[?:?]

For ledger 46095, we get the metadata by the bookkeeper shell

bin/bookkeeper shell ledgermetadata -ledgerid 46095

2023-06-30T16:19:10,233+0000 [main] INFO  org.apache.bookkeeper.tools.cli.commands.client.LedgerMetaDataCommand - LedgerMetadata{formatVersion=3, ensembleSize=3, writeQuorumSize=3, ackQuorumSize=2, state=CLOSED, length=192, lastEntryId=3, digestType=CRC32, password=base64:, ensembles={0=[bookie-0:3181, bookie-2:3181, bookie-1:3181]}, customMetadata={}}

The ledger ensemble is {0=[bookie-0:3181, bookie-2:3181, bookie-1:3181]}, and the LAC is 3.

Then we try to read entries 0-3 from each bookie.

Read from bookie-0:3181

bin/bookkeeper shell readledger -bookie bookie-0:3181 -msg -ledgerid 46095 -fe 0 -le 4
2023-06-30T09:59:09,866+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - --------- Lid=46095, Eid=0 ---------
2023-06-30T09:59:09,867+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Data:    xxx

2023-06-30T09:59:09,872+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - --------- Lid=46095, Eid=1 ---------
2023-06-30T09:59:09,873+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Data:  xxx
2023-06-30T09:59:09,874+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - --------- Lid=46095, Eid=2 ---------
2023-06-30T09:59:09,875+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Data:     xxx
2023-06-30T09:59:09,876+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - --------- Lid=46095, Eid=3 ---------
2023-06-30T09:59:09,876+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Data:    xxx

Read from bookie-1:3181

bin/bookkeeper shell readledger -bookie bookie-1:3181 -msg -ledgerid 46095 -fe 0 -le 4

2023-06-30T09:58:50,806+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - --------- Lid=46095, Eid=0 ---------
2023-06-30T09:58:50,807+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Data:    xxx
2023-06-30T09:58:50,809+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - --------- Lid=46095, Eid=1 ---------
2023-06-30T09:58:50,810+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Data:     xxx
2023-06-30T09:58:50,811+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - --------- Lid=46095, Eid=2 ---------
2023-06-30T09:58:50,812+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Data:     xxx
2023-06-30T09:58:50,813+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - --------- Lid=46095, Eid=3 ---------
2023-06-30T09:58:50,813+0000 [BookieClientScheduler-OrderedExecutor-0-0] INFO  org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Data:    xxx

Read from bookie-2:3181

bin/bookkeeper shell readledger -bookie bookie-2:3181 -msg -ledgerid 46095 -fe 0 -le 4
2023-06-30T09:59:24,094+0000 [main] ERROR org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand - Error future.get while reading entries from ledger 46095
java.util.concurrent.ExecutionException: org.apache.bookkeeper.client.BKException$BKNoSuchLedgerExistsException: No such ledger exists on Bookies
	at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:395) ~[?:?]
	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1999) ~[?:?]
	at org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand.lambda$readledger$1(ReadLedgerCommand.java:215) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
	at java.util.stream.Streams$RangeLongSpliterator.forEachRemaining(Streams.java:228) ~[?:?]
	at java.util.stream.LongPipeline$Head.forEach(LongPipeline.java:577) ~[?:?]
	at org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand.readledger(ReadLedgerCommand.java:192) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
	at org.apache.bookkeeper.tools.cli.commands.bookie.ReadLedgerCommand.apply(ReadLedgerCommand.java:136) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
	at org.apache.bookkeeper.bookie.BookieShell$ReadLedgerEntriesCmd.runCmd(BookieShell.java:633) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
	at org.apache.bookkeeper.bookie.BookieShell$MyCommand.runCmd(BookieShell.java:237) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
	at org.apache.bookkeeper.bookie.BookieShell.run(BookieShell.java:2281) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]
	at org.apache.bookkeeper.bookie.BookieShell.main(BookieShell.java:2372) ~[org.apache.bookkeeper-bookkeeper-server-4.14.7.jar:4.14.7]

Ledger 46095 does not exist in bookie-2:3181.

Root cause

The config E:W:A=3:3:2, that means: "Once the entry is written to two bookies, the write request is considered successful."

So ledger 46095 just be stored at bookie-0:3181 and bookie-1:3181.

When the AutoRecovery works, it finds that ledger 46095 does not exist in bookie-2:3181, it will start the replication process to copy 46095 to a new node. However, there are only three bookies in the entire cluster [bookie-0:3181, bookie-1:3181, bookie-2:3181].

In the current logic, when selecting a new node, all ensemble bookies of the current ledger 46095 are excluded, so there are no new nodes available. That's why we see the exception message."

Action

If there are no extra bookies available at present, we can downgrade: the node to be replaced is not down, and we can re-replicate the data to itself.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging a pull request may close this issue.

1 participant