This is an automated email from the ASF dual-hosted git repository.
mmerli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/master by this push:
new 09aec9ca52 Optimize bookie decommission check wait interval (#4070)
09aec9ca52 is described below
commit 09aec9ca520af7164fe5de378413e63d0c3889f9
Author: Hang Chen <[email protected]>
AuthorDate: Sat Sep 9 00:48:23 2023 +0800
Optimize bookie decommission check wait interval (#4070)
* Optimise bookie decommission check wait interval
* fix a bug
---
.../main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java | 9 ++++++---
1 file changed, 6 insertions(+), 3 deletions(-)
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
index f9ac460bde..ed2db8a032 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
@@ -1631,14 +1631,17 @@ public class BookKeeperAdmin implements AutoCloseable {
private void waitForLedgersToBeReplicated(Collection<Long> ledgers,
BookieId thisBookieAddress,
LedgerManager ledgerManager) throws InterruptedException,
TimeoutException {
- int maxSleepTimeInBetweenChecks = 10 * 60 * 1000; // 10 minutes
- int sleepTimePerLedger = 10 * 1000; // 10 secs
+ int maxSleepTimeInBetweenChecks = 5 * 60 * 1000; // 5 minutes
+ int sleepTimePerLedger = 3 * 1000; // 3 secs
Predicate<Long> validateBookieIsNotPartOfEnsemble = ledgerId ->
!areEntriesOfLedgerStoredInTheBookie(ledgerId,
thisBookieAddress, ledgerManager);
+ ledgers.removeIf(validateBookieIsNotPartOfEnsemble);
+
while (!ledgers.isEmpty()) {
- LOG.info("Count of Ledgers which need to be rereplicated: {}",
ledgers.size());
int sleepTimeForThisCheck = (long) ledgers.size() *
sleepTimePerLedger > maxSleepTimeInBetweenChecks
? maxSleepTimeInBetweenChecks : ledgers.size() *
sleepTimePerLedger;
+ LOG.info("Count of Ledgers which need to be rereplicated: {},
waiting {} seconds for next check",
+ ledgers.size(), sleepTimeForThisCheck / 1000);
Thread.sleep(sleepTimeForThisCheck);
if (LOG.isDebugEnabled()) {
LOG.debug("Making sure following ledgers replication to be
completed: {}", ledgers);