|
@@ -543,8 +543,8 @@ public class BlockManager {
|
|
|
int usableReplicas = numReplicas.liveReplicas() +
|
|
|
numReplicas.decommissionedAndDecommissioning();
|
|
|
|
|
|
- if (block instanceof BlockInfoContiguous) {
|
|
|
- BlockCollection bc = ((BlockInfoContiguous) block).getBlockCollection();
|
|
|
+ if (block instanceof BlockInfo) {
|
|
|
+ BlockCollection bc = ((BlockInfo) block).getBlockCollection();
|
|
|
String fileName = (bc == null) ? "[orphaned]" : bc.getName();
|
|
|
out.print(fileName + ": ");
|
|
|
}
|
|
@@ -598,15 +598,14 @@ public class BlockManager {
|
|
|
* @throws IOException if the block does not have at least a minimal number
|
|
|
* of replicas reported from data-nodes.
|
|
|
*/
|
|
|
- private static boolean commitBlock(
|
|
|
- final BlockInfoContiguousUnderConstruction block,
|
|
|
+ private static boolean commitBlock(final BlockInfo block,
|
|
|
final Block commitBlock) throws IOException {
|
|
|
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();
|
|
|
- block.commitBlock(commitBlock);
|
|
|
+ BlockInfo.commitBlock(block, commitBlock);
|
|
|
return true;
|
|
|
}
|
|
|
|
|
@@ -624,16 +623,16 @@ public class BlockManager {
|
|
|
Block commitBlock) throws IOException {
|
|
|
if(commitBlock == null)
|
|
|
return false; // not committing, this is a block allocation retry
|
|
|
- BlockInfoContiguous lastBlock = bc.getLastBlock();
|
|
|
+ BlockInfo lastBlock = bc.getLastBlock();
|
|
|
if(lastBlock == null)
|
|
|
return false; // no blocks in file yet
|
|
|
if(lastBlock.isComplete())
|
|
|
return false; // already completed (e.g. by syncBlock)
|
|
|
|
|
|
- final boolean b = commitBlock(
|
|
|
- (BlockInfoContiguousUnderConstruction)lastBlock, commitBlock);
|
|
|
- if(countNodes(lastBlock).liveReplicas() >= minReplication)
|
|
|
- completeBlock(bc, bc.numBlocks()-1, false);
|
|
|
+ final boolean b = commitBlock(lastBlock, commitBlock);
|
|
|
+ if (countNodes(lastBlock).liveReplicas() >= minReplication) {
|
|
|
+ completeBlock(bc, bc.numBlocks() - 1, false);
|
|
|
+ }
|
|
|
return b;
|
|
|
}
|
|
|
|
|
@@ -646,22 +645,25 @@ public class BlockManager {
|
|
|
*/
|
|
|
private BlockInfo completeBlock(final BlockCollection bc,
|
|
|
final int blkIndex, boolean force) throws IOException {
|
|
|
- if(blkIndex < 0)
|
|
|
+ if (blkIndex < 0) {
|
|
|
return null;
|
|
|
- BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex];
|
|
|
- if (curBlock.isComplete())
|
|
|
+ }
|
|
|
+ BlockInfo curBlock = bc.getBlocks()[blkIndex];
|
|
|
+ if (curBlock.isComplete()) {
|
|
|
return curBlock;
|
|
|
- // TODO: support BlockInfoStripedUC
|
|
|
- BlockInfoContiguousUnderConstruction ucBlock =
|
|
|
- (BlockInfoContiguousUnderConstruction)curBlock;
|
|
|
- int numNodes = ucBlock.numNodes();
|
|
|
- if (!force && numNodes < minReplication)
|
|
|
+ }
|
|
|
+
|
|
|
+ int numNodes = curBlock.numNodes();
|
|
|
+ if (!force && numNodes < minReplication) {
|
|
|
throw new IOException("Cannot complete block: " +
|
|
|
"block does not satisfy minimal replication requirement.");
|
|
|
- if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
|
|
|
+ }
|
|
|
+ if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
|
|
|
throw new IOException(
|
|
|
"Cannot complete block: block has not been COMMITTED by the client");
|
|
|
- BlockInfoContiguous completeBlock = ucBlock.convertToCompleteBlock();
|
|
|
+ }
|
|
|
+
|
|
|
+ final BlockInfo completeBlock = BlockInfo.convertToCompleteBlock(curBlock);
|
|
|
// replace penultimate block in file
|
|
|
bc.setBlock(blkIndex, completeBlock);
|
|
|
|
|
@@ -679,10 +681,9 @@ public class BlockManager {
|
|
|
return blocksMap.replaceBlock(completeBlock);
|
|
|
}
|
|
|
|
|
|
- // TODO: support BlockInfoStrippedUC
|
|
|
private BlockInfo completeBlock(final BlockCollection bc,
|
|
|
final BlockInfo block, boolean force) throws IOException {
|
|
|
- BlockInfoContiguous[] fileBlocks = bc.getBlocks();
|
|
|
+ BlockInfo[] fileBlocks = bc.getBlocks();
|
|
|
for (int idx = 0; idx < fileBlocks.length; idx++) {
|
|
|
if (fileBlocks[idx] == block) {
|
|
|
return completeBlock(bc, idx, force);
|
|
@@ -698,6 +699,7 @@ public class BlockManager {
|
|
|
*/
|
|
|
public BlockInfo forceCompleteBlock(final BlockCollection bc,
|
|
|
final BlockInfoContiguousUnderConstruction block) throws IOException {
|
|
|
+ // TODO: support BlockInfoStripedUC for editlog
|
|
|
block.commitBlock(block);
|
|
|
return completeBlock(bc, block, true);
|
|
|
}
|
|
@@ -719,7 +721,7 @@ public class BlockManager {
|
|
|
*/
|
|
|
public LocatedBlock convertLastBlockToUnderConstruction(
|
|
|
BlockCollection bc, long bytesToRemove) throws IOException {
|
|
|
- BlockInfoContiguous oldBlock = bc.getLastBlock();
|
|
|
+ BlockInfo oldBlock = bc.getLastBlock();
|
|
|
if(oldBlock == null ||
|
|
|
bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
|
|
|
return null;
|
|
@@ -728,8 +730,10 @@ public class BlockManager {
|
|
|
|
|
|
DatanodeStorageInfo[] targets = getStorages(oldBlock);
|
|
|
|
|
|
- BlockInfoContiguousUnderConstruction ucBlock = bc.setLastBlock(oldBlock,
|
|
|
- targets);
|
|
|
+ // convert the last block to UC
|
|
|
+ bc.convertLastBlockToUC(oldBlock, targets);
|
|
|
+ // get the new created uc block
|
|
|
+ BlockInfo ucBlock = bc.getLastBlock();
|
|
|
blocksMap.replaceBlock(ucBlock);
|
|
|
|
|
|
// Remove block from replication queue.
|
|
@@ -771,11 +775,10 @@ public class BlockManager {
|
|
|
return locations;
|
|
|
}
|
|
|
|
|
|
- private List<LocatedBlock> createLocatedBlockList(
|
|
|
- final BlockInfoContiguous[] blocks,
|
|
|
+ private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
|
|
|
final long offset, final long length, final int nrBlocksToReturn,
|
|
|
final AccessMode mode) throws IOException {
|
|
|
- int curBlk = 0;
|
|
|
+ int curBlk;
|
|
|
long curPos = 0, blkSize = 0;
|
|
|
int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
|
|
|
for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
|
|
@@ -788,10 +791,10 @@ public class BlockManager {
|
|
|
}
|
|
|
|
|
|
if (nrBlocks > 0 && curBlk == nrBlocks) // offset >= end of file
|
|
|
- return Collections.<LocatedBlock>emptyList();
|
|
|
+ return Collections.emptyList();
|
|
|
|
|
|
long endOff = offset + length;
|
|
|
- List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
|
|
|
+ List<LocatedBlock> results = new ArrayList<>(blocks.length);
|
|
|
do {
|
|
|
results.add(createLocatedBlock(blocks[curBlk], curPos, mode));
|
|
|
curPos += blocks[curBlk].getNumBytes();
|
|
@@ -802,9 +805,9 @@ public class BlockManager {
|
|
|
return results;
|
|
|
}
|
|
|
|
|
|
- private LocatedBlock createLocatedBlock(final BlockInfoContiguous[] blocks,
|
|
|
+ private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
|
|
|
final long endPos, final AccessMode mode) throws IOException {
|
|
|
- int curBlk = 0;
|
|
|
+ int curBlk;
|
|
|
long curPos = 0;
|
|
|
int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
|
|
|
for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
|
|
@@ -817,8 +820,8 @@ public class BlockManager {
|
|
|
|
|
|
return createLocatedBlock(blocks[curBlk], curPos, mode);
|
|
|
}
|
|
|
-
|
|
|
- private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos,
|
|
|
+
|
|
|
+ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
|
|
|
final AccessMode mode) throws IOException {
|
|
|
final LocatedBlock lb = createLocatedBlock(blk, pos);
|
|
|
if (mode != null) {
|
|
@@ -828,8 +831,8 @@ public class BlockManager {
|
|
|
}
|
|
|
|
|
|
/** @return a LocatedBlock for the given block */
|
|
|
- private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos
|
|
|
- ) throws IOException {
|
|
|
+ private LocatedBlock createLocatedBlock(final BlockInfo blk,
|
|
|
+ final long pos) throws IOException {
|
|
|
if (blk instanceof BlockInfoContiguousUnderConstruction) {
|
|
|
if (blk.isComplete()) {
|
|
|
throw new IOException(
|
|
@@ -842,6 +845,7 @@ public class BlockManager {
|
|
|
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
|
|
|
return newLocatedBlock(eb, storages, pos, false);
|
|
|
}
|
|
|
+ // TODO support BlockInfoStripedUC
|
|
|
|
|
|
// get block locations
|
|
|
final int numCorruptNodes = countNodes(blk).corruptReplicas();
|
|
@@ -877,7 +881,7 @@ public class BlockManager {
|
|
|
}
|
|
|
|
|
|
/** Create a LocatedBlocks. */
|
|
|
- public LocatedBlocks createLocatedBlocks(final BlockInfoContiguous[] blocks,
|
|
|
+ public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks,
|
|
|
final long fileSizeExcludeBlocksUnderConstruction,
|
|
|
final boolean isFileUnderConstruction, final long offset,
|
|
|
final long length, final boolean needBlockToken,
|
|
@@ -900,7 +904,7 @@ public class BlockManager {
|
|
|
final LocatedBlock lastlb;
|
|
|
final boolean isComplete;
|
|
|
if (!inSnapshot) {
|
|
|
- final BlockInfoContiguous last = blocks[blocks.length - 1];
|
|
|
+ final BlockInfo last = blocks[blocks.length - 1];
|
|
|
final long lastPos = last.isComplete()?
|
|
|
fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
|
|
|
: fileSizeExcludeBlocksUnderConstruction;
|
|
@@ -1724,12 +1728,15 @@ public class BlockManager {
|
|
|
* reported by the datanode in the block report.
|
|
|
*/
|
|
|
static class StatefulBlockInfo {
|
|
|
- final BlockInfoContiguousUnderConstruction storedBlock;
|
|
|
+ final BlockInfo storedBlock; // should be UC block
|
|
|
final Block reportedBlock;
|
|
|
final ReplicaState reportedState;
|
|
|
|
|
|
- StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
|
|
|
+ StatefulBlockInfo(BlockInfo storedBlock,
|
|
|
Block reportedBlock, ReplicaState reportedState) {
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ storedBlock instanceof BlockInfoContiguousUnderConstruction ||
|
|
|
+ storedBlock instanceof BlockInfoStripedUnderConstruction);
|
|
|
this.storedBlock = storedBlock;
|
|
|
this.reportedBlock = reportedBlock;
|
|
|
this.reportedState = reportedState;
|
|
@@ -2133,15 +2140,12 @@ public class BlockManager {
|
|
|
|
|
|
// If block is under construction, add this replica to its list
|
|
|
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
|
|
|
- ((BlockInfoContiguousUnderConstruction) storedBlock)
|
|
|
- .addReplicaIfNotPresent(storageInfo, iblk, reportedState);
|
|
|
+ BlockInfo.addReplica(storedBlock, 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
|
|
|
- BlockInfoContiguousUnderConstruction blockUC =
|
|
|
- (BlockInfoContiguousUnderConstruction) storedBlock;
|
|
|
- if (namesystem.isInSnapshot(blockUC)) {
|
|
|
- int numOfReplicas = blockUC.getNumExpectedLocations();
|
|
|
+ if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
|
|
|
+ int numOfReplicas = BlockInfo.getNumExpectedLocations(storedBlock);
|
|
|
namesystem.incrementSafeBlockCount(numOfReplicas);
|
|
|
}
|
|
|
//and fall through to next clause
|
|
@@ -2164,7 +2168,7 @@ public class BlockManager {
|
|
|
// place a delimiter in the list which separates blocks
|
|
|
// that have been reported from those that have not
|
|
|
Block delimiterBlock = new Block();
|
|
|
- BlockInfoContiguous delimiter = new BlockInfoContiguous(delimiterBlock,
|
|
|
+ BlockInfo delimiter = new BlockInfoContiguous(delimiterBlock,
|
|
|
(short) 1);
|
|
|
AddBlockResult result = storageInfo.addBlock(delimiter, delimiterBlock);
|
|
|
assert result == AddBlockResult.ADDED
|
|
@@ -2294,9 +2298,8 @@ public class BlockManager {
|
|
|
}
|
|
|
|
|
|
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
|
|
|
- toUC.add(new StatefulBlockInfo(
|
|
|
- (BlockInfoContiguousUnderConstruction) storedBlock,
|
|
|
- new Block(block), reportedState));
|
|
|
+ toUC.add(new StatefulBlockInfo(storedBlock, new Block(block),
|
|
|
+ reportedState));
|
|
|
return storedBlock;
|
|
|
}
|
|
|
|
|
@@ -2487,9 +2490,8 @@ public class BlockManager {
|
|
|
|
|
|
void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
|
|
|
DatanodeStorageInfo storageInfo) throws IOException {
|
|
|
- BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock;
|
|
|
- block.addReplicaIfNotPresent(
|
|
|
- storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
|
|
|
+ BlockInfo block = ucBlock.storedBlock;
|
|
|
+ BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
|
|
|
|
|
|
if (ucBlock.reportedState == ReplicaState.FINALIZED &&
|
|
|
(block.findStorageInfo(storageInfo) < 0)) {
|
|
@@ -2549,7 +2551,8 @@ public class BlockManager {
|
|
|
assert block != null && namesystem.hasWriteLock();
|
|
|
BlockInfo storedBlock;
|
|
|
DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
|
|
|
- if (block instanceof BlockInfoContiguousUnderConstruction) {
|
|
|
+ if (block instanceof BlockInfoContiguousUnderConstruction ||
|
|
|
+ block instanceof BlockInfoStripedUnderConstruction) {
|
|
|
//refresh our copy in case the block got completed in another thread
|
|
|
storedBlock = getStoredBlock(block);
|
|
|
} else {
|
|
@@ -2565,7 +2568,6 @@ public class BlockManager {
|
|
|
return block;
|
|
|
}
|
|
|
BlockCollection bc = storedBlock.getBlockCollection();
|
|
|
- assert bc != null : "Block must belong to a file";
|
|
|
|
|
|
// add block to the datanode
|
|
|
AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock);
|
|
@@ -3493,8 +3495,8 @@ public class BlockManager {
|
|
|
* replicated.
|
|
|
*/
|
|
|
public boolean checkBlocksProperlyReplicated(
|
|
|
- String src, BlockInfoContiguous[] blocks) {
|
|
|
- for (BlockInfoContiguous b: blocks) {
|
|
|
+ String src, BlockInfo[] blocks) {
|
|
|
+ for (BlockInfo b: blocks) {
|
|
|
if (!b.isComplete()) {
|
|
|
final BlockInfoContiguousUnderConstruction uc =
|
|
|
(BlockInfoContiguousUnderConstruction)b;
|
|
@@ -3563,7 +3565,7 @@ public class BlockManager {
|
|
|
if (!this.shouldCheckForEnoughRacks) {
|
|
|
return true;
|
|
|
}
|
|
|
- boolean enoughRacks = false;;
|
|
|
+ boolean enoughRacks = false;
|
|
|
Collection<DatanodeDescriptor> corruptNodes =
|
|
|
corruptReplicas.getNodes(b);
|
|
|
int numExpectedReplicas = getReplication(b);
|
|
@@ -3609,21 +3611,15 @@ public class BlockManager {
|
|
|
return this.neededReplications.getCorruptReplOneBlockSize();
|
|
|
}
|
|
|
|
|
|
- public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block,
|
|
|
+ public BlockInfo addBlockCollection(BlockInfo block,
|
|
|
BlockCollection bc) {
|
|
|
- // TODO
|
|
|
- return (BlockInfoContiguous) blocksMap.addBlockCollection(block, bc);
|
|
|
+ return blocksMap.addBlockCollection(block, bc);
|
|
|
}
|
|
|
|
|
|
public BlockCollection getBlockCollection(Block b) {
|
|
|
return blocksMap.getBlockCollection(b);
|
|
|
}
|
|
|
|
|
|
- /** @return an iterator of the datanodes. */
|
|
|
- public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
|
|
|
- return blocksMap.getStorages(block);
|
|
|
- }
|
|
|
-
|
|
|
public int numCorruptReplicas(Block block) {
|
|
|
return corruptReplicas.numCorruptReplicas(block);
|
|
|
}
|
|
@@ -3656,26 +3652,6 @@ public class BlockManager {
|
|
|
public int getCapacity() {
|
|
|
return blocksMap.getCapacity();
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Return a range of corrupt replica block ids. Up to numExpectedBlocks
|
|
|
- * blocks starting at the next block after startingBlockId are returned
|
|
|
- * (fewer if numExpectedBlocks blocks are unavailable). If startingBlockId
|
|
|
- * is null, up to numExpectedBlocks blocks are returned from the beginning.
|
|
|
- * If startingBlockId cannot be found, null is returned.
|
|
|
- *
|
|
|
- * @param numExpectedBlocks Number of block ids to return.
|
|
|
- * 0 <= numExpectedBlocks <= 100
|
|
|
- * @param startingBlockId Block id from which to start. If null, start at
|
|
|
- * beginning.
|
|
|
- * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
|
|
|
- *
|
|
|
- */
|
|
|
- public long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
|
|
|
- Long startingBlockId) {
|
|
|
- return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
|
|
|
- startingBlockId);
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Return an iterator over the set of blocks for which there are no replicas.
|