|
@@ -23,6 +23,7 @@ import java.io.Closeable;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.locks.Condition;
|
|
@@ -41,11 +42,12 @@ import org.apache.hadoop.fs.permission.FsAction;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
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.FSLimitException;
|
|
|
+import org.apache.hadoop.hdfs.protocol.FSLimitException.IllegalNameException;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
@@ -1355,7 +1357,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
readLock();
|
|
|
try {
|
|
|
- if (srcs.endsWith(Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR)) {
|
|
|
+ if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
|
|
|
return getSnapshotsListing(srcs, startAfter);
|
|
|
}
|
|
|
final INodesInPath inodesInPath = rootDir.getLastINodeInPath(srcs, true);
|
|
@@ -1393,10 +1395,10 @@ public class FSDirectory implements Closeable {
|
|
|
*/
|
|
|
private DirectoryListing getSnapshotsListing(String src, byte[] startAfter)
|
|
|
throws UnresolvedLinkException, IOException {
|
|
|
- assert hasReadLock();
|
|
|
- final String dotSnapshot = Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR;
|
|
|
- Preconditions.checkArgument(src.endsWith(dotSnapshot),
|
|
|
- src + " does not end with " + dotSnapshot);
|
|
|
+ Preconditions.checkState(hasReadLock());
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ src.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR),
|
|
|
+ "%s does not end with %s", src, HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR);
|
|
|
|
|
|
final String dirPath = normalizePath(src.substring(0,
|
|
|
src.length() - HdfsConstants.DOT_SNAPSHOT_DIR.length()));
|
|
@@ -1428,7 +1430,7 @@ public class FSDirectory implements Closeable {
|
|
|
String srcs = normalizePath(src);
|
|
|
readLock();
|
|
|
try {
|
|
|
- if (srcs.endsWith(Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR)) {
|
|
|
+ if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
|
|
|
return getFileInfo4DotSnapshot(srcs);
|
|
|
}
|
|
|
final INodesInPath inodesInPath = rootDir.getLastINodeInPath(srcs, resolveLink);
|
|
@@ -1442,9 +1444,9 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
private HdfsFileStatus getFileInfo4DotSnapshot(String src)
|
|
|
throws UnresolvedLinkException {
|
|
|
- final String dotSnapshot = Path.SEPARATOR + HdfsConstants.DOT_SNAPSHOT_DIR;
|
|
|
- Preconditions.checkArgument(src.endsWith(dotSnapshot),
|
|
|
- src + " does not end with " + dotSnapshot);
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ src.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR),
|
|
|
+ "%s does not end with %s", src, HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR);
|
|
|
|
|
|
final String dirPath = normalizePath(src.substring(0,
|
|
|
src.length() - HdfsConstants.DOT_SNAPSHOT_DIR.length()));
|
|
@@ -1927,38 +1929,47 @@ public class FSDirectory implements Closeable {
|
|
|
verifyQuota(dst, dstIndex, delta.get(Quota.NAMESPACE),
|
|
|
delta.get(Quota.DISKSPACE), src[i - 1]);
|
|
|
}
|
|
|
+
|
|
|
+ /** Verify if the snapshot name is legal. */
|
|
|
+ void verifySnapshotName(String snapshotName, String path)
|
|
|
+ throws PathComponentTooLongException, IllegalNameException {
|
|
|
+ final byte[] bytes = DFSUtil.string2Bytes(snapshotName);
|
|
|
+ verifyINodeName(bytes);
|
|
|
+ verifyMaxComponentLength(bytes, path, 0);
|
|
|
+ }
|
|
|
|
|
|
- /**
|
|
|
- * Verify that filesystem limit constraints are not violated
|
|
|
- */
|
|
|
- void verifyFsLimits(INode[] pathComponents, int pos, INode child)
|
|
|
- throws FSLimitException {
|
|
|
- verifyMaxComponentLength(child.getLocalName(), pathComponents, pos);
|
|
|
- verifyMaxDirItems(pathComponents, pos);
|
|
|
+ /** Verify if the inode name is legal. */
|
|
|
+ void verifyINodeName(byte[] childName) throws IllegalNameException {
|
|
|
+ if (Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, childName)) {
|
|
|
+ String s = "\"" + HdfsConstants.DOT_SNAPSHOT_DIR + "\" is a reserved name.";
|
|
|
+ if (!ready) {
|
|
|
+ s += " Please rename it before upgrade.";
|
|
|
+ }
|
|
|
+ throw new IllegalNameException(s);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* 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 {
|
|
|
+ void verifyMaxComponentLength(byte[] childName, Object parentPath, int pos)
|
|
|
+ throws PathComponentTooLongException {
|
|
|
if (maxComponentLength == 0) {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- final int length = childName.length();
|
|
|
+ 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);
|
|
|
+ maxComponentLength, length, p, DFSUtil.bytes2String(childName));
|
|
|
if (ready) {
|
|
|
throw e;
|
|
|
} else {
|
|
|
// Do not throw if edits log is still being processed
|
|
|
- NameNode.LOG.error("FSDirectory.verifyMaxComponentLength: "
|
|
|
- + e.getLocalizedMessage());
|
|
|
+ NameNode.LOG.error("ERROR in FSDirectory.verifyINodeName", e);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1967,7 +1978,7 @@ public class FSDirectory implements Closeable {
|
|
|
* Verify children size for fs limit.
|
|
|
* @throws MaxDirectoryItemsExceededException too many children.
|
|
|
*/
|
|
|
- private void verifyMaxDirItems(INode[] pathComponents, int pos)
|
|
|
+ void verifyMaxDirItems(INode[] pathComponents, int pos)
|
|
|
throws MaxDirectoryItemsExceededException {
|
|
|
if (maxDirItems == 0) {
|
|
|
return;
|
|
@@ -2015,8 +2026,11 @@ public class FSDirectory implements Closeable {
|
|
|
// 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(inodes, pos, child);
|
|
|
+ verifyMaxComponentLength(child.getLocalNameBytes(), inodes, pos);
|
|
|
+ verifyMaxDirItems(inodes, pos);
|
|
|
}
|
|
|
+ // always verify inode name
|
|
|
+ verifyINodeName(child.getLocalNameBytes());
|
|
|
|
|
|
final Quota.Counts counts = child.computeQuotaUsage();
|
|
|
updateCount(iip, pos,
|