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 @@ static long delete(
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 @@ static void deleteForEditLog(FSDirectory fsd, String src, long mtime)
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 @@ static void deleteForEditLog(FSDirectory fsd, String src, long mtime)
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 @@ static BlocksMapUpdateInfo deleteInternal(
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 @@ private static boolean deleteAllowed(final INodesInPath iip,
* @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 @@ private static long unprotectedDelete(
// 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 @@ void updateMtimeAndLease(long timestamp) throws QuotaExceededException {
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 @@ boolean cleanDst(BlockStoragePolicySuite bsps, BlocksMapUpdateInfo collectedBloc
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.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 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
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 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
"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 @@ fsDir, renameReservedPathsOnUpgrade(timesOp.path, logVersion),
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:
@ -739,7 +740,7 @@ fsDir, renameReservedPathsOnUpgrade(timesOp.path, logVersion),
collectedBlocks.clear(); collectedBlocks.clear();
fsNamesys.dir.removeFromInodeMap(removedINodes); fsNamesys.dir.removeFromInodeMap(removedINodes);
removedINodes.clear(); removedINodes.clear();
if (toAddRetryCache) { if (toAddRetryCache) {
fsNamesys.addCacheEntry(deleteSnapshotOp.rpcClientId, fsNamesys.addCacheEntry(deleteSnapshotOp.rpcClientId,
deleteSnapshotOp.rpcCallId); deleteSnapshotOp.rpcCallId);

View File

@ -966,8 +966,7 @@ LayoutVersion.Feature.ADD_INODE_ID, getLayoutVersion())) {
} }
if (!inSnapshot) { if (!inSnapshot) {
namesystem.leaseManager.addLease(cons namesystem.leaseManager.addLease(uc.getClientName(), oldnode.getId());
.getFileUnderConstructionFeature().getClientName(), path);
} }
} }
} }
@ -1297,7 +1296,7 @@ void save(File newFile, FSImageCompression compression) throws IOException {
// 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 @@ private void saveINode2Image(INode inode, DataOutputStream out,
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.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 @@ void loadFilesUnderConstructionSection(InputStream in) throws IOException {
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 @@ void serializeINodeSection(OutputStream out) throws IOException {
} }
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 @@ Block prepareFileForTruncate(INodesInPath iip,
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 @@ private BlocksMapUpdateInfo startFileInternal(FSPermissionChecker pc,
} 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 @@ private BlocksMapUpdateInfo startFileInternal(FSPermissionChecker pc,
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 @@ LocatedBlock prepareFileForAppend(String src, INodesInPath iip,
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 @@ void recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip,
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 @@ void removeBlocks(BlocksMapUpdateInfo blocks) {
/** /**
* 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 @@ private Lease reassignLease(Lease lease, String src, String newHolder,
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 @@ private void finalizeINodeFileUnderConstruction(String src,
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 @@ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
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

@ -390,7 +390,7 @@ public INodeSymlink asSymlink() {
* @param bsps * @param bsps
* block storage policy suite to calculate intended storage type usage * block storage policy suite to calculate intended storage type usage
* @param snapshotId * @param snapshotId
* The id of the snapshot to delete. * The id of the snapshot to delete.
* {@link Snapshot#CURRENT_STATE_ID} means to delete the current * {@link Snapshot#CURRENT_STATE_ID} means to delete the current
* file/directory. * file/directory.
* @param priorSnapshotId * @param priorSnapshotId
@ -401,14 +401,16 @@ public INodeSymlink asSymlink() {
* blocks collected from the descents for further block * blocks collected from the descents for further block
* deletion/update will be added to the given map. * deletion/update will be added to the given map.
* @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 QuotaCounts cleanSubtree(final BlockStoragePolicySuite bsps,
* 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 QuotaCounts cleanSubtree(final BlockStoragePolicySuite bsps,
* @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 void clear() {
} }
/** 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 QuotaCounts cleanSubtreeRecursively(final BlockStoragePolicySuite bsps,
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 QuotaCounts cleanSubtreeRecursively(final BlockStoragePolicySuite bsps,
} }
@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 void destroyAndCollectBlocks(final BlockStoragePolicySuite bsps,
} }
@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 QuotaCounts cleanSubtree(final BlockStoragePolicySuite bsps,
// 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 void setBlocks(BlockInfoContiguous[] blocks) {
} }
@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 QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, final int snapshot
} }
@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 void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
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 @@ void recordModification(int latestSnapshotId) {
} }
@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 ContentSummaryComputationContext computeContentSummary(
} }
@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 @@ final void recordModification(int latestSnapshotId) {
} }
@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 final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
} }
@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 QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps,
} }
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 QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps,
} }
@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 void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
} }
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 DstReference(INodeDirectory parent, WithCount referred,
} }
@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 QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, int snapshot, int
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 QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, int snapshot, int
* 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 void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
// 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 void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
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 byte[] getSymlink() {
} }
@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 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 @@ Lease getLease(String holder) {
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 @@ synchronized Lease addLease(String holder, String src) {
} 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 @@ synchronized void removeLease(Lease lease, String src) {
/** /**
* 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 @@ synchronized void renewAllLeases() {
* 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,127 +254,43 @@ public boolean expiredSoftLimit() {
} }
/** 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
public int hashCode() { public int hashCode() {
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 void run() {
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 @@ Map<String, INodeFile> getINodesUnderConstruction() {
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 @@ synchronized boolean checkLeases() {
} 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 synchronized String toString() {
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 @@ void stopMonitor() {
* 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

@ -65,7 +65,7 @@ public void clear() {
* @param snapshot The id of the snapshot to be deleted * @param snapshot The id of the snapshot to be deleted
* @param prior The id of the snapshot taken before the to-be-deleted snapshot * @param prior The id of the snapshot taken before the to-be-deleted snapshot
* @param collectedBlocks Used to collect information for blocksMap update * @param collectedBlocks Used to collect information for blocksMap update
* @return delta in namespace. * @return delta in namespace.
*/ */
public final QuotaCounts deleteSnapshotDiff(BlockStoragePolicySuite bsps, public final QuotaCounts deleteSnapshotDiff(BlockStoragePolicySuite bsps,
final int snapshot, final int snapshot,

View File

@ -216,7 +216,7 @@ public Snapshot removeSnapshot(BlockStoragePolicySuite bsps, INodeDirectory snap
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 @@ private boolean removeChild(ListType type, final INode child) {
/** 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 @@ private QuotaCounts destroyCreatedList(
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 @@ boolean isSnapshotRoot() {
@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 @@ QuotaCounts combinePosteriorAndCollectBlocks(
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 @@ QuotaCounts destroyDiffAndCollectBlocks(
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 @@ private static Map<INode, INode> cloneDiffList(List<INode> diffList) {
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 static void destroyDstSubtree(
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 static void destroyDstSubtree(
continue; continue;
} }
destroyDstSubtree(bsps, child, snapshot, prior, collectedBlocks, destroyDstSubtree(bsps, child, snapshot, prior, collectedBlocks,
removedINodes); removedINodes, removedUCFiles);
} }
} }
} }
@ -466,13 +471,13 @@ public static void destroyDstSubtree(
* @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 @@ private static QuotaCounts cleanDeletedINode(
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 @@ private static QuotaCounts cleanDeletedINode(
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 INode saveChild2Snapshot(INodeDirectory currentINode,
} }
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 @@ boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
} }
} }
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 QuotaCounts cleanDirectory(final BlockStoragePolicySuite bsps, final INod
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 QuotaCounts cleanDirectory(final BlockStoragePolicySuite bsps, final INod
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 QuotaCounts cleanDirectory(final BlockStoragePolicySuite bsps, final INod
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 QuotaCounts cleanDirectory(final BlockStoragePolicySuite bsps, final INod
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 void collectBlocksAndClear(final BlockStoragePolicySuite bsps, final INod
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

@ -228,7 +228,7 @@ public String createSnapshot(final INodesInPath iip, String snapshotRoot,
/** /**
* Delete a snapshot for a snapshottable directory * Delete a snapshot for a snapshottable directory
* @param snapshotName Name of the snapshot to be deleted * @param snapshotName Name of the snapshot to be deleted
* @param collectedBlocks Used to collect information to update blocksMap * @param collectedBlocks Used to collect information to update blocksMap
* @throws IOException * @throws IOException
*/ */
public void deleteSnapshot(final INodesInPath iip, final String snapshotName, public void deleteSnapshot(final INodesInPath iip, final String snapshotName,
@ -266,7 +266,7 @@ public int getNumSnapshottableDirs() {
public int getNumSnapshots() { public int getNumSnapshots() {
return numSnapshots.get(); return numSnapshots.get();
} }
void setNumSnapshots(int num) { void setNumSnapshots(int num) {
numSnapshots.set(num); numSnapshots.set(num);
} }

View File

@ -59,8 +59,8 @@
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 static void setLeasePeriod(final FSNamesystem namesystem, long soft, long
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 static String getLeaseHolderForPath(NameNode namenode, String path) {
* 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 void testAppendOverStorageQuota() throws Exception {
// 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 void testAppendOverTypeQuota() throws Exception {
} }
// 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 void testTruncateOverQuota() throws Exception {
} }
// 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 @@ private void testPersistHelper(Configuration conf) throws IOException {
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 void testFSNamespaceClearLeases() throws Exception {
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 void testGetBlockLocationsRacingWithDelete() throws IOException {
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.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 void testCheckLeaseNotInfiniteLoop() {
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 void testSaveNamespaceWithDanglingLease() throws Exception {
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.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 void testGetBlockLocations() throws Exception {
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);
}
}
} }