HBASE-27043 Let lock wait timeout to improve performance of SnapshotHFileCleaner (#4437)
Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
bf5f0c7e7f
commit
1aa07d5e6b
|
@ -22,6 +22,7 @@ import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Timer;
|
import java.util.Timer;
|
||||||
import java.util.TimerTask;
|
import java.util.TimerTask;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.locks.Lock;
|
import java.util.concurrent.locks.Lock;
|
||||||
import org.apache.commons.lang3.ArrayUtils;
|
import org.apache.commons.lang3.ArrayUtils;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -97,6 +98,8 @@ public class SnapshotFileCache implements Stoppable {
|
||||||
private ImmutableMap<String, SnapshotDirectoryInfo> snapshots = ImmutableMap.of();
|
private ImmutableMap<String, SnapshotDirectoryInfo> snapshots = ImmutableMap.of();
|
||||||
private final Timer refreshTimer;
|
private final Timer refreshTimer;
|
||||||
|
|
||||||
|
private static final int LOCK_TIMEOUT_MS = 30000;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
|
* Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
|
||||||
* filesystem.
|
* filesystem.
|
||||||
|
@ -192,7 +195,8 @@ public class SnapshotFileCache implements Stoppable {
|
||||||
if (snapshotManager != null) {
|
if (snapshotManager != null) {
|
||||||
lock = snapshotManager.getTakingSnapshotLock().writeLock();
|
lock = snapshotManager.getTakingSnapshotLock().writeLock();
|
||||||
}
|
}
|
||||||
if (lock == null || lock.tryLock()) {
|
try {
|
||||||
|
if (lock == null || lock.tryLock(LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) {
|
||||||
try {
|
try {
|
||||||
if (snapshotManager != null && snapshotManager.isTakingAnySnapshot()) {
|
if (snapshotManager != null && snapshotManager.isTakingAnySnapshot()) {
|
||||||
LOG.warn("Not checking unreferenced files since snapshot is running, it will "
|
LOG.warn("Not checking unreferenced files since snapshot is running, it will "
|
||||||
|
@ -225,6 +229,13 @@ public class SnapshotFileCache implements Stoppable {
|
||||||
lock.unlock();
|
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;
|
return unReferencedFiles;
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue