commit | 025d99f5a2a4cc02f3780a11b58a9b9d6c9940c3 | [log] [tgz] |
---|---|---|
author | hangc0276 <hangc0276@163.com> | Tue Mar 10 21:17:53 2020 +0800 |
committer | GitHub <noreply@github.com> | Tue Mar 10 14:17:53 2020 +0100 |
tree | 1ea7e64682084fec19121997d3cb5da9d63c3b02 | |
parent | 92f75da3f8711cb98eda2f435bbd57ffe67e87b7 [diff] |
fix bookie decommission sleep timeout value is negative bug when decommission a bookie, and the ledger size of the bookie is big enough, the thread timeout will get negative, and the decommission operation will give up by throw exceptions as follow ``` 14:12:56.982 [main] INFO org.apache.bookkeeper.client.BookKeeperAdmin - Count of Ledgers which need to be rereplicated: 272752 14:12:56.983 [main] ERROR org.apache.bookkeeper.bookie.BookieShell - Received exception in DecommissionBookieCmd java.lang.IllegalArgumentException: timeout value is negative at java.lang.Thread.sleep(Native Method) ~[?:?] at org.apache.bookkeeper.client.BookKeeperAdmin.waitForLedgersToBeReplicated(BookKeeperAdmin.java:1528) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.client.BookKeeperAdmin.decommissionBookie(BookKeeperAdmin.java:1500) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.bookie.BookieShell$DecommissionBookieCmd.runCmd(BookieShell.java:2664) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.bookie.BookieShell$MyCommand.runCmd(BookieShell.java:277) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.bookie.BookieShell.run(BookieShell.java:3081) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] at org.apache.bookkeeper.bookie.BookieShell.main(BookieShell.java:3172) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2] 14:12:57.013 [main] INFO org.apache.zookeeper.ZooKeeper - Session: 0x206189927840052 closed ``` The exception code is ``` private void waitForLedgersToBeReplicated(Collection<Long> ledgers, BookieSocketAddress thisBookieAddress, LedgerManager ledgerManager) throws InterruptedException, TimeoutException { int maxSleepTimeInBetweenChecks = 10 * 60 * 1000; // 10 minutes int sleepTimePerLedger = 10 * 1000; // 10 secs Predicate<Long> validateBookieIsNotPartOfEnsemble = ledgerId -> !areEntriesOfLedgerStoredInTheBookie(ledgerId, thisBookieAddress, ledgerManager); while (!ledgers.isEmpty()) { LOG.info("Count of Ledgers which need to be rereplicated: {}", ledgers.size()); int sleepTimeForThisCheck = ledgers.size() * sleepTimePerLedger > maxSleepTimeInBetweenChecks ? maxSleepTimeInBetweenChecks : ledgers.size() * sleepTimePerLedger; Thread.sleep(sleepTimeForThisCheck); LOG.debug("Making sure following ledgers replication to be completed: {}", ledgers); ledgers.removeIf(validateBookieIsNotPartOfEnsemble); } } ``` the ledger size is `272752`, when computing sleepTimeForThisCheck, `ledgers.size() * sleepTimePerLedger` is `272752 * 10 * 1000 = 2727520000`, the value exceeds max int value `2147483647`, it will turn to `-1567447296`, then the sleepTimeForThisCheck will be `-1567447296`. Thread.sleep will throw `java.lang.IllegalArgumentException: timeout value is negative` exception Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Jia Zhai <zhaijia@apache.org> This closes #2284 from hangc0276/bug_fix
Apache BookKeeper is a scalable, fault tolerant and low latency storage service optimized for append-only workloads.
It is suitable for being used in following scenarios:
You can also read Turning Ledgers into Logs to learn how to turn ledgers into continuous log streams. If you are looking for a high level log stream API, you can checkout DistributedLog.
For filing bugs, suggesting improvements, or requesting new features, help us out by opening a Github issue or opening an Apache jira.
Subscribe or mail the user@bookkeeper.apache.org list - Ask questions, find answers, and also help other users.
Subscribe or mail the dev@bookkeeper.apache.org list - Join development discussions, propose new ideas and connect with contributors.
Join us on Slack - This is the most immediate way to connect with Apache BookKeeper committers and contributors.
We feel that a welcoming open community is important and welcome contributions.
See Developer Setup to get your local environment setup.
Take a look at our open issues: JIRA Issues Github Issues.
Review our coding style and follow our pull requests to learn about our conventions.
Make your changes according to our contribution guide.