HDFS-6757. Simplify lease manager with INodeID. Contributed by Haohui Mai.

This commit is contained in:
Haohui Mai 2015-05-08 23:04:31 -07:00
parent 6471d18bc7
commit 00fe1ed3a4
27 changed files with 414 additions and 458 deletions

View File

@ -41,7 +41,8 @@ class FSDirDeleteOp {
*/ */
static long delete( static long delete(
FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks, FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
List<INode> removedINodes, long mtime) throws IOException { List<INode> removedINodes, List<Long> removedUCFiles,
long mtime) throws IOException {
if (NameNode.stateChangeLog.isDebugEnabled()) { if (NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + iip.getPath()); NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + iip.getPath());
} }
@ -54,7 +55,7 @@ class FSDirDeleteOp {
List<INodeDirectory> snapshottableDirs = new ArrayList<>(); List<INodeDirectory> snapshottableDirs = new ArrayList<>();
FSDirSnapshotOp.checkSnapshot(iip.getLastINode(), snapshottableDirs); FSDirSnapshotOp.checkSnapshot(iip.getLastINode(), snapshottableDirs);
filesRemoved = unprotectedDelete(fsd, iip, collectedBlocks, filesRemoved = unprotectedDelete(fsd, iip, collectedBlocks,
removedINodes, mtime); removedINodes, removedUCFiles, mtime);
fsd.getFSNamesystem().removeSnapshottableDirs(snapshottableDirs); fsd.getFSNamesystem().removeSnapshottableDirs(snapshottableDirs);
} }
} finally { } finally {
@ -118,6 +119,7 @@ class FSDirDeleteOp {
FSNamesystem fsn = fsd.getFSNamesystem(); FSNamesystem fsn = fsd.getFSNamesystem();
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo(); BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
List<INode> removedINodes = new ChunkedArrayList<>(); List<INode> removedINodes = new ChunkedArrayList<>();
List<Long> removedUCFiles = new ChunkedArrayList<>();
final INodesInPath iip = fsd.getINodesInPath4Write( final INodesInPath iip = fsd.getINodesInPath4Write(
FSDirectory.normalizePath(src), false); FSDirectory.normalizePath(src), false);
@ -127,11 +129,11 @@ class FSDirDeleteOp {
List<INodeDirectory> snapshottableDirs = new ArrayList<>(); List<INodeDirectory> snapshottableDirs = new ArrayList<>();
FSDirSnapshotOp.checkSnapshot(iip.getLastINode(), snapshottableDirs); FSDirSnapshotOp.checkSnapshot(iip.getLastINode(), snapshottableDirs);
long filesRemoved = unprotectedDelete( long filesRemoved = unprotectedDelete(
fsd, iip, collectedBlocks, removedINodes, mtime); fsd, iip, collectedBlocks, removedINodes, removedUCFiles, mtime);
fsn.removeSnapshottableDirs(snapshottableDirs); fsn.removeSnapshottableDirs(snapshottableDirs);
if (filesRemoved >= 0) { if (filesRemoved >= 0) {
fsn.removeLeasesAndINodes(src, removedINodes, false); fsn.removeLeasesAndINodes(removedUCFiles, removedINodes, false);
fsn.removeBlocksAndUpdateSafemodeTotal(collectedBlocks); fsn.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
} }
} }
@ -163,18 +165,19 @@ class FSDirDeleteOp {
FSDirectory fsd = fsn.getFSDirectory(); FSDirectory fsd = fsn.getFSDirectory();
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo(); BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
List<INode> removedINodes = new ChunkedArrayList<>(); List<INode> removedINodes = new ChunkedArrayList<>();
List<Long> removedUCFiles = new ChunkedArrayList<>();
long mtime = now(); long mtime = now();
// Unlink the target directory from directory tree // Unlink the target directory from directory tree
long filesRemoved = delete( long filesRemoved = delete(
fsd, iip, collectedBlocks, removedINodes, mtime); fsd, iip, collectedBlocks, removedINodes, removedUCFiles, mtime);
if (filesRemoved < 0) { if (filesRemoved < 0) {
return null; return null;
} }
fsd.getEditLog().logDelete(src, mtime, logRetryCache); fsd.getEditLog().logDelete(src, mtime, logRetryCache);
incrDeletedFileCount(filesRemoved); incrDeletedFileCount(filesRemoved);
fsn.removeLeasesAndINodes(src, removedINodes, true); fsn.removeLeasesAndINodes(removedUCFiles, removedINodes, true);
if (NameNode.stateChangeLog.isDebugEnabled()) { if (NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("DIR* Namesystem.delete: " NameNode.stateChangeLog.debug("DIR* Namesystem.delete: "
@ -212,12 +215,13 @@ class FSDirDeleteOp {
* @param iip the inodes resolved from the path * @param iip the inodes resolved from the path
* @param collectedBlocks blocks collected from the deleted path * @param collectedBlocks blocks collected from the deleted path
* @param removedINodes inodes that should be removed from inodeMap * @param removedINodes inodes that should be removed from inodeMap
* @param removedUCFiles inodes whose leases need to be released
* @param mtime the time the inode is removed * @param mtime the time the inode is removed
* @return the number of inodes deleted; 0 if no inodes are deleted. * @return the number of inodes deleted; 0 if no inodes are deleted.
*/ */
private static long unprotectedDelete( private static long unprotectedDelete(
FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks, FSDirectory fsd, INodesInPath iip, BlocksMapUpdateInfo collectedBlocks,
List<INode> removedINodes, long mtime) { List<INode> removedINodes, List<Long> removedUCFiles, long mtime) {
assert fsd.hasWriteLock(); assert fsd.hasWriteLock();
// check if target node exists // check if target node exists
@ -248,11 +252,11 @@ class FSDirDeleteOp {
// collect block and update quota // collect block and update quota
if (!targetNode.isInLatestSnapshot(latestSnapshot)) { if (!targetNode.isInLatestSnapshot(latestSnapshot)) {
targetNode.destroyAndCollectBlocks(fsd.getBlockStoragePolicySuite(), targetNode.destroyAndCollectBlocks(fsd.getBlockStoragePolicySuite(),
collectedBlocks, removedINodes); collectedBlocks, removedINodes, removedUCFiles);
} else { } else {
QuotaCounts counts = targetNode.cleanSubtree( QuotaCounts counts = targetNode.cleanSubtree(
fsd.getBlockStoragePolicySuite(), CURRENT_STATE_ID, fsd.getBlockStoragePolicySuite(), CURRENT_STATE_ID,
latestSnapshot, collectedBlocks, removedINodes); latestSnapshot, collectedBlocks, removedINodes, removedUCFiles);
removed = counts.getNameSpace(); removed = counts.getNameSpace();
fsd.updateCountNoQuotaCheck(iip, iip.length() -1, counts.negation()); fsd.updateCountNoQuotaCheck(iip, iip.length() -1, counts.negation());
} }

View File

@ -680,8 +680,6 @@ class FSDirRenameOp {
srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshotId()); srcParent.updateModificationTime(timestamp, srcIIP.getLatestSnapshotId());
final INode dstParent = dstParentIIP.getLastINode(); final INode dstParent = dstParentIIP.getLastINode();
dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshotId()); dstParent.updateModificationTime(timestamp, dstIIP.getLatestSnapshotId());
// update moved lease with new filename
fsd.getFSNamesystem().unprotectedChangeLease(src, dst);
} }
void restoreSource() throws QuotaExceededException { void restoreSource() throws QuotaExceededException {
@ -731,16 +729,20 @@ class FSDirRenameOp {
throws QuotaExceededException { throws QuotaExceededException {
Preconditions.checkState(oldDstChild != null); Preconditions.checkState(oldDstChild != null);
List<INode> removedINodes = new ChunkedArrayList<>(); List<INode> removedINodes = new ChunkedArrayList<>();
List<Long> removedUCFiles = new ChunkedArrayList<>();
final boolean filesDeleted; final boolean filesDeleted;
if (!oldDstChild.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) { if (!oldDstChild.isInLatestSnapshot(dstIIP.getLatestSnapshotId())) {
oldDstChild.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); oldDstChild.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
filesDeleted = true; filesDeleted = true;
} else { } else {
filesDeleted = oldDstChild.cleanSubtree(bsps, Snapshot.CURRENT_STATE_ID, filesDeleted = oldDstChild.cleanSubtree(
dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes) bsps, Snapshot.CURRENT_STATE_ID,
.getNameSpace() >= 0; dstIIP.getLatestSnapshotId(), collectedBlocks,
removedINodes, removedUCFiles).getNameSpace() >= 0;
} }
fsd.getFSNamesystem().removeLeasesAndINodes(src, removedINodes, false); fsd.getFSNamesystem().removeLeasesAndINodes(
removedUCFiles, removedINodes, false);
return filesDeleted; return filesDeleted;
} }

View File

@ -29,6 +29,7 @@ import java.util.EnumMap;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.List; import java.util.List;
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
@ -374,7 +375,7 @@ public class FSEditLogLoader {
addCloseOp.clientMachine, addCloseOp.clientMachine,
addCloseOp.storagePolicyId); addCloseOp.storagePolicyId);
iip = INodesInPath.replace(iip, iip.length() - 1, newFile); iip = INodesInPath.replace(iip, iip.length() - 1, newFile);
fsNamesys.leaseManager.addLease(addCloseOp.clientName, path); fsNamesys.leaseManager.addLease(addCloseOp.clientName, newFile.getId());
// add the op into retry cache if necessary // add the op into retry cache if necessary
if (toAddRetryCache) { if (toAddRetryCache) {
@ -446,9 +447,9 @@ public class FSEditLogLoader {
"File is not under construction: " + path); "File is not under construction: " + path);
} }
// One might expect that you could use removeLease(holder, path) here, // One might expect that you could use removeLease(holder, path) here,
// but OP_CLOSE doesn't serialize the holder. So, remove by path. // but OP_CLOSE doesn't serialize the holder. So, remove the inode.
if (file.isUnderConstruction()) { if (file.isUnderConstruction()) {
fsNamesys.leaseManager.removeLeaseWithPrefixPath(path); fsNamesys.leaseManager.removeLeases(Lists.newArrayList(file.getId()));
file.toCompleteFile(file.getModificationTime()); file.toCompleteFile(file.getModificationTime());
} }
break; break;
@ -701,8 +702,8 @@ public class FSEditLogLoader {
renameReservedPathsOnUpgrade(reassignLeaseOp.path, logVersion); renameReservedPathsOnUpgrade(reassignLeaseOp.path, logVersion);
INodeFile pendingFile = fsDir.getINode(path).asFile(); INodeFile pendingFile = fsDir.getINode(path).asFile();
Preconditions.checkState(pendingFile.isUnderConstruction()); Preconditions.checkState(pendingFile.isUnderConstruction());
fsNamesys.reassignLeaseInternal(lease, fsNamesys.reassignLeaseInternal(lease, reassignLeaseOp.newHolder,
path, reassignLeaseOp.newHolder, pendingFile); pendingFile);
break; break;
} }
case OP_START_LOG_SEGMENT: case OP_START_LOG_SEGMENT:

View File

@ -966,8 +966,7 @@ public class FSImageFormat {
} }
if (!inSnapshot) { if (!inSnapshot) {
namesystem.leaseManager.addLease(cons namesystem.leaseManager.addLease(uc.getClientName(), oldnode.getId());
.getFileUnderConstructionFeature().getClientName(), path);
} }
} }
} }
@ -1297,7 +1296,7 @@ public class FSImageFormat {
// paths, so that when loading fsimage we do not put them into the lease // paths, so that when loading fsimage we do not put them into the lease
// map. In the future, we can remove this hack when we can bump the // map. In the future, we can remove this hack when we can bump the
// layout version. // layout version.
sourceNamesystem.saveFilesUnderConstruction(out, snapshotUCMap); saveFilesUnderConstruction(sourceNamesystem, out, snapshotUCMap);
context.checkCancelled(); context.checkCancelled();
sourceNamesystem.saveSecretManagerStateCompat(out, sdPath); sourceNamesystem.saveSecretManagerStateCompat(out, sdPath);
@ -1448,5 +1447,46 @@ public class FSImageFormat {
counter.increment(); counter.increment();
} }
} }
/**
* Serializes leases.
*/
void saveFilesUnderConstruction(FSNamesystem fsn, DataOutputStream out,
Map<Long, INodeFile> snapshotUCMap) throws IOException {
// This is run by an inferior thread of saveNamespace, which holds a read
// lock on our behalf. If we took the read lock here, we could block
// for fairness if a writer is waiting on the lock.
final LeaseManager leaseManager = fsn.getLeaseManager();
final FSDirectory dir = fsn.getFSDirectory();
synchronized (leaseManager) {
Collection<Long> filesWithUC = leaseManager.getINodeIdWithLeases();
for (Long id : filesWithUC) {
// TODO: for HDFS-5428, because of rename operations, some
// under-construction files that are
// in the current fs directory can also be captured in the
// snapshotUCMap. We should remove them from the snapshotUCMap.
snapshotUCMap.remove(id);
}
out.writeInt(filesWithUC.size() + snapshotUCMap.size()); // write the size
for (Long id : filesWithUC) {
INodeFile file = dir.getInode(id).asFile();
String path = file.getFullPathName();
FSImageSerialization.writeINodeUnderConstruction(
out, file, path);
}
for (Map.Entry<Long, INodeFile> entry : snapshotUCMap.entrySet()) {
// for those snapshot INodeFileUC, we use "/.reserved/.inodes/<inodeid>"
// as their paths
StringBuilder b = new StringBuilder();
b.append(FSDirectory.DOT_RESERVED_PATH_PREFIX)
.append(Path.SEPARATOR).append(FSDirectory.DOT_INODES_STRING)
.append(Path.SEPARATOR).append(entry.getValue().getId());
FSImageSerialization.writeINodeUnderConstruction(
out, entry.getValue(), b.toString());
}
}
}
} }
} }

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -279,7 +280,8 @@ public final class FSImageFormatPBINode {
INodeFile file = dir.getInode(entry.getInodeId()).asFile(); INodeFile file = dir.getInode(entry.getInodeId()).asFile();
FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature(); FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
Preconditions.checkState(uc != null); // file must be under-construction Preconditions.checkState(uc != null); // file must be under-construction
fsn.leaseManager.addLease(uc.getClientName(), entry.getFullPath()); fsn.leaseManager.addLease(uc.getClientName(),
entry.getInodeId());
} }
} }
@ -576,10 +578,21 @@ public final class FSImageFormatPBINode {
} }
void serializeFilesUCSection(OutputStream out) throws IOException { void serializeFilesUCSection(OutputStream out) throws IOException {
Map<String, INodeFile> ucMap = fsn.getFilesUnderConstruction(); Collection<Long> filesWithUC = fsn.getLeaseManager()
for (Map.Entry<String, INodeFile> entry : ucMap.entrySet()) { .getINodeIdWithLeases();
String path = entry.getKey(); for (Long id : filesWithUC) {
INodeFile file = entry.getValue(); INode inode = fsn.getFSDirectory().getInode(id);
if (inode == null) {
LOG.warn("Fail to find inode " + id + " when saving the leases.");
continue;
}
INodeFile file = inode.asFile();
if (!file.isUnderConstruction()) {
LOG.warn("Fail to save the lease for inode id " + id
+ " as the file is not under construction");
continue;
}
String path = file.getFullPathName();
FileUnderConstructionEntry.Builder b = FileUnderConstructionEntry FileUnderConstructionEntry.Builder b = FileUnderConstructionEntry
.newBuilder().setInodeId(file.getId()).setFullPath(path); .newBuilder().setInodeId(file.getId()).setFullPath(path);
FileUnderConstructionEntry e = b.build(); FileUnderConstructionEntry e = b.build();

View File

@ -2087,12 +2087,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
Block newBlock) Block newBlock)
throws IOException { throws IOException {
INodeFile file = iip.getLastINode().asFile(); INodeFile file = iip.getLastINode().asFile();
String src = iip.getPath();
file.recordModification(iip.getLatestSnapshotId()); file.recordModification(iip.getLatestSnapshotId());
file.toUnderConstruction(leaseHolder, clientMachine); file.toUnderConstruction(leaseHolder, clientMachine);
assert file.isUnderConstruction() : "inode should be under construction."; assert file.isUnderConstruction() : "inode should be under construction.";
leaseManager.addLease( leaseManager.addLease(
file.getFileUnderConstructionFeature().getClientName(), src); file.getFileUnderConstructionFeature().getClientName(), file.getId());
boolean shouldRecoverNow = (newBlock == null); boolean shouldRecoverNow = (newBlock == null);
BlockInfoContiguous oldBlock = file.getLastBlock(); BlockInfoContiguous oldBlock = file.getLastBlock();
boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock); boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
@ -2568,13 +2567,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
} else { } else {
if (overwrite) { if (overwrite) {
toRemoveBlocks = new BlocksMapUpdateInfo(); toRemoveBlocks = new BlocksMapUpdateInfo();
List<INode> toRemoveINodes = new ChunkedArrayList<INode>(); List<INode> toRemoveINodes = new ChunkedArrayList<>();
long ret = FSDirDeleteOp.delete(dir, iip, toRemoveBlocks, List<Long> toRemoveUCFiles = new ChunkedArrayList<>();
toRemoveINodes, now()); long ret = FSDirDeleteOp.delete(
dir, iip, toRemoveBlocks, toRemoveINodes,
toRemoveUCFiles, now());
if (ret >= 0) { if (ret >= 0) {
iip = INodesInPath.replace(iip, iip.length() - 1, null); iip = INodesInPath.replace(iip, iip.length() - 1, null);
FSDirDeleteOp.incrDeletedFileCount(ret); FSDirDeleteOp.incrDeletedFileCount(ret);
removeLeasesAndINodes(src, toRemoveINodes, true); removeLeasesAndINodes(toRemoveUCFiles, toRemoveINodes, true);
} }
} else { } else {
// If lease soft limit time is expired, recover the lease // If lease soft limit time is expired, recover the lease
@ -2601,7 +2602,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
throw new IOException("Unable to add " + src + " to namespace"); throw new IOException("Unable to add " + src + " to namespace");
} }
leaseManager.addLease(newNode.getFileUnderConstructionFeature() leaseManager.addLease(newNode.getFileUnderConstructionFeature()
.getClientName(), src); .getClientName(), newNode.getId());
// Set encryption attributes if necessary // Set encryption attributes if necessary
if (feInfo != null) { if (feInfo != null) {
@ -2745,7 +2746,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
file.toUnderConstruction(leaseHolder, clientMachine); file.toUnderConstruction(leaseHolder, clientMachine);
leaseManager.addLease( leaseManager.addLease(
file.getFileUnderConstructionFeature().getClientName(), src); file.getFileUnderConstructionFeature().getClientName(), file.getId());
LocatedBlock ret = null; LocatedBlock ret = null;
if (!newBlock) { if (!newBlock) {
@ -2897,7 +2898,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
Lease lease = leaseManager.getLease(holder); Lease lease = leaseManager.getLease(holder);
if (!force && lease != null) { if (!force && lease != null) {
Lease leaseFile = leaseManager.getLeaseByPath(src); Lease leaseFile = leaseManager.getLease(file);
if (leaseFile != null && leaseFile.equals(lease)) { if (leaseFile != null && leaseFile.equals(lease)) {
// We found the lease for this file but the original // We found the lease for this file but the original
// holder is trying to obtain it again. // holder is trying to obtain it again.
@ -3758,15 +3759,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
/** /**
* Remove leases and inodes related to a given path * Remove leases and inodes related to a given path
* @param src The given path * @param removedUCFiles INodes whose leases need to be released
* @param removedINodes Containing the list of inodes to be removed from * @param removedINodes Containing the list of inodes to be removed from
* inodesMap * inodesMap
* @param acquireINodeMapLock Whether to acquire the lock for inode removal * @param acquireINodeMapLock Whether to acquire the lock for inode removal
*/ */
void removeLeasesAndINodes(String src, List<INode> removedINodes, void removeLeasesAndINodes(List<Long> removedUCFiles,
List<INode> removedINodes,
final boolean acquireINodeMapLock) { final boolean acquireINodeMapLock) {
assert hasWriteLock(); assert hasWriteLock();
leaseManager.removeLeaseWithPrefixPath(src); leaseManager.removeLeases(removedUCFiles);
// remove inodes from inodesMap // remove inodes from inodesMap
if (removedINodes != null) { if (removedINodes != null) {
if (acquireINodeMapLock) { if (acquireINodeMapLock) {
@ -4156,14 +4158,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return lease; return lease;
// The following transaction is not synced. Make sure it's sync'ed later. // The following transaction is not synced. Make sure it's sync'ed later.
logReassignLease(lease.getHolder(), src, newHolder); logReassignLease(lease.getHolder(), src, newHolder);
return reassignLeaseInternal(lease, src, newHolder, pendingFile); return reassignLeaseInternal(lease, newHolder, pendingFile);
} }
Lease reassignLeaseInternal(Lease lease, String src, String newHolder, Lease reassignLeaseInternal(Lease lease, String newHolder, INodeFile pendingFile) {
INodeFile pendingFile) {
assert hasWriteLock(); assert hasWriteLock();
pendingFile.getFileUnderConstructionFeature().setClientName(newHolder); pendingFile.getFileUnderConstructionFeature().setClientName(newHolder);
return leaseManager.reassignLease(lease, src, newHolder); return leaseManager.reassignLease(lease, pendingFile, newHolder);
} }
private void commitOrCompleteLastBlock(final INodeFile fileINode, private void commitOrCompleteLastBlock(final INodeFile fileINode,
@ -4191,7 +4192,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature(); FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature();
Preconditions.checkArgument(uc != null); Preconditions.checkArgument(uc != null);
leaseManager.removeLease(uc.getClientName(), src); leaseManager.removeLease(uc.getClientName(), pendingFile);
pendingFile.recordModification(latestSnapshot); pendingFile.recordModification(latestSnapshot);
@ -6401,58 +6402,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
persistBlocks(src, pendingFile, logRetryCache); persistBlocks(src, pendingFile, logRetryCache);
} }
// rename was successful. If any part of the renamed subtree had
// files that were being written to, update with new filename.
void unprotectedChangeLease(String src, String dst) {
assert hasWriteLock();
leaseManager.changeLease(src, dst);
}
/**
* Serializes leases.
*/
void saveFilesUnderConstruction(DataOutputStream out,
Map<Long, INodeFile> snapshotUCMap) throws IOException {
// This is run by an inferior thread of saveNamespace, which holds a read
// lock on our behalf. If we took the read lock here, we could block
// for fairness if a writer is waiting on the lock.
synchronized (leaseManager) {
Map<String, INodeFile> nodes = leaseManager.getINodesUnderConstruction();
for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) {
// TODO: for HDFS-5428, because of rename operations, some
// under-construction files that are
// in the current fs directory can also be captured in the
// snapshotUCMap. We should remove them from the snapshotUCMap.
snapshotUCMap.remove(entry.getValue().getId());
}
out.writeInt(nodes.size() + snapshotUCMap.size()); // write the size
for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) {
FSImageSerialization.writeINodeUnderConstruction(
out, entry.getValue(), entry.getKey());
}
for (Map.Entry<Long, INodeFile> entry : snapshotUCMap.entrySet()) {
// for those snapshot INodeFileUC, we use "/.reserved/.inodes/<inodeid>"
// as their paths
StringBuilder b = new StringBuilder();
b.append(FSDirectory.DOT_RESERVED_PATH_PREFIX)
.append(Path.SEPARATOR).append(FSDirectory.DOT_INODES_STRING)
.append(Path.SEPARATOR).append(entry.getValue().getId());
FSImageSerialization.writeINodeUnderConstruction(
out, entry.getValue(), b.toString());
}
}
}
/**
* @return all the under-construction files in the lease map
*/
Map<String, INodeFile> getFilesUnderConstruction() {
synchronized (leaseManager) {
return leaseManager.getINodesUnderConstruction();
}
}
/** /**
* Register a Backup name-node, verifying that it belongs * Register a Backup name-node, verifying that it belongs
* to the correct namespace, and adding it to the set of * to the correct namespace, and adding it to the set of

View File

@ -403,12 +403,14 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
* @param removedINodes * @param removedINodes
* INodes collected from the descents for further cleaning up of * INodes collected from the descents for further cleaning up of
* inodeMap * inodeMap
* @param removedUCFiles
* INodes whose leases need to be released
* @return quota usage delta when deleting a snapshot * @return quota usage delta when deleting a snapshot
*/ */
public abstract QuotaCounts cleanSubtree(final BlockStoragePolicySuite bsps, public abstract QuotaCounts cleanSubtree(
final int snapshotId, final BlockStoragePolicySuite bsps, final int snapshotId,
int priorSnapshotId, BlocksMapUpdateInfo collectedBlocks, int priorSnapshotId, BlocksMapUpdateInfo collectedBlocks,
List<INode> removedINodes); List<INode> removedINodes, List<Long> removedUCFiles);
/** /**
* Destroy self and clear everything! If the INode is a file, this method * Destroy self and clear everything! If the INode is a file, this method
@ -416,7 +418,6 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
* directory, the method goes down the subtree and collects blocks from the * directory, the method goes down the subtree and collects blocks from the
* descents, and clears its parent/children references as well. The method * descents, and clears its parent/children references as well. The method
* also clears the diff list if the INode contains snapshot diff list. * also clears the diff list if the INode contains snapshot diff list.
*
* @param bsps * @param bsps
* block storage policy suite to calculate intended storage type usage * block storage policy suite to calculate intended storage type usage
* This is needed because INodeReference#destroyAndCollectBlocks() needs * This is needed because INodeReference#destroyAndCollectBlocks() needs
@ -427,10 +428,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
* @param removedINodes * @param removedINodes
* INodes collected from the descents for further cleaning up of * INodes collected from the descents for further cleaning up of
* inodeMap * inodeMap
* @param removedUCFiles
* INodes whose leases need to be released
*/ */
public abstract void destroyAndCollectBlocks( public abstract void destroyAndCollectBlocks(
BlockStoragePolicySuite bsps, BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes); List<INode> removedINodes, List<Long> removedUCFiles);
/** Compute {@link ContentSummary}. Blocking call */ /** Compute {@link ContentSummary}. Blocking call */
public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps) { public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps) {

View File

@ -753,10 +753,11 @@ public class INodeDirectory extends INodeWithAdditionalFields
} }
/** Call cleanSubtree(..) recursively down the subtree. */ /** Call cleanSubtree(..) recursively down the subtree. */
public QuotaCounts cleanSubtreeRecursively(final BlockStoragePolicySuite bsps, public QuotaCounts cleanSubtreeRecursively(
final int snapshot, final BlockStoragePolicySuite bsps, final int snapshot, int prior,
int prior, final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes, final Map<INode, INode> excludedNodes) { final List<INode> removedINodes, List<Long> removedUCFiles,
final Map<INode, INode> excludedNodes) {
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
// in case of deletion snapshot, since this call happens after we modify // in case of deletion snapshot, since this call happens after we modify
// the diff list, the snapshot to be deleted has been combined or renamed // the diff list, the snapshot to be deleted has been combined or renamed
@ -771,7 +772,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
continue; continue;
} else { } else {
QuotaCounts childCounts = child.cleanSubtree(bsps, snapshot, prior, QuotaCounts childCounts = child.cleanSubtree(bsps, snapshot, prior,
collectedBlocks, removedINodes); collectedBlocks, removedINodes, removedUCFiles);
counts.add(childCounts); counts.add(childCounts);
} }
} }
@ -779,15 +780,17 @@ public class INodeDirectory extends INodeWithAdditionalFields
} }
@Override @Override
public void destroyAndCollectBlocks(final BlockStoragePolicySuite bsps, public void destroyAndCollectBlocks(
final BlockStoragePolicySuite bsps,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature(); final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
if (sf != null) { if (sf != null) {
sf.clear(bsps, this, collectedBlocks, removedINodes); sf.clear(bsps, this, collectedBlocks, removedINodes, removedUCFiles);
} }
for (INode child : getChildrenList(Snapshot.CURRENT_STATE_ID)) { for (INode child : getChildrenList(Snapshot.CURRENT_STATE_ID)) {
child.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); child.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
} }
if (getAclFeature() != null) { if (getAclFeature() != null) {
AclStorage.removeAclFeature(getAclFeature()); AclStorage.removeAclFeature(getAclFeature());
@ -797,15 +800,15 @@ public class INodeDirectory extends INodeWithAdditionalFields
} }
@Override @Override
public QuotaCounts cleanSubtree(final BlockStoragePolicySuite bsps, public QuotaCounts cleanSubtree(
final int snapshotId, int priorSnapshotId, final BlockStoragePolicySuite bsps, final int snapshotId, int priorSnapshotId,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature(); DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
// there is snapshot data // there is snapshot data
if (sf != null) { if (sf != null) {
return sf.cleanDirectory(bsps, this, snapshotId, priorSnapshotId, return sf.cleanDirectory(bsps, this, snapshotId, priorSnapshotId,
collectedBlocks, removedINodes); collectedBlocks, removedINodes, removedUCFiles);
} }
// there is no snapshot data // there is no snapshot data
if (priorSnapshotId == Snapshot.NO_SNAPSHOT_ID if (priorSnapshotId == Snapshot.NO_SNAPSHOT_ID
@ -813,12 +816,13 @@ public class INodeDirectory extends INodeWithAdditionalFields
// destroy the whole subtree and collect blocks that should be deleted // destroy the whole subtree and collect blocks that should be deleted
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
this.computeQuotaUsage(bsps, counts, true); this.computeQuotaUsage(bsps, counts, true);
destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
return counts; return counts;
} else { } else {
// process recursively down the subtree // process recursively down the subtree
QuotaCounts counts = cleanSubtreeRecursively(bsps, snapshotId, priorSnapshotId, QuotaCounts counts = cleanSubtreeRecursively(bsps, snapshotId, priorSnapshotId,
collectedBlocks, removedINodes, null); collectedBlocks, removedINodes, removedUCFiles, null);
if (isQuotaSet()) { if (isQuotaSet()) {
getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(counts.negation()); getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(counts.negation());
} }

View File

@ -494,28 +494,33 @@ public class INodeFile extends INodeWithAdditionalFields
} }
@Override @Override
public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, final int snapshot, public QuotaCounts cleanSubtree(
int priorSnapshotId, BlockStoragePolicySuite bsps, final int snapshot, int priorSnapshotId,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
if (sf != null) { if (sf != null) {
return sf.cleanFile(bsps, this, snapshot, priorSnapshotId, collectedBlocks, return sf.cleanFile(bsps, this, snapshot, priorSnapshotId, collectedBlocks,
removedINodes); removedINodes);
} }
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
if (snapshot == CURRENT_STATE_ID) { if (snapshot == CURRENT_STATE_ID) {
if (priorSnapshotId == NO_SNAPSHOT_ID) { if (priorSnapshotId == NO_SNAPSHOT_ID) {
// this only happens when deleting the current file and the file is not // this only happens when deleting the current file and the file is not
// in any snapshot // in any snapshot
computeQuotaUsage(bsps, counts, false); computeQuotaUsage(bsps, counts, false);
destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
} else { } else {
FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
// when deleting the current file and the file is in snapshot, we should // when deleting the current file and the file is in snapshot, we should
// clean the 0-sized block if the file is UC // clean the 0-sized block if the file is UC
FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
if (uc != null) { if (uc != null) {
uc.cleanZeroSizeBlock(this, collectedBlocks); uc.cleanZeroSizeBlock(this, collectedBlocks);
if (removedUCFiles != null) {
removedUCFiles.add(getId());
}
} }
} }
} }
@ -523,8 +528,9 @@ public class INodeFile extends INodeWithAdditionalFields
} }
@Override @Override
public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps, public void destroyAndCollectBlocks(
BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) { BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes, List<Long> removedUCFiles) {
if (blocks != null && collectedBlocks != null) { if (blocks != null && collectedBlocks != null) {
for (BlockInfoContiguous blk : blocks) { for (BlockInfoContiguous blk : blocks) {
collectedBlocks.addDeleteBlock(blk); collectedBlocks.addDeleteBlock(blk);
@ -542,6 +548,9 @@ public class INodeFile extends INodeWithAdditionalFields
sf.getDiffs().destroyAndCollectSnapshotBlocks(collectedBlocks); sf.getDiffs().destroyAndCollectSnapshotBlocks(collectedBlocks);
sf.clearDiffs(); sf.clearDiffs();
} }
if (isUnderConstruction() && removedUCFiles != null) {
removedUCFiles.add(getId());
}
} }
@Override @Override

View File

@ -97,8 +97,9 @@ public class INodeMap {
} }
@Override @Override
public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps, public void destroyAndCollectBlocks(
BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) { BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
List<INode> removedINodes, List<Long> removedUCFiles) {
// Nothing to do // Nothing to do
} }
@ -116,9 +117,10 @@ public class INodeMap {
} }
@Override @Override
public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, public QuotaCounts cleanSubtree(
int snapshotId, int priorSnapshotId, BlockStoragePolicySuite bsps, int snapshotId, int priorSnapshotId,
BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) { BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
List<Long> removedUCFiles) {
return null; return null;
} }

View File

@ -302,19 +302,20 @@ public abstract class INodeReference extends INode {
} }
@Override // used by WithCount @Override // used by WithCount
public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, int snapshot, public QuotaCounts cleanSubtree(
int prior, BlocksMapUpdateInfo collectedBlocks, BlockStoragePolicySuite bsps, int snapshot, int prior, BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
return referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks, return referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks,
removedINodes); removedINodes, removedUCFiles);
} }
@Override // used by WithCount @Override // used by WithCount
public void destroyAndCollectBlocks( public void destroyAndCollectBlocks(
BlockStoragePolicySuite bsps, BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
if (removeReference(this) <= 0) { if (removeReference(this) <= 0) {
referred.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); referred.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
} }
} }
@ -542,9 +543,9 @@ public abstract class INodeReference extends INode {
} }
@Override @Override
public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, public QuotaCounts cleanSubtree(
final int snapshot, int prior, final BlocksMapUpdateInfo collectedBlocks, BlockStoragePolicySuite bsps, final int snapshot, int prior, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
// since WithName node resides in deleted list acting as a snapshot copy, // since WithName node resides in deleted list acting as a snapshot copy,
// the parameter snapshot must be non-null // the parameter snapshot must be non-null
Preconditions.checkArgument(snapshot != Snapshot.CURRENT_STATE_ID); Preconditions.checkArgument(snapshot != Snapshot.CURRENT_STATE_ID);
@ -560,7 +561,7 @@ public abstract class INodeReference extends INode {
} }
QuotaCounts counts = getReferredINode().cleanSubtree(bsps, snapshot, prior, QuotaCounts counts = getReferredINode().cleanSubtree(bsps, snapshot, prior,
collectedBlocks, removedINodes); collectedBlocks, removedINodes, removedUCFiles);
INodeReference ref = getReferredINode().getParentReference(); INodeReference ref = getReferredINode().getParentReference();
if (ref != null) { if (ref != null) {
try { try {
@ -581,13 +582,13 @@ public abstract class INodeReference extends INode {
} }
@Override @Override
public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps, public void destroyAndCollectBlocks(
BlocksMapUpdateInfo collectedBlocks, BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
int snapshot = getSelfSnapshot(); int snapshot = getSelfSnapshot();
if (removeReference(this) <= 0) { if (removeReference(this) <= 0) {
getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks, getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks,
removedINodes); removedINodes, removedUCFiles);
} else { } else {
int prior = getPriorSnapshot(this); int prior = getPriorSnapshot(this);
INode referred = getReferredINode().asReference().getReferredINode(); INode referred = getReferredINode().asReference().getReferredINode();
@ -607,7 +608,7 @@ public abstract class INodeReference extends INode {
} }
try { try {
QuotaCounts counts = referred.cleanSubtree(bsps, snapshot, prior, QuotaCounts counts = referred.cleanSubtree(bsps, snapshot, prior,
collectedBlocks, removedINodes); collectedBlocks, removedINodes, removedUCFiles);
INodeReference ref = getReferredINode().getParentReference(); INodeReference ref = getReferredINode().getParentReference();
if (ref != null) { if (ref != null) {
ref.addSpaceConsumed(counts.negation(), true); ref.addSpaceConsumed(counts.negation(), true);
@ -661,13 +662,16 @@ public abstract class INodeReference extends INode {
} }
@Override @Override
public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, int snapshot, int prior, public QuotaCounts cleanSubtree(
BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes) { BlockStoragePolicySuite bsps, int snapshot, int prior,
BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes,
List<Long> removedUCFiles) {
if (snapshot == Snapshot.CURRENT_STATE_ID if (snapshot == Snapshot.CURRENT_STATE_ID
&& prior == Snapshot.NO_SNAPSHOT_ID) { && prior == Snapshot.NO_SNAPSHOT_ID) {
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
this.computeQuotaUsage(bsps, counts, true); this.computeQuotaUsage(bsps, counts, true);
destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
return counts; return counts;
} else { } else {
// if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to // if prior is NO_SNAPSHOT_ID, we need to check snapshot belonging to
@ -684,7 +688,7 @@ public abstract class INodeReference extends INode {
return new QuotaCounts.Builder().build(); return new QuotaCounts.Builder().build();
} }
return getReferredINode().cleanSubtree(bsps, snapshot, prior, return getReferredINode().cleanSubtree(bsps, snapshot, prior,
collectedBlocks, removedINodes); collectedBlocks, removedINodes, removedUCFiles);
} }
} }
@ -699,11 +703,12 @@ public abstract class INodeReference extends INode {
* WithName nodes. * WithName nodes.
*/ */
@Override @Override
public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps, public void destroyAndCollectBlocks(
BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) { BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes, List<Long> removedUCFiles) {
if (removeReference(this) <= 0) { if (removeReference(this) <= 0) {
getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks, getReferredINode().destroyAndCollectBlocks(bsps, collectedBlocks,
removedINodes); removedINodes, removedUCFiles);
} else { } else {
// we will clean everything, including files, directories, and // we will clean everything, including files, directories, and
// snapshots, that were created after this prior snapshot // snapshots, that were created after this prior snapshot
@ -726,7 +731,7 @@ public abstract class INodeReference extends INode {
// compute quota usage updates before calling this destroy // compute quota usage updates before calling this destroy
// function, we use true for countDiffChange // function, we use true for countDiffChange
referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks, referred.cleanSubtree(bsps, snapshot, prior, collectedBlocks,
removedINodes); removedINodes, removedUCFiles);
} else if (referred.isDirectory()) { } else if (referred.isDirectory()) {
// similarly, if referred is a directory, it must be an // similarly, if referred is a directory, it must be an
// INodeDirectory with snapshot // INodeDirectory with snapshot
@ -734,7 +739,7 @@ public abstract class INodeReference extends INode {
Preconditions.checkState(dir.isWithSnapshot()); Preconditions.checkState(dir.isWithSnapshot());
try { try {
DirectoryWithSnapshotFeature.destroyDstSubtree(bsps, dir, snapshot, DirectoryWithSnapshotFeature.destroyDstSubtree(bsps, dir, snapshot,
prior, collectedBlocks, removedINodes); prior, collectedBlocks, removedINodes, removedUCFiles);
} catch (QuotaExceededException e) { } catch (QuotaExceededException e) {
LOG.error("should not exceed quota while snapshot deletion", e); LOG.error("should not exceed quota while snapshot deletion", e);
} }

View File

@ -73,21 +73,23 @@ public class INodeSymlink extends INodeWithAdditionalFields {
} }
@Override @Override
public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, public QuotaCounts cleanSubtree(
final int snapshotId, int priorSnapshotId, BlockStoragePolicySuite bsps, final int snapshotId, int priorSnapshotId,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
if (snapshotId == Snapshot.CURRENT_STATE_ID if (snapshotId == Snapshot.CURRENT_STATE_ID
&& priorSnapshotId == Snapshot.NO_SNAPSHOT_ID) { && priorSnapshotId == Snapshot.NO_SNAPSHOT_ID) {
destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
} }
return new QuotaCounts.Builder().nameSpace(1).build(); return new QuotaCounts.Builder().nameSpace(1).build();
} }
@Override @Override
public void destroyAndCollectBlocks(final BlockStoragePolicySuite bsps, public void destroyAndCollectBlocks(
final BlockStoragePolicySuite bsps,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
removedINodes.add(this); removedINodes.add(this);
} }

View File

@ -22,20 +22,17 @@ import static org.apache.hadoop.util.Time.monotonicNow;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.PriorityQueue;
import java.util.NavigableSet;
import java.util.NoSuchElementException;
import java.util.SortedMap; import java.util.SortedMap;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
@ -78,15 +75,17 @@ public class LeaseManager {
// Used for handling lock-leases // Used for handling lock-leases
// Mapping: leaseHolder -> Lease // Mapping: leaseHolder -> Lease
// //
private final SortedMap<String, Lease> leases = new TreeMap<String, Lease>(); private final SortedMap<String, Lease> leases = new TreeMap<>();
// Set of: Lease // Set of: Lease
private final NavigableSet<Lease> sortedLeases = new TreeSet<Lease>(); private final PriorityQueue<Lease> sortedLeases = new PriorityQueue<>(512,
new Comparator<Lease>() {
// @Override
// Map path names to leases. It is protected by the sortedLeases lock. public int compare(Lease o1, Lease o2) {
// The map stores pathnames in lexicographical order. return Long.signum(o1.getLastUpdate() - o2.getLastUpdate());
// }
private final SortedMap<String, Lease> sortedLeasesByPath = new TreeMap<String, Lease>(); });
// INodeID -> Lease
private final HashMap<Long, Lease> leasesById = new HashMap<>();
private Daemon lmthread; private Daemon lmthread;
private volatile boolean shouldRunMonitor; private volatile boolean shouldRunMonitor;
@ -97,60 +96,44 @@ public class LeaseManager {
return leases.get(holder); return leases.get(holder);
} }
@VisibleForTesting
int getNumSortedLeases() {return sortedLeases.size();}
/** /**
* This method iterates through all the leases and counts the number of blocks * This method iterates through all the leases and counts the number of blocks
* which are not COMPLETE. The FSNamesystem read lock MUST be held before * which are not COMPLETE. The FSNamesystem read lock MUST be held before
* calling this method. * calling this method.
* @return
*/ */
synchronized long getNumUnderConstructionBlocks() { synchronized long getNumUnderConstructionBlocks() {
assert this.fsnamesystem.hasReadLock() : "The FSNamesystem read lock wasn't" assert this.fsnamesystem.hasReadLock() : "The FSNamesystem read lock wasn't"
+ "acquired before counting under construction blocks"; + "acquired before counting under construction blocks";
long numUCBlocks = 0; long numUCBlocks = 0;
for (Lease lease : sortedLeases) { for (Long id : getINodeIdWithLeases()) {
for (String path : lease.getPaths()) { final INodeFile cons = fsnamesystem.getFSDirectory().getInode(id).asFile();
final INodeFile cons; Preconditions.checkState(cons.isUnderConstruction());
try { BlockInfoContiguous[] blocks = cons.getBlocks();
cons = this.fsnamesystem.getFSDirectory().getINode(path).asFile(); if(blocks == null) {
Preconditions.checkState(cons.isUnderConstruction()); continue;
} catch (UnresolvedLinkException e) { }
throw new AssertionError("Lease files should reside on this FS"); for(BlockInfoContiguous b : blocks) {
} if(!b.isComplete())
BlockInfoContiguous[] blocks = cons.getBlocks(); numUCBlocks++;
if(blocks == null)
continue;
for(BlockInfoContiguous b : blocks) {
if(!b.isComplete())
numUCBlocks++;
}
} }
} }
LOG.info("Number of blocks under construction: " + numUCBlocks); LOG.info("Number of blocks under construction: " + numUCBlocks);
return numUCBlocks; return numUCBlocks;
} }
Collection<Long> getINodeIdWithLeases() {return leasesById.keySet();}
/** @return the lease containing src */ /** @return the lease containing src */
public Lease getLeaseByPath(String src) {return sortedLeasesByPath.get(src);} public synchronized Lease getLease(INodeFile src) {return leasesById.get(src.getId());}
/** @return the number of leases currently in the system */ /** @return the number of leases currently in the system */
@VisibleForTesting
public synchronized int countLease() {return sortedLeases.size();} public synchronized int countLease() {return sortedLeases.size();}
/** @return the number of paths contained in all leases */
synchronized int countPath() {
int count = 0;
for(Lease lease : sortedLeases) {
count += lease.getPaths().size();
}
return count;
}
/** /**
* Adds (or re-adds) the lease for the specified file. * Adds (or re-adds) the lease for the specified file.
*/ */
synchronized Lease addLease(String holder, String src) { synchronized Lease addLease(String holder, long inodeId) {
Lease lease = getLease(holder); Lease lease = getLease(holder);
if (lease == null) { if (lease == null) {
lease = new Lease(holder); lease = new Lease(holder);
@ -159,23 +142,24 @@ public class LeaseManager {
} else { } else {
renewLease(lease); renewLease(lease);
} }
sortedLeasesByPath.put(src, lease); leasesById.put(inodeId, lease);
lease.paths.add(src); lease.files.add(inodeId);
return lease; return lease;
} }
/** /**
* Remove the specified lease and src. * Remove the specified lease and src.
*/ */
synchronized void removeLease(Lease lease, String src) { private synchronized void removeLease(Lease lease, long inodeId) {
sortedLeasesByPath.remove(src); leasesById.remove(inodeId);
if (!lease.removePath(src)) { if (!lease.removeFile(inodeId)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(src + " not found in lease.paths (=" + lease.paths + ")"); LOG.debug("inode " + inodeId + " not found in lease.files (=" + lease
+ ")");
} }
} }
if (!lease.hasPath()) { if (!lease.hasFiles()) {
leases.remove(lease.holder); leases.remove(lease.holder);
if (!sortedLeases.remove(lease)) { if (!sortedLeases.remove(lease)) {
LOG.error(lease + " not found in sortedLeases"); LOG.error(lease + " not found in sortedLeases");
@ -186,31 +170,32 @@ public class LeaseManager {
/** /**
* Remove the lease for the specified holder and src * Remove the lease for the specified holder and src
*/ */
synchronized void removeLease(String holder, String src) { synchronized void removeLease(String holder, INodeFile src) {
Lease lease = getLease(holder); Lease lease = getLease(holder);
if (lease != null) { if (lease != null) {
removeLease(lease, src); removeLease(lease, src.getId());
} else { } else {
LOG.warn("Removing non-existent lease! holder=" + holder + LOG.warn("Removing non-existent lease! holder=" + holder +
" src=" + src); " src=" + src.getFullPathName());
} }
} }
synchronized void removeAllLeases() { synchronized void removeAllLeases() {
sortedLeases.clear(); sortedLeases.clear();
sortedLeasesByPath.clear(); leasesById.clear();
leases.clear(); leases.clear();
} }
/** /**
* Reassign lease for file src to the new holder. * Reassign lease for file src to the new holder.
*/ */
synchronized Lease reassignLease(Lease lease, String src, String newHolder) { synchronized Lease reassignLease(Lease lease, INodeFile src,
String newHolder) {
assert newHolder != null : "new lease holder is null"; assert newHolder != null : "new lease holder is null";
if (lease != null) { if (lease != null) {
removeLease(lease, src); removeLease(lease, src.getId());
} }
return addLease(newHolder, src); return addLease(newHolder, src.getId());
} }
/** /**
@ -243,10 +228,10 @@ public class LeaseManager {
* checks in. If the client dies and allows its lease to * checks in. If the client dies and allows its lease to
* expire, all the corresponding locks can be released. * expire, all the corresponding locks can be released.
*************************************************************/ *************************************************************/
class Lease implements Comparable<Lease> { class Lease {
private final String holder; private final String holder;
private long lastUpdate; private long lastUpdate;
private final Collection<String> paths = new TreeSet<String>(); private final HashSet<Long> files = new HashSet<>();
/** Only LeaseManager object can create a lease */ /** Only LeaseManager object can create a lease */
private Lease(String holder) { private Lease(String holder) {
@ -269,44 +254,16 @@ public class LeaseManager {
} }
/** Does this lease contain any path? */ /** Does this lease contain any path? */
boolean hasPath() {return !paths.isEmpty();} boolean hasFiles() {return !files.isEmpty();}
boolean removePath(String src) { boolean removeFile(long inodeId) {
return paths.remove(src); return files.remove(inodeId);
} }
@Override @Override
public String toString() { public String toString() {
return "[Lease. Holder: " + holder return "[Lease. Holder: " + holder
+ ", pendingcreates: " + paths.size() + "]"; + ", pending creates: " + files.size() + "]";
}
@Override
public int compareTo(Lease o) {
Lease l1 = this;
Lease l2 = o;
long lu1 = l1.lastUpdate;
long lu2 = l2.lastUpdate;
if (lu1 < lu2) {
return -1;
} else if (lu1 > lu2) {
return 1;
} else {
return l1.holder.compareTo(l2.holder);
}
}
@Override
public boolean equals(Object o) {
if (!(o instanceof Lease)) {
return false;
}
Lease obj = (Lease) o;
if (lastUpdate == obj.lastUpdate &&
holder.equals(obj.holder)) {
return true;
}
return false;
} }
@Override @Override
@ -314,82 +271,26 @@ public class LeaseManager {
return holder.hashCode(); return holder.hashCode();
} }
Collection<String> getPaths() { private Collection<Long> getFiles() { return files; }
return paths;
}
String getHolder() { String getHolder() {
return holder; return holder;
} }
void replacePath(String oldpath, String newpath) {
paths.remove(oldpath);
paths.add(newpath);
}
@VisibleForTesting @VisibleForTesting
long getLastUpdate() { long getLastUpdate() {
return lastUpdate; return lastUpdate;
} }
} }
synchronized void changeLease(String src, String dst) { @VisibleForTesting
if (LOG.isDebugEnabled()) { synchronized void removeLeases(Collection<Long> inodes) {
LOG.debug(getClass().getSimpleName() + ".changelease: " + for (long inode : inodes) {
" src=" + src + ", dest=" + dst); Lease lease = leasesById.get(inode);
} if (lease != null) {
removeLease(lease, inode);
final int len = src.length();
for(Map.Entry<String, Lease> entry
: findLeaseWithPrefixPath(src, sortedLeasesByPath).entrySet()) {
final String oldpath = entry.getKey();
final Lease lease = entry.getValue();
// replace stem of src with new destination
final String newpath = dst + oldpath.substring(len);
if (LOG.isDebugEnabled()) {
LOG.debug("changeLease: replacing " + oldpath + " with " + newpath);
}
lease.replacePath(oldpath, newpath);
sortedLeasesByPath.remove(oldpath);
sortedLeasesByPath.put(newpath, lease);
}
}
synchronized void removeLeaseWithPrefixPath(String prefix) {
for(Map.Entry<String, Lease> entry
: findLeaseWithPrefixPath(prefix, sortedLeasesByPath).entrySet()) {
if (LOG.isDebugEnabled()) {
LOG.debug(LeaseManager.class.getSimpleName()
+ ".removeLeaseWithPrefixPath: entry=" + entry);
}
removeLease(entry.getValue(), entry.getKey());
}
}
static private Map<String, Lease> findLeaseWithPrefixPath(
String prefix, SortedMap<String, Lease> path2lease) {
if (LOG.isDebugEnabled()) {
LOG.debug(LeaseManager.class.getSimpleName() + ".findLease: prefix=" + prefix);
}
final Map<String, Lease> entries = new HashMap<String, Lease>();
int srclen = prefix.length();
// prefix may ended with '/'
if (prefix.charAt(srclen - 1) == Path.SEPARATOR_CHAR) {
srclen -= 1;
}
for(Map.Entry<String, Lease> entry : path2lease.tailMap(prefix).entrySet()) {
final String p = entry.getKey();
if (!p.startsWith(prefix)) {
return entries;
}
if (p.length() == srclen || p.charAt(srclen) == Path.SEPARATOR_CHAR) {
entries.put(entry.getKey(), entry.getValue());
} }
} }
return entries;
} }
public void setLeasePeriod(long softLimit, long hardLimit) { public void setLeasePeriod(long softLimit, long hardLimit) {
@ -428,30 +329,13 @@ public class LeaseManager {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(name + " is interrupted", ie); LOG.debug(name + " is interrupted", ie);
} }
} catch(Throwable e) {
LOG.warn("Unexpected throwable: ", e);
} }
} }
} }
} }
/**
* Get the list of inodes corresponding to valid leases.
* @return list of inodes
*/
Map<String, INodeFile> getINodesUnderConstruction() {
Map<String, INodeFile> inodes = new TreeMap<String, INodeFile>();
for (String p : sortedLeasesByPath.keySet()) {
// verify that path exists in namespace
try {
INodeFile node = INodeFile.valueOf(fsnamesystem.dir.getINode(p), p);
Preconditions.checkState(node.isUnderConstruction());
inodes.put(p, node);
} catch (IOException ioe) {
LOG.error(ioe);
}
}
return inodes;
}
/** Check the leases beginning from the oldest. /** Check the leases beginning from the oldest.
* @return true is sync is needed. * @return true is sync is needed.
*/ */
@ -459,34 +343,35 @@ public class LeaseManager {
synchronized boolean checkLeases() { synchronized boolean checkLeases() {
boolean needSync = false; boolean needSync = false;
assert fsnamesystem.hasWriteLock(); assert fsnamesystem.hasWriteLock();
Lease leaseToCheck = null;
try {
leaseToCheck = sortedLeases.first();
} catch(NoSuchElementException e) {}
while(leaseToCheck != null) {
if (!leaseToCheck.expiredHardLimit()) {
break;
}
while(!sortedLeases.isEmpty() && sortedLeases.peek().expiredHardLimit()) {
Lease leaseToCheck = sortedLeases.poll();
LOG.info(leaseToCheck + " has expired hard limit"); LOG.info(leaseToCheck + " has expired hard limit");
final List<String> removing = new ArrayList<String>(); final List<Long> removing = new ArrayList<>();
// need to create a copy of the oldest lease paths, because // need to create a copy of the oldest lease files, because
// internalReleaseLease() removes paths corresponding to empty files, // internalReleaseLease() removes files corresponding to empty files,
// i.e. it needs to modify the collection being iterated over // i.e. it needs to modify the collection being iterated over
// causing ConcurrentModificationException // causing ConcurrentModificationException
String[] leasePaths = new String[leaseToCheck.getPaths().size()]; Collection<Long> files = leaseToCheck.getFiles();
leaseToCheck.getPaths().toArray(leasePaths); Long[] leaseINodeIds = files.toArray(new Long[files.size()]);
for(String p : leasePaths) { FSDirectory fsd = fsnamesystem.getFSDirectory();
String p = null;
for(Long id : leaseINodeIds) {
try { try {
INodesInPath iip = fsnamesystem.getFSDirectory().getINodesInPath(p, INodesInPath iip = INodesInPath.fromINode(fsd.getInode(id));
true); p = iip.getPath();
boolean completed = fsnamesystem.internalReleaseLease(leaseToCheck, p, // Sanity check to make sure the path is correct
iip, HdfsServerConstants.NAMENODE_LEASE_HOLDER); if (!p.startsWith("/")) {
throw new IOException("Invalid path in the lease " + p);
}
boolean completed = fsnamesystem.internalReleaseLease(
leaseToCheck, p, iip,
HdfsServerConstants.NAMENODE_LEASE_HOLDER);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
if (completed) { if (completed) {
LOG.debug("Lease recovery for " + p + " is complete. File closed."); LOG.debug("Lease recovery for inode " + id + " is complete. " +
"File closed.");
} else { } else {
LOG.debug("Started block recovery " + p + " lease " + leaseToCheck); LOG.debug("Started block recovery " + p + " lease " + leaseToCheck);
} }
@ -498,22 +383,15 @@ public class LeaseManager {
} catch (IOException e) { } catch (IOException e) {
LOG.error("Cannot release the path " + p + " in the lease " LOG.error("Cannot release the path " + p + " in the lease "
+ leaseToCheck, e); + leaseToCheck, e);
removing.add(p); removing.add(id);
} }
} }
for(String p : removing) { for(Long id : removing) {
removeLease(leaseToCheck, p); removeLease(leaseToCheck, id);
} }
leaseToCheck = sortedLeases.higher(leaseToCheck);
} }
try {
if(leaseToCheck != sortedLeases.first()) {
LOG.warn("Unable to release hard-limit expired lease: "
+ sortedLeases.first());
}
} catch(NoSuchElementException e) {}
return needSync; return needSync;
} }
@ -522,7 +400,7 @@ public class LeaseManager {
return getClass().getSimpleName() + "= {" return getClass().getSimpleName() + "= {"
+ "\n leases=" + leases + "\n leases=" + leases
+ "\n sortedLeases=" + sortedLeases + "\n sortedLeases=" + sortedLeases
+ "\n sortedLeasesByPath=" + sortedLeasesByPath + "\n leasesById=" + leasesById
+ "\n}"; + "\n}";
} }
@ -552,9 +430,15 @@ public class LeaseManager {
* its leases immediately. This is for use by unit tests. * its leases immediately. This is for use by unit tests.
*/ */
@VisibleForTesting @VisibleForTesting
void triggerMonitorCheckNow() { public void triggerMonitorCheckNow() {
Preconditions.checkState(lmthread != null, Preconditions.checkState(lmthread != null,
"Lease monitor is not running"); "Lease monitor is not running");
lmthread.interrupt(); lmthread.interrupt();
} }
@VisibleForTesting
public void runLeaseChecks() {
checkLeases();
}
} }

View File

@ -216,7 +216,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId()); int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId());
try { try {
QuotaCounts counts = snapshotRoot.cleanSubtree(bsps, snapshot.getId(), QuotaCounts counts = snapshotRoot.cleanSubtree(bsps, snapshot.getId(),
prior, collectedBlocks, removedINodes); prior, collectedBlocks, removedINodes, null);
INodeDirectory parent = snapshotRoot.getParent(); INodeDirectory parent = snapshotRoot.getParent();
if (parent != null) { if (parent != null) {
// there will not be any WithName node corresponding to the deleted // there will not be any WithName node corresponding to the deleted

View File

@ -97,15 +97,15 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
/** clear the created list */ /** clear the created list */
private QuotaCounts destroyCreatedList( private QuotaCounts destroyCreatedList(
final BlockStoragePolicySuite bsps, final BlockStoragePolicySuite bsps, final INodeDirectory currentINode,
final INodeDirectory currentINode,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
final List<INode> createdList = getList(ListType.CREATED); final List<INode> createdList = getList(ListType.CREATED);
for (INode c : createdList) { for (INode c : createdList) {
c.computeQuotaUsage(bsps, counts, true); c.computeQuotaUsage(bsps, counts, true);
c.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); c.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
// c should be contained in the children list, remove it // c should be contained in the children list, remove it
currentINode.removeChild(c); currentINode.removeChild(c);
} }
@ -117,12 +117,13 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
private QuotaCounts destroyDeletedList( private QuotaCounts destroyDeletedList(
final BlockStoragePolicySuite bsps, final BlockStoragePolicySuite bsps,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
final List<INode> deletedList = getList(ListType.DELETED); final List<INode> deletedList = getList(ListType.DELETED);
for (INode d : deletedList) { for (INode d : deletedList) {
d.computeQuotaUsage(bsps, counts, false); d.computeQuotaUsage(bsps, counts, false);
d.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); d.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
removedUCFiles);
} }
deletedList.clear(); deletedList.clear();
return counts; return counts;
@ -210,8 +211,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
@Override @Override
QuotaCounts combinePosteriorAndCollectBlocks( QuotaCounts combinePosteriorAndCollectBlocks(
final BlockStoragePolicySuite bsps, final BlockStoragePolicySuite bsps, final INodeDirectory currentDir,
final INodeDirectory currentDir, final DirectoryDiff posterior, final DirectoryDiff posterior,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes) {
final QuotaCounts counts = new QuotaCounts.Builder().build(); final QuotaCounts counts = new QuotaCounts.Builder().build();
@ -221,7 +222,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
public void process(INode inode) { public void process(INode inode) {
if (inode != null) { if (inode != null) {
inode.computeQuotaUsage(bsps, counts, false); inode.computeQuotaUsage(bsps, counts, false);
inode.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes); inode.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
null);
} }
} }
}); });
@ -324,7 +326,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) { BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
// this diff has been deleted // this diff has been deleted
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
counts.add(diff.destroyDeletedList(bsps, collectedBlocks, removedINodes)); counts.add(diff.destroyDeletedList(bsps, collectedBlocks, removedINodes,
null));
INodeDirectoryAttributes snapshotINode = getSnapshotINode(); INodeDirectoryAttributes snapshotINode = getSnapshotINode();
if (snapshotINode != null && snapshotINode.getAclFeature() != null) { if (snapshotINode != null && snapshotINode.getAclFeature() != null) {
AclStorage.removeAclFeature(snapshotINode.getAclFeature()); AclStorage.removeAclFeature(snapshotINode.getAclFeature());
@ -411,21 +414,23 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
public static void destroyDstSubtree( public static void destroyDstSubtree(
final BlockStoragePolicySuite bsps, INode inode, final int snapshot, final BlockStoragePolicySuite bsps, INode inode, final int snapshot,
final int prior, final BlocksMapUpdateInfo collectedBlocks, final int prior, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) throws QuotaExceededException { final List<INode> removedINodes, List<Long> removedUCFiles) throws QuotaExceededException {
Preconditions.checkArgument(prior != Snapshot.NO_SNAPSHOT_ID); Preconditions.checkArgument(prior != Snapshot.NO_SNAPSHOT_ID);
if (inode.isReference()) { if (inode.isReference()) {
if (inode instanceof INodeReference.WithName if (inode instanceof INodeReference.WithName
&& snapshot != Snapshot.CURRENT_STATE_ID) { && snapshot != Snapshot.CURRENT_STATE_ID) {
// this inode has been renamed before the deletion of the DstReference // this inode has been renamed before the deletion of the DstReference
// subtree // subtree
inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes); inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes,
removedUCFiles);
} else { } else {
// for DstReference node, continue this process to its subtree // for DstReference node, continue this process to its subtree
destroyDstSubtree(bsps, inode.asReference().getReferredINode(), snapshot, destroyDstSubtree(bsps, inode.asReference().getReferredINode(), snapshot,
prior, collectedBlocks, removedINodes); prior, collectedBlocks, removedINodes, removedUCFiles);
} }
} else if (inode.isFile()) { } else if (inode.isFile()) {
inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes); inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes,
removedUCFiles);
} else if (inode.isDirectory()) { } else if (inode.isDirectory()) {
Map<INode, INode> excludedNodes = null; Map<INode, INode> excludedNodes = null;
INodeDirectory dir = inode.asDirectory(); INodeDirectory dir = inode.asDirectory();
@ -445,7 +450,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
priorDiff = diffList.getDiffById(prior); priorDiff = diffList.getDiffById(prior);
if (priorDiff != null && priorDiff.getSnapshotId() == prior) { if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
priorDiff.diff.destroyCreatedList(bsps, dir, collectedBlocks, priorDiff.diff.destroyCreatedList(bsps, dir, collectedBlocks,
removedINodes); removedINodes, removedUCFiles);
} }
} }
for (INode child : inode.asDirectory().getChildrenList(prior)) { for (INode child : inode.asDirectory().getChildrenList(prior)) {
@ -453,7 +458,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
continue; continue;
} }
destroyDstSubtree(bsps, child, snapshot, prior, collectedBlocks, destroyDstSubtree(bsps, child, snapshot, prior, collectedBlocks,
removedINodes); removedINodes, removedUCFiles);
} }
} }
} }
@ -466,13 +471,13 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
* @param post The post snapshot. * @param post The post snapshot.
* @param prior The id of the prior snapshot. * @param prior The id of the prior snapshot.
* @param collectedBlocks Used to collect blocks for later deletion. * @param collectedBlocks Used to collect blocks for later deletion.
* @param removedUCFiles
* @return Quota usage update. * @return Quota usage update.
*/ */
private static QuotaCounts cleanDeletedINode( private static QuotaCounts cleanDeletedINode(
final BlockStoragePolicySuite bsps, INode inode, final BlockStoragePolicySuite bsps, INode inode, final int post, final int prior,
final int post, final int prior,
final BlocksMapUpdateInfo collectedBlocks, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
Deque<INode> queue = new ArrayDeque<INode>(); Deque<INode> queue = new ArrayDeque<INode>();
queue.addLast(inode); queue.addLast(inode);
@ -481,7 +486,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
if (topNode instanceof INodeReference.WithName) { if (topNode instanceof INodeReference.WithName) {
INodeReference.WithName wn = (INodeReference.WithName) topNode; INodeReference.WithName wn = (INodeReference.WithName) topNode;
if (wn.getLastSnapshotId() >= post) { if (wn.getLastSnapshotId() >= post) {
wn.cleanSubtree(bsps, post, prior, collectedBlocks, removedINodes); wn.cleanSubtree(bsps, post, prior, collectedBlocks, removedINodes,
removedUCFiles);
} }
// For DstReference node, since the node is not in the created list of // For DstReference node, since the node is not in the created list of
// prior, we should treat it as regular file/dir // prior, we should treat it as regular file/dir
@ -500,7 +506,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
if (priorDiff != null && priorDiff.getSnapshotId() == prior) { if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
priorChildrenDiff = priorDiff.getChildrenDiff(); priorChildrenDiff = priorDiff.getChildrenDiff();
counts.add(priorChildrenDiff.destroyCreatedList(bsps, dir, counts.add(priorChildrenDiff.destroyCreatedList(bsps, dir,
collectedBlocks, removedINodes)); collectedBlocks, removedINodes, removedUCFiles));
} }
} }
@ -631,7 +637,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
} }
public void clear(BlockStoragePolicySuite bsps, INodeDirectory currentINode, public void clear(BlockStoragePolicySuite bsps, INodeDirectory currentINode,
final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) { final BlocksMapUpdateInfo collectedBlocks, final List<INode>
removedINodes, final List<Long> removedUCFiles) {
// destroy its diff list // destroy its diff list
for (DirectoryDiff diff : diffs) { for (DirectoryDiff diff : diffs) {
diff.destroyDiffAndCollectBlocks(bsps, currentINode, collectedBlocks, diff.destroyDiffAndCollectBlocks(bsps, currentINode, collectedBlocks,
@ -721,10 +728,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
} }
} }
public QuotaCounts cleanDirectory(final BlockStoragePolicySuite bsps, final INodeDirectory currentINode, public QuotaCounts cleanDirectory(
final int snapshot, int prior, final BlockStoragePolicySuite bsps, final INodeDirectory currentINode,
final BlocksMapUpdateInfo collectedBlocks, final int snapshot, int prior, final BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) { final List<INode> removedINodes, List<Long> removedUCFiles) {
QuotaCounts counts = new QuotaCounts.Builder().build(); QuotaCounts counts = new QuotaCounts.Builder().build();
Map<INode, INode> priorCreated = null; Map<INode, INode> priorCreated = null;
Map<INode, INode> priorDeleted = null; Map<INode, INode> priorDeleted = null;
@ -734,10 +741,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
DirectoryDiff lastDiff = diffs.getLast(); DirectoryDiff lastDiff = diffs.getLast();
if (lastDiff != null) { if (lastDiff != null) {
counts.add(lastDiff.diff.destroyCreatedList(bsps, currentINode, counts.add(lastDiff.diff.destroyCreatedList(bsps, currentINode,
collectedBlocks, removedINodes)); collectedBlocks, removedINodes, removedUCFiles));
} }
counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior, counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior,
collectedBlocks, removedINodes, priorDeleted)); collectedBlocks, removedINodes, removedUCFiles, priorDeleted));
} else { } else {
// update prior // update prior
prior = getDiffs().updatePrior(snapshot, prior); prior = getDiffs().updatePrior(snapshot, prior);
@ -756,7 +763,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
counts.add(getDiffs().deleteSnapshotDiff(bsps, snapshot, prior, counts.add(getDiffs().deleteSnapshotDiff(bsps, snapshot, prior,
currentINode, collectedBlocks, removedINodes)); currentINode, collectedBlocks, removedINodes));
counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior, counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior,
collectedBlocks, removedINodes, priorDeleted)); collectedBlocks, removedINodes, removedUCFiles, priorDeleted));
// check priorDiff again since it may be created during the diff deletion // check priorDiff again since it may be created during the diff deletion
if (prior != Snapshot.NO_SNAPSHOT_ID) { if (prior != Snapshot.NO_SNAPSHOT_ID) {
@ -773,7 +780,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
ListType.CREATED)) { ListType.CREATED)) {
if (priorCreated.containsKey(cNode)) { if (priorCreated.containsKey(cNode)) {
counts.add(cNode.cleanSubtree(bsps, snapshot, Snapshot.NO_SNAPSHOT_ID, counts.add(cNode.cleanSubtree(bsps, snapshot, Snapshot.NO_SNAPSHOT_ID,
collectedBlocks, removedINodes)); collectedBlocks, removedINodes, removedUCFiles));
} }
} }
} }
@ -790,7 +797,7 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
ListType.DELETED)) { ListType.DELETED)) {
if (priorDeleted == null || !priorDeleted.containsKey(dNode)) { if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
counts.add(cleanDeletedINode(bsps, dNode, snapshot, prior, counts.add(cleanDeletedINode(bsps, dNode, snapshot, prior,
collectedBlocks, removedINodes)); collectedBlocks, removedINodes, removedUCFiles));
} }
} }
} }

View File

@ -213,7 +213,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
final BlocksMapUpdateInfo info, final List<INode> removedINodes) { final BlocksMapUpdateInfo info, final List<INode> removedINodes) {
// check if everything is deleted. // check if everything is deleted.
if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) { if (isCurrentFileDeleted() && getDiffs().asList().isEmpty()) {
file.destroyAndCollectBlocks(bsps, info, removedINodes); file.destroyAndCollectBlocks(bsps, info, removedINodes, null);
return; return;
} }
// find max file size. // find max file size.

View File

@ -59,8 +59,8 @@ import org.mockito.Mockito;
public class TestLease { public class TestLease {
static boolean hasLease(MiniDFSCluster cluster, Path src) { static boolean hasLease(MiniDFSCluster cluster, Path src) {
return NameNodeAdapter.getLeaseManager(cluster.getNamesystem() return NameNodeAdapter.getLeaseForPath(cluster.getNameNode(),
).getLeaseByPath(src.toString()) != null; src.toString()) != null;
} }
static int leaseCount(MiniDFSCluster cluster) { static int leaseCount(MiniDFSCluster cluster) {

View File

@ -135,8 +135,19 @@ public class NameNodeAdapter {
namesystem.leaseManager.triggerMonitorCheckNow(); namesystem.leaseManager.triggerMonitorCheckNow();
} }
public static Lease getLeaseForPath(NameNode nn, String path) {
final FSNamesystem fsn = nn.getNamesystem();
INode inode;
try {
inode = fsn.getFSDirectory().getINode(path, false);
} catch (UnresolvedLinkException e) {
throw new RuntimeException("Lease manager should not support symlinks");
}
return inode == null ? null : fsn.leaseManager.getLease((INodeFile) inode);
}
public static String getLeaseHolderForPath(NameNode namenode, String path) { public static String getLeaseHolderForPath(NameNode namenode, String path) {
Lease l = namenode.getNamesystem().leaseManager.getLeaseByPath(path); Lease l = getLeaseForPath(namenode, path);
return l == null? null: l.getHolder(); return l == null? null: l.getHolder();
} }
@ -145,12 +156,8 @@ public class NameNodeAdapter {
* or -1 in the case that the lease doesn't exist. * or -1 in the case that the lease doesn't exist.
*/ */
public static long getLeaseRenewalTime(NameNode nn, String path) { public static long getLeaseRenewalTime(NameNode nn, String path) {
LeaseManager lm = nn.getNamesystem().leaseManager; Lease l = getLeaseForPath(nn, path);
Lease l = lm.getLeaseByPath(path); return l == null ? -1 : l.getLastUpdate();
if (l == null) {
return -1;
}
return l.getLastUpdate();
} }
/** /**

View File

@ -213,11 +213,12 @@ public class TestDiskspaceQuotaUpdate {
// ignore // ignore
} }
LeaseManager lm = cluster.getNamesystem().getLeaseManager();
// check that the file exists, isn't UC, and has no dangling lease // check that the file exists, isn't UC, and has no dangling lease
INodeFile inode = fsdir.getINode(file.toString()).asFile(); INodeFile inode = fsdir.getINode(file.toString()).asFile();
Assert.assertNotNull(inode); Assert.assertNotNull(inode);
Assert.assertFalse("should not be UC", inode.isUnderConstruction()); Assert.assertFalse("should not be UC", inode.isUnderConstruction());
Assert.assertNull("should not have a lease", cluster.getNamesystem().getLeaseManager().getLeaseByPath(file.toString())); Assert.assertNull("should not have a lease", lm.getLease(inode));
// make sure the quota usage is unchanged // make sure the quota usage is unchanged
final long newSpaceUsed = dirNode.getDirectoryWithQuotaFeature() final long newSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
.getSpaceConsumed().getStorageSpace(); .getSpaceConsumed().getStorageSpace();
@ -256,11 +257,11 @@ public class TestDiskspaceQuotaUpdate {
} }
// check that the file exists, isn't UC, and has no dangling lease // check that the file exists, isn't UC, and has no dangling lease
LeaseManager lm = cluster.getNamesystem().getLeaseManager();
INodeFile inode = fsdir.getINode(file.toString()).asFile(); INodeFile inode = fsdir.getINode(file.toString()).asFile();
Assert.assertNotNull(inode); Assert.assertNotNull(inode);
Assert.assertFalse("should not be UC", inode.isUnderConstruction()); Assert.assertFalse("should not be UC", inode.isUnderConstruction());
Assert.assertNull("should not have a lease", cluster.getNamesystem() Assert.assertNull("should not have a lease", lm.getLease(inode));
.getLeaseManager().getLeaseByPath(file.toString()));
// make sure the quota usage is unchanged // make sure the quota usage is unchanged
final long newSpaceUsed = dirNode.getDirectoryWithQuotaFeature() final long newSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
.getSpaceConsumed().getStorageSpace(); .getSpaceConsumed().getStorageSpace();
@ -296,11 +297,11 @@ public class TestDiskspaceQuotaUpdate {
} }
// check that the file exists, isn't UC, and has no dangling lease // check that the file exists, isn't UC, and has no dangling lease
LeaseManager lm = cluster.getNamesystem().getLeaseManager();
INodeFile inode = fsdir.getINode(file.toString()).asFile(); INodeFile inode = fsdir.getINode(file.toString()).asFile();
Assert.assertNotNull(inode); Assert.assertNotNull(inode);
Assert.assertFalse("should not be UC", inode.isUnderConstruction()); Assert.assertFalse("should not be UC", inode.isUnderConstruction());
Assert.assertNull("should not have a lease", cluster.getNamesystem() Assert.assertNull("should not have a lease", lm.getLease(inode));
.getLeaseManager().getLeaseByPath(file.toString()));
// make sure the quota usage is unchanged // make sure the quota usage is unchanged
final long newSpaceUsed = dirNode.getDirectoryWithQuotaFeature() final long newSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
.getSpaceConsumed().getStorageSpace(); .getSpaceConsumed().getStorageSpace();

View File

@ -109,7 +109,7 @@ public class TestFSImage {
assertEquals(1, blks.length); assertEquals(1, blks.length);
assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState()); assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState());
// check lease manager // check lease manager
Lease lease = fsn.leaseManager.getLeaseByPath(file2.toString()); Lease lease = fsn.leaseManager.getLease(file2Node);
Assert.assertNotNull(lease); Assert.assertNotNull(lease);
} finally { } finally {
if (cluster != null) { if (cluster != null) {

View File

@ -76,7 +76,7 @@ public class TestFSNamesystem {
DFSTestUtil.formatNameNode(conf); DFSTestUtil.formatNameNode(conf);
FSNamesystem fsn = FSNamesystem.loadFromDisk(conf); FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
LeaseManager leaseMan = fsn.getLeaseManager(); LeaseManager leaseMan = fsn.getLeaseManager();
leaseMan.addLease("client1", "importantFile"); leaseMan.addLease("client1", fsn.getFSDirectory().allocateNewInodeId());
assertEquals(1, leaseMan.countLease()); assertEquals(1, leaseMan.countLease());
fsn.clear(); fsn.clear();
leaseMan = fsn.getLeaseManager(); leaseMan = fsn.getLeaseManager();

View File

@ -70,7 +70,8 @@ public class TestGetBlockLocations {
public Void answer(InvocationOnMock invocation) throws Throwable { public Void answer(InvocationOnMock invocation) throws Throwable {
INodesInPath iip = fsd.getINodesInPath(FILE_PATH, true); INodesInPath iip = fsd.getINodesInPath(FILE_PATH, true);
FSDirDeleteOp.delete(fsd, iip, new INode.BlocksMapUpdateInfo(), FSDirDeleteOp.delete(fsd, iip, new INode.BlocksMapUpdateInfo(),
new ArrayList<INode>(), now()); new ArrayList<INode>(), new ArrayList<Long>(),
now());
invocation.callRealMethod(); invocation.callRealMethod();
return null; return null;
} }

View File

@ -21,39 +21,31 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import org.apache.hadoop.conf.Configuration; import com.google.common.collect.Lists;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
import java.util.ArrayList;
import static org.mockito.Mockito.*;
public class TestLeaseManager { public class TestLeaseManager {
final Configuration conf = new HdfsConfiguration();
@Test @Test
public void testRemoveLeaseWithPrefixPath() throws Exception { public void testRemoveLeases() throws Exception {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); FSNamesystem fsn = mock(FSNamesystem.class);
cluster.waitActive(); LeaseManager lm = new LeaseManager(fsn);
ArrayList<Long> ids = Lists.newArrayList(INodeId.ROOT_INODE_ID + 1,
INodeId.ROOT_INODE_ID + 2, INodeId.ROOT_INODE_ID + 3,
INodeId.ROOT_INODE_ID + 4);
for (long id : ids) {
lm.addLease("foo", id);
}
LeaseManager lm = NameNodeAdapter.getLeaseManager(cluster.getNamesystem()); assertEquals(4, lm.getINodeIdWithLeases().size());
lm.addLease("holder1", "/a/b"); synchronized (lm) {
lm.addLease("holder2", "/a/c"); lm.removeLeases(ids);
assertNotNull(lm.getLeaseByPath("/a/b")); }
assertNotNull(lm.getLeaseByPath("/a/c")); assertEquals(0, lm.getINodeIdWithLeases().size());
lm.removeLeaseWithPrefixPath("/a");
assertNull(lm.getLeaseByPath("/a/b"));
assertNull(lm.getLeaseByPath("/a/c"));
lm.addLease("holder1", "/a/b");
lm.addLease("holder2", "/a/c");
lm.removeLeaseWithPrefixPath("/a/");
assertNull(lm.getLeaseByPath("/a/b"));
assertNull(lm.getLeaseByPath("/a/c"));
} }
/** Check that even if LeaseManager.checkLease is not able to relinquish /** Check that even if LeaseManager.checkLease is not able to relinquish
@ -70,13 +62,13 @@ public class TestLeaseManager {
LeaseManager lm = new LeaseManager(fsn); LeaseManager lm = new LeaseManager(fsn);
//Make sure the leases we are going to add exceed the hard limit //Make sure the leases we are going to add exceed the hard limit
lm.setLeasePeriod(0,0); lm.setLeasePeriod(0, 0);
//Add some leases to the LeaseManager //Add some leases to the LeaseManager
lm.addLease("holder1", "src1"); lm.addLease("holder1", INodeId.ROOT_INODE_ID + 1);
lm.addLease("holder2", "src2"); lm.addLease("holder2", INodeId.ROOT_INODE_ID + 2);
lm.addLease("holder3", "src3"); lm.addLease("holder3", INodeId.ROOT_INODE_ID + 3);
assertEquals(lm.getNumSortedLeases(), 3); assertEquals(lm.countLease(), 3);
//Initiate a call to checkLease. This should exit within the test timeout //Initiate a call to checkLease. This should exit within the test timeout
lm.checkLeases(); lm.checkLeases();

View File

@ -615,7 +615,8 @@ public class TestSaveNamespace {
cluster.waitActive(); cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem(); DistributedFileSystem fs = cluster.getFileSystem();
try { try {
cluster.getNamesystem().leaseManager.addLease("me", "/non-existent"); cluster.getNamesystem().leaseManager.addLease("me",
INodeId.ROOT_INODE_ID + 1);
fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER); fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
cluster.getNameNodeRpc().saveNamespace(0, 0); cluster.getNameNodeRpc().saveNamespace(0, 0);
fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE); fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);

View File

@ -40,11 +40,13 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory; import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory; import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff; import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.junit.After; import org.junit.After;
@ -279,4 +281,31 @@ public class TestINodeFileUnderConstructionWithSnapshot {
assertEquals(BLOCKSIZE - 1, lastBlock.getBlockSize()); assertEquals(BLOCKSIZE - 1, lastBlock.getBlockSize());
out.close(); out.close();
} }
@Test
public void testLease() throws Exception {
try {
NameNodeAdapter.setLeasePeriod(fsn, 100, 200);
final Path foo = new Path(dir, "foo");
final Path bar = new Path(foo, "bar");
DFSTestUtil.createFile(hdfs, bar, BLOCKSIZE, REPLICATION, 0);
HdfsDataOutputStream out = appendFileWithoutClosing(bar, 100);
out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
hdfs.delete(foo, true);
Thread.sleep(1000);
try {
fsn.writeLock();
NameNodeAdapter.getLeaseManager(fsn).runLeaseChecks();
} finally {
fsn.writeUnlock();
}
} finally {
NameNodeAdapter.setLeasePeriod(
fsn,
HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD,
HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
}
}
} }