|
@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
@@ -838,7 +839,8 @@ public class FSDirectory implements Closeable {
|
|
long dsDelta = (replication - oldRepl) * (fileNode.diskspaceConsumed()/oldRepl);
|
|
long dsDelta = (replication - oldRepl) * (fileNode.diskspaceConsumed()/oldRepl);
|
|
updateCount(inodesInPath, 0, dsDelta, true);
|
|
updateCount(inodesInPath, 0, dsDelta, true);
|
|
|
|
|
|
- fileNode.setFileReplication(replication, inodesInPath.getLatestSnapshot());
|
|
|
|
|
|
+ fileNode = fileNode.setFileReplication(
|
|
|
|
+ replication, inodesInPath.getLatestSnapshot());
|
|
|
|
|
|
if (oldReplication != null) {
|
|
if (oldReplication != null) {
|
|
oldReplication[0] = oldRepl;
|
|
oldReplication[0] = oldRepl;
|
|
@@ -877,7 +879,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
void setPermission(String src, FsPermission permission)
|
|
void setPermission(String src, FsPermission permission)
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
- SnapshotAccessControlException {
|
|
|
|
|
|
+ NSQuotaExceededException, SnapshotAccessControlException {
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
unprotectedSetPermission(src, permission);
|
|
unprotectedSetPermission(src, permission);
|
|
@@ -889,7 +891,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
void unprotectedSetPermission(String src, FsPermission permissions)
|
|
void unprotectedSetPermission(String src, FsPermission permissions)
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
- SnapshotAccessControlException {
|
|
|
|
|
|
+ NSQuotaExceededException, SnapshotAccessControlException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
|
|
final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
|
|
final INode inode = inodesInPath.getLastINode();
|
|
final INode inode = inodesInPath.getLastINode();
|
|
@@ -901,7 +903,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
void setOwner(String src, String username, String groupname)
|
|
void setOwner(String src, String username, String groupname)
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
- SnapshotAccessControlException {
|
|
|
|
|
|
+ NSQuotaExceededException, SnapshotAccessControlException {
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
unprotectedSetOwner(src, username, groupname);
|
|
unprotectedSetOwner(src, username, groupname);
|
|
@@ -913,7 +915,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
void unprotectedSetOwner(String src, String username, String groupname)
|
|
void unprotectedSetOwner(String src, String username, String groupname)
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
- SnapshotAccessControlException {
|
|
|
|
|
|
+ NSQuotaExceededException, SnapshotAccessControlException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
|
|
final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(src, true);
|
|
INode inode = inodesInPath.getLastINode();
|
|
INode inode = inodesInPath.getLastINode();
|
|
@@ -932,7 +934,8 @@ public class FSDirectory implements Closeable {
|
|
* Concat all the blocks from srcs to trg and delete the srcs files
|
|
* Concat all the blocks from srcs to trg and delete the srcs files
|
|
*/
|
|
*/
|
|
public void concat(String target, String [] srcs)
|
|
public void concat(String target, String [] srcs)
|
|
- throws UnresolvedLinkException, SnapshotAccessControlException {
|
|
|
|
|
|
+ throws UnresolvedLinkException, NSQuotaExceededException,
|
|
|
|
+ SnapshotAccessControlException {
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
// actual move
|
|
// actual move
|
|
@@ -956,7 +959,8 @@ public class FSDirectory implements Closeable {
|
|
* NOTE: - it does not update quota (not needed for concat)
|
|
* NOTE: - it does not update quota (not needed for concat)
|
|
*/
|
|
*/
|
|
public void unprotectedConcat(String target, String [] srcs, long timestamp)
|
|
public void unprotectedConcat(String target, String [] srcs, long timestamp)
|
|
- throws UnresolvedLinkException, SnapshotAccessControlException {
|
|
|
|
|
|
+ throws UnresolvedLinkException, NSQuotaExceededException,
|
|
|
|
+ SnapshotAccessControlException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
NameNode.stateChangeLog.debug("DIR* FSNamesystem.concat to "+target);
|
|
NameNode.stateChangeLog.debug("DIR* FSNamesystem.concat to "+target);
|
|
@@ -1093,8 +1097,8 @@ public class FSDirectory implements Closeable {
|
|
* @param mtime the time the inode is removed
|
|
* @param mtime the time the inode is removed
|
|
* @throws SnapshotAccessControlException if path is in RO snapshot
|
|
* @throws SnapshotAccessControlException if path is in RO snapshot
|
|
*/
|
|
*/
|
|
- void unprotectedDelete(String src, long mtime)
|
|
|
|
- throws UnresolvedLinkException, SnapshotAccessControlException {
|
|
|
|
|
|
+ void unprotectedDelete(String src, long mtime) throws UnresolvedLinkException,
|
|
|
|
+ NSQuotaExceededException, SnapshotAccessControlException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
|
|
|
|
@@ -1116,7 +1120,8 @@ public class FSDirectory implements Closeable {
|
|
* @return the number of inodes deleted; 0 if no inodes are deleted.
|
|
* @return the number of inodes deleted; 0 if no inodes are deleted.
|
|
*/
|
|
*/
|
|
int unprotectedDelete(INodesInPath inodesInPath,
|
|
int unprotectedDelete(INodesInPath inodesInPath,
|
|
- BlocksMapUpdateInfo collectedBlocks, long mtime) {
|
|
|
|
|
|
+ BlocksMapUpdateInfo collectedBlocks, long mtime)
|
|
|
|
+ throws NSQuotaExceededException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
|
|
|
|
// check if target node exists
|
|
// check if target node exists
|
|
@@ -1812,37 +1817,62 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Verify that filesystem limit constraints are not violated
|
|
* Verify that filesystem limit constraints are not violated
|
|
- * @throws PathComponentTooLongException child's name is too long
|
|
|
|
- * @throws MaxDirectoryItemsExceededException items per directory is exceeded
|
|
|
|
*/
|
|
*/
|
|
- protected <T extends INode> void verifyFsLimits(INode[] pathComponents,
|
|
|
|
- int pos, T child) throws FSLimitException {
|
|
|
|
- boolean includeChildName = false;
|
|
|
|
- try {
|
|
|
|
- if (maxComponentLength != 0) {
|
|
|
|
- int length = child.getLocalName().length();
|
|
|
|
- if (length > maxComponentLength) {
|
|
|
|
- includeChildName = true;
|
|
|
|
- throw new PathComponentTooLongException(maxComponentLength, length);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- if (maxDirItems != 0) {
|
|
|
|
- INodeDirectory parent = (INodeDirectory)pathComponents[pos-1];
|
|
|
|
- int count = parent.getChildrenList(null).size();
|
|
|
|
- if (count >= maxDirItems) {
|
|
|
|
- throw new MaxDirectoryItemsExceededException(maxDirItems, count);
|
|
|
|
- }
|
|
|
|
|
|
+ void verifyFsLimits(INode[] pathComponents, int pos, INode child)
|
|
|
|
+ throws FSLimitException {
|
|
|
|
+ verifyMaxComponentLength(child.getLocalName(), pathComponents, pos);
|
|
|
|
+ verifyMaxDirItems(pathComponents, pos);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Verify child's name for fs limit.
|
|
|
|
+ * @throws PathComponentTooLongException child's name is too long.
|
|
|
|
+ */
|
|
|
|
+ public void verifyMaxComponentLength(String childName,
|
|
|
|
+ Object parentPath, int pos) throws PathComponentTooLongException {
|
|
|
|
+ if (maxComponentLength == 0) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ final int length = childName.length();
|
|
|
|
+ if (length > maxComponentLength) {
|
|
|
|
+ final String p = parentPath instanceof INode[]?
|
|
|
|
+ getFullPathName((INode[])parentPath, pos - 1): (String)parentPath;
|
|
|
|
+ final PathComponentTooLongException e = new PathComponentTooLongException(
|
|
|
|
+ maxComponentLength, length, p, childName);
|
|
|
|
+ if (ready) {
|
|
|
|
+ throw e;
|
|
|
|
+ } else {
|
|
|
|
+ // Do not throw if edits log is still being processed
|
|
|
|
+ NameNode.LOG.error("FSDirectory.verifyMaxComponentLength: "
|
|
|
|
+ + e.getLocalizedMessage());
|
|
}
|
|
}
|
|
- } catch (FSLimitException e) {
|
|
|
|
- String badPath = getFullPathName(pathComponents, pos-1);
|
|
|
|
- if (includeChildName) {
|
|
|
|
- badPath += Path.SEPARATOR + child.getLocalName();
|
|
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Verify children size for fs limit.
|
|
|
|
+ * @throws MaxDirectoryItemsExceededException too many children.
|
|
|
|
+ */
|
|
|
|
+ private void verifyMaxDirItems(INode[] pathComponents, int pos)
|
|
|
|
+ throws MaxDirectoryItemsExceededException {
|
|
|
|
+ if (maxDirItems == 0) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ final INodeDirectory parent = (INodeDirectory)pathComponents[pos-1];
|
|
|
|
+ final int count = parent.getChildrenList(null).size();
|
|
|
|
+ if (count >= maxDirItems) {
|
|
|
|
+ final MaxDirectoryItemsExceededException e
|
|
|
|
+ = new MaxDirectoryItemsExceededException(maxDirItems, count);
|
|
|
|
+ if (ready) {
|
|
|
|
+ e.setPathName(getFullPathName(pathComponents, pos - 1));
|
|
|
|
+ throw e;
|
|
|
|
+ } else {
|
|
|
|
+ // Do not throw if edits log is still being processed
|
|
|
|
+ NameNode.LOG.error("FSDirectory.verifyMaxDirItems: "
|
|
|
|
+ + e.getLocalizedMessage());
|
|
}
|
|
}
|
|
- e.setPathName(badPath);
|
|
|
|
- // Do not throw if edits log is still being processed
|
|
|
|
- if (ready) throw(e);
|
|
|
|
- // log pre-existing paths that exceed limits
|
|
|
|
- NameNode.LOG.error("FSDirectory.verifyFsLimits - " + e.getLocalizedMessage());
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1900,20 +1930,22 @@ public class FSDirectory implements Closeable {
|
|
* Remove the last inode in the path from the namespace.
|
|
* Remove the last inode in the path from the namespace.
|
|
* Count of each ancestor with quota is also updated.
|
|
* 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.
|
|
|
|
+ * @throws NSQuotaExceededException
|
|
*/
|
|
*/
|
|
- private INode removeLastINode(final INodesInPath inodesInPath) {
|
|
|
|
|
|
+ private INode removeLastINode(final INodesInPath inodesInPath)
|
|
|
|
+ throws NSQuotaExceededException {
|
|
|
|
+ final Snapshot latestSnapshot = inodesInPath.getLatestSnapshot();
|
|
final INode[] inodes = inodesInPath.getINodes();
|
|
final INode[] inodes = inodesInPath.getINodes();
|
|
final int pos = inodes.length - 1;
|
|
final int pos = inodes.length - 1;
|
|
final boolean removed = ((INodeDirectory)inodes[pos-1]).removeChild(
|
|
final boolean removed = ((INodeDirectory)inodes[pos-1]).removeChild(
|
|
- inodes[pos], inodesInPath.getLatestSnapshot());
|
|
|
|
- if (removed) {
|
|
|
|
|
|
+ inodes[pos], latestSnapshot);
|
|
|
|
+ if (removed && latestSnapshot == null) {
|
|
inodesInPath.setINode(pos - 1, inodes[pos].getParent());
|
|
inodesInPath.setINode(pos - 1, inodes[pos].getParent());
|
|
final Quota.Counts counts = inodes[pos].computeQuotaUsage();
|
|
final Quota.Counts counts = inodes[pos].computeQuotaUsage();
|
|
updateCountNoQuotaCheck(inodesInPath, pos,
|
|
updateCountNoQuotaCheck(inodesInPath, pos,
|
|
-counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
|
|
-counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
|
|
- return inodes[pos];
|
|
|
|
}
|
|
}
|
|
- return null;
|
|
|
|
|
|
+ return removed? inodes[pos]: null;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1986,7 +2018,7 @@ public class FSDirectory implements Closeable {
|
|
final Snapshot latest = iip.getLatestSnapshot();
|
|
final Snapshot latest = iip.getLatestSnapshot();
|
|
if (dirNode instanceof INodeDirectoryWithQuota) {
|
|
if (dirNode instanceof INodeDirectoryWithQuota) {
|
|
// a directory with quota; so set the quota to the new value
|
|
// a directory with quota; so set the quota to the new value
|
|
- ((INodeDirectoryWithQuota)dirNode).setQuota(nsQuota, dsQuota, latest);
|
|
|
|
|
|
+ ((INodeDirectoryWithQuota)dirNode).setQuota(nsQuota, dsQuota);
|
|
if (!dirNode.isQuotaSet() && latest == null) {
|
|
if (!dirNode.isQuotaSet() && latest == null) {
|
|
// will not come here for root because root's nsQuota is always set
|
|
// will not come here for root because root's nsQuota is always set
|
|
return dirNode.replaceSelf4INodeDirectory();
|
|
return dirNode.replaceSelf4INodeDirectory();
|
|
@@ -2033,7 +2065,7 @@ public class FSDirectory implements Closeable {
|
|
* Sets the access time on the file/directory. Logs it in the transaction log.
|
|
* Sets the access time on the file/directory. Logs it in the transaction log.
|
|
*/
|
|
*/
|
|
void setTimes(String src, INode inode, long mtime, long atime, boolean force,
|
|
void setTimes(String src, INode inode, long mtime, long atime, boolean force,
|
|
- Snapshot latest) {
|
|
|
|
|
|
+ Snapshot latest) throws NSQuotaExceededException {
|
|
boolean status = false;
|
|
boolean status = false;
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
@@ -2047,7 +2079,7 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
|
|
|
|
boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force)
|
|
boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force)
|
|
- throws UnresolvedLinkException {
|
|
|
|
|
|
+ throws UnresolvedLinkException, NSQuotaExceededException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
final INodesInPath i = getLastINodeInPath(src);
|
|
final INodesInPath i = getLastINodeInPath(src);
|
|
return unprotectedSetTimes(src, i.getLastINode(), mtime, atime, force,
|
|
return unprotectedSetTimes(src, i.getLastINode(), mtime, atime, force,
|
|
@@ -2055,7 +2087,8 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
|
|
|
|
private boolean unprotectedSetTimes(String src, INode inode, long mtime,
|
|
private boolean unprotectedSetTimes(String src, INode inode, long mtime,
|
|
- long atime, boolean force, Snapshot latest) {
|
|
|
|
|
|
+ long atime, boolean force, Snapshot latest)
|
|
|
|
+ throws NSQuotaExceededException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
boolean status = false;
|
|
boolean status = false;
|
|
if (mtime != -1) {
|
|
if (mtime != -1) {
|