|
@@ -644,19 +644,13 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
*/
|
|
|
private static boolean commitBlock(final BlockInfo block,
|
|
|
final Block commitBlock) throws IOException {
|
|
|
- if (block instanceof BlockInfoUnderConstruction
|
|
|
- && block.getBlockUCState() != BlockUCState.COMMITTED) {
|
|
|
- final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
|
|
|
-
|
|
|
- assert block.getNumBytes() <= commitBlock.getNumBytes() :
|
|
|
+ if (block.getBlockUCState() == BlockUCState.COMMITTED)
|
|
|
+ return false;
|
|
|
+ assert block.getNumBytes() <= commitBlock.getNumBytes() :
|
|
|
"commitBlock length is less than the stored one "
|
|
|
- + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
|
|
|
-
|
|
|
- uc.commitBlock(commitBlock);
|
|
|
- return true;
|
|
|
- }
|
|
|
-
|
|
|
- return false;
|
|
|
+ + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
|
|
|
+ block.commitBlock(commitBlock);
|
|
|
+ return true;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -713,9 +707,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
"Cannot complete block: block has not been COMMITTED by the client");
|
|
|
}
|
|
|
|
|
|
- final BlockInfo completeBlock
|
|
|
- = !(curBlock instanceof BlockInfoUnderConstruction)? curBlock
|
|
|
- : ((BlockInfoUnderConstruction)curBlock).convertToCompleteBlock();
|
|
|
+ final BlockInfo completeBlock = curBlock.convertToCompleteBlock();
|
|
|
|
|
|
// replace penultimate block in file
|
|
|
bc.setBlock(blkIndex, completeBlock);
|
|
@@ -754,9 +746,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
*/
|
|
|
public BlockInfo forceCompleteBlock(final BlockCollection bc,
|
|
|
final BlockInfo block) throws IOException {
|
|
|
- if (block instanceof BlockInfoUnderConstruction) {
|
|
|
- ((BlockInfoUnderConstruction)block).commitBlock(block);
|
|
|
- }
|
|
|
+ block.commitBlock(block);
|
|
|
return completeBlock(bc, block, true);
|
|
|
}
|
|
|
|
|
@@ -777,30 +767,28 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
*/
|
|
|
public LocatedBlock convertLastBlockToUnderConstruction(
|
|
|
BlockCollection bc, long bytesToRemove) throws IOException {
|
|
|
- BlockInfo oldBlock = bc.getLastBlock();
|
|
|
- if(oldBlock == null ||
|
|
|
- bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
|
|
|
+ BlockInfo lastBlock = bc.getLastBlock();
|
|
|
+ if(lastBlock == null ||
|
|
|
+ bc.getPreferredBlockSize() == lastBlock.getNumBytes() - bytesToRemove)
|
|
|
return null;
|
|
|
- assert oldBlock == getStoredBlock(oldBlock) :
|
|
|
+ assert lastBlock == getStoredBlock(lastBlock) :
|
|
|
"last block of the file is not in blocksMap";
|
|
|
|
|
|
- DatanodeStorageInfo[] targets = getStorages(oldBlock);
|
|
|
+ DatanodeStorageInfo[] targets = getStorages(lastBlock);
|
|
|
|
|
|
- // convert the last block to UC
|
|
|
- bc.convertLastBlockToUC(oldBlock, targets);
|
|
|
- // get the new created uc block
|
|
|
- BlockInfo ucBlock = bc.getLastBlock();
|
|
|
- blocksMap.replaceBlock(ucBlock);
|
|
|
+ // convert the last block to under construction. note no block replacement
|
|
|
+ // is happening
|
|
|
+ bc.convertLastBlockToUC(lastBlock, targets);
|
|
|
|
|
|
// Remove block from replication queue.
|
|
|
- NumberReplicas replicas = countNodes(ucBlock);
|
|
|
- neededReplications.remove(ucBlock, replicas.liveReplicas(),
|
|
|
- replicas.decommissionedAndDecommissioning(), getReplication(ucBlock));
|
|
|
- pendingReplications.remove(ucBlock);
|
|
|
+ NumberReplicas replicas = countNodes(lastBlock);
|
|
|
+ neededReplications.remove(lastBlock, replicas.liveReplicas(),
|
|
|
+ replicas.decommissionedAndDecommissioning(), getReplication(lastBlock));
|
|
|
+ pendingReplications.remove(lastBlock);
|
|
|
|
|
|
// remove this block from the list of pending blocks to be deleted.
|
|
|
for (DatanodeStorageInfo storage : targets) {
|
|
|
- final Block b = getBlockOnStorage(oldBlock, storage);
|
|
|
+ final Block b = getBlockOnStorage(lastBlock, storage);
|
|
|
if (b != null) {
|
|
|
invalidateBlocks.remove(storage.getDatanodeDescriptor(), b);
|
|
|
}
|
|
@@ -810,13 +798,15 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
// count in safe-mode.
|
|
|
namesystem.adjustSafeModeBlockTotals(
|
|
|
// decrement safe if we had enough
|
|
|
- hasMinStorage(oldBlock, targets.length) ? -1 : 0,
|
|
|
+ hasMinStorage(lastBlock, targets.length) ? -1 : 0,
|
|
|
// always decrement total blocks
|
|
|
-1);
|
|
|
|
|
|
- final long fileLength = bc.computeContentSummary(getStoragePolicySuite()).getLength();
|
|
|
- final long pos = fileLength - ucBlock.getNumBytes();
|
|
|
- return createLocatedBlock(ucBlock, pos, BlockTokenIdentifier.AccessMode.WRITE);
|
|
|
+ final long fileLength = bc.computeContentSummary(
|
|
|
+ getStoragePolicySuite()).getLength();
|
|
|
+ final long pos = fileLength - lastBlock.getNumBytes();
|
|
|
+ return createLocatedBlock(lastBlock, pos,
|
|
|
+ BlockTokenIdentifier.AccessMode.WRITE);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -895,18 +885,14 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
|
|
|
throws IOException {
|
|
|
if (!blk.isComplete()) {
|
|
|
+ final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
|
|
|
if (blk.isStriped()) {
|
|
|
- final BlockInfoStripedUnderConstruction uc =
|
|
|
- (BlockInfoStripedUnderConstruction) blk;
|
|
|
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
|
|
|
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
|
|
|
blk);
|
|
|
return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
|
|
|
false);
|
|
|
} else {
|
|
|
- assert blk instanceof BlockInfoContiguousUnderConstruction;
|
|
|
- final BlockInfoContiguousUnderConstruction uc =
|
|
|
- (BlockInfoContiguousUnderConstruction) blk;
|
|
|
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
|
|
|
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
|
|
|
blk);
|
|
@@ -1923,9 +1909,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
StatefulBlockInfo(BlockInfo storedBlock,
|
|
|
Block reportedBlock, ReplicaState reportedState) {
|
|
|
- Preconditions.checkArgument(
|
|
|
- storedBlock instanceof BlockInfoContiguousUnderConstruction ||
|
|
|
- storedBlock instanceof BlockInfoStripedUnderConstruction);
|
|
|
+ Preconditions.checkArgument(!storedBlock.isComplete());
|
|
|
this.storedBlock = storedBlock;
|
|
|
this.reportedBlock = reportedBlock;
|
|
|
this.reportedState = reportedState;
|
|
@@ -2335,13 +2319,14 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
// If block is under construction, add this replica to its list
|
|
|
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
|
|
|
- final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)storedBlock;
|
|
|
- uc.addReplicaIfNotPresent(storageInfo, iblk, reportedState);
|
|
|
+ storedBlock.getUnderConstructionFeature()
|
|
|
+ .addReplicaIfNotPresent(storageInfo, iblk, reportedState);
|
|
|
// OpenFileBlocks only inside snapshots also will be added to safemode
|
|
|
// threshold. So we need to update such blocks to safemode
|
|
|
// refer HDFS-5283
|
|
|
if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
|
|
|
- int numOfReplicas = uc.getNumExpectedLocations();
|
|
|
+ int numOfReplicas = storedBlock.getUnderConstructionFeature()
|
|
|
+ .getNumExpectedLocations();
|
|
|
namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock);
|
|
|
}
|
|
|
//and fall through to next clause
|
|
@@ -2469,11 +2454,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
// Ignore replicas already scheduled to be removed from the DN
|
|
|
if(invalidateBlocks.contains(dn, block)) {
|
|
|
- /*
|
|
|
- * TODO: following assertion is incorrect, see HDFS-2668 assert
|
|
|
- * storedBlock.findDatanode(dn) < 0 : "Block " + block +
|
|
|
- * " in recentInvalidatesSet should not appear in DN " + dn;
|
|
|
- */
|
|
|
return storedBlock;
|
|
|
}
|
|
|
|
|
@@ -2704,9 +2684,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
|
|
|
DatanodeStorageInfo storageInfo) throws IOException {
|
|
|
BlockInfo block = ucBlock.storedBlock;
|
|
|
- final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)block;
|
|
|
- uc.addReplicaIfNotPresent(storageInfo, ucBlock.reportedBlock,
|
|
|
- ucBlock.reportedState);
|
|
|
+ block.getUnderConstructionFeature().addReplicaIfNotPresent(storageInfo,
|
|
|
+ ucBlock.reportedBlock, ucBlock.reportedState);
|
|
|
|
|
|
if (ucBlock.reportedState == ReplicaState.FINALIZED &&
|
|
|
(block.findStorageInfo(storageInfo) < 0)) {
|
|
@@ -2766,8 +2745,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
assert block != null && namesystem.hasWriteLock();
|
|
|
BlockInfo storedBlock;
|
|
|
DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
|
|
|
- if (block instanceof BlockInfoContiguousUnderConstruction ||
|
|
|
- block instanceof BlockInfoStripedUnderConstruction) {
|
|
|
+ if (!block.isComplete()) {
|
|
|
//refresh our copy in case the block got completed in another thread
|
|
|
storedBlock = getStoredBlock(block);
|
|
|
} else {
|
|
@@ -4221,7 +4199,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
final LocatedBlock lb;
|
|
|
if (info.isStriped()) {
|
|
|
lb = newLocatedStripedBlock(eb, locs,
|
|
|
- ((BlockInfoStripedUnderConstruction)info).getBlockIndices(),
|
|
|
+ info.getUnderConstructionFeature().getBlockIndices(),
|
|
|
offset, false);
|
|
|
} else {
|
|
|
lb = newLocatedBlock(eb, locs, offset, false);
|