|
@@ -1399,6 +1399,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
|
//
|
|
|
for (BlockInfo block: v.getBlocks()) {
|
|
|
if (!blockManager.checkMinReplication(block)) {
|
|
|
+ LOG.info("BLOCK* NameSystem.checkFileProgress: "
|
|
|
+ + "block " + block + " has not reached minimal replication "
|
|
|
+ + blockManager.minReplication);
|
|
|
return false;
|
|
|
}
|
|
|
}
|
|
@@ -1408,6 +1411,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
|
//
|
|
|
BlockInfo b = v.getPenultimateBlock();
|
|
|
if (b != null && !blockManager.checkMinReplication(b)) {
|
|
|
+ LOG.info("BLOCK* NameSystem.checkFileProgress: "
|
|
|
+ + "block " + b + " has not reached minimal replication "
|
|
|
+ + blockManager.minReplication);
|
|
|
return false;
|
|
|
}
|
|
|
}
|
|
@@ -3942,6 +3948,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
|
synchronized void updatePipeline(String clientName, Block oldBlock,
|
|
|
Block newBlock, DatanodeID[] newNodes)
|
|
|
throws IOException {
|
|
|
+ assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
|
|
|
+ + oldBlock + " has different block identifier";
|
|
|
LOG.info("updatePipeline(block=" + oldBlock
|
|
|
+ ", newGenerationStamp=" + newBlock.getGenerationStamp()
|
|
|
+ ", newLength=" + newBlock.getNumBytes()
|
|
@@ -3952,27 +3960,24 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
|
// check the vadility of the block and lease holder name
|
|
|
final INodeFileUnderConstruction pendingFile =
|
|
|
checkUCBlock(oldBlock, clientName);
|
|
|
- final BlockInfo oldblockinfo = pendingFile.getLastBlock();
|
|
|
+ final BlockInfoUnderConstruction blockinfo = pendingFile.getLastBlock();
|
|
|
|
|
|
// check new GS & length: this is not expected
|
|
|
- if (newBlock.getGenerationStamp() <= oldblockinfo.getGenerationStamp() ||
|
|
|
- newBlock.getNumBytes() < oldblockinfo.getNumBytes()) {
|
|
|
+ if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||
|
|
|
+ newBlock.getNumBytes() < blockinfo.getNumBytes()) {
|
|
|
String msg = "Update " + oldBlock + " (len = " +
|
|
|
- oldblockinfo.getNumBytes() + ") to an older state: " + newBlock +
|
|
|
+ blockinfo.getNumBytes() + ") to an older state: " + newBlock +
|
|
|
" (len = " + newBlock.getNumBytes() +")";
|
|
|
LOG.warn(msg);
|
|
|
throw new IOException(msg);
|
|
|
}
|
|
|
|
|
|
- // Remove old block from blocks map. This always have to be done
|
|
|
+ // Remove old block from the raw map in blocks map.
|
|
|
+ // This does not change any other value of the oldblockinfo
|
|
|
+ // This always have to be done
|
|
|
// because the generation stamp of this block is changing.
|
|
|
- blockManager.removeBlockFromMap(oldblockinfo);
|
|
|
-
|
|
|
- // update last block, construct newblockinfo and add it to the blocks map
|
|
|
- BlockInfoUnderConstruction newblockinfo =
|
|
|
- new BlockInfoUnderConstruction(
|
|
|
- newBlock, pendingFile.getReplication());
|
|
|
- blockManager.addINode(newblockinfo, pendingFile);
|
|
|
+ blockManager.updateBlock(oldBlock,
|
|
|
+ newBlock.getGenerationStamp(), newBlock.getNumBytes());
|
|
|
|
|
|
// find the DatanodeDescriptor objects
|
|
|
DatanodeDescriptor[] descriptors = null;
|
|
@@ -3982,8 +3987,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
|
descriptors[i] = getDatanode(newNodes[i]);
|
|
|
}
|
|
|
}
|
|
|
- // add locations into the INodeUnderConstruction
|
|
|
- pendingFile.setLastBlock(newblockinfo, descriptors);
|
|
|
+ blockinfo.setExpectedLocations(descriptors);
|
|
|
|
|
|
// persist blocks only if append is supported
|
|
|
String src = leaseManager.findPath(pendingFile);
|