|
@@ -151,7 +151,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FsServerDefaults;
|
|
|
import org.apache.hadoop.fs.InvalidPathException;
|
|
|
import org.apache.hadoop.fs.Options;
|
|
|
-import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.XAttr;
|
|
@@ -275,7 +274,6 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
-import org.apache.hadoop.util.ChunkedArrayList;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
@@ -2279,8 +2277,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @return chosen protocol version
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private CryptoProtocolVersion chooseProtocolVersion(EncryptionZone zone,
|
|
|
- CryptoProtocolVersion[] supportedVersions)
|
|
|
+ CryptoProtocolVersion chooseProtocolVersion(
|
|
|
+ EncryptionZone zone, CryptoProtocolVersion[] supportedVersions)
|
|
|
throws UnknownCryptoProtocolVersionException, UnresolvedLinkException,
|
|
|
SnapshotAccessControlException {
|
|
|
Preconditions.checkNotNull(zone);
|
|
@@ -2342,11 +2340,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String holder, String clientMachine, EnumSet<CreateFlag> flag,
|
|
|
boolean createParent, short replication, long blockSize,
|
|
|
CryptoProtocolVersion[] supportedVersions, boolean logRetryCache)
|
|
|
- throws AccessControlException, SafeModeException,
|
|
|
- FileAlreadyExistsException, UnresolvedLinkException,
|
|
|
- FileNotFoundException, ParentNotDirectoryException, IOException {
|
|
|
+ throws IOException {
|
|
|
|
|
|
- HdfsFileStatus status = null;
|
|
|
+ HdfsFileStatus status;
|
|
|
try {
|
|
|
status = startFileInt(src, permissions, holder, clientMachine, flag,
|
|
|
createParent, replication, blockSize, supportedVersions,
|
|
@@ -2355,54 +2351,42 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
logAuditEvent(false, "create", src);
|
|
|
throw e;
|
|
|
}
|
|
|
+ logAuditEvent(true, "create", src, null, status);
|
|
|
return status;
|
|
|
}
|
|
|
|
|
|
- private HdfsFileStatus startFileInt(final String srcArg,
|
|
|
+ private HdfsFileStatus startFileInt(final String src,
|
|
|
PermissionStatus permissions, String holder, String clientMachine,
|
|
|
EnumSet<CreateFlag> flag, boolean createParent, short replication,
|
|
|
long blockSize, CryptoProtocolVersion[] supportedVersions,
|
|
|
boolean logRetryCache)
|
|
|
- throws AccessControlException, SafeModeException,
|
|
|
- FileAlreadyExistsException, UnresolvedLinkException,
|
|
|
- FileNotFoundException, ParentNotDirectoryException, IOException {
|
|
|
- String src = srcArg;
|
|
|
+ throws IOException {
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
StringBuilder builder = new StringBuilder();
|
|
|
- builder.append("DIR* NameSystem.startFile: src=" + src
|
|
|
- + ", holder=" + holder
|
|
|
- + ", clientMachine=" + clientMachine
|
|
|
- + ", createParent=" + createParent
|
|
|
- + ", replication=" + replication
|
|
|
- + ", createFlag=" + flag.toString()
|
|
|
- + ", blockSize=" + blockSize);
|
|
|
- builder.append(", supportedVersions=");
|
|
|
- if (supportedVersions != null) {
|
|
|
- builder.append(Arrays.toString(supportedVersions));
|
|
|
- } else {
|
|
|
- builder.append("null");
|
|
|
- }
|
|
|
+ builder.append("DIR* NameSystem.startFile: src=").append(src)
|
|
|
+ .append(", holder=").append(holder)
|
|
|
+ .append(", clientMachine=").append(clientMachine)
|
|
|
+ .append(", createParent=").append(createParent)
|
|
|
+ .append(", replication=").append(replication)
|
|
|
+ .append(", createFlag=").append(flag.toString())
|
|
|
+ .append(", blockSize=").append(blockSize)
|
|
|
+ .append(", supportedVersions=")
|
|
|
+ .append(supportedVersions == null ? null : Arrays.toString
|
|
|
+ (supportedVersions));
|
|
|
NameNode.stateChangeLog.debug(builder.toString());
|
|
|
}
|
|
|
if (!DFSUtil.isValidName(src)) {
|
|
|
throw new InvalidPathException(src);
|
|
|
}
|
|
|
blockManager.verifyReplication(src, replication, clientMachine);
|
|
|
-
|
|
|
- boolean skipSync = false;
|
|
|
- HdfsFileStatus stat = null;
|
|
|
- FSPermissionChecker pc = getPermissionChecker();
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
if (blockSize < minBlockSize) {
|
|
|
throw new IOException("Specified block size is less than configured" +
|
|
|
" minimum value (" + DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY
|
|
|
+ "): " + blockSize + " < " + minBlockSize);
|
|
|
}
|
|
|
- byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
- boolean create = flag.contains(CreateFlag.CREATE);
|
|
|
- boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
|
|
|
- boolean isLazyPersist = flag.contains(CreateFlag.LAZY_PERSIST);
|
|
|
|
|
|
+ FSPermissionChecker pc = getPermissionChecker();
|
|
|
waitForLoadingFSImage();
|
|
|
|
|
|
/**
|
|
@@ -2417,245 +2401,61 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* special RetryStartFileException to ask the DFSClient to try the create
|
|
|
* again later.
|
|
|
*/
|
|
|
- CryptoProtocolVersion protocolVersion = null;
|
|
|
- CipherSuite suite = null;
|
|
|
- String ezKeyName = null;
|
|
|
- EncryptedKeyVersion edek = null;
|
|
|
+ FSDirWriteFileOp.EncryptionKeyInfo ezInfo = null;
|
|
|
|
|
|
if (provider != null) {
|
|
|
readLock();
|
|
|
try {
|
|
|
- src = dir.resolvePath(pc, src, pathComponents);
|
|
|
- INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
- // Nothing to do if the path is not within an EZ
|
|
|
- final EncryptionZone zone = dir.getEZForPath(iip);
|
|
|
- if (zone != null) {
|
|
|
- protocolVersion = chooseProtocolVersion(zone, supportedVersions);
|
|
|
- suite = zone.getSuite();
|
|
|
- ezKeyName = zone.getKeyName();
|
|
|
-
|
|
|
- Preconditions.checkNotNull(protocolVersion);
|
|
|
- Preconditions.checkNotNull(suite);
|
|
|
- Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN),
|
|
|
- "Chose an UNKNOWN CipherSuite!");
|
|
|
- Preconditions.checkNotNull(ezKeyName);
|
|
|
- }
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
+ ezInfo = FSDirWriteFileOp
|
|
|
+ .getEncryptionKeyInfo(this, pc, src, supportedVersions);
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
|
|
|
|
- Preconditions.checkState(
|
|
|
- (suite == null && ezKeyName == null) ||
|
|
|
- (suite != null && ezKeyName != null),
|
|
|
- "Both suite and ezKeyName should both be null or not null");
|
|
|
-
|
|
|
// Generate EDEK if necessary while not holding the lock
|
|
|
- edek = generateEncryptedDataEncryptionKey(ezKeyName);
|
|
|
+ if (ezInfo != null) {
|
|
|
+ ezInfo.edek = generateEncryptedDataEncryptionKey(ezInfo.ezKeyName);
|
|
|
+ }
|
|
|
EncryptionFaultInjector.getInstance().startFileAfterGenerateKey();
|
|
|
}
|
|
|
|
|
|
- // Proceed with the create, using the computed cipher suite and
|
|
|
+ boolean skipSync = false;
|
|
|
+ HdfsFileStatus stat = null;
|
|
|
+
|
|
|
+ // Proceed with the create, using the computed cipher suite and
|
|
|
// generated EDEK
|
|
|
- BlocksMapUpdateInfo toRemoveBlocks = null;
|
|
|
+ BlocksMapUpdateInfo toRemoveBlocks = new BlocksMapUpdateInfo();
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot create file" + src);
|
|
|
dir.writeLock();
|
|
|
try {
|
|
|
- src = dir.resolvePath(pc, src, pathComponents);
|
|
|
- final INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
- toRemoveBlocks = startFileInternal(
|
|
|
- pc, iip, permissions, holder,
|
|
|
- clientMachine, create, overwrite,
|
|
|
- createParent, replication, blockSize,
|
|
|
- isLazyPersist, suite, protocolVersion, edek,
|
|
|
- logRetryCache);
|
|
|
- stat = FSDirStatAndListingOp.getFileInfo(
|
|
|
- dir, src, false, FSDirectory.isReservedRawName(srcArg), true);
|
|
|
+ stat = FSDirWriteFileOp.startFile(this, pc, src, permissions, holder,
|
|
|
+ clientMachine, flag, createParent,
|
|
|
+ replication, blockSize, ezInfo,
|
|
|
+ toRemoveBlocks, logRetryCache);
|
|
|
} finally {
|
|
|
dir.writeUnlock();
|
|
|
}
|
|
|
- } catch (StandbyException se) {
|
|
|
- skipSync = true;
|
|
|
- throw se;
|
|
|
+ } catch (IOException e) {
|
|
|
+ skipSync = e instanceof StandbyException;
|
|
|
+ throw e;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
// There might be transactions logged while trying to recover the lease.
|
|
|
// They need to be sync'ed even when an exception was thrown.
|
|
|
if (!skipSync) {
|
|
|
getEditLog().logSync();
|
|
|
- if (toRemoveBlocks != null) {
|
|
|
- removeBlocks(toRemoveBlocks);
|
|
|
- toRemoveBlocks.clear();
|
|
|
- }
|
|
|
+ removeBlocks(toRemoveBlocks);
|
|
|
+ toRemoveBlocks.clear();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- logAuditEvent(true, "create", srcArg, null, stat);
|
|
|
return stat;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Create a new file or overwrite an existing file<br>
|
|
|
- *
|
|
|
- * Once the file is create the client then allocates a new block with the next
|
|
|
- * call using {@link ClientProtocol#addBlock}.
|
|
|
- * <p>
|
|
|
- * For description of parameters and exceptions thrown see
|
|
|
- * {@link ClientProtocol#create}
|
|
|
- */
|
|
|
- private BlocksMapUpdateInfo startFileInternal(FSPermissionChecker pc,
|
|
|
- INodesInPath iip, PermissionStatus permissions, String holder,
|
|
|
- String clientMachine, boolean create, boolean overwrite,
|
|
|
- boolean createParent, short replication, long blockSize,
|
|
|
- boolean isLazyPersist, CipherSuite suite, CryptoProtocolVersion version,
|
|
|
- EncryptedKeyVersion edek, boolean logRetryEntry)
|
|
|
- throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
- // Verify that the destination does not exist as a directory already.
|
|
|
- final INode inode = iip.getLastINode();
|
|
|
- final String src = iip.getPath();
|
|
|
- if (inode != null && inode.isDirectory()) {
|
|
|
- throw new FileAlreadyExistsException(src +
|
|
|
- " already exists as a directory");
|
|
|
- }
|
|
|
-
|
|
|
- final INodeFile myFile = INodeFile.valueOf(inode, src, true);
|
|
|
- if (isPermissionEnabled) {
|
|
|
- if (overwrite && myFile != null) {
|
|
|
- dir.checkPathAccess(pc, iip, FsAction.WRITE);
|
|
|
- }
|
|
|
- /*
|
|
|
- * To overwrite existing file, need to check 'w' permission
|
|
|
- * of parent (equals to ancestor in this case)
|
|
|
- */
|
|
|
- dir.checkAncestorAccess(pc, iip, FsAction.WRITE);
|
|
|
- }
|
|
|
- if (!createParent) {
|
|
|
- dir.verifyParentDir(iip, src);
|
|
|
- }
|
|
|
-
|
|
|
- FileEncryptionInfo feInfo = null;
|
|
|
-
|
|
|
- final EncryptionZone zone = dir.getEZForPath(iip);
|
|
|
- if (zone != null) {
|
|
|
- // The path is now within an EZ, but we're missing encryption parameters
|
|
|
- if (suite == null || edek == null) {
|
|
|
- throw new RetryStartFileException();
|
|
|
- }
|
|
|
- // Path is within an EZ and we have provided encryption parameters.
|
|
|
- // Make sure that the generated EDEK matches the settings of the EZ.
|
|
|
- final String ezKeyName = zone.getKeyName();
|
|
|
- if (!ezKeyName.equals(edek.getEncryptionKeyName())) {
|
|
|
- throw new RetryStartFileException();
|
|
|
- }
|
|
|
- feInfo = new FileEncryptionInfo(suite, version,
|
|
|
- edek.getEncryptedKeyVersion().getMaterial(),
|
|
|
- edek.getEncryptedKeyIv(),
|
|
|
- ezKeyName, edek.getEncryptionKeyVersionName());
|
|
|
- }
|
|
|
-
|
|
|
- try {
|
|
|
- BlocksMapUpdateInfo toRemoveBlocks = null;
|
|
|
- if (myFile == null) {
|
|
|
- if (!create) {
|
|
|
- throw new FileNotFoundException("Can't overwrite non-existent " +
|
|
|
- src + " for client " + clientMachine);
|
|
|
- }
|
|
|
- } else {
|
|
|
- if (overwrite) {
|
|
|
- toRemoveBlocks = new BlocksMapUpdateInfo();
|
|
|
- List<INode> toRemoveINodes = new ChunkedArrayList<>();
|
|
|
- List<Long> toRemoveUCFiles = new ChunkedArrayList<>();
|
|
|
- long ret = FSDirDeleteOp.delete(
|
|
|
- dir, iip, toRemoveBlocks, toRemoveINodes,
|
|
|
- toRemoveUCFiles, now());
|
|
|
- if (ret >= 0) {
|
|
|
- iip = INodesInPath.replace(iip, iip.length() - 1, null);
|
|
|
- FSDirDeleteOp.incrDeletedFileCount(ret);
|
|
|
- removeLeasesAndINodes(toRemoveUCFiles, toRemoveINodes, true);
|
|
|
- }
|
|
|
- } else {
|
|
|
- // If lease soft limit time is expired, recover the lease
|
|
|
- recoverLeaseInternal(RecoverLeaseOp.CREATE_FILE,
|
|
|
- iip, src, holder, clientMachine, false);
|
|
|
- throw new FileAlreadyExistsException(src + " for client " +
|
|
|
- clientMachine + " already exists");
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- checkFsObjectLimit();
|
|
|
- INodeFile newNode = null;
|
|
|
-
|
|
|
- // Always do an implicit mkdirs for parent directory tree.
|
|
|
- Map.Entry<INodesInPath, String> parent = FSDirMkdirOp
|
|
|
- .createAncestorDirectories(dir, iip, permissions);
|
|
|
- if (parent != null) {
|
|
|
- iip = dir.addFile(parent.getKey(), parent.getValue(), permissions,
|
|
|
- replication, blockSize, holder, clientMachine);
|
|
|
- newNode = iip != null ? iip.getLastINode().asFile() : null;
|
|
|
- }
|
|
|
-
|
|
|
- if (newNode == null) {
|
|
|
- throw new IOException("Unable to add " + src + " to namespace");
|
|
|
- }
|
|
|
- leaseManager.addLease(newNode.getFileUnderConstructionFeature()
|
|
|
- .getClientName(), newNode.getId());
|
|
|
-
|
|
|
- // Set encryption attributes if necessary
|
|
|
- if (feInfo != null) {
|
|
|
- dir.setFileEncryptionInfo(src, feInfo);
|
|
|
- newNode = dir.getInode(newNode.getId()).asFile();
|
|
|
- }
|
|
|
-
|
|
|
- setNewINodeStoragePolicy(newNode, iip, isLazyPersist);
|
|
|
-
|
|
|
- // record file record in log, record new generation stamp
|
|
|
- getEditLog().logOpenFile(src, newNode, overwrite, logRetryEntry);
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " +
|
|
|
- src + " inode " + newNode.getId() + " " + holder);
|
|
|
- }
|
|
|
- return toRemoveBlocks;
|
|
|
- } catch (IOException ie) {
|
|
|
- NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: " + src + " " +
|
|
|
- ie.getMessage());
|
|
|
- throw ie;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void setNewINodeStoragePolicy(INodeFile inode,
|
|
|
- INodesInPath iip,
|
|
|
- boolean isLazyPersist)
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- if (isLazyPersist) {
|
|
|
- BlockStoragePolicy lpPolicy =
|
|
|
- blockManager.getStoragePolicy("LAZY_PERSIST");
|
|
|
-
|
|
|
- // Set LAZY_PERSIST storage policy if the flag was passed to
|
|
|
- // CreateFile.
|
|
|
- if (lpPolicy == null) {
|
|
|
- throw new HadoopIllegalArgumentException(
|
|
|
- "The LAZY_PERSIST storage policy has been disabled " +
|
|
|
- "by the administrator.");
|
|
|
- }
|
|
|
- inode.setStoragePolicyID(lpPolicy.getId(),
|
|
|
- iip.getLatestSnapshotId());
|
|
|
- } else {
|
|
|
- BlockStoragePolicy effectivePolicy =
|
|
|
- blockManager.getStoragePolicy(inode.getStoragePolicyID());
|
|
|
-
|
|
|
- if (effectivePolicy != null &&
|
|
|
- effectivePolicy.isCopyOnCreateFile()) {
|
|
|
- // Copy effective policy from ancestor directory to current file.
|
|
|
- inode.setStoragePolicyID(effectivePolicy.getId(),
|
|
|
- iip.getLatestSnapshotId());
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Append to an existing file for append.
|
|
|
* <p>
|
|
@@ -2871,7 +2671,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- private enum RecoverLeaseOp {
|
|
|
+ enum RecoverLeaseOp {
|
|
|
CREATE_FILE,
|
|
|
APPEND_FILE,
|
|
|
TRUNCATE_FILE,
|