|
@@ -268,7 +268,6 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
|
import org.apache.hadoop.metrics2.util.MBeans;
|
|
|
import org.apache.hadoop.net.NetworkTopology;
|
|
|
import org.apache.hadoop.net.Node;
|
|
|
-import org.apache.hadoop.net.NodeBase;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
|
|
@@ -484,7 +483,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
private final long maxFsObjects; // maximum number of fs objects
|
|
|
|
|
|
private final long minBlockSize; // minimum block size
|
|
|
- private final long maxBlocksPerFile; // maximum # of blocks per file
|
|
|
+ final long maxBlocksPerFile; // maximum # of blocks per file
|
|
|
|
|
|
// precision of access times.
|
|
|
private final long accessTimePrecision;
|
|
@@ -614,7 +613,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
boolean isHaEnabled() {
|
|
|
return haEnabled;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Check the supplied configuration for correctness.
|
|
|
* @param conf Supplies the configuration to validate.
|
|
@@ -1863,8 +1862,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
: dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
|
|
|
|
|
|
final LocatedBlocks blocks = blockManager.createLocatedBlocks(
|
|
|
- inode.getBlocks(iip.getPathSnapshotId()), fileSize,
|
|
|
- isUc, offset, length, needBlockToken, iip.isSnapshot(), feInfo);
|
|
|
+ inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
|
|
|
+ length, needBlockToken, iip.isSnapshot(), feInfo);
|
|
|
|
|
|
// Set caching information for the located blocks.
|
|
|
for (LocatedBlock lb : blocks.getLocatedBlocks()) {
|
|
@@ -2232,8 +2231,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot set storage policy for " + src);
|
|
|
- auditStat = FSDirAttrOp.setStoragePolicy(
|
|
|
- dir, blockManager, src, policyName);
|
|
|
+ auditStat = FSDirAttrOp.setStoragePolicy(dir, blockManager, src,
|
|
|
+ policyName);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "setStoragePolicy", src);
|
|
|
throw e;
|
|
@@ -2621,7 +2620,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return toRemoveBlocks;
|
|
|
} catch (IOException ie) {
|
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: " + src + " " +
|
|
|
- ie.getMessage());
|
|
|
+ ie.getMessage());
|
|
|
throw ie;
|
|
|
}
|
|
|
}
|
|
@@ -2703,8 +2702,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
"Cannot append to lazy persist file " + src);
|
|
|
}
|
|
|
// Opening an existing file for append - may need to recover lease.
|
|
|
- recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE,
|
|
|
- iip, src, holder, clientMachine, false);
|
|
|
+ recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, iip, src, holder,
|
|
|
+ clientMachine, false);
|
|
|
|
|
|
final BlockInfoContiguous lastBlock = myFile.getLastBlock();
|
|
|
// Check that the block has at least minimum replication.
|
|
@@ -3042,290 +3041,49 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* are replicated. Will return an empty 2-elt array if we want the
|
|
|
* client to "try again later".
|
|
|
*/
|
|
|
- LocatedBlock getAdditionalBlock(String src, long fileId, String clientName,
|
|
|
- ExtendedBlock previous, Set<Node> excludedNodes,
|
|
|
- List<String> favoredNodes) throws IOException {
|
|
|
- LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
|
- DatanodeStorageInfo targets[] = getNewBlockTargets(src, fileId,
|
|
|
- clientName, previous, excludedNodes, favoredNodes, onRetryBlock);
|
|
|
- if (targets == null) {
|
|
|
- assert onRetryBlock[0] != null : "Retry block is null";
|
|
|
- // This is a retry. Just return the last block.
|
|
|
- return onRetryBlock[0];
|
|
|
- }
|
|
|
- LocatedBlock newBlock = storeAllocatedBlock(
|
|
|
- src, fileId, clientName, previous, targets);
|
|
|
- return newBlock;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Part I of getAdditionalBlock().
|
|
|
- * Analyze the state of the file under read lock to determine if the client
|
|
|
- * can add a new block, detect potential retries, lease mismatches,
|
|
|
- * and minimal replication of the penultimate block.
|
|
|
- *
|
|
|
- * Generate target DataNode locations for the new block,
|
|
|
- * but do not create the new block yet.
|
|
|
- */
|
|
|
- DatanodeStorageInfo[] getNewBlockTargets(String src, long fileId,
|
|
|
- String clientName, ExtendedBlock previous, Set<Node> excludedNodes,
|
|
|
- List<String> favoredNodes, LocatedBlock[] onRetryBlock) throws IOException {
|
|
|
- final long blockSize;
|
|
|
- final int replication;
|
|
|
- final byte storagePolicyID;
|
|
|
- Node clientNode = null;
|
|
|
- String clientMachine = null;
|
|
|
-
|
|
|
+ LocatedBlock getAdditionalBlock(
|
|
|
+ String src, long fileId, String clientName, ExtendedBlock previous,
|
|
|
+ DatanodeInfo[] excludedNodes, String[] favoredNodes) throws IOException {
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: "
|
|
|
+ src + " inodeId " + fileId + " for " + clientName);
|
|
|
}
|
|
|
|
|
|
- checkOperation(OperationCategory.READ);
|
|
|
- byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
+ waitForLoadingFSImage();
|
|
|
+ LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
|
+ FSDirWriteFileOp.ValidateAddBlockResult r;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.READ);
|
|
|
- src = dir.resolvePath(pc, src, pathComponents);
|
|
|
- FileState fileState = analyzeFileState(
|
|
|
- src, fileId, clientName, previous, onRetryBlock);
|
|
|
- final INodeFile pendingFile = fileState.inode;
|
|
|
- // Check if the penultimate block is minimally replicated
|
|
|
- if (!checkFileProgress(src, pendingFile, false)) {
|
|
|
- throw new NotReplicatedYetException("Not replicated yet: " + src);
|
|
|
- }
|
|
|
- src = fileState.path;
|
|
|
-
|
|
|
- if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
|
|
|
- // This is a retry. No need to generate new locations.
|
|
|
- // Use the last block if it has locations.
|
|
|
- return null;
|
|
|
- }
|
|
|
- if (pendingFile.getBlocks().length >= maxBlocksPerFile) {
|
|
|
- throw new IOException("File has reached the limit on maximum number of"
|
|
|
- + " blocks (" + DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY
|
|
|
- + "): " + pendingFile.getBlocks().length + " >= "
|
|
|
- + maxBlocksPerFile);
|
|
|
- }
|
|
|
- blockSize = pendingFile.getPreferredBlockSize();
|
|
|
- clientMachine = pendingFile.getFileUnderConstructionFeature()
|
|
|
- .getClientMachine();
|
|
|
- clientNode = blockManager.getDatanodeManager().getDatanodeByHost(
|
|
|
- clientMachine);
|
|
|
- replication = pendingFile.getFileReplication();
|
|
|
- storagePolicyID = pendingFile.getStoragePolicyID();
|
|
|
+ r = FSDirWriteFileOp.validateAddBlock(this, pc, src, fileId, clientName,
|
|
|
+ previous, onRetryBlock);
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
|
|
|
|
- if (clientNode == null) {
|
|
|
- clientNode = getClientNode(clientMachine);
|
|
|
+ if (r == null) {
|
|
|
+ assert onRetryBlock[0] != null : "Retry block is null";
|
|
|
+ // This is a retry. Just return the last block.
|
|
|
+ return onRetryBlock[0];
|
|
|
}
|
|
|
|
|
|
- // choose targets for the new block to be allocated.
|
|
|
- return getBlockManager().chooseTarget4NewBlock(
|
|
|
- src, replication, clientNode, excludedNodes, blockSize, favoredNodes,
|
|
|
- storagePolicyID);
|
|
|
- }
|
|
|
+ DatanodeStorageInfo[] targets = FSDirWriteFileOp.chooseTargetForNewBlock(
|
|
|
+ blockManager, src, excludedNodes, favoredNodes, r);
|
|
|
|
|
|
- /**
|
|
|
- * Part II of getAdditionalBlock().
|
|
|
- * Should repeat the same analysis of the file state as in Part 1,
|
|
|
- * but under the write lock.
|
|
|
- * If the conditions still hold, then allocate a new block with
|
|
|
- * the new targets, add it to the INode and to the BlocksMap.
|
|
|
- */
|
|
|
- LocatedBlock storeAllocatedBlock(String src, long fileId, String clientName,
|
|
|
- ExtendedBlock previous, DatanodeStorageInfo[] targets) throws IOException {
|
|
|
- Block newBlock = null;
|
|
|
- long offset;
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- waitForLoadingFSImage();
|
|
|
writeLock();
|
|
|
+ LocatedBlock lb;
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- // Run the full analysis again, since things could have changed
|
|
|
- // while chooseTarget() was executing.
|
|
|
- LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
|
- FileState fileState =
|
|
|
- analyzeFileState(src, fileId, clientName, previous, onRetryBlock);
|
|
|
- final INodeFile pendingFile = fileState.inode;
|
|
|
- src = fileState.path;
|
|
|
-
|
|
|
- if (onRetryBlock[0] != null) {
|
|
|
- if (onRetryBlock[0].getLocations().length > 0) {
|
|
|
- // This is a retry. Just return the last block if having locations.
|
|
|
- return onRetryBlock[0];
|
|
|
- } else {
|
|
|
- // add new chosen targets to already allocated block and return
|
|
|
- BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
|
|
|
- ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
|
|
|
- .setExpectedLocations(targets);
|
|
|
- offset = pendingFile.computeFileSize();
|
|
|
- return makeLocatedBlock(lastBlockInFile, targets, offset);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // commit the last block and complete it if it has minimum replicas
|
|
|
- commitOrCompleteLastBlock(pendingFile, fileState.iip,
|
|
|
- ExtendedBlock.getLocalBlock(previous));
|
|
|
-
|
|
|
- // allocate new block, record block locations in INode.
|
|
|
- newBlock = createNewBlock();
|
|
|
- INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
|
|
|
- saveAllocatedBlock(src, inodesInPath, newBlock, targets);
|
|
|
-
|
|
|
- persistNewBlock(src, pendingFile);
|
|
|
- offset = pendingFile.computeFileSize();
|
|
|
+ lb = FSDirWriteFileOp.storeAllocatedBlock(
|
|
|
+ this, src, fileId, clientName, previous, targets);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
-
|
|
|
- // Return located block
|
|
|
- return makeLocatedBlock(newBlock, targets, offset);
|
|
|
- }
|
|
|
-
|
|
|
- /*
|
|
|
- * Resolve clientmachine address to get a network location path
|
|
|
- */
|
|
|
- private Node getClientNode(String clientMachine) {
|
|
|
- List<String> hosts = new ArrayList<String>(1);
|
|
|
- hosts.add(clientMachine);
|
|
|
- List<String> rName = getBlockManager().getDatanodeManager()
|
|
|
- .resolveNetworkLocation(hosts);
|
|
|
- Node clientNode = null;
|
|
|
- if (rName != null) {
|
|
|
- // Able to resolve clientMachine mapping.
|
|
|
- // Create a temp node to findout the rack local nodes
|
|
|
- clientNode = new NodeBase(rName.get(0) + NodeBase.PATH_SEPARATOR_STR
|
|
|
- + clientMachine);
|
|
|
- }
|
|
|
- return clientNode;
|
|
|
- }
|
|
|
-
|
|
|
- static class FileState {
|
|
|
- public final INodeFile inode;
|
|
|
- public final String path;
|
|
|
- public final INodesInPath iip;
|
|
|
-
|
|
|
- public FileState(INodeFile inode, String fullPath, INodesInPath iip) {
|
|
|
- this.inode = inode;
|
|
|
- this.path = fullPath;
|
|
|
- this.iip = iip;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- FileState analyzeFileState(String src,
|
|
|
- long fileId,
|
|
|
- String clientName,
|
|
|
- ExtendedBlock previous,
|
|
|
- LocatedBlock[] onRetryBlock)
|
|
|
- throws IOException {
|
|
|
- assert hasReadLock();
|
|
|
-
|
|
|
- checkBlock(previous);
|
|
|
- onRetryBlock[0] = null;
|
|
|
- checkNameNodeSafeMode("Cannot add block to " + src);
|
|
|
-
|
|
|
- // have we exceeded the configured limit of fs objects.
|
|
|
- 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 = 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);
|
|
|
- iip = INodesInPath.fromINode(inode);
|
|
|
- if (inode != null) {
|
|
|
- src = iip.getPath();
|
|
|
- }
|
|
|
- }
|
|
|
- final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
|
|
|
- BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
|
|
|
- if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
|
|
|
- // The block that the client claims is the current last block
|
|
|
- // doesn't match up with what we think is the last block. There are
|
|
|
- // four possibilities:
|
|
|
- // 1) This is the first block allocation of an append() pipeline
|
|
|
- // which started appending exactly at or exceeding the block boundary.
|
|
|
- // In this case, the client isn't passed the previous block,
|
|
|
- // so it makes the allocateBlock() call with previous=null.
|
|
|
- // We can distinguish this since the last block of the file
|
|
|
- // will be exactly a full block.
|
|
|
- // 2) This is a retry from a client that missed the response of a
|
|
|
- // prior getAdditionalBlock() call, perhaps because of a network
|
|
|
- // timeout, or because of an HA failover. In that case, we know
|
|
|
- // by the fact that the client is re-issuing the RPC that it
|
|
|
- // never began to write to the old block. Hence it is safe to
|
|
|
- // to return the existing block.
|
|
|
- // 3) This is an entirely bogus request/bug -- we should error out
|
|
|
- // rather than potentially appending a new block with an empty
|
|
|
- // one in the middle, etc
|
|
|
- // 4) This is a retry from a client that timed out while
|
|
|
- // the prior getAdditionalBlock() is still being processed,
|
|
|
- // currently working on chooseTarget().
|
|
|
- // There are no means to distinguish between the first and
|
|
|
- // the second attempts in Part I, because the first one hasn't
|
|
|
- // changed the namesystem state yet.
|
|
|
- // We run this analysis again in Part II where case 4 is impossible.
|
|
|
-
|
|
|
- BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
|
|
|
- if (previous == null &&
|
|
|
- lastBlockInFile != null &&
|
|
|
- lastBlockInFile.getNumBytes() >= pendingFile.getPreferredBlockSize() &&
|
|
|
- lastBlockInFile.isComplete()) {
|
|
|
- // Case 1
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug(
|
|
|
- "BLOCK* NameSystem.allocateBlock: handling block allocation" +
|
|
|
- " writing to a file with a complete previous block: src=" +
|
|
|
- src + " lastBlock=" + lastBlockInFile);
|
|
|
- }
|
|
|
- } else if (Block.matchingIdAndGenStamp(penultimateBlock, previousBlock)) {
|
|
|
- if (lastBlockInFile.getNumBytes() != 0) {
|
|
|
- throw new IOException(
|
|
|
- "Request looked like a retry to allocate block " +
|
|
|
- lastBlockInFile + " but it already contains " +
|
|
|
- lastBlockInFile.getNumBytes() + " bytes");
|
|
|
- }
|
|
|
-
|
|
|
- // Case 2
|
|
|
- // Return the last block.
|
|
|
- NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
|
|
|
- "caught retry for allocation of a new block in " +
|
|
|
- src + ". Returning previously allocated block " + lastBlockInFile);
|
|
|
- long offset = pendingFile.computeFileSize();
|
|
|
- onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
|
|
|
- ((BlockInfoContiguousUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
|
|
|
- offset);
|
|
|
- return new FileState(pendingFile, src, iip);
|
|
|
- } else {
|
|
|
- // Case 3
|
|
|
- throw new IOException("Cannot allocate block in " + src + ": " +
|
|
|
- "passed 'previous' block " + previous + " does not match actual " +
|
|
|
- "last block in file " + lastBlockInFile);
|
|
|
- }
|
|
|
- }
|
|
|
- return new FileState(pendingFile, src, iip);
|
|
|
- }
|
|
|
-
|
|
|
- LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
|
|
|
- long offset) throws IOException {
|
|
|
- LocatedBlock lBlk = BlockManager.newLocatedBlock(
|
|
|
- getExtendedBlock(blk), locs, offset, false);
|
|
|
- getBlockManager().setBlockToken(
|
|
|
- lBlk, BlockTokenIdentifier.AccessMode.WRITE);
|
|
|
- return lBlk;
|
|
|
+ return lb;
|
|
|
}
|
|
|
|
|
|
/** @see ClientProtocol#getAdditionalDatanode */
|
|
@@ -3378,7 +3136,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
if (clientnode == null) {
|
|
|
- clientnode = getClientNode(clientMachine);
|
|
|
+ clientnode = FSDirWriteFileOp.getClientNode(blockManager, clientMachine);
|
|
|
}
|
|
|
|
|
|
// choose new datanodes.
|
|
@@ -3394,60 +3152,32 @@ 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)
|
|
|
+ void abandonBlock(ExtendedBlock b, long fileId, String src, String holder)
|
|
|
throws IOException {
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: " + b
|
|
|
+ "of file " + src);
|
|
|
}
|
|
|
+ waitForLoadingFSImage();
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
- waitForLoadingFSImage();
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
|
|
|
- src = dir.resolvePath(pc, src, pathComponents);
|
|
|
-
|
|
|
- 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 = dir.getINodesInPath(src, true);
|
|
|
- inode = iip.getLastINode();
|
|
|
- } else {
|
|
|
- inode = dir.getInode(fileId);
|
|
|
- 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, iip, file,
|
|
|
- ExtendedBlock.getLocalBlock(b));
|
|
|
- if (!removed) {
|
|
|
- return true;
|
|
|
- }
|
|
|
+ FSDirWriteFileOp.abandonBlock(dir, pc, b, fileId, src, holder);
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
|
|
|
+ b + " is removed from pendingCreates");
|
|
|
}
|
|
|
- persistBlocks(src, file, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
-
|
|
|
- return true;
|
|
|
}
|
|
|
|
|
|
- private INodeFile checkLease(String src, String holder, INode inode,
|
|
|
- long fileId) throws LeaseExpiredException, FileNotFoundException {
|
|
|
+ INodeFile checkLease(
|
|
|
+ String src, String holder, INode inode, long fileId) throws LeaseExpiredException, FileNotFoundException {
|
|
|
assert hasReadLock();
|
|
|
final String ident = src + " (inode " + fileId + ")";
|
|
|
if (inode == null) {
|
|
@@ -3492,120 +3222,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* (e.g if not all blocks have reached minimum replication yet)
|
|
|
* @throws IOException on error (eg lease mismatch, file not open, file deleted)
|
|
|
*/
|
|
|
- boolean completeFile(final String srcArg, String holder,
|
|
|
+ boolean completeFile(final String src, String holder,
|
|
|
ExtendedBlock last, long fileId)
|
|
|
- throws SafeModeException, UnresolvedLinkException, IOException {
|
|
|
- String src = srcArg;
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " +
|
|
|
- src + " for " + holder);
|
|
|
- }
|
|
|
- checkBlock(last);
|
|
|
+ throws IOException {
|
|
|
boolean success = false;
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
- FSPermissionChecker pc = getPermissionChecker();
|
|
|
waitForLoadingFSImage();
|
|
|
+ FSPermissionChecker pc = getPermissionChecker();
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot complete file " + src);
|
|
|
- src = dir.resolvePath(pc, src, pathComponents);
|
|
|
- success = completeFileInternal(src, holder,
|
|
|
- ExtendedBlock.getLocalBlock(last), fileId);
|
|
|
+ success = FSDirWriteFileOp.completeFile(this, pc, src, holder, last,
|
|
|
+ fileId);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
- if (success) {
|
|
|
- NameNode.stateChangeLog.info("DIR* completeFile: " + srcArg
|
|
|
- + " is closed by " + holder);
|
|
|
- }
|
|
|
return success;
|
|
|
}
|
|
|
|
|
|
- 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 {
|
|
|
- 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 = dir.getINodesInPath(src, true);
|
|
|
- inode = iip.getLastINode();
|
|
|
- } else {
|
|
|
- inode = dir.getInode(fileId);
|
|
|
- iip = INodesInPath.fromINode(inode);
|
|
|
- if (inode != null) {
|
|
|
- src = iip.getPath();
|
|
|
- }
|
|
|
- }
|
|
|
- pendingFile = checkLease(src, holder, inode, fileId);
|
|
|
- } catch (LeaseExpiredException lee) {
|
|
|
- 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
|
|
|
- // the client's view of the last block matches the actual
|
|
|
- // last block, then we'll treat it as a successful close.
|
|
|
- // See HDFS-3031.
|
|
|
- final Block realLastBlock = inode.asFile().getLastBlock();
|
|
|
- if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
|
|
|
- NameNode.stateChangeLog.info("DIR* completeFile: " +
|
|
|
- "request from " + holder + " to complete inode " + fileId +
|
|
|
- "(" + src + ") which is already closed. But, it appears to be " +
|
|
|
- "an RPC retry. Returning success");
|
|
|
- return true;
|
|
|
- }
|
|
|
- }
|
|
|
- throw lee;
|
|
|
- }
|
|
|
- // Check the state of the penultimate block. It should be completed
|
|
|
- // before attempting to complete the last one.
|
|
|
- if (!checkFileProgress(src, pendingFile, false)) {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- // commit the last block and complete it if it has minimum replicas
|
|
|
- commitOrCompleteLastBlock(pendingFile, iip, last);
|
|
|
-
|
|
|
- if (!checkFileProgress(src, pendingFile, true)) {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
- Snapshot.CURRENT_STATE_ID);
|
|
|
- return true;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Save allocated block at the given pending filename
|
|
|
- *
|
|
|
- * @param src path to the file
|
|
|
- * @param inodesInPath representing each of the components of src.
|
|
|
- * The last INode is the INode for {@code src} file.
|
|
|
- * @param newBlock newly allocated block to be save
|
|
|
- * @param targets target datanodes where replicas of the new block is placed
|
|
|
- * @throws QuotaExceededException If addition of block exceeds space quota
|
|
|
- */
|
|
|
- private void saveAllocatedBlock(String src, INodesInPath inodesInPath,
|
|
|
- Block newBlock, DatanodeStorageInfo[] targets)
|
|
|
- throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
- BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets);
|
|
|
- NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
|
|
|
- DatanodeStorageInfo.incrementBlocksScheduled(targets);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Create new block with a unique block id and a new generation stamp.
|
|
|
*/
|
|
|
- private Block createNewBlock() throws IOException {
|
|
|
+ Block createNewBlock() throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
Block b = new Block(nextBlockId(), 0, 0);
|
|
|
// Increment the generation stamp for every new block.
|
|
@@ -3997,7 +3637,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
pendingFile.getFileUnderConstructionFeature().updateLengthOfLastBlock(
|
|
|
pendingFile, lastBlockLength);
|
|
|
}
|
|
|
- persistBlocks(src, pendingFile, false);
|
|
|
+ FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -4167,8 +3807,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return leaseManager.reassignLease(lease, pendingFile, newHolder);
|
|
|
}
|
|
|
|
|
|
- private void commitOrCompleteLastBlock(final INodeFile fileINode,
|
|
|
- final INodesInPath iip, final Block commitBlock) throws IOException {
|
|
|
+ void commitOrCompleteLastBlock(
|
|
|
+ final INodeFile fileINode, final INodesInPath iip,
|
|
|
+ final Block commitBlock) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
Preconditions.checkArgument(fileINode.isUnderConstruction());
|
|
|
if (!blockManager.commitOrCompleteLastBlock(fileINode, commitBlock)) {
|
|
@@ -4186,14 +3827,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void finalizeINodeFileUnderConstruction(String src,
|
|
|
- INodeFile pendingFile, int latestSnapshot) throws IOException {
|
|
|
+ void finalizeINodeFileUnderConstruction(
|
|
|
+ String src, INodeFile pendingFile, int latestSnapshot) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
|
|
|
FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature();
|
|
|
Preconditions.checkArgument(uc != null);
|
|
|
leaseManager.removeLease(uc.getClientName(), pendingFile);
|
|
|
-
|
|
|
+
|
|
|
pendingFile.recordModification(latestSnapshot);
|
|
|
|
|
|
// The file is no longer pending.
|
|
@@ -4405,7 +4046,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
} else {
|
|
|
// If this commit does not want to close the file, persist blocks
|
|
|
src = iFile.getFullPathName();
|
|
|
- persistBlocks(src, iFile, false);
|
|
|
+ FSDirWriteFileOp.persistBlocks(dir, src, iFile, false);
|
|
|
}
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -4595,24 +4236,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
hasResourcesAvailable = nnResourceChecker.hasAvailableDiskSpace();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Persist the block list for the inode.
|
|
|
- * @param path
|
|
|
- * @param file
|
|
|
- * @param logRetryCache
|
|
|
- */
|
|
|
- private void persistBlocks(String path, INodeFile file,
|
|
|
- boolean logRetryCache) {
|
|
|
- assert hasWriteLock();
|
|
|
- Preconditions.checkArgument(file.isUnderConstruction());
|
|
|
- getEditLog().logUpdateBlocks(path, file, logRetryCache);
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("persistBlocks: " + path
|
|
|
- + " with " + file.getBlocks().length + " blocks is persisted to" +
|
|
|
- " the file system");
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Close file.
|
|
|
* @param path
|
|
@@ -4800,13 +4423,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
public FSEditLog getEditLog() {
|
|
|
return getFSImage().getEditLog();
|
|
|
- }
|
|
|
-
|
|
|
- private void checkBlock(ExtendedBlock block) throws IOException {
|
|
|
- if (block != null && !this.blockPoolId.equals(block.getBlockPoolId())) {
|
|
|
- throw new IOException("Unexpected BlockPoolId " + block.getBlockPoolId()
|
|
|
- + " - expected " + blockPoolId);
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
@Metric({"MissingBlocks", "Number of missing blocks"})
|
|
@@ -5079,21 +4695,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Persist the new block (the last block of the given file).
|
|
|
- * @param path
|
|
|
- * @param file
|
|
|
- */
|
|
|
- private void persistNewBlock(String path, INodeFile file) {
|
|
|
- Preconditions.checkArgument(file.isUnderConstruction());
|
|
|
- getEditLog().logAddBlock(path, file);
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("persistNewBlock: "
|
|
|
- + path + " with new block " + file.getLastBlock().toString()
|
|
|
- + ", current total block count is " + file.getBlocks().length);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* SafeModeInfo contains information related to the safe mode.
|
|
|
* <p>
|
|
@@ -6399,7 +6000,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
blockinfo.setExpectedLocations(storages);
|
|
|
|
|
|
String src = pendingFile.getFullPathName();
|
|
|
- persistBlocks(src, pendingFile, logRetryCache);
|
|
|
+ FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache);
|
|
|
}
|
|
|
|
|
|
/**
|