|
@@ -17,9 +17,10 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import org.apache.commons.io.Charsets;
|
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
|
import org.apache.hadoop.fs.InvalidPathException;
|
|
|
-import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.permission.AclEntry;
|
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
@@ -29,18 +30,19 @@ import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.AclException;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
|
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.AbstractMap;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
|
|
|
import static org.apache.hadoop.util.Time.now;
|
|
|
|
|
|
class FSDirMkdirOp {
|
|
|
- static HdfsFileStatus mkdirs(
|
|
|
- FSNamesystem fsn, String src, PermissionStatus permissions,
|
|
|
- boolean createParent) throws IOException {
|
|
|
+
|
|
|
+ static HdfsFileStatus mkdirs(FSNamesystem fsn, String src,
|
|
|
+ PermissionStatus permissions, boolean createParent) throws IOException {
|
|
|
FSDirectory fsd = fsn.getFSDirectory();
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
|
|
@@ -50,188 +52,193 @@ class FSDirMkdirOp {
|
|
|
}
|
|
|
FSPermissionChecker pc = fsd.getPermissionChecker();
|
|
|
byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
- src = fsd.resolvePath(pc, src, pathComponents);
|
|
|
- INodesInPath iip = fsd.getINodesInPath4Write(src);
|
|
|
- if (fsd.isPermissionEnabled()) {
|
|
|
- fsd.checkTraverse(pc, iip);
|
|
|
- }
|
|
|
-
|
|
|
- if (!isDirMutable(fsd, iip)) {
|
|
|
+ fsd.writeLock();
|
|
|
+ try {
|
|
|
+ src = fsd.resolvePath(pc, src, pathComponents);
|
|
|
+ INodesInPath iip = fsd.getINodesInPath4Write(src);
|
|
|
if (fsd.isPermissionEnabled()) {
|
|
|
- fsd.checkAncestorAccess(pc, iip, FsAction.WRITE);
|
|
|
+ fsd.checkTraverse(pc, iip);
|
|
|
}
|
|
|
|
|
|
- if (!createParent) {
|
|
|
- fsd.verifyParentDir(iip, src);
|
|
|
+ final INode lastINode = iip.getLastINode();
|
|
|
+ if (lastINode != null && lastINode.isFile()) {
|
|
|
+ throw new FileAlreadyExistsException("Path is not a directory: " + src);
|
|
|
}
|
|
|
|
|
|
- // validate that we have enough inodes. This is, at best, a
|
|
|
- // heuristic because the mkdirs() operation might need to
|
|
|
- // create multiple inodes.
|
|
|
- fsn.checkFsObjectLimit();
|
|
|
- iip = mkdirsRecursively(fsd, iip, permissions, false, now());
|
|
|
- if (iip == null) {
|
|
|
- throw new IOException("Failed to create directory: " + src);
|
|
|
+ INodesInPath existing = lastINode != null ? iip : iip.getExistingINodes();
|
|
|
+ if (lastINode == null) {
|
|
|
+ if (fsd.isPermissionEnabled()) {
|
|
|
+ fsd.checkAncestorAccess(pc, iip, FsAction.WRITE);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!createParent) {
|
|
|
+ fsd.verifyParentDir(iip, src);
|
|
|
+ }
|
|
|
+
|
|
|
+ // validate that we have enough inodes. This is, at best, a
|
|
|
+ // heuristic because the mkdirs() operation might need to
|
|
|
+ // create multiple inodes.
|
|
|
+ fsn.checkFsObjectLimit();
|
|
|
+
|
|
|
+ List<String> nonExisting = iip.getPath(existing.length(),
|
|
|
+ iip.length() - existing.length());
|
|
|
+ int length = nonExisting.size();
|
|
|
+ if (length > 1) {
|
|
|
+ List<String> ancestors = nonExisting.subList(0, length - 1);
|
|
|
+ // Ensure that the user can traversal the path by adding implicit
|
|
|
+ // u+wx permission to all ancestor directories
|
|
|
+ existing = createChildrenDirectories(fsd, existing, ancestors,
|
|
|
+ addImplicitUwx(permissions, permissions));
|
|
|
+ if (existing == null) {
|
|
|
+ throw new IOException("Failed to create directory: " + src);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if ((existing = createChildrenDirectories(fsd, existing,
|
|
|
+ nonExisting.subList(length - 1, length), permissions)) == null) {
|
|
|
+ throw new IOException("Failed to create directory: " + src);
|
|
|
+ }
|
|
|
}
|
|
|
+ return fsd.getAuditFileInfo(existing);
|
|
|
+ } finally {
|
|
|
+ fsd.writeUnlock();
|
|
|
}
|
|
|
- return fsd.getAuditFileInfo(iip);
|
|
|
}
|
|
|
|
|
|
- static INode unprotectedMkdir(
|
|
|
- FSDirectory fsd, long inodeId, String src,
|
|
|
- PermissionStatus permissions, List<AclEntry> aclEntries, long timestamp)
|
|
|
- throws QuotaExceededException, UnresolvedLinkException, AclException {
|
|
|
- assert fsd.hasWriteLock();
|
|
|
- byte[][] components = INode.getPathComponents(src);
|
|
|
- final INodesInPath iip = fsd.getExistingPathINodes(components);
|
|
|
- final int pos = iip.length() - 1;
|
|
|
- final INodesInPath newiip = unprotectedMkdir(fsd, inodeId, iip, pos,
|
|
|
- components[pos], permissions, aclEntries, timestamp);
|
|
|
- return newiip.getINode(pos);
|
|
|
+ /**
|
|
|
+ * For a given absolute path, create all ancestors as directories along the
|
|
|
+ * path. All ancestors inherit their parent's permission plus an implicit
|
|
|
+ * u+wx permission. This is used by create() and addSymlink() for
|
|
|
+ * implicitly creating all directories along the path.
|
|
|
+ *
|
|
|
+ * For example, path="/foo/bar/spam", "/foo" is an existing directory,
|
|
|
+ * "/foo/bar" is not existing yet, the function will create directory bar.
|
|
|
+ *
|
|
|
+ * @return a tuple which contains both the new INodesInPath (with all the
|
|
|
+ * existing and newly created directories) and the last component in the
|
|
|
+ * relative path. Or return null if there are errors.
|
|
|
+ */
|
|
|
+ static Map.Entry<INodesInPath, String> createAncestorDirectories(
|
|
|
+ FSDirectory fsd, INodesInPath iip, PermissionStatus permission)
|
|
|
+ throws IOException {
|
|
|
+ final String last = new String(iip.getLastLocalName(), Charsets.UTF_8);
|
|
|
+ INodesInPath existing = iip.getExistingINodes();
|
|
|
+ List<String> children = iip.getPath(existing.length(),
|
|
|
+ iip.length() - existing.length());
|
|
|
+ int size = children.size();
|
|
|
+ if (size > 1) { // otherwise all ancestors have been created
|
|
|
+ List<String> directories = children.subList(0, size - 1);
|
|
|
+ INode parentINode = existing.getLastINode();
|
|
|
+ // Ensure that the user can traversal the path by adding implicit
|
|
|
+ // u+wx permission to all ancestor directories
|
|
|
+ existing = createChildrenDirectories(fsd, existing, directories,
|
|
|
+ addImplicitUwx(parentINode.getPermissionStatus(), permission));
|
|
|
+ if (existing == null) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return new AbstractMap.SimpleImmutableEntry<>(existing, last);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Create a directory
|
|
|
- * If ancestor directories do not exist, automatically create them.
|
|
|
-
|
|
|
+ * Create the directory {@code parent} / {@code children} and all ancestors
|
|
|
+ * along the path.
|
|
|
+ *
|
|
|
* @param fsd FSDirectory
|
|
|
- * @param iip the INodesInPath instance containing all the existing INodes
|
|
|
- * and null elements for non-existing components in the path
|
|
|
- * @param permissions the permission of the directory
|
|
|
- * @param inheritPermission
|
|
|
- * if the permission of the directory should inherit from its parent or not.
|
|
|
- * u+wx is implicitly added to the automatically created directories,
|
|
|
- * and to the given directory if inheritPermission is true
|
|
|
- * @param now creation time
|
|
|
- * @return non-null INodesInPath instance if operation succeeds
|
|
|
- * @throws QuotaExceededException if directory creation violates
|
|
|
- * any quota limit
|
|
|
- * @throws UnresolvedLinkException if a symlink is encountered in src.
|
|
|
- * @throws SnapshotAccessControlException if path is in RO snapshot
|
|
|
+ * @param existing The INodesInPath instance containing all the existing
|
|
|
+ * ancestral INodes
|
|
|
+ * @param children The relative path from the parent towards children,
|
|
|
+ * starting with "/"
|
|
|
+ * @param perm the permission of the directory. Note that all ancestors
|
|
|
+ * created along the path has implicit {@code u+wx} permissions.
|
|
|
+ *
|
|
|
+ * @return {@link INodesInPath} which contains all inodes to the
|
|
|
+ * target directory, After the execution parentPath points to the path of
|
|
|
+ * the returned INodesInPath. The function return null if the operation has
|
|
|
+ * failed.
|
|
|
*/
|
|
|
- static INodesInPath mkdirsRecursively(FSDirectory fsd, INodesInPath iip,
|
|
|
- PermissionStatus permissions, boolean inheritPermission, long now)
|
|
|
- throws FileAlreadyExistsException, QuotaExceededException,
|
|
|
- UnresolvedLinkException, SnapshotAccessControlException,
|
|
|
- AclException {
|
|
|
- final int lastInodeIndex = iip.length() - 1;
|
|
|
- final byte[][] components = iip.getPathComponents();
|
|
|
- final String[] names = new String[components.length];
|
|
|
- for (int i = 0; i < components.length; i++) {
|
|
|
- names[i] = DFSUtil.bytes2String(components[i]);
|
|
|
- }
|
|
|
+ private static INodesInPath createChildrenDirectories(FSDirectory fsd,
|
|
|
+ INodesInPath existing, List<String> children, PermissionStatus perm)
|
|
|
+ throws IOException {
|
|
|
+ assert fsd.hasWriteLock();
|
|
|
|
|
|
- fsd.writeLock();
|
|
|
- try {
|
|
|
- if (iip.isSnapshot()) {
|
|
|
- throw new SnapshotAccessControlException(
|
|
|
- "Modification on RO snapshot is disallowed");
|
|
|
- }
|
|
|
- final int length = iip.length();
|
|
|
- // find the index of the first null in inodes[]
|
|
|
- StringBuilder pathbuilder = new StringBuilder();
|
|
|
- int i = 1;
|
|
|
- INode curNode;
|
|
|
- for(; i < length && (curNode = iip.getINode(i)) != null; i++) {
|
|
|
- pathbuilder.append(Path.SEPARATOR).append(names[i]);
|
|
|
- if (!curNode.isDirectory()) {
|
|
|
- throw new FileAlreadyExistsException("Parent path is not a directory: "
|
|
|
- + pathbuilder + " " + curNode.getLocalName());
|
|
|
- }
|
|
|
+ for (String component : children) {
|
|
|
+ existing = createSingleDirectory(fsd, existing, component, perm);
|
|
|
+ if (existing == null) {
|
|
|
+ return null;
|
|
|
}
|
|
|
+ }
|
|
|
+ return existing;
|
|
|
+ }
|
|
|
|
|
|
- // default to creating parent dirs with the given perms
|
|
|
- PermissionStatus parentPermissions = permissions;
|
|
|
-
|
|
|
- // if not inheriting and it's the last inode, there's no use in
|
|
|
- // computing perms that won't be used
|
|
|
- if (inheritPermission || (i < lastInodeIndex)) {
|
|
|
- // if inheriting (ie. creating a file or symlink), use the parent dir,
|
|
|
- // else the supplied permissions
|
|
|
- // NOTE: the permissions of the auto-created directories violate posix
|
|
|
- FsPermission parentFsPerm = inheritPermission ?
|
|
|
- iip.getINode(i-1).getFsPermission() : permissions.getPermission();
|
|
|
-
|
|
|
- // ensure that the permissions allow user write+execute
|
|
|
- if (!parentFsPerm.getUserAction().implies(FsAction.WRITE_EXECUTE)) {
|
|
|
- parentFsPerm = new FsPermission(
|
|
|
- parentFsPerm.getUserAction().or(FsAction.WRITE_EXECUTE),
|
|
|
- parentFsPerm.getGroupAction(),
|
|
|
- parentFsPerm.getOtherAction()
|
|
|
- );
|
|
|
- }
|
|
|
+ static void mkdirForEditLog(FSDirectory fsd, long inodeId, String src,
|
|
|
+ PermissionStatus permissions, List<AclEntry> aclEntries, long timestamp)
|
|
|
+ throws QuotaExceededException, UnresolvedLinkException, AclException,
|
|
|
+ FileAlreadyExistsException {
|
|
|
+ assert fsd.hasWriteLock();
|
|
|
+ INodesInPath iip = fsd.getINodesInPath(src, false);
|
|
|
+ final byte[] localName = iip.getLastLocalName();
|
|
|
+ final INodesInPath existing = iip.getParentINodesInPath();
|
|
|
+ Preconditions.checkState(existing.getLastINode() != null);
|
|
|
+ unprotectedMkdir(fsd, inodeId, existing, localName, permissions, aclEntries,
|
|
|
+ timestamp);
|
|
|
+ }
|
|
|
|
|
|
- if (!parentPermissions.getPermission().equals(parentFsPerm)) {
|
|
|
- parentPermissions = new PermissionStatus(
|
|
|
- parentPermissions.getUserName(),
|
|
|
- parentPermissions.getGroupName(),
|
|
|
- parentFsPerm
|
|
|
- );
|
|
|
- // when inheriting, use same perms for entire path
|
|
|
- if (inheritPermission) permissions = parentPermissions;
|
|
|
- }
|
|
|
- }
|
|
|
+ private static INodesInPath createSingleDirectory(FSDirectory fsd,
|
|
|
+ INodesInPath existing, String localName, PermissionStatus perm)
|
|
|
+ throws IOException {
|
|
|
+ assert fsd.hasWriteLock();
|
|
|
+ existing = unprotectedMkdir(fsd, fsd.allocateNewInodeId(), existing,
|
|
|
+ localName.getBytes(Charsets.UTF_8), perm, null, now());
|
|
|
+ if (existing == null) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
|
|
|
- // create directories beginning from the first null index
|
|
|
- for(; i < length; i++) {
|
|
|
- pathbuilder.append(Path.SEPARATOR).append(names[i]);
|
|
|
- iip = unprotectedMkdir(fsd, fsd.allocateNewInodeId(), iip, i,
|
|
|
- components[i], (i < lastInodeIndex) ? parentPermissions :
|
|
|
- permissions, null, now);
|
|
|
- if (iip.getINode(i) == null) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- // Directory creation also count towards FilesCreated
|
|
|
- // to match count of FilesDeleted metric.
|
|
|
- NameNode.getNameNodeMetrics().incrFilesCreated();
|
|
|
-
|
|
|
- final String cur = pathbuilder.toString();
|
|
|
- fsd.getEditLog().logMkDir(cur, iip.getINode(i));
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug(
|
|
|
- "mkdirs: created directory " + cur);
|
|
|
- }
|
|
|
- }
|
|
|
- } finally {
|
|
|
- fsd.writeUnlock();
|
|
|
+ final INode newNode = existing.getLastINode();
|
|
|
+ // Directory creation also count towards FilesCreated
|
|
|
+ // to match count of FilesDeleted metric.
|
|
|
+ NameNode.getNameNodeMetrics().incrFilesCreated();
|
|
|
+
|
|
|
+ String cur = existing.getPath();
|
|
|
+ fsd.getEditLog().logMkDir(cur, newNode);
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
+ NameNode.stateChangeLog.debug("mkdirs: created directory " + cur);
|
|
|
}
|
|
|
- return iip;
|
|
|
+ return existing;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Check whether the path specifies a directory
|
|
|
- * @throws SnapshotAccessControlException if path is in RO snapshot
|
|
|
- */
|
|
|
- private static boolean isDirMutable(FSDirectory fsd, INodesInPath iip)
|
|
|
- throws SnapshotAccessControlException {
|
|
|
- fsd.readLock();
|
|
|
- try {
|
|
|
- INode node = iip.getLastINode();
|
|
|
- return node != null && node.isDirectory();
|
|
|
- } finally {
|
|
|
- fsd.readUnlock();
|
|
|
- }
|
|
|
+ private static PermissionStatus addImplicitUwx(PermissionStatus parentPerm,
|
|
|
+ PermissionStatus perm) {
|
|
|
+ FsPermission p = parentPerm.getPermission();
|
|
|
+ FsPermission ancestorPerm = new FsPermission(
|
|
|
+ p.getUserAction().or(FsAction.WRITE_EXECUTE),
|
|
|
+ p.getGroupAction(),
|
|
|
+ p.getOtherAction());
|
|
|
+ return new PermissionStatus(perm.getUserName(), perm.getGroupName(),
|
|
|
+ ancestorPerm);
|
|
|
}
|
|
|
|
|
|
- /** 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.
|
|
|
+ /**
|
|
|
+ * create a directory at path specified by parent
|
|
|
*/
|
|
|
- private static INodesInPath unprotectedMkdir(
|
|
|
- FSDirectory fsd, long inodeId, INodesInPath inodesInPath, int pos,
|
|
|
- byte[] name, PermissionStatus permission, List<AclEntry> aclEntries,
|
|
|
- long timestamp)
|
|
|
- throws QuotaExceededException, AclException {
|
|
|
+ private static INodesInPath unprotectedMkdir(FSDirectory fsd, long inodeId,
|
|
|
+ INodesInPath parent, byte[] name, PermissionStatus permission,
|
|
|
+ List<AclEntry> aclEntries, long timestamp)
|
|
|
+ throws QuotaExceededException, AclException, FileAlreadyExistsException {
|
|
|
assert fsd.hasWriteLock();
|
|
|
+ assert parent.getLastINode() != null;
|
|
|
+ if (!parent.getLastINode().isDirectory()) {
|
|
|
+ throw new FileAlreadyExistsException("Parent path is not a directory: " +
|
|
|
+ parent.getPath() + " " + DFSUtil.bytes2String(name));
|
|
|
+ }
|
|
|
final INodeDirectory dir = new INodeDirectory(inodeId, name, permission,
|
|
|
timestamp);
|
|
|
- if (fsd.addChild(inodesInPath, pos, dir, true)) {
|
|
|
- if (aclEntries != null) {
|
|
|
- AclStorage.updateINodeAcl(dir, aclEntries, Snapshot.CURRENT_STATE_ID);
|
|
|
- }
|
|
|
- return INodesInPath.replace(inodesInPath, pos, dir);
|
|
|
- } else {
|
|
|
- return inodesInPath;
|
|
|
+
|
|
|
+ INodesInPath iip = fsd.addLastINode(parent, dir, true);
|
|
|
+ if (iip != null && aclEntries != null) {
|
|
|
+ AclStorage.updateINodeAcl(dir, aclEntries, Snapshot.CURRENT_STATE_ID);
|
|
|
}
|
|
|
+ return iip;
|
|
|
}
|
|
|
}
|
|
|
+
|