Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.util.List;
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.hadoop.conf.Configuration;
Expand Down Expand Up @@ -97,6 +98,8 @@ Collection<String> filesUnderSnapshot(final FileSystem fs, final Path snapshotDi
private ImmutableMap<String, SnapshotDirectoryInfo> snapshots = ImmutableMap.of();
private final Timer refreshTimer;

private static final int LOCK_TIMEOUT_MS = 30000;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we make it configurable? Just asking, I'm neutral on whether to introduce a configuration.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After HBASE-27590 I don't think it is necessary because the locking time is drastically reduced. Earlier I have seen the Failed to acquire write lock on taking snapshot after waiting 60000ms WARN messages continuously on this cluster.


/**
* Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
* filesystem.
Expand Down Expand Up @@ -192,39 +195,47 @@ public Iterable<FileStatus> getUnreferencedFiles(Iterable<FileStatus> files,
if (snapshotManager != null) {
lock = snapshotManager.getTakingSnapshotLock().writeLock();
}
if (lock == null || lock.tryLock()) {
try {
if (snapshotManager != null && snapshotManager.isTakingAnySnapshot()) {
LOG.warn("Not checking unreferenced files since snapshot is running, it will "
+ "skip to clean the HFiles this time");
return unReferencedFiles;
}
ImmutableSet<String> currentCache = cache;
for (FileStatus file : files) {
String fileName = file.getPath().getName();
if (!refreshed && !currentCache.contains(fileName)) {
synchronized (this) {
refreshCache();
currentCache = cache;
refreshed = true;
}
}
if (currentCache.contains(fileName)) {
continue;
try {
if (lock == null || lock.tryLock(LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) {
try {
if (snapshotManager != null && snapshotManager.isTakingAnySnapshot()) {
LOG.warn("Not checking unreferenced files since snapshot is running, it will "
+ "skip to clean the HFiles this time");
return unReferencedFiles;
}
if (snapshotsInProgress == null) {
snapshotsInProgress = getSnapshotsInProgress();
ImmutableSet<String> currentCache = cache;
for (FileStatus file : files) {
String fileName = file.getPath().getName();
if (!refreshed && !currentCache.contains(fileName)) {
synchronized (this) {
refreshCache();
currentCache = cache;
refreshed = true;
}
}
if (currentCache.contains(fileName)) {
continue;
}
if (snapshotsInProgress == null) {
snapshotsInProgress = getSnapshotsInProgress();
}
if (snapshotsInProgress.contains(fileName)) {
continue;
}
unReferencedFiles.add(file);
}
if (snapshotsInProgress.contains(fileName)) {
continue;
} finally {
if (lock != null) {
lock.unlock();
}
unReferencedFiles.add(file);
}
} finally {
if (lock != null) {
lock.unlock();
}
} else {
LOG.warn("Failed to acquire write lock on taking snapshot after waiting {}ms",
LOCK_TIMEOUT_MS);
}
} catch (InterruptedException e) {
LOG.warn("Interrupted while acquiring write lock on taking snapshot");
Thread.currentThread().interrupt(); // restore the interrupt flag
}
return unReferencedFiles;
}
Expand Down