|
@@ -44,10 +44,10 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
@@ -57,6 +57,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
|
|
|
import org.apache.hadoop.hdfs.util.ByteArray;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
@@ -140,7 +142,7 @@ public class FSDirectory implements Closeable {
|
|
|
DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT);
|
|
|
NameNode.LOG.info("Caching file names occuring more than " + threshold
|
|
|
- + " times ");
|
|
|
+ + " times");
|
|
|
nameCache = new NameCache<ByteArray>(threshold);
|
|
|
namesystem = ns;
|
|
|
}
|
|
@@ -253,15 +255,12 @@ public class FSDirectory implements Closeable {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
if (newNode == null) {
|
|
|
- NameNode.stateChangeLog.info("DIR* FSDirectory.addFile: "
|
|
|
- +"failed to add "+path
|
|
|
- +" to the file system");
|
|
|
+ NameNode.stateChangeLog.info("DIR* addFile: failed to add " + path);
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
|
|
|
- +path+" is added to the file system");
|
|
|
+ NameNode.stateChangeLog.debug("DIR* addFile: " + path + " is added");
|
|
|
}
|
|
|
return newNode;
|
|
|
}
|
|
@@ -315,7 +314,7 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
if(newParent == null)
|
|
|
return null;
|
|
|
- if(!newNode.isDirectory() && !newNode.isLink()) {
|
|
|
+ if(!newNode.isDirectory() && !newNode.isSymlink()) {
|
|
|
// Add file->block mapping
|
|
|
INodeFile newF = (INodeFile)newNode;
|
|
|
BlockInfo[] blocks = newF.getBlocks();
|
|
@@ -332,22 +331,18 @@ public class FSDirectory implements Closeable {
|
|
|
/**
|
|
|
* Add a block to the file. Returns a reference to the added block.
|
|
|
*/
|
|
|
- BlockInfo addBlock(String path,
|
|
|
- INode[] inodes,
|
|
|
- Block block,
|
|
|
- DatanodeDescriptor targets[]
|
|
|
- ) throws QuotaExceededException {
|
|
|
+ BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
|
|
|
+ DatanodeDescriptor targets[]) throws IOException {
|
|
|
waitForReady();
|
|
|
|
|
|
writeLock();
|
|
|
try {
|
|
|
- assert inodes[inodes.length-1].isUnderConstruction() :
|
|
|
- "INode should correspond to a file under construction";
|
|
|
- INodeFileUnderConstruction fileINode =
|
|
|
- (INodeFileUnderConstruction)inodes[inodes.length-1];
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
+ final INodeFileUnderConstruction fileINode =
|
|
|
+ INodeFileUnderConstruction.valueOf(inodes[inodes.length-1], path);
|
|
|
|
|
|
// check quota limits and updated space consumed
|
|
|
- updateCount(inodes, inodes.length-1, 0,
|
|
|
+ updateCount(inodesInPath, inodes.length-1, 0,
|
|
|
fileINode.getPreferredBlockSize()*fileINode.getBlockReplication(), true);
|
|
|
|
|
|
// associate new last block for the file
|
|
@@ -443,8 +438,9 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
|
|
|
// update space consumed
|
|
|
- INode[] pathINodes = getExistingPathINodes(path);
|
|
|
- updateCount(pathINodes, pathINodes.length-1, 0,
|
|
|
+ final INodesInPath inodesInPath = rootDir.getExistingPathINodes(path, true);
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
+ updateCount(inodesInPath, inodes.length-1, 0,
|
|
|
-fileNode.getPreferredBlockSize()*fileNode.getBlockReplication(), true);
|
|
|
}
|
|
|
|
|
@@ -512,7 +508,8 @@ public class FSDirectory implements Closeable {
|
|
|
throws QuotaExceededException, UnresolvedLinkException,
|
|
|
FileAlreadyExistsException {
|
|
|
assert hasWriteLock();
|
|
|
- INode[] srcInodes = rootDir.getExistingPathINodes(src, false);
|
|
|
+ INodesInPath srcInodesInPath = rootDir.getExistingPathINodes(src, false);
|
|
|
+ INode[] srcInodes = srcInodesInPath.getINodes();
|
|
|
INode srcInode = srcInodes[srcInodes.length-1];
|
|
|
|
|
|
// check the validation of the source
|
|
@@ -535,7 +532,7 @@ public class FSDirectory implements Closeable {
|
|
|
if (dst.equals(src)) {
|
|
|
return true;
|
|
|
}
|
|
|
- if (srcInode.isLink() &&
|
|
|
+ if (srcInode.isSymlink() &&
|
|
|
dst.equals(((INodeSymlink)srcInode).getLinkValue())) {
|
|
|
throw new FileAlreadyExistsException(
|
|
|
"Cannot rename symlink "+src+" to its target "+dst);
|
|
@@ -551,8 +548,9 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
|
|
|
byte[][] dstComponents = INode.getPathComponents(dst);
|
|
|
- INode[] dstInodes = new INode[dstComponents.length];
|
|
|
- rootDir.getExistingPathINodes(dstComponents, dstInodes, false);
|
|
|
+ INodesInPath dstInodesInPath = rootDir.getExistingPathINodes(dstComponents,
|
|
|
+ dstComponents.length, false);
|
|
|
+ INode[] dstInodes = dstInodesInPath.getINodes();
|
|
|
if (dstInodes[dstInodes.length-1] != null) {
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
+"failed to rename "+src+" to "+dst+
|
|
@@ -567,14 +565,14 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
|
|
|
// Ensure dst has quota to accommodate rename
|
|
|
- verifyQuotaForRename(srcInodes,dstInodes);
|
|
|
+ verifyQuotaForRename(srcInodes, dstInodes);
|
|
|
|
|
|
INode dstChild = null;
|
|
|
INode srcChild = null;
|
|
|
String srcChildName = null;
|
|
|
try {
|
|
|
// remove src
|
|
|
- srcChild = removeChild(srcInodes, srcInodes.length-1);
|
|
|
+ srcChild = removeChild(srcInodesInPath, srcInodes.length-1);
|
|
|
if (srcChild == null) {
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
+ "failed to rename " + src + " to " + dst
|
|
@@ -585,7 +583,7 @@ public class FSDirectory implements Closeable {
|
|
|
srcChild.setLocalName(dstComponents[dstInodes.length-1]);
|
|
|
|
|
|
// add src to the destination
|
|
|
- dstChild = addChildNoQuotaCheck(dstInodes, dstInodes.length - 1,
|
|
|
+ dstChild = addChildNoQuotaCheck(dstInodesInPath, dstInodes.length-1,
|
|
|
srcChild, UNKNOWN_DISK_SPACE);
|
|
|
if (dstChild != null) {
|
|
|
srcChild = null;
|
|
@@ -602,7 +600,7 @@ public class FSDirectory implements Closeable {
|
|
|
if (dstChild == null && srcChild != null) {
|
|
|
// put it back
|
|
|
srcChild.setLocalName(srcChildName);
|
|
|
- addChildNoQuotaCheck(srcInodes, srcInodes.length - 1, srcChild,
|
|
|
+ addChildNoQuotaCheck(srcInodesInPath, srcInodes.length - 1, srcChild,
|
|
|
UNKNOWN_DISK_SPACE);
|
|
|
}
|
|
|
}
|
|
@@ -635,7 +633,8 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
String error = null;
|
|
|
- final INode[] srcInodes = rootDir.getExistingPathINodes(src, false);
|
|
|
+ final INodesInPath srcInodesInPath = rootDir.getExistingPathINodes(src, false);
|
|
|
+ final INode[] srcInodes = srcInodesInPath.getINodes();
|
|
|
final INode srcInode = srcInodes[srcInodes.length - 1];
|
|
|
// validate source
|
|
|
if (srcInode == null) {
|
|
@@ -656,7 +655,7 @@ public class FSDirectory implements Closeable {
|
|
|
throw new FileAlreadyExistsException(
|
|
|
"The source "+src+" and destination "+dst+" are the same");
|
|
|
}
|
|
|
- if (srcInode.isLink() &&
|
|
|
+ if (srcInode.isSymlink() &&
|
|
|
dst.equals(((INodeSymlink)srcInode).getLinkValue())) {
|
|
|
throw new FileAlreadyExistsException(
|
|
|
"Cannot rename symlink "+src+" to its target "+dst);
|
|
@@ -671,8 +670,9 @@ public class FSDirectory implements Closeable {
|
|
|
throw new IOException(error);
|
|
|
}
|
|
|
final byte[][] dstComponents = INode.getPathComponents(dst);
|
|
|
- final INode[] dstInodes = new INode[dstComponents.length];
|
|
|
- rootDir.getExistingPathINodes(dstComponents, dstInodes, false);
|
|
|
+ INodesInPath dstInodesInPath = rootDir.getExistingPathINodes(dstComponents,
|
|
|
+ dstComponents.length, false);
|
|
|
+ final INode[] dstInodes = dstInodesInPath.getINodes();
|
|
|
INode dstInode = dstInodes[dstInodes.length - 1];
|
|
|
if (dstInodes.length == 1) {
|
|
|
error = "rename destination cannot be the root";
|
|
@@ -696,7 +696,7 @@ public class FSDirectory implements Closeable {
|
|
|
throw new FileAlreadyExistsException(error);
|
|
|
}
|
|
|
List<INode> children = dstInode.isDirectory() ?
|
|
|
- ((INodeDirectory) dstInode).getChildrenRaw() : null;
|
|
|
+ ((INodeDirectory) dstInode).getChildren() : null;
|
|
|
if (children != null && children.size() != 0) {
|
|
|
error = "rename cannot overwrite non empty destination directory "
|
|
|
+ dst;
|
|
@@ -720,7 +720,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
// Ensure dst has quota to accommodate rename
|
|
|
verifyQuotaForRename(srcInodes, dstInodes);
|
|
|
- INode removedSrc = removeChild(srcInodes, srcInodes.length - 1);
|
|
|
+ INode removedSrc = removeChild(srcInodesInPath, srcInodes.length - 1);
|
|
|
if (removedSrc == null) {
|
|
|
error = "Failed to rename " + src + " to " + dst
|
|
|
+ " because the source can not be removed";
|
|
@@ -733,14 +733,14 @@ public class FSDirectory implements Closeable {
|
|
|
INode removedDst = null;
|
|
|
try {
|
|
|
if (dstInode != null) { // dst exists remove it
|
|
|
- removedDst = removeChild(dstInodes, dstInodes.length - 1);
|
|
|
+ removedDst = removeChild(dstInodesInPath, dstInodes.length - 1);
|
|
|
dstChildName = removedDst.getLocalName();
|
|
|
}
|
|
|
|
|
|
INode dstChild = null;
|
|
|
removedSrc.setLocalName(dstComponents[dstInodes.length - 1]);
|
|
|
// add src as dst to complete rename
|
|
|
- dstChild = addChildNoQuotaCheck(dstInodes, dstInodes.length - 1,
|
|
|
+ dstChild = addChildNoQuotaCheck(dstInodesInPath, dstInodes.length - 1,
|
|
|
removedSrc, UNKNOWN_DISK_SPACE);
|
|
|
|
|
|
int filesDeleted = 0;
|
|
@@ -758,7 +758,7 @@ public class FSDirectory implements Closeable {
|
|
|
if (removedDst != null) {
|
|
|
INode rmdst = removedDst;
|
|
|
removedDst = null;
|
|
|
- List<Block> collectedBlocks = new ArrayList<Block>();
|
|
|
+ BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
|
filesDeleted = rmdst.collectSubtreeBlocksAndClear(collectedBlocks);
|
|
|
getFSNamesystem().removePathAndBlocks(src, collectedBlocks);
|
|
|
}
|
|
@@ -768,13 +768,13 @@ public class FSDirectory implements Closeable {
|
|
|
if (removedSrc != null) {
|
|
|
// Rename failed - restore src
|
|
|
removedSrc.setLocalName(srcChildName);
|
|
|
- addChildNoQuotaCheck(srcInodes, srcInodes.length - 1, removedSrc,
|
|
|
+ addChildNoQuotaCheck(srcInodesInPath, srcInodes.length - 1, removedSrc,
|
|
|
UNKNOWN_DISK_SPACE);
|
|
|
}
|
|
|
if (removedDst != null) {
|
|
|
// Rename failed - restore dst
|
|
|
removedDst.setLocalName(dstChildName);
|
|
|
- addChildNoQuotaCheck(dstInodes, dstInodes.length - 1, removedDst,
|
|
|
+ addChildNoQuotaCheck(dstInodesInPath, dstInodes.length - 1, removedDst,
|
|
|
UNKNOWN_DISK_SPACE);
|
|
|
}
|
|
|
}
|
|
@@ -814,12 +814,13 @@ public class FSDirectory implements Closeable {
|
|
|
UnresolvedLinkException {
|
|
|
assert hasWriteLock();
|
|
|
|
|
|
- INode[] inodes = rootDir.getExistingPathINodes(src, true);
|
|
|
+ final INodesInPath inodesInPath = rootDir.getExistingPathINodes(src, true);
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
INode inode = inodes[inodes.length - 1];
|
|
|
if (inode == null) {
|
|
|
return null;
|
|
|
}
|
|
|
- assert !inode.isLink();
|
|
|
+ assert !inode.isSymlink();
|
|
|
if (inode.isDirectory()) {
|
|
|
return null;
|
|
|
}
|
|
@@ -828,7 +829,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
// check disk quota
|
|
|
long dsDelta = (replication - oldRepl) * (fileNode.diskspaceConsumed()/oldRepl);
|
|
|
- updateCount(inodes, inodes.length-1, 0, dsDelta, true);
|
|
|
+ updateCount(inodesInPath, inodes.length-1, 0, dsDelta, true);
|
|
|
|
|
|
fileNode.setReplication(replication);
|
|
|
|
|
@@ -851,7 +852,7 @@ public class FSDirectory implements Closeable {
|
|
|
if (inode == null) {
|
|
|
throw new FileNotFoundException("File does not exist: " + filename);
|
|
|
}
|
|
|
- if (inode.isDirectory() || inode.isLink()) {
|
|
|
+ if (inode.isDirectory() || inode.isSymlink()) {
|
|
|
throw new IOException("Getting block size of non-file: "+ filename);
|
|
|
}
|
|
|
return ((INodeFile)inode).getPreferredBlockSize();
|
|
@@ -868,7 +869,7 @@ public class FSDirectory implements Closeable {
|
|
|
if (inode == null) {
|
|
|
return false;
|
|
|
}
|
|
|
- return inode.isDirectory() || inode.isLink()
|
|
|
+ return inode.isDirectory() || inode.isSymlink()
|
|
|
? true
|
|
|
: ((INodeFile)inode).getBlocks() != null;
|
|
|
} finally {
|
|
@@ -958,7 +959,8 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
// do the move
|
|
|
|
|
|
- INode [] trgINodes = getExistingPathINodes(target);
|
|
|
+ final INodesInPath trgINodesInPath = rootDir.getExistingPathINodes(target, true);
|
|
|
+ final INode[] trgINodes = trgINodesInPath.getINodes();
|
|
|
INodeFile trgInode = (INodeFile) trgINodes[trgINodes.length-1];
|
|
|
INodeDirectory trgParent = (INodeDirectory)trgINodes[trgINodes.length-2];
|
|
|
|
|
@@ -966,9 +968,9 @@ public class FSDirectory implements Closeable {
|
|
|
int i = 0;
|
|
|
int totalBlocks = 0;
|
|
|
for(String src : srcs) {
|
|
|
- INodeFile srcInode = getFileINode(src);
|
|
|
+ INodeFile srcInode = (INodeFile)getINode(src);
|
|
|
allSrcInodes[i++] = srcInode;
|
|
|
- totalBlocks += srcInode.blocks.length;
|
|
|
+ totalBlocks += srcInode.numBlocks();
|
|
|
}
|
|
|
trgInode.appendBlocks(allSrcInodes, totalBlocks); // copy the blocks
|
|
|
|
|
@@ -977,7 +979,7 @@ public class FSDirectory implements Closeable {
|
|
|
for(INodeFile nodeToRemove: allSrcInodes) {
|
|
|
if(nodeToRemove == null) continue;
|
|
|
|
|
|
- nodeToRemove.blocks = null;
|
|
|
+ nodeToRemove.setBlocks(null);
|
|
|
trgParent.removeChild(nodeToRemove);
|
|
|
count++;
|
|
|
}
|
|
@@ -985,7 +987,7 @@ public class FSDirectory implements Closeable {
|
|
|
trgInode.setModificationTimeForce(timestamp);
|
|
|
trgParent.setModificationTime(timestamp);
|
|
|
// update quota on the parent directory ('count' files removed, 0 space)
|
|
|
- unprotectedUpdateCount(trgINodes, trgINodes.length-1, - count, 0);
|
|
|
+ unprotectedUpdateCount(trgINodesInPath, trgINodes.length-1, -count, 0);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -995,7 +997,7 @@ public class FSDirectory implements Closeable {
|
|
|
* @param collectedBlocks Blocks under the deleted directory
|
|
|
* @return true on successful deletion; else false
|
|
|
*/
|
|
|
- boolean delete(String src, List<Block>collectedBlocks)
|
|
|
+ boolean delete(String src, BlocksMapUpdateInfo collectedBlocks)
|
|
|
throws UnresolvedLinkException {
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + src);
|
|
@@ -1019,35 +1021,21 @@ public class FSDirectory implements Closeable {
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- /** Return if a directory is empty or not **/
|
|
|
- boolean isDirEmpty(String src) throws UnresolvedLinkException {
|
|
|
- boolean dirNotEmpty = true;
|
|
|
- if (!isDir(src)) {
|
|
|
- return true;
|
|
|
- }
|
|
|
+ /**
|
|
|
+ * @return true if the path is a non-empty directory; otherwise, return false.
|
|
|
+ */
|
|
|
+ boolean isNonEmptyDirectory(String path) throws UnresolvedLinkException {
|
|
|
readLock();
|
|
|
try {
|
|
|
- INode targetNode = rootDir.getNode(src, false);
|
|
|
- assert targetNode != null : "should be taken care in isDir() above";
|
|
|
- if (((INodeDirectory)targetNode).getChildren().size() != 0) {
|
|
|
- dirNotEmpty = false;
|
|
|
+ final INode inode = rootDir.getNode(path, false);
|
|
|
+ if (inode == null || !inode.isDirectory()) {
|
|
|
+ //not found or not a directory
|
|
|
+ return false;
|
|
|
}
|
|
|
+ return ((INodeDirectory)inode).getChildrenList().size() != 0;
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
|
- return dirNotEmpty;
|
|
|
- }
|
|
|
-
|
|
|
- boolean isEmpty() {
|
|
|
- try {
|
|
|
- return isDirEmpty("/");
|
|
|
- } catch (UnresolvedLinkException e) {
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("/ cannot be a symlink");
|
|
|
- }
|
|
|
- assert false : "/ cannot be a symlink";
|
|
|
- return true;
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1062,7 +1050,7 @@ public class FSDirectory implements Closeable {
|
|
|
void unprotectedDelete(String src, long mtime)
|
|
|
throws UnresolvedLinkException {
|
|
|
assert hasWriteLock();
|
|
|
- List<Block> collectedBlocks = new ArrayList<Block>();
|
|
|
+ BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
|
int filesRemoved = unprotectedDelete(src, collectedBlocks, mtime);
|
|
|
if (filesRemoved > 0) {
|
|
|
getFSNamesystem().removePathAndBlocks(src, collectedBlocks);
|
|
@@ -1077,12 +1065,13 @@ public class FSDirectory implements Closeable {
|
|
|
* @param mtime the time the inode is removed
|
|
|
* @return the number of inodes deleted; 0 if no inodes are deleted.
|
|
|
*/
|
|
|
- int unprotectedDelete(String src, List<Block> collectedBlocks,
|
|
|
+ int unprotectedDelete(String src, BlocksMapUpdateInfo collectedBlocks,
|
|
|
long mtime) throws UnresolvedLinkException {
|
|
|
assert hasWriteLock();
|
|
|
src = normalizePath(src);
|
|
|
|
|
|
- INode[] inodes = rootDir.getExistingPathINodes(src, false);
|
|
|
+ final INodesInPath inodesInPath = rootDir.getExistingPathINodes(src, false);
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
INode targetNode = inodes[inodes.length-1];
|
|
|
|
|
|
if (targetNode == null) { // non-existent src
|
|
@@ -1100,7 +1089,7 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
int pos = inodes.length - 1;
|
|
|
// Remove the node from the namespace
|
|
|
- targetNode = removeChild(inodes, pos);
|
|
|
+ targetNode = removeChild(inodesInPath, pos);
|
|
|
if (targetNode == null) {
|
|
|
return 0;
|
|
|
}
|
|
@@ -1171,7 +1160,7 @@ public class FSDirectory implements Closeable {
|
|
|
targetNode, needLocation)}, 0);
|
|
|
}
|
|
|
INodeDirectory dirInode = (INodeDirectory)targetNode;
|
|
|
- List<INode> contents = dirInode.getChildren();
|
|
|
+ List<INode> contents = dirInode.getChildrenList();
|
|
|
int startChild = dirInode.nextChild(startAfter);
|
|
|
int totalNumChildren = contents.size();
|
|
|
int numOfListing = Math.min(totalNumChildren-startChild, this.lsLimit);
|
|
@@ -1222,7 +1211,7 @@ public class FSDirectory implements Closeable {
|
|
|
return null;
|
|
|
if (targetNode.isDirectory())
|
|
|
return null;
|
|
|
- if (targetNode.isLink())
|
|
|
+ if (targetNode.isSymlink())
|
|
|
return null;
|
|
|
return ((INodeFile)targetNode).getBlocks();
|
|
|
} finally {
|
|
@@ -1230,47 +1219,13 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Get {@link INode} associated with the file.
|
|
|
- */
|
|
|
- INodeFile getFileINode(String src) throws UnresolvedLinkException {
|
|
|
- INode inode = getINode(src);
|
|
|
- if (inode == null || inode.isDirectory())
|
|
|
- return null;
|
|
|
- assert !inode.isLink();
|
|
|
- return (INodeFile) inode;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Get {@link INode} associated with the file / directory.
|
|
|
*/
|
|
|
INode getINode(String src) throws UnresolvedLinkException {
|
|
|
readLock();
|
|
|
try {
|
|
|
- INode iNode = rootDir.getNode(src, true);
|
|
|
- return iNode;
|
|
|
- } finally {
|
|
|
- readUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Retrieve the existing INodes along the given path.
|
|
|
- *
|
|
|
- * @param path the path to explore
|
|
|
- * @return INodes array containing the existing INodes in the order they
|
|
|
- * appear when following the path from the root INode to the
|
|
|
- * deepest INodes. The array size will be the number of expected
|
|
|
- * components in the path, and non existing components will be
|
|
|
- * filled with null
|
|
|
- *
|
|
|
- * @see INodeDirectory#getExistingPathINodes(byte[][], INode[])
|
|
|
- */
|
|
|
- INode[] getExistingPathINodes(String path)
|
|
|
- throws UnresolvedLinkException {
|
|
|
- readLock();
|
|
|
- try {
|
|
|
- return rootDir.getExistingPathINodes(path, true);
|
|
|
+ return rootDir.getNode(src, true);
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -1340,13 +1295,14 @@ public class FSDirectory implements Closeable {
|
|
|
UnresolvedLinkException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
- INode[] inodes = rootDir.getExistingPathINodes(path, false);
|
|
|
+ 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.");
|
|
|
}
|
|
|
- updateCount(inodes, len-1, nsDelta, dsDelta, true);
|
|
|
+ updateCount(inodesInPath, len-1, nsDelta, dsDelta, true);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -1361,7 +1317,7 @@ public class FSDirectory implements Closeable {
|
|
|
* @param checkQuota if true then check if quota is exceeded
|
|
|
* @throws QuotaExceededException if the new count violates any quota limit
|
|
|
*/
|
|
|
- private void updateCount(INode[] inodes, int numOfINodes,
|
|
|
+ private void updateCount(INodesInPath inodesInPath, int numOfINodes,
|
|
|
long nsDelta, long dsDelta, boolean checkQuota)
|
|
|
throws QuotaExceededException {
|
|
|
assert hasWriteLock();
|
|
@@ -1369,29 +1325,25 @@ public class FSDirectory implements Closeable {
|
|
|
//still initializing. do not check or update quotas.
|
|
|
return;
|
|
|
}
|
|
|
- if (numOfINodes>inodes.length) {
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
+ if (numOfINodes > inodes.length) {
|
|
|
numOfINodes = inodes.length;
|
|
|
}
|
|
|
if (checkQuota) {
|
|
|
verifyQuota(inodes, numOfINodes, nsDelta, dsDelta, null);
|
|
|
}
|
|
|
- for(int i = 0; i < numOfINodes; i++) {
|
|
|
- if (inodes[i].isQuotaSet()) { // a directory with quota
|
|
|
- INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i];
|
|
|
- node.updateNumItemsInTree(nsDelta, dsDelta);
|
|
|
- }
|
|
|
- }
|
|
|
+ unprotectedUpdateCount(inodesInPath, numOfINodes, nsDelta, dsDelta);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* update quota of each inode and check to see if quota is exceeded.
|
|
|
* See {@link #updateCount(INode[], int, long, long, boolean)}
|
|
|
*/
|
|
|
- private void updateCountNoQuotaCheck(INode[] inodes, int numOfINodes,
|
|
|
- long nsDelta, long dsDelta) {
|
|
|
+ private void updateCountNoQuotaCheck(INodesInPath inodesInPath,
|
|
|
+ int numOfINodes, long nsDelta, long dsDelta) {
|
|
|
assert hasWriteLock();
|
|
|
try {
|
|
|
- updateCount(inodes, numOfINodes, nsDelta, dsDelta, false);
|
|
|
+ updateCount(inodesInPath, numOfINodes, nsDelta, dsDelta, false);
|
|
|
} catch (QuotaExceededException e) {
|
|
|
NameNode.LOG.warn("FSDirectory.updateCountNoQuotaCheck - unexpected ", e);
|
|
|
}
|
|
@@ -1405,9 +1357,10 @@ public class FSDirectory implements Closeable {
|
|
|
* @param nsDelta
|
|
|
* @param dsDelta
|
|
|
*/
|
|
|
- void unprotectedUpdateCount(INode[] inodes, int numOfINodes,
|
|
|
- long nsDelta, long dsDelta) {
|
|
|
- assert hasWriteLock();
|
|
|
+ private void unprotectedUpdateCount(INodesInPath inodesInPath,
|
|
|
+ int numOfINodes, long nsDelta, long dsDelta) {
|
|
|
+ assert hasWriteLock();
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
for(int i=0; i < numOfINodes; i++) {
|
|
|
if (inodes[i].isQuotaSet()) { // a directory with quota
|
|
|
INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i];
|
|
@@ -1472,18 +1425,19 @@ public class FSDirectory implements Closeable {
|
|
|
src = normalizePath(src);
|
|
|
String[] names = INode.getPathNames(src);
|
|
|
byte[][] components = INode.getPathComponents(names);
|
|
|
- INode[] inodes = new INode[components.length];
|
|
|
- final int lastInodeIndex = inodes.length - 1;
|
|
|
+ final int lastInodeIndex = components.length - 1;
|
|
|
|
|
|
writeLock();
|
|
|
try {
|
|
|
- rootDir.getExistingPathINodes(components, inodes, false);
|
|
|
+ INodesInPath inodesInPath = rootDir.getExistingPathINodes(components,
|
|
|
+ components.length, false);
|
|
|
+ INode[] inodes = inodesInPath.getINodes();
|
|
|
|
|
|
// find the index of the first null in inodes[]
|
|
|
StringBuilder pathbuilder = new StringBuilder();
|
|
|
int i = 1;
|
|
|
for(; i < inodes.length && inodes[i] != null; i++) {
|
|
|
- pathbuilder.append(Path.SEPARATOR + names[i]);
|
|
|
+ pathbuilder.append(Path.SEPARATOR).append(names[i]);
|
|
|
if (!inodes[i].isDirectory()) {
|
|
|
throw new FileAlreadyExistsException("Parent path is not a directory: "
|
|
|
+ pathbuilder+ " "+inodes[i].getLocalName());
|
|
@@ -1525,8 +1479,7 @@ public class FSDirectory implements Closeable {
|
|
|
// create directories beginning from the first null index
|
|
|
for(; i < inodes.length; i++) {
|
|
|
pathbuilder.append(Path.SEPARATOR + names[i]);
|
|
|
- String cur = pathbuilder.toString();
|
|
|
- unprotectedMkdir(inodes, i, components[i],
|
|
|
+ unprotectedMkdir(inodesInPath, i, components[i],
|
|
|
(i < lastInodeIndex) ? parentPermissions : permissions, now);
|
|
|
if (inodes[i] == null) {
|
|
|
return false;
|
|
@@ -1535,6 +1488,8 @@ public class FSDirectory implements Closeable {
|
|
|
// to match count of FilesDeleted metric.
|
|
|
if (getFSNamesystem() != null)
|
|
|
NameNode.getNameNodeMetrics().incrFilesCreated();
|
|
|
+
|
|
|
+ final String cur = pathbuilder.toString();
|
|
|
fsImage.getEditLog().logMkDir(cur, inodes[i]);
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug(
|
|
@@ -1547,49 +1502,48 @@ public class FSDirectory implements Closeable {
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- */
|
|
|
INode unprotectedMkdir(String src, PermissionStatus permissions,
|
|
|
long timestamp) throws QuotaExceededException,
|
|
|
UnresolvedLinkException {
|
|
|
assert hasWriteLock();
|
|
|
byte[][] components = INode.getPathComponents(src);
|
|
|
- INode[] inodes = new INode[components.length];
|
|
|
-
|
|
|
- rootDir.getExistingPathINodes(components, inodes, false);
|
|
|
- unprotectedMkdir(inodes, inodes.length-1, components[inodes.length-1],
|
|
|
- permissions, timestamp);
|
|
|
- return inodes[inodes.length-1];
|
|
|
+ INodesInPath inodesInPath = rootDir.getExistingPathINodes(components,
|
|
|
+ components.length, false);
|
|
|
+ INode[] inodes = inodesInPath.getINodes();
|
|
|
+ final int pos = inodes.length - 1;
|
|
|
+ unprotectedMkdir(inodesInPath, pos, components[pos], permissions, timestamp);
|
|
|
+ return inodes[pos];
|
|
|
}
|
|
|
|
|
|
/** create a directory at index pos.
|
|
|
* The parent path to the directory is at [0, pos-1].
|
|
|
* All ancestors exist. Newly created one stored at index pos.
|
|
|
*/
|
|
|
- private void unprotectedMkdir(INode[] inodes, int pos,
|
|
|
+ private void unprotectedMkdir(INodesInPath inodesInPath, int pos,
|
|
|
byte[] name, PermissionStatus permission,
|
|
|
long timestamp) throws QuotaExceededException {
|
|
|
assert hasWriteLock();
|
|
|
- inodes[pos] = addChild(inodes, pos,
|
|
|
- new INodeDirectory(name, permission, timestamp),
|
|
|
- -1);
|
|
|
+ final INodeDirectory dir = new INodeDirectory(name, permission, timestamp);
|
|
|
+ final INode inode = addChild(inodesInPath, pos, dir, -1, true);
|
|
|
+ inodesInPath.setINode(pos, inode);
|
|
|
}
|
|
|
|
|
|
/** Add a node child to the namespace. The full path name of the node is src.
|
|
|
* childDiskspace should be -1, if unknown.
|
|
|
- * QuotaExceededException is thrown if it violates quota limit */
|
|
|
- private <T extends INode> T addNode(String src, T child,
|
|
|
- long childDiskspace)
|
|
|
- throws QuotaExceededException, UnresolvedLinkException {
|
|
|
+ * @throw QuotaExceededException is thrown if it violates quota limit
|
|
|
+ */
|
|
|
+ private <T extends INode> T addNode(String src, T child, long childDiskspace
|
|
|
+ ) throws QuotaExceededException, UnresolvedLinkException {
|
|
|
byte[][] components = INode.getPathComponents(src);
|
|
|
byte[] path = components[components.length-1];
|
|
|
child.setLocalName(path);
|
|
|
cacheName(child);
|
|
|
- INode[] inodes = new INode[components.length];
|
|
|
writeLock();
|
|
|
try {
|
|
|
- rootDir.getExistingPathINodes(components, inodes, false);
|
|
|
- return addChild(inodes, inodes.length-1, child, childDiskspace);
|
|
|
+ INodesInPath inodesInPath = rootDir.getExistingPathINodes(components,
|
|
|
+ components.length, false);
|
|
|
+ return addChild(inodesInPath, inodesInPath.getINodes().length-1, child,
|
|
|
+ childDiskspace, true);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -1695,7 +1649,7 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
if (maxDirItems != 0) {
|
|
|
INodeDirectory parent = (INodeDirectory)pathComponents[pos-1];
|
|
|
- int count = parent.getChildren().size();
|
|
|
+ int count = parent.getChildrenList().size();
|
|
|
if (count >= maxDirItems) {
|
|
|
throw new MaxDirectoryItemsExceededException(maxDirItems, count);
|
|
|
}
|
|
@@ -1714,19 +1668,22 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
|
|
|
/** Add a node child to the inodes at index pos.
|
|
|
- * Its ancestors are stored at [0, pos-1].
|
|
|
- * QuotaExceededException is thrown if it violates quota limit */
|
|
|
- private <T extends INode> T addChild(INode[] pathComponents, int pos,
|
|
|
+ * Its ancestors are stored at [0, pos-1].
|
|
|
+ * @return the added node.
|
|
|
+ * @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 {
|
|
|
- // 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
|
|
|
- // to the dest, if that fails, re-adds the src from whence it came.
|
|
|
- // The rename code disables the quota when it's restoring to the
|
|
|
- // original location becase a quota violation would cause the the item
|
|
|
- // to go "poof". The fs limits must be bypassed for the same reason.
|
|
|
+ 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
|
|
|
+ // to the dest, if that fails, re-adds the src from whence it came.
|
|
|
+ // The rename code disables the quota when it's restoring to the
|
|
|
+ // original location becase a quota violation would cause the the item
|
|
|
+ // to go "poof". The fs limits must be bypassed for the same reason.
|
|
|
if (checkQuota) {
|
|
|
- verifyFsLimits(pathComponents, pos, child);
|
|
|
+ verifyFsLimits(inodes, pos, child);
|
|
|
}
|
|
|
|
|
|
INode.DirCounts counts = new INode.DirCounts();
|
|
@@ -1734,31 +1691,22 @@ public class FSDirectory implements Closeable {
|
|
|
if (childDiskspace < 0) {
|
|
|
childDiskspace = counts.getDsCount();
|
|
|
}
|
|
|
- updateCount(pathComponents, pos, counts.getNsCount(), childDiskspace,
|
|
|
- checkQuota);
|
|
|
- if (pathComponents[pos-1] == null) {
|
|
|
+ updateCount(inodesInPath, pos, counts.getNsCount(), childDiskspace, checkQuota);
|
|
|
+ if (inodes[pos-1] == null) {
|
|
|
throw new NullPointerException("Panic: parent does not exist");
|
|
|
}
|
|
|
- T addedNode = ((INodeDirectory)pathComponents[pos-1]).addChild(
|
|
|
- child, true);
|
|
|
+ final T addedNode = ((INodeDirectory)inodes[pos-1]).addChild(child, true);
|
|
|
if (addedNode == null) {
|
|
|
- updateCount(pathComponents, pos, -counts.getNsCount(),
|
|
|
- -childDiskspace, true);
|
|
|
+ updateCount(inodesInPath, pos, -counts.getNsCount(), -childDiskspace, true);
|
|
|
}
|
|
|
return addedNode;
|
|
|
}
|
|
|
-
|
|
|
- private <T extends INode> T addChild(INode[] pathComponents, int pos,
|
|
|
- T child, long childDiskspace)
|
|
|
- throws QuotaExceededException {
|
|
|
- return addChild(pathComponents, pos, child, childDiskspace, true);
|
|
|
- }
|
|
|
|
|
|
- private <T extends INode> T addChildNoQuotaCheck(INode[] pathComponents,
|
|
|
+ private <T extends INode> T addChildNoQuotaCheck(INodesInPath inodesInPath,
|
|
|
int pos, T child, long childDiskspace) {
|
|
|
T inode = null;
|
|
|
try {
|
|
|
- inode = addChild(pathComponents, pos, child, childDiskspace, false);
|
|
|
+ inode = addChild(inodesInPath, pos, child, childDiskspace, false);
|
|
|
} catch (QuotaExceededException e) {
|
|
|
NameNode.LOG.warn("FSDirectory.addChildNoQuotaCheck - unexpected", e);
|
|
|
}
|
|
@@ -1770,13 +1718,13 @@ public class FSDirectory implements Closeable {
|
|
|
* Count of each ancestor with quota is also updated.
|
|
|
* Return the removed node; null if the removal fails.
|
|
|
*/
|
|
|
- private INode removeChild(INode[] pathComponents, int pos) {
|
|
|
- INode removedNode =
|
|
|
- ((INodeDirectory)pathComponents[pos-1]).removeChild(pathComponents[pos]);
|
|
|
+ private INode removeChild(final INodesInPath inodesInPath, int pos) {
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
+ INode removedNode = ((INodeDirectory)inodes[pos-1]).removeChild(inodes[pos]);
|
|
|
if (removedNode != null) {
|
|
|
INode.DirCounts counts = new INode.DirCounts();
|
|
|
removedNode.spaceConsumedInTree(counts);
|
|
|
- updateCountNoQuotaCheck(pathComponents, pos,
|
|
|
+ updateCountNoQuotaCheck(inodesInPath, pos,
|
|
|
-counts.getNsCount(), -counts.getDsCount());
|
|
|
}
|
|
|
return removedNode;
|
|
@@ -1844,11 +1792,11 @@ public class FSDirectory implements Closeable {
|
|
|
* INode. using 'parent' is not currently recommended. */
|
|
|
nodesInPath.add(dir);
|
|
|
|
|
|
- for (INode child : dir.getChildren()) {
|
|
|
+ for (INode child : dir.getChildrenList()) {
|
|
|
if (child.isDirectory()) {
|
|
|
updateCountForINodeWithQuota((INodeDirectory)child,
|
|
|
counts, nodesInPath);
|
|
|
- } else if (child.isLink()) {
|
|
|
+ } else if (child.isSymlink()) {
|
|
|
counts.nsCount += 1;
|
|
|
} else { // reduce recursive calls
|
|
|
counts.nsCount += 1;
|
|
@@ -1911,7 +1859,8 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
String srcs = normalizePath(src);
|
|
|
|
|
|
- INode[] inodes = rootDir.getExistingPathINodes(src, true);
|
|
|
+ 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);
|
|
@@ -2077,7 +2026,7 @@ public class FSDirectory implements Closeable {
|
|
|
node.getFsPermission(),
|
|
|
node.getUserName(),
|
|
|
node.getGroupName(),
|
|
|
- node.isLink() ? ((INodeSymlink)node).getSymlink() : null,
|
|
|
+ node.isSymlink() ? ((INodeSymlink)node).getSymlink() : null,
|
|
|
path);
|
|
|
}
|
|
|
|
|
@@ -2113,7 +2062,7 @@ public class FSDirectory implements Closeable {
|
|
|
node.getFsPermission(),
|
|
|
node.getUserName(),
|
|
|
node.getGroupName(),
|
|
|
- node.isLink() ? ((INodeSymlink)node).getSymlink() : null,
|
|
|
+ node.isSymlink() ? ((INodeSymlink)node).getSymlink() : null,
|
|
|
path,
|
|
|
loc);
|
|
|
}
|
|
@@ -2145,16 +2094,13 @@ public class FSDirectory implements Closeable {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
if (newNode == null) {
|
|
|
- NameNode.stateChangeLog.info("DIR* FSDirectory.addSymlink: "
|
|
|
- +"failed to add "+path
|
|
|
- +" to the file system");
|
|
|
+ NameNode.stateChangeLog.info("DIR* addSymlink: failed to add " + path);
|
|
|
return null;
|
|
|
}
|
|
|
fsImage.getEditLog().logSymlink(path, target, modTime, modTime, newNode);
|
|
|
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* FSDirectory.addSymlink: "
|
|
|
- +path+" is added to the file system");
|
|
|
+ NameNode.stateChangeLog.debug("DIR* addSymlink: " + path + " is added");
|
|
|
}
|
|
|
return newNode;
|
|
|
}
|
|
@@ -2187,7 +2133,7 @@ public class FSDirectory implements Closeable {
|
|
|
*/
|
|
|
void cacheName(INode inode) {
|
|
|
// Name is cached only for files
|
|
|
- if (inode.isDirectory() || inode.isLink()) {
|
|
|
+ if (inode.isDirectory() || inode.isSymlink()) {
|
|
|
return;
|
|
|
}
|
|
|
ByteArray name = new ByteArray(inode.getLocalNameBytes());
|