|
@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.EncryptionZone;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
@@ -116,25 +115,10 @@ class FSDirWriteFileOp {
|
|
|
static void abandonBlock(
|
|
|
FSDirectory fsd, FSPermissionChecker pc, ExtendedBlock b, long fileId,
|
|
|
String src, String holder) throws IOException {
|
|
|
- src = fsd.resolvePath(pc, src);
|
|
|
-
|
|
|
- final INode inode;
|
|
|
- final INodesInPath iip;
|
|
|
- if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
|
|
|
- // Older clients may not have given us an inode ID to work with.
|
|
|
- // In this case, we have to try to resolve the path and hope it
|
|
|
- // hasn't changed or been deleted since the file was opened for write.
|
|
|
- iip = fsd.getINodesInPath(src, true);
|
|
|
- inode = iip.getLastINode();
|
|
|
- } else {
|
|
|
- inode = fsd.getInode(fileId);
|
|
|
- iip = INodesInPath.fromINode(inode);
|
|
|
- if (inode != null) {
|
|
|
- src = iip.getPath();
|
|
|
- }
|
|
|
- }
|
|
|
+ final INodesInPath iip = fsd.resolvePath(pc, src, fileId);
|
|
|
+ src = iip.getPath();
|
|
|
FSNamesystem fsn = fsd.getFSNamesystem();
|
|
|
- final INodeFile file = fsn.checkLease(src, holder, inode, fileId);
|
|
|
+ final INodeFile file = fsn.checkLease(iip, holder, fileId);
|
|
|
Preconditions.checkState(file.isUnderConstruction());
|
|
|
|
|
|
Block localBlock = ExtendedBlock.getLocalBlock(b);
|
|
@@ -177,8 +161,8 @@ class FSDirWriteFileOp {
|
|
|
final byte storagePolicyID;
|
|
|
String clientMachine;
|
|
|
|
|
|
- src = fsn.dir.resolvePath(pc, src);
|
|
|
- FileState fileState = analyzeFileState(fsn, src, fileId, clientName,
|
|
|
+ INodesInPath iip = fsn.dir.resolvePath(pc, src, fileId);
|
|
|
+ FileState fileState = analyzeFileState(fsn, iip, fileId, clientName,
|
|
|
previous, onRetryBlock);
|
|
|
if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
|
|
|
// This is a retry. No need to generate new locations.
|
|
@@ -228,7 +212,8 @@ class FSDirWriteFileOp {
|
|
|
// Run the full analysis again, since things could have changed
|
|
|
// while chooseTarget() was executing.
|
|
|
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
|
- FileState fileState = analyzeFileState(fsn, src, fileId, clientName,
|
|
|
+ INodesInPath iip = fsn.dir.resolvePath(null, src, fileId);
|
|
|
+ FileState fileState = analyzeFileState(fsn, iip, fileId, clientName,
|
|
|
previous, onRetryBlock);
|
|
|
final INodeFile pendingFile = fileState.inode;
|
|
|
src = fileState.path;
|
|
@@ -339,8 +324,8 @@ class FSDirWriteFileOp {
|
|
|
|
|
|
boolean isRawPath = FSDirectory.isReservedRawName(src);
|
|
|
FSDirectory fsd = fsn.getFSDirectory();
|
|
|
- src = fsd.resolvePath(pc, src);
|
|
|
- INodesInPath iip = fsd.getINodesInPath4Write(src);
|
|
|
+ INodesInPath iip = fsd.resolvePathForWrite(pc, src);
|
|
|
+ src = iip.getPath();
|
|
|
|
|
|
// Verify that the destination does not exist as a directory already.
|
|
|
final INode inode = iip.getLastINode();
|
|
@@ -449,8 +434,7 @@ class FSDirWriteFileOp {
|
|
|
CryptoProtocolVersion[] supportedVersions)
|
|
|
throws IOException {
|
|
|
FSDirectory fsd = fsn.getFSDirectory();
|
|
|
- src = fsd.resolvePath(pc, src);
|
|
|
- INodesInPath iip = fsd.getINodesInPath4Write(src);
|
|
|
+ INodesInPath iip = fsd.resolvePathForWrite(pc, src);
|
|
|
// Nothing to do if the path is not within an EZ
|
|
|
final EncryptionZone zone = FSDirEncryptionZoneOp.getEZForPath(fsd, iip);
|
|
|
if (zone == null) {
|
|
@@ -583,11 +567,11 @@ class FSDirWriteFileOp {
|
|
|
}
|
|
|
|
|
|
private static FileState analyzeFileState(
|
|
|
- FSNamesystem fsn, String src, long fileId, String clientName,
|
|
|
+ FSNamesystem fsn, INodesInPath iip, long fileId, String clientName,
|
|
|
ExtendedBlock previous, LocatedBlock[] onRetryBlock)
|
|
|
throws IOException {
|
|
|
assert fsn.hasReadLock();
|
|
|
-
|
|
|
+ String src = iip.getPath();
|
|
|
checkBlock(fsn, previous);
|
|
|
onRetryBlock[0] = null;
|
|
|
fsn.checkNameNodeSafeMode("Cannot add block to " + src);
|
|
@@ -596,24 +580,7 @@ class FSDirWriteFileOp {
|
|
|
fsn.checkFsObjectLimit();
|
|
|
|
|
|
Block previousBlock = ExtendedBlock.getLocalBlock(previous);
|
|
|
- final INode inode;
|
|
|
- final INodesInPath iip;
|
|
|
- if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
|
|
|
- // Older clients may not have given us an inode ID to work with.
|
|
|
- // In this case, we have to try to resolve the path and hope it
|
|
|
- // hasn't changed or been deleted since the file was opened for write.
|
|
|
- iip = fsn.dir.getINodesInPath4Write(src);
|
|
|
- inode = iip.getLastINode();
|
|
|
- } else {
|
|
|
- // Newer clients pass the inode ID, so we can just get the inode
|
|
|
- // directly.
|
|
|
- inode = fsn.dir.getInode(fileId);
|
|
|
- iip = INodesInPath.fromINode(inode);
|
|
|
- if (inode != null) {
|
|
|
- src = iip.getPath();
|
|
|
- }
|
|
|
- }
|
|
|
- final INodeFile file = fsn.checkLease(src, clientName, inode, fileId);
|
|
|
+ final INodeFile file = fsn.checkLease(iip, clientName, fileId);
|
|
|
BlockInfo lastBlockInFile = file.getLastBlock();
|
|
|
if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
|
|
|
// The block that the client claims is the current last block
|
|
@@ -692,8 +659,8 @@ class FSDirWriteFileOp {
|
|
|
src + " for " + holder);
|
|
|
}
|
|
|
checkBlock(fsn, last);
|
|
|
- src = fsn.dir.resolvePath(pc, src);
|
|
|
- boolean success = completeFileInternal(fsn, src, holder,
|
|
|
+ INodesInPath iip = fsn.dir.resolvePath(pc, src, fileId);
|
|
|
+ boolean success = completeFileInternal(fsn, iip, holder,
|
|
|
ExtendedBlock.getLocalBlock(last),
|
|
|
fileId);
|
|
|
if (success) {
|
|
@@ -704,27 +671,16 @@ class FSDirWriteFileOp {
|
|
|
}
|
|
|
|
|
|
private static boolean completeFileInternal(
|
|
|
- FSNamesystem fsn, String src, String holder, Block last, long fileId)
|
|
|
+ FSNamesystem fsn, INodesInPath iip,
|
|
|
+ String holder, Block last, long fileId)
|
|
|
throws IOException {
|
|
|
assert fsn.hasWriteLock();
|
|
|
+ final String src = iip.getPath();
|
|
|
final INodeFile pendingFile;
|
|
|
- final INodesInPath iip;
|
|
|
INode inode = null;
|
|
|
try {
|
|
|
- if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
|
|
|
- // Older clients may not have given us an inode ID to work with.
|
|
|
- // In this case, we have to try to resolve the path and hope it
|
|
|
- // hasn't changed or been deleted since the file was opened for write.
|
|
|
- iip = fsn.dir.getINodesInPath(src, true);
|
|
|
- inode = iip.getLastINode();
|
|
|
- } else {
|
|
|
- inode = fsn.dir.getInode(fileId);
|
|
|
- iip = INodesInPath.fromINode(inode);
|
|
|
- if (inode != null) {
|
|
|
- src = iip.getPath();
|
|
|
- }
|
|
|
- }
|
|
|
- pendingFile = fsn.checkLease(src, holder, inode, fileId);
|
|
|
+ inode = iip.getLastINode();
|
|
|
+ pendingFile = fsn.checkLease(iip, holder, fileId);
|
|
|
} catch (LeaseExpiredException lee) {
|
|
|
if (inode != null && inode.isFile() &&
|
|
|
!inode.asFile().isUnderConstruction()) {
|