HDFS-15481. Ordered snapshot deletion: garbage collect deleted snapshots (#2165)
This commit is contained in:
parent
e0c9653166
commit
05b3337a46
@ -502,11 +502,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
||||
"dfs.namenode.snapshot.max.limit";
|
||||
public static final int DFS_NAMENODE_SNAPSHOT_MAX_LIMIT_DEFAULT = 65536;
|
||||
|
||||
public static final String DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED =
|
||||
"dfs.namenode.snapshot.deletion.ordered";
|
||||
public static final boolean DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_DEFAULT
|
||||
= false;
|
||||
|
||||
public static final String DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL =
|
||||
"dfs.namenode.snapshot.skiplist.interval";
|
||||
public static final int DFS_NAMENODE_SNAPSHOT_SKIPLIST_SKIP_INTERVAL_DEFAULT =
|
||||
|
@ -366,7 +366,7 @@ enum BlockUCState {
|
||||
"security.hdfs.unreadable.by.superuser";
|
||||
String XATTR_ERASURECODING_POLICY =
|
||||
"system.hdfs.erasurecoding.policy";
|
||||
String SNAPSHOT_XATTR_NAME = "system.hdfs.snapshot.deleted";
|
||||
String XATTR_SNAPSHOT_DELETED = "system.hdfs.snapshot.deleted";
|
||||
|
||||
String XATTR_SATISFY_STORAGE_POLICY = "user.hdfs.sps";
|
||||
|
||||
|
@ -44,7 +44,7 @@
|
||||
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
|
||||
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
|
||||
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
|
||||
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SNAPSHOT_XATTR_NAME;
|
||||
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SNAPSHOT_DELETED;
|
||||
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
|
||||
|
||||
public class FSDirXAttrOp {
|
||||
@ -328,10 +328,10 @@ public static INode unprotectedSetXAttrs(
|
||||
SECURITY_XATTR_UNREADABLE_BY_SUPERUSER + "' on a file.");
|
||||
}
|
||||
|
||||
if (xaName.equals(SNAPSHOT_XATTR_NAME) && !(inode.isDirectory() &&
|
||||
if (xaName.equals(XATTR_SNAPSHOT_DELETED) && !(inode.isDirectory() &&
|
||||
inode.getParent().isSnapshottable())) {
|
||||
throw new IOException("Can only set '" +
|
||||
SNAPSHOT_XATTR_NAME + "' on a snapshot root.");
|
||||
XATTR_SNAPSHOT_DELETED + "' on a snapshot root.");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -187,7 +187,6 @@ private static INodeDirectory createRoot(FSNamesystem namesystem) {
|
||||
private boolean posixAclInheritanceEnabled;
|
||||
private final boolean xattrsEnabled;
|
||||
private final int xattrMaxSize;
|
||||
private final boolean snapshotDeletionOrdered;
|
||||
|
||||
// precision of access times.
|
||||
private final long accessTimePrecision;
|
||||
@ -354,12 +353,6 @@ public enum DirOp {
|
||||
+ " hard limit " + DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_HARD_LIMIT
|
||||
+ ": (%s).", DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY);
|
||||
|
||||
this.snapshotDeletionOrdered =
|
||||
conf.getBoolean(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED,
|
||||
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_DEFAULT);
|
||||
LOG.info("{} = {}", DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED,
|
||||
snapshotDeletionOrdered);
|
||||
|
||||
this.accessTimePrecision = conf.getLong(
|
||||
DFS_NAMENODE_ACCESSTIME_PRECISION_KEY,
|
||||
DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT);
|
||||
@ -618,10 +611,6 @@ boolean isXattrsEnabled() {
|
||||
}
|
||||
int getXattrMaxSize() { return xattrMaxSize; }
|
||||
|
||||
public boolean isSnapshotDeletionOrdered() {
|
||||
return snapshotDeletionOrdered;
|
||||
}
|
||||
|
||||
boolean isAccessTimeSupported() {
|
||||
return accessTimePrecision > 0;
|
||||
}
|
||||
|
@ -109,6 +109,7 @@
|
||||
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotDeletionGc;
|
||||
import org.apache.hadoop.thirdparty.protobuf.ByteString;
|
||||
import org.apache.hadoop.hdfs.protocol.BatchedDirectoryListing;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsPartialListing;
|
||||
@ -493,6 +494,7 @@ private void logAuditEvent(boolean succeeded,
|
||||
FSDirectory dir;
|
||||
private BlockManager blockManager;
|
||||
private final SnapshotManager snapshotManager;
|
||||
private final SnapshotDeletionGc snapshotDeletionGc;
|
||||
private final CacheManager cacheManager;
|
||||
private final DatanodeStatistics datanodeStatistics;
|
||||
|
||||
@ -974,6 +976,9 @@ static FSNamesystem loadFromDisk(Configuration conf) throws IOException {
|
||||
this.dtSecretManager = createDelegationTokenSecretManager(conf);
|
||||
this.dir = new FSDirectory(this, conf);
|
||||
this.snapshotManager = new SnapshotManager(conf, dir);
|
||||
this.snapshotDeletionGc = snapshotManager.isSnapshotDeletionOrdered()?
|
||||
new SnapshotDeletionGc(this, conf): null;
|
||||
|
||||
this.cacheManager = new CacheManager(this, conf, blockManager);
|
||||
// Init ErasureCodingPolicyManager instance.
|
||||
ErasureCodingPolicyManager.getInstance().init(conf);
|
||||
@ -1360,6 +1365,10 @@ void startActiveServices() throws IOException {
|
||||
dir.enableQuotaChecks();
|
||||
dir.ezManager.startReencryptThreads();
|
||||
|
||||
if (snapshotDeletionGc != null) {
|
||||
snapshotDeletionGc.schedule();
|
||||
}
|
||||
|
||||
if (haEnabled) {
|
||||
// Renew all of the leases before becoming active.
|
||||
// This is because, while we were in standby mode,
|
||||
@ -5480,6 +5489,9 @@ private void registerMBean() {
|
||||
* Shutdown FSNamesystem.
|
||||
*/
|
||||
void shutdown() {
|
||||
if (snapshotDeletionGc != null) {
|
||||
snapshotDeletionGc.cancel();
|
||||
}
|
||||
if (snapshotManager != null) {
|
||||
snapshotManager.shutdown();
|
||||
}
|
||||
@ -7198,6 +7210,30 @@ void deleteSnapshot(String snapshotRoot, String snapshotName,
|
||||
logAuditEvent(true, operationName, rootPath, null, null);
|
||||
}
|
||||
|
||||
public void gcDeletedSnapshot(String snapshotRoot, String snapshotName)
|
||||
throws IOException {
|
||||
final String operationName = "gcDeletedSnapshot";
|
||||
String rootPath = null;
|
||||
final INode.BlocksMapUpdateInfo blocksToBeDeleted;
|
||||
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
|
||||
checkNameNodeSafeMode("Cannot gcDeletedSnapshot for " + rootPath);
|
||||
|
||||
final long now = Time.now();
|
||||
final INodesInPath iip = dir.resolvePath(null, snapshotRoot, DirOp.WRITE);
|
||||
snapshotManager.assertMarkedAsDeleted(iip, snapshotName);
|
||||
blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(
|
||||
dir, snapshotManager, iip, snapshotName, now);
|
||||
} finally {
|
||||
writeUnlock(operationName, getLockReportInfoSupplier(rootPath));
|
||||
}
|
||||
removeBlocks(blocksToBeDeleted);
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a list of INodeDirectorySnapshottable from the SnapshotManager
|
||||
* @param toRemove the list of INodeDirectorySnapshottable to be removed
|
||||
|
@ -38,6 +38,7 @@
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
|
||||
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
@ -294,11 +295,11 @@ public Snapshot addSnapshot(int id, String name,
|
||||
* @param snapshotName Name of the snapshot.
|
||||
* @param mtime The snapshot deletion time set by Time.now().
|
||||
*/
|
||||
public Snapshot removeSnapshot(
|
||||
ReclaimContext reclaimContext, String snapshotName, long mtime)
|
||||
public Snapshot removeSnapshot(ReclaimContext reclaimContext,
|
||||
String snapshotName, long mtime, SnapshotManager snapshotManager)
|
||||
throws SnapshotException {
|
||||
return getDirectorySnapshottableFeature().removeSnapshot(
|
||||
reclaimContext, this, snapshotName, mtime);
|
||||
reclaimContext, this, snapshotName, mtime, snapshotManager);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -133,6 +133,17 @@ final void addFirst(D diff) {
|
||||
diff.setPosterior(first);
|
||||
}
|
||||
|
||||
/** @return the first diff. */
|
||||
final D getFirst() {
|
||||
return diffs == null || diffs.isEmpty()? null: diffs.get(0);
|
||||
}
|
||||
|
||||
/** @return the first snapshot INode. */
|
||||
final A getFirstSnapshotINode() {
|
||||
final D first = getFirst();
|
||||
return first == null? null: first.getSnapshotINode();
|
||||
}
|
||||
|
||||
/** @return the last diff. */
|
||||
public final D getLast() {
|
||||
if (diffs == null) {
|
||||
|
@ -239,7 +239,8 @@ public Snapshot addSnapshot(INodeDirectory snapshotRoot, int id, String name,
|
||||
*/
|
||||
public Snapshot removeSnapshot(
|
||||
INode.ReclaimContext reclaimContext, INodeDirectory snapshotRoot,
|
||||
String snapshotName, long now) throws SnapshotException {
|
||||
String snapshotName, long now, SnapshotManager snapshotManager)
|
||||
throws SnapshotException {
|
||||
final int i = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
|
||||
if (i < 0) {
|
||||
throw new SnapshotException("Cannot delete snapshot " + snapshotName
|
||||
@ -248,6 +249,7 @@ public Snapshot removeSnapshot(
|
||||
} else {
|
||||
final Snapshot snapshot = snapshotsByNames.get(i);
|
||||
int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId());
|
||||
snapshotManager.assertPrior(snapshotRoot, snapshotName, prior);
|
||||
snapshotRoot.cleanSubtree(reclaimContext, snapshot.getId(), prior);
|
||||
// remove from snapshotsByNames after successfully cleaning the subtree
|
||||
snapshotsByNames.remove(i);
|
||||
|
@ -438,7 +438,7 @@ public void serializeSnapshotSection(OutputStream out) throws IOException {
|
||||
.setSnapshotCounter(sm.getSnapshotCounter())
|
||||
.setNumSnapshots(sm.getNumSnapshots());
|
||||
|
||||
INodeDirectory[] snapshottables = sm.getSnapshottableDirs();
|
||||
final List<INodeDirectory> snapshottables = sm.getSnapshottableDirs();
|
||||
for (INodeDirectory sdir : snapshottables) {
|
||||
b.addSnapshottableDir(sdir.getId());
|
||||
}
|
||||
|
@ -40,6 +40,8 @@
|
||||
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
|
||||
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SNAPSHOT_DELETED;
|
||||
|
||||
/** Snapshot of a sub-tree in the namesystem. */
|
||||
@InterfaceAudience.Private
|
||||
public class Snapshot implements Comparable<byte[]> {
|
||||
@ -156,6 +158,11 @@ static public class Root extends INodeDirectory {
|
||||
}).collect(Collectors.toList()).toArray(new Feature[0]));
|
||||
}
|
||||
|
||||
boolean isMarkedAsDeleted() {
|
||||
final XAttrFeature f = getXAttrFeature();
|
||||
return f != null && f.getXAttr(XATTR_SNAPSHOT_DELETED) != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadOnlyList<INode> getChildrenList(int snapshotId) {
|
||||
return getParent().getChildrenList(snapshotId);
|
||||
|
@ -0,0 +1,110 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.Timer;
|
||||
import java.util.TimerTask;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS_DEFAULT;
|
||||
|
||||
public class SnapshotDeletionGc {
|
||||
public static final Logger LOG = LoggerFactory.getLogger(
|
||||
SnapshotDeletionGc.class);
|
||||
|
||||
private final FSNamesystem namesystem;
|
||||
private final long deletionOrderedGcPeriodMs;
|
||||
private final AtomicReference<Timer> timer = new AtomicReference<>();
|
||||
|
||||
public SnapshotDeletionGc(FSNamesystem namesystem, Configuration conf) {
|
||||
this.namesystem = namesystem;
|
||||
|
||||
this.deletionOrderedGcPeriodMs = conf.getLong(
|
||||
DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS,
|
||||
DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS_DEFAULT);
|
||||
LOG.info("{} = {}", DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS,
|
||||
deletionOrderedGcPeriodMs);
|
||||
}
|
||||
|
||||
public void schedule() {
|
||||
if (timer.get() != null) {
|
||||
return;
|
||||
}
|
||||
final Timer t = new Timer(getClass().getSimpleName(), true);
|
||||
if (timer.compareAndSet(null, t)) {
|
||||
LOG.info("Schedule at fixed rate {}",
|
||||
StringUtils.formatTime(deletionOrderedGcPeriodMs));
|
||||
t.scheduleAtFixedRate(new GcTask(),
|
||||
deletionOrderedGcPeriodMs, deletionOrderedGcPeriodMs);
|
||||
}
|
||||
}
|
||||
|
||||
public void cancel() {
|
||||
final Timer t = timer.getAndSet(null);
|
||||
if (t != null) {
|
||||
LOG.info("cancel");
|
||||
t.cancel();
|
||||
}
|
||||
}
|
||||
|
||||
private void gcDeletedSnapshot(String name) {
|
||||
final Snapshot.Root deleted;
|
||||
namesystem.readLock();
|
||||
try {
|
||||
deleted = namesystem.getSnapshotManager().chooseDeletedSnapshot();
|
||||
} catch (Throwable e) {
|
||||
LOG.error("Failed to chooseDeletedSnapshot", e);
|
||||
throw e;
|
||||
} finally {
|
||||
namesystem.readUnlock();
|
||||
}
|
||||
if (deleted == null) {
|
||||
LOG.trace("{}: no snapshots are marked as deleted.", name);
|
||||
return;
|
||||
}
|
||||
|
||||
final String snapshotRoot = deleted.getRootFullPathName();
|
||||
final String snapshotName = deleted.getLocalName();
|
||||
LOG.info("{}: delete snapshot {} from {}",
|
||||
name, snapshotName, snapshotRoot);
|
||||
|
||||
try {
|
||||
namesystem.gcDeletedSnapshot(snapshotRoot, snapshotName);
|
||||
} catch (Throwable e) {
|
||||
LOG.error("Failed to gcDeletedSnapshot " + deleted.getFullPathName(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private class GcTask extends TimerTask {
|
||||
private final AtomicInteger count = new AtomicInteger();
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
final int id = count.incrementAndGet();
|
||||
gcDeletedSnapshot(getClass().getSimpleName() + " #" + id);
|
||||
}
|
||||
}
|
||||
}
|
@ -31,6 +31,7 @@
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import javax.management.ObjectName;
|
||||
@ -84,6 +85,16 @@ public class SnapshotManager implements SnapshotStatsMXBean {
|
||||
public static final Logger LOG =
|
||||
LoggerFactory.getLogger(SnapshotManager.class);
|
||||
|
||||
// The following are private configurations
|
||||
static final String DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED
|
||||
= "dfs.namenode.snapshot.deletion.ordered";
|
||||
static final boolean DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_DEFAULT
|
||||
= false;
|
||||
static final String DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS
|
||||
= "dfs.namenode.snapshot.deletion.ordered.gc.period.ms";
|
||||
static final long DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS_DEFAULT
|
||||
= 5 * 60_000L; //5 minutes
|
||||
|
||||
private final FSDirectory fsdir;
|
||||
private boolean captureOpenFiles;
|
||||
/**
|
||||
@ -106,12 +117,13 @@ public class SnapshotManager implements SnapshotStatsMXBean {
|
||||
private static final int SNAPSHOT_ID_BIT_WIDTH = 28;
|
||||
|
||||
private boolean allowNestedSnapshots = false;
|
||||
private final boolean snapshotDeletionOrdered;
|
||||
private int snapshotCounter = 0;
|
||||
private final int maxSnapshotLimit;
|
||||
|
||||
/** All snapshottable directories in the namesystem. */
|
||||
private final Map<Long, INodeDirectory> snapshottables =
|
||||
new HashMap<Long, INodeDirectory>();
|
||||
new ConcurrentHashMap<>();
|
||||
|
||||
public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
|
||||
this.fsdir = fsdir;
|
||||
@ -136,6 +148,12 @@ public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
|
||||
+ ", maxSnapshotLimit: "
|
||||
+ maxSnapshotLimit);
|
||||
|
||||
this.snapshotDeletionOrdered = conf.getBoolean(
|
||||
DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED,
|
||||
DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_DEFAULT);
|
||||
LOG.info("{} = {}", DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED,
|
||||
snapshotDeletionOrdered);
|
||||
|
||||
final int maxLevels = conf.getInt(
|
||||
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_LEVELS,
|
||||
DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_SKIPLIST_MAX_SKIP_LEVELS_DEFAULT);
|
||||
@ -145,6 +163,10 @@ public SnapshotManager(final Configuration conf, final FSDirectory fsdir) {
|
||||
DirectoryDiffListFactory.init(skipInterval, maxLevels, LOG);
|
||||
}
|
||||
|
||||
public boolean isSnapshotDeletionOrdered() {
|
||||
return snapshotDeletionOrdered;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void setCaptureOpenFiles(boolean captureOpenFiles) {
|
||||
this.captureOpenFiles = captureOpenFiles;
|
||||
@ -275,6 +297,49 @@ public INodeDirectory getSnapshottableRoot(final INodesInPath iip)
|
||||
return dir;
|
||||
}
|
||||
|
||||
public void assertMarkedAsDeleted(INodesInPath iip, String snapshotName)
|
||||
throws IOException {
|
||||
final INodeDirectory dir = getSnapshottableRoot(iip);
|
||||
final Snapshot.Root snapshotRoot = dir.getDirectorySnapshottableFeature()
|
||||
.getSnapshotByName(dir, snapshotName)
|
||||
.getRoot();
|
||||
|
||||
if (!snapshotRoot.isMarkedAsDeleted()) {
|
||||
throw new SnapshotException("Failed to gcDeletedSnapshot "
|
||||
+ snapshotName + " from " + dir.getFullPathName()
|
||||
+ ": snapshot is not marked as deleted");
|
||||
}
|
||||
}
|
||||
|
||||
void assertPrior(INodeDirectory dir, String snapshotName, int prior)
|
||||
throws SnapshotException {
|
||||
if (!isSnapshotDeletionOrdered()) {
|
||||
return;
|
||||
}
|
||||
// prior must not exist
|
||||
if (prior != Snapshot.NO_SNAPSHOT_ID) {
|
||||
throw new SnapshotException("Failed to removeSnapshot "
|
||||
+ snapshotName + " from " + dir.getFullPathName()
|
||||
+ ": Unexpected prior (=" + prior + ") when "
|
||||
+ DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED
|
||||
+ " is " + isSnapshotDeletionOrdered());
|
||||
}
|
||||
}
|
||||
|
||||
void assertFirstSnapshot(INodeDirectory dir,
|
||||
DirectorySnapshottableFeature snapshottable, Snapshot snapshot)
|
||||
throws SnapshotException {
|
||||
final INodeDirectoryAttributes first
|
||||
= snapshottable.getDiffs().getFirstSnapshotINode();
|
||||
if (snapshot.getRoot() != first) {
|
||||
throw new SnapshotException("Failed to delete snapshot " + snapshot
|
||||
+ " from " + dir.getFullPathName() + " since " + snapshot
|
||||
+ " is not the first snapshot (=" + first + ") and "
|
||||
+ DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED
|
||||
+ " is " + isSnapshotDeletionOrdered());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the snapshot root directory for the given directory. The given
|
||||
* directory must either be a snapshot root or a descendant of any
|
||||
@ -360,15 +425,14 @@ public String createSnapshot(final LeaseManager leaseManager,
|
||||
public void deleteSnapshot(final INodesInPath iip, final String snapshotName,
|
||||
INode.ReclaimContext reclaimContext, long now) throws IOException {
|
||||
final INodeDirectory srcRoot = getSnapshottableRoot(iip);
|
||||
if (fsdir.isSnapshotDeletionOrdered()) {
|
||||
if (isSnapshotDeletionOrdered()) {
|
||||
final DirectorySnapshottableFeature snapshottable
|
||||
= srcRoot.getDirectorySnapshottableFeature();
|
||||
final Snapshot snapshot = snapshottable.getSnapshotByName(
|
||||
srcRoot, snapshotName);
|
||||
|
||||
// Diffs must be not empty since a snapshot exists in the list
|
||||
final int earliest = snapshottable.getDiffs().iterator().next()
|
||||
.getSnapshotId();
|
||||
final int earliest = snapshottable.getDiffs().getFirst().getSnapshotId();
|
||||
if (snapshot.getId() != earliest) {
|
||||
final XAttr snapshotXAttr = buildXAttr();
|
||||
final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
|
||||
@ -390,8 +454,11 @@ public void deleteSnapshot(final INodesInPath iip, final String snapshotName,
|
||||
EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
|
||||
return;
|
||||
}
|
||||
|
||||
assertFirstSnapshot(srcRoot, snapshottable, snapshot);
|
||||
}
|
||||
srcRoot.removeSnapshot(reclaimContext, snapshotName, now);
|
||||
|
||||
srcRoot.removeSnapshot(reclaimContext, snapshotName, now, this);
|
||||
numSnapshots.getAndDecrement();
|
||||
}
|
||||
|
||||
@ -435,9 +502,8 @@ void setSnapshotCounter(int counter) {
|
||||
snapshotCounter = counter;
|
||||
}
|
||||
|
||||
INodeDirectory[] getSnapshottableDirs() {
|
||||
return snapshottables.values().toArray(
|
||||
new INodeDirectory[snapshottables.size()]);
|
||||
List<INodeDirectory> getSnapshottableDirs() {
|
||||
return new ArrayList<>(snapshottables.values());
|
||||
}
|
||||
|
||||
/**
|
||||
@ -613,7 +679,7 @@ public int getMaxSnapshotID() {
|
||||
}
|
||||
|
||||
public static XAttr buildXAttr() {
|
||||
return XAttrHelper.buildXAttr(HdfsServerConstants.SNAPSHOT_XATTR_NAME);
|
||||
return XAttrHelper.buildXAttr(HdfsServerConstants.XATTR_SNAPSHOT_DELETED);
|
||||
}
|
||||
|
||||
private ObjectName mxBeanName;
|
||||
@ -666,4 +732,35 @@ public static SnapshotInfo.Bean toBean(Snapshot s) {
|
||||
s.getRoot().getLocalName(), s.getRoot().getFullPathName(),
|
||||
s.getRoot().getModificationTime());
|
||||
}
|
||||
}
|
||||
|
||||
private List<INodeDirectory> getSnapshottableDirsForGc() {
|
||||
final List<INodeDirectory> dirs = getSnapshottableDirs();
|
||||
Collections.shuffle(dirs);
|
||||
return dirs;
|
||||
}
|
||||
|
||||
Snapshot.Root chooseDeletedSnapshot() {
|
||||
for(INodeDirectory dir : getSnapshottableDirsForGc()) {
|
||||
final Snapshot.Root root = chooseDeletedSnapshot(dir);
|
||||
if (root != null) {
|
||||
return root;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private static Snapshot.Root chooseDeletedSnapshot(INodeDirectory dir) {
|
||||
final DirectorySnapshottableFeature snapshottable
|
||||
= dir.getDirectorySnapshottableFeature();
|
||||
if (snapshottable == null) {
|
||||
return null;
|
||||
}
|
||||
final DirectoryWithSnapshotFeature.DirectoryDiffList diffs
|
||||
= snapshottable.getDiffs();
|
||||
final Snapshot.Root first = (Snapshot.Root)diffs.getFirstSnapshotINode();
|
||||
if (first == null || !first.isMarkedAsDeleted()) {
|
||||
return null;
|
||||
}
|
||||
return first;
|
||||
}
|
||||
}
|
@ -15,7 +15,7 @@
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@ -26,9 +26,10 @@
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.XAttrHelper;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
||||
import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
@ -37,9 +38,8 @@
|
||||
import java.util.EnumSet;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.
|
||||
DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SNAPSHOT_DELETED;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
@ -71,7 +71,7 @@ public void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@Test(timeout = 60000)
|
||||
public void testConf() throws Exception {
|
||||
public void testOrderedSnapshotDeletion() throws Exception {
|
||||
DistributedFileSystem hdfs = cluster.getFileSystem();
|
||||
hdfs.mkdirs(snapshottableDir);
|
||||
hdfs.allowSnapshot(snapshottableDir);
|
||||
@ -96,22 +96,54 @@ public void testConf() throws Exception {
|
||||
hdfs.deleteSnapshot(snapshottableDir, "s2");
|
||||
}
|
||||
|
||||
static void assertMarkedAsDeleted(Path snapshotRoot, MiniDFSCluster cluster)
|
||||
throws IOException {
|
||||
// Check if the path exists
|
||||
Assert.assertNotNull(cluster.getFileSystem().getFileStatus(snapshotRoot));
|
||||
|
||||
// Check xAttr for snapshotRoot
|
||||
final INode inode = cluster.getNamesystem().getFSDirectory()
|
||||
.getINode(snapshotRoot.toString());
|
||||
final XAttrFeature f = inode.getXAttrFeature();
|
||||
final XAttr xAttr = f.getXAttr(XATTR_SNAPSHOT_DELETED);
|
||||
Assert.assertNotNull(xAttr);
|
||||
Assert.assertEquals(XATTR_SNAPSHOT_DELETED.substring("system.".length()),
|
||||
xAttr.getName());
|
||||
Assert.assertEquals(XAttr.NameSpace.SYSTEM, xAttr.getNameSpace());
|
||||
Assert.assertNull(xAttr.getValue());
|
||||
|
||||
// Check inode
|
||||
Assert.assertTrue(inode instanceof Snapshot.Root);
|
||||
Assert.assertTrue(((Snapshot.Root)inode).isMarkedAsDeleted());
|
||||
}
|
||||
|
||||
static void assertNotMarkedAsDeleted(Path snapshotRoot,
|
||||
MiniDFSCluster cluster) throws IOException {
|
||||
// Check if the path exists
|
||||
Assert.assertNotNull(cluster.getFileSystem().getFileStatus(snapshotRoot));
|
||||
|
||||
// Check xAttr for snapshotRoot
|
||||
final INode inode = cluster.getNamesystem().getFSDirectory()
|
||||
.getINode(snapshotRoot.toString());
|
||||
final XAttrFeature f = inode.getXAttrFeature();
|
||||
if (f != null) {
|
||||
final XAttr xAttr = f.getXAttr(XATTR_SNAPSHOT_DELETED);
|
||||
Assert.assertNull(xAttr);
|
||||
}
|
||||
|
||||
// Check inode
|
||||
Assert.assertTrue(inode instanceof Snapshot.Root);
|
||||
Assert.assertFalse(((Snapshot.Root)inode).isMarkedAsDeleted());
|
||||
}
|
||||
|
||||
void assertXAttrSet(String snapshot,
|
||||
DistributedFileSystem hdfs, XAttr newXattr)
|
||||
throws IOException {
|
||||
hdfs.deleteSnapshot(snapshottableDir, snapshot);
|
||||
// Check xAttr for parent directory
|
||||
FSNamesystem namesystem = cluster.getNamesystem();
|
||||
Path snapshotRoot = SnapshotTestHelper.getSnapshotRoot(snapshottableDir,
|
||||
snapshot);
|
||||
INode inode = namesystem.getFSDirectory().getINode(snapshotRoot.toString());
|
||||
XAttrFeature f = inode.getXAttrFeature();
|
||||
XAttr xAttr = f.getXAttr(HdfsServerConstants.SNAPSHOT_XATTR_NAME);
|
||||
assertTrue("Snapshot xAttr should exist", xAttr != null);
|
||||
assertTrue(xAttr.getName().equals(HdfsServerConstants.SNAPSHOT_XATTR_NAME.
|
||||
replace("system.", "")));
|
||||
assertTrue(xAttr.getNameSpace().equals(XAttr.NameSpace.SYSTEM));
|
||||
assertNull(xAttr.getValue());
|
||||
assertMarkedAsDeleted(snapshotRoot, cluster);
|
||||
|
||||
// Make sure its not user visible
|
||||
if (cluster.getNameNode().getConf().getBoolean(DFSConfigKeys.
|
@ -0,0 +1,184 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.event.Level;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion.assertMarkedAsDeleted;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion.assertNotMarkedAsDeleted;
|
||||
|
||||
/**
|
||||
* Test {@link SnapshotDeletionGc}.
|
||||
*/
|
||||
public class TestOrderedSnapshotDeletionGc {
|
||||
private static final int GC_PERIOD = 10;
|
||||
|
||||
private MiniDFSCluster cluster;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
final Configuration conf = new Configuration();
|
||||
conf.setBoolean(DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED, true);
|
||||
conf.setInt(DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS, GC_PERIOD);
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
|
||||
cluster.waitActive();
|
||||
|
||||
GenericTestUtils.setLogLevel(SnapshotDeletionGc.LOG, Level.TRACE);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 60000)
|
||||
public void testSingleDir() throws Exception {
|
||||
final DistributedFileSystem hdfs = cluster.getFileSystem();
|
||||
|
||||
final Path snapshottableDir = new Path("/dir");
|
||||
hdfs.mkdirs(snapshottableDir);
|
||||
hdfs.allowSnapshot(snapshottableDir);
|
||||
|
||||
final Path sub0 = new Path(snapshottableDir, "sub0");
|
||||
hdfs.mkdirs(sub0);
|
||||
final Path s0path = hdfs.createSnapshot(snapshottableDir, "s0");
|
||||
Assert.assertTrue(exist(s0path, hdfs));
|
||||
|
||||
final Path sub1 = new Path(snapshottableDir, "sub1");
|
||||
hdfs.mkdirs(sub1);
|
||||
final Path s1path = hdfs.createSnapshot(snapshottableDir, "s1");
|
||||
Assert.assertTrue(exist(s1path, hdfs));
|
||||
|
||||
final Path sub2 = new Path(snapshottableDir, "sub2");
|
||||
hdfs.mkdirs(sub2);
|
||||
final Path s2path = hdfs.createSnapshot(snapshottableDir, "s2");
|
||||
Assert.assertTrue(exist(s2path, hdfs));
|
||||
|
||||
assertNotMarkedAsDeleted(s0path, cluster);
|
||||
assertNotMarkedAsDeleted(s1path, cluster);
|
||||
assertNotMarkedAsDeleted(s2path, cluster);
|
||||
|
||||
hdfs.deleteSnapshot(snapshottableDir, "s2");
|
||||
assertNotMarkedAsDeleted(s0path, cluster);
|
||||
assertNotMarkedAsDeleted(s1path, cluster);
|
||||
assertMarkedAsDeleted(s2path, cluster);
|
||||
|
||||
hdfs.deleteSnapshot(snapshottableDir, "s1");
|
||||
assertNotMarkedAsDeleted(s0path, cluster);
|
||||
assertMarkedAsDeleted(s1path, cluster);
|
||||
assertMarkedAsDeleted(s2path, cluster);
|
||||
|
||||
// should not be gc'ed
|
||||
Thread.sleep(10*GC_PERIOD);
|
||||
assertNotMarkedAsDeleted(s0path, cluster);
|
||||
assertMarkedAsDeleted(s1path, cluster);
|
||||
assertMarkedAsDeleted(s2path, cluster);
|
||||
|
||||
hdfs.deleteSnapshot(snapshottableDir, "s0");
|
||||
Assert.assertFalse(exist(s0path, hdfs));
|
||||
|
||||
waitForGc(Arrays.asList(s1path, s2path), hdfs);
|
||||
}
|
||||
|
||||
static boolean exist(Path snapshotRoot, DistributedFileSystem hdfs)
|
||||
throws IOException {
|
||||
try {
|
||||
hdfs.getFileStatus(snapshotRoot);
|
||||
return true;
|
||||
} catch (FileNotFoundException ignored) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
static void waitForGc(List<Path> snapshotPaths, DistributedFileSystem hdfs)
|
||||
throws Exception {
|
||||
final Iterator<Path> i = snapshotPaths.iterator();
|
||||
for(Path p = i.next();; Thread.sleep(GC_PERIOD)) {
|
||||
for(; !exist(p, hdfs); p = i.next()) {
|
||||
if (!i.hasNext()) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 60000)
|
||||
public void testMultipleDirs() throws Exception {
|
||||
final int numSnapshottables = 10;
|
||||
final DistributedFileSystem hdfs = cluster.getFileSystem();
|
||||
|
||||
final List<Path> snapshottableDirs = new ArrayList<>();
|
||||
for(int i = 0; i < numSnapshottables; i++) {
|
||||
final Path p = new Path("/dir" + i);
|
||||
snapshottableDirs.add(p);
|
||||
hdfs.mkdirs(p);
|
||||
hdfs.allowSnapshot(p);
|
||||
}
|
||||
|
||||
final Random random = new Random();
|
||||
final List<Path> snapshotPaths = new ArrayList<>();
|
||||
for(Path s : snapshottableDirs) {
|
||||
final int numSnapshots = random.nextInt(10) + 1;
|
||||
createSnapshots(s, numSnapshots, snapshotPaths, hdfs);
|
||||
}
|
||||
|
||||
// Randomly delete snapshots
|
||||
Collections.shuffle(snapshotPaths);
|
||||
for(Path p : snapshotPaths) {
|
||||
hdfs.deleteSnapshot(p.getParent().getParent(), p.getName());
|
||||
}
|
||||
|
||||
waitForGc(snapshotPaths, hdfs);
|
||||
}
|
||||
|
||||
static void createSnapshots(Path snapshottableDir, int numSnapshots,
|
||||
List<Path> snapshotPaths, DistributedFileSystem hdfs)
|
||||
throws IOException {
|
||||
for(int i = 0; i < numSnapshots; i++) {
|
||||
final Path sub = new Path(snapshottableDir, "sub" + i);
|
||||
hdfs.mkdirs(sub);
|
||||
final Path p = hdfs.createSnapshot(snapshottableDir, "s" + i);
|
||||
snapshotPaths.add(p);
|
||||
Assert.assertTrue(exist(p, hdfs));
|
||||
}
|
||||
}
|
||||
}
|
@ -156,7 +156,7 @@ public void testApplyEditLogForDeletion() throws Exception {
|
||||
assertEquals(2, cluster.getNamesystem().getSnapshotManager()
|
||||
.getNumSnapshottableDirs());
|
||||
assertEquals(2, cluster.getNamesystem().getSnapshotManager()
|
||||
.getSnapshottableDirs().length);
|
||||
.getSnapshottableDirs().size());
|
||||
|
||||
// delete /foo
|
||||
hdfs.delete(foo, true);
|
||||
@ -165,7 +165,7 @@ public void testApplyEditLogForDeletion() throws Exception {
|
||||
assertEquals(0, cluster.getNamesystem().getSnapshotManager()
|
||||
.getNumSnapshottableDirs());
|
||||
assertEquals(0, cluster.getNamesystem().getSnapshotManager()
|
||||
.getSnapshottableDirs().length);
|
||||
.getSnapshottableDirs().size());
|
||||
hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
|
||||
hdfs.saveNamespace();
|
||||
hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
|
||||
|
Loading…
Reference in New Issue
Block a user