This is an automated email from the ASF dual-hosted git repository. yong pushed a commit to branch branch-4.15 in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
commit 4e839a73752ce213070abc3c4f7932f403747032 Author: Hang Chen <[email protected]> AuthorDate: Tue Sep 12 14:43:30 2023 +0800 Print compaction progress (#4071) ### Motivation When the garbage collection is triggered, we don't know the actual compaction progress, for example, the total entrylog files waiting to be compacted and how many entrylog files have been compacted. Without the information, we don't know when will the garbage collection will finish. ### Changes Print the garbage collection progress every one minutes, and the progress contains how many EntryLog files have been compaction and the total EntryLog files waiting to be compacted. (cherry picked from commit 056fe7b554bb66279c373df346495fa37d781584) --- .../bookkeeper/bookie/GarbageCollectorThread.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index bbf8ebdf4d..40a007dcd5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -35,7 +35,7 @@ import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; - +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; @@ -62,6 +62,7 @@ import org.slf4j.LoggerFactory; public class GarbageCollectorThread extends SafeRunnable { private static final Logger LOG = LoggerFactory.getLogger(GarbageCollectorThread.class); private static final int SECOND = 1000; + private static final long MINUTE = TimeUnit.MINUTES.toMillis(1); // Maps entry log files to the set of ledgers that comprise the file and the size usage per ledger private EntryLogMetadataMap entryLogMetaMap; @@ -583,6 +584,13 @@ public class GarbageCollectorThread extends SafeRunnable { entryLogUsageBuckets); final int maxBucket = calculateUsageIndex(numBuckets, threshold); + int totalEntryLogIds = 0; + for (int currBucket = 0; currBucket <= maxBucket; currBucket++) { + totalEntryLogIds += compactableBuckets.get(currBucket).size(); + } + long lastPrintTimestamp = 0; + AtomicInteger processedEntryLogCnt = new AtomicInteger(0); + stopCompaction: for (int currBucket = 0; currBucket <= maxBucket; currBucket++) { LinkedList<Long> entryLogIds = compactableBuckets.get(currBucket); @@ -600,7 +608,11 @@ public class GarbageCollectorThread extends SafeRunnable { final int bucketIndex = currBucket; final long logId = entryLogIds.remove(); - + if (System.currentTimeMillis() - lastPrintTimestamp >= MINUTE) { + lastPrintTimestamp = System.currentTimeMillis(); + LOG.info("Compaction progress {} / {}, current compaction entryLogId: {}", + processedEntryLogCnt.get(), totalEntryLogIds, logId); + } entryLogMetaMap.forKey(logId, (entryLogId, meta) -> { if (meta == null) { if (LOG.isDebugEnabled()) { @@ -617,6 +629,7 @@ public class GarbageCollectorThread extends SafeRunnable { compactEntryLog(meta); gcStats.getReclaimedSpaceViaCompaction().add(meta.getTotalSize() - priorRemainingSize); compactedBuckets[bucketIndex]++; + processedEntryLogCnt.getAndIncrement(); }); } }
