commit | f5ddc36cc30c5bab5d2e8ebc5fa552c2ad0d6eea | [log] [tgz] |
---|---|---|
author | Rajan Dhabalia <rdhabalia@apache.org> | Tue May 14 23:35:18 2019 -0700 |
committer | Enrico Olivelli <eolivelli@gmail.com> | Wed May 15 08:35:18 2019 +0200 |
tree | aa8bbcefb14bee6c1c6f6523385e9bf00359edd0 | |
parent | d35aa22ade87969a4f8e932d925c5d134feb764b [diff] |
[BK-GC] avoid blocking call in gc-thread ### Motivation Right now, we have below 3 issues because of which gc thread gets blocked forever and it can't perform gc-task further. Below issues are mainly related to blocking call while doing zk-operation without timeout. bug-fixes: 1. right now, [GC - ScanAndCompareGarbageCollector](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java#L142) passes timeout in millisecond to [LedgerManager](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LongHierarchicalLedgerManager.java#L166) but it takes it as second and again try to convert it in millis so, 30Kms timeout becomes [30M ms timeout](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java#L245). Sp, fix timeout unit during gc. 2. Right now, GC makes blocking call to get list of children on ledger znode and sometime zk-call back doesn't comeback which blocks the gc-thread forever. However, recently we added the timeout on the [object-waiting-lock](https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ZkUtils.java#L243-L248) which doesn't work because it's in while loop and `object.wait(timeout)` completes without any exception and GC threads keep running in while loop. 3. add zk-timeout during delete ledgers in bookie else it can also block the GC thread. ### Changes add timeout while bk-gc makes zk-call to verify deleted ledgers. Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Sijie Guo <sijie@apache.org>, Rajan Dhabalia <rdhabalia@apache.org> This closes #1940 from rdhabalia/verify_gc
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.