|
@@ -2208,13 +2208,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
final DatanodeDescriptor clientNode =
|
|
|
blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
|
|
|
|
|
|
- INodeFileUnderConstruction newNode = dir.addFile(src, permissions,
|
|
|
- replication, blockSize, holder, clientMachine, clientNode);
|
|
|
+ INodeFile newNode = dir.addFile(src, permissions, replication, blockSize,
|
|
|
+ holder, clientMachine, clientNode);
|
|
|
if (newNode == null) {
|
|
|
throw new IOException("DIR* NameSystem.startFile: " +
|
|
|
"Unable to add file to namespace.");
|
|
|
}
|
|
|
- leaseManager.addLease(newNode.getClientName(), src);
|
|
|
+ leaseManager.addLease(newNode.getFileUnderConstructionFeature()
|
|
|
+ .getClientName(), src);
|
|
|
|
|
|
// record file record in log, record new generation stamp
|
|
|
getEditLog().logOpenFile(src, newNode, logRetryEntry);
|
|
@@ -2306,11 +2307,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
boolean writeToEditLog, Snapshot latestSnapshot, boolean logRetryCache)
|
|
|
throws IOException {
|
|
|
file = file.recordModification(latestSnapshot, dir.getINodeMap());
|
|
|
- final INodeFileUnderConstruction cons = file.toUnderConstruction(
|
|
|
- leaseHolder, clientMachine, clientNode);
|
|
|
+ final INodeFile cons = file.toUnderConstruction(leaseHolder, clientMachine,
|
|
|
+ clientNode);
|
|
|
|
|
|
- dir.replaceINodeFile(src, file, cons);
|
|
|
- leaseManager.addLease(cons.getClientName(), src);
|
|
|
+ leaseManager.addLease(cons.getFileUnderConstructionFeature()
|
|
|
+ .getClientName(), src);
|
|
|
|
|
|
LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
|
|
|
if (writeToEditLog) {
|
|
@@ -2373,7 +2374,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
if (fileInode != null && fileInode.isUnderConstruction()) {
|
|
|
- INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction) fileInode;
|
|
|
//
|
|
|
// If the file is under construction , then it must be in our
|
|
|
// leases. Find the appropriate lease record.
|
|
@@ -2396,7 +2396,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
//
|
|
|
// Find the original holder.
|
|
|
//
|
|
|
- lease = leaseManager.getLease(pendingFile.getClientName());
|
|
|
+ FileUnderConstructionFeature uc = fileInode.getFileUnderConstructionFeature();
|
|
|
+ String clientName = uc.getClientName();
|
|
|
+ lease = leaseManager.getLease(clientName);
|
|
|
if (lease == null) {
|
|
|
throw new AlreadyBeingCreatedException(
|
|
|
"failed to create file " + src + " for " + holder +
|
|
@@ -2407,26 +2409,26 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// close now: no need to wait for soft lease expiration and
|
|
|
// close only the file src
|
|
|
LOG.info("recoverLease: " + lease + ", src=" + src +
|
|
|
- " from client " + pendingFile.getClientName());
|
|
|
+ " from client " + clientName);
|
|
|
internalReleaseLease(lease, src, holder);
|
|
|
} else {
|
|
|
- assert lease.getHolder().equals(pendingFile.getClientName()) :
|
|
|
+ assert lease.getHolder().equals(clientName) :
|
|
|
"Current lease holder " + lease.getHolder() +
|
|
|
- " does not match file creator " + pendingFile.getClientName();
|
|
|
+ " does not match file creator " + clientName;
|
|
|
//
|
|
|
// If the original holder has not renewed in the last SOFTLIMIT
|
|
|
// period, then start lease recovery.
|
|
|
//
|
|
|
if (lease.expiredSoftLimit()) {
|
|
|
LOG.info("startFile: recover " + lease + ", src=" + src + " client "
|
|
|
- + pendingFile.getClientName());
|
|
|
+ + clientName);
|
|
|
boolean isClosed = internalReleaseLease(lease, src, null);
|
|
|
if(!isClosed)
|
|
|
throw new RecoveryInProgressException(
|
|
|
"Failed to close file " + src +
|
|
|
". Lease recovery is in progress. Try again later.");
|
|
|
} else {
|
|
|
- final BlockInfo lastBlock = pendingFile.getLastBlock();
|
|
|
+ final BlockInfo lastBlock = fileInode.getLastBlock();
|
|
|
if (lastBlock != null
|
|
|
&& lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
|
|
|
throw new RecoveryInProgressException("Recovery in progress, file ["
|
|
@@ -2435,8 +2437,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throw new AlreadyBeingCreatedException("Failed to create file ["
|
|
|
+ src + "] for [" + holder + "] on client [" + clientMachine
|
|
|
+ "], because this file is already being created by ["
|
|
|
- + pendingFile.getClientName() + "] on ["
|
|
|
- + pendingFile.getClientMachine() + "]");
|
|
|
+ + clientName + "] on ["
|
|
|
+ + uc.getClientMachine() + "]");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -2566,8 +2568,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
|
final INode[] inodes = analyzeFileState(
|
|
|
src, fileId, clientName, previous, onRetryBlock).getINodes();
|
|
|
- final INodeFileUnderConstruction pendingFile =
|
|
|
- (INodeFileUnderConstruction) inodes[inodes.length - 1].asFile();
|
|
|
+ final INodeFile pendingFile = inodes[inodes.length - 1].asFile();
|
|
|
|
|
|
if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
|
|
|
// This is a retry. Just return the last block if having locations.
|
|
@@ -2580,7 +2581,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
+ maxBlocksPerFile);
|
|
|
}
|
|
|
blockSize = pendingFile.getPreferredBlockSize();
|
|
|
- clientNode = pendingFile.getClientNode();
|
|
|
+ clientNode = pendingFile.getFileUnderConstructionFeature().getClientNode();
|
|
|
replication = pendingFile.getFileReplication();
|
|
|
} finally {
|
|
|
readUnlock();
|
|
@@ -2604,8 +2605,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
INodesInPath inodesInPath =
|
|
|
analyzeFileState(src, fileId, clientName, previous, onRetryBlock);
|
|
|
INode[] inodes = inodesInPath.getINodes();
|
|
|
- final INodeFileUnderConstruction pendingFile =
|
|
|
- (INodeFileUnderConstruction) inodes[inodes.length - 1].asFile();
|
|
|
+ final INodeFile pendingFile = inodes[inodes.length - 1].asFile();
|
|
|
|
|
|
if (onRetryBlock[0] != null) {
|
|
|
if (onRetryBlock[0].getLocations().length > 0) {
|
|
@@ -2660,7 +2660,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
Block previousBlock = ExtendedBlock.getLocalBlock(previous);
|
|
|
final INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
- final INodeFileUnderConstruction pendingFile
|
|
|
+ final INodeFile pendingFile
|
|
|
= checkLease(src, fileId, clientName, iip.getLastINode());
|
|
|
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
|
|
|
if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
|
|
@@ -2766,8 +2766,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
|
|
|
//check lease
|
|
|
- final INodeFileUnderConstruction file = checkLease(src, clientName);
|
|
|
- clientnode = file.getClientNode();
|
|
|
+ final INodeFile file = checkLease(src, clientName);
|
|
|
+ clientnode = file.getFileUnderConstructionFeature().getClientNode();
|
|
|
preferredblocksize = file.getPreferredBlockSize();
|
|
|
|
|
|
//find datanode descriptors
|
|
@@ -2813,7 +2813,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
//
|
|
|
// Remove the block from the pending creates list
|
|
|
//
|
|
|
- INodeFileUnderConstruction file = checkLease(src, holder);
|
|
|
+ INodeFile file = checkLease(src, holder);
|
|
|
boolean removed = dir.removeBlock(src, file,
|
|
|
ExtendedBlock.getLocalBlock(b));
|
|
|
if (!removed) {
|
|
@@ -2835,16 +2835,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
/** make sure that we still have the lease on this file. */
|
|
|
- private INodeFileUnderConstruction checkLease(String src, String holder)
|
|
|
+ private INodeFile checkLease(String src, String holder)
|
|
|
throws LeaseExpiredException, UnresolvedLinkException,
|
|
|
FileNotFoundException {
|
|
|
return checkLease(src, INodeId.GRANDFATHER_INODE_ID, holder,
|
|
|
dir.getINode(src));
|
|
|
}
|
|
|
|
|
|
- private INodeFileUnderConstruction checkLease(String src, long fileId,
|
|
|
- String holder, INode inode) throws LeaseExpiredException,
|
|
|
- FileNotFoundException {
|
|
|
+ private INodeFile checkLease(String src, long fileId, String holder,
|
|
|
+ INode inode) throws LeaseExpiredException, FileNotFoundException {
|
|
|
assert hasReadLock();
|
|
|
if (inode == null || !inode.isFile()) {
|
|
|
Lease lease = leaseManager.getLease(holder);
|
|
@@ -2861,13 +2860,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
+ (lease != null ? lease.toString()
|
|
|
: "Holder " + holder + " does not have any open files."));
|
|
|
}
|
|
|
- INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)file;
|
|
|
- if (holder != null && !pendingFile.getClientName().equals(holder)) {
|
|
|
+ String clientName = file.getFileUnderConstructionFeature().getClientName();
|
|
|
+ if (holder != null && !clientName.equals(holder)) {
|
|
|
throw new LeaseExpiredException("Lease mismatch on " + src + " owned by "
|
|
|
- + pendingFile.getClientName() + " but is accessed by " + holder);
|
|
|
+ + clientName + " but is accessed by " + holder);
|
|
|
}
|
|
|
- INodeId.checkId(fileId, pendingFile);
|
|
|
- return pendingFile;
|
|
|
+ INodeId.checkId(fileId, file);
|
|
|
+ return file;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -2908,7 +2907,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
assert hasWriteLock();
|
|
|
final INodesInPath iip = dir.getLastINodeInPath(src);
|
|
|
- final INodeFileUnderConstruction pendingFile;
|
|
|
+ final INodeFile pendingFile;
|
|
|
try {
|
|
|
pendingFile = checkLease(src, fileId, holder, iip.getINode(0));
|
|
|
} catch (LeaseExpiredException lee) {
|
|
@@ -3599,9 +3598,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot fsync file " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
- INodeFileUnderConstruction pendingFile = checkLease(src, clientName);
|
|
|
+ INodeFile pendingFile = checkLease(src, clientName);
|
|
|
if (lastBlockLength > 0) {
|
|
|
- pendingFile.updateLengthOfLastBlock(lastBlockLength);
|
|
|
+ pendingFile.getFileUnderConstructionFeature().updateLengthOfLastBlock(
|
|
|
+ pendingFile, lastBlockLength);
|
|
|
}
|
|
|
dir.persistBlocks(src, pendingFile, false);
|
|
|
} finally {
|
|
@@ -3632,8 +3632,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
assert hasWriteLock();
|
|
|
|
|
|
final INodesInPath iip = dir.getLastINodeInPath(src);
|
|
|
- final INodeFileUnderConstruction pendingFile
|
|
|
- = INodeFileUnderConstruction.valueOf(iip.getINode(0), src);
|
|
|
+ final INodeFile pendingFile = iip.getINode(0).asFile();
|
|
|
int nrBlocks = pendingFile.numBlocks();
|
|
|
BlockInfo[] blocks = pendingFile.getBlocks();
|
|
|
|
|
@@ -3755,7 +3754,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
private Lease reassignLease(Lease lease, String src, String newHolder,
|
|
|
- INodeFileUnderConstruction pendingFile) {
|
|
|
+ INodeFile pendingFile) {
|
|
|
assert hasWriteLock();
|
|
|
if(newHolder == null)
|
|
|
return lease;
|
|
@@ -3765,15 +3764,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
Lease reassignLeaseInternal(Lease lease, String src, String newHolder,
|
|
|
- INodeFileUnderConstruction pendingFile) {
|
|
|
+ INodeFile pendingFile) {
|
|
|
assert hasWriteLock();
|
|
|
- pendingFile.setClientName(newHolder);
|
|
|
+ pendingFile.getFileUnderConstructionFeature().setClientName(newHolder);
|
|
|
return leaseManager.reassignLease(lease, src, newHolder);
|
|
|
}
|
|
|
|
|
|
- private void commitOrCompleteLastBlock(final INodeFileUnderConstruction fileINode,
|
|
|
+ private void commitOrCompleteLastBlock(final INodeFile fileINode,
|
|
|
final Block commitBlock) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
+ Preconditions.checkArgument(fileINode.isUnderConstruction());
|
|
|
if (!blockManager.commitOrCompleteLastBlock(fileINode, commitBlock)) {
|
|
|
return;
|
|
|
}
|
|
@@ -3790,19 +3790,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void finalizeINodeFileUnderConstruction(String src,
|
|
|
- INodeFileUnderConstruction pendingFile, Snapshot latestSnapshot)
|
|
|
- throws IOException, UnresolvedLinkException {
|
|
|
+ private void finalizeINodeFileUnderConstruction(String src,
|
|
|
+ INodeFile pendingFile, Snapshot latestSnapshot) throws IOException,
|
|
|
+ UnresolvedLinkException {
|
|
|
assert hasWriteLock();
|
|
|
- leaseManager.removeLease(pendingFile.getClientName(), src);
|
|
|
+ FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature();
|
|
|
+ Preconditions.checkArgument(uc != null);
|
|
|
+ leaseManager.removeLease(uc.getClientName(), src);
|
|
|
|
|
|
pendingFile = pendingFile.recordModification(latestSnapshot,
|
|
|
dir.getINodeMap());
|
|
|
|
|
|
// The file is no longer pending.
|
|
|
- // Create permanent INode, update blocks
|
|
|
- final INodeFile newFile = pendingFile.toINodeFile(now());
|
|
|
- dir.replaceINodeFile(src, pendingFile, newFile);
|
|
|
+ // 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());
|
|
|
|
|
|
// close file and persist block allocations for this file
|
|
|
dir.closeFile(src, newFile);
|
|
@@ -3819,12 +3821,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
|
|
|
assert hasReadLock();
|
|
|
final BlockCollection bc = blockUC.getBlockCollection();
|
|
|
- if (bc == null || !(bc instanceof INodeFileUnderConstruction)) {
|
|
|
+ if (bc == null || !(bc instanceof INodeFile)
|
|
|
+ || !((INodeFile) bc).isUnderConstruction()) {
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- INodeFileUnderConstruction inodeUC = (INodeFileUnderConstruction) blockUC
|
|
|
- .getBlockCollection();
|
|
|
+ INodeFile inodeUC = (INodeFile) bc;
|
|
|
String fullName = inodeUC.getName();
|
|
|
try {
|
|
|
if (fullName != null && fullName.startsWith(Path.SEPARATOR)
|
|
@@ -3902,11 +3904,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
+ recoveryId + " for block " + lastblock);
|
|
|
}
|
|
|
|
|
|
- INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)iFile;
|
|
|
-
|
|
|
if (deleteblock) {
|
|
|
Block blockToDel = ExtendedBlock.getLocalBlock(lastblock);
|
|
|
- boolean remove = pendingFile.removeLastBlock(blockToDel);
|
|
|
+ boolean remove = iFile.removeLastBlock(blockToDel);
|
|
|
if (remove) {
|
|
|
blockManager.removeBlockFromMap(storedBlock);
|
|
|
}
|
|
@@ -3944,14 +3944,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// add pipeline locations into the INodeUnderConstruction
|
|
|
DatanodeDescriptor[] targetArray =
|
|
|
new DatanodeDescriptor[targetList.size()];
|
|
|
- pendingFile.setLastBlock(storedBlock, targetList.toArray(targetArray));
|
|
|
+ iFile.setLastBlock(storedBlock, targetList.toArray(targetArray));
|
|
|
}
|
|
|
|
|
|
if (closeFile) {
|
|
|
- src = closeFileCommitBlocks(pendingFile, storedBlock);
|
|
|
+ src = closeFileCommitBlocks(iFile, storedBlock);
|
|
|
} else {
|
|
|
// If this commit does not want to close the file, persist blocks
|
|
|
- src = persistBlocks(pendingFile, false);
|
|
|
+ src = persistBlocks(iFile, false);
|
|
|
}
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -3976,10 +3976,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- String closeFileCommitBlocks(INodeFileUnderConstruction pendingFile,
|
|
|
- BlockInfo storedBlock)
|
|
|
+ String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock)
|
|
|
throws IOException {
|
|
|
-
|
|
|
String src = leaseManager.findPath(pendingFile);
|
|
|
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
@@ -3987,7 +3985,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
//remove lease, close file
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
- Snapshot.findLatestSnapshot(pendingFile, null));
|
|
|
+ Snapshot.findLatestSnapshot(pendingFile, null));
|
|
|
|
|
|
return src;
|
|
|
}
|
|
@@ -4000,8 +3998,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- String persistBlocks(INodeFileUnderConstruction pendingFile,
|
|
|
- boolean logRetryCache) throws IOException {
|
|
|
+ String persistBlocks(INodeFile pendingFile, boolean logRetryCache)
|
|
|
+ throws IOException {
|
|
|
String src = leaseManager.findPath(pendingFile);
|
|
|
dir.persistBlocks(src, pendingFile, logRetryCache);
|
|
|
return src;
|
|
@@ -5186,13 +5184,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
for (Lease lease : leaseManager.getSortedLeases()) {
|
|
|
for (String path : lease.getPaths()) {
|
|
|
- final INodeFileUnderConstruction cons;
|
|
|
+ final INodeFile cons;
|
|
|
try {
|
|
|
- cons = INodeFileUnderConstruction.valueOf(dir.getINode(path), path);
|
|
|
+ cons = dir.getINode(path).asFile();
|
|
|
+ Preconditions.checkState(cons.isUnderConstruction());
|
|
|
} catch (UnresolvedLinkException e) {
|
|
|
throw new AssertionError("Lease files should reside on this FS");
|
|
|
- } catch (IOException e) {
|
|
|
- throw new RuntimeException(e);
|
|
|
}
|
|
|
BlockInfo[] blocks = cons.getBlocks();
|
|
|
if(blocks == null)
|
|
@@ -5768,7 +5765,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
return blockId;
|
|
|
}
|
|
|
|
|
|
- private INodeFileUnderConstruction checkUCBlock(ExtendedBlock block,
|
|
|
+ private INodeFile checkUCBlock(ExtendedBlock block,
|
|
|
String clientName) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
checkNameNodeSafeMode("Cannot get a new generation stamp and an "
|
|
@@ -5784,19 +5781,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
// check file inode
|
|
|
final INodeFile file = ((INode)storedBlock.getBlockCollection()).asFile();
|
|
|
- if (file==null || !file.isUnderConstruction()) {
|
|
|
+ if (file == null || !file.isUnderConstruction()) {
|
|
|
throw new IOException("The file " + storedBlock +
|
|
|
" belonged to does not exist or it is not under construction.");
|
|
|
}
|
|
|
|
|
|
// check lease
|
|
|
- INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)file;
|
|
|
- if (clientName == null || !clientName.equals(pendingFile.getClientName())) {
|
|
|
+ if (clientName == null
|
|
|
+ || !clientName.equals(file.getFileUnderConstructionFeature()
|
|
|
+ .getClientName())) {
|
|
|
throw new LeaseExpiredException("Lease mismatch: " + block +
|
|
|
" is accessed by a non lease holder " + clientName);
|
|
|
}
|
|
|
|
|
|
- return pendingFile;
|
|
|
+ return file;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -5905,8 +5903,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
// check the vadility of the block and lease holder name
|
|
|
- final INodeFileUnderConstruction pendingFile
|
|
|
- = checkUCBlock(oldBlock, clientName);
|
|
|
+ final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
|
|
|
final BlockInfoUnderConstruction blockinfo
|
|
|
= (BlockInfoUnderConstruction)pendingFile.getLastBlock();
|
|
|
|
|
@@ -5950,15 +5947,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* Serializes leases.
|
|
|
*/
|
|
|
void saveFilesUnderConstruction(DataOutputStream out,
|
|
|
- Map<Long, INodeFileUnderConstruction> snapshotUCMap) throws IOException {
|
|
|
+ Map<Long, INodeFile> snapshotUCMap) throws IOException {
|
|
|
// This is run by an inferior thread of saveNamespace, which holds a read
|
|
|
// lock on our behalf. If we took the read lock here, we could block
|
|
|
// for fairness if a writer is waiting on the lock.
|
|
|
synchronized (leaseManager) {
|
|
|
- Map<String, INodeFileUnderConstruction> nodes =
|
|
|
- leaseManager.getINodesUnderConstruction();
|
|
|
- for (Map.Entry<String, INodeFileUnderConstruction> entry
|
|
|
- : nodes.entrySet()) {
|
|
|
+ Map<String, INodeFile> nodes = leaseManager.getINodesUnderConstruction();
|
|
|
+ for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) {
|
|
|
// TODO: for HDFS-5428, because of rename operations, some
|
|
|
// under-construction files that are
|
|
|
// in the current fs directory can also be captured in the
|
|
@@ -5967,13 +5962,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
out.writeInt(nodes.size() + snapshotUCMap.size()); // write the size
|
|
|
- for (Map.Entry<String, INodeFileUnderConstruction> entry
|
|
|
- : nodes.entrySet()) {
|
|
|
+ for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) {
|
|
|
FSImageSerialization.writeINodeUnderConstruction(
|
|
|
out, entry.getValue(), entry.getKey());
|
|
|
}
|
|
|
- for (Map.Entry<Long, INodeFileUnderConstruction> entry
|
|
|
- : snapshotUCMap.entrySet()) {
|
|
|
+ for (Map.Entry<Long, INodeFile> entry : snapshotUCMap.entrySet()) {
|
|
|
// for those snapshot INodeFileUC, we use "/.reserved/.inodes/<inodeid>"
|
|
|
// as their paths
|
|
|
StringBuilder b = new StringBuilder();
|