in bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsMonitor.java [69:193]
private void check(final LedgerDirsManager ldm) {
final ConcurrentMap<File, Float> diskUsages = ldm.getDiskUsages();
boolean someDiskFulled = false;
boolean highPriorityWritesAllowed = true;
boolean someDiskRecovered = false;
try {
List<File> writableDirs = ldm.getWritableLedgerDirs();
// Check all writable dirs disk space usage.
for (File dir : writableDirs) {
try {
diskUsages.put(dir, diskChecker.checkDir(dir));
} catch (DiskErrorException e) {
LOG.error("Ledger directory {} failed on disk checking : ", dir, e);
// Notify disk failure to all listeners
for (LedgerDirsListener listener : ldm.getListeners()) {
listener.diskFailed(dir);
}
} catch (DiskWarnThresholdException e) {
diskUsages.compute(dir, (d, prevUsage) -> {
if (null == prevUsage || e.getUsage() != prevUsage) {
LOG.warn("Ledger directory {} is almost full : usage {}", dir, e.getUsage());
}
return e.getUsage();
});
for (LedgerDirsListener listener : ldm.getListeners()) {
listener.diskAlmostFull(dir);
}
} catch (DiskOutOfSpaceException e) {
diskUsages.compute(dir, (d, prevUsage) -> {
if (null == prevUsage || e.getUsage() != prevUsage) {
LOG.error("Ledger directory {} is out-of-space : usage {}", dir, e.getUsage());
}
return e.getUsage();
});
// Notify disk full to all listeners
ldm.addToFilledDirs(dir);
someDiskFulled = true;
}
}
// Let's get NoWritableLedgerDirException without waiting for the next iteration
// in case we are out of writable dirs
// otherwise for the duration of {interval} we end up in the state where
// bookie cannot get writable dir but considered to be writable
ldm.getWritableLedgerDirs();
} catch (NoWritableLedgerDirException e) {
LOG.warn("LedgerDirsMonitor check process: All ledger directories are non writable");
try {
// disk check can be frequent, so disable 'loggingNoWritable' to avoid log flooding.
ldm.getDirsAboveUsableThresholdSize(minUsableSizeForHighPriorityWrites, false);
} catch (NoWritableLedgerDirException e1) {
highPriorityWritesAllowed = false;
}
for (LedgerDirsListener listener : ldm.getListeners()) {
listener.allDisksFull(highPriorityWritesAllowed);
}
}
List<File> fulfilledDirs = new ArrayList<File>(ldm.getFullFilledLedgerDirs());
boolean makeWritable = ldm.hasWritableLedgerDirs();
// When bookie is in READONLY mode, i.e there are no writableLedgerDirs:
// - Update fulfilledDirs disk usage.
// - If the total disk usage is below DiskLowWaterMarkUsageThreshold
// add fulfilledDirs back to writableLedgerDirs list if their usage is < conf.getDiskUsageThreshold.
try {
if (!makeWritable) {
float totalDiskUsage = diskChecker.getTotalDiskUsage(ldm.getAllLedgerDirs());
if (totalDiskUsage < conf.getDiskLowWaterMarkUsageThreshold()) {
makeWritable = true;
} else if (LOG.isDebugEnabled()) {
LOG.debug(
"Current TotalDiskUsage: {} is greater than LWMThreshold: {}."
+ " So not adding any filledDir to WritableDirsList",
totalDiskUsage, conf.getDiskLowWaterMarkUsageThreshold());
}
}
// Update all full-filled disk space usage
for (File dir : fulfilledDirs) {
try {
diskUsages.put(dir, diskChecker.checkDir(dir));
if (makeWritable) {
ldm.addToWritableDirs(dir, true);
}
someDiskRecovered = true;
} catch (DiskErrorException e) {
// Notify disk failure to all the listeners
for (LedgerDirsListener listener : ldm.getListeners()) {
listener.diskFailed(dir);
}
} catch (DiskWarnThresholdException e) {
diskUsages.put(dir, e.getUsage());
// the full-filled dir become writable but still above the warn threshold
if (makeWritable) {
ldm.addToWritableDirs(dir, false);
}
someDiskRecovered = true;
} catch (DiskOutOfSpaceException e) {
// the full-filled dir is still full-filled
diskUsages.put(dir, e.getUsage());
}
}
} catch (IOException ioe) {
LOG.error("Got IOException while monitoring Dirs", ioe);
for (LedgerDirsListener listener : ldm.getListeners()) {
listener.fatalError();
}
}
if (conf.isReadOnlyModeOnAnyDiskFullEnabled()) {
if (someDiskFulled && !ldm.getFullFilledLedgerDirs().isEmpty()) {
// notify any disk full.
for (LedgerDirsListener listener : ldm.getListeners()) {
listener.anyDiskFull(highPriorityWritesAllowed);
}
}
if (someDiskRecovered && ldm.getFullFilledLedgerDirs().isEmpty()) {
// notify all disk recovered.
for (LedgerDirsListener listener : ldm.getListeners()) {
listener.allDisksWritable();
}
}
}
}