|
@@ -1883,6 +1883,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
QuotaExceededException, SafeModeException, UnresolvedLinkException,
|
|
|
IOException {
|
|
|
checkBlock(previous);
|
|
|
+ Block previousBlock = ExtendedBlock.getLocalBlock(previous);
|
|
|
long fileLength, blockSize;
|
|
|
int replication;
|
|
|
DatanodeDescriptor clientNode = null;
|
|
@@ -1905,10 +1906,65 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// have we exceeded the configured limit of fs objects.
|
|
|
checkFsObjectLimit();
|
|
|
|
|
|
- INodeFileUnderConstruction pendingFile = checkLease(src, clientName);
|
|
|
+ INodeFileUnderConstruction pendingFile = checkLease(src, clientName);
|
|
|
+ BlockInfo 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
|
|
|
+ // three possibilities:
|
|
|
+ // 1) This is the first block allocation of an append() pipeline
|
|
|
+ // which started appending exactly at a 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
|
|
|
+ // abandon it and allocate a new one.
|
|
|
+ // 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
|
|
|
+
|
|
|
+ BlockInfo 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)) {
|
|
|
+ // Case 2
|
|
|
+ if (lastBlockInFile.getNumBytes() != 0) {
|
|
|
+ throw new IOException(
|
|
|
+ "Request looked like a retry to allocate block " +
|
|
|
+ lastBlockInFile + " but it already contains " +
|
|
|
+ lastBlockInFile.getNumBytes() + " bytes");
|
|
|
+ }
|
|
|
+
|
|
|
+ // The retry case ("b" above) -- abandon the old block.
|
|
|
+ NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: " +
|
|
|
+ "caught retry for allocation of a new block in " +
|
|
|
+ src + ". Abandoning old block " + lastBlockInFile);
|
|
|
+ dir.removeBlock(src, pendingFile, lastBlockInFile);
|
|
|
+ dir.persistBlocks(src, pendingFile);
|
|
|
+ } else {
|
|
|
+
|
|
|
+ throw new IOException("Cannot allocate block in " + src + ": " +
|
|
|
+ "passed 'previous' block " + previous + " does not match actual " +
|
|
|
+ "last block in file " + lastBlockInFile);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
|
- commitOrCompleteLastBlock(pendingFile, ExtendedBlock.getLocalBlock(previous));
|
|
|
+ commitOrCompleteLastBlock(pendingFile, previousBlock);
|
|
|
|
|
|
//
|
|
|
// If we fail this, bad things happen!
|
|
@@ -2119,7 +2175,29 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throw new SafeModeException("Cannot complete file " + src, safeMode);
|
|
|
}
|
|
|
|
|
|
- INodeFileUnderConstruction pendingFile = checkLease(src, holder);
|
|
|
+ INodeFileUnderConstruction pendingFile;
|
|
|
+ try {
|
|
|
+ pendingFile = checkLease(src, holder);
|
|
|
+ } catch (LeaseExpiredException lee) {
|
|
|
+ INodeFile file = dir.getFileINode(src);
|
|
|
+ if (file != null && !file.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.
|
|
|
+ Block realLastBlock = file.getLastBlock();
|
|
|
+ if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
|
|
|
+ NameNode.stateChangeLog.info("DIR* NameSystem.completeFile: " +
|
|
|
+ "received request from " + holder + " to complete file " + src +
|
|
|
+ " which is already closed. But, it appears to be an RPC " +
|
|
|
+ "retry. Returning success.");
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw lee;
|
|
|
+ }
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
|
commitOrCompleteLastBlock(pendingFile, last);
|
|
|
|