HDFS-8787. Erasure coding: rename BlockInfoContiguousUC and BlockInfoStripedUC to be consistent with trunk.

This commit is contained in:
Zhe Zhang 2015-07-15 20:13:04 -07:00
parent 7e091de136
commit 4fdd9abd7e
27 changed files with 107 additions and 110 deletions

View File

@ -359,3 +359,6 @@
HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous
blocks in INodeFile. (Zhe Zhang and Yi Liu via zhz) blocks in INodeFile. (Zhe Zhang and Yi Liu via zhz)
HDFS-8787. Erasure coding: rename BlockInfoContiguousUC and BlockInfoStripedUC
to be consistent with trunk. (zhz)

View File

@ -304,8 +304,8 @@ public abstract class BlockInfo extends Block
/** /**
* BlockInfo represents a block that is not being constructed. * BlockInfo represents a block that is not being constructed.
* In order to start modifying the block, the BlockInfo should be converted to * In order to start modifying the block, the BlockInfo should be converted to
* {@link BlockInfoContiguousUnderConstruction} or * {@link BlockInfoUnderConstructionContiguous} or
* {@link BlockInfoStripedUnderConstruction}. * {@link BlockInfoUnderConstructionStriped}.
* @return {@link HdfsServerConstants.BlockUCState#COMPLETE} * @return {@link HdfsServerConstants.BlockUCState#COMPLETE}
*/ */
public HdfsServerConstants.BlockUCState getBlockUCState() { public HdfsServerConstants.BlockUCState getBlockUCState() {

View File

@ -127,18 +127,18 @@ public class BlockInfoContiguous extends BlockInfo {
* Convert a complete block to an under construction block. * Convert a complete block to an under construction block.
* @return BlockInfoUnderConstruction - an under construction block. * @return BlockInfoUnderConstruction - an under construction block.
*/ */
public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction( public BlockInfoUnderConstructionContiguous convertToBlockUnderConstruction(
BlockUCState s, DatanodeStorageInfo[] targets) { BlockUCState s, DatanodeStorageInfo[] targets) {
if(isComplete()) { if(isComplete()) {
BlockInfoContiguousUnderConstruction ucBlock = BlockInfoUnderConstructionContiguous ucBlock =
new BlockInfoContiguousUnderConstruction(this, new BlockInfoUnderConstructionContiguous(this,
getBlockCollection().getPreferredBlockReplication(), s, targets); getBlockCollection().getPreferredBlockReplication(), s, targets);
ucBlock.setBlockCollection(getBlockCollection()); ucBlock.setBlockCollection(getBlockCollection());
return ucBlock; return ucBlock;
} }
// the block is already under construction // the block is already under construction
BlockInfoContiguousUnderConstruction ucBlock = BlockInfoUnderConstructionContiguous ucBlock =
(BlockInfoContiguousUnderConstruction) this; (BlockInfoUnderConstructionContiguous) this;
ucBlock.setBlockUCState(s); ucBlock.setBlockUCState(s);
ucBlock.setExpectedLocations(targets); ucBlock.setExpectedLocations(targets);
ucBlock.setBlockCollection(getBlockCollection()); ucBlock.setBlockCollection(getBlockCollection());

View File

@ -256,16 +256,16 @@ public class BlockInfoStriped extends BlockInfo {
* Convert a complete block to an under construction block. * Convert a complete block to an under construction block.
* @return BlockInfoUnderConstruction - an under construction block. * @return BlockInfoUnderConstruction - an under construction block.
*/ */
public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction( public BlockInfoUnderConstructionStriped convertToBlockUnderConstruction(
BlockUCState s, DatanodeStorageInfo[] targets) { BlockUCState s, DatanodeStorageInfo[] targets) {
final BlockInfoStripedUnderConstruction ucBlock; final BlockInfoUnderConstructionStriped ucBlock;
if(isComplete()) { if(isComplete()) {
ucBlock = new BlockInfoStripedUnderConstruction(this, schema, cellSize, ucBlock = new BlockInfoUnderConstructionStriped(this, schema, cellSize,
s, targets); s, targets);
ucBlock.setBlockCollection(getBlockCollection()); ucBlock.setBlockCollection(getBlockCollection());
} else { } else {
// the block is already under construction // the block is already under construction
ucBlock = (BlockInfoStripedUnderConstruction) this; ucBlock = (BlockInfoUnderConstructionStriped) this;
ucBlock.setBlockUCState(s); ucBlock.setBlockUCState(s);
ucBlock.setExpectedLocations(targets); ucBlock.setExpectedLocations(targets);
ucBlock.setBlockCollection(getBlockCollection()); ucBlock.setBlockCollection(getBlockCollection());

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
* Represents a block that is currently being constructed.<br> * Represents a block that is currently being constructed.<br>
* This is usually the last block of a file opened for write or append. * This is usually the last block of a file opened for write or append.
*/ */
public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous public class BlockInfoUnderConstructionContiguous extends BlockInfoContiguous
implements BlockInfoUnderConstruction{ implements BlockInfoUnderConstruction{
/** Block state. See {@link BlockUCState} */ /** Block state. See {@link BlockUCState} */
private BlockUCState blockUCState; private BlockUCState blockUCState;
@ -64,18 +64,18 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
* Create block and set its state to * Create block and set its state to
* {@link BlockUCState#UNDER_CONSTRUCTION}. * {@link BlockUCState#UNDER_CONSTRUCTION}.
*/ */
public BlockInfoContiguousUnderConstruction(Block blk, short replication) { public BlockInfoUnderConstructionContiguous(Block blk, short replication) {
this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null); this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
} }
/** /**
* Create a block that is currently being constructed. * Create a block that is currently being constructed.
*/ */
public BlockInfoContiguousUnderConstruction(Block blk, short replication, public BlockInfoUnderConstructionContiguous(Block blk, short replication,
BlockUCState state, DatanodeStorageInfo[] targets) { BlockUCState state, DatanodeStorageInfo[] targets) {
super(blk, replication); super(blk, replication);
assert getBlockUCState() != BlockUCState.COMPLETE : assert getBlockUCState() != BlockUCState.COMPLETE :
"BlockInfoContiguousUnderConstruction cannot be in COMPLETE state"; "BlockInfoUnderConstructionContiguous cannot be in COMPLETE state";
this.blockUCState = state; this.blockUCState = state;
setExpectedLocations(targets); setExpectedLocations(targets);
} }
@ -179,7 +179,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
blockRecoveryId = recoveryId; blockRecoveryId = recoveryId;
if (replicas.size() == 0) { if (replicas.size() == 0) {
NameNode.blockStateChangeLog.warn("BLOCK*" NameNode.blockStateChangeLog.warn("BLOCK*"
+ " BlockInfoContiguousUnderConstruction.initLeaseRecovery:" + " BlockInfoUnderConstructionContiguous.initLeaseRecovery:"
+ " No blocks found, lease removed."); + " No blocks found, lease removed.");
} }
boolean allLiveReplicasTriedAsPrimary = true; boolean allLiveReplicasTriedAsPrimary = true;

View File

@ -32,7 +32,7 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCSt
* Represents a striped block that is currently being constructed. * Represents a striped block that is currently being constructed.
* This is usually the last block of a file opened for write or append. * This is usually the last block of a file opened for write or append.
*/ */
public class BlockInfoStripedUnderConstruction extends BlockInfoStriped public class BlockInfoUnderConstructionStriped extends BlockInfoStriped
implements BlockInfoUnderConstruction{ implements BlockInfoUnderConstruction{
private BlockUCState blockUCState; private BlockUCState blockUCState;
@ -57,7 +57,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
/** /**
* Constructor with null storage targets. * Constructor with null storage targets.
*/ */
public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema, public BlockInfoUnderConstructionStriped(Block blk, ECSchema schema,
int cellSize) { int cellSize) {
this(blk, schema, cellSize, UNDER_CONSTRUCTION, null); this(blk, schema, cellSize, UNDER_CONSTRUCTION, null);
} }
@ -65,11 +65,11 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
/** /**
* Create a striped block that is currently being constructed. * Create a striped block that is currently being constructed.
*/ */
public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema, public BlockInfoUnderConstructionStriped(Block blk, ECSchema schema,
int cellSize, BlockUCState state, DatanodeStorageInfo[] targets) { int cellSize, BlockUCState state, DatanodeStorageInfo[] targets) {
super(blk, schema, cellSize); super(blk, schema, cellSize);
assert getBlockUCState() != COMPLETE : assert getBlockUCState() != COMPLETE :
"BlockInfoStripedUnderConstruction cannot be in COMPLETE state"; "BlockInfoUnderConstructionStriped cannot be in COMPLETE state";
this.blockUCState = state; this.blockUCState = state;
setExpectedLocations(targets); setExpectedLocations(targets);
} }
@ -188,7 +188,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
blockRecoveryId = recoveryId; blockRecoveryId = recoveryId;
if (replicas == null || replicas.length == 0) { if (replicas == null || replicas.length == 0) {
NameNode.blockStateChangeLog.warn("BLOCK*" + NameNode.blockStateChangeLog.warn("BLOCK*" +
" BlockInfoStripedUnderConstruction.initLeaseRecovery:" + " BlockInfoUnderConstructionStriped.initLeaseRecovery:" +
" No blocks found, lease removed."); " No blocks found, lease removed.");
// sets primary node index and return. // sets primary node index and return.
primaryNodeIndex = -1; primaryNodeIndex = -1;

View File

@ -872,17 +872,17 @@ public class BlockManager {
private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) { private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) {
if (!blk.isComplete()) { if (!blk.isComplete()) {
if (blk.isStriped()) { if (blk.isStriped()) {
final BlockInfoStripedUnderConstruction uc = final BlockInfoUnderConstructionStriped uc =
(BlockInfoStripedUnderConstruction) blk; (BlockInfoUnderConstructionStriped) blk;
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
blk); blk);
return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos, return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
false); false);
} else { } else {
assert blk instanceof BlockInfoContiguousUnderConstruction; assert blk instanceof BlockInfoUnderConstructionContiguous;
final BlockInfoContiguousUnderConstruction uc = final BlockInfoUnderConstructionContiguous uc =
(BlockInfoContiguousUnderConstruction) blk; (BlockInfoUnderConstructionContiguous) blk;
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
blk); blk);
@ -1863,8 +1863,8 @@ public class BlockManager {
StatefulBlockInfo(BlockInfo storedBlock, StatefulBlockInfo(BlockInfo storedBlock,
Block reportedBlock, ReplicaState reportedState) { Block reportedBlock, ReplicaState reportedState) {
Preconditions.checkArgument( Preconditions.checkArgument(
storedBlock instanceof BlockInfoContiguousUnderConstruction || storedBlock instanceof BlockInfoUnderConstructionContiguous ||
storedBlock instanceof BlockInfoStripedUnderConstruction); storedBlock instanceof BlockInfoUnderConstructionStriped);
this.storedBlock = storedBlock; this.storedBlock = storedBlock;
this.reportedBlock = reportedBlock; this.reportedBlock = reportedBlock;
this.reportedState = reportedState; this.reportedState = reportedState;
@ -2692,8 +2692,8 @@ public class BlockManager {
assert block != null && namesystem.hasWriteLock(); assert block != null && namesystem.hasWriteLock();
BlockInfo storedBlock; BlockInfo storedBlock;
DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
if (block instanceof BlockInfoContiguousUnderConstruction || if (block instanceof BlockInfoUnderConstructionContiguous ||
block instanceof BlockInfoStripedUnderConstruction) { block instanceof BlockInfoUnderConstructionStriped) {
//refresh our copy in case the block got completed in another thread //refresh our copy in case the block got completed in another thread
storedBlock = getStoredBlock(block); storedBlock = getStoredBlock(block);
} else { } else {
@ -4118,7 +4118,7 @@ public class BlockManager {
final LocatedBlock lb; final LocatedBlock lb;
if (info.isStriped()) { if (info.isStriped()) {
lb = newLocatedStripedBlock(eb, locs, lb = newLocatedStripedBlock(eb, locs,
((BlockInfoStripedUnderConstruction)info).getBlockIndices(), ((BlockInfoUnderConstructionStriped)info).getBlockIndices(),
offset, false); offset, false);
} else { } else {
lb = newLocatedBlock(eb, locs, offset, false); lb = newLocatedBlock(eb, locs, offset, false);

View File

@ -44,8 +44,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@ -540,7 +540,7 @@ class FSDirWriteFileOp {
// check quota limits and updated space consumed // check quota limits and updated space consumed
fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
numLocations, true); numLocations, true);
blockInfo = new BlockInfoStripedUnderConstruction(block, ecSchema, blockInfo = new BlockInfoUnderConstructionStriped(block, ecSchema,
ecZone.getCellSize(), ecZone.getCellSize(),
HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
} else { } else {
@ -549,7 +549,7 @@ class FSDirWriteFileOp {
fileINode.getPreferredBlockReplication(), true); fileINode.getPreferredBlockReplication(), true);
short numLocations = fileINode.getFileReplication(); short numLocations = fileINode.getFileReplication();
blockInfo = new BlockInfoContiguousUnderConstruction(block, blockInfo = new BlockInfoUnderConstructionContiguous(block,
numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, numLocations, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
targets); targets);
} }

View File

@ -42,14 +42,14 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@ -105,7 +105,6 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter; import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step; import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
import org.apache.hadoop.hdfs.util.Holder; import org.apache.hadoop.hdfs.util.Holder;
import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.util.ChunkedArrayList; import org.apache.hadoop.util.ChunkedArrayList;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
@ -992,10 +991,10 @@ public class FSEditLogLoader {
final BlockInfo newBlockInfo; final BlockInfo newBlockInfo;
boolean isStriped = ecZone != null; boolean isStriped = ecZone != null;
if (isStriped) { if (isStriped) {
newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, newBlockInfo = new BlockInfoUnderConstructionStriped(newBlock,
ecZone.getSchema(), ecZone.getCellSize()); ecZone.getSchema(), ecZone.getCellSize());
} else { } else {
newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock, newBlockInfo = new BlockInfoUnderConstructionContiguous(newBlock,
file.getPreferredBlockReplication()); file.getPreferredBlockReplication());
} }
fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file); fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file);
@ -1078,10 +1077,10 @@ public class FSEditLogLoader {
// what about an old-version fsync() where fsync isn't called // what about an old-version fsync() where fsync isn't called
// until several blocks in? // until several blocks in?
if (isStriped) { if (isStriped) {
newBI = new BlockInfoStripedUnderConstruction(newBlock, newBI = new BlockInfoUnderConstructionStriped(newBlock,
ecZone.getSchema(), ecZone.getCellSize()); ecZone.getSchema(), ecZone.getCellSize());
} else { } else {
newBI = new BlockInfoContiguousUnderConstruction(newBlock, newBI = new BlockInfoUnderConstructionContiguous(newBlock,
file.getPreferredBlockReplication()); file.getPreferredBlockReplication());
} }
} else { } else {

View File

@ -55,7 +55,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature; import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@ -780,7 +780,7 @@ public class FSImageFormat {
if (blocks.length > 0) { if (blocks.length > 0) {
Block lastBlk = blocks[blocks.length - 1]; Block lastBlk = blocks[blocks.length - 1];
blocks[blocks.length - 1] = blocks[blocks.length - 1] =
new BlockInfoContiguousUnderConstruction(lastBlk, replication); new BlockInfoUnderConstructionContiguous(lastBlk, replication);
} }
} }
} }

View File

@ -44,9 +44,9 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper; import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@ -373,10 +373,10 @@ public final class FSImageFormatPBINode {
final BlockInfo ucBlk; final BlockInfo ucBlk;
if (isStriped) { if (isStriped) {
BlockInfoStriped striped = (BlockInfoStriped) lastBlk; BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
ucBlk = new BlockInfoStripedUnderConstruction(striped, ucBlk = new BlockInfoUnderConstructionStriped(striped,
schema, (int)f.getStripingCellSize()); schema, (int)f.getStripingCellSize());
} else { } else {
ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, ucBlk = new BlockInfoUnderConstructionContiguous(lastBlk,
replication); replication);
} }
file.setBlock(file.numBlocks() - 1, ucBlk); file.setBlock(file.numBlocks() - 1, ucBlk);

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.LayoutVersion; import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@ -138,7 +138,7 @@ public class FSImageSerialization {
// last block is UNDER_CONSTRUCTION // last block is UNDER_CONSTRUCTION
if(numBlocks > 0) { if(numBlocks > 0) {
blk.readFields(in); blk.readFields(in);
blocksContiguous[i] = new BlockInfoContiguousUnderConstruction( blocksContiguous[i] = new BlockInfoUnderConstructionContiguous(
blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null); blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
} }

View File

@ -207,7 +207,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@ -2041,7 +2041,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final BlockInfo last = file.getLastBlock(); final BlockInfo last = file.getLastBlock();
if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) { if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
final Block truncateBlock final Block truncateBlock
= ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock(); = ((BlockInfoUnderConstructionContiguous)last).getTruncateBlock();
if (truncateBlock != null) { if (truncateBlock != null) {
final long truncateLength = file.computeFileSize(false, false) final long truncateLength = file.computeFileSize(false, false)
+ truncateBlock.getNumBytes(); + truncateBlock.getNumBytes();
@ -2124,11 +2124,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock))); nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
} }
BlockInfoContiguousUnderConstruction truncatedBlockUC; BlockInfoUnderConstructionContiguous truncatedBlockUC;
if(shouldCopyOnTruncate) { if(shouldCopyOnTruncate) {
// Add new truncateBlock into blocksMap and // Add new truncateBlock into blocksMap and
// use oldBlock as a source for copy-on-truncate recovery // use oldBlock as a source for copy-on-truncate recovery
truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock, truncatedBlockUC = new BlockInfoUnderConstructionContiguous(newBlock,
file.getPreferredBlockReplication()); file.getPreferredBlockReplication());
truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta); truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
truncatedBlockUC.setTruncateBlock(oldBlock); truncatedBlockUC.setTruncateBlock(oldBlock);
@ -2145,7 +2145,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta); blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
oldBlock = file.getLastBlock(); oldBlock = file.getLastBlock();
assert !oldBlock.isComplete() : "oldBlock should be under construction"; assert !oldBlock.isComplete() : "oldBlock should be under construction";
truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock; truncatedBlockUC = (BlockInfoUnderConstructionContiguous) oldBlock;
truncatedBlockUC.setTruncateBlock(new Block(oldBlock)); truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
truncatedBlockUC.getTruncateBlock().setNumBytes( truncatedBlockUC.getTruncateBlock().setNumBytes(
oldBlock.getNumBytes() - lastBlockDelta); oldBlock.getNumBytes() - lastBlockDelta);

View File

@ -21,9 +21,6 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
/** /**

View File

@ -23,7 +23,7 @@ import java.util.List;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@ -136,7 +136,7 @@ public class FileDiffList extends
Block dontRemoveBlock = null; Block dontRemoveBlock = null;
if (lastBlock != null && lastBlock.getBlockUCState().equals( if (lastBlock != null && lastBlock.getBlockUCState().equals(
HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) { HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock) dontRemoveBlock = ((BlockInfoUnderConstructionContiguous) lastBlock)
.getTruncateBlock(); .getTruncateBlock();
} }
// Collect the remaining blocks of the file, ignoring truncate block // Collect the remaining blocks of the file, ignoring truncate block

View File

@ -116,7 +116,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@ -1626,9 +1626,9 @@ public class DFSTestUtil {
BlockInfo storedBlock = fsn.getStoredBlock(blk.getLocalBlock()); BlockInfo storedBlock = fsn.getStoredBlock(blk.getLocalBlock());
assertTrue("Block " + blk + " should be under construction, " + assertTrue("Block " + blk + " should be under construction, " +
"got: " + storedBlock, "got: " + storedBlock,
storedBlock instanceof BlockInfoContiguousUnderConstruction); storedBlock instanceof BlockInfoUnderConstructionContiguous);
BlockInfoContiguousUnderConstruction ucBlock = BlockInfoUnderConstructionContiguous ucBlock =
(BlockInfoContiguousUnderConstruction)storedBlock; (BlockInfoUnderConstructionContiguous)storedBlock;
// We expect that the replica with the most recent heart beat will be // We expect that the replica with the most recent heart beat will be
// the one to be in charge of the synchronization / recovery protocol. // the one to be in charge of the synchronization / recovery protocol.
final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations(); final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();

View File

@ -23,7 +23,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.util.Time;
import org.junit.Test; import org.junit.Test;
/** /**
@ -40,7 +39,7 @@ public class TestBlockInfoUnderConstruction {
DatanodeDescriptor dd3 = s3.getDatanodeDescriptor(); DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
dd1.isAlive = dd2.isAlive = dd3.isAlive = true; dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( BlockInfoUnderConstructionContiguous blockInfo = new BlockInfoUnderConstructionContiguous(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
(short) 3, (short) 3,
BlockUCState.UNDER_CONSTRUCTION, BlockUCState.UNDER_CONSTRUCTION,

View File

@ -727,7 +727,7 @@ public class TestBlockManager {
// verify the storage info is correct // verify the storage info is correct
assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo
(ds) >= 0); (ds) >= 0);
assertTrue(((BlockInfoContiguousUnderConstruction) bm. assertTrue(((BlockInfoUnderConstructionContiguous) bm.
getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0); getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0);
assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId)) assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId))
.findStorageInfo(ds) >= 0); .findStorageInfo(ds) >= 0);
@ -748,8 +748,8 @@ public class TestBlockManager {
private BlockInfoContiguous addUcBlockToBM(long blkId) { private BlockInfoContiguous addUcBlockToBM(long blkId) {
Block block = new Block(blkId); Block block = new Block(blkId);
BlockInfoContiguousUnderConstruction blockInfo = BlockInfoUnderConstructionContiguous blockInfo =
new BlockInfoContiguousUnderConstruction(block, (short) 3); new BlockInfoUnderConstructionContiguous(block, (short) 3);
BlockCollection bc = Mockito.mock(BlockCollection.class); BlockCollection bc = Mockito.mock(BlockCollection.class);
Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication(); Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
bm.blocksMap.addBlockCollection(blockInfo, bc); bm.blocksMap.addBlockCollection(blockInfo, bc);

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.Time;
import org.junit.Test; import org.junit.Test;
/** /**
@ -173,7 +172,7 @@ public class TestHeartbeatHandling {
dd1.getStorageInfos()[0], dd1.getStorageInfos()[0],
dd2.getStorageInfos()[0], dd2.getStorageInfos()[0],
dd3.getStorageInfos()[0]}; dd3.getStorageInfos()[0]};
BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( BlockInfoUnderConstructionContiguous blockInfo = new BlockInfoUnderConstructionContiguous(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages); BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo); dd1.addBlockToBeRecovered(blockInfo);
@ -195,7 +194,7 @@ public class TestHeartbeatHandling {
// More than the default stale interval of 30 seconds. // More than the default stale interval of 30 seconds.
DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000);
DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0); DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
blockInfo = new BlockInfoContiguousUnderConstruction( blockInfo = new BlockInfoUnderConstructionContiguous(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages); BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo); dd1.addBlockToBeRecovered(blockInfo);
@ -216,7 +215,7 @@ public class TestHeartbeatHandling {
// More than the default stale interval of 30 seconds. // More than the default stale interval of 30 seconds.
DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000);
DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000); DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000);
blockInfo = new BlockInfoContiguousUnderConstruction( blockInfo = new BlockInfoUnderConstructionContiguous(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3, new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages); BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo); dd1.addBlockToBeRecovered(blockInfo);

View File

@ -1182,7 +1182,7 @@ public class TestReplicationPolicy {
// block under construction, the BlockManager will realize the expected // block under construction, the BlockManager will realize the expected
// replication has been achieved and remove it from the under-replicated // replication has been achieved and remove it from the under-replicated
// queue. // queue.
BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1); BlockInfoUnderConstructionContiguous info = new BlockInfoUnderConstructionContiguous(block1, (short) 1);
BlockCollection bc = mock(BlockCollection.class); BlockCollection bc = mock(BlockCollection.class);
when(bc.getPreferredBlockReplication()).thenReturn((short)1); when(bc.getPreferredBlockReplication()).thenReturn((short)1);
bm.addBlockCollection(info, bc); bm.addBlockCollection(info, bc);
@ -1247,7 +1247,7 @@ public class TestReplicationPolicy {
DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo( DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
dataNodes[0], new DatanodeStorage("s1"))}; dataNodes[0], new DatanodeStorage("s1"))};
final BlockInfoContiguousUnderConstruction ucBlock = final BlockInfoUnderConstructionContiguous ucBlock =
info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
storageAry); storageAry);
DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class); DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class);

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@ -170,8 +170,8 @@ public class TestAddStripedBlocks {
Assert.assertEquals(0, Assert.assertEquals(0,
block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
final BlockInfoStripedUnderConstruction blockUC = final BlockInfoUnderConstructionStriped blockUC =
(BlockInfoStripedUnderConstruction) block; (BlockInfoUnderConstructionStriped) block;
Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
blockUC.getBlockUCState()); blockUC.getBlockUCState());
if (checkReplica) { if (checkReplica) {
@ -205,8 +205,8 @@ public class TestAddStripedBlocks {
FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
BlockInfoStripedUnderConstruction lastBlk = BlockInfoUnderConstructionStriped lastBlk =
(BlockInfoStripedUnderConstruction) fileNode.getLastBlock(); (BlockInfoUnderConstructionStriped) fileNode.getLastBlock();
DatanodeInfo[] expectedDNs = DatanodeStorageInfo DatanodeInfo[] expectedDNs = DatanodeStorageInfo
.toDatanodeInfos(lastBlk.getExpectedStorageLocations()); .toDatanodeInfos(lastBlk.getExpectedStorageLocations());
int[] indices = lastBlk.getBlockIndices(); int[] indices = lastBlk.getBlockIndices();
@ -228,7 +228,7 @@ public class TestAddStripedBlocks {
} }
/** /**
* Test BlockInfoStripedUnderConstruction#addReplicaIfNotPresent in different * Test BlockInfoUnderConstructionStriped#addReplicaIfNotPresent in different
* scenarios. * scenarios.
*/ */
@Test @Test
@ -246,8 +246,8 @@ public class TestAddStripedBlocks {
cluster.getNamesystem().getAdditionalBlock(file.toString(), cluster.getNamesystem().getAdditionalBlock(file.toString(),
fileNode.getId(), dfs.getClient().getClientName(), null, null, null); fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
BlockInfo lastBlock = fileNode.getLastBlock(); BlockInfo lastBlock = fileNode.getLastBlock();
BlockInfoStripedUnderConstruction ucBlock = BlockInfoUnderConstructionStriped ucBlock =
(BlockInfoStripedUnderConstruction) lastBlock; (BlockInfoUnderConstructionStriped) lastBlock;
DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations(); DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
int[] indices = ucBlock.getBlockIndices(); int[] indices = ucBlock.getBlockIndices();
@ -255,7 +255,7 @@ public class TestAddStripedBlocks {
Assert.assertEquals(GROUP_SIZE, indices.length); Assert.assertEquals(GROUP_SIZE, indices.length);
// 2. mimic incremental block reports and make sure the uc-replica list in // 2. mimic incremental block reports and make sure the uc-replica list in
// the BlockStripedUC is correct // the BlockInfoUCStriped is correct
int i = 0; int i = 0;
for (DataNode dn : cluster.getDataNodes()) { for (DataNode dn : cluster.getDataNodes()) {
final Block block = new Block(lastBlock.getBlockId() + i++, final Block block = new Block(lastBlock.getBlockId() + i++,
@ -307,8 +307,8 @@ public class TestAddStripedBlocks {
bpId, reports, null); bpId, reports, null);
} }
BlockInfoStripedUnderConstruction ucBlock = BlockInfoUnderConstructionStriped ucBlock =
(BlockInfoStripedUnderConstruction) lastBlock; (BlockInfoUnderConstructionStriped) lastBlock;
DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations(); DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
int[] indices = ucBlock.getBlockIndices(); int[] indices = ucBlock.getBlockIndices();
Assert.assertEquals(GROUP_SIZE, locs.length); Assert.assertEquals(GROUP_SIZE, locs.length);

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -170,7 +170,7 @@ public class TestBlockUnderConstruction {
final List<LocatedBlock> blocks = lb.getLocatedBlocks(); final List<LocatedBlock> blocks = lb.getLocatedBlocks();
assertEquals(i, blocks.size()); assertEquals(i, blocks.size());
final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock(); final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock();
assertTrue(b instanceof BlockInfoContiguousUnderConstruction); assertTrue(b instanceof BlockInfoUnderConstructionContiguous);
if (++i < NUM_BLOCKS) { if (++i < NUM_BLOCKS) {
// write one more block // write one more block

View File

@ -23,7 +23,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.junit.Test; import org.junit.Test;
@ -67,7 +67,7 @@ public class TestCommitBlockSynchronization {
namesystem.dir.getINodeMap().put(file); namesystem.dir.getINodeMap().put(file);
FSNamesystem namesystemSpy = spy(namesystem); FSNamesystem namesystemSpy = spy(namesystem);
BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction( BlockInfoUnderConstructionContiguous blockInfo = new BlockInfoUnderConstructionContiguous(
block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
blockInfo.setBlockCollection(file); blockInfo.setBlockCollection(file);
blockInfo.setGenerationStamp(genStamp); blockInfo.setGenerationStamp(genStamp);

View File

@ -55,7 +55,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
@ -1020,7 +1020,7 @@ public class TestFileTruncate {
is(fsn.getBlockIdManager().getGenerationStampV2())); is(fsn.getBlockIdManager().getGenerationStampV2()));
assertThat(file.getLastBlock().getBlockUCState(), assertThat(file.getLastBlock().getBlockUCState(),
is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY)); is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock()) long blockRecoveryId = ((BlockInfoUnderConstructionContiguous) file.getLastBlock())
.getBlockRecoveryId(); .getBlockRecoveryId();
assertThat(blockRecoveryId, is(initialGenStamp + 1)); assertThat(blockRecoveryId, is(initialGenStamp + 1));
fsn.getEditLog().logTruncate( fsn.getEditLog().logTruncate(
@ -1054,7 +1054,7 @@ public class TestFileTruncate {
is(fsn.getBlockIdManager().getGenerationStampV2())); is(fsn.getBlockIdManager().getGenerationStampV2()));
assertThat(file.getLastBlock().getBlockUCState(), assertThat(file.getLastBlock().getBlockUCState(),
is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY)); is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock()) long blockRecoveryId = ((BlockInfoUnderConstructionContiguous) file.getLastBlock())
.getBlockRecoveryId(); .getBlockRecoveryId();
assertThat(blockRecoveryId, is(initialGenStamp + 1)); assertThat(blockRecoveryId, is(initialGenStamp + 1));
fsn.getEditLog().logTruncate( fsn.getEditLog().logTruncate(

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.ECSchema;
@ -146,14 +146,14 @@ public class TestStripedINodeFile {
} }
@Test @Test
public void testBlockStripedUCFileSize() public void testBlockUCStripedFileSize()
throws IOException, InterruptedException { throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile(); INodeFile inf = createStripedINodeFile();
Block blk = new Block(1); Block blk = new Block(1);
BlockInfoStripedUnderConstruction bInfoStripedUC BlockInfoUnderConstructionStriped bInfoUCStriped
= new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize); = new BlockInfoUnderConstructionStriped(blk, testSchema, cellSize);
bInfoStripedUC.setNumBytes(100); bInfoUCStriped.setNumBytes(100);
inf.addBlock(bInfoStripedUC); inf.addBlock(bInfoUCStriped);
assertEquals(100, inf.computeFileSize()); assertEquals(100, inf.computeFileSize());
assertEquals(0, inf.computeFileSize(false, false)); assertEquals(0, inf.computeFileSize(false, false));
} }
@ -180,21 +180,21 @@ public class TestStripedINodeFile {
} }
@Test @Test
public void testBlockStripedUCComputeQuotaUsage() public void testBlockUCStripedComputeQuotaUsage()
throws IOException, InterruptedException { throws IOException, InterruptedException {
INodeFile inf = createStripedINodeFile(); INodeFile inf = createStripedINodeFile();
Block blk = new Block(1); Block blk = new Block(1);
BlockInfoStripedUnderConstruction bInfoStripedUC BlockInfoUnderConstructionStriped bInfoUCStriped
= new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize); = new BlockInfoUnderConstructionStriped(blk, testSchema, cellSize);
bInfoStripedUC.setNumBytes(100); bInfoUCStriped.setNumBytes(100);
inf.addBlock(bInfoStripedUC); inf.addBlock(bInfoUCStriped);
QuotaCounts counts QuotaCounts counts
= inf.computeQuotaUsageWithStriped(defaultPolicy, = inf.computeQuotaUsageWithStriped(defaultPolicy,
new QuotaCounts.Builder().build()); new QuotaCounts.Builder().build());
assertEquals(1024, inf.getPreferredBlockSize()); assertEquals(1024, inf.getPreferredBlockSize());
assertEquals(1, counts.getNameSpace()); assertEquals(1, counts.getNameSpace());
// Consumed space in the case of BlockInfoStripedUC can be calculated // Consumed space in the case of BlockInfoUCStriped can be calculated
// by using preferred block size. This is 1024 and total block num // by using preferred block size. This is 1024 and total block num
// is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216. // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216.
assertEquals(9216, counts.getStorageSpace()); assertEquals(9216, counts.getStorageSpace());

View File

@ -72,7 +72,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@ -744,8 +744,8 @@ public class TestRetryCacheWithHA {
boolean checkNamenodeBeforeReturn() throws Exception { boolean checkNamenodeBeforeReturn() throws Exception {
INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory() INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory()
.getINode4Write(file).asFile(); .getINode4Write(file).asFile();
BlockInfoContiguousUnderConstruction blkUC = BlockInfoUnderConstructionContiguous blkUC =
(BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1]; (BlockInfoUnderConstructionContiguous) (fileNode.getBlocks())[1];
int datanodeNum = blkUC.getExpectedStorageLocations().length; int datanodeNum = blkUC.getExpectedStorageLocations().length;
for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) { for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) {
Thread.sleep(1000); Thread.sleep(1000);

View File

@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage; import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
import org.apache.hadoop.hdfs.server.datanode.BlockScanner; import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
@ -177,7 +177,7 @@ public class SnapshotTestHelper {
* Specific information for different types of INode: * Specific information for different types of INode:
* {@link INodeDirectory}:childrenSize * {@link INodeDirectory}:childrenSize
* {@link INodeFile}: fileSize, block list. Check {@link BlockInfoContiguous#toString()} * {@link INodeFile}: fileSize, block list. Check {@link BlockInfoContiguous#toString()}
* and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information. * and {@link BlockInfoUnderConstructionContiguous#toString()} for detailed information.
* {@link FileWithSnapshot}: next link * {@link FileWithSnapshot}: next link
* </pre> * </pre>
* @see INode#dumpTreeRecursively() * @see INode#dumpTreeRecursively()