HDFS-7743. Code cleanup of BlockInfo and rename BlockInfo to BlockInfoContiguous. Contributed by Jing Zhao.

(cherry picked from commit 1382ae525c)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
This commit is contained in:
Jing Zhao 2015-02-08 11:51:44 -08:00
parent 6933975143
commit 1e49b40f1f
47 changed files with 449 additions and 432 deletions

View File

@ -307,6 +307,9 @@ Release 2.7.0 - UNRELEASED
tests such as truncate with HA setup, negative tests, truncate with other
operations and multiple truncates. (szetszwo)
HDFS-7743. Code cleanup of BlockInfo and rename BlockInfo to
BlockInfoContiguous. (jing9)
OPTIMIZATIONS
HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

View File

@ -31,7 +31,7 @@ public interface BlockCollection {
/**
* Get the last block of the collection.
*/
public BlockInfo getLastBlock();
public BlockInfoContiguous getLastBlock();
/**
* Get content summary.
@ -46,7 +46,7 @@ public interface BlockCollection {
/**
* Get the blocks.
*/
public BlockInfo[] getBlocks();
public BlockInfoContiguous[] getBlocks();
/**
* Get preferred block size for the collection
@ -73,14 +73,14 @@ public interface BlockCollection {
/**
* Set the block at the given index.
*/
public void setBlock(int index, BlockInfo blk);
public void setBlock(int index, BlockInfoContiguous blk);
/**
* Convert the last block of the collection to an under-construction block
* and set the locations.
*/
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeStorageInfo[] locations) throws IOException;
public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfoContiguous lastBlock,
DatanodeStorageInfo[] targets) throws IOException;
/**
* @return whether the block collection is under construction.

View File

@ -33,8 +33,9 @@ import org.apache.hadoop.util.LightWeightGSet;
* the block are stored.
*/
@InterfaceAudience.Private
public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
public static final BlockInfo[] EMPTY_ARRAY = {};
public class BlockInfoContiguous extends Block
implements LightWeightGSet.LinkedElement {
public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
private BlockCollection bc;
@ -59,12 +60,12 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* Construct an entry for blocksmap
* @param replication the block's replication factor
*/
public BlockInfo(short replication) {
public BlockInfoContiguous(short replication) {
this.triplets = new Object[3*replication];
this.bc = null;
}
public BlockInfo(Block blk, short replication) {
public BlockInfoContiguous(Block blk, short replication) {
super(blk);
this.triplets = new Object[3*replication];
this.bc = null;
@ -75,7 +76,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* This is used to convert BlockInfoUnderConstruction
* @param from BlockInfo to copy from.
*/
protected BlockInfo(BlockInfo from) {
protected BlockInfoContiguous(BlockInfoContiguous from) {
this(from, from.bc.getBlockReplication());
this.bc = from.bc;
}
@ -99,23 +100,23 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
return (DatanodeStorageInfo)triplets[index*3];
}
private BlockInfo getPrevious(int index) {
private BlockInfoContiguous getPrevious(int index) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
BlockInfo info = (BlockInfo)triplets[index*3+1];
BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
assert info == null ||
info.getClass().getName().startsWith(BlockInfo.class.getName()) :
info.getClass().getName().startsWith(BlockInfoContiguous.class.getName()) :
"BlockInfo is expected at " + index*3;
return info;
}
BlockInfo getNext(int index) {
BlockInfoContiguous getNext(int index) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
BlockInfo info = (BlockInfo)triplets[index*3+2];
assert info == null ||
info.getClass().getName().startsWith(BlockInfo.class.getName()) :
"BlockInfo is expected at " + index*3;
BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
assert info == null || info.getClass().getName().startsWith(
BlockInfoContiguous.class.getName()) :
"BlockInfo is expected at " + index*3;
return info;
}
@ -133,10 +134,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* @param to - block to be set to previous on the list of blocks
* @return current previous block on the list of blocks
*/
private BlockInfo setPrevious(int index, BlockInfo to) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
BlockInfo info = (BlockInfo)triplets[index*3+1];
private BlockInfoContiguous setPrevious(int index, BlockInfoContiguous to) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
triplets[index*3+1] = to;
return info;
}
@ -149,10 +150,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* @param to - block to be set to next on the list of blocks
* * @return current next block on the list of blocks
*/
private BlockInfo setNext(int index, BlockInfo to) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
BlockInfo info = (BlockInfo)triplets[index*3+2];
private BlockInfoContiguous setNext(int index, BlockInfoContiguous to) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
triplets[index*3+2] = to;
return info;
}
@ -244,6 +245,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
}
return false;
}
/**
* Find specified DatanodeStorageInfo.
* @return DatanodeStorageInfo or null if not found.
@ -268,10 +270,12 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
int len = getCapacity();
for(int idx = 0; idx < len; idx++) {
DatanodeStorageInfo cur = getStorageInfo(idx);
if(cur == storageInfo)
if (cur == storageInfo) {
return idx;
if(cur == null)
}
if (cur == null) {
break;
}
}
return -1;
}
@ -282,7 +286,8 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* If the head is null then form a new list.
* @return current block as the new head of the list.
*/
BlockInfo listInsert(BlockInfo head, DatanodeStorageInfo storage) {
BlockInfoContiguous listInsert(BlockInfoContiguous head,
DatanodeStorageInfo storage) {
int dnIndex = this.findStorageInfo(storage);
assert dnIndex >= 0 : "Data node is not found: current";
assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
@ -302,15 +307,16 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* @return the new head of the list or null if the list becomes
* empy after deletion.
*/
BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) {
BlockInfoContiguous listRemove(BlockInfoContiguous head,
DatanodeStorageInfo storage) {
if(head == null)
return null;
int dnIndex = this.findStorageInfo(storage);
if(dnIndex < 0) // this block is not on the data-node list
return head;
BlockInfo next = this.getNext(dnIndex);
BlockInfo prev = this.getPrevious(dnIndex);
BlockInfoContiguous next = this.getNext(dnIndex);
BlockInfoContiguous prev = this.getPrevious(dnIndex);
this.setNext(dnIndex, null);
this.setPrevious(dnIndex, null);
if(prev != null)
@ -328,25 +334,26 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
*
* @return the new head of the list.
*/
public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage,
int curIndex, int headIndex) {
public BlockInfoContiguous moveBlockToHead(BlockInfoContiguous head,
DatanodeStorageInfo storage, int curIndex, int headIndex) {
if (head == this) {
return this;
}
BlockInfo next = this.setNext(curIndex, head);
BlockInfo prev = this.setPrevious(curIndex, null);
BlockInfoContiguous next = this.setNext(curIndex, head);
BlockInfoContiguous prev = this.setPrevious(curIndex, null);
head.setPrevious(headIndex, this);
prev.setNext(prev.findStorageInfo(storage), next);
if (next != null)
if (next != null) {
next.setPrevious(next.findStorageInfo(storage), prev);
}
return this;
}
/**
* BlockInfo represents a block that is not being constructed.
* In order to start modifying the block, the BlockInfo should be converted
* to {@link BlockInfoUnderConstruction}.
* to {@link BlockInfoContiguousUnderConstruction}.
* @return {@link BlockUCState#COMPLETE}
*/
public BlockUCState getBlockUCState() {
@ -366,16 +373,18 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* Convert a complete block to an under construction block.
* @return BlockInfoUnderConstruction - an under construction block.
*/
public BlockInfoUnderConstruction convertToBlockUnderConstruction(
public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
BlockUCState s, DatanodeStorageInfo[] targets) {
if(isComplete()) {
BlockInfoUnderConstruction ucBlock = new BlockInfoUnderConstruction(this,
BlockInfoContiguousUnderConstruction ucBlock =
new BlockInfoContiguousUnderConstruction(this,
getBlockCollection().getBlockReplication(), s, targets);
ucBlock.setBlockCollection(getBlockCollection());
return ucBlock;
}
// the block is already under construction
BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;
BlockInfoContiguousUnderConstruction ucBlock =
(BlockInfoContiguousUnderConstruction)this;
ucBlock.setBlockUCState(s);
ucBlock.setExpectedLocations(targets);
ucBlock.setBlockCollection(getBlockCollection());
@ -387,7 +396,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
// Super implementation is sufficient
return super.hashCode();
}
@Override
public boolean equals(Object obj) {
// Sufficient to rely on super's implementation

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
* Represents a block that is currently being constructed.<br>
* This is usually the last block of a file opened for write or append.
*/
public class BlockInfoUnderConstruction extends BlockInfo {
public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
/** Block state. See {@link BlockUCState} */
private BlockUCState blockUCState;
@ -158,16 +158,14 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* Create block and set its state to
* {@link BlockUCState#UNDER_CONSTRUCTION}.
*/
public BlockInfoUnderConstruction(Block blk, short replication) {
public BlockInfoContiguousUnderConstruction(Block blk, short replication) {
this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
}
/**
* Create a block that is currently being constructed.
*/
public BlockInfoUnderConstruction(Block blk, short replication,
BlockUCState state,
DatanodeStorageInfo[] targets) {
public BlockInfoContiguousUnderConstruction(Block blk, short replication, BlockUCState state, DatanodeStorageInfo[] targets) {
super(blk, replication);
assert getBlockUCState() != BlockUCState.COMPLETE :
"BlockInfoUnderConstruction cannot be in COMPLETE state";
@ -184,10 +182,10 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* the client or it does not have at least a minimal number of replicas
* reported from data-nodes.
*/
BlockInfo convertToCompleteBlock() throws IOException {
BlockInfoContiguous convertToCompleteBlock() throws IOException {
assert getBlockUCState() != BlockUCState.COMPLETE :
"Trying to convert a COMPLETE block";
return new BlockInfo(this);
return new BlockInfoContiguous(this);
}
/** Set expected locations */

View File

@ -535,8 +535,8 @@ public class BlockManager {
int usableReplicas = numReplicas.liveReplicas() +
numReplicas.decommissionedReplicas();
if (block instanceof BlockInfo) {
BlockCollection bc = ((BlockInfo) block).getBlockCollection();
if (block instanceof BlockInfoContiguous) {
BlockCollection bc = ((BlockInfoContiguous) block).getBlockCollection();
String fileName = (bc == null) ? "[orphaned]" : bc.getName();
out.print(fileName + ": ");
}
@ -590,8 +590,9 @@ 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 BlockInfoUnderConstruction block,
final Block commitBlock) throws IOException {
private static boolean commitBlock(
final BlockInfoContiguousUnderConstruction block, final Block commitBlock)
throws IOException {
if (block.getBlockUCState() == BlockUCState.COMMITTED)
return false;
assert block.getNumBytes() <= commitBlock.getNumBytes() :
@ -615,13 +616,14 @@ public class BlockManager {
Block commitBlock) throws IOException {
if(commitBlock == null)
return false; // not committing, this is a block allocation retry
BlockInfo lastBlock = bc.getLastBlock();
BlockInfoContiguous 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((BlockInfoUnderConstruction)lastBlock, commitBlock);
final boolean b = commitBlock(
(BlockInfoContiguousUnderConstruction) lastBlock, commitBlock);
if(countNodes(lastBlock).liveReplicas() >= minReplication)
completeBlock(bc, bc.numBlocks()-1, false);
return b;
@ -634,14 +636,15 @@ public class BlockManager {
* @throws IOException if the block does not have at least a minimal number
* of replicas reported from data-nodes.
*/
private BlockInfo completeBlock(final BlockCollection bc,
private BlockInfoContiguous completeBlock(final BlockCollection bc,
final int blkIndex, boolean force) throws IOException {
if(blkIndex < 0)
return null;
BlockInfo curBlock = bc.getBlocks()[blkIndex];
BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex];
if(curBlock.isComplete())
return curBlock;
BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
BlockInfoContiguousUnderConstruction ucBlock =
(BlockInfoContiguousUnderConstruction) curBlock;
int numNodes = ucBlock.numNodes();
if (!force && numNodes < minReplication)
throw new IOException("Cannot complete block: " +
@ -649,7 +652,7 @@ public class BlockManager {
if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
throw new IOException(
"Cannot complete block: block has not been COMMITTED by the client");
BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
BlockInfoContiguous completeBlock = ucBlock.convertToCompleteBlock();
// replace penultimate block in file
bc.setBlock(blkIndex, completeBlock);
@ -667,9 +670,9 @@ public class BlockManager {
return blocksMap.replaceBlock(completeBlock);
}
private BlockInfo completeBlock(final BlockCollection bc,
final BlockInfo block, boolean force) throws IOException {
BlockInfo[] fileBlocks = bc.getBlocks();
private BlockInfoContiguous completeBlock(final BlockCollection bc,
final BlockInfoContiguous block, boolean force) throws IOException {
BlockInfoContiguous[] fileBlocks = bc.getBlocks();
for(int idx = 0; idx < fileBlocks.length; idx++)
if(fileBlocks[idx] == block) {
return completeBlock(bc, idx, force);
@ -682,8 +685,8 @@ public class BlockManager {
* regardless of whether enough replicas are present. This is necessary
* when tailing edit logs as a Standby.
*/
public BlockInfo forceCompleteBlock(final BlockCollection bc,
final BlockInfoUnderConstruction block) throws IOException {
public BlockInfoContiguous forceCompleteBlock(final BlockCollection bc,
final BlockInfoContiguousUnderConstruction block) throws IOException {
block.commitBlock(block);
return completeBlock(bc, block, true);
}
@ -705,7 +708,7 @@ public class BlockManager {
*/
public LocatedBlock convertLastBlockToUnderConstruction(
BlockCollection bc, long bytesToRemove) throws IOException {
BlockInfo oldBlock = bc.getLastBlock();
BlockInfoContiguous oldBlock = bc.getLastBlock();
if(oldBlock == null ||
bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
return null;
@ -714,7 +717,8 @@ public class BlockManager {
DatanodeStorageInfo[] targets = getStorages(oldBlock);
BlockInfoUnderConstruction ucBlock = bc.setLastBlock(oldBlock, targets);
BlockInfoContiguousUnderConstruction ucBlock =
bc.setLastBlock(oldBlock, targets);
blocksMap.replaceBlock(ucBlock);
// Remove block from replication queue.
@ -756,7 +760,8 @@ public class BlockManager {
return locations;
}
private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
private List<LocatedBlock> createLocatedBlockList(
final BlockInfoContiguous[] blocks,
final long offset, final long length, final int nrBlocksToReturn,
final AccessMode mode) throws IOException {
int curBlk = 0;
@ -786,7 +791,7 @@ public class BlockManager {
return results;
}
private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
private LocatedBlock createLocatedBlock(final BlockInfoContiguous[] blocks,
final long endPos, final AccessMode mode) throws IOException {
int curBlk = 0;
long curPos = 0;
@ -802,7 +807,7 @@ public class BlockManager {
return createLocatedBlock(blocks[curBlk], curPos, mode);
}
private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos,
final BlockTokenSecretManager.AccessMode mode) throws IOException {
final LocatedBlock lb = createLocatedBlock(blk, pos);
if (mode != null) {
@ -812,15 +817,16 @@ public class BlockManager {
}
/** @return a LocatedBlock for the given block */
private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos
) throws IOException {
if (blk instanceof BlockInfoUnderConstruction) {
if (blk instanceof BlockInfoContiguousUnderConstruction) {
if (blk.isComplete()) {
throw new IOException(
"blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
+ ", blk=" + blk);
}
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
final BlockInfoContiguousUnderConstruction uc =
(BlockInfoContiguousUnderConstruction) blk;
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
return new LocatedBlock(eb, storages, pos, false);
@ -859,7 +865,7 @@ public class BlockManager {
}
/** Create a LocatedBlocks. */
public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks,
public LocatedBlocks createLocatedBlocks(final BlockInfoContiguous[] blocks,
final long fileSizeExcludeBlocksUnderConstruction,
final boolean isFileUnderConstruction, final long offset,
final long length, final boolean needBlockToken,
@ -882,7 +888,7 @@ public class BlockManager {
final LocatedBlock lastlb;
final boolean isComplete;
if (!inSnapshot) {
final BlockInfo last = blocks[blocks.length - 1];
final BlockInfoContiguous last = blocks[blocks.length - 1];
final long lastPos = last.isComplete()?
fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
: fileSizeExcludeBlocksUnderConstruction;
@ -971,7 +977,7 @@ public class BlockManager {
/**
* Check if a block is replicated to at least the minimum replication.
*/
public boolean isSufficientlyReplicated(BlockInfo b) {
public boolean isSufficientlyReplicated(BlockInfoContiguous b) {
// Compare against the lesser of the minReplication and number of live DNs.
final int replication =
Math.min(minReplication, getDatanodeManager().getNumLiveDataNodes());
@ -1012,7 +1018,7 @@ public class BlockManager {
if(numBlocks == 0) {
return new BlocksWithLocations(new BlockWithLocations[0]);
}
Iterator<BlockInfo> iter = node.getBlockIterator();
Iterator<BlockInfoContiguous> iter = node.getBlockIterator();
int startBlock = DFSUtil.getRandom().nextInt(numBlocks); // starting from a random block
// skip blocks
for(int i=0; i<startBlock; i++) {
@ -1020,7 +1026,7 @@ public class BlockManager {
}
List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
long totalSize = 0;
BlockInfo curBlock;
BlockInfoContiguous curBlock;
while(totalSize<size && iter.hasNext()) {
curBlock = iter.next();
if(!curBlock.isComplete()) continue;
@ -1119,7 +1125,7 @@ public class BlockManager {
public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
final DatanodeInfo dn, String storageID, String reason) throws IOException {
assert namesystem.hasWriteLock();
final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
final BlockInfoContiguous storedBlock = getStoredBlock(blk.getLocalBlock());
if (storedBlock == null) {
// Check if the replica is in the blockMap, if not
// ignore the request for now. This could happen when BlockScanner
@ -1699,11 +1705,11 @@ public class BlockManager {
* reported by the datanode in the block report.
*/
static class StatefulBlockInfo {
final BlockInfoUnderConstruction storedBlock;
final BlockInfoContiguousUnderConstruction storedBlock;
final Block reportedBlock;
final ReplicaState reportedState;
StatefulBlockInfo(BlockInfoUnderConstruction storedBlock,
StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
Block reportedBlock, ReplicaState reportedState) {
this.storedBlock = storedBlock;
this.reportedBlock = reportedBlock;
@ -1717,15 +1723,16 @@ public class BlockManager {
*/
private static class BlockToMarkCorrupt {
/** The corrupted block in a datanode. */
final BlockInfo corrupted;
final BlockInfoContiguous corrupted;
/** The corresponding block stored in the BlockManager. */
final BlockInfo stored;
final BlockInfoContiguous stored;
/** The reason to mark corrupt. */
final String reason;
/** The reason code to be stored */
final Reason reasonCode;
BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason,
BlockToMarkCorrupt(BlockInfoContiguous corrupted,
BlockInfoContiguous stored, String reason,
Reason reasonCode) {
Preconditions.checkNotNull(corrupted, "corrupted is null");
Preconditions.checkNotNull(stored, "stored is null");
@ -1736,13 +1743,14 @@ public class BlockManager {
this.reasonCode = reasonCode;
}
BlockToMarkCorrupt(BlockInfo stored, String reason, Reason reasonCode) {
BlockToMarkCorrupt(BlockInfoContiguous stored, String reason,
Reason reasonCode) {
this(stored, stored, reason, reasonCode);
}
BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
BlockToMarkCorrupt(BlockInfoContiguous stored, long gs, String reason,
Reason reasonCode) {
this(new BlockInfo(stored), stored, reason, reasonCode);
this(new BlockInfoContiguous(stored), stored, reason, reasonCode);
//the corrupted block in datanode has a different generation stamp
corrupted.setGenerationStamp(gs);
}
@ -1867,7 +1875,7 @@ public class BlockManager {
break;
}
BlockInfo bi = blocksMap.getStoredBlock(b);
BlockInfoContiguous bi = blocksMap.getStoredBlock(b);
if (bi == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@ -1907,7 +1915,7 @@ public class BlockManager {
// Modify the (block-->datanode) map, according to the difference
// between the old and new block report.
//
Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
Collection<BlockInfoContiguous> toAdd = new LinkedList<BlockInfoContiguous>();
Collection<Block> toRemove = new TreeSet<Block>();
Collection<Block> toInvalidate = new LinkedList<Block>();
Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
@ -1924,7 +1932,7 @@ public class BlockManager {
removeStoredBlock(b, node);
}
int numBlocksLogged = 0;
for (BlockInfo b : toAdd) {
for (BlockInfoContiguous b : toAdd) {
addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog);
numBlocksLogged++;
}
@ -1972,7 +1980,7 @@ public class BlockManager {
continue;
}
BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(iblk);
// If block does not belong to any file, we are done.
if (storedBlock == null) continue;
@ -1995,12 +2003,13 @@ public class BlockManager {
// If block is under construction, add this replica to its list
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
storageInfo, iblk, reportedState);
((BlockInfoContiguousUnderConstruction)storedBlock)
.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
BlockInfoUnderConstruction blockUC = (BlockInfoUnderConstruction) storedBlock;
BlockInfoContiguousUnderConstruction blockUC =
(BlockInfoContiguousUnderConstruction) storedBlock;
if (namesystem.isInSnapshot(blockUC)) {
int numOfReplicas = blockUC.getNumExpectedLocations();
namesystem.incrementSafeBlockCount(numOfReplicas);
@ -2016,7 +2025,7 @@ public class BlockManager {
private void reportDiff(DatanodeStorageInfo storageInfo,
BlockListAsLongs newReport,
Collection<BlockInfo> toAdd, // add to DatanodeDescriptor
Collection<BlockInfoContiguous> toAdd, // add to DatanodeDescriptor
Collection<Block> toRemove, // remove from DatanodeDescriptor
Collection<Block> toInvalidate, // should be removed from DN
Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
@ -2024,7 +2033,7 @@ public class BlockManager {
// place a delimiter in the list which separates blocks
// that have been reported from those that have not
BlockInfo delimiter = new BlockInfo(new Block(), (short) 1);
BlockInfoContiguous delimiter = new BlockInfoContiguous(new Block(), (short) 1);
AddBlockResult result = storageInfo.addBlock(delimiter);
assert result == AddBlockResult.ADDED
: "Delimiting block cannot be present in the node";
@ -2039,7 +2048,7 @@ public class BlockManager {
while(itBR.hasNext()) {
Block iblk = itBR.next();
ReplicaState iState = itBR.getCurrentReplicaState();
BlockInfo storedBlock = processReportedBlock(storageInfo,
BlockInfoContiguous storedBlock = processReportedBlock(storageInfo,
iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
// move block to the head of the list
@ -2051,7 +2060,8 @@ public class BlockManager {
// collect blocks that have not been reported
// all of them are next to the delimiter
Iterator<BlockInfo> it = storageInfo.new BlockIterator(delimiter.getNext(0));
Iterator<BlockInfoContiguous> it =
storageInfo.new BlockIterator(delimiter.getNext(0));
while(it.hasNext())
toRemove.add(it.next());
storageInfo.removeBlock(delimiter);
@ -2088,10 +2098,10 @@ public class BlockManager {
* @return the up-to-date stored block, if it should be kept.
* Otherwise, null.
*/
private BlockInfo processReportedBlock(
private BlockInfoContiguous processReportedBlock(
final DatanodeStorageInfo storageInfo,
final Block block, final ReplicaState reportedState,
final Collection<BlockInfo> toAdd,
final Collection<BlockInfoContiguous> toAdd,
final Collection<Block> toInvalidate,
final Collection<BlockToMarkCorrupt> toCorrupt,
final Collection<StatefulBlockInfo> toUC) {
@ -2112,7 +2122,7 @@ public class BlockManager {
}
// find block by blockId
BlockInfo storedBlock = blocksMap.getStoredBlock(block);
BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(block);
if(storedBlock == null) {
// If blocksMap does not contain reported block id,
// the replica should be removed from the data-node.
@ -2155,7 +2165,8 @@ public class BlockManager {
}
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
toUC.add(new StatefulBlockInfo((BlockInfoUnderConstruction) storedBlock,
toUC.add(new StatefulBlockInfo(
(BlockInfoContiguousUnderConstruction) storedBlock,
new Block(block), reportedState));
return storedBlock;
}
@ -2243,7 +2254,7 @@ public class BlockManager {
*/
private BlockToMarkCorrupt checkReplicaCorrupt(
Block reported, ReplicaState reportedState,
BlockInfo storedBlock, BlockUCState ucState,
BlockInfoContiguous storedBlock, BlockUCState ucState,
DatanodeDescriptor dn) {
switch(reportedState) {
case FINALIZED:
@ -2316,7 +2327,7 @@ public class BlockManager {
}
}
private boolean isBlockUnderConstruction(BlockInfo storedBlock,
private boolean isBlockUnderConstruction(BlockInfoContiguous storedBlock,
BlockUCState ucState, ReplicaState reportedState) {
switch(reportedState) {
case FINALIZED:
@ -2339,7 +2350,7 @@ public class BlockManager {
void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
DatanodeStorageInfo storageInfo) throws IOException {
BlockInfoUnderConstruction block = ucBlock.storedBlock;
BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock;
block.addReplicaIfNotPresent(
storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
@ -2350,9 +2361,8 @@ public class BlockManager {
}
/**
* Faster version of
* {@link #addStoredBlock(BlockInfo, DatanodeStorageInfo, DatanodeDescriptor, boolean)}
* , intended for use with initial block report at startup. If not in startup
* Faster version of {@link #addStoredBlock},
* intended for use with initial block report at startup. If not in startup
* safe mode, will call standard addStoredBlock(). Assumes this method is
* called "immediately" so there is no need to refresh the storedBlock from
* blocksMap. Doesn't handle underReplication/overReplication, or worry about
@ -2361,7 +2371,7 @@ public class BlockManager {
*
* @throws IOException
*/
private void addStoredBlockImmediate(BlockInfo storedBlock,
private void addStoredBlockImmediate(BlockInfoContiguous storedBlock,
DatanodeStorageInfo storageInfo)
throws IOException {
assert (storedBlock != null && namesystem.hasWriteLock());
@ -2393,15 +2403,15 @@ public class BlockManager {
* needed replications if this takes care of the problem.
* @return the block that is stored in blockMap.
*/
private Block addStoredBlock(final BlockInfo block,
private Block addStoredBlock(final BlockInfoContiguous block,
DatanodeStorageInfo storageInfo,
DatanodeDescriptor delNodeHint,
boolean logEveryBlock)
throws IOException {
assert block != null && namesystem.hasWriteLock();
BlockInfo storedBlock;
BlockInfoContiguous storedBlock;
DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
if (block instanceof BlockInfoUnderConstruction) {
if (block instanceof BlockInfoContiguousUnderConstruction) {
//refresh our copy in case the block got completed in another thread
storedBlock = blocksMap.getStoredBlock(block);
} else {
@ -2500,7 +2510,8 @@ public class BlockManager {
return storedBlock;
}
private void logAddStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
private void logAddStoredBlock(BlockInfoContiguous storedBlock,
DatanodeDescriptor node) {
if (!blockLog.isInfoEnabled()) {
return;
}
@ -2527,7 +2538,7 @@ public class BlockManager {
*
* @param blk Block whose corrupt replicas need to be invalidated
*/
private void invalidateCorruptReplicas(BlockInfo blk) {
private void invalidateCorruptReplicas(BlockInfoContiguous blk) {
Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
boolean removedFromBlocksMap = true;
if (nodes == null)
@ -2606,7 +2617,7 @@ public class BlockManager {
long nrInvalid = 0, nrOverReplicated = 0;
long nrUnderReplicated = 0, nrPostponed = 0, nrUnderConstruction = 0;
long startTimeMisReplicatedScan = Time.now();
Iterator<BlockInfo> blocksItr = blocksMap.getBlocks().iterator();
Iterator<BlockInfoContiguous> blocksItr = blocksMap.getBlocks().iterator();
long totalBlocks = blocksMap.size();
replicationQueuesInitProgress = 0;
long totalProcessed = 0;
@ -2618,7 +2629,7 @@ public class BlockManager {
namesystem.writeLockInterruptibly();
try {
while (processed < numBlocksPerIteration && blocksItr.hasNext()) {
BlockInfo block = blocksItr.next();
BlockInfoContiguous block = blocksItr.next();
MisReplicationResult res = processMisReplicatedBlock(block);
if (LOG.isTraceEnabled()) {
LOG.trace("block " + block + ": " + res);
@ -2692,7 +2703,7 @@ public class BlockManager {
* appropriate queues if necessary, and returns a result code indicating
* what happened with it.
*/
private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
private MisReplicationResult processMisReplicatedBlock(BlockInfoContiguous block) {
BlockCollection bc = block.getBlockCollection();
if (bc == null) {
// block does not belong to any file
@ -3021,7 +3032,7 @@ public class BlockManager {
ReplicaState reportedState, DatanodeDescriptor delHintNode)
throws IOException {
// blockReceived reports a finalized block
Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
Collection<BlockInfoContiguous> toAdd = new LinkedList<BlockInfoContiguous>();
Collection<Block> toInvalidate = new LinkedList<Block>();
Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
@ -3038,7 +3049,7 @@ public class BlockManager {
addStoredBlockUnderConstruction(b, storageInfo);
}
long numBlocksLogged = 0;
for (BlockInfo b : toAdd) {
for (BlockInfoContiguous b : toAdd) {
addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog);
numBlocksLogged++;
}
@ -3162,7 +3173,7 @@ public class BlockManager {
* @param b - the block being tested
* @return count of live nodes for this block
*/
int countLiveNodes(BlockInfo b) {
int countLiveNodes(BlockInfoContiguous b) {
if (!namesystem.isInStartupSafeMode()) {
return countNodes(b).liveReplicas();
}
@ -3317,7 +3328,7 @@ public class BlockManager {
return blocksMap.size();
}
public DatanodeStorageInfo[] getStorages(BlockInfo block) {
public DatanodeStorageInfo[] getStorages(BlockInfoContiguous block) {
final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
int i = 0;
for(DatanodeStorageInfo s : blocksMap.getStorages(block)) {
@ -3347,7 +3358,7 @@ public class BlockManager {
}
}
public BlockInfo getStoredBlock(Block block) {
public BlockInfoContiguous getStoredBlock(Block block) {
return blocksMap.getStoredBlock(block);
}
@ -3495,7 +3506,8 @@ public class BlockManager {
return this.neededReplications.getCorruptReplOneBlockSize();
}
public BlockInfo addBlockCollection(BlockInfo block, BlockCollection bc) {
public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block,
BlockCollection bc) {
return blocksMap.addBlockCollection(block, bc);
}
@ -3704,7 +3716,7 @@ public class BlockManager {
/**
* A simple result enum for the result of
* {@link BlockManager#processMisReplicatedBlock(BlockInfo)}.
* {@link BlockManager#processMisReplicatedBlock(BlockInfoContiguous)}.
*/
enum MisReplicationResult {
/** The block should be invalidated since it belongs to a deleted file. */

View File

@ -36,10 +36,10 @@ import com.google.common.collect.Iterables;
*/
class BlocksMap {
private static class StorageIterator implements Iterator<DatanodeStorageInfo> {
private final BlockInfo blockInfo;
private final BlockInfoContiguous blockInfo;
private int nextIdx = 0;
StorageIterator(BlockInfo blkInfo) {
StorageIterator(BlockInfoContiguous blkInfo) {
this.blockInfo = blkInfo;
}
@ -63,14 +63,14 @@ class BlocksMap {
/** Constant {@link LightWeightGSet} capacity. */
private final int capacity;
private GSet<Block, BlockInfo> blocks;
private GSet<Block, BlockInfoContiguous> blocks;
BlocksMap(int capacity) {
// Use 2% of total memory to size the GSet capacity
this.capacity = capacity;
this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity) {
this.blocks = new LightWeightGSet<Block, BlockInfoContiguous>(capacity) {
@Override
public Iterator<BlockInfo> iterator() {
public Iterator<BlockInfoContiguous> iterator() {
SetIterator iterator = new SetIterator();
/*
* Not tracking any modifications to set. As this set will be used
@ -97,15 +97,15 @@ class BlocksMap {
}
BlockCollection getBlockCollection(Block b) {
BlockInfo info = blocks.get(b);
BlockInfoContiguous info = blocks.get(b);
return (info != null) ? info.getBlockCollection() : null;
}
/**
* Add block b belonging to the specified block collection to the map.
*/
BlockInfo addBlockCollection(BlockInfo b, BlockCollection bc) {
BlockInfo info = blocks.get(b);
BlockInfoContiguous addBlockCollection(BlockInfoContiguous b, BlockCollection bc) {
BlockInfoContiguous info = blocks.get(b);
if (info != b) {
info = b;
blocks.put(info);
@ -120,7 +120,7 @@ class BlocksMap {
* and remove all data-node locations associated with the block.
*/
void removeBlock(Block block) {
BlockInfo blockInfo = blocks.remove(block);
BlockInfoContiguous blockInfo = blocks.remove(block);
if (blockInfo == null)
return;
@ -132,7 +132,7 @@ class BlocksMap {
}
/** Returns the block object it it exists in the map. */
BlockInfo getStoredBlock(Block b) {
BlockInfoContiguous getStoredBlock(Block b) {
return blocks.get(b);
}
@ -164,7 +164,7 @@ class BlocksMap {
* For a block that has already been retrieved from the BlocksMap
* returns {@link Iterable} of the storages the block belongs to.
*/
Iterable<DatanodeStorageInfo> getStorages(final BlockInfo storedBlock) {
Iterable<DatanodeStorageInfo> getStorages(final BlockInfoContiguous storedBlock) {
return new Iterable<DatanodeStorageInfo>() {
@Override
public Iterator<DatanodeStorageInfo> iterator() {
@ -175,7 +175,7 @@ class BlocksMap {
/** counts number of containing nodes. Better than using iterator. */
int numNodes(Block b) {
BlockInfo info = blocks.get(b);
BlockInfoContiguous info = blocks.get(b);
return info == null ? 0 : info.numNodes();
}
@ -185,7 +185,7 @@ class BlocksMap {
* only if it does not belong to any file and data-nodes.
*/
boolean removeNode(Block b, DatanodeDescriptor node) {
BlockInfo info = blocks.get(b);
BlockInfoContiguous info = blocks.get(b);
if (info == null)
return false;
@ -203,7 +203,7 @@ class BlocksMap {
return blocks.size();
}
Iterable<BlockInfo> getBlocks() {
Iterable<BlockInfoContiguous> getBlocks() {
return blocks;
}
@ -218,8 +218,8 @@ class BlocksMap {
* @param newBlock - block for replacement
* @return new block
*/
BlockInfo replaceBlock(BlockInfo newBlock) {
BlockInfo currentBlock = blocks.get(newBlock);
BlockInfoContiguous replaceBlock(BlockInfoContiguous newBlock) {
BlockInfoContiguous currentBlock = blocks.get(newBlock);
assert currentBlock != null : "the block if not in blocksMap";
// replace block in data-node lists
for (int i = currentBlock.numNodes() - 1; i >= 0; i--) {

View File

@ -369,7 +369,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
* @param file The file.
*/
private void rescanFile(CacheDirective directive, INodeFile file) {
BlockInfo[] blockInfos = file.getBlocks();
BlockInfoContiguous[] blockInfos = file.getBlocks();
// Increment the "needed" statistics
directive.addFilesNeeded(1);
@ -394,7 +394,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
}
long cachedTotal = 0;
for (BlockInfo blockInfo : blockInfos) {
for (BlockInfoContiguous blockInfo : blockInfos) {
if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) {
// We don't try to cache blocks that are under construction.
LOG.trace("Directive {}: can't cache block {} because it is in state "
@ -453,7 +453,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
}
private String findReasonForNotCaching(CachedBlock cblock,
BlockInfo blockInfo) {
BlockInfoContiguous blockInfo) {
if (blockInfo == null) {
// Somehow, a cache report with the block arrived, but the block
// reports from the DataNode haven't (yet?) described such a block.
@ -513,7 +513,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
iter.remove();
}
}
BlockInfo blockInfo = blockManager.
BlockInfoContiguous blockInfo = blockManager.
getStoredBlock(new Block(cblock.getBlockId()));
String reason = findReasonForNotCaching(cblock, blockInfo);
int neededCached = 0;
@ -628,7 +628,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
List<DatanodeDescriptor> pendingCached) {
// To figure out which replicas can be cached, we consult the
// blocksMap. We don't want to try to cache a corrupt replica, though.
BlockInfo blockInfo = blockManager.
BlockInfoContiguous blockInfo = blockManager.
getStoredBlock(new Block(cachedBlock.getBlockId()));
if (blockInfo == null) {
LOG.debug("Block {}: can't add new cached replicas," +
@ -667,7 +667,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
Iterator<CachedBlock> it = datanode.getPendingCached().iterator();
while (it.hasNext()) {
CachedBlock cBlock = it.next();
BlockInfo info =
BlockInfoContiguous info =
blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
if (info != null) {
pendingBytes -= info.getNumBytes();
@ -677,7 +677,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
// Add pending uncached blocks from effective capacity
while (it.hasNext()) {
CachedBlock cBlock = it.next();
BlockInfo info =
BlockInfoContiguous info =
blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
if (info != null) {
pendingBytes += info.getNumBytes();

View File

@ -199,8 +199,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
/** A queue of blocks to be replicated by this datanode */
private final BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
/** A queue of blocks to be recovered by this datanode */
private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
new BlockQueue<BlockInfoUnderConstruction>();
private final BlockQueue<BlockInfoContiguousUnderConstruction> recoverBlocks =
new BlockQueue<BlockInfoContiguousUnderConstruction>();
/** A set of blocks to be invalidated by this datanode */
private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
@ -286,7 +286,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
* Remove block from the list of blocks belonging to the data-node. Remove
* data-node from the block.
*/
boolean removeBlock(BlockInfo b) {
boolean removeBlock(BlockInfoContiguous b) {
final DatanodeStorageInfo s = b.findStorageInfo(this);
// if block exists on this datanode
if (s != null) {
@ -299,7 +299,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
* Remove block from the list of blocks belonging to the data-node. Remove
* data-node from the block.
*/
boolean removeBlock(String storageID, BlockInfo b) {
boolean removeBlock(String storageID, BlockInfoContiguous b) {
DatanodeStorageInfo s = getStorageInfo(storageID);
if (s != null) {
return s.removeBlock(b);
@ -470,12 +470,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
}
}
private static class BlockIterator implements Iterator<BlockInfo> {
private static class BlockIterator implements Iterator<BlockInfoContiguous> {
private int index = 0;
private final List<Iterator<BlockInfo>> iterators;
private final List<Iterator<BlockInfoContiguous>> iterators;
private BlockIterator(final DatanodeStorageInfo... storages) {
List<Iterator<BlockInfo>> iterators = new ArrayList<Iterator<BlockInfo>>();
List<Iterator<BlockInfoContiguous>> iterators = new ArrayList<Iterator<BlockInfoContiguous>>();
for (DatanodeStorageInfo e : storages) {
iterators.add(e.getBlockIterator());
}
@ -489,7 +489,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
}
@Override
public BlockInfo next() {
public BlockInfoContiguous next() {
update();
return iterators.get(index).next();
}
@ -506,10 +506,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
}
}
Iterator<BlockInfo> getBlockIterator() {
Iterator<BlockInfoContiguous> getBlockIterator() {
return new BlockIterator(getStorageInfos());
}
Iterator<BlockInfo> getBlockIterator(final String storageID) {
Iterator<BlockInfoContiguous> getBlockIterator(final String storageID) {
return new BlockIterator(getStorageInfo(storageID));
}
@ -532,7 +532,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
/**
* Store block recovery work.
*/
void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
if(recoverBlocks.contains(block)) {
// this prevents adding the same block twice to the recovery queue
BlockManager.LOG.info(block + " is already in the recovery queue");
@ -574,11 +574,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
return replicateBlocks.poll(maxTransfers);
}
public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
if(blocks == null)
return null;
return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]);
}
/**

View File

@ -1416,12 +1416,12 @@ public class DatanodeManager {
}
//check lease recovery
BlockInfoUnderConstruction[] blocks = nodeinfo
BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
.getLeaseRecoveryCommand(Integer.MAX_VALUE);
if (blocks != null) {
BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
blocks.length);
for (BlockInfoUnderConstruction b : blocks) {
for (BlockInfoContiguousUnderConstruction b : blocks) {
final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
// Skip stale nodes during recovery - not heart beated for some time (30s by default).
final List<DatanodeStorageInfo> recoveryLocations =

View File

@ -80,10 +80,10 @@ public class DatanodeStorageInfo {
/**
* Iterates over the list of blocks belonging to the data-node.
*/
class BlockIterator implements Iterator<BlockInfo> {
private BlockInfo current;
class BlockIterator implements Iterator<BlockInfoContiguous> {
private BlockInfoContiguous current;
BlockIterator(BlockInfo head) {
BlockIterator(BlockInfoContiguous head) {
this.current = head;
}
@ -91,8 +91,8 @@ public class DatanodeStorageInfo {
return current != null;
}
public BlockInfo next() {
BlockInfo res = current;
public BlockInfoContiguous next() {
BlockInfoContiguous res = current;
current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this));
return res;
}
@ -112,7 +112,7 @@ public class DatanodeStorageInfo {
private volatile long remaining;
private long blockPoolUsed;
private volatile BlockInfo blockList = null;
private volatile BlockInfoContiguous blockList = null;
private int numBlocks = 0;
/** The number of block reports received */
@ -215,7 +215,7 @@ public class DatanodeStorageInfo {
return blockPoolUsed;
}
public AddBlockResult addBlock(BlockInfo b) {
public AddBlockResult addBlock(BlockInfoContiguous b) {
// First check whether the block belongs to a different storage
// on the same DN.
AddBlockResult result = AddBlockResult.ADDED;
@ -240,7 +240,7 @@ public class DatanodeStorageInfo {
return result;
}
public boolean removeBlock(BlockInfo b) {
public boolean removeBlock(BlockInfoContiguous b) {
blockList = b.listRemove(blockList, this);
if (b.removeStorage(this)) {
numBlocks--;
@ -254,7 +254,7 @@ public class DatanodeStorageInfo {
return numBlocks;
}
Iterator<BlockInfo> getBlockIterator() {
Iterator<BlockInfoContiguous> getBlockIterator() {
return new BlockIterator(blockList);
}
@ -263,7 +263,7 @@ public class DatanodeStorageInfo {
* Move block to the head of the list of blocks belonging to the data-node.
* @return the index of the head of the blockList
*/
int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
int moveBlockToHead(BlockInfoContiguous b, int curIndex, int headIndex) {
blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
return curIndex;
}
@ -273,7 +273,7 @@ public class DatanodeStorageInfo {
* @return the head of the blockList
*/
@VisibleForTesting
BlockInfo getBlockListHeadForTesting(){
BlockInfoContiguous getBlockListHeadForTesting(){
return blockList;
}

View File

@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.SnapshotException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import java.io.IOException;
import java.util.Arrays;
@ -96,7 +96,7 @@ class FSDirConcatOp {
long blockSize = trgInode.getPreferredBlockSize();
// check the end block to be full
final BlockInfo last = trgInode.getLastBlock();
final BlockInfoContiguous last = trgInode.getLastBlock();
if(blockSize != last.getNumBytes()) {
throw new HadoopIllegalArgumentException("The last block in " + target
+ " is not full; last block size = " + last.getNumBytes()
@ -133,7 +133,7 @@ class FSDirConcatOp {
//boolean endBlock=false;
// verify that all the blocks are of the same length as target
// should be enough to check the end blocks
final BlockInfo[] srcBlocks = srcInode.getBlocks();
final BlockInfoContiguous[] srcBlocks = srcInode.getBlocks();
int idx = srcBlocks.length-1;
if(endSrc)
idx = srcBlocks.length-2; // end block of endSrc is OK not to be full

View File

@ -52,8 +52,8 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@ -364,7 +364,7 @@ public class FSDirectory implements Closeable {
long mtime, long atime, short replication, long preferredBlockSize,
byte storagePolicyId) {
return new INodeFile(id, null, permissions, mtime, atime,
BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
BlockInfoContiguous.EMPTY_ARRAY, replication, preferredBlockSize,
storagePolicyId);
}
@ -444,8 +444,8 @@ public class FSDirectory implements Closeable {
/**
* Add a block to the file. Returns a reference to the added block.
*/
BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
DatanodeStorageInfo[] targets) throws IOException {
BlockInfoContiguous addBlock(String path, INodesInPath inodesInPath,
Block block, DatanodeStorageInfo[] targets) throws IOException {
writeLock();
try {
final INodeFile fileINode = inodesInPath.getLastINode().asFile();
@ -455,8 +455,8 @@ public class FSDirectory implements Closeable {
updateCount(inodesInPath, 0, fileINode.getBlockDiskspace(), true);
// associate new last block for the file
BlockInfoUnderConstruction blockInfo =
new BlockInfoUnderConstruction(
BlockInfoContiguousUnderConstruction blockInfo =
new BlockInfoContiguousUnderConstruction(
block,
fileINode.getFileReplication(),
BlockUCState.UNDER_CONSTRUCTION,
@ -974,7 +974,7 @@ public class FSDirectory implements Closeable {
unprotectedTruncate(iip, newLength, collectedBlocks, mtime);
if(! onBlockBoundary) {
BlockInfo oldBlock = file.getLastBlock();
BlockInfoContiguous oldBlock = file.getLastBlock();
Block tBlk =
getFSNamesystem().prepareFileForTruncate(iip,
clientName, clientMachine, file.computeFileSize() - newLength,

View File

@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@ -758,10 +758,10 @@ public class FSEditLog implements LogsPurgeable {
public void logAddBlock(String path, INodeFile file) {
Preconditions.checkArgument(file.isUnderConstruction());
BlockInfo[] blocks = file.getBlocks();
BlockInfoContiguous[] blocks = file.getBlocks();
Preconditions.checkState(blocks != null && blocks.length > 0);
BlockInfo pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
BlockInfo lastBlock = blocks[blocks.length - 1];
BlockInfoContiguous pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
BlockInfoContiguous lastBlock = blocks[blocks.length - 1];
AddBlockOp op = AddBlockOp.getInstance(cache.get()).setPath(path)
.setPenultimateBlock(pBlock).setLastBlock(lastBlock);
logEdit(op);

View File

@ -43,8 +43,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
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.Storage;
@ -903,7 +903,7 @@ public class FSEditLogLoader {
*/
private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file)
throws IOException {
BlockInfo[] oldBlocks = file.getBlocks();
BlockInfoContiguous[] oldBlocks = file.getBlocks();
Block pBlock = op.getPenultimateBlock();
Block newBlock= op.getLastBlock();
@ -920,16 +920,16 @@ public class FSEditLogLoader {
}
oldLastBlock.setNumBytes(pBlock.getNumBytes());
if (oldLastBlock instanceof BlockInfoUnderConstruction) {
if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
fsNamesys.getBlockManager().forceCompleteBlock(file,
(BlockInfoUnderConstruction) oldLastBlock);
(BlockInfoContiguousUnderConstruction) oldLastBlock);
fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
}
} else { // the penultimate block is null
Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
}
// add the new block
BlockInfo newBI = new BlockInfoUnderConstruction(
BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction(
newBlock, file.getBlockReplication());
fsNamesys.getBlockManager().addBlockCollection(newBI, file);
file.addBlock(newBI);
@ -943,7 +943,7 @@ public class FSEditLogLoader {
private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
INodesInPath iip, INodeFile file) throws IOException {
// Update its block list
BlockInfo[] oldBlocks = file.getBlocks();
BlockInfoContiguous[] oldBlocks = file.getBlocks();
Block[] newBlocks = op.getBlocks();
String path = op.getPath();
@ -952,7 +952,7 @@ public class FSEditLogLoader {
// First, update blocks in common
for (int i = 0; i < oldBlocks.length && i < newBlocks.length; i++) {
BlockInfo oldBlock = oldBlocks[i];
BlockInfoContiguous oldBlock = oldBlocks[i];
Block newBlock = newBlocks[i];
boolean isLastBlock = i == newBlocks.length - 1;
@ -970,11 +970,11 @@ public class FSEditLogLoader {
oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
if (oldBlock instanceof BlockInfoUnderConstruction &&
if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
(!isLastBlock || op.shouldCompleteLastBlock())) {
changeMade = true;
fsNamesys.getBlockManager().forceCompleteBlock(file,
(BlockInfoUnderConstruction) oldBlock);
(BlockInfoContiguousUnderConstruction) oldBlock);
}
if (changeMade) {
// The state or gen-stamp of the block has changed. So, we may be
@ -1003,19 +1003,19 @@ public class FSEditLogLoader {
// We're adding blocks
for (int i = oldBlocks.length; i < newBlocks.length; i++) {
Block newBlock = newBlocks[i];
BlockInfo newBI;
BlockInfoContiguous newBI;
if (!op.shouldCompleteLastBlock()) {
// TODO: shouldn't this only be true for the last block?
// what about an old-version fsync() where fsync isn't called
// until several blocks in?
newBI = new BlockInfoUnderConstruction(
newBI = new BlockInfoContiguousUnderConstruction(
newBlock, file.getBlockReplication());
} else {
// OP_CLOSE should add finalized blocks. This code path
// is only executed when loading edits written by prior
// versions of Hadoop. Current versions always log
// OP_ADD operations as each block is allocated.
newBI = new BlockInfo(newBlock, file.getBlockReplication());
newBI = new BlockInfoContiguous(newBlock, file.getBlockReplication());
}
fsNamesys.getBlockManager().addBlockCollection(newBI, file);
file.addBlock(newBI);

View File

@ -52,8 +52,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutFlags;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@ -684,7 +684,7 @@ public class FSImageFormat {
public void updateBlocksMap(INodeFile file) {
// Add file->block mapping
final BlockInfo[] blocks = file.getBlocks();
final BlockInfoContiguous[] blocks = file.getBlocks();
if (blocks != null) {
final BlockManager bm = namesystem.getBlockManager();
for (int i = 0; i < blocks.length; i++) {
@ -751,9 +751,9 @@ public class FSImageFormat {
// file
// read blocks
BlockInfo[] blocks = new BlockInfo[numBlocks];
BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
for (int j = 0; j < numBlocks; j++) {
blocks[j] = new BlockInfo(replication);
blocks[j] = new BlockInfoContiguous(replication);
blocks[j].readFields(in);
}
@ -773,8 +773,8 @@ public class FSImageFormat {
clientMachine = FSImageSerialization.readString(in);
// convert the last block to BlockUC
if (blocks.length > 0) {
BlockInfo lastBlk = blocks[blocks.length - 1];
blocks[blocks.length - 1] = new BlockInfoUnderConstruction(
BlockInfoContiguous lastBlk = blocks[blocks.length - 1];
blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
lastBlk, replication);
}
}
@ -952,9 +952,9 @@ public class FSImageFormat {
FileUnderConstructionFeature uc = cons.getFileUnderConstructionFeature();
oldnode.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
if (oldnode.numBlocks() > 0) {
BlockInfo ucBlock = cons.getLastBlock();
BlockInfoContiguous ucBlock = cons.getLastBlock();
// we do not replace the inode, just replace the last block of oldnode
BlockInfo info = namesystem.getBlockManager().addBlockCollection(
BlockInfoContiguous info = namesystem.getBlockManager().addBlockCollection(
ucBlock, oldnode);
oldnode.setBlock(oldnode.numBlocks() - 1, info);
}

View File

@ -40,8 +40,8 @@ import org.apache.hadoop.fs.XAttr;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
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.SaverContext;
@ -169,7 +169,7 @@ public final class FSImageFormatPBINode {
public static void updateBlocksMap(INodeFile file, BlockManager bm) {
// Add file->block mapping
final BlockInfo[] blocks = file.getBlocks();
final BlockInfoContiguous[] blocks = file.getBlocks();
if (blocks != null) {
for (int i = 0; i < blocks.length; i++) {
file.setBlock(i, bm.addBlockCollection(blocks[i], file));
@ -281,9 +281,9 @@ public final class FSImageFormatPBINode {
short replication = (short) f.getReplication();
LoaderContext state = parent.getLoaderContext();
BlockInfo[] blocks = new BlockInfo[bp.size()];
BlockInfoContiguous[] blocks = new BlockInfoContiguous[bp.size()];
for (int i = 0, e = bp.size(); i < e; ++i) {
blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
blocks[i] = new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication);
}
final PermissionStatus permissions = loadPermission(f.getPermission(),
parent.getLoaderContext().getStringTable());
@ -309,9 +309,9 @@ public final class FSImageFormatPBINode {
INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
if (blocks.length > 0) {
BlockInfo lastBlk = file.getLastBlock();
BlockInfoContiguous lastBlk = file.getLastBlock();
// replace the last block of file
file.setBlock(file.numBlocks() - 1, new BlockInfoUnderConstruction(
file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
lastBlk, replication));
}
}

View File

@ -33,8 +33,8 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
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.ReferenceMap;
@ -126,17 +126,17 @@ public class FSImageSerialization {
long preferredBlockSize = in.readLong();
int numBlocks = in.readInt();
BlockInfo[] blocks = new BlockInfo[numBlocks];
BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
Block blk = new Block();
int i = 0;
for (; i < numBlocks-1; i++) {
blk.readFields(in);
blocks[i] = new BlockInfo(blk, blockReplication);
blocks[i] = new BlockInfoContiguous(blk, blockReplication);
}
// last block is UNDER_CONSTRUCTION
if(numBlocks > 0) {
blk.readFields(in);
blocks[i] = new BlockInfoUnderConstruction(
blocks[i] = new BlockInfoContiguousUnderConstruction(
blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
}
PermissionStatus perm = PermissionStatus.read(in);

View File

@ -205,8 +205,8 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@ -2070,7 +2070,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
leaseManager.addLease(
file.getFileUnderConstructionFeature().getClientName(), src);
boolean shouldRecoverNow = (newBlock == null);
BlockInfo oldBlock = file.getLastBlock();
BlockInfoContiguous oldBlock = file.getLastBlock();
boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
if(newBlock == null) {
newBlock = (shouldCopyOnTruncate) ? createNewBlock() :
@ -2078,11 +2078,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
}
BlockInfoUnderConstruction truncatedBlockUC;
BlockInfoContiguousUnderConstruction truncatedBlockUC;
if(shouldCopyOnTruncate) {
// Add new truncateBlock into blocksMap and
// use oldBlock as a source for copy-on-truncate recovery
truncatedBlockUC = new BlockInfoUnderConstruction(newBlock,
truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
file.getBlockReplication());
truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
truncatedBlockUC.setTruncateBlock(oldBlock);
@ -2098,7 +2098,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
oldBlock = file.getLastBlock();
assert !oldBlock.isComplete() : "oldBlock should be under construction";
truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;
truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
truncatedBlockUC.getTruncateBlock().setNumBytes(
oldBlock.getNumBytes() - lastBlockDelta);
@ -2124,7 +2124,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
* Defines if a replica needs to be copied on truncate or
* can be truncated in place.
*/
boolean shouldCopyOnTruncate(INodeFile file, BlockInfo blk) {
boolean shouldCopyOnTruncate(INodeFile file, BlockInfoContiguous blk) {
if(!isUpgradeFinalized()) {
return true;
}
@ -2618,7 +2618,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE,
iip, src, holder, clientMachine, false);
final BlockInfo lastBlock = myFile.getLastBlock();
final BlockInfoContiguous lastBlock = myFile.getLastBlock();
// Check that the block has at least minimum replication.
if(lastBlock != null && lastBlock.isComplete() &&
!getBlockManager().isSufficientlyReplicated(lastBlock)) {
@ -2791,7 +2791,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
op.getExceptionMessage(src, holder, clientMachine,
"lease recovery is in progress. Try again later."));
} else {
final BlockInfo lastBlock = file.getLastBlock();
final BlockInfoContiguous lastBlock = file.getLastBlock();
if (lastBlock != null
&& lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
throw new RecoveryInProgressException(
@ -2976,8 +2976,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return onRetryBlock[0];
} else {
// add new chosen targets to already allocated block and return
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
((BlockInfoUnderConstruction) lastBlockInFile)
BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
((BlockInfoContiguousUnderConstruction) lastBlockInFile)
.setExpectedLocations(targets);
offset = pendingFile.computeFileSize();
return makeLocatedBlock(lastBlockInFile, targets, offset);
@ -3068,7 +3068,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
}
}
final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
BlockInfo lastBlockInFile = pendingFile.getLastBlock();
BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
// The block that the client claims is the current last block
// doesn't match up with what we think is the last block. There are
@ -3096,7 +3096,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
// changed the namesystem state yet.
// We run this analysis again in Part II where case 4 is impossible.
BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
if (previous == null &&
lastBlockInFile != null &&
lastBlockInFile.getNumBytes() == pendingFile.getPreferredBlockSize() &&
@ -3123,7 +3123,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
src + ". Returning previously allocated block " + lastBlockInFile);
long offset = pendingFile.computeFileSize();
onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
((BlockInfoContiguousUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
offset);
return new FileState(pendingFile, src, iip);
} else {
@ -3414,11 +3414,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
* @param targets target datanodes where replicas of the new block is placed
* @throws QuotaExceededException If addition of block exceeds space quota
*/
BlockInfo saveAllocatedBlock(String src, INodesInPath inodes,
BlockInfoContiguous saveAllocatedBlock(String src, INodesInPath inodesInPath,
Block newBlock, DatanodeStorageInfo[] targets)
throws IOException {
assert hasWriteLock();
BlockInfo b = dir.addBlock(src, inodes, newBlock, targets);
BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets);
NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
DatanodeStorageInfo.incrementBlocksScheduled(targets);
return b;
@ -3445,14 +3445,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
try {
if (checkall) {
// check all blocks of the file.
for (BlockInfo block: v.getBlocks()) {
for (BlockInfoContiguous block: v.getBlocks()) {
if (!isCompleteBlock(src, block, blockManager.minReplication)) {
return false;
}
}
} else {
// check the penultimate block of this file
BlockInfo b = v.getPenultimateBlock();
BlockInfoContiguous b = v.getPenultimateBlock();
if (b != null
&& !isCompleteBlock(src, b, blockManager.minReplication)) {
return false;
@ -3464,9 +3464,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
}
}
private static boolean isCompleteBlock(String src, BlockInfo b, int minRepl) {
private static boolean isCompleteBlock(String src, BlockInfoContiguous b, int minRepl) {
if (!b.isComplete()) {
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)b;
final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)b;
final int numNodes = b.numNodes();
LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
+ uc.getBlockUCState() + ", replication# = " + numNodes
@ -3656,7 +3656,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
for (Block b : blocks.getToDeleteList()) {
if (trackBlockCounts) {
BlockInfo bi = getStoredBlock(b);
BlockInfoContiguous bi = getStoredBlock(b);
if (bi.isComplete()) {
numRemovedComplete++;
if (bi.numNodes() >= blockManager.minReplication) {
@ -3875,10 +3875,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
final INodeFile pendingFile = iip.getLastINode().asFile();
int nrBlocks = pendingFile.numBlocks();
BlockInfo[] blocks = pendingFile.getBlocks();
BlockInfoContiguous[] blocks = pendingFile.getBlocks();
int nrCompleteBlocks;
BlockInfo curBlock = null;
BlockInfoContiguous curBlock = null;
for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) {
curBlock = blocks[nrCompleteBlocks];
if(!curBlock.isComplete())
@ -3913,9 +3913,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
// The last block is not COMPLETE, and
// that the penultimate block if exists is either COMPLETE or COMMITTED
final BlockInfo lastBlock = pendingFile.getLastBlock();
final BlockInfoContiguous lastBlock = pendingFile.getLastBlock();
BlockUCState lastBlockState = lastBlock.getBlockUCState();
BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
// If penultimate block doesn't exist then its minReplication is met
boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
@ -3948,7 +3948,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
throw new AlreadyBeingCreatedException(message);
case UNDER_CONSTRUCTION:
case UNDER_RECOVERY:
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock;
final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
// determine if last block was intended to be truncated
Block recoveryBlock = uc.getTruncateBlock();
boolean truncateRecovery = recoveryBlock != null;
@ -4059,12 +4059,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
}
@VisibleForTesting
BlockInfo getStoredBlock(Block block) {
BlockInfoContiguous getStoredBlock(Block block) {
return blockManager.getStoredBlock(block);
}
@Override
public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) {
assert hasReadLock();
final BlockCollection bc = blockUC.getBlockCollection();
if (bc == null || !(bc instanceof INodeFile)
@ -4117,7 +4117,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
checkNameNodeSafeMode(
"Cannot commitBlockSynchronization while in safe mode");
final BlockInfo storedBlock = getStoredBlock(
final BlockInfoContiguous storedBlock = getStoredBlock(
ExtendedBlock.getLocalBlock(oldBlock));
if (storedBlock == null) {
if (deleteblock) {
@ -4165,8 +4165,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return;
}
BlockInfoUnderConstruction truncatedBlock =
(BlockInfoUnderConstruction) iFile.getLastBlock();
BlockInfoContiguousUnderConstruction truncatedBlock =
(BlockInfoContiguousUnderConstruction) iFile.getLastBlock();
long recoveryId = truncatedBlock.getBlockRecoveryId();
boolean copyTruncate =
truncatedBlock.getBlockId() != storedBlock.getBlockId();
@ -4275,7 +4275,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
* @throws IOException on error
*/
@VisibleForTesting
String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock)
String closeFileCommitBlocks(INodeFile pendingFile, BlockInfoContiguous storedBlock)
throws IOException {
final INodesInPath iip = INodesInPath.fromINode(pendingFile);
final String src = iip.getPath();
@ -4583,7 +4583,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
while (it.hasNext()) {
Block b = it.next();
BlockInfo blockInfo = blockManager.getStoredBlock(b);
BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b);
if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) {
filesToDelete.add(blockInfo.getBlockCollection());
}
@ -5530,7 +5530,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
SafeModeInfo safeMode = this.safeMode;
if (safeMode == null) // mostly true
return;
BlockInfo storedBlock = getStoredBlock(b);
BlockInfoContiguous storedBlock = getStoredBlock(b);
if (storedBlock.isComplete()) {
safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
}
@ -6047,7 +6047,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
+ "access token for block " + block);
// check stored block state
BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
BlockInfoContiguous storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
if (storedBlock == null ||
storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
throw new IOException(block +
@ -6176,8 +6176,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
assert hasWriteLock();
// check the vadility of the block and lease holder name
final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
final BlockInfoUnderConstruction blockinfo
= (BlockInfoUnderConstruction)pendingFile.getLastBlock();
final BlockInfoContiguousUnderConstruction blockinfo
= (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock();
// check new GS & length: this is not expected
if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||

View File

@ -20,9 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
/**
@ -59,10 +58,10 @@ public class FileUnderConstructionFeature implements INode.Feature {
*/
void updateLengthOfLastBlock(INodeFile f, long lastBlockLength)
throws IOException {
BlockInfo lastBlock = f.getLastBlock();
BlockInfoContiguous lastBlock = f.getLastBlock();
assert (lastBlock != null) : "The last block for path "
+ f.getFullPathName() + " is null when updating its length";
assert (lastBlock instanceof BlockInfoUnderConstruction)
assert (lastBlock instanceof BlockInfoContiguousUnderConstruction)
: "The last block for path " + f.getFullPathName()
+ " is not a BlockInfoUnderConstruction when updating its length";
lastBlock.setNumBytes(lastBlockLength);
@ -75,11 +74,11 @@ public class FileUnderConstructionFeature implements INode.Feature {
*/
void cleanZeroSizeBlock(final INodeFile f,
final BlocksMapUpdateInfo collectedBlocks) {
final BlockInfo[] blocks = f.getBlocks();
final BlockInfoContiguous[] blocks = f.getBlocks();
if (blocks != null && blocks.length > 0
&& blocks[blocks.length - 1] instanceof BlockInfoUnderConstruction) {
BlockInfoUnderConstruction lastUC =
(BlockInfoUnderConstruction) blocks[blocks.length - 1];
&& blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) {
BlockInfoContiguousUnderConstruction lastUC =
(BlockInfoContiguousUnderConstruction) blocks[blocks.length - 1];
if (lastUC.getNumBytes() == 0) {
// this is a 0-sized block. do not need check its UC state here
collectedBlocks.addDeleteBlock(lastUC);

View File

@ -33,8 +33,8 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@ -114,17 +114,17 @@ public class INodeFile extends INodeWithAdditionalFields
private long header = 0L;
private BlockInfo[] blocks;
private BlockInfoContiguous[] blocks;
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
long atime, BlockInfo[] blklist, short replication,
long atime, BlockInfoContiguous[] blklist, short replication,
long preferredBlockSize) {
this(id, name, permissions, mtime, atime, blklist, replication,
preferredBlockSize, (byte) 0);
}
INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
long atime, BlockInfo[] blklist, short replication,
long atime, BlockInfoContiguous[] blklist, short replication,
long preferredBlockSize, byte storagePolicyID) {
super(id, name, permissions, mtime, atime);
header = HeaderFormat.toLong(preferredBlockSize, replication,
@ -220,20 +220,21 @@ public class INodeFile extends INodeWithAdditionalFields
}
@Override // BlockCollection
public void setBlock(int index, BlockInfo blk) {
public void setBlock(int index, BlockInfoContiguous blk) {
this.blocks[index] = blk;
}
@Override // BlockCollection, the file should be under construction
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeStorageInfo[] locations) throws IOException {
public BlockInfoContiguousUnderConstruction setLastBlock(
BlockInfoContiguous lastBlock, DatanodeStorageInfo[] locations)
throws IOException {
Preconditions.checkState(isUnderConstruction(),
"file is no longer under construction");
if (numBlocks() == 0) {
throw new IOException("Failed to set last block: File is empty.");
}
BlockInfoUnderConstruction ucBlock =
BlockInfoContiguousUnderConstruction ucBlock =
lastBlock.convertToBlockUnderConstruction(
BlockUCState.UNDER_CONSTRUCTION, locations);
setBlock(numBlocks() - 1, ucBlock);
@ -256,7 +257,7 @@ public class INodeFile extends INodeWithAdditionalFields
}
//copy to a new list
BlockInfo[] newlist = new BlockInfo[size_1];
BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
System.arraycopy(blocks, 0, newlist, 0, size_1);
setBlocks(newlist);
return true;
@ -416,16 +417,17 @@ public class INodeFile extends INodeWithAdditionalFields
/** @return the blocks of the file. */
@Override
public BlockInfo[] getBlocks() {
public BlockInfoContiguous[] getBlocks() {
return this.blocks;
}
/** @return blocks of the file corresponding to the snapshot. */
public BlockInfo[] getBlocks(int snapshot) {
public BlockInfoContiguous[] getBlocks(int snapshot) {
if(snapshot == CURRENT_STATE_ID || getDiffs() == null)
return getBlocks();
FileDiff diff = getDiffs().getDiffById(snapshot);
BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks();
BlockInfoContiguous[] snapshotBlocks =
diff == null ? getBlocks() : diff.getBlocks();
if(snapshotBlocks != null)
return snapshotBlocks;
// Blocks are not in the current snapshot
@ -436,7 +438,7 @@ public class INodeFile extends INodeWithAdditionalFields
void updateBlockCollection() {
if (blocks != null) {
for(BlockInfo b : blocks) {
for(BlockInfoContiguous b : blocks) {
b.setBlockCollection(this);
}
}
@ -452,7 +454,8 @@ public class INodeFile extends INodeWithAdditionalFields
totalAddedBlocks += f.blocks.length;
}
BlockInfo[] newlist = new BlockInfo[size + totalAddedBlocks];
BlockInfoContiguous[] newlist =
new BlockInfoContiguous[size + totalAddedBlocks];
System.arraycopy(this.blocks, 0, newlist, 0, size);
for(INodeFile in: inodes) {
@ -467,12 +470,12 @@ public class INodeFile extends INodeWithAdditionalFields
/**
* add a block to the block list
*/
void addBlock(BlockInfo newblock) {
void addBlock(BlockInfoContiguous newblock) {
if (this.blocks == null) {
this.setBlocks(new BlockInfo[]{newblock});
this.setBlocks(new BlockInfoContiguous[]{newblock});
} else {
int size = this.blocks.length;
BlockInfo[] newlist = new BlockInfo[size + 1];
BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1];
System.arraycopy(this.blocks, 0, newlist, 0, size);
newlist[size] = newblock;
this.setBlocks(newlist);
@ -480,7 +483,7 @@ public class INodeFile extends INodeWithAdditionalFields
}
/** Set the blocks. */
public void setBlocks(BlockInfo[] blocks) {
public void setBlocks(BlockInfoContiguous[] blocks) {
this.blocks = blocks;
}
@ -516,7 +519,7 @@ public class INodeFile extends INodeWithAdditionalFields
public void destroyAndCollectBlocks(BlocksMapUpdateInfo collectedBlocks,
final List<INode> removedINodes) {
if (blocks != null && collectedBlocks != null) {
for (BlockInfo blk : blocks) {
for (BlockInfoContiguous blk : blocks) {
collectedBlocks.addDeleteBlock(blk);
blk.setBlockCollection(null);
}
@ -637,7 +640,7 @@ public class INodeFile extends INodeWithAdditionalFields
final int last = blocks.length - 1;
//check if the last block is BlockInfoUnderConstruction
long size = blocks[last].getNumBytes();
if (blocks[last] instanceof BlockInfoUnderConstruction) {
if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) {
if (!includesLastUcBlock) {
size = 0;
} else if (usePreferredBlockSize4LastUcBlock) {
@ -667,7 +670,7 @@ public class INodeFile extends INodeWithAdditionalFields
Set<Block> allBlocks = new HashSet<Block>(Arrays.asList(getBlocks()));
List<FileDiff> diffs = sf.getDiffs().asList();
for(FileDiff diff : diffs) {
BlockInfo[] diffBlocks = diff.getBlocks();
BlockInfoContiguous[] diffBlocks = diff.getBlocks();
if (diffBlocks != null) {
allBlocks.addAll(Arrays.asList(diffBlocks));
}
@ -676,8 +679,9 @@ public class INodeFile extends INodeWithAdditionalFields
size += block.getNumBytes();
}
// check if the last block is under construction
BlockInfo lastBlock = getLastBlock();
if(lastBlock != null && lastBlock instanceof BlockInfoUnderConstruction) {
BlockInfoContiguous lastBlock = getLastBlock();
if(lastBlock != null &&
lastBlock instanceof BlockInfoContiguousUnderConstruction) {
size += getPreferredBlockSize() - lastBlock.getNumBytes();
}
return size * getBlockReplication();
@ -695,7 +699,7 @@ public class INodeFile extends INodeWithAdditionalFields
/**
* Return the penultimate allocated block for this file.
*/
BlockInfo getPenultimateBlock() {
BlockInfoContiguous getPenultimateBlock() {
if (blocks == null || blocks.length <= 1) {
return null;
}
@ -703,7 +707,7 @@ public class INodeFile extends INodeWithAdditionalFields
}
@Override
public BlockInfo getLastBlock() {
public BlockInfoContiguous getLastBlock() {
return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
}
@ -730,7 +734,7 @@ public class INodeFile extends INodeWithAdditionalFields
*/
public long collectBlocksBeyondMax(final long max,
final BlocksMapUpdateInfo collectedBlocks) {
final BlockInfo[] oldBlocks = getBlocks();
final BlockInfoContiguous[] oldBlocks = getBlocks();
if (oldBlocks == null)
return 0;
// find the minimum n such that the size of the first n blocks > max
@ -756,20 +760,20 @@ public class INodeFile extends INodeWithAdditionalFields
}
void truncateBlocksTo(int n) {
final BlockInfo[] newBlocks;
final BlockInfoContiguous[] newBlocks;
if (n == 0) {
newBlocks = BlockInfo.EMPTY_ARRAY;
newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
} else {
newBlocks = new BlockInfo[n];
newBlocks = new BlockInfoContiguous[n];
System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
}
// set new blocks
setBlocks(newBlocks);
}
public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks,
public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
BlocksMapUpdateInfo collectedBlocks) {
BlockInfo[] oldBlocks = getBlocks();
BlockInfoContiguous[] oldBlocks = getBlocks();
if(snapshotBlocks == null || oldBlocks == null)
return;
// Skip blocks in common between the file and the snapshot
@ -793,7 +797,7 @@ public class INodeFile extends INodeWithAdditionalFields
FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
if(sf == null)
return;
BlockInfo[] snapshotBlocks =
BlockInfoContiguous[] snapshotBlocks =
getDiffs().findEarlierSnapshotBlocks(snapshotId);
if(snapshotBlocks == null)
return;
@ -807,14 +811,14 @@ public class INodeFile extends INodeWithAdditionalFields
/**
* @return true if the block is contained in a snapshot or false otherwise.
*/
boolean isBlockInLatestSnapshot(BlockInfo block) {
boolean isBlockInLatestSnapshot(BlockInfoContiguous block) {
FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
if (sf == null || sf.getDiffs() == null)
if (sf == null || sf.getDiffs() == null) {
return false;
BlockInfo[] snapshotBlocks =
getDiffs().findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
if(snapshotBlocks == null)
return false;
return Arrays.asList(snapshotBlocks).contains(block);
}
BlockInfoContiguous[] snapshotBlocks = getDiffs()
.findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
return snapshotBlocks != null &&
Arrays.asList(snapshotBlocks).contains(block);
}
}

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.util.Daemon;
@ -120,10 +120,10 @@ public class LeaseManager {
} catch (UnresolvedLinkException e) {
throw new AssertionError("Lease files should reside on this FS");
}
BlockInfo[] blocks = cons.getBlocks();
BlockInfoContiguous[] blocks = cons.getBlocks();
if(blocks == null)
continue;
for(BlockInfo b : blocks) {
for(BlockInfoContiguous b : blocks) {
if(!b.isComplete())
numUCBlocks++;
}

View File

@ -61,7 +61,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactor
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
@ -228,7 +228,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
//get blockInfo
Block block = new Block(Block.getBlockId(blockId));
//find which file this block belongs to
BlockInfo blockInfo = bm.getStoredBlock(block);
BlockInfoContiguous blockInfo = bm.getStoredBlock(block);
if(blockInfo == null) {
out.println("Block "+ blockId +" " + NONEXISTENT_STATUS);
LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS);

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.util.RwLock;
import org.apache.hadoop.ipc.StandbyException;
@ -45,5 +45,5 @@ public interface Namesystem extends RwLock, SafeMode {
public void checkOperation(OperationCategory read) throws StandbyException;
public boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
}

View File

@ -39,7 +39,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat;
import org.apache.hadoop.hdfs.server.namenode.AclFeature;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@ -235,13 +235,13 @@ public class FSImageFormatPBSnapshot {
FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
pbf.getFileSize());
List<BlockProto> bpl = pbf.getBlocksList();
BlockInfo[] blocks = new BlockInfo[bpl.size()];
BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()];
for(int j = 0, e = bpl.size(); j < e; ++j) {
Block blk = PBHelper.convert(bpl.get(j));
BlockInfo storedBlock = fsn.getBlockManager().getStoredBlock(blk);
BlockInfoContiguous storedBlock = fsn.getBlockManager().getStoredBlock(blk);
if(storedBlock == null) {
storedBlock = fsn.getBlockManager().addBlockCollection(
new BlockInfo(blk, copy.getFileReplication()), file);
new BlockInfoContiguous(blk, copy.getFileReplication()), file);
}
blocks[j] = storedBlock;
}

View File

@ -22,7 +22,7 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@ -40,7 +40,7 @@ public class FileDiff extends
/** The file size at snapshot creation time. */
private final long fileSize;
/** A copy of the INodeFile block list. Used in truncate. */
private BlockInfo[] blocks;
private BlockInfoContiguous[] blocks;
FileDiff(int snapshotId, INodeFile file) {
super(snapshotId, null, null);
@ -66,7 +66,7 @@ public class FileDiff extends
* up to the current {@link #fileSize}.
* Should be done only once.
*/
public void setBlocks(BlockInfo[] blocks) {
public void setBlocks(BlockInfoContiguous[] blocks) {
if(this.blocks != null)
return;
int numBlocks = 0;
@ -75,7 +75,7 @@ public class FileDiff extends
this.blocks = Arrays.copyOf(blocks, numBlocks);
}
public BlockInfo[] getBlocks() {
public BlockInfoContiguous[] getBlocks() {
return blocks;
}
@ -121,7 +121,7 @@ public class FileDiff extends
BlocksMapUpdateInfo collectedBlocks) {
if(blocks == null || collectedBlocks == null)
return;
for(BlockInfo blk : blocks)
for(BlockInfoContiguous blk : blocks)
collectedBlocks.addDeleteBlock(blk);
blocks = null;
}

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@ -54,14 +54,14 @@ public class FileDiffList extends
diff.setBlocks(iNodeFile.getBlocks());
}
public BlockInfo[] findEarlierSnapshotBlocks(int snapshotId) {
public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
if(snapshotId == Snapshot.CURRENT_STATE_ID) {
return null;
}
List<FileDiff> diffs = this.asList();
int i = Collections.binarySearch(diffs, snapshotId);
BlockInfo[] blocks = null;
BlockInfoContiguous[] blocks = null;
for(i = i >= 0 ? i : -i; i < diffs.size(); i--) {
blocks = diffs.get(i).getBlocks();
if(blocks != null) {
@ -71,14 +71,14 @@ public class FileDiffList extends
return blocks;
}
public BlockInfo[] findLaterSnapshotBlocks(int snapshotId) {
public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) {
assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
if(snapshotId == Snapshot.CURRENT_STATE_ID) {
return null;
}
List<FileDiff> diffs = this.asList();
int i = Collections.binarySearch(diffs, snapshotId);
BlockInfo[] blocks = null;
BlockInfoContiguous[] blocks = null;
for(i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
blocks = diffs.get(i).getBlocks();
if(blocks != null) {
@ -97,7 +97,7 @@ public class FileDiffList extends
FileDiff removed,
BlocksMapUpdateInfo collectedBlocks,
List<INode> removedINodes) {
BlockInfo[] removedBlocks = removed.getBlocks();
BlockInfoContiguous[] removedBlocks = removed.getBlocks();
if(removedBlocks == null) {
FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
assert sf != null : "FileWithSnapshotFeature is null";
@ -110,10 +110,10 @@ public class FileDiffList extends
// Copy blocks to the previous snapshot if not set already
if(earlierDiff != null)
earlierDiff.setBlocks(removedBlocks);
BlockInfo[] earlierBlocks =
(earlierDiff == null ? new BlockInfo[]{} : earlierDiff.getBlocks());
BlockInfoContiguous[] earlierBlocks =
(earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
// Find later snapshot (or file itself) with blocks
BlockInfo[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks;
// Skip blocks, which belong to either the earlier or the later lists
int i = 0;

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.namenode.AclFeature;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
@ -183,7 +183,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
// Collect blocks that should be deleted
FileDiff last = diffs.getLast();
BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
BlockInfoContiguous[] snapshotBlocks = last == null ? null : last.getBlocks();
if(snapshotBlocks == null)
file.collectBlocksBeyondMax(max, info);
else

View File

@ -47,8 +47,8 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@ -1542,12 +1542,12 @@ public class DFSTestUtil {
public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn,
ExtendedBlock blk) {
BlockManager bm0 = nn.getNamesystem().getBlockManager();
BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
BlockInfoContiguous storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
assertTrue("Block " + blk + " should be under construction, " +
"got: " + storedBlock,
storedBlock instanceof BlockInfoUnderConstruction);
BlockInfoUnderConstruction ucBlock =
(BlockInfoUnderConstruction)storedBlock;
storedBlock instanceof BlockInfoContiguousUnderConstruction);
BlockInfoContiguousUnderConstruction ucBlock =
(BlockInfoContiguousUnderConstruction)storedBlock;
// We expect that the replica with the most recent heart beat will be
// the one to be in charge of the synchronization / recovery protocol.
final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();

View File

@ -49,7 +49,7 @@ public class TestBlockInfo {
@Test
public void testAddStorage() throws Exception {
BlockInfo blockInfo = new BlockInfo((short) 3);
BlockInfoContiguous blockInfo = new BlockInfoContiguous((short) 3);
final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
@ -67,11 +67,11 @@ public class TestBlockInfo {
final DatanodeStorageInfo storage1 = DFSTestUtil.createDatanodeStorageInfo("storageID1", "127.0.0.1");
final DatanodeStorageInfo storage2 = new DatanodeStorageInfo(storage1.getDatanodeDescriptor(), new DatanodeStorage("storageID2"));
final int NUM_BLOCKS = 10;
BlockInfo[] blockInfos = new BlockInfo[NUM_BLOCKS];
BlockInfoContiguous[] blockInfos = new BlockInfoContiguous[NUM_BLOCKS];
// Create a few dummy blocks and add them to the first storage.
for (int i = 0; i < NUM_BLOCKS; ++i) {
blockInfos[i] = new BlockInfo((short) 3);
blockInfos[i] = new BlockInfoContiguous((short) 3);
storage1.addBlock(blockInfos[i]);
}
@ -90,14 +90,14 @@ public class TestBlockInfo {
DatanodeStorageInfo dd = DFSTestUtil.createDatanodeStorageInfo("s1", "1.1.1.1");
ArrayList<Block> blockList = new ArrayList<Block>(MAX_BLOCKS);
ArrayList<BlockInfo> blockInfoList = new ArrayList<BlockInfo>();
ArrayList<BlockInfoContiguous> blockInfoList = new ArrayList<BlockInfoContiguous>();
int headIndex;
int curIndex;
LOG.info("Building block list...");
for (int i = 0; i < MAX_BLOCKS; i++) {
blockList.add(new Block(i, 0, GenerationStamp.LAST_RESERVED_STAMP));
blockInfoList.add(new BlockInfo(blockList.get(i), (short) 3));
blockInfoList.add(new BlockInfoContiguous(blockList.get(i), (short) 3));
dd.addBlock(blockInfoList.get(i));
// index of the datanode should be 0
@ -108,7 +108,7 @@ public class TestBlockInfo {
// list length should be equal to the number of blocks we inserted
LOG.info("Checking list length...");
assertEquals("Length should be MAX_BLOCK", MAX_BLOCKS, dd.numBlocks());
Iterator<BlockInfo> it = dd.getBlockIterator();
Iterator<BlockInfoContiguous> it = dd.getBlockIterator();
int len = 0;
while (it.hasNext()) {
it.next();
@ -130,7 +130,7 @@ public class TestBlockInfo {
// move head of the list to the head - this should not change the list
LOG.info("Moving head to the head...");
BlockInfo temp = dd.getBlockListHeadForTesting();
BlockInfoContiguous temp = dd.getBlockListHeadForTesting();
curIndex = 0;
headIndex = 0;
dd.moveBlockToHead(temp, curIndex, headIndex);

View File

@ -39,7 +39,7 @@ public class TestBlockInfoUnderConstruction {
DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
(short) 3,
BlockUCState.UNDER_CONSTRUCTION,
@ -51,7 +51,7 @@ public class TestBlockInfoUnderConstruction {
dd2.setLastUpdate(currentTime - 1 * 1000);
dd3.setLastUpdate(currentTime - 2 * 1000);
blockInfo.initializeBlockRecovery(1);
BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
assertEquals(blockInfoRecovery[0], blockInfo);
// Recovery attempt #2.

View File

@ -141,7 +141,7 @@ public class TestBlockManager {
private void doBasicTest(int testIndex) {
List<DatanodeStorageInfo> origStorages = getStorages(0, 1);
List<DatanodeDescriptor> origNodes = getNodes(origStorages);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo);
assertEquals(2, pipeline.length);
@ -173,7 +173,7 @@ public class TestBlockManager {
// Block originally on A1, A2, B1
List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
List<DatanodeDescriptor> origNodes = getNodes(origStorages);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission two of the nodes (A1, A2)
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1);
@ -217,7 +217,7 @@ public class TestBlockManager {
// Block originally on A1, A2, B1
List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
List<DatanodeDescriptor> origNodes = getNodes(origStorages);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission all of the nodes
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 3);
@ -270,7 +270,7 @@ public class TestBlockManager {
// Block originally on A1, A2, B1
List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
List<DatanodeDescriptor> origNodes = getNodes(origStorages);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission all of the nodes in rack A
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 2);
@ -329,7 +329,7 @@ public class TestBlockManager {
private void doTestSufficientlyReplBlocksUsesNewRack(int testIndex) {
// Originally on only nodes in rack A.
List<DatanodeDescriptor> origNodes = rackA;
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
DatanodeStorageInfo pipeline[] = scheduleSingleReplication(blockInfo);
assertEquals(2, pipeline.length); // single new copy
@ -372,7 +372,7 @@ public class TestBlockManager {
* Tell the block manager that replication is completed for the given
* pipeline.
*/
private void fulfillPipeline(BlockInfo blockInfo,
private void fulfillPipeline(BlockInfoContiguous blockInfo,
DatanodeStorageInfo[] pipeline) throws IOException {
for (int i = 1; i < pipeline.length; i++) {
DatanodeStorageInfo storage = pipeline[i];
@ -381,9 +381,9 @@ public class TestBlockManager {
}
}
private BlockInfo blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
private BlockInfoContiguous blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
Block block = new Block(blkId);
BlockInfo blockInfo = new BlockInfo(block, (short) 3);
BlockInfoContiguous blockInfo = new BlockInfoContiguous(block, (short) 3);
for (DatanodeDescriptor dn : nodes) {
for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
@ -425,10 +425,10 @@ public class TestBlockManager {
return nodes;
}
private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
private BlockInfoContiguous addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
BlockCollection bc = Mockito.mock(BlockCollection.class);
Mockito.doReturn((short)3).when(bc).getBlockReplication();
BlockInfo blockInfo = blockOnNodes(blockId, nodes);
BlockInfoContiguous blockInfo = blockOnNodes(blockId, nodes);
bm.blocksMap.addBlockCollection(blockInfo, bc);
return blockInfo;

View File

@ -58,8 +58,8 @@ public class TestDatanodeDescriptor {
public void testBlocksCounter() throws Exception {
DatanodeDescriptor dd = BlockManagerTestUtil.getLocalDatanodeDescriptor(true);
assertEquals(0, dd.numBlocks());
BlockInfo blk = new BlockInfo(new Block(1L), (short) 1);
BlockInfo blk1 = new BlockInfo(new Block(2L), (short) 2);
BlockInfoContiguous blk = new BlockInfoContiguous(new Block(1L), (short) 1);
BlockInfoContiguous blk1 = new BlockInfoContiguous(new Block(2L), (short) 2);
DatanodeStorageInfo[] storages = dd.getStorageInfos();
assertTrue(storages.length > 0);
// add first block

View File

@ -171,7 +171,7 @@ public class TestHeartbeatHandling {
dd1.getStorageInfos()[0],
dd2.getStorageInfos()[0],
dd3.getStorageInfos()[0]};
BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);
@ -193,7 +193,7 @@ public class TestHeartbeatHandling {
// More than the default stale interval of 30 seconds.
dd2.setLastUpdate(System.currentTimeMillis() - 40 * 1000);
dd3.setLastUpdate(System.currentTimeMillis());
blockInfo = new BlockInfoUnderConstruction(
blockInfo = new BlockInfoContiguousUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);
@ -214,7 +214,7 @@ public class TestHeartbeatHandling {
// More than the default stale interval of 30 seconds.
dd2.setLastUpdate(System.currentTimeMillis() - 40 * 1000);
dd3.setLastUpdate(System.currentTimeMillis() - 80 * 1000);
blockInfo = new BlockInfoUnderConstruction(
blockInfo = new BlockInfoContiguousUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);

View File

@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.Storage;
@ -66,10 +66,10 @@ public class CreateEditsLog {
INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p,
0L);
editLog.logMkDir(BASE_PATH, dirInode);
BlockInfo[] blocks = new BlockInfo[blocksPerFile];
BlockInfoContiguous[] blocks = new BlockInfoContiguous[blocksPerFile];
for (int iB = 0; iB < blocksPerFile; ++iB) {
blocks[iB] =
new BlockInfo(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
replication);
}
@ -97,7 +97,7 @@ public class CreateEditsLog {
editLog.logMkDir(currentDir, dirInode);
}
INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize, (byte)0);
p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
fileUc.toUnderConstruction("", "");
editLog.logOpenFile(filePath, fileUc, false, false);
editLog.logCloseFile(filePath, inode);

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.junit.After;
import org.junit.Before;
@ -87,21 +87,21 @@ public class TestAddBlock {
// check file1
INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
BlockInfo[] file1Blocks = file1Node.getBlocks();
BlockInfoContiguous[] file1Blocks = file1Node.getBlocks();
assertEquals(1, file1Blocks.length);
assertEquals(BLOCKSIZE - 1, file1Blocks[0].getNumBytes());
assertEquals(BlockUCState.COMPLETE, file1Blocks[0].getBlockUCState());
// check file2
INodeFile file2Node = fsdir.getINode4Write(file2.toString()).asFile();
BlockInfo[] file2Blocks = file2Node.getBlocks();
BlockInfoContiguous[] file2Blocks = file2Node.getBlocks();
assertEquals(1, file2Blocks.length);
assertEquals(BLOCKSIZE, file2Blocks[0].getNumBytes());
assertEquals(BlockUCState.COMPLETE, file2Blocks[0].getBlockUCState());
// check file3
INodeFile file3Node = fsdir.getINode4Write(file3.toString()).asFile();
BlockInfo[] file3Blocks = file3Node.getBlocks();
BlockInfoContiguous[] file3Blocks = file3Node.getBlocks();
assertEquals(2, file3Blocks.length);
assertEquals(BLOCKSIZE, file3Blocks[0].getNumBytes());
assertEquals(BlockUCState.COMPLETE, file3Blocks[0].getBlockUCState());
@ -110,7 +110,7 @@ public class TestAddBlock {
// check file4
INodeFile file4Node = fsdir.getINode4Write(file4.toString()).asFile();
BlockInfo[] file4Blocks = file4Node.getBlocks();
BlockInfoContiguous[] file4Blocks = file4Node.getBlocks();
assertEquals(2, file4Blocks.length);
assertEquals(BLOCKSIZE, file4Blocks[0].getNumBytes());
assertEquals(BlockUCState.COMPLETE, file4Blocks[0].getBlockUCState());
@ -141,7 +141,7 @@ public class TestAddBlock {
FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
INodeFile fileNode = fsdir.getINode4Write(file1.toString()).asFile();
BlockInfo[] fileBlocks = fileNode.getBlocks();
BlockInfoContiguous[] fileBlocks = fileNode.getBlocks();
assertEquals(2, fileBlocks.length);
assertEquals(BLOCKSIZE, fileBlocks[0].getNumBytes());
assertEquals(BlockUCState.COMPLETE, fileBlocks[0].getBlockUCState());

View File

@ -35,8 +35,8 @@ import org.apache.hadoop.hdfs.TestFileCreation;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.junit.AfterClass;
@ -91,12 +91,12 @@ public class TestBlockUnderConstruction {
" isUnderConstruction = " + inode.isUnderConstruction() +
" expected to be " + isFileOpen,
inode.isUnderConstruction() == isFileOpen);
BlockInfo[] blocks = inode.getBlocks();
BlockInfoContiguous[] blocks = inode.getBlocks();
assertTrue("File does not have blocks: " + inode.toString(),
blocks != null && blocks.length > 0);
int idx = 0;
BlockInfo curBlock;
BlockInfoContiguous curBlock;
// all blocks but the last two should be regular blocks
for(; idx < blocks.length - 2; idx++) {
curBlock = blocks[idx];
@ -170,7 +170,7 @@ public class TestBlockUnderConstruction {
final List<LocatedBlock> blocks = lb.getLocatedBlocks();
assertEquals(i, blocks.size());
final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock();
assertTrue(b instanceof BlockInfoUnderConstruction);
assertTrue(b instanceof BlockInfoContiguousUnderConstruction);
if (++i < NUM_BLOCKS) {
// write one more block

View File

@ -22,8 +22,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.junit.Test;
@ -64,7 +64,7 @@ public class TestCommitBlockSynchronization {
namesystem.dir.getINodeMap().put(file);
FSNamesystem namesystemSpy = spy(namesystem);
BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
blockInfo.setBlockCollection(file);
blockInfo.setGenerationStamp(genStamp);
@ -75,7 +75,7 @@ public class TestCommitBlockSynchronization {
doReturn(blockInfo).when(namesystemSpy).getStoredBlock(any(Block.class));
doReturn(blockInfo).when(file).getLastBlock();
doReturn("").when(namesystemSpy).closeFileCommitBlocks(
any(INodeFile.class), any(BlockInfo.class));
any(INodeFile.class), any(BlockInfoContiguous.class));
doReturn(mock(FSEditLog.class)).when(namesystemSpy).getEditLog();
return namesystemSpy;
@ -103,7 +103,7 @@ public class TestCommitBlockSynchronization {
lastBlock, genStamp, length, false, false, newTargets, null);
// Simulate 'completing' the block.
BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
BlockInfoContiguous completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
completedBlockInfo.setBlockCollection(file);
completedBlockInfo.setGenerationStamp(genStamp);
doReturn(completedBlockInfo).when(namesystemSpy)
@ -175,7 +175,7 @@ public class TestCommitBlockSynchronization {
namesystemSpy.commitBlockSynchronization(
lastBlock, genStamp, length, true, false, newTargets, null);
BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
BlockInfoContiguous completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
completedBlockInfo.setBlockCollection(file);
completedBlockInfo.setGenerationStamp(genStamp);
doReturn(completedBlockInfo).when(namesystemSpy)

View File

@ -70,7 +70,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
@ -205,7 +205,7 @@ public class TestEditLog {
for (int i = 0; i < numTransactions; i++) {
INodeFile inode = new INodeFile(namesystem.dir.allocateNewInodeId(), null,
p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize, (byte)0);
p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
inode.toUnderConstruction("", "");
editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false);

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
@ -97,7 +97,7 @@ public class TestFSImage {
INodeFile file2Node = fsn.dir.getINode4Write(file2.toString()).asFile();
assertEquals("hello".length(), file2Node.computeFileSize());
assertTrue(file2Node.isUnderConstruction());
BlockInfo[] blks = file2Node.getBlocks();
BlockInfoContiguous[] blks = file2Node.getBlocks();
assertEquals(1, blks.length);
assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState());
// check lease manager

View File

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

View File

@ -65,7 +65,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@ -714,7 +714,7 @@ public class TestFsck {
// intentionally corrupt NN data structure
INodeFile node = (INodeFile) cluster.getNamesystem().dir.getINode
(fileName, true);
final BlockInfo[] blocks = node.getBlocks();
final BlockInfoContiguous[] blocks = node.getBlocks();
assertEquals(blocks.length, 1);
blocks[0].setNumBytes(-1L); // set the block length to be negative

View File

@ -58,7 +58,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.io.IOUtils;
@ -288,7 +288,7 @@ public class TestINodeFile {
iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
preferredBlockSize, (byte)0);
iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
BlockInfo newblock = new BlockInfo(replication);
BlockInfoContiguous newblock = new BlockInfoContiguous(replication);
iNodes[i].addBlock(newblock);
}

View File

@ -28,7 +28,6 @@ import java.util.concurrent.TimeoutException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -42,12 +41,8 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@ -60,7 +55,6 @@ import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.Test;
import org.mockito.Mockito;

View File

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

View File

@ -34,7 +34,6 @@ import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -44,8 +43,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
@ -63,7 +62,6 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine.Server;
import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.junit.Assert;
/**
@ -178,8 +176,8 @@ public class SnapshotTestHelper {
*
* Specific information for different types of INode:
* {@link INodeDirectory}:childrenSize
* {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
* and {@link BlockInfoUnderConstruction#toString()} for detailed information.
* {@link INodeFile}: fileSize, block list. Check {@link BlockInfoContiguous#toString()}
* and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
* {@link FileWithSnapshot}: next link
* </pre>
* @see INode#dumpTreeRecursively()

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@ -108,14 +108,14 @@ public class TestSnapshotBlocksMap {
final FSDirectory dir, final BlockManager blkManager) throws Exception {
final INodeFile file = INodeFile.valueOf(dir.getINode(path), path);
assertEquals(numBlocks, file.getBlocks().length);
for(BlockInfo b : file.getBlocks()) {
for(BlockInfoContiguous b : file.getBlocks()) {
assertBlockCollection(blkManager, file, b);
}
return file;
}
static void assertBlockCollection(final BlockManager blkManager,
final INodeFile file, final BlockInfo b) {
final INodeFile file, final BlockInfoContiguous b) {
Assert.assertSame(b, blkManager.getStoredBlock(b));
Assert.assertSame(file, blkManager.getBlockCollection(b));
Assert.assertSame(file, b.getBlockCollection());
@ -146,10 +146,10 @@ public class TestSnapshotBlocksMap {
{
final INodeFile f2 = assertBlockCollection(file2.toString(), 3, fsdir,
blockmanager);
BlockInfo[] blocks = f2.getBlocks();
BlockInfoContiguous[] blocks = f2.getBlocks();
hdfs.delete(sub2, true);
// The INode should have been removed from the blocksMap
for(BlockInfo b : blocks) {
for(BlockInfoContiguous b : blocks) {
assertNull(blockmanager.getBlockCollection(b));
}
}
@ -177,7 +177,7 @@ public class TestSnapshotBlocksMap {
// Check the block information for file0
final INodeFile f0 = assertBlockCollection(file0.toString(), 4, fsdir,
blockmanager);
BlockInfo[] blocks0 = f0.getBlocks();
BlockInfoContiguous[] blocks0 = f0.getBlocks();
// Also check the block information for snapshot of file0
Path snapshotFile0 = SnapshotTestHelper.getSnapshotPath(sub1, "s0",
@ -187,7 +187,7 @@ public class TestSnapshotBlocksMap {
// Delete file0
hdfs.delete(file0, true);
// Make sure the blocks of file0 is still in blocksMap
for(BlockInfo b : blocks0) {
for(BlockInfoContiguous b : blocks0) {
assertNotNull(blockmanager.getBlockCollection(b));
}
assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
@ -201,7 +201,7 @@ public class TestSnapshotBlocksMap {
hdfs.deleteSnapshot(sub1, "s1");
// Make sure the first block of file0 is still in blocksMap
for(BlockInfo b : blocks0) {
for(BlockInfoContiguous b : blocks0) {
assertNotNull(blockmanager.getBlockCollection(b));
}
assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
@ -293,7 +293,7 @@ public class TestSnapshotBlocksMap {
hdfs.append(bar);
INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
BlockInfo[] blks = barNode.getBlocks();
BlockInfoContiguous[] blks = barNode.getBlocks();
assertEquals(1, blks.length);
assertEquals(BLOCKSIZE, blks[0].getNumBytes());
ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
@ -331,7 +331,7 @@ public class TestSnapshotBlocksMap {
hdfs.append(bar);
INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
BlockInfo[] blks = barNode.getBlocks();
BlockInfoContiguous[] blks = barNode.getBlocks();
assertEquals(1, blks.length);
ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
cluster.getNameNodeRpc()
@ -370,7 +370,7 @@ public class TestSnapshotBlocksMap {
hdfs.append(bar);
INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
BlockInfo[] blks = barNode.getBlocks();
BlockInfoContiguous[] blks = barNode.getBlocks();
assertEquals(1, blks.length);
ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
cluster.getNameNodeRpc()
@ -421,7 +421,7 @@ public class TestSnapshotBlocksMap {
out.write(testData);
out.close();
INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
BlockInfo[] blks = barNode.getBlocks();
BlockInfoContiguous[] blks = barNode.getBlocks();
assertEquals(1, blks.length);
assertEquals(testData.length, blks[0].getNumBytes());

View File

@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@ -260,12 +260,12 @@ public class TestSnapshotDeletion {
DFSTestUtil.createFile(hdfs, tempFile, BLOCKSIZE, REPLICATION, seed);
final INodeFile temp = TestSnapshotBlocksMap.assertBlockCollection(
tempFile.toString(), 1, fsdir, blockmanager);
BlockInfo[] blocks = temp.getBlocks();
BlockInfoContiguous[] blocks = temp.getBlocks();
hdfs.delete(tempDir, true);
// check dir's quota usage
checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
// check blocks of tempFile
for (BlockInfo b : blocks) {
for (BlockInfoContiguous b : blocks) {
assertNull(blockmanager.getBlockCollection(b));
}
@ -342,7 +342,7 @@ public class TestSnapshotDeletion {
// while deletion, we add diff for subsub and metaChangeFile1, and remove
// newFile
checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
for (BlockInfo b : blocks) {
for (BlockInfoContiguous b : blocks) {
assertNull(blockmanager.getBlockCollection(b));
}
@ -479,7 +479,7 @@ public class TestSnapshotDeletion {
final INodeFile toDeleteFileNode = TestSnapshotBlocksMap
.assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager);
BlockInfo[] blocks = toDeleteFileNode.getBlocks();
BlockInfoContiguous[] blocks = toDeleteFileNode.getBlocks();
// create snapshot s0 on dir
SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
@ -505,7 +505,7 @@ public class TestSnapshotDeletion {
// metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and
// metaChangeFile's replication factor decreases
checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
for (BlockInfo b : blocks) {
for (BlockInfoContiguous b : blocks) {
assertNull(blockmanager.getBlockCollection(b));
}
@ -799,7 +799,7 @@ public class TestSnapshotDeletion {
FileStatus statusBeforeDeletion13 = hdfs.getFileStatus(file13_s1);
INodeFile file14Node = TestSnapshotBlocksMap.assertBlockCollection(
file14_s2.toString(), 1, fsdir, blockmanager);
BlockInfo[] blocks_14 = file14Node.getBlocks();
BlockInfoContiguous[] blocks_14 = file14Node.getBlocks();
TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir,
blockmanager);
@ -836,7 +836,7 @@ public class TestSnapshotDeletion {
modDirStr + "file15");
assertFalse(hdfs.exists(file14_s1));
assertFalse(hdfs.exists(file15_s1));
for (BlockInfo b : blocks_14) {
for (BlockInfoContiguous b : blocks_14) {
assertNull(blockmanager.getBlockCollection(b));
}