Skip to content
Open
Show file tree
Hide file tree
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 @@ -97,8 +97,7 @@ 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;
private volatile long lastKnownSnapshotStateVersion = -1;

/**
* Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
Expand Down Expand Up @@ -186,54 +185,91 @@ public Iterable<FileStatus> getUnreferencedFiles(List<FileStatus> files,
final SnapshotManager snapshotManager) throws IOException {
List<FileStatus> unReferencedFiles = Lists.newArrayList();
List<String> snapshotsInProgress = null;
boolean refreshed = false;
Lock lock = null;
if (snapshotManager != null) {
lock = snapshotManager.getTakingSnapshotLock().writeLock();
}
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;
}
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;

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 (snapshotManager == null) {
for (FileStatus file : files) {
String fileName = file.getPath().getName();
if (!cache.contains(fileName)) {
synchronized (this) {
if (!cache.contains(fileName)) {
refreshCache();
}
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);
if (cache.contains(fileName)) {
continue;
}
if (snapshotsInProgress == null) {
snapshotsInProgress = getSnapshotsInProgress();
}
if (snapshotsInProgress.contains(fileName)) {
continue;
}
unReferencedFiles.add(file);
}
} catch (InterruptedException e) {
LOG.warn("Interrupted while acquiring write lock on taking snapshot");
Thread.currentThread().interrupt(); // restore the interrupt flag
return unReferencedFiles;
}

long currentStateVersion = snapshotManager.getSnapshotStateVersion();
if (this.lastKnownSnapshotStateVersion < currentStateVersion) {
lock.lock();
try {
currentStateVersion = snapshotManager.getSnapshotStateVersion();
if (this.lastKnownSnapshotStateVersion < currentStateVersion) {
refreshCache();
LOG.debug(
"Snapshot state version changed from " + this.lastKnownSnapshotStateVersion + " to "
+ currentStateVersion + ", will refresh cache");
this.lastKnownSnapshotStateVersion = currentStateVersion;
}
} finally {
lock.unlock();
}
}

for (FileStatus file : files) {
String fileName = file.getPath().getName();
if (cache.contains(fileName)) {
continue;
}
if (snapshotsInProgress == null) {
snapshotsInProgress = getSnapshotsInProgress();
}
if (snapshotsInProgress.contains(fileName)) {
continue;
}
unReferencedFiles.add(file);
}

lock.lock();
try {
if (snapshotManager.isTakingAnySnapshot()) {
LOG.warn("Not checking unreferenced files since snapshot is running, it will " +
"skip to clean the HFiles this time");
return Lists.newArrayList();
}
currentStateVersion = snapshotManager.getSnapshotStateVersion();
if (this.lastKnownSnapshotStateVersion < currentStateVersion) {
LOG.warn("Snapshot state version changed from " +
this.lastKnownSnapshotStateVersion + " to " +
currentStateVersion + ", will skip to clean the HFiles this time");
return Lists.newArrayList();
}

return unReferencedFiles;
} finally {
lock.unlock();
}
return unReferencedFiles;
}

private void refreshCache() throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.conf.Configuration;
Expand Down Expand Up @@ -190,6 +191,8 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
*/
private ReentrantReadWriteLock takingSnapshotLock = new ReentrantReadWriteLock(true);

private final AtomicLong snapshotStateVersion = new AtomicLong(0);

public SnapshotManager() {
}

Expand Down Expand Up @@ -343,6 +346,7 @@ public void deleteSnapshot(SnapshotDescription snapshot) throws IOException {
cpHost.postDeleteSnapshot(snapshotPOJO);
}

incrementSnapshotStateVersion();
}

/**
Expand Down Expand Up @@ -714,6 +718,14 @@ else if (master.getTableStateManager().isTableState(snapshotTable, TableState.St
}
}

public long getSnapshotStateVersion() {
return snapshotStateVersion.get();
}

void incrementSnapshotStateVersion() {
snapshotStateVersion.incrementAndGet();
}

/**
* Set the handler for the current snapshot
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,7 @@ public void process() {
// complete the snapshot, atomically moving from tmp to .snapshot dir.
SnapshotDescriptionUtils.completeSnapshot(this.snapshotDir, this.workingDir, this.rootFs,
this.workingDirFs, this.conf);
snapshotManager.incrementSnapshotStateVersion();
finished = true;
msg = "Snapshot " + snapshot.getName() + " of table " + snapshotTable + " completed";
status.markComplete(msg);
Expand Down