|
@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.Options;
|
|
|
import org.apache.hadoop.fs.Options.Rename;
|
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.PathIsNotDirectoryException;
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
@@ -83,7 +84,6 @@ public class FSDirectory implements Closeable {
|
|
|
FSImage fsImage;
|
|
|
private final FSNamesystem namesystem;
|
|
|
private volatile boolean ready = false;
|
|
|
- private static final long UNKNOWN_DISK_SPACE = -1;
|
|
|
private final int maxComponentLength;
|
|
|
private final int maxDirItems;
|
|
|
private final int lsLimit; // max list limit
|
|
@@ -256,13 +256,14 @@ public class FSDirectory implements Closeable {
|
|
|
permissions,replication,
|
|
|
preferredBlockSize, modTime, clientName,
|
|
|
clientMachine, clientNode);
|
|
|
+ boolean added = false;
|
|
|
writeLock();
|
|
|
try {
|
|
|
- newNode = addNode(path, newNode, UNKNOWN_DISK_SPACE);
|
|
|
+ added = addINode(path, newNode);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
- if (newNode == null) {
|
|
|
+ if (!added) {
|
|
|
NameNode.stateChangeLog.info("DIR* addFile: failed to add " + path);
|
|
|
return null;
|
|
|
}
|
|
@@ -282,7 +283,7 @@ public class FSDirectory implements Closeable {
|
|
|
boolean underConstruction,
|
|
|
String clientName,
|
|
|
String clientMachine) {
|
|
|
- INode newNode;
|
|
|
+ final INode newNode;
|
|
|
assert hasWriteLock();
|
|
|
if (underConstruction) {
|
|
|
newNode = new INodeFileUnderConstruction(
|
|
@@ -295,16 +296,17 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- newNode = addNode(path, newNode, UNKNOWN_DISK_SPACE);
|
|
|
+ if (addINode(path, newNode)) {
|
|
|
+ return newNode;
|
|
|
+ }
|
|
|
} catch (IOException e) {
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug(
|
|
|
"DIR* FSDirectory.unprotectedAddFile: exception when add " + path
|
|
|
+ " to the file system", e);
|
|
|
}
|
|
|
- return null;
|
|
|
}
|
|
|
- return newNode;
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -543,12 +545,12 @@ public class FSDirectory implements Closeable {
|
|
|
// Ensure dst has quota to accommodate rename
|
|
|
verifyQuotaForRename(srcInodes, dstInodes);
|
|
|
|
|
|
- INode dstChild = null;
|
|
|
+ boolean added = false;
|
|
|
INode srcChild = null;
|
|
|
String srcChildName = null;
|
|
|
try {
|
|
|
// remove src
|
|
|
- srcChild = removeChild(srcInodesInPath, srcInodes.length-1);
|
|
|
+ srcChild = removeLastINode(srcInodesInPath);
|
|
|
if (srcChild == null) {
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
+ "failed to rename " + src + " to " + dst
|
|
@@ -559,9 +561,8 @@ public class FSDirectory implements Closeable {
|
|
|
srcChild.setLocalName(dstComponents[dstInodes.length-1]);
|
|
|
|
|
|
// add src to the destination
|
|
|
- dstChild = addChildNoQuotaCheck(dstInodesInPath, dstInodes.length-1,
|
|
|
- srcChild, UNKNOWN_DISK_SPACE);
|
|
|
- if (dstChild != null) {
|
|
|
+ added = addLastINodeNoQuotaCheck(dstInodesInPath, srcChild);
|
|
|
+ if (added) {
|
|
|
srcChild = null;
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: "
|
|
@@ -575,11 +576,10 @@ public class FSDirectory implements Closeable {
|
|
|
return true;
|
|
|
}
|
|
|
} finally {
|
|
|
- if (dstChild == null && srcChild != null) {
|
|
|
+ if (!added && srcChild != null) {
|
|
|
// put it back
|
|
|
srcChild.setLocalName(srcChildName);
|
|
|
- addChildNoQuotaCheck(srcInodesInPath, srcInodes.length - 1, srcChild,
|
|
|
- UNKNOWN_DISK_SPACE);
|
|
|
+ addLastINodeNoQuotaCheck(srcInodesInPath, srcChild);
|
|
|
}
|
|
|
}
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
@@ -699,7 +699,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
// Ensure dst has quota to accommodate rename
|
|
|
verifyQuotaForRename(srcInodes, dstInodes);
|
|
|
- INode removedSrc = removeChild(srcInodesInPath, srcInodes.length - 1);
|
|
|
+ INode removedSrc = removeLastINode(srcInodesInPath);
|
|
|
if (removedSrc == null) {
|
|
|
error = "Failed to rename " + src + " to " + dst
|
|
|
+ " because the source can not be removed";
|
|
@@ -712,18 +712,13 @@ public class FSDirectory implements Closeable {
|
|
|
INode removedDst = null;
|
|
|
try {
|
|
|
if (dstInode != null) { // dst exists remove it
|
|
|
- removedDst = removeChild(dstInodesInPath, dstInodes.length - 1);
|
|
|
+ removedDst = removeLastINode(dstInodesInPath);
|
|
|
dstChildName = removedDst.getLocalName();
|
|
|
}
|
|
|
|
|
|
- INode dstChild = null;
|
|
|
removedSrc.setLocalName(dstComponents[dstInodes.length - 1]);
|
|
|
// add src as dst to complete rename
|
|
|
- dstChild = addChildNoQuotaCheck(dstInodesInPath, dstInodes.length - 1,
|
|
|
- removedSrc, UNKNOWN_DISK_SPACE);
|
|
|
-
|
|
|
- int filesDeleted = 0;
|
|
|
- if (dstChild != null) {
|
|
|
+ if (addLastINodeNoQuotaCheck(dstInodesInPath, removedSrc)) {
|
|
|
removedSrc = null;
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug(
|
|
@@ -736,6 +731,7 @@ public class FSDirectory implements Closeable {
|
|
|
getFSNamesystem().unprotectedChangeLease(src, dst);
|
|
|
|
|
|
// Collect the blocks and remove the lease for previous dst
|
|
|
+ int filesDeleted = 0;
|
|
|
if (removedDst != null) {
|
|
|
INode rmdst = removedDst;
|
|
|
removedDst = null;
|
|
@@ -749,14 +745,12 @@ public class FSDirectory implements Closeable {
|
|
|
if (removedSrc != null) {
|
|
|
// Rename failed - restore src
|
|
|
removedSrc.setLocalName(srcChildName);
|
|
|
- addChildNoQuotaCheck(srcInodesInPath, srcInodes.length - 1, removedSrc,
|
|
|
- UNKNOWN_DISK_SPACE);
|
|
|
+ addLastINodeNoQuotaCheck(srcInodesInPath, removedSrc);
|
|
|
}
|
|
|
if (removedDst != null) {
|
|
|
// Rename failed - restore dst
|
|
|
removedDst.setLocalName(dstChildName);
|
|
|
- addChildNoQuotaCheck(dstInodesInPath, dstInodes.length - 1, removedDst,
|
|
|
- UNKNOWN_DISK_SPACE);
|
|
|
+ addLastINodeNoQuotaCheck(dstInodesInPath, removedDst);
|
|
|
}
|
|
|
}
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
@@ -1055,14 +1049,13 @@ public class FSDirectory implements Closeable {
|
|
|
" because the root is not allowed to be deleted");
|
|
|
return 0;
|
|
|
}
|
|
|
- int pos = inodes.length - 1;
|
|
|
// Remove the node from the namespace
|
|
|
- targetNode = removeChild(inodesInPath, pos);
|
|
|
+ targetNode = removeLastINode(inodesInPath);
|
|
|
if (targetNode == null) {
|
|
|
return 0;
|
|
|
}
|
|
|
// set the parent's modification time
|
|
|
- inodes[pos-1].setModificationTime(mtime);
|
|
|
+ inodes[inodes.length - 2].setModificationTime(mtime);
|
|
|
int filesRemoved = targetNode.collectSubtreeBlocksAndClear(collectedBlocks);
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
|
|
@@ -1091,7 +1084,7 @@ public class FSDirectory implements Closeable {
|
|
|
/* Currently oldnode and newnode are assumed to contain the same
|
|
|
* blocks. Otherwise, blocks need to be removed from the blocksMap.
|
|
|
*/
|
|
|
- rootDir.addNode(path, newnode);
|
|
|
+ rootDir.addINode(path, newnode);
|
|
|
|
|
|
int index = 0;
|
|
|
for (BlockInfo b : newnode.getBlocks()) {
|
|
@@ -1193,21 +1186,6 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Get the parent node of path.
|
|
|
- *
|
|
|
- * @param path the path to explore
|
|
|
- * @return its parent node
|
|
|
- */
|
|
|
- INodeDirectory getParent(byte[][] path)
|
|
|
- throws FileNotFoundException, UnresolvedLinkException {
|
|
|
- readLock();
|
|
|
- try {
|
|
|
- return rootDir.getParent(path);
|
|
|
- } finally {
|
|
|
- readUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Check whether the filepath could be created
|
|
@@ -1249,20 +1227,17 @@ public class FSDirectory implements Closeable {
|
|
|
* @param nsDelta the delta change of namespace
|
|
|
* @param dsDelta the delta change of diskspace
|
|
|
* @throws QuotaExceededException if the new count violates any quota limit
|
|
|
- * @throws FileNotFound if path does not exist.
|
|
|
+ * @throws FileNotFoundException if path does not exist.
|
|
|
*/
|
|
|
void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
|
|
|
- throws QuotaExceededException,
|
|
|
- FileNotFoundException,
|
|
|
- UnresolvedLinkException {
|
|
|
+ throws QuotaExceededException, FileNotFoundException, UnresolvedLinkException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
final INodesInPath inodesInPath = rootDir.getExistingPathINodes(path, false);
|
|
|
final INode[] inodes = inodesInPath.getINodes();
|
|
|
int len = inodes.length;
|
|
|
if (inodes[len - 1] == null) {
|
|
|
- throw new FileNotFoundException(path +
|
|
|
- " does not exist under rootDir.");
|
|
|
+ throw new FileNotFoundException("Path not found: " + path);
|
|
|
}
|
|
|
updateCount(inodesInPath, len-1, nsDelta, dsDelta, true);
|
|
|
} finally {
|
|
@@ -1491,15 +1466,17 @@ public class FSDirectory implements Closeable {
|
|
|
long timestamp) throws QuotaExceededException {
|
|
|
assert hasWriteLock();
|
|
|
final INodeDirectory dir = new INodeDirectory(name, permission, timestamp);
|
|
|
- final INode inode = addChild(inodesInPath, pos, dir, -1, true);
|
|
|
- inodesInPath.setINode(pos, inode);
|
|
|
+ if (addChild(inodesInPath, pos, dir, true)) {
|
|
|
+ inodesInPath.setINode(pos, dir);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- /** Add a node child to the namespace. The full path name of the node is src.
|
|
|
- * childDiskspace should be -1, if unknown.
|
|
|
+ /**
|
|
|
+ * Add the given child to the namespace.
|
|
|
+ * @param src The full path name of the child node.
|
|
|
* @throw QuotaExceededException is thrown if it violates quota limit
|
|
|
*/
|
|
|
- private <T extends INode> T addNode(String src, T child, long childDiskspace
|
|
|
+ private boolean addINode(String src, INode child
|
|
|
) throws QuotaExceededException, UnresolvedLinkException {
|
|
|
byte[][] components = INode.getPathComponents(src);
|
|
|
byte[] path = components[components.length-1];
|
|
@@ -1509,8 +1486,7 @@ public class FSDirectory implements Closeable {
|
|
|
try {
|
|
|
INodesInPath inodesInPath = rootDir.getExistingPathINodes(components,
|
|
|
components.length, false);
|
|
|
- return addChild(inodesInPath, inodesInPath.getINodes().length-1, child,
|
|
|
- childDiskspace, true);
|
|
|
+ return addLastINode(inodesInPath, child, true);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -1634,14 +1610,24 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * The same as {@link #addChild(INodesInPath, int, INode, boolean)}
|
|
|
+ * with pos = length - 1.
|
|
|
+ */
|
|
|
+ private boolean addLastINode(INodesInPath inodesInPath,
|
|
|
+ INode inode, boolean checkQuota) throws QuotaExceededException {
|
|
|
+ final int pos = inodesInPath.getINodes().length - 1;
|
|
|
+ return addChild(inodesInPath, pos, inode, checkQuota);
|
|
|
+ }
|
|
|
+
|
|
|
/** Add a node child to the inodes at index pos.
|
|
|
* Its ancestors are stored at [0, pos-1].
|
|
|
- * @return the added node.
|
|
|
+ * @return false if the child with this name already exists;
|
|
|
+ * otherwise return true;
|
|
|
* @throw QuotaExceededException is thrown if it violates quota limit
|
|
|
*/
|
|
|
- private <T extends INode> T addChild(INodesInPath inodesInPath, int pos,
|
|
|
- T child, long childDiskspace,
|
|
|
- boolean checkQuota) throws QuotaExceededException {
|
|
|
+ private boolean addChild(INodesInPath inodesInPath, int pos,
|
|
|
+ INode child, boolean checkQuota) throws QuotaExceededException {
|
|
|
final INode[] inodes = inodesInPath.getINodes();
|
|
|
// The filesystem limits are not really quotas, so this check may appear
|
|
|
// odd. It's because a rename operation deletes the src, tries to add
|
|
@@ -1655,38 +1641,34 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
INode.DirCounts counts = new INode.DirCounts();
|
|
|
child.spaceConsumedInTree(counts);
|
|
|
- if (childDiskspace < 0) {
|
|
|
- childDiskspace = counts.getDsCount();
|
|
|
- }
|
|
|
- updateCount(inodesInPath, pos, counts.getNsCount(), childDiskspace, checkQuota);
|
|
|
+ updateCount(inodesInPath, pos, counts.getNsCount(), counts.getDsCount(), checkQuota);
|
|
|
if (inodes[pos-1] == null) {
|
|
|
throw new NullPointerException("Panic: parent does not exist");
|
|
|
}
|
|
|
- final T addedNode = ((INodeDirectory)inodes[pos-1]).addChild(child, true);
|
|
|
- if (addedNode == null) {
|
|
|
- updateCount(inodesInPath, pos, -counts.getNsCount(), -childDiskspace, true);
|
|
|
+ final boolean added = ((INodeDirectory)inodes[pos-1]).addChild(child, true);
|
|
|
+ if (!added) {
|
|
|
+ updateCount(inodesInPath, pos, -counts.getNsCount(), -counts.getDsCount(), true);
|
|
|
}
|
|
|
- return addedNode;
|
|
|
+ return added;
|
|
|
}
|
|
|
|
|
|
- private <T extends INode> T addChildNoQuotaCheck(INodesInPath inodesInPath,
|
|
|
- int pos, T child, long childDiskspace) {
|
|
|
- T inode = null;
|
|
|
+ private boolean addLastINodeNoQuotaCheck(INodesInPath inodesInPath, INode i) {
|
|
|
try {
|
|
|
- inode = addChild(inodesInPath, pos, child, childDiskspace, false);
|
|
|
+ return addLastINode(inodesInPath, i, false);
|
|
|
} catch (QuotaExceededException e) {
|
|
|
NameNode.LOG.warn("FSDirectory.addChildNoQuotaCheck - unexpected", e);
|
|
|
}
|
|
|
- return inode;
|
|
|
+ return false;
|
|
|
}
|
|
|
|
|
|
- /** Remove an inode at index pos from the namespace.
|
|
|
- * Its ancestors are stored at [0, pos-1].
|
|
|
+ /**
|
|
|
+ * Remove the last inode in the path from the namespace.
|
|
|
* Count of each ancestor with quota is also updated.
|
|
|
- * Return the removed node; null if the removal fails.
|
|
|
+ * @return the removed node; null if the removal fails.
|
|
|
*/
|
|
|
- private INode removeChild(final INodesInPath inodesInPath, int pos) {
|
|
|
+ private INode removeLastINode(final INodesInPath inodesInPath) {
|
|
|
final INode[] inodes = inodesInPath.getINodes();
|
|
|
+ final int pos = inodes.length - 1;
|
|
|
INode removedNode = ((INodeDirectory)inodes[pos-1]).removeChild(inodes[pos]);
|
|
|
if (removedNode != null) {
|
|
|
INode.DirCounts counts = new INode.DirCounts();
|
|
@@ -1805,14 +1787,15 @@ public class FSDirectory implements Closeable {
|
|
|
* See {@link ClientProtocol#setQuota(String, long, long)} for the contract.
|
|
|
* Sets quota for for a directory.
|
|
|
* @returns INodeDirectory if any of the quotas have changed. null other wise.
|
|
|
- * @throws FileNotFoundException if the path does not exist or is a file
|
|
|
+ * @throws FileNotFoundException if the path does not exist.
|
|
|
+ * @throws PathIsNotDirectoryException if the path is not a directory.
|
|
|
* @throws QuotaExceededException if the directory tree size is
|
|
|
* greater than the given quota
|
|
|
* @throws UnresolvedLinkException if a symlink is encountered in src.
|
|
|
*/
|
|
|
INodeDirectory unprotectedSetQuota(String src, long nsQuota, long dsQuota)
|
|
|
- throws FileNotFoundException, QuotaExceededException,
|
|
|
- UnresolvedLinkException {
|
|
|
+ throws FileNotFoundException, PathIsNotDirectoryException,
|
|
|
+ QuotaExceededException, UnresolvedLinkException {
|
|
|
assert hasWriteLock();
|
|
|
// sanity check
|
|
|
if ((nsQuota < 0 && nsQuota != HdfsConstants.QUOTA_DONT_SET &&
|
|
@@ -1828,15 +1811,10 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
final INodesInPath inodesInPath = rootDir.getExistingPathINodes(src, true);
|
|
|
final INode[] inodes = inodesInPath.getINodes();
|
|
|
- INode targetNode = inodes[inodes.length-1];
|
|
|
- if (targetNode == null) {
|
|
|
- throw new FileNotFoundException("Directory does not exist: " + srcs);
|
|
|
- } else if (!targetNode.isDirectory()) {
|
|
|
- throw new FileNotFoundException("Cannot set quota on a file: " + srcs);
|
|
|
- } else if (targetNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
|
|
|
+ INodeDirectory dirNode = INodeDirectory.valueOf(inodes[inodes.length-1], srcs);
|
|
|
+ if (dirNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
|
|
|
throw new IllegalArgumentException("Cannot clear namespace quota on root.");
|
|
|
} else { // a directory inode
|
|
|
- INodeDirectory dirNode = (INodeDirectory)targetNode;
|
|
|
long oldNsQuota = dirNode.getNsQuota();
|
|
|
long oldDsQuota = dirNode.getDsQuota();
|
|
|
if (nsQuota == HdfsConstants.QUOTA_DONT_SET) {
|
|
@@ -1870,13 +1848,12 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * See {@link ClientProtocol#setQuota(String, long, long)} for the
|
|
|
- * contract.
|
|
|
+ * See {@link ClientProtocol#setQuota(String, long, long)} for the contract.
|
|
|
* @see #unprotectedSetQuota(String, long, long)
|
|
|
*/
|
|
|
void setQuota(String src, long nsQuota, long dsQuota)
|
|
|
- throws FileNotFoundException, QuotaExceededException,
|
|
|
- UnresolvedLinkException {
|
|
|
+ throws FileNotFoundException, PathIsNotDirectoryException,
|
|
|
+ QuotaExceededException, UnresolvedLinkException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
INodeDirectory dir = unprotectedSetQuota(src, nsQuota, dsQuota);
|
|
@@ -2085,7 +2062,7 @@ public class FSDirectory implements Closeable {
|
|
|
throws UnresolvedLinkException, QuotaExceededException {
|
|
|
assert hasWriteLock();
|
|
|
final INodeSymlink symlink = new INodeSymlink(target, mtime, atime, perm);
|
|
|
- return addNode(path, symlink, UNKNOWN_DISK_SPACE);
|
|
|
+ return addINode(path, symlink)? symlink: null;
|
|
|
}
|
|
|
|
|
|
/**
|