|
@@ -25,6 +25,9 @@ import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.permission.*;
|
|
|
+import org.apache.hadoop.metrics.MetricsRecord;
|
|
|
+import org.apache.hadoop.metrics.MetricsUtil;
|
|
|
+import org.apache.hadoop.metrics.MetricsContext;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
@@ -46,6 +49,8 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
final INodeDirectoryWithQuota rootDir;
|
|
|
FSImage fsImage;
|
|
|
private boolean ready = false;
|
|
|
+ // Metrics record
|
|
|
+ private MetricsRecord directoryMetrics = null;
|
|
|
|
|
|
/** Access an existing dfs name directory. */
|
|
|
FSDirectory(FSNamesystem ns, Configuration conf) {
|
|
@@ -60,6 +65,13 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
Integer.MAX_VALUE, -1);
|
|
|
this.fsImage = fsImage;
|
|
|
namesystem = ns;
|
|
|
+ initialize(conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void initialize(Configuration conf) {
|
|
|
+ MetricsContext metricsContext = MetricsUtil.getContext("dfs");
|
|
|
+ directoryMetrics = MetricsUtil.createRecord(metricsContext, "FSDirectory");
|
|
|
+ directoryMetrics.setTag("sessionId", conf.get("session.id"));
|
|
|
}
|
|
|
|
|
|
void loadFSImage(Collection<File> dataDirs,
|
|
@@ -73,7 +85,7 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
}
|
|
|
try {
|
|
|
if (fsImage.recoverTransitionRead(dataDirs, editsDirs, startOpt)) {
|
|
|
- fsImage.saveNamespace(true);
|
|
|
+ fsImage.saveFSImage();
|
|
|
}
|
|
|
FSEditLog editLog = fsImage.getEditLog();
|
|
|
assert editLog != null : "editLog must be initialized";
|
|
@@ -91,8 +103,8 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
}
|
|
|
|
|
|
private void incrDeletedFileCount(int count) {
|
|
|
- if (namesystem != null)
|
|
|
- NameNode.getNameNodeMetrics().numFilesDeleted.inc(count);
|
|
|
+ directoryMetrics.incrMetric("files_deleted", count);
|
|
|
+ directoryMetrics.update();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -250,8 +262,8 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
INodeFile fileNode = (INodeFile) inodes[inodes.length-1];
|
|
|
|
|
|
// check quota limits and updated space consumed
|
|
|
- updateCount(inodes, inodes.length-1, 0,
|
|
|
- fileNode.getPreferredBlockSize()*fileNode.getReplication(), true);
|
|
|
+ updateCount(inodes, inodes.length-1, 0,
|
|
|
+ fileNode.getPreferredBlockSize()*fileNode.getReplication());
|
|
|
|
|
|
// associate the new list of blocks with this file
|
|
|
namesystem.blocksMap.addINode(block, fileNode);
|
|
@@ -351,11 +363,9 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
// check the validation of the source
|
|
|
if (srcInodes[srcInodes.length-1] == null) {
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
- + "failed to rename " + src + " to " + dst
|
|
|
- + " because source does not exist");
|
|
|
+ +"failed to rename "+src+" to "+dst+ " because source does not exist");
|
|
|
return false;
|
|
|
- }
|
|
|
- if (srcInodes.length == 1) {
|
|
|
+ } else if (srcInodes.length == 1) {
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
+"failed to rename "+src+" to "+dst+ " because source is the root");
|
|
|
return false;
|
|
@@ -364,78 +374,71 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
dst += Path.SEPARATOR + new Path(src).getName();
|
|
|
}
|
|
|
|
|
|
- // check the validity of the destination
|
|
|
- if (dst.equals(src)) {
|
|
|
- return true;
|
|
|
+ // remove source
|
|
|
+ INode srcChild = null;
|
|
|
+ try {
|
|
|
+ srcChild = removeChild(srcInodes, srcInodes.length-1);
|
|
|
+ } catch (IOException e) {
|
|
|
+ // srcChild == null; go to next if statement
|
|
|
}
|
|
|
- // dst cannot be directory or a file under src
|
|
|
- if (dst.startsWith(src) &&
|
|
|
- dst.charAt(src.length()) == Path.SEPARATOR_CHAR) {
|
|
|
+ if (srcChild == null) {
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
- + "failed to rename " + src + " to " + dst
|
|
|
- + " because destination starts with src");
|
|
|
+ +"failed to rename "+src+" to "+dst+ " because the source can not be removed");
|
|
|
return false;
|
|
|
}
|
|
|
+
|
|
|
+ String srcChildName = srcChild.getLocalName();
|
|
|
|
|
|
+ // check the validity of the destination
|
|
|
+ INode dstChild = null;
|
|
|
+ QuotaExceededException failureByQuota = null;
|
|
|
+
|
|
|
byte[][] dstComponents = INode.getPathComponents(dst);
|
|
|
INode[] dstInodes = new INode[dstComponents.length];
|
|
|
rootDir.getExistingPathINodes(dstComponents, dstInodes);
|
|
|
- if (dstInodes[dstInodes.length-1] != null) {
|
|
|
+ if (dstInodes[dstInodes.length-1] != null) { //check if destination exists
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
+"failed to rename "+src+" to "+dst+
|
|
|
" because destination exists");
|
|
|
- return false;
|
|
|
- }
|
|
|
- if (dstInodes[dstInodes.length-2] == null) {
|
|
|
+ } else if (dstInodes[dstInodes.length-2] == null) { // check if its parent exists
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
+"failed to rename "+src+" to "+dst+
|
|
|
- " because destination's parent does not exist");
|
|
|
- return false;
|
|
|
+ " because destination's parent does not exists");
|
|
|
}
|
|
|
-
|
|
|
- // Ensure dst has quota to accommodate rename
|
|
|
- verifyQuotaForRename(srcInodes,dstInodes);
|
|
|
-
|
|
|
- INode dstChild = null;
|
|
|
- INode srcChild = null;
|
|
|
- String srcChildName = null;
|
|
|
- try {
|
|
|
- // remove src
|
|
|
- srcChild = removeChild(srcInodes, srcInodes.length-1);
|
|
|
- if (srcChild == null) {
|
|
|
- NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
- + "failed to rename " + src + " to " + dst
|
|
|
- + " because the source can not be removed");
|
|
|
- return false;
|
|
|
- }
|
|
|
- srcChildName = srcChild.getLocalName();
|
|
|
+ else {
|
|
|
+ // add to the destination
|
|
|
srcChild.setLocalName(dstComponents[dstInodes.length-1]);
|
|
|
-
|
|
|
- // add src to the destination
|
|
|
- dstChild = addChildNoQuotaCheck(dstInodes, dstInodes.length - 1,
|
|
|
- srcChild, -1, false);
|
|
|
- if (dstChild != null) {
|
|
|
- srcChild = null;
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: " + src
|
|
|
- + " is renamed to " + dst);
|
|
|
- }
|
|
|
- // update modification time of dst and the parent of src
|
|
|
- srcInodes[srcInodes.length-2].setModificationTime(timestamp);
|
|
|
- dstInodes[dstInodes.length-2].setModificationTime(timestamp);
|
|
|
- return true;
|
|
|
+ try {
|
|
|
+ // add it to the namespace
|
|
|
+ dstChild = addChild(dstInodes, dstInodes.length-1, srcChild, false);
|
|
|
+ } catch (QuotaExceededException qe) {
|
|
|
+ failureByQuota = qe;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (dstChild != null) {
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
+ NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
+ +src+" is renamed to "+dst);
|
|
|
}
|
|
|
- } finally {
|
|
|
- if (dstChild == null && srcChild != null) {
|
|
|
+
|
|
|
+ // update modification time of dst and the parent of src
|
|
|
+ srcInodes[srcInodes.length-2].setModificationTime(timestamp);
|
|
|
+ dstInodes[dstInodes.length-2].setModificationTime(timestamp);
|
|
|
+ return true;
|
|
|
+ } else {
|
|
|
+ NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
+ +"failed to rename "+src+" to "+dst);
|
|
|
+ try {
|
|
|
// put it back
|
|
|
srcChild.setLocalName(srcChildName);
|
|
|
- addChildNoQuotaCheck(srcInodes, srcInodes.length - 1, srcChild, -1,
|
|
|
- false);
|
|
|
+ addChild(srcInodes, srcInodes.length-1, srcChild, false);
|
|
|
+ } catch (IOException ignored) {}
|
|
|
+ if (failureByQuota != null) {
|
|
|
+ throw failureByQuota;
|
|
|
+ } else {
|
|
|
+ return false;
|
|
|
}
|
|
|
}
|
|
|
- NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
|
|
|
- +"failed to rename "+src+" to "+dst);
|
|
|
- return false;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -480,7 +483,7 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
// check disk quota
|
|
|
long dsDelta = (replication - oldReplication[0]) *
|
|
|
(fileNode.diskspaceConsumed()/oldReplication[0]);
|
|
|
- updateCount(inodes, inodes.length-1, 0, dsDelta, true);
|
|
|
+ updateCount(inodes, inodes.length-1, 0, dsDelta);
|
|
|
|
|
|
fileNode.setReplication(replication);
|
|
|
fileBlocks = fileNode.getBlocks();
|
|
@@ -557,19 +560,17 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
/**
|
|
|
* Remove the file from management, return blocks
|
|
|
*/
|
|
|
- boolean delete(String src) {
|
|
|
+ INode delete(String src) {
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: "+src);
|
|
|
}
|
|
|
waitForReady();
|
|
|
long now = FSNamesystem.now();
|
|
|
- int filesRemoved = unprotectedDelete(src, now);
|
|
|
- if (filesRemoved <= 0) {
|
|
|
- return false;
|
|
|
+ INode deletedNode = unprotectedDelete(src, now);
|
|
|
+ if (deletedNode != null) {
|
|
|
+ fsImage.getEditLog().logDelete(src, now);
|
|
|
}
|
|
|
- incrDeletedFileCount(filesRemoved);
|
|
|
- fsImage.getEditLog().logDelete(src, now);
|
|
|
- return true;
|
|
|
+ return deletedNode;
|
|
|
}
|
|
|
|
|
|
/** Return if a directory is empty or not **/
|
|
@@ -594,9 +595,9 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
* @param src a string representation of a path to an inode
|
|
|
* @param modificationTime the time the inode is removed
|
|
|
* @param deletedBlocks the place holder for the blocks to be removed
|
|
|
- * @return the number of inodes deleted; 0 if no inodes are deleted.
|
|
|
+ * @return if the deletion succeeds
|
|
|
*/
|
|
|
- int unprotectedDelete(String src, long modificationTime) {
|
|
|
+ INode unprotectedDelete(String src, long modificationTime) {
|
|
|
src = normalizePath(src);
|
|
|
|
|
|
synchronized (rootDir) {
|
|
@@ -606,12 +607,12 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
if (targetNode == null) { // non-existent src
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
|
|
|
+"failed to remove "+src+" because it does not exist");
|
|
|
- return 0;
|
|
|
+ return null;
|
|
|
} else if (inodes.length == 1) { // src is the root
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: " +
|
|
|
"failed to remove " + src +
|
|
|
" because the root is not allowed to be deleted");
|
|
|
- return 0;
|
|
|
+ return null;
|
|
|
} else {
|
|
|
try {
|
|
|
// Remove the node from the namespace
|
|
@@ -621,16 +622,17 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
// GC all the blocks underneath the node.
|
|
|
ArrayList<Block> v = new ArrayList<Block>();
|
|
|
int filesRemoved = targetNode.collectSubtreeBlocksAndClear(v);
|
|
|
+ incrDeletedFileCount(filesRemoved);
|
|
|
namesystem.removePathAndBlocks(src, v);
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
|
|
|
+src+" is removed");
|
|
|
}
|
|
|
- return filesRemoved;
|
|
|
+ return targetNode;
|
|
|
} catch (IOException e) {
|
|
|
NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: " +
|
|
|
"failed to remove " + src + " because " + e.getMessage());
|
|
|
- return 0;
|
|
|
+ return null;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -639,9 +641,19 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
/**
|
|
|
* Replaces the specified inode with the specified one.
|
|
|
*/
|
|
|
- void replaceNode(String path, INodeFile oldnode, INodeFile newnode)
|
|
|
- throws IOException {
|
|
|
+ void replaceNode(String path, INodeFile oldnode, INodeFile newnode)
|
|
|
+ throws IOException {
|
|
|
+ replaceNode(path, oldnode, newnode, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @see #replaceNode(String, INodeFile, INodeFile)
|
|
|
+ */
|
|
|
+ private void replaceNode(String path, INodeFile oldnode, INodeFile newnode,
|
|
|
+ boolean updateDiskspace) throws IOException {
|
|
|
synchronized (rootDir) {
|
|
|
+ long dsOld = oldnode.diskspaceConsumed();
|
|
|
+
|
|
|
//
|
|
|
// Remove the node from the namespace
|
|
|
//
|
|
@@ -658,6 +670,18 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
|
|
|
rootDir.addNode(path, newnode);
|
|
|
|
|
|
+ //check if disk space needs to be updated.
|
|
|
+ long dsNew = 0;
|
|
|
+ if (updateDiskspace && (dsNew = newnode.diskspaceConsumed()) != dsOld) {
|
|
|
+ try {
|
|
|
+ updateSpaceConsumed(path, 0, dsNew-dsOld);
|
|
|
+ } catch (QuotaExceededException e) {
|
|
|
+ // undo
|
|
|
+ replaceNode(path, newnode, oldnode, false);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
int index = 0;
|
|
|
for (Block b : newnode.getBlocks()) {
|
|
|
BlockInfo info = namesystem.blocksMap.addINode(b, newnode);
|
|
@@ -804,7 +828,7 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
throw new FileNotFoundException(path +
|
|
|
" does not exist under rootDir.");
|
|
|
}
|
|
|
- updateCount(inodes, len-1, nsDelta, dsDelta, true);
|
|
|
+ updateCount(inodes, len-1, nsDelta, dsDelta);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -814,11 +838,10 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
* @param numOfINodes the number of inodes to update starting from index 0
|
|
|
* @param nsDelta the delta change of namespace
|
|
|
* @param dsDelta the delta change of diskspace
|
|
|
- * @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,
|
|
|
- long nsDelta, long dsDelta, boolean checkQuota)
|
|
|
+ long nsDelta, long dsDelta)
|
|
|
throws QuotaExceededException {
|
|
|
if (!ready) {
|
|
|
//still intializing. do not check or update quotas.
|
|
@@ -827,27 +850,28 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
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);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * 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) {
|
|
|
+ // check existing components in the path
|
|
|
+ int i=0;
|
|
|
try {
|
|
|
- updateCount(inodes, numOfINodes, nsDelta, dsDelta, false);
|
|
|
+ for(; i < numOfINodes; i++) {
|
|
|
+ if (inodes[i].isQuotaSet()) { // a directory with quota
|
|
|
+ INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i];
|
|
|
+ node.updateNumItemsInTree(nsDelta, dsDelta);
|
|
|
+ }
|
|
|
+ }
|
|
|
} catch (QuotaExceededException e) {
|
|
|
- NameNode.LOG.warn("FSDirectory.updateCountNoQuotaCheck - unexpected ", e);
|
|
|
+ e.setPathName(getFullPathName(inodes, i));
|
|
|
+ // undo updates
|
|
|
+ for( ; i-- > 0; ) {
|
|
|
+ try {
|
|
|
+ if (inodes[i].isQuotaSet()) { // a directory with quota
|
|
|
+ INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i];
|
|
|
+ node.updateNumItemsInTree(-nsDelta, -dsDelta);
|
|
|
+ }
|
|
|
+ } catch (IOException ingored) {
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -907,7 +931,7 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
return false;
|
|
|
}
|
|
|
// Directory creation also count towards FilesCreated
|
|
|
- // to match count of FilesDeleted metric.
|
|
|
+ // to match count of files_deleted metric.
|
|
|
if (namesystem != null)
|
|
|
NameNode.getNameNodeMetrics().numFilesCreated.inc();
|
|
|
fsImage.getEditLog().logMkDir(cur, inodes[i]);
|
|
@@ -941,7 +965,7 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
long timestamp) throws QuotaExceededException {
|
|
|
inodes[pos] = addChild(inodes, pos,
|
|
|
new INodeDirectory(name, permission, timestamp),
|
|
|
- -1, inheritPermission );
|
|
|
+ inheritPermission );
|
|
|
}
|
|
|
|
|
|
/** Add a node child to the namespace. The full path name of the node is src.
|
|
@@ -959,127 +983,48 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
inheritPermission);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Verify quota for adding or moving a new INode with required
|
|
|
- * namespace and diskspace to a given position.
|
|
|
- *
|
|
|
- * @param inodes INodes corresponding to a path
|
|
|
- * @param pos position where a new INode will be added
|
|
|
- * @param nsDelta needed namespace
|
|
|
- * @param dsDelta needed diskspace
|
|
|
- * @param commonAncestor Last node in inodes array that is a common ancestor
|
|
|
- * for a INode that is being moved from one location to the other.
|
|
|
- * Pass null if a node is not being moved.
|
|
|
- * @throws QuotaExceededException if quota limit is exceeded.
|
|
|
- */
|
|
|
- private void verifyQuota(INode[] inodes, int pos, long nsDelta, long dsDelta,
|
|
|
- INode commonAncestor) throws QuotaExceededException {
|
|
|
- if (!ready) {
|
|
|
- // Do not check quota if edits log is still being processed
|
|
|
- return;
|
|
|
- }
|
|
|
- if (pos>inodes.length) {
|
|
|
- pos = inodes.length;
|
|
|
- }
|
|
|
- int i = pos - 1;
|
|
|
- try {
|
|
|
- // check existing components in the path
|
|
|
- for(; i >= 0; i--) {
|
|
|
- if (commonAncestor == inodes[i]) {
|
|
|
- // Moving an existing node. Stop checking for quota when common
|
|
|
- // ancestor is reached
|
|
|
- return;
|
|
|
- }
|
|
|
- if (inodes[i].isQuotaSet()) { // a directory with quota
|
|
|
- INodeDirectoryWithQuota node =(INodeDirectoryWithQuota)inodes[i];
|
|
|
- node.verifyQuota(nsDelta, dsDelta);
|
|
|
- }
|
|
|
- }
|
|
|
- } catch (QuotaExceededException e) {
|
|
|
- e.setPathName(getFullPathName(inodes, i));
|
|
|
- throw e;
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- /**
|
|
|
- * Verify quota for rename operation where srcInodes[srcInodes.length-1] moves
|
|
|
- * dstInodes[dstInodes.length-1]
|
|
|
- *
|
|
|
- * @param srcInodes directory from where node is being moved.
|
|
|
- * @param dstInodes directory to where node is moved to.
|
|
|
- * @throws QuotaExceededException if quota limit is exceeded.
|
|
|
- */
|
|
|
- private void verifyQuotaForRename(INode[] srcInodes, INode[]dstInodes)
|
|
|
- throws QuotaExceededException {
|
|
|
- if (!ready) {
|
|
|
- // Do not check quota if edits log is still being processed
|
|
|
- return;
|
|
|
- }
|
|
|
- INode srcInode = srcInodes[srcInodes.length - 1];
|
|
|
- INode commonAncestor = null;
|
|
|
- for(int i =0;srcInodes[i] == dstInodes[i]; i++) {
|
|
|
- commonAncestor = srcInodes[i];
|
|
|
- }
|
|
|
- INode.DirCounts counts = new INode.DirCounts();
|
|
|
- srcInode.spaceConsumedInTree(counts);
|
|
|
- verifyQuota(dstInodes, dstInodes.length - 1, counts.getNsCount(),
|
|
|
- counts.getDsCount(), commonAncestor);
|
|
|
+ /** 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, T child,
|
|
|
+ boolean inheritPermission) throws QuotaExceededException {
|
|
|
+ return addChild(pathComponents, pos, child, -1, inheritPermission);
|
|
|
}
|
|
|
|
|
|
/** 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,
|
|
|
- T child, long childDiskspace, boolean inheritPermission,
|
|
|
- boolean checkQuota) throws QuotaExceededException {
|
|
|
+ private <T extends INode> T addChild(INode[] pathComponents, int pos, T child,
|
|
|
+ long childDiskspace, boolean inheritPermission) throws QuotaExceededException {
|
|
|
INode.DirCounts counts = new INode.DirCounts();
|
|
|
child.spaceConsumedInTree(counts);
|
|
|
if (childDiskspace < 0) {
|
|
|
childDiskspace = counts.getDsCount();
|
|
|
}
|
|
|
- updateCount(pathComponents, pos, counts.getNsCount(), childDiskspace,
|
|
|
- checkQuota);
|
|
|
+ updateCount(pathComponents, pos, counts.getNsCount(), childDiskspace);
|
|
|
T addedNode = ((INodeDirectory)pathComponents[pos-1]).addChild(
|
|
|
child, inheritPermission);
|
|
|
if (addedNode == null) {
|
|
|
- updateCount(pathComponents, pos, -counts.getNsCount(),
|
|
|
- -childDiskspace, true);
|
|
|
+ updateCount(pathComponents, pos,
|
|
|
+ -counts.getNsCount(), -childDiskspace);
|
|
|
}
|
|
|
return addedNode;
|
|
|
}
|
|
|
-
|
|
|
- private <T extends INode> T addChild(INode[] pathComponents, int pos,
|
|
|
- T child, long childDiskspace, boolean inheritPermission)
|
|
|
- throws QuotaExceededException {
|
|
|
- return addChild(pathComponents, pos, child, childDiskspace,
|
|
|
- inheritPermission, true);
|
|
|
- }
|
|
|
-
|
|
|
- private <T extends INode> T addChildNoQuotaCheck(INode[] pathComponents,
|
|
|
- int pos, T child, long childDiskspace, boolean inheritPermission) {
|
|
|
- T inode = null;
|
|
|
- try {
|
|
|
- inode = addChild(pathComponents, pos, child, childDiskspace,
|
|
|
- inheritPermission, false);
|
|
|
- } catch (QuotaExceededException e) {
|
|
|
- NameNode.LOG.warn("FSDirectory.addChildNoQuotaCheck - unexpected", e);
|
|
|
- }
|
|
|
- return inode;
|
|
|
- }
|
|
|
|
|
|
/** Remove an inode at index pos from the namespace.
|
|
|
* Its ancestors are stored at [0, pos-1].
|
|
|
* 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) {
|
|
|
+ private INode removeChild(INode[] pathComponents, int pos)
|
|
|
+ throws QuotaExceededException {
|
|
|
INode removedNode =
|
|
|
((INodeDirectory)pathComponents[pos-1]).removeChild(pathComponents[pos]);
|
|
|
if (removedNode != null) {
|
|
|
INode.DirCounts counts = new INode.DirCounts();
|
|
|
removedNode.spaceConsumedInTree(counts);
|
|
|
- updateCountNoQuotaCheck(pathComponents, pos,
|
|
|
+ updateCount(pathComponents, pos,
|
|
|
-counts.getNsCount(), -counts.getDsCount());
|
|
|
}
|
|
|
return removedNode;
|
|
@@ -1129,6 +1074,7 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
* @param dir the root of the tree that represents the directory
|
|
|
* @param counters counters for name space and disk space
|
|
|
* @param nodesInPath INodes for the each of components in the path.
|
|
|
+ * @return the size of the tree
|
|
|
*/
|
|
|
private static void updateCountForINodeWithQuota(INodeDirectory dir,
|
|
|
INode.DirCounts counts,
|
|
@@ -1210,8 +1156,6 @@ class FSDirectory implements FSConstants, Closeable {
|
|
|
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 == FSConstants.QUOTA_RESET) {
|
|
|
- throw new IllegalArgumentException("Cannot clear namespace quota on root.");
|
|
|
} else { // a directory inode
|
|
|
INodeDirectory dirNode = (INodeDirectory)targetNode;
|
|
|
long oldNsQuota = dirNode.getNsQuota();
|