HDFS-15481. Ordered snapshot deletion: garbage collect deleted snapshots (#2165)

This commit is contained in:
Tsz-Wo Nicholas Sze 2020-07-30 10:36:51 -07:00 committed by GitHub
parent e0c9653166
commit 05b3337a46
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 517 additions and 53 deletions

View File

@ -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 =

View File

@ -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";

View File

@ -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.");
}
}

View File

@ -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;
}

View File

@ -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

View File

@ -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);
}
/**

View File

@ -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) {

View File

@ -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);

View File

@ -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());
}

View File

@ -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);

View File

@ -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);
}
}
}

View File

@ -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;
}
}

View File

@ -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.

View File

@ -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));
}
}
}

View File

@ -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);