You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by yo...@apache.org on 2021/10/22 02:43:31 UTC

[bookkeeper] branch master updated: Add ensemble check to over-replicated ledger GC (#2813)

This is an automated email from the ASF dual-hosted git repository.

yong 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 53954ca  Add ensemble check to over-replicated ledger GC (#2813)
53954ca is described below

commit 53954caae3a4f40cf0a45718d306b6a8555c54e2
Author: shustsud <51...@users.noreply.github.com>
AuthorDate: Fri Oct 22 11:43:23 2021 +0900

    Add ensemble check to over-replicated ledger GC (#2813)
    
    ### Motivation
    - Issue is as described in [PR#2797](https://github.com/apache/bookkeeper/pull/2797).
    > In one day, zookeepers became high cpu usage and disk full.
    > The cause of this is bookie's gc of overreplicated ledgers.
    > Gc created/deleted zk nodes under /ledgers/underreplication/locks very frequently and some bookies ran gc at same time.
    > As a result, zookeepers created a lot of snapshots and became disk full.
    
    - I want to reduce the number of lock node creations and deletions in ZK.
    
    ### Changes
    - Add an ensemble check before creating the lock node.
    This is to reduce the number of lock node creations and deletions in ZK.
    
    - ~~If [PR#2797](https://github.com/apache/bookkeeper/pull/2797) was merged, this PR needs to be fixed.~~
---
 .../bookie/ScanAndCompareGarbageCollector.java     | 47 +++++++++++++---------
 1 file changed, 29 insertions(+), 18 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java
index 6f67bae..4c778a1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ScanAndCompareGarbageCollector.java
@@ -231,8 +231,12 @@ public class ScanAndCompareGarbageCollector implements GarbageCollector {
 
         for (final Long ledgerId : bkActiveledgers) {
             try {
-                // check if the ledger is being replicated already by the replication worker
-                if (lum.isLedgerBeingReplicated(ledgerId)) {
+                // check ledger ensembles before creating lock nodes.
+                // this is to reduce the number of lock node creations and deletions in ZK.
+                // the ensemble check is done again after the lock node is created.
+                // also, check if the ledger is being replicated already by the replication worker
+                Versioned<LedgerMetadata> preCheckMetadata = ledgerManager.readLedgerMetadata(ledgerId).get();
+                if (!isNotBookieIncludedInLedgerEnsembles(preCheckMetadata) || lum.isLedgerBeingReplicated(ledgerId)) {
                     latch.countDown();
                     continue;
                 }
@@ -245,23 +249,12 @@ public class ScanAndCompareGarbageCollector implements GarbageCollector {
                     .whenComplete((metadata, exception) -> {
                             try {
                                 if (exception == null) {
-                                    // do not delete a ledger that is not closed, since the ensemble might
-                                    // change again and include the current bookie while we are deleting it
-                                    if (!metadata.getValue().isClosed()) {
-                                        return;
+                                    if (isNotBookieIncludedInLedgerEnsembles(metadata)) {
+                                        // this bookie is not supposed to have this ledger,
+                                        // thus we can delete this ledger now
+                                        overReplicatedLedgers.add(ledgerId);
+                                        garbageCleaner.clean(ledgerId);
                                     }
-                                    SortedMap<Long, ? extends List<BookieId>> ensembles =
-                                        metadata.getValue().getAllEnsembles();
-                                    for (List<BookieId> ensemble : ensembles.values()) {
-                                        // check if this bookie is supposed to have this ledger
-                                        if (ensemble.contains(selfBookieAddress)) {
-                                            return;
-                                        }
-                                    }
-                                    // this bookie is not supposed to have this ledger,
-                                    // thus we can delete this ledger now
-                                    overReplicatedLedgers.add(ledgerId);
-                                    garbageCleaner.clean(ledgerId);
                                 }
                             } finally {
                                 semaphore.release();
@@ -302,4 +295,22 @@ public class ScanAndCompareGarbageCollector implements GarbageCollector {
         }
     }
 
+    private boolean isNotBookieIncludedInLedgerEnsembles(Versioned<LedgerMetadata> metadata) {
+        // do not delete a ledger that is not closed, since the ensemble might
+        // change again and include the current bookie while we are deleting it
+        if (!metadata.getValue().isClosed()) {
+            return false;
+        }
+
+        SortedMap<Long, ? extends List<BookieId>> ensembles =
+                metadata.getValue().getAllEnsembles();
+        for (List<BookieId> ensemble : ensembles.values()) {
+            // check if this bookie is supposed to have this ledger
+            if (ensemble.contains(selfBookieAddress)) {
+                return false;
+            }
+        }
+
+        return true;
+    }
 }