in bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java [433:541]
public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMinor) {
long threadStart = MathUtils.nowInNano();
if (force) {
LOG.info("Garbage collector thread forced to perform GC before expiry of wait time.");
}
// Recover and clean up previous state if using transactional compaction
compactor.cleanUpAndRecover();
try {
// gc inactive/deleted ledgers
// this is used in extractMetaFromEntryLogs to calculate the usage of entry log
doGcLedgers();
long extractMetaStart = MathUtils.nowInNano();
try {
// Extract all of the ledger ID's that comprise all of the entry logs
// (except for the current new one which is still being written to).
extractMetaFromEntryLogs();
// gc entry logs
doGcEntryLogs();
gcStats.getExtractMetaRuntime()
.registerSuccessfulEvent(MathUtils.elapsedNanos(extractMetaStart), TimeUnit.NANOSECONDS);
} catch (EntryLogMetadataMapException e) {
gcStats.getExtractMetaRuntime()
.registerFailedEvent(MathUtils.elapsedNanos(extractMetaStart), TimeUnit.NANOSECONDS);
throw e;
}
if (suspendMajor) {
LOG.info("Disk almost full, suspend major compaction to slow down filling disk.");
}
if (suspendMinor) {
LOG.info("Disk full, suspend minor compaction to slow down filling disk.");
}
long curTime = System.currentTimeMillis();
long compactStart = MathUtils.nowInNano();
if (((isForceMajorCompactionAllow && force) || (enableMajorCompaction
&& (force || curTime - lastMajorCompactionTime > majorCompactionInterval)))
&& (!suspendMajor)) {
// enter major compaction
LOG.info("Enter major compaction, suspendMajor {}, lastMajorCompactionTime {}", suspendMajor,
lastMajorCompactionTime);
majorCompacting.set(true);
try {
doCompactEntryLogs(majorCompactionThreshold, majorCompactionMaxTimeMillis);
} catch (EntryLogMetadataMapException e) {
gcStats.getCompactRuntime()
.registerFailedEvent(MathUtils.elapsedNanos(compactStart), TimeUnit.NANOSECONDS);
throw e;
} finally {
lastMajorCompactionTime = System.currentTimeMillis();
// and also move minor compaction time
lastMinorCompactionTime = lastMajorCompactionTime;
gcStats.getMajorCompactionCounter().inc();
majorCompacting.set(false);
}
} else if (((isForceMinorCompactionAllow && force) || (enableMinorCompaction
&& (force || curTime - lastMinorCompactionTime > minorCompactionInterval)))
&& (!suspendMinor)) {
// enter minor compaction
LOG.info("Enter minor compaction, suspendMinor {}, lastMinorCompactionTime {}", suspendMinor,
lastMinorCompactionTime);
minorCompacting.set(true);
try {
doCompactEntryLogs(minorCompactionThreshold, minorCompactionMaxTimeMillis);
} catch (EntryLogMetadataMapException e) {
gcStats.getCompactRuntime()
.registerFailedEvent(MathUtils.elapsedNanos(compactStart), TimeUnit.NANOSECONDS);
throw e;
} finally {
lastMinorCompactionTime = System.currentTimeMillis();
gcStats.getMinorCompactionCounter().inc();
minorCompacting.set(false);
}
}
if (entryLocationCompactionInterval > 0 && (curTime - lastEntryLocationCompactionTime > (
entryLocationCompactionInterval + randomCompactionDelay))) {
// enter entry location compaction
LOG.info(
"Enter entry location compaction, entryLocationCompactionInterval {}, randomCompactionDelay "
+ "{}, lastEntryLocationCompactionTime {}",
entryLocationCompactionInterval, randomCompactionDelay, lastEntryLocationCompactionTime);
ledgerStorage.entryLocationCompact();
lastEntryLocationCompactionTime = System.currentTimeMillis();
randomCompactionDelay = ThreadLocalRandom.current().nextLong(entryLocationCompactionInterval);
LOG.info("Next entry location compaction interval {}",
entryLocationCompactionInterval + randomCompactionDelay);
gcStats.getEntryLocationCompactionCounter().inc();
}
gcStats.getCompactRuntime()
.registerSuccessfulEvent(MathUtils.elapsedNanos(compactStart), TimeUnit.NANOSECONDS);
gcStats.getGcThreadRuntime().registerSuccessfulEvent(
MathUtils.nowInNano() - threadStart, TimeUnit.NANOSECONDS);
} catch (EntryLogMetadataMapException e) {
LOG.error("Error in entryLog-metadatamap, Failed to complete GC/Compaction due to entry-log {}",
e.getMessage(), e);
gcStats.getGcThreadRuntime().registerFailedEvent(MathUtils.elapsedNanos(threadStart), TimeUnit.NANOSECONDS);
} finally {
if (force && forceGarbageCollection.compareAndSet(true, false)) {
LOG.info("{} Set forceGarbageCollection to false after force GC to make it forceGC-able again.",
Thread.currentThread().getName());
}
}
}