|
@@ -154,7 +154,6 @@ import org.apache.hadoop.fs.InvalidPathException;
|
|
import org.apache.hadoop.fs.Options;
|
|
import org.apache.hadoop.fs.Options;
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
-import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
|
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
import org.apache.hadoop.fs.XAttr;
|
|
import org.apache.hadoop.fs.XAttr;
|
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
|
@@ -2284,11 +2283,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
if (overwrite) {
|
|
if (overwrite) {
|
|
toRemoveBlocks = new BlocksMapUpdateInfo();
|
|
toRemoveBlocks = new BlocksMapUpdateInfo();
|
|
List<INode> toRemoveINodes = new ChunkedArrayList<INode>();
|
|
List<INode> toRemoveINodes = new ChunkedArrayList<INode>();
|
|
- long ret = dir.delete(iip, toRemoveBlocks, toRemoveINodes, now());
|
|
|
|
|
|
+ long ret = FSDirDeleteOp.delete(dir, iip, toRemoveBlocks,
|
|
|
|
+ toRemoveINodes, now());
|
|
if (ret >= 0) {
|
|
if (ret >= 0) {
|
|
iip = INodesInPath.replace(iip, iip.length() - 1, null);
|
|
iip = INodesInPath.replace(iip, iip.length() - 1, null);
|
|
- incrDeletedFileCount(ret);
|
|
|
|
- removePathAndBlocks(src, null, toRemoveINodes, true);
|
|
|
|
|
|
+ FSDirDeleteOp.incrDeletedFileCount(ret);
|
|
|
|
+ removeLeasesAndINodes(src, toRemoveINodes, true);
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
// If lease soft limit time is expired, recover the lease
|
|
// If lease soft limit time is expired, recover the lease
|
|
@@ -3346,99 +3346,35 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
* description of exceptions
|
|
* description of exceptions
|
|
*/
|
|
*/
|
|
boolean delete(String src, boolean recursive, boolean logRetryCache)
|
|
boolean delete(String src, boolean recursive, boolean logRetryCache)
|
|
- throws AccessControlException, SafeModeException,
|
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
|
-
|
|
|
|
|
|
+ throws IOException {
|
|
|
|
+ waitForLoadingFSImage();
|
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
|
+ BlocksMapUpdateInfo toRemovedBlocks = null;
|
|
|
|
+ writeLock();
|
|
boolean ret = false;
|
|
boolean ret = false;
|
|
try {
|
|
try {
|
|
- ret = deleteInt(src, recursive, logRetryCache);
|
|
|
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
|
+ checkNameNodeSafeMode("Cannot delete " + src);
|
|
|
|
+ toRemovedBlocks = FSDirDeleteOp.delete(
|
|
|
|
+ this, src, recursive, logRetryCache);
|
|
|
|
+ ret = toRemovedBlocks != null;
|
|
} catch (AccessControlException e) {
|
|
} catch (AccessControlException e) {
|
|
logAuditEvent(false, "delete", src);
|
|
logAuditEvent(false, "delete", src);
|
|
throw e;
|
|
throw e;
|
|
|
|
+ } finally {
|
|
|
|
+ writeUnlock();
|
|
}
|
|
}
|
|
- return ret;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private boolean deleteInt(String src, boolean recursive, boolean logRetryCache)
|
|
|
|
- throws AccessControlException, SafeModeException,
|
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + src);
|
|
|
|
- }
|
|
|
|
- boolean status = deleteInternal(src, recursive, true, logRetryCache);
|
|
|
|
- if (status) {
|
|
|
|
- logAuditEvent(true, "delete", src);
|
|
|
|
|
|
+ if (toRemovedBlocks != null) {
|
|
|
|
+ removeBlocks(toRemovedBlocks); // Incremental deletion of blocks
|
|
}
|
|
}
|
|
- return status;
|
|
|
|
|
|
+ logAuditEvent(true, "delete", src);
|
|
|
|
+ return ret;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
FSPermissionChecker getPermissionChecker()
|
|
FSPermissionChecker getPermissionChecker()
|
|
throws AccessControlException {
|
|
throws AccessControlException {
|
|
return dir.getPermissionChecker();
|
|
return dir.getPermissionChecker();
|
|
}
|
|
}
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Remove a file/directory from the namespace.
|
|
|
|
- * <p>
|
|
|
|
- * For large directories, deletion is incremental. The blocks under
|
|
|
|
- * the directory are collected and deleted a small number at a time holding
|
|
|
|
- * the {@link FSNamesystem} lock.
|
|
|
|
- * <p>
|
|
|
|
- * For small directory or file the deletion is done in one shot.
|
|
|
|
- *
|
|
|
|
- * @see ClientProtocol#delete(String, boolean) for description of exceptions
|
|
|
|
- */
|
|
|
|
- private boolean deleteInternal(String src, boolean recursive,
|
|
|
|
- boolean enforcePermission, boolean logRetryCache)
|
|
|
|
- throws AccessControlException, SafeModeException, UnresolvedLinkException,
|
|
|
|
- IOException {
|
|
|
|
- BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
|
|
- List<INode> removedINodes = new ChunkedArrayList<INode>();
|
|
|
|
- FSPermissionChecker pc = getPermissionChecker();
|
|
|
|
- checkOperation(OperationCategory.WRITE);
|
|
|
|
- byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
|
- boolean ret = false;
|
|
|
|
-
|
|
|
|
- waitForLoadingFSImage();
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- checkOperation(OperationCategory.WRITE);
|
|
|
|
- checkNameNodeSafeMode("Cannot delete " + src);
|
|
|
|
- src = dir.resolvePath(pc, src, pathComponents);
|
|
|
|
- final INodesInPath iip = dir.getINodesInPath4Write(src, false);
|
|
|
|
- if (!recursive && dir.isNonEmptyDirectory(iip)) {
|
|
|
|
- throw new PathIsNotEmptyDirectoryException(src + " is non empty");
|
|
|
|
- }
|
|
|
|
- if (enforcePermission && isPermissionEnabled) {
|
|
|
|
- dir.checkPermission(pc, iip, false, null, FsAction.WRITE, null,
|
|
|
|
- FsAction.ALL, true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- long mtime = now();
|
|
|
|
- // Unlink the target directory from directory tree
|
|
|
|
- long filesRemoved = dir.delete(iip, collectedBlocks, removedINodes,
|
|
|
|
- mtime);
|
|
|
|
- if (filesRemoved < 0) {
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
- getEditLog().logDelete(src, mtime, logRetryCache);
|
|
|
|
- incrDeletedFileCount(filesRemoved);
|
|
|
|
- // Blocks/INodes will be handled later
|
|
|
|
- removePathAndBlocks(src, null, removedINodes, true);
|
|
|
|
- ret = true;
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- getEditLog().logSync();
|
|
|
|
- removeBlocks(collectedBlocks); // Incremental deletion of blocks
|
|
|
|
- collectedBlocks.clear();
|
|
|
|
-
|
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
- NameNode.stateChangeLog.debug("DIR* Namesystem.delete: "
|
|
|
|
- + src +" is removed");
|
|
|
|
- }
|
|
|
|
- return ret;
|
|
|
|
- }
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* From the given list, incrementally remove the blocks from blockManager
|
|
* From the given list, incrementally remove the blocks from blockManager
|
|
@@ -3465,15 +3401,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Remove leases, inodes and blocks related to a given path
|
|
|
|
|
|
+ * Remove leases and inodes related to a given path
|
|
* @param src The given path
|
|
* @param src The given path
|
|
- * @param blocks Containing the list of blocks to be deleted from blocksMap
|
|
|
|
- * @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 removePathAndBlocks(String src, BlocksMapUpdateInfo blocks,
|
|
|
|
- List<INode> removedINodes, final boolean acquireINodeMapLock) {
|
|
|
|
|
|
+ void removeLeasesAndINodes(String src, List<INode> removedINodes,
|
|
|
|
+ final boolean acquireINodeMapLock) {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
leaseManager.removeLeaseWithPrefixPath(src);
|
|
leaseManager.removeLeaseWithPrefixPath(src);
|
|
// remove inodes from inodesMap
|
|
// remove inodes from inodesMap
|
|
@@ -3490,11 +3425,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
}
|
|
}
|
|
removedINodes.clear();
|
|
removedINodes.clear();
|
|
}
|
|
}
|
|
- if (blocks == null) {
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- removeBlocksAndUpdateSafemodeTotal(blocks);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -4278,10 +4208,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- void incrDeletedFileCount(long count) {
|
|
|
|
- NameNode.getNameNodeMetrics().incrFilesDeleted(count);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Close file.
|
|
* Close file.
|
|
* @param path
|
|
* @param path
|
|
@@ -4416,7 +4342,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
for (BlockCollection bc : filesToDelete) {
|
|
for (BlockCollection bc : filesToDelete) {
|
|
LOG.warn("Removing lazyPersist file " + bc.getName() + " with no replicas.");
|
|
LOG.warn("Removing lazyPersist file " + bc.getName() + " with no replicas.");
|
|
- deleteInternal(bc.getName(), false, false, false);
|
|
|
|
|
|
+ BlocksMapUpdateInfo toRemoveBlocks =
|
|
|
|
+ FSDirDeleteOp.deleteInternal(
|
|
|
|
+ FSNamesystem.this, bc.getName(),
|
|
|
|
+ INodesInPath.fromINode((INodeFile) bc), false);
|
|
|
|
+ if (toRemoveBlocks != null) {
|
|
|
|
+ removeBlocks(toRemoveBlocks); // Incremental deletion of blocks
|
|
|
|
+ }
|
|
}
|
|
}
|
|
} finally {
|
|
} finally {
|
|
writeUnlock();
|
|
writeUnlock();
|