|
@@ -459,6 +459,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
private final long minBlockSize; // minimum block size
|
|
|
final long maxBlocksPerFile; // maximum # of blocks per file
|
|
|
+ private final int numCommittedAllowed;
|
|
|
|
|
|
/** Lock to protect FSNamesystem. */
|
|
|
private final FSNamesystemLock fsLock;
|
|
@@ -756,6 +757,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT);
|
|
|
this.maxBlocksPerFile = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT);
|
|
|
+ this.numCommittedAllowed = conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_KEY,
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_FILE_CLOSE_NUM_COMMITTED_ALLOWED_DEFAULT);
|
|
|
|
|
|
this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
|
|
|
|
|
@@ -2594,17 +2598,36 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
boolean checkFileProgress(String src, INodeFile v, boolean checkall) {
|
|
|
assert hasReadLock();
|
|
|
if (checkall) {
|
|
|
- return blockManager.checkBlocksProperlyReplicated(src, v
|
|
|
- .getBlocks());
|
|
|
+ return checkBlocksComplete(src, true, v.getBlocks());
|
|
|
} else {
|
|
|
- // check the penultimate block of this file
|
|
|
- BlockInfo b = v.getPenultimateBlock();
|
|
|
- return b == null ||
|
|
|
- blockManager.checkBlocksProperlyReplicated(
|
|
|
- src, new BlockInfo[] { b });
|
|
|
+ final BlockInfo[] blocks = v.getBlocks();
|
|
|
+ final int i = blocks.length - numCommittedAllowed - 2;
|
|
|
+ return i < 0 || blocks[i] == null
|
|
|
+ || checkBlocksComplete(src, false, blocks[i]);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Check if the blocks are COMPLETE;
|
|
|
+ * it may allow the last block to be COMMITTED.
|
|
|
+ */
|
|
|
+ private boolean checkBlocksComplete(String src, boolean allowCommittedBlock,
|
|
|
+ BlockInfo... blocks) {
|
|
|
+ final int n = allowCommittedBlock? numCommittedAllowed: 0;
|
|
|
+ for(int i = 0; i < blocks.length; i++) {
|
|
|
+ final short min = blockManager.getMinStorageNum(blocks[i]);
|
|
|
+ final String err = INodeFile.checkBlockComplete(blocks, i, n, min);
|
|
|
+ if (err != null) {
|
|
|
+ final int numNodes = blocks[i].numNodes();
|
|
|
+ LOG.info("BLOCK* " + err + "(numNodes= " + numNodes
|
|
|
+ + (numNodes < min ? " < " : " >= ")
|
|
|
+ + " minimum = " + min + ") in file " + src);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Change the indicated filename.
|
|
|
* @deprecated Use {@link #renameTo(String, String, boolean,
|
|
@@ -2735,7 +2758,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
List<INode> removedINodes,
|
|
|
final boolean acquireINodeMapLock) {
|
|
|
assert hasWriteLock();
|
|
|
- leaseManager.removeLeases(removedUCFiles);
|
|
|
+ for(long i : removedUCFiles) {
|
|
|
+ leaseManager.removeLease(i);
|
|
|
+ }
|
|
|
// remove inodes from inodesMap
|
|
|
if (removedINodes != null) {
|
|
|
if (acquireINodeMapLock) {
|
|
@@ -2994,7 +3019,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
// then reap lease immediately and close the file.
|
|
|
if(nrCompleteBlocks == nrBlocks) {
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
- iip.getLatestSnapshotId());
|
|
|
+ iip.getLatestSnapshotId(), false);
|
|
|
NameNode.stateChangeLog.warn("BLOCK*"
|
|
|
+ " internalReleaseLease: All existing blocks are COMPLETE,"
|
|
|
+ " lease removed, file closed.");
|
|
@@ -3033,7 +3058,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if(penultimateBlockMinStorage &&
|
|
|
blockManager.hasMinStorage(lastBlock)) {
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
- iip.getLatestSnapshotId());
|
|
|
+ iip.getLatestSnapshotId(), false);
|
|
|
NameNode.stateChangeLog.warn("BLOCK*"
|
|
|
+ " internalReleaseLease: Committed blocks are minimally replicated,"
|
|
|
+ " lease removed, file closed.");
|
|
@@ -3077,7 +3102,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
// We can remove this block and close the file.
|
|
|
pendingFile.removeLastBlock(lastBlock);
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
- iip.getLatestSnapshotId());
|
|
|
+ iip.getLatestSnapshotId(), false);
|
|
|
NameNode.stateChangeLog.warn("BLOCK* internalReleaseLease: "
|
|
|
+ "Removed empty last block and closed file.");
|
|
|
return true;
|
|
@@ -3163,8 +3188,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void finalizeINodeFileUnderConstruction(
|
|
|
- String src, INodeFile pendingFile, int latestSnapshot) throws IOException {
|
|
|
+ void addCommittedBlocksToPending(final INodeFile pendingFile) {
|
|
|
+ final BlockInfo[] blocks = pendingFile.getBlocks();
|
|
|
+ int i = blocks.length - numCommittedAllowed;
|
|
|
+ if (i < 0) {
|
|
|
+ i = 0;
|
|
|
+ }
|
|
|
+ for(; i < blocks.length; i++) {
|
|
|
+ final BlockInfo b = blocks[i];
|
|
|
+ if (b != null && b.getBlockUCState() == BlockUCState.COMMITTED) {
|
|
|
+ // b is COMMITTED but not yet COMPLETE, add it to pending replication.
|
|
|
+ blockManager.addExpectedReplicasToPending(b, pendingFile);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ void finalizeINodeFileUnderConstruction(String src, INodeFile pendingFile,
|
|
|
+ int latestSnapshot, boolean allowCommittedBlock) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
|
|
|
FileUnderConstructionFeature uc = pendingFile.getFileUnderConstructionFeature();
|
|
@@ -3179,7 +3219,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
// The file is no longer pending.
|
|
|
// Create permanent INode, update blocks. No need to replace the inode here
|
|
|
// since we just remove the uc feature from pendingFile
|
|
|
- pendingFile.toCompleteFile(now());
|
|
|
+ pendingFile.toCompleteFile(now(),
|
|
|
+ allowCommittedBlock? numCommittedAllowed: 0,
|
|
|
+ blockManager.getMinReplication());
|
|
|
|
|
|
// close file and persist block allocations for this file
|
|
|
closeFile(src, pendingFile);
|
|
@@ -3412,8 +3454,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
commitOrCompleteLastBlock(pendingFile, iip, storedBlock);
|
|
|
|
|
|
//remove lease, close file
|
|
|
- finalizeINodeFileUnderConstruction(src, pendingFile,
|
|
|
- Snapshot.findLatestSnapshot(pendingFile, Snapshot.CURRENT_STATE_ID));
|
|
|
+ int s = Snapshot.findLatestSnapshot(pendingFile, Snapshot.CURRENT_STATE_ID);
|
|
|
+ finalizeINodeFileUnderConstruction(src, pendingFile, s, false);
|
|
|
}
|
|
|
|
|
|
/**
|