|
@@ -2668,9 +2668,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkOperation(OperationCategory.READ);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
|
- final INode[] inodes = analyzeFileState(
|
|
|
- src, fileId, clientName, previous, onRetryBlock).getINodes();
|
|
|
- final INodeFile pendingFile = inodes[inodes.length - 1].asFile();
|
|
|
+ final INodeFile pendingFile = analyzeFileState(
|
|
|
+ src, fileId, clientName, previous, onRetryBlock);
|
|
|
+ src = pendingFile.getFullPathName();
|
|
|
|
|
|
if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
|
|
|
// This is a retry. Just return the last block if having locations.
|
|
@@ -2704,10 +2704,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// Run the full analysis again, since things could have changed
|
|
|
// while chooseTarget() was executing.
|
|
|
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
|
- INodesInPath inodesInPath =
|
|
|
+ final INodeFile pendingFile =
|
|
|
analyzeFileState(src, fileId, clientName, previous, onRetryBlock);
|
|
|
- INode[] inodes = inodesInPath.getINodes();
|
|
|
- final INodeFile pendingFile = inodes[inodes.length - 1].asFile();
|
|
|
|
|
|
if (onRetryBlock[0] != null) {
|
|
|
if (onRetryBlock[0].getLocations().length > 0) {
|
|
@@ -2729,6 +2727,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
// allocate new block, record block locations in INode.
|
|
|
newBlock = createNewBlock();
|
|
|
+ INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
|
|
|
saveAllocatedBlock(src, inodesInPath, newBlock, targets);
|
|
|
|
|
|
dir.persistNewBlock(src, pendingFile);
|
|
@@ -2742,7 +2741,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
return makeLocatedBlock(newBlock, targets, offset);
|
|
|
}
|
|
|
|
|
|
- INodesInPath analyzeFileState(String src,
|
|
|
+ INodeFile analyzeFileState(String src,
|
|
|
long fileId,
|
|
|
String clientName,
|
|
|
ExtendedBlock previous,
|
|
@@ -2759,9 +2758,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkFsObjectLimit();
|
|
|
|
|
|
Block previousBlock = ExtendedBlock.getLocalBlock(previous);
|
|
|
- final INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
- final INodeFile pendingFile
|
|
|
- = checkLease(src, fileId, clientName, iip.getLastINode());
|
|
|
+ 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.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();
|
|
|
+ }
|
|
|
+ final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
|
|
|
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
|
|
|
if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
|
|
|
// The block that the client claims is the current last block
|
|
@@ -2819,7 +2829,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
|
|
|
((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
|
|
|
offset);
|
|
|
- return iip;
|
|
|
+ return pendingFile;
|
|
|
} else {
|
|
|
// Case 3
|
|
|
throw new IOException("Cannot allocate block in " + src + ": " +
|
|
@@ -2832,7 +2842,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
if (!checkFileProgress(pendingFile, false)) {
|
|
|
throw new NotReplicatedYetException("Not replicated yet: " + src);
|
|
|
}
|
|
|
- return iip;
|
|
|
+ return pendingFile;
|
|
|
}
|
|
|
|
|
|
LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
|
|
@@ -2845,8 +2855,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
/** @see ClientProtocol#getAdditionalDatanode */
|
|
|
- LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
|
|
|
- final DatanodeInfo[] existings, final String[] storageIDs,
|
|
|
+ LocatedBlock getAdditionalDatanode(String src, long fileId,
|
|
|
+ final ExtendedBlock blk, final DatanodeInfo[] existings,
|
|
|
+ final String[] storageIDs,
|
|
|
final Set<Node> excludes,
|
|
|
final int numAdditionalNodes, final String clientName
|
|
|
) throws IOException {
|
|
@@ -2866,7 +2877,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
|
|
|
//check lease
|
|
|
- final INodeFile file = checkLease(src, clientName);
|
|
|
+ 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.
|
|
|
+ inode = dir.getINode(src);
|
|
|
+ } else {
|
|
|
+ inode = dir.getInode(fileId);
|
|
|
+ if (inode != null) src = inode.getFullPathName();
|
|
|
+ }
|
|
|
+ final INodeFile file = checkLease(src, clientName, inode, fileId);
|
|
|
clientnode = file.getFileUnderConstructionFeature().getClientNode();
|
|
|
preferredblocksize = file.getPreferredBlockSize();
|
|
|
|
|
@@ -2890,7 +2911,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
/**
|
|
|
* The client would like to let go of the given block
|
|
|
*/
|
|
|
- boolean abandonBlock(ExtendedBlock b, String src, String holder)
|
|
|
+ boolean abandonBlock(ExtendedBlock b, long fileId, String src, String holder)
|
|
|
throws LeaseExpiredException, FileNotFoundException,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
@@ -2902,13 +2923,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- checkNameNodeSafeMode("Cannot abandon block " + b + " for fle" + src);
|
|
|
+ checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
|
|
|
+ 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.
|
|
|
+ inode = dir.getINode(src);
|
|
|
+ } else {
|
|
|
+ inode = dir.getInode(fileId);
|
|
|
+ if (inode != null) src = inode.getFullPathName();
|
|
|
+ }
|
|
|
+ final INodeFile file = checkLease(src, holder, inode, fileId);
|
|
|
+
|
|
|
//
|
|
|
// Remove the block from the pending creates list
|
|
|
//
|
|
|
- INodeFile file = checkLease(src, holder);
|
|
|
boolean removed = dir.removeBlock(src, file,
|
|
|
ExtendedBlock.getLocalBlock(b));
|
|
|
if (!removed) {
|
|
@@ -2926,39 +2958,39 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
return true;
|
|
|
}
|
|
|
-
|
|
|
- /** make sure that we still have the lease on this file. */
|
|
|
- private INodeFile checkLease(String src, String holder)
|
|
|
- throws LeaseExpiredException, UnresolvedLinkException,
|
|
|
- FileNotFoundException {
|
|
|
- return checkLease(src, INodeId.GRANDFATHER_INODE_ID, holder,
|
|
|
- dir.getINode(src));
|
|
|
- }
|
|
|
-
|
|
|
- private INodeFile checkLease(String src, long fileId, String holder,
|
|
|
- INode inode) throws LeaseExpiredException, FileNotFoundException {
|
|
|
+
|
|
|
+ private INodeFile checkLease(String src, String holder, INode inode,
|
|
|
+ long fileId)
|
|
|
+ throws LeaseExpiredException, FileNotFoundException {
|
|
|
assert hasReadLock();
|
|
|
- if (inode == null || !inode.isFile()) {
|
|
|
+ final String ident = src + " (inode " + fileId + ")";
|
|
|
+ if (inode == null) {
|
|
|
Lease lease = leaseManager.getLease(holder);
|
|
|
throw new LeaseExpiredException(
|
|
|
- "No lease on " + src + ": File does not exist. "
|
|
|
+ "No lease on " + ident + ": File does not exist. "
|
|
|
+ (lease != null ? lease.toString()
|
|
|
: "Holder " + holder + " does not have any open files."));
|
|
|
}
|
|
|
+ if (!inode.isFile()) {
|
|
|
+ Lease lease = leaseManager.getLease(holder);
|
|
|
+ throw new LeaseExpiredException(
|
|
|
+ "No lease on " + ident + ": INode is not a regular file. "
|
|
|
+ + (lease != null ? lease.toString()
|
|
|
+ : "Holder " + holder + " does not have any open files."));
|
|
|
+ }
|
|
|
final INodeFile file = inode.asFile();
|
|
|
if (!file.isUnderConstruction()) {
|
|
|
Lease lease = leaseManager.getLease(holder);
|
|
|
throw new LeaseExpiredException(
|
|
|
- "No lease on " + src + ": File is not open for writing. "
|
|
|
+ "No lease on " + ident + ": File is not open for writing. "
|
|
|
+ (lease != null ? lease.toString()
|
|
|
: "Holder " + holder + " does not have any open files."));
|
|
|
}
|
|
|
String clientName = file.getFileUnderConstructionFeature().getClientName();
|
|
|
if (holder != null && !clientName.equals(holder)) {
|
|
|
- throw new LeaseExpiredException("Lease mismatch on " + src + " owned by "
|
|
|
- + clientName + " but is accessed by " + holder);
|
|
|
+ throw new LeaseExpiredException("Lease mismatch on " + ident +
|
|
|
+ " owned by " + clientName + " but is accessed by " + holder);
|
|
|
}
|
|
|
- INodeId.checkId(fileId, file);
|
|
|
return file;
|
|
|
}
|
|
|
|
|
@@ -3001,10 +3033,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
String holder, Block last, long fileId) throws SafeModeException,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
assert hasWriteLock();
|
|
|
- final INodesInPath iip = dir.getLastINodeInPath(src);
|
|
|
final INodeFile pendingFile;
|
|
|
try {
|
|
|
- pendingFile = checkLease(src, fileId, holder, iip.getINode(0));
|
|
|
+ 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);
|
|
|
+ } else {
|
|
|
+ inode = dir.getInode(fileId);
|
|
|
+ if (inode != null) src = inode.getFullPathName();
|
|
|
+ }
|
|
|
+ pendingFile = checkLease(src, holder, inode, fileId);
|
|
|
} catch (LeaseExpiredException lee) {
|
|
|
final INode inode = dir.getINode(src);
|
|
|
if (inode != null
|
|
@@ -3019,9 +3061,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
final Block realLastBlock = inode.asFile().getLastBlock();
|
|
|
if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
|
|
|
NameNode.stateChangeLog.info("DIR* completeFile: " +
|
|
|
- "request from " + holder + " to complete " + src +
|
|
|
- " which is already closed. But, it appears to be an RPC " +
|
|
|
- "retry. Returning success");
|
|
|
+ "request from " + holder + " to complete inode " + fileId +
|
|
|
+ "(" + src + ") which is already closed. But, it appears to be " +
|
|
|
+ "an RPC retry. Returning success");
|
|
|
return true;
|
|
|
}
|
|
|
}
|
|
@@ -3041,7 +3083,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
- iip.getLatestSnapshotId());
|
|
|
+ Snapshot.CURRENT_STATE_ID);
|
|
|
return true;
|
|
|
}
|
|
|
|
|
@@ -3682,12 +3724,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
/** Persist all metadata about this file.
|
|
|
* @param src The string representation of the path
|
|
|
+ * @param fileId The inode ID that we're fsyncing. Older clients will pass
|
|
|
+ * INodeId.GRANDFATHER_INODE_ID here.
|
|
|
* @param clientName The string representation of the client
|
|
|
* @param lastBlockLength The length of the last block
|
|
|
* under construction reported from client.
|
|
|
* @throws IOException if path does not exist
|
|
|
*/
|
|
|
- void fsync(String src, String clientName, long lastBlockLength)
|
|
|
+ void fsync(String src, long fileId, String clientName, long lastBlockLength)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -3697,7 +3741,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot fsync file " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
- INodeFile pendingFile = checkLease(src, clientName);
|
|
|
+ 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.
|
|
|
+ inode = dir.getINode(src);
|
|
|
+ } else {
|
|
|
+ inode = dir.getInode(fileId);
|
|
|
+ if (inode != null) src = inode.getFullPathName();
|
|
|
+ }
|
|
|
+ final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
|
|
|
if (lastBlockLength > 0) {
|
|
|
pendingFile.getFileUnderConstructionFeature().updateLengthOfLastBlock(
|
|
|
pendingFile, lastBlockLength);
|