|
@@ -2028,7 +2028,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (!createParent) {
|
|
|
dir.verifyParentDir(iip, link);
|
|
|
}
|
|
|
- if (!dir.isValidToCreate(link)) {
|
|
|
+ if (!dir.isValidToCreate(link, iip)) {
|
|
|
throw new IOException("failed to create link " + link
|
|
|
+" either because the filename is invalid or the file exists");
|
|
|
}
|
|
@@ -2039,7 +2039,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkFsObjectLimit();
|
|
|
|
|
|
// add symbolic link to namespace
|
|
|
- addSymlink(link, target, dirPerms, createParent, logRetryCache);
|
|
|
+ addSymlink(link, iip, target, dirPerms, createParent, logRetryCache);
|
|
|
resultingStat = getAuditFileInfo(link, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -2191,11 +2191,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.READ);
|
|
|
filename = dir.resolvePath(pc, filename, pathComponents);
|
|
|
- final INodesInPath iip = dir.getINodesInPath(filename, true);
|
|
|
+ final INodesInPath iip = dir.getINodesInPath(filename, false);
|
|
|
if (isPermissionEnabled) {
|
|
|
dir.checkTraverse(pc, iip);
|
|
|
}
|
|
|
- return dir.getPreferredBlockSize(filename);
|
|
|
+ return INodeFile.valueOf(iip.getLastINode(), filename)
|
|
|
+ .getPreferredBlockSize();
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -2491,14 +2492,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (overwrite) {
|
|
|
toRemoveBlocks = new BlocksMapUpdateInfo();
|
|
|
List<INode> toRemoveINodes = new ChunkedArrayList<INode>();
|
|
|
- long ret = dir.delete(src, toRemoveBlocks, toRemoveINodes, now());
|
|
|
+ long ret = dir.delete(iip, toRemoveBlocks, toRemoveINodes, now());
|
|
|
if (ret >= 0) {
|
|
|
incrDeletedFileCount(ret);
|
|
|
removePathAndBlocks(src, null, toRemoveINodes, true);
|
|
|
}
|
|
|
} else {
|
|
|
// If lease soft limit time is expired, recover the lease
|
|
|
- recoverLeaseInternal(myFile, src, holder, clientMachine, false);
|
|
|
+ recoverLeaseInternal(iip, src, holder, clientMachine, false);
|
|
|
throw new FileAlreadyExistsException(src + " for client " +
|
|
|
clientMachine + " already exists");
|
|
|
}
|
|
@@ -2508,10 +2509,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
INodeFile newNode = null;
|
|
|
|
|
|
// Always do an implicit mkdirs for parent directory tree.
|
|
|
- Path parent = new Path(src).getParent();
|
|
|
- if (parent != null && FSDirMkdirOp.mkdirsRecursively(dir,
|
|
|
- parent.toString(), permissions, true, now())) {
|
|
|
- newNode = dir.addFile(src, permissions, replication, blockSize,
|
|
|
+ INodesInPath parentIIP = iip.getParentINodesInPath();
|
|
|
+ if (parentIIP != null && (parentIIP = FSDirMkdirOp.mkdirsRecursively(dir,
|
|
|
+ parentIIP, permissions, true, now())) != null) {
|
|
|
+ iip = INodesInPath.append(parentIIP, newNode, iip.getLastLocalName());
|
|
|
+ newNode = dir.addFile(iip, src, permissions, replication, blockSize,
|
|
|
holder, clientMachine);
|
|
|
}
|
|
|
|
|
@@ -2621,12 +2623,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
"Cannot append to lazy persist file " + src);
|
|
|
}
|
|
|
// Opening an existing file for write - may need to recover lease.
|
|
|
- recoverLeaseInternal(myFile, src, holder, clientMachine, false);
|
|
|
+ recoverLeaseInternal(iip, src, holder, clientMachine, false);
|
|
|
|
|
|
- // recoverLeaseInternal may create a new InodeFile via
|
|
|
- // finalizeINodeFileUnderConstruction so we need to refresh
|
|
|
- // the referenced file.
|
|
|
- myFile = INodeFile.valueOf(dir.getINode(src), src, true);
|
|
|
final BlockInfo lastBlock = myFile.getLastBlock();
|
|
|
// Check that the block has at least minimum replication.
|
|
|
if(lastBlock != null && lastBlock.isComplete() &&
|
|
@@ -2634,8 +2632,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
throw new IOException("append: lastBlock=" + lastBlock +
|
|
|
" of src=" + src + " is not sufficiently replicated yet.");
|
|
|
}
|
|
|
- return prepareFileForWrite(src, myFile, holder, clientMachine, true,
|
|
|
- iip.getLatestSnapshotId(), logRetryCache);
|
|
|
+ return prepareFileForWrite(src, iip, holder, clientMachine, true,
|
|
|
+ logRetryCache);
|
|
|
} catch (IOException ie) {
|
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.append: " +ie.getMessage());
|
|
|
throw ie;
|
|
@@ -2643,11 +2641,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Replace current node with a INodeUnderConstruction.
|
|
|
+ * Convert current node to under construction.
|
|
|
* Recreate in-memory lease record.
|
|
|
*
|
|
|
* @param src path to the file
|
|
|
- * @param file existing file object
|
|
|
* @param leaseHolder identifier of the lease holder on this file
|
|
|
* @param clientMachine identifier of the client machine
|
|
|
* @param writeToEditLog whether to persist this change to the edit log
|
|
@@ -2657,26 +2654,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @throws UnresolvedLinkException
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- LocatedBlock prepareFileForWrite(String src, INodeFile file,
|
|
|
- String leaseHolder, String clientMachine,
|
|
|
- boolean writeToEditLog,
|
|
|
- int latestSnapshot, boolean logRetryCache)
|
|
|
- throws IOException {
|
|
|
- file.recordModification(latestSnapshot);
|
|
|
- final INodeFile cons = file.toUnderConstruction(leaseHolder, clientMachine);
|
|
|
+ LocatedBlock prepareFileForWrite(String src, INodesInPath iip,
|
|
|
+ String leaseHolder, String clientMachine, boolean writeToEditLog,
|
|
|
+ boolean logRetryCache) throws IOException {
|
|
|
+ final INodeFile file = iip.getLastINode().asFile();
|
|
|
+ file.recordModification(iip.getLatestSnapshotId());
|
|
|
+ file.toUnderConstruction(leaseHolder, clientMachine);
|
|
|
|
|
|
- leaseManager.addLease(cons.getFileUnderConstructionFeature()
|
|
|
- .getClientName(), src);
|
|
|
+ leaseManager.addLease(
|
|
|
+ file.getFileUnderConstructionFeature().getClientName(), src);
|
|
|
|
|
|
- LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
|
|
|
+ LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(file);
|
|
|
if (ret != null) {
|
|
|
// update the quota: use the preferred block size for UC block
|
|
|
final long diff = file.getPreferredBlockSize() - ret.getBlockSize();
|
|
|
- dir.updateSpaceConsumed(src, 0, diff * file.getBlockReplication());
|
|
|
+ dir.updateSpaceConsumed(iip, 0, diff * file.getBlockReplication());
|
|
|
}
|
|
|
|
|
|
if (writeToEditLog) {
|
|
|
- getEditLog().logOpenFile(src, cons, false, logRetryCache);
|
|
|
+ getEditLog().logOpenFile(src, file, false, logRetryCache);
|
|
|
}
|
|
|
return ret;
|
|
|
}
|
|
@@ -2716,7 +2712,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
dir.checkPathAccess(pc, iip, FsAction.WRITE);
|
|
|
}
|
|
|
|
|
|
- recoverLeaseInternal(inode, src, holder, clientMachine, true);
|
|
|
+ recoverLeaseInternal(iip, src, holder, clientMachine, true);
|
|
|
} catch (StandbyException se) {
|
|
|
skipSync = true;
|
|
|
throw se;
|
|
@@ -2731,11 +2727,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- private void recoverLeaseInternal(INodeFile fileInode,
|
|
|
+ private void recoverLeaseInternal(INodesInPath iip,
|
|
|
String src, String holder, String clientMachine, boolean force)
|
|
|
throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
- if (fileInode != null && fileInode.isUnderConstruction()) {
|
|
|
+ INodeFile file = iip.getLastINode().asFile();
|
|
|
+ if (file != null && file.isUnderConstruction()) {
|
|
|
//
|
|
|
// If the file is under construction , then it must be in our
|
|
|
// leases. Find the appropriate lease record.
|
|
@@ -2758,7 +2755,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
//
|
|
|
// Find the original holder.
|
|
|
//
|
|
|
- FileUnderConstructionFeature uc = fileInode.getFileUnderConstructionFeature();
|
|
|
+ FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
|
|
|
String clientName = uc.getClientName();
|
|
|
lease = leaseManager.getLease(clientName);
|
|
|
if (lease == null) {
|
|
@@ -2772,7 +2769,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
// close only the file src
|
|
|
LOG.info("recoverLease: " + lease + ", src=" + src +
|
|
|
" from client " + clientName);
|
|
|
- internalReleaseLease(lease, src, holder);
|
|
|
+ internalReleaseLease(lease, src, iip, holder);
|
|
|
} else {
|
|
|
assert lease.getHolder().equals(clientName) :
|
|
|
"Current lease holder " + lease.getHolder() +
|
|
@@ -2784,13 +2781,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (lease.expiredSoftLimit()) {
|
|
|
LOG.info("startFile: recover " + lease + ", src=" + src + " client "
|
|
|
+ clientName);
|
|
|
- boolean isClosed = internalReleaseLease(lease, src, null);
|
|
|
+ boolean isClosed = internalReleaseLease(lease, src, iip, null);
|
|
|
if(!isClosed)
|
|
|
throw new RecoveryInProgressException(
|
|
|
"Failed to close file " + src +
|
|
|
". Lease recovery is in progress. Try again later.");
|
|
|
} else {
|
|
|
- final BlockInfo lastBlock = fileInode.getLastBlock();
|
|
|
+ final BlockInfo lastBlock = file.getLastBlock();
|
|
|
if (lastBlock != null
|
|
|
&& lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
|
|
|
throw new RecoveryInProgressException("Recovery in progress, file ["
|
|
@@ -2822,10 +2819,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
private LastBlockWithStatus appendFileInt(final String srcArg, String holder,
|
|
|
- String clientMachine, boolean logRetryCache)
|
|
|
- throws AccessControlException, SafeModeException,
|
|
|
- FileAlreadyExistsException, FileNotFoundException,
|
|
|
- ParentNotDirectoryException, IOException {
|
|
|
+ String clientMachine, boolean logRetryCache) throws IOException {
|
|
|
String src = srcArg;
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src=" + src
|
|
@@ -2892,10 +2886,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
*/
|
|
|
LocatedBlock getAdditionalBlock(String src, long fileId, String clientName,
|
|
|
ExtendedBlock previous, Set<Node> excludedNodes,
|
|
|
- List<String> favoredNodes)
|
|
|
- throws LeaseExpiredException, NotReplicatedYetException,
|
|
|
- QuotaExceededException, SafeModeException, UnresolvedLinkException,
|
|
|
- IOException {
|
|
|
+ List<String> favoredNodes) throws IOException {
|
|
|
final long blockSize;
|
|
|
final int replication;
|
|
|
final byte storagePolicyID;
|
|
@@ -2983,7 +2974,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
|
- commitOrCompleteLastBlock(pendingFile,
|
|
|
+ commitOrCompleteLastBlock(pendingFile, fileState.iip,
|
|
|
ExtendedBlock.getLocalBlock(previous));
|
|
|
|
|
|
// allocate new block, record block locations in INode.
|
|
@@ -3023,10 +3014,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
static class FileState {
|
|
|
public final INodeFile inode;
|
|
|
public final String path;
|
|
|
+ public final INodesInPath iip;
|
|
|
|
|
|
- public FileState(INodeFile inode, String fullPath) {
|
|
|
+ public FileState(INodeFile inode, String fullPath, INodesInPath iip) {
|
|
|
this.inode = inode;
|
|
|
this.path = fullPath;
|
|
|
+ this.iip = iip;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -3046,18 +3039,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkFsObjectLimit();
|
|
|
|
|
|
Block previousBlock = ExtendedBlock.getLocalBlock(previous);
|
|
|
- INode inode;
|
|
|
+ final INode inode;
|
|
|
+ final INodesInPath iip;
|
|
|
if (fileId == INodeId.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.
|
|
|
- final INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
+ iip = dir.getINodesInPath4Write(src);
|
|
|
inode = iip.getLastINode();
|
|
|
} else {
|
|
|
// Newer clients pass the inode ID, so we can just get the inode
|
|
|
// directly.
|
|
|
inode = dir.getInode(fileId);
|
|
|
- if (inode != null) src = inode.getFullPathName();
|
|
|
+ iip = INodesInPath.fromINode(inode);
|
|
|
+ if (inode != null) {
|
|
|
+ src = iip.getPath();
|
|
|
+ }
|
|
|
}
|
|
|
final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
|
|
|
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
|
|
@@ -3117,7 +3114,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
|
|
|
((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
|
|
|
offset);
|
|
|
- return new FileState(pendingFile, src);
|
|
|
+ return new FileState(pendingFile, src, iip);
|
|
|
} else {
|
|
|
// Case 3
|
|
|
throw new IOException("Cannot allocate block in " + src + ": " +
|
|
@@ -3130,7 +3127,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (!checkFileProgress(src, pendingFile, false)) {
|
|
|
throw new NotReplicatedYetException("Not replicated yet: " + src);
|
|
|
}
|
|
|
- return new FileState(pendingFile, src);
|
|
|
+ return new FileState(pendingFile, src, iip);
|
|
|
}
|
|
|
|
|
|
LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
|
|
@@ -3208,8 +3205,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* The client would like to let go of the given block
|
|
|
*/
|
|
|
boolean abandonBlock(ExtendedBlock b, long fileId, String src, String holder)
|
|
|
- throws LeaseExpiredException, FileNotFoundException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ throws IOException {
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: " + b
|
|
|
+ "of file " + src);
|
|
@@ -3225,21 +3221,24 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
src = dir.resolvePath(pc, src, pathComponents);
|
|
|
|
|
|
final INode inode;
|
|
|
+ final INodesInPath iip;
|
|
|
if (fileId == INodeId.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.
|
|
|
- inode = dir.getINode(src);
|
|
|
+ iip = dir.getINodesInPath(src, true);
|
|
|
+ inode = iip.getLastINode();
|
|
|
} else {
|
|
|
inode = dir.getInode(fileId);
|
|
|
- if (inode != null) src = inode.getFullPathName();
|
|
|
+ iip = INodesInPath.fromINode(inode);
|
|
|
+ if (inode != null) {
|
|
|
+ src = iip.getPath();
|
|
|
+ }
|
|
|
}
|
|
|
final INodeFile file = checkLease(src, holder, inode, fileId);
|
|
|
|
|
|
- //
|
|
|
// Remove the block from the pending creates list
|
|
|
- //
|
|
|
- boolean removed = dir.removeBlock(src, file,
|
|
|
+ boolean removed = dir.removeBlock(src, iip, file,
|
|
|
ExtendedBlock.getLocalBlock(b));
|
|
|
if (!removed) {
|
|
|
return true;
|
|
@@ -3258,8 +3257,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
private INodeFile checkLease(String src, String holder, INode inode,
|
|
|
- long fileId)
|
|
|
- throws LeaseExpiredException, FileNotFoundException {
|
|
|
+ long fileId) throws LeaseExpiredException, FileNotFoundException {
|
|
|
assert hasReadLock();
|
|
|
final String ident = src + " (inode " + fileId + ")";
|
|
|
if (inode == null) {
|
|
@@ -3336,29 +3334,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return success;
|
|
|
}
|
|
|
|
|
|
- private boolean completeFileInternal(String src,
|
|
|
- String holder, Block last, long fileId) throws SafeModeException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ private boolean completeFileInternal(String src, String holder, Block last,
|
|
|
+ long fileId) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
final INodeFile pendingFile;
|
|
|
+ final INodesInPath iip;
|
|
|
+ INode inode = null;
|
|
|
try {
|
|
|
- final INode inode;
|
|
|
if (fileId == INodeId.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.
|
|
|
- final INodesInPath iip = dir.getLastINodeInPath(src);
|
|
|
- inode = iip.getINode(0);
|
|
|
+ iip = dir.getINodesInPath(src, true);
|
|
|
+ inode = iip.getLastINode();
|
|
|
} else {
|
|
|
inode = dir.getInode(fileId);
|
|
|
- if (inode != null) src = inode.getFullPathName();
|
|
|
+ iip = INodesInPath.fromINode(inode);
|
|
|
+ if (inode != null) {
|
|
|
+ src = iip.getPath();
|
|
|
+ }
|
|
|
}
|
|
|
pendingFile = checkLease(src, holder, inode, fileId);
|
|
|
} catch (LeaseExpiredException lee) {
|
|
|
- final INode inode = dir.getINode(src);
|
|
|
- if (inode != null
|
|
|
- && inode.isFile()
|
|
|
- && !inode.asFile().isUnderConstruction()) {
|
|
|
+ if (inode != null && inode.isFile() &&
|
|
|
+ !inode.asFile().isUnderConstruction()) {
|
|
|
// This could be a retry RPC - i.e the client tried to close
|
|
|
// the file, but missed the RPC response. Thus, it is trying
|
|
|
// again to close the file. If the file still exists and
|
|
@@ -3383,7 +3382,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
|
- commitOrCompleteLastBlock(pendingFile, last);
|
|
|
+ commitOrCompleteLastBlock(pendingFile, iip, last);
|
|
|
|
|
|
if (!checkFileProgress(src, pendingFile, true)) {
|
|
|
return false;
|
|
@@ -3618,7 +3617,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
long mtime = now();
|
|
|
// Unlink the target directory from directory tree
|
|
|
- long filesRemoved = dir.delete(src, collectedBlocks, removedINodes,
|
|
|
+ long filesRemoved = dir.delete(iip, collectedBlocks, removedINodes,
|
|
|
mtime);
|
|
|
if (filesRemoved < 0) {
|
|
|
return false;
|
|
@@ -3885,7 +3884,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @throws IOException if path does not exist
|
|
|
*/
|
|
|
void fsync(String src, long fileId, String clientName, long lastBlockLength)
|
|
|
- throws IOException, UnresolvedLinkException {
|
|
|
+ throws IOException {
|
|
|
NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
@@ -3933,15 +3932,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* false if block recovery has been initiated. Since the lease owner
|
|
|
* has been changed and logged, caller should call logSync().
|
|
|
*/
|
|
|
- boolean internalReleaseLease(Lease lease, String src,
|
|
|
- String recoveryLeaseHolder) throws AlreadyBeingCreatedException,
|
|
|
- IOException, UnresolvedLinkException {
|
|
|
+ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip,
|
|
|
+ String recoveryLeaseHolder) throws IOException {
|
|
|
LOG.info("Recovering " + lease + ", src=" + src);
|
|
|
assert !isInSafeMode();
|
|
|
assert hasWriteLock();
|
|
|
|
|
|
- final INodesInPath iip = dir.getLastINodeInPath(src);
|
|
|
- final INodeFile pendingFile = iip.getINode(0).asFile();
|
|
|
+ final INodeFile pendingFile = iip.getLastINode().asFile();
|
|
|
int nrBlocks = pendingFile.numBlocks();
|
|
|
BlockInfo[] blocks = pendingFile.getBlocks();
|
|
|
|
|
@@ -4070,7 +4067,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
private void commitOrCompleteLastBlock(final INodeFile fileINode,
|
|
|
- final Block commitBlock) throws IOException {
|
|
|
+ final INodesInPath iip, final Block commitBlock) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
Preconditions.checkArgument(fileINode.isUnderConstruction());
|
|
|
if (!blockManager.commitOrCompleteLastBlock(fileINode, commitBlock)) {
|
|
@@ -4081,8 +4078,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();
|
|
|
if (diff > 0) {
|
|
|
try {
|
|
|
- String path = fileINode.getFullPathName();
|
|
|
- dir.updateSpaceConsumed(path, 0, -diff*fileINode.getFileReplication());
|
|
|
+ dir.updateSpaceConsumed(iip, 0, -diff*fileINode.getFileReplication());
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("Unexpected exception while updating disk space.", e);
|
|
|
}
|
|
@@ -4090,8 +4086,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
private void finalizeINodeFileUnderConstruction(String src,
|
|
|
- INodeFile pendingFile, int latestSnapshot) throws IOException,
|
|
|
- UnresolvedLinkException {
|
|
|
+ INodeFile pendingFile, int latestSnapshot) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
|
|
|
FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature();
|
|
@@ -4103,13 +4098,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
// The file is no longer pending.
|
|
|
// Create permanent INode, update blocks. No need to replace the inode here
|
|
|
// since we just remove the uc feature from pendingFile
|
|
|
- final INodeFile newFile = pendingFile.toCompleteFile(now());
|
|
|
+ pendingFile.toCompleteFile(now());
|
|
|
|
|
|
waitForLoadingFSImage();
|
|
|
// close file and persist block allocations for this file
|
|
|
- closeFile(src, newFile);
|
|
|
+ closeFile(src, pendingFile);
|
|
|
|
|
|
- blockManager.checkReplication(newFile);
|
|
|
+ blockManager.checkReplication(pendingFile);
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@@ -4126,11 +4121,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- INodeFile inodeUC = (INodeFile) bc;
|
|
|
- String fullName = inodeUC.getName();
|
|
|
+ String fullName = bc.getName();
|
|
|
try {
|
|
|
if (fullName != null && fullName.startsWith(Path.SEPARATOR)
|
|
|
- && dir.getINode(fullName) == inodeUC) {
|
|
|
+ && dir.getINode(fullName) == bc) {
|
|
|
// If file exists in normal path then no need to look in snapshot
|
|
|
return false;
|
|
|
}
|
|
@@ -4139,7 +4133,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return false;
|
|
|
}
|
|
|
/*
|
|
|
- * 1. if bc is an instance of INodeFileUnderConstructionWithSnapshot, and
|
|
|
+ * 1. if bc is under construction and also with snapshot, and
|
|
|
* bc is not in the current fsdirectory tree, bc must represent a snapshot
|
|
|
* file.
|
|
|
* 2. if fullName is not an absolute path, bc cannot be existent in the
|
|
@@ -4153,8 +4147,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
void commitBlockSynchronization(ExtendedBlock lastblock,
|
|
|
long newgenerationstamp, long newlength,
|
|
|
boolean closeFile, boolean deleteblock, DatanodeID[] newtargets,
|
|
|
- String[] newtargetstorages)
|
|
|
- throws IOException, UnresolvedLinkException {
|
|
|
+ String[] newtargetstorages) throws IOException {
|
|
|
LOG.info("commitBlockSynchronization(lastblock=" + lastblock
|
|
|
+ ", newgenerationstamp=" + newgenerationstamp
|
|
|
+ ", newlength=" + newlength
|
|
@@ -4312,10 +4305,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
@VisibleForTesting
|
|
|
String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock)
|
|
|
throws IOException {
|
|
|
- String src = pendingFile.getFullPathName();
|
|
|
+ final INodesInPath iip = INodesInPath.fromINode(pendingFile);
|
|
|
+ final String src = iip.getPath();
|
|
|
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
|
- commitOrCompleteLastBlock(pendingFile, storedBlock);
|
|
|
+ commitOrCompleteLastBlock(pendingFile, iip, storedBlock);
|
|
|
|
|
|
//remove lease, close file
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
@@ -4515,7 +4509,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
/**
|
|
|
* Add the given symbolic link to the fs. Record it in the edits log.
|
|
|
*/
|
|
|
- private INodeSymlink addSymlink(String path, String target,
|
|
|
+ private INodeSymlink addSymlink(String path, INodesInPath iip, String target,
|
|
|
PermissionStatus dirPerms,
|
|
|
boolean createParent, boolean logRetryCache)
|
|
|
throws UnresolvedLinkException, FileAlreadyExistsException,
|
|
@@ -4524,15 +4518,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
final long modTime = now();
|
|
|
if (createParent) {
|
|
|
- final String parent = new Path(path).getParent().toString();
|
|
|
- if (!FSDirMkdirOp.mkdirsRecursively(dir, parent, dirPerms, true,
|
|
|
- modTime)) {
|
|
|
+ INodesInPath parentIIP = iip.getParentINodesInPath();
|
|
|
+ if (parentIIP == null || (parentIIP = FSDirMkdirOp.mkdirsRecursively(dir,
|
|
|
+ parentIIP, dirPerms, true, modTime)) == null) {
|
|
|
return null;
|
|
|
+ } else {
|
|
|
+ iip = INodesInPath.append(parentIIP, null, iip.getLastLocalName());
|
|
|
}
|
|
|
}
|
|
|
final String userName = dirPerms.getUserName();
|
|
|
long id = dir.allocateNewInodeId();
|
|
|
- INodeSymlink newNode = dir.addSymlink(id, path, target, modTime, modTime,
|
|
|
+ INodeSymlink newNode = dir.addSymlink(iip, id, target, modTime, modTime,
|
|
|
new PermissionStatus(userName, null, FsPermission.getDefault()));
|
|
|
if (newNode == null) {
|
|
|
NameNode.stateChangeLog.info("addSymlink: failed to add " + path);
|