blob: f1b2ee27322a1e9127fb6be586944aea3205c6e8 [file] [log] [blame]
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.nio.charset.StandardCharsets;
import org.apache.hadoop.fs.permission.FsCreateModes;
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.AclEntry;
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.DFSUtil;
import org.apache.hadoop.hdfs.protocol.AclException;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.security.AccessControlException;
import java.io.IOException;
import java.util.List;
import static org.apache.hadoop.util.Time.now;
class FSDirMkdirOp {
static FileStatus mkdirs(FSNamesystem fsn, FSPermissionChecker pc, String src,
PermissionStatus permissions, boolean createParent) throws IOException {
FSDirectory fsd = fsn.getFSDirectory();
if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
}
fsd.writeLock();
try {
INodesInPath iip = fsd.resolvePath(pc, src, DirOp.CREATE);
final INode lastINode = iip.getLastINode();
if (lastINode != null && lastINode.isFile()) {
throw new FileAlreadyExistsException("Path is not a directory: " + src);
}
if (lastINode == null) {
if (fsd.isPermissionEnabled()) {
fsd.checkAncestorAccess(pc, iip, FsAction.WRITE);
}
if (!createParent) {
fsd.verifyParentDir(iip);
}
// 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 = createMissingDirs(fsd, iip, permissions, false);
}
return fsd.getAuditFileInfo(iip);
} finally {
fsd.writeUnlock();
}
}
static INodesInPath createMissingDirs(FSDirectory fsd, INodesInPath iip,
PermissionStatus permissions, boolean inheritPerms) throws IOException {
PermissionStatus basePerm = inheritPerms ?
iip.getExistingINodes().getLastINode().getPermissionStatus() :
permissions;
// create all missing directories along the path,
// but don't add them to the INodeMap yet
permissions = addImplicitUwx(basePerm, permissions);
INode[] missing = createPathDirectories(fsd, iip, permissions);
iip = iip.getExistingINodes();
if (missing.length == 0) {
return iip;
}
// switch the locks
fsd.getINodeMap().latchWriteLock(iip, missing);
int counter = 0;
// Add missing inodes to the INodeMap
for (INode dir : missing) {
if (counter++ == missing.length - 1) {
//Last folder in the path, use the user given permission
//For MKDIR - refers to the permission given by the user
//For create - refers to the parent directory permission.
permissions = basePerm;
}
iip = addSingleDirectory(fsd, iip, dir, permissions);
assert iip != null : "iip should not be null";
}
return iip;
}
/**
* 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 INodesInPath with all the existing and newly created
* ancestor directories created.
* Or return null if there are errors.
*/
static INodesInPath createAncestorDirectories(
FSDirectory fsd, INodesInPath iip, PermissionStatus permission)
throws IOException {
return createParentDirectories(fsd, iip, permission, true);
}
/**
* Create all ancestor directories and return the parent inodes.
*
* @param fsd FSDirectory
* @param iip inodes in path to the fs directory
* @param perm the permission of the directory. Note that all ancestors
* created along the path has implicit {@code u+wx} permissions.
* @param inheritPerms if the ancestor directories should inherit permissions
* or use the specified 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.
*/
private static INodesInPath createParentDirectories(FSDirectory fsd,
INodesInPath iip, PermissionStatus perm, boolean inheritPerms)
throws IOException {
assert fsd.hasWriteLock();
// this is the desired parent iip if the subsequent delta is 1.
INodesInPath existing = iip.getExistingINodes();
int missing = iip.length() - existing.length();
if (missing == 0) { // full path exists, return parents.
existing = iip.getParentINodesInPath();
} else if (missing > 1) { // need to create at least one ancestor dir.
FSNamesystem.LOG.error("missing = " + missing);
// Ensure that the user can traversal the path by adding implicit
// u+wx permission to all ancestor directories.
PermissionStatus basePerm = inheritPerms
? existing.getLastINode().getPermissionStatus()
: perm;
perm = addImplicitUwx(basePerm, perm);
// create all the missing directories.
final int last = iip.length() - 2;
for (int i = existing.length(); existing != null && i <= last; i++) {
byte[] component = iip.getPathComponent(i);
existing = createSingleDirectory(fsd, existing, component, perm);
if(existing == null) {
FSNamesystem.LOG.error("unprotectedMkdir returned null for "
+ iip.getPath() + " for "
+ new String(component, StandardCharsets.US_ASCII) + " i = " + i);
// Somebody already created the parent. Recalculate existing
existing = INodesInPath.resolve(fsd.getRoot(), iip.getPathComponents());
i = existing.length() - 1;
}
}
}
return existing;
}
static void mkdirForEditLog(FSDirectory fsd, long inodeId, String src,
PermissionStatus permissions, List<AclEntry> aclEntries, long timestamp)
throws QuotaExceededException, UnresolvedLinkException, AclException,
FileAlreadyExistsException, ParentNotDirectoryException,
AccessControlException {
assert fsd.hasWriteLock();
INodesInPath iip = fsd.getINodesInPath(src, DirOp.WRITE_LINK);
final byte[] localName = iip.getLastLocalName();
final INodesInPath existing = iip.getParentINodesInPath();
Preconditions.checkState(existing.getLastINode() != null);
unprotectedMkdir(fsd, inodeId, existing, localName, permissions, aclEntries,
timestamp);
}
private static INodesInPath createSingleDirectory(FSDirectory fsd,
INodesInPath existing, byte[] localName, PermissionStatus perm)
throws IOException {
assert fsd.hasWriteLock();
existing = unprotectedMkdir(fsd, fsd.allocateNewInodeId(), existing,
localName, perm, null, now());
if (existing == null) {
return null;
}
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 existing;
}
private static PermissionStatus addImplicitUwx(PermissionStatus parentPerm,
PermissionStatus perm) {
FsPermission p = parentPerm.getPermission();
FsPermission ancestorPerm;
if (p.getUnmasked() == null) {
ancestorPerm = new FsPermission(
p.getUserAction().or(FsAction.WRITE_EXECUTE),
p.getGroupAction(),
p.getOtherAction());
} else {
ancestorPerm = FsCreateModes.create(
new FsPermission(
p.getUserAction().or(FsAction.WRITE_EXECUTE),
p.getGroupAction(),
p.getOtherAction()), p.getUnmasked());
}
return new PermissionStatus(perm.getUserName(), perm.getGroupName(),
ancestorPerm);
}
/**
* create a directory at path specified by parent
*/
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);
INodesInPath iip =
fsd.addLastINode(parent, dir, permission.getPermission(), true);
if (iip != null && aclEntries != null) {
AclStorage.updateINodeAcl(dir, aclEntries, Snapshot.CURRENT_STATE_ID);
}
return iip;
}
private static INode createDirectoryINode(FSDirectory fsd,
INodesInPath parent, byte[] name, PermissionStatus permission)
throws FileAlreadyExistsException {
assert fsd.hasReadLock();
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(
fsd.allocateNewInodeId(), name, permission, now());
return dir;
}
/**
* Find-out missing iNodes for the current mkdir OP.
*/
private static INode[] createPathDirectories(FSDirectory fsd,
INodesInPath iip, PermissionStatus perm)
throws IOException {
assert fsd.hasWriteLock();
INodesInPath existing = iip.getExistingINodes();
assert existing != null : "existing should not be null";
int numMissing = iip.length() - existing.length();
if (numMissing == 0) { // full path exists
return new INode[0];
}
// create the missing directories along the path
INode[] missing = new INode[numMissing];
final int last = iip.length();
for (int i = existing.length(); i < last; i++) {
byte[] component = iip.getPathComponent(i);
missing[i - existing.length()] =
createDirectoryINode(fsd, existing, component, perm);
}
return missing;
}
private static INodesInPath addSingleDirectory(FSDirectory fsd,
INodesInPath existing, INode dir, PermissionStatus perm)
throws IOException {
assert fsd.hasWriteLock();
INodesInPath iip = fsd.addLastINode(existing, dir, perm.getPermission(), true);
if (iip == null) {
FSNamesystem.LOG.debug("somebody already created {} on path {}", dir, existing.getPath());
final INodeDirectory parent = existing.getLastINode().asDirectory();
dir = parent.getChild(dir.getLocalNameBytes(), Snapshot.CURRENT_STATE_ID);
return INodesInPath.append(existing, dir, dir.getLocalNameBytes());
}
existing = iip;
assert dir.equals(existing.getLastINode()) : "dir is not the last INode";
// Directory creation also count towards FilesCreated
// to match count of FilesDeleted metric.
NameNode.getNameNodeMetrics().incrFilesCreated();
assert dir.getPermissionStatus().getGroupName() != null :
"GroupName is null for " + existing.getPath();
String cur = existing.getPath();
fsd.getEditLog().logMkDir(cur, dir);
NameNode.stateChangeLog.debug("mkdirs: created directory {}", cur);
return existing;
}
}