HDFS-2228. Move block and datanode code from FSNamesystem to BlockManager and DatanodeManager. (szetszwo)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1154899 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2011-08-08 10:06:45 +00:00
parent 8dc420ba36
commit 371f4a5905
25 changed files with 636 additions and 770 deletions

View File

@ -641,6 +641,9 @@ Trunk (unreleased changes)
HDFS-2226. Clean up counting of operations in FSEditLogLoader (todd) HDFS-2226. Clean up counting of operations in FSEditLogLoader (todd)
HDFS-2228. Move block and datanode code from FSNamesystem to
BlockManager and DatanodeManager. (szetszwo)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-1458. Improve checkpoint performance by avoiding unnecessary image HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

View File

@ -259,26 +259,6 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
return head; return head;
} }
boolean listIsConsistent(DatanodeDescriptor dn) {
// going forward
int count = 0;
BlockInfo next, nextPrev;
BlockInfo cur = this;
while(cur != null) {
next = cur.getNext(cur.findDatanode(dn));
if(next != null) {
nextPrev = next.getPrevious(next.findDatanode(dn));
if(cur != nextPrev) {
System.out.println("Inconsistent list: cur->next->prev != cur");
return false;
}
}
cur = next;
count++;
}
return true;
}
/** /**
* BlockInfo represents a block that is not being constructed. * BlockInfo represents a block that is not being constructed.
* In order to start modifying the block, the BlockInfo should be converted * In order to start modifying the block, the BlockInfo should be converted

View File

@ -46,13 +46,15 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks.BlockIterator; import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks.BlockIterator;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@ -60,8 +62,9 @@ import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.net.Node; import org.apache.hadoop.net.Node;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
/** /**
@ -81,18 +84,13 @@ public class BlockManager {
private volatile long pendingReplicationBlocksCount = 0L; private volatile long pendingReplicationBlocksCount = 0L;
private volatile long corruptReplicaBlocksCount = 0L; private volatile long corruptReplicaBlocksCount = 0L;
private volatile long underReplicatedBlocksCount = 0L; private volatile long underReplicatedBlocksCount = 0L;
public volatile long scheduledReplicationBlocksCount = 0L; private volatile long scheduledReplicationBlocksCount = 0L;
private volatile long excessBlocksCount = 0L; private volatile long excessBlocksCount = 0L;
private volatile long pendingDeletionBlocksCount = 0L; private volatile long pendingDeletionBlocksCount = 0L;
private boolean isBlockTokenEnabled; private boolean isBlockTokenEnabled;
private long blockKeyUpdateInterval; private long blockKeyUpdateInterval;
private long blockTokenLifetime; private long blockTokenLifetime;
private BlockTokenSecretManager blockTokenSecretManager; private BlockTokenSecretManager blockTokenSecretManager;
/** returns the isBlockTokenEnabled - true if block token enabled ,else false */
public boolean isBlockTokenEnabled() {
return isBlockTokenEnabled;
}
/** get the BlockTokenSecretManager */ /** get the BlockTokenSecretManager */
public BlockTokenSecretManager getBlockTokenSecretManager() { public BlockTokenSecretManager getBlockTokenSecretManager() {
@ -131,7 +129,7 @@ public class BlockManager {
* Mapping: Block -> { INode, datanodes, self ref } * Mapping: Block -> { INode, datanodes, self ref }
* Updated only in response to client-sent information. * Updated only in response to client-sent information.
*/ */
public final BlocksMap blocksMap; final BlocksMap blocksMap;
private final DatanodeManager datanodeManager; private final DatanodeManager datanodeManager;
private final HeartbeatManager heartbeatManager; private final HeartbeatManager heartbeatManager;
@ -168,13 +166,13 @@ public class BlockManager {
private final PendingReplicationBlocks pendingReplications; private final PendingReplicationBlocks pendingReplications;
/** The maximum number of replicas allowed for a block */ /** The maximum number of replicas allowed for a block */
public final int maxReplication; public final short maxReplication;
/** The maximum number of outgoing replication streams /** The maximum number of outgoing replication streams
* a given node should have at one time * a given node should have at one time
*/ */
int maxReplicationStreams; int maxReplicationStreams;
/** Minimum copies needed or else write is disallowed */ /** Minimum copies needed or else write is disallowed */
public final int minReplication; public final short minReplication;
/** Default number of replicas */ /** Default number of replicas */
public final int defaultReplication; public final int defaultReplication;
/** The maximum number of entries returned by getCorruptInodes() */ /** The maximum number of entries returned by getCorruptInodes() */
@ -189,30 +187,6 @@ public class BlockManager {
/** for block replicas placement */ /** for block replicas placement */
private BlockPlacementPolicy blockplacement; private BlockPlacementPolicy blockplacement;
/**
* Get access keys
*
* @return current access keys
*/
public ExportedBlockKeys getBlockKeys() {
return isBlockTokenEnabled ? blockTokenSecretManager.exportKeys()
: ExportedBlockKeys.DUMMY_KEYS;
}
/** Generate block token for a LocatedBlock. */
public void setBlockToken(LocatedBlock l) throws IOException {
Token<BlockTokenIdentifier> token = blockTokenSecretManager.generateToken(l
.getBlock(), EnumSet.of(BlockTokenSecretManager.AccessMode.READ));
l.setBlockToken(token);
}
/** Generate block tokens for the blocks to be returned. */
public void setBlockTokens(List<LocatedBlock> locatedBlocks) throws IOException {
for(LocatedBlock l : locatedBlocks) {
setBlockToken(l);
}
}
public BlockManager(FSNamesystem fsn, Configuration conf) throws IOException { public BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
namesystem = fsn; namesystem = fsn;
datanodeManager = new DatanodeManager(this, fsn, conf); datanodeManager = new DatanodeManager(this, fsn, conf);
@ -249,25 +223,28 @@ public class BlockManager {
DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED); DFSConfigKeys.DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED);
this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, this.defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
DFSConfigKeys.DFS_REPLICATION_DEFAULT); DFSConfigKeys.DFS_REPLICATION_DEFAULT);
this.maxReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_MAX_KEY,
DFSConfigKeys.DFS_REPLICATION_MAX_DEFAULT); final int maxR = conf.getInt(DFSConfigKeys.DFS_REPLICATION_MAX_KEY,
this.minReplication = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, DFSConfigKeys.DFS_REPLICATION_MAX_DEFAULT);
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT); final int minR = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
if (minReplication <= 0) DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
throw new IOException( if (minR <= 0)
"Unexpected configuration parameters: dfs.namenode.replication.min = " throw new IOException("Unexpected configuration parameters: "
+ minReplication + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
+ " must be greater than 0"); + " = " + minR + " <= 0");
if (maxReplication >= (int)Short.MAX_VALUE) if (maxR > Short.MAX_VALUE)
throw new IOException( throw new IOException("Unexpected configuration parameters: "
"Unexpected configuration parameters: dfs.replication.max = " + DFSConfigKeys.DFS_REPLICATION_MAX_KEY
+ maxReplication + " must be less than " + (Short.MAX_VALUE)); + " = " + maxR + " > " + Short.MAX_VALUE);
if (maxReplication < minReplication) if (minR > maxR)
throw new IOException( throw new IOException("Unexpected configuration parameters: "
"Unexpected configuration parameters: dfs.namenode.replication.min = " + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY
+ minReplication + " = " + minR + " > "
+ " must be less than dfs.replication.max = " + DFSConfigKeys.DFS_REPLICATION_MAX_KEY
+ maxReplication); + " = " + maxR);
this.minReplication = (short)minR;
this.maxReplication = (short)maxR;
this.maxReplicationStreams = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, this.maxReplicationStreams = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT); DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null ? false this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null ? false
@ -517,7 +494,7 @@ public class BlockManager {
} }
long fileLength = fileINode.computeContentSummary().getLength(); long fileLength = fileINode.computeContentSummary().getLength();
return getBlockLocation(ucBlock, fileLength - ucBlock.getNumBytes()); return createLocatedBlock(ucBlock, fileLength - ucBlock.getNumBytes());
} }
/** /**
@ -537,8 +514,9 @@ public class BlockManager {
return machineSet; return machineSet;
} }
public List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset, private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
long length, int nrBlocksToReturn) throws IOException { final long offset, final long length, final int nrBlocksToReturn
) throws IOException {
int curBlk = 0; int curBlk = 0;
long curPos = 0, blkSize = 0; long curPos = 0, blkSize = 0;
int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length; int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
@ -557,7 +535,7 @@ public class BlockManager {
long endOff = offset + length; long endOff = offset + length;
List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length); List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
do { do {
results.add(getBlockLocation(blocks[curBlk], curPos)); results.add(createLocatedBlock(blocks[curBlk], curPos));
curPos += blocks[curBlk].getNumBytes(); curPos += blocks[curBlk].getNumBytes();
curBlk++; curBlk++;
} while (curPos < endOff } while (curPos < endOff
@ -567,7 +545,7 @@ public class BlockManager {
} }
/** @return a LocatedBlock for the given block */ /** @return a LocatedBlock for the given block */
public LocatedBlock getBlockLocation(final BlockInfo blk, final long pos private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
) throws IOException { ) throws IOException {
if (blk instanceof BlockInfoUnderConstruction) { if (blk instanceof BlockInfoUnderConstruction) {
if (blk.isComplete()) { if (blk.isComplete()) {
@ -608,6 +586,76 @@ public class BlockManager {
return new LocatedBlock(eb, machines, pos, isCorrupt); return new LocatedBlock(eb, machines, pos, isCorrupt);
} }
/** Create a LocatedBlocks. */
public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks,
final long fileSizeExcludeBlocksUnderConstruction,
final boolean isFileUnderConstruction,
final long offset, final long length, final boolean needBlockToken
) throws IOException {
assert namesystem.hasReadOrWriteLock();
if (blocks == null) {
return null;
} else if (blocks.length == 0) {
return new LocatedBlocks(0, isFileUnderConstruction,
Collections.<LocatedBlock>emptyList(), null, false);
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
}
final List<LocatedBlock> locatedblocks = createLocatedBlockList(
blocks, offset, length, Integer.MAX_VALUE);
final BlockInfo last = blocks[blocks.length - 1];
final long lastPos = last.isComplete()?
fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
: fileSizeExcludeBlocksUnderConstruction;
final LocatedBlock lastlb = createLocatedBlock(last, lastPos);
if (isBlockTokenEnabled && needBlockToken) {
for(LocatedBlock lb : locatedblocks) {
setBlockToken(lb, AccessMode.READ);
}
setBlockToken(lastlb, AccessMode.READ);
}
return new LocatedBlocks(
fileSizeExcludeBlocksUnderConstruction, isFileUnderConstruction,
locatedblocks, lastlb, last.isComplete());
}
}
/** @return current access keys. */
public ExportedBlockKeys getBlockKeys() {
return isBlockTokenEnabled? blockTokenSecretManager.exportKeys()
: ExportedBlockKeys.DUMMY_KEYS;
}
/** Generate a block token for the located block. */
public void setBlockToken(final LocatedBlock b,
final BlockTokenSecretManager.AccessMode mode) throws IOException {
if (isBlockTokenEnabled) {
b.setBlockToken(blockTokenSecretManager.generateToken(b.getBlock(),
EnumSet.of(mode)));
}
}
void addKeyUpdateCommand(final List<DatanodeCommand> cmds,
final DatanodeDescriptor nodeinfo) {
// check access key update
if (isBlockTokenEnabled && nodeinfo.needKeyUpdate) {
cmds.add(new KeyUpdateCommand(blockTokenSecretManager.exportKeys()));
nodeinfo.needKeyUpdate = false;
}
}
/**
* Clamp the specified replication between the minimum and the maximum
* replication levels.
*/
public short adjustReplication(short replication) {
return replication < minReplication? minReplication
: replication > maxReplication? maxReplication: replication;
}
/** /**
* Check whether the replication parameter is within the range * Check whether the replication parameter is within the range
* determined by system configuration. * determined by system configuration.
@ -639,7 +687,7 @@ public class BlockManager {
final long size) throws UnregisteredNodeException { final long size) throws UnregisteredNodeException {
final DatanodeDescriptor node = getDatanodeManager().getDatanode(datanode); final DatanodeDescriptor node = getDatanodeManager().getDatanode(datanode);
if (node == null) { if (node == null) {
NameNode.stateChangeLog.warn("BLOCK* NameSystem.getBlocks: " NameNode.stateChangeLog.warn("BLOCK* getBlocks: "
+ "Asking for blocks from an unrecorded node " + datanode.getName()); + "Asking for blocks from an unrecorded node " + datanode.getName());
throw new HadoopIllegalArgumentException( throw new HadoopIllegalArgumentException(
"Datanode " + datanode.getName() + " not found."); "Datanode " + datanode.getName() + " not found.");
@ -711,7 +759,7 @@ public class BlockManager {
* @param dn datanode * @param dn datanode
* @param log true to create an entry in the log * @param log true to create an entry in the log
*/ */
void addToInvalidates(Block b, DatanodeInfo dn, boolean log) { private void addToInvalidates(Block b, DatanodeInfo dn, boolean log) {
Collection<Block> invalidateSet = recentInvalidateSets Collection<Block> invalidateSet = recentInvalidateSets
.get(dn.getStorageID()); .get(dn.getStorageID());
if (invalidateSet == null) { if (invalidateSet == null) {
@ -721,7 +769,7 @@ public class BlockManager {
if (invalidateSet.add(b)) { if (invalidateSet.add(b)) {
pendingDeletionBlocksCount++; pendingDeletionBlocksCount++;
if (log) { if (log) {
NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: " NameNode.stateChangeLog.info("BLOCK* addToInvalidates: "
+ b + " to " + dn.getName()); + b + " to " + dn.getName());
} }
} }
@ -734,7 +782,7 @@ public class BlockManager {
* @param b block * @param b block
* @param dn datanode * @param dn datanode
*/ */
public void addToInvalidates(Block b, DatanodeInfo dn) { void addToInvalidates(Block b, DatanodeInfo dn) {
addToInvalidates(b, dn, true); addToInvalidates(b, dn, true);
} }
@ -751,7 +799,7 @@ public class BlockManager {
datanodes.append(node.getName()).append(" "); datanodes.append(node.getName()).append(" ");
} }
if (datanodes.length() != 0) { if (datanodes.length() != 0) {
NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: " NameNode.stateChangeLog.info("BLOCK* addToInvalidates: "
+ b + " to " + datanodes.toString()); + b + " to " + datanodes.toString());
} }
} }
@ -775,20 +823,29 @@ public class BlockManager {
} }
} }
public void findAndMarkBlockAsCorrupt(Block blk, /**
DatanodeInfo dn) throws IOException { * Mark the block belonging to datanode as corrupt
BlockInfo storedBlock = getStoredBlock(blk); * @param blk Block to be marked as corrupt
if (storedBlock == null) { * @param dn Datanode which holds the corrupt replica
// Check if the replica is in the blockMap, if not */
// ignore the request for now. This could happen when BlockScanner public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
// thread of Datanode reports bad block before Block reports are sent final DatanodeInfo dn) throws IOException {
// by the Datanode on startup namesystem.writeLock();
NameNode.stateChangeLog.info("BLOCK* NameSystem.markBlockAsCorrupt: " + try {
"block " + blk + " could not be marked as " + final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
"corrupt as it does not exist in blocksMap"); if (storedBlock == null) {
return; // Check if the replica is in the blockMap, if not
// ignore the request for now. This could happen when BlockScanner
// thread of Datanode reports bad block before Block reports are sent
// by the Datanode on startup
NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
+ blk + " not found.");
return;
}
markBlockAsCorrupt(storedBlock, dn);
} finally {
namesystem.writeUnlock();
} }
markBlockAsCorrupt(storedBlock, dn);
} }
private void markBlockAsCorrupt(BlockInfo storedBlock, private void markBlockAsCorrupt(BlockInfo storedBlock,
@ -804,7 +861,7 @@ public class BlockManager {
INodeFile inode = storedBlock.getINode(); INodeFile inode = storedBlock.getINode();
if (inode == null) { if (inode == null) {
NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: " + NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " +
"block " + storedBlock + "block " + storedBlock +
" could not be marked as corrupt as it" + " could not be marked as corrupt as it" +
" does not belong to any file"); " does not belong to any file");
@ -831,13 +888,12 @@ public class BlockManager {
*/ */
private void invalidateBlock(Block blk, DatanodeInfo dn) private void invalidateBlock(Block blk, DatanodeInfo dn)
throws IOException { throws IOException {
NameNode.stateChangeLog.info("DIR* NameSystem.invalidateBlock: " NameNode.stateChangeLog.info("BLOCK* invalidateBlock: "
+ blk + " on " + dn.getName()); + blk + " on " + dn.getName());
DatanodeDescriptor node = getDatanodeManager().getDatanode(dn); DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
if (node == null) { if (node == null) {
throw new IOException("Cannot invalidate block " + blk + throw new IOException("Cannot invalidate block " + blk
" because datanode " + dn.getName() + + " because datanode " + dn.getName() + " does not exist.");
" does not exist.");
} }
// Check how many copies we have of the block. If we have at least one // Check how many copies we have of the block. If we have at least one
@ -847,14 +903,12 @@ public class BlockManager {
addToInvalidates(blk, dn); addToInvalidates(blk, dn);
removeStoredBlock(blk, node); removeStoredBlock(blk, node);
if(NameNode.stateChangeLog.isDebugEnabled()) { if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("BLOCK* NameSystem.invalidateBlocks: " NameNode.stateChangeLog.debug("BLOCK* invalidateBlocks: "
+ blk + " on " + blk + " on " + dn.getName() + " listed for deletion.");
+ dn.getName() + " listed for deletion.");
} }
} else { } else {
NameNode.stateChangeLog.info("BLOCK* NameSystem.invalidateBlocks: " NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: " + blk + " on "
+ blk + " on " + dn.getName() + dn.getName() + " is the only copy and was not deleted.");
+ " is the only copy and was not deleted.");
} }
} }
@ -1286,20 +1340,51 @@ public class BlockManager {
} }
/** /**
* The given node is reporting all its blocks. Use this info to * The given datanode is reporting all its blocks.
* update the (datanode-->blocklist) and (block-->nodelist) tables. * Update the (machine-->blocklist) and (block-->machinelist) maps.
*/ */
public void processReport(DatanodeDescriptor node, BlockListAsLongs report) public void processReport(final DatanodeID nodeID, final String poolId,
throws IOException { final BlockListAsLongs newReport) throws IOException {
namesystem.writeLock();
boolean isFirstBlockReport = (node.numBlocks() == 0); final long startTime = Util.now(); //after acquiring write lock
if (isFirstBlockReport) { final long endTime;
// Initial block reports can be processed a lot more efficiently than try {
// ordinary block reports. This shortens NN restart times. final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
processFirstBlockReport(node, report); if (node == null || !node.isAlive) {
return; throw new IOException("ProcessReport from dead or unregistered node: "
} + nodeID.getName());
}
// To minimize startup time, we discard any second (or later) block reports
// that we receive while still in startup phase.
if (namesystem.isInStartupSafeMode() && node.numBlocks() > 0) {
NameNode.stateChangeLog.info("BLOCK* processReport: "
+ "discarded non-initial block report from " + nodeID.getName()
+ " because namenode still in startup phase");
return;
}
if (node.numBlocks() == 0) {
// The first block report can be processed a lot more efficiently than
// ordinary block reports. This shortens restart times.
processFirstBlockReport(node, newReport);
} else {
processReport(node, newReport);
}
} finally {
endTime = Util.now();
namesystem.writeUnlock();
}
// Log the block report processing stats from Namenode perspective
NameNode.getNameNodeMetrics().addBlockReport((int) (endTime - startTime));
NameNode.stateChangeLog.info("BLOCK* processReport: from "
+ nodeID.getName() + ", blocks: " + newReport.getNumberOfBlocks()
+ ", processing time: " + (endTime - startTime) + " msecs");
}
private void processReport(final DatanodeDescriptor node,
final BlockListAsLongs report) throws IOException {
// Normal case: // Normal case:
// Modify the (block-->datanode) map, according to the difference // Modify the (block-->datanode) map, according to the difference
// between the old and new block report. // between the old and new block report.
@ -1322,7 +1407,7 @@ public class BlockManager {
addStoredBlock(b, node, null, true); addStoredBlock(b, node, null, true);
} }
for (Block b : toInvalidate) { for (Block b : toInvalidate) {
NameNode.stateChangeLog.info("BLOCK* NameSystem.processReport: block " NameNode.stateChangeLog.info("BLOCK* processReport: block "
+ b + " on " + node.getName() + " size " + b.getNumBytes() + b + " on " + node.getName() + " size " + b.getNumBytes()
+ " does not belong to any file."); + " does not belong to any file.");
addToInvalidates(b, node); addToInvalidates(b, node);
@ -1343,8 +1428,8 @@ public class BlockManager {
* @param report - the initial block report, to be processed * @param report - the initial block report, to be processed
* @throws IOException * @throws IOException
*/ */
void processFirstBlockReport(DatanodeDescriptor node, BlockListAsLongs report) private void processFirstBlockReport(final DatanodeDescriptor node,
throws IOException { final BlockListAsLongs report) throws IOException {
if (report == null) return; if (report == null) return;
assert (namesystem.hasWriteLock()); assert (namesystem.hasWriteLock());
assert (node.numBlocks() == 0); assert (node.numBlocks() == 0);
@ -1441,12 +1526,12 @@ public class BlockManager {
* @param toUC replicas of blocks currently under construction * @param toUC replicas of blocks currently under construction
* @return * @return
*/ */
BlockInfo processReportedBlock(DatanodeDescriptor dn, private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
Block block, ReplicaState reportedState, final Block block, final ReplicaState reportedState,
Collection<BlockInfo> toAdd, final Collection<BlockInfo> toAdd,
Collection<Block> toInvalidate, final Collection<Block> toInvalidate,
Collection<BlockInfo> toCorrupt, final Collection<BlockInfo> toCorrupt,
Collection<StatefulBlockInfo> toUC) { final Collection<StatefulBlockInfo> toUC) {
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
LOG.debug("Reported block " + block LOG.debug("Reported block " + block
@ -1616,11 +1701,9 @@ public class BlockManager {
} }
if (storedBlock == null || storedBlock.getINode() == null) { if (storedBlock == null || storedBlock.getINode() == null) {
// If this block does not belong to anyfile, then we are done. // If this block does not belong to anyfile, then we are done.
NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: " NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
+ "addStoredBlock request received for " + node.getName() + " size " + block.getNumBytes()
+ block + " on " + node.getName() + " but it does not belong to any file.");
+ " size " + block.getNumBytes()
+ " But it does not belong to any file.");
// we could add this block to invalidate set of this datanode. // we could add this block to invalidate set of this datanode.
// it will happen in next block report otherwise. // it will happen in next block report otherwise.
return block; return block;
@ -1636,13 +1719,13 @@ public class BlockManager {
if (added) { if (added) {
curReplicaDelta = 1; curReplicaDelta = 1;
if (logEveryBlock) { if (logEveryBlock) {
NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: " NameNode.stateChangeLog.info("BLOCK* addStoredBlock: "
+ "blockMap updated: " + node.getName() + " is added to " + + "blockMap updated: " + node.getName() + " is added to " +
storedBlock + " size " + storedBlock.getNumBytes()); storedBlock + " size " + storedBlock.getNumBytes());
} }
} else { } else {
curReplicaDelta = 0; curReplicaDelta = 0;
NameNode.stateChangeLog.warn("BLOCK* NameSystem.addStoredBlock: " NameNode.stateChangeLog.warn("BLOCK* addStoredBlock: "
+ "Redundant addStoredBlock request received for " + storedBlock + "Redundant addStoredBlock request received for " + storedBlock
+ " on " + node.getName() + " size " + storedBlock.getNumBytes()); + " on " + node.getName() + " size " + storedBlock.getNumBytes());
} }
@ -1778,13 +1861,39 @@ public class BlockManager {
LOG.info("Number of over-replicated blocks = " + nrOverReplicated); LOG.info("Number of over-replicated blocks = " + nrOverReplicated);
} }
/** Set replication for the blocks. */
public void setReplication(final short oldRepl, final short newRepl,
final String src, final Block... blocks) throws IOException {
if (newRepl == oldRepl) {
return;
}
// update needReplication priority queues
for(Block b : blocks) {
updateNeededReplications(b, 0, newRepl-oldRepl);
}
if (oldRepl > newRepl) {
// old replication > the new one; need to remove copies
LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
+ " for " + src);
for(Block b : blocks) {
processOverReplicatedBlock(b, newRepl, null, null);
}
} else { // replication factor is increased
LOG.info("Increasing replication from " + oldRepl + " to " + newRepl
+ " for " + src);
}
}
/** /**
* Find how many of the containing nodes are "extra", if any. * Find how many of the containing nodes are "extra", if any.
* If there are any extras, call chooseExcessReplicates() to * If there are any extras, call chooseExcessReplicates() to
* mark them in the excessReplicateMap. * mark them in the excessReplicateMap.
*/ */
public void processOverReplicatedBlock(Block block, short replication, private void processOverReplicatedBlock(final Block block,
DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) { final short replication, final DatanodeDescriptor addedNode,
DatanodeDescriptor delNodeHint) {
assert namesystem.hasWriteLock(); assert namesystem.hasWriteLock();
if (addedNode == delNodeHint) { if (addedNode == delNodeHint) {
delNodeHint = null; delNodeHint = null;
@ -1806,12 +1915,112 @@ public class BlockManager {
} }
} }
} }
namesystem.chooseExcessReplicates(nonExcess, block, replication, chooseExcessReplicates(nonExcess, block, replication,
addedNode, delNodeHint, blockplacement); addedNode, delNodeHint, blockplacement);
} }
public void addToExcessReplicate(DatanodeInfo dn, Block block) { /**
* We want "replication" replicates for the block, but we now have too many.
* In this method, copy enough nodes from 'srcNodes' into 'dstNodes' such that:
*
* srcNodes.size() - dstNodes.size() == replication
*
* We pick node that make sure that replicas are spread across racks and
* also try hard to pick one with least free space.
* The algorithm is first to pick a node with least free space from nodes
* that are on a rack holding more than one replicas of the block.
* So removing such a replica won't remove a rack.
* If no such a node is available,
* then pick a node with least free space
*/
private void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess,
Block b, short replication,
DatanodeDescriptor addedNode,
DatanodeDescriptor delNodeHint,
BlockPlacementPolicy replicator) {
assert namesystem.hasWriteLock();
// first form a rack to datanodes map and
INodeFile inode = getINode(b);
final Map<String, List<DatanodeDescriptor>> rackMap
= new HashMap<String, List<DatanodeDescriptor>>();
for(final Iterator<DatanodeDescriptor> iter = nonExcess.iterator();
iter.hasNext(); ) {
final DatanodeDescriptor node = iter.next();
final String rackName = node.getNetworkLocation();
List<DatanodeDescriptor> datanodeList = rackMap.get(rackName);
if (datanodeList == null) {
datanodeList = new ArrayList<DatanodeDescriptor>();
rackMap.put(rackName, datanodeList);
}
datanodeList.add(node);
}
// split nodes into two sets
// priSet contains nodes on rack with more than one replica
// remains contains the remaining nodes
final List<DatanodeDescriptor> priSet = new ArrayList<DatanodeDescriptor>();
final List<DatanodeDescriptor> remains = new ArrayList<DatanodeDescriptor>();
for(List<DatanodeDescriptor> datanodeList : rackMap.values()) {
if (datanodeList.size() == 1 ) {
remains.add(datanodeList.get(0));
} else {
priSet.addAll(datanodeList);
}
}
// pick one node to delete that favors the delete hint
// otherwise pick one with least space from priSet if it is not empty
// otherwise one node with least space from remains
boolean firstOne = true;
while (nonExcess.size() - replication > 0) {
// check if we can delete delNodeHint
final DatanodeInfo cur;
if (firstOne && delNodeHint !=null && nonExcess.contains(delNodeHint)
&& (priSet.contains(delNodeHint)
|| (addedNode != null && !priSet.contains(addedNode))) ) {
cur = delNodeHint;
} else { // regular excessive replica removal
cur = replicator.chooseReplicaToDelete(inode, b, replication,
priSet, remains);
}
firstOne = false;
// adjust rackmap, priSet, and remains
String rack = cur.getNetworkLocation();
final List<DatanodeDescriptor> datanodes = rackMap.get(rack);
datanodes.remove(cur);
if (datanodes.isEmpty()) {
rackMap.remove(rack);
}
if (priSet.remove(cur)) {
if (datanodes.size() == 1) {
priSet.remove(datanodes.get(0));
remains.add(datanodes.get(0));
}
} else {
remains.remove(cur);
}
nonExcess.remove(cur);
addToExcessReplicate(cur, b);
//
// The 'excessblocks' tracks blocks until we get confirmation
// that the datanode has deleted them; the only way we remove them
// is when we get a "removeBlock" message.
//
// The 'invalidate' list is used to inform the datanode the block
// should be deleted. Items are removed from the invalidate list
// upon giving instructions to the namenode.
//
addToInvalidates(b, cur);
NameNode.stateChangeLog.info("BLOCK* chooseExcessReplicates: "
+"("+cur.getName()+", "+b+") is added to recentInvalidateSets");
}
}
private void addToExcessReplicate(DatanodeInfo dn, Block block) {
assert namesystem.hasWriteLock(); assert namesystem.hasWriteLock();
Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID()); Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
if (excessBlocks == null) { if (excessBlocks == null) {
@ -1821,7 +2030,7 @@ public class BlockManager {
if (excessBlocks.add(block)) { if (excessBlocks.add(block)) {
excessBlocksCount++; excessBlocksCount++;
if(NameNode.stateChangeLog.isDebugEnabled()) { if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("BLOCK* NameSystem.chooseExcessReplicates:" NameNode.stateChangeLog.debug("BLOCK* addToExcessReplicate:"
+ " (" + dn.getName() + ", " + block + " (" + dn.getName() + ", " + block
+ ") is added to excessReplicateMap"); + ") is added to excessReplicateMap");
} }
@ -1834,14 +2043,14 @@ public class BlockManager {
*/ */
private void removeStoredBlock(Block block, DatanodeDescriptor node) { private void removeStoredBlock(Block block, DatanodeDescriptor node) {
if(NameNode.stateChangeLog.isDebugEnabled()) { if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: " NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
+ block + " from " + node.getName()); + block + " from " + node.getName());
} }
assert (namesystem.hasWriteLock()); assert (namesystem.hasWriteLock());
{ {
if (!blocksMap.removeNode(block, node)) { if (!blocksMap.removeNode(block, node)) {
if(NameNode.stateChangeLog.isDebugEnabled()) { if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: " NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
+ block + " has already been removed from node " + node); + block + " has already been removed from node " + node);
} }
return; return;
@ -1869,8 +2078,7 @@ public class BlockManager {
if (excessBlocks.remove(block)) { if (excessBlocks.remove(block)) {
excessBlocksCount--; excessBlocksCount--;
if(NameNode.stateChangeLog.isDebugEnabled()) { if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug( NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
"BLOCK* NameSystem.removeStoredBlock: "
+ block + " is removed from excessBlocks"); + block + " is removed from excessBlocks");
} }
if (excessBlocks.size() == 0) { if (excessBlocks.size() == 0) {
@ -1902,7 +2110,7 @@ public class BlockManager {
/** /**
* The given node is reporting that it received a certain block. * The given node is reporting that it received a certain block.
*/ */
public void addBlock(DatanodeDescriptor node, Block block, String delHint) private void addBlock(DatanodeDescriptor node, Block block, String delHint)
throws IOException { throws IOException {
// decrement number of blocks scheduled to this datanode. // decrement number of blocks scheduled to this datanode.
node.decBlocksScheduled(); node.decBlocksScheduled();
@ -1912,9 +2120,8 @@ public class BlockManager {
if (delHint != null && delHint.length() != 0) { if (delHint != null && delHint.length() != 0) {
delHintNode = datanodeManager.getDatanode(delHint); delHintNode = datanodeManager.getDatanode(delHint);
if (delHintNode == null) { if (delHintNode == null) {
NameNode.stateChangeLog.warn("BLOCK* NameSystem.blockReceived: " NameNode.stateChangeLog.warn("BLOCK* blockReceived: " + block
+ block + " is expected to be removed from an unrecorded node " + " is expected to be removed from an unrecorded node " + delHint);
+ delHint);
} }
} }
@ -1942,7 +2149,7 @@ public class BlockManager {
addStoredBlock(b, node, delHintNode, true); addStoredBlock(b, node, delHintNode, true);
} }
for (Block b : toInvalidate) { for (Block b : toInvalidate) {
NameNode.stateChangeLog.info("BLOCK* NameSystem.addBlock: block " NameNode.stateChangeLog.info("BLOCK* addBlock: block "
+ b + " on " + node.getName() + " size " + b.getNumBytes() + b + " on " + node.getName() + " size " + b.getNumBytes()
+ " does not belong to any file."); + " does not belong to any file.");
addToInvalidates(b, node); addToInvalidates(b, node);
@ -1952,6 +2159,30 @@ public class BlockManager {
} }
} }
/** The given node is reporting that it received a certain block. */
public void blockReceived(final DatanodeID nodeID, final String poolId,
final Block block, final String delHint) throws IOException {
namesystem.writeLock();
try {
final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
if (node == null || !node.isAlive) {
final String s = block + " is received from dead or unregistered node "
+ nodeID.getName();
NameNode.stateChangeLog.warn("BLOCK* blockReceived: " + s);
throw new IOException(s);
}
if (NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("BLOCK* blockReceived: " + block
+ " is received from " + nodeID.getName());
}
addBlock(node, block, delHint);
} finally {
namesystem.writeUnlock();
}
}
/** /**
* Return the number of nodes that are live and decommissioned. * Return the number of nodes that are live and decommissioned.
*/ */
@ -2142,9 +2373,9 @@ public class BlockManager {
return b; return b;
} }
/* updates a block in under replication queue */ /** updates a block in under replication queue */
public void updateNeededReplications(Block block, int curReplicasDelta, private void updateNeededReplications(final Block block,
int expectedReplicasDelta) { final int curReplicasDelta, int expectedReplicasDelta) {
namesystem.writeLock(); namesystem.writeLock();
try { try {
NumberReplicas repl = countNodes(block); NumberReplicas repl = countNodes(block);
@ -2303,8 +2534,9 @@ public class BlockManager {
return blocksMap.getINode(b); return blocksMap.getINode(b);
} }
public void removeFromCorruptReplicasMap(Block block) { /** @return an iterator of the datanodes. */
corruptReplicas.removeFromCorruptReplicasMap(block); public Iterator<DatanodeDescriptor> datanodeIterator(final Block block) {
return blocksMap.nodeIterator(block);
} }
public int numCorruptReplicas(Block block) { public int numCorruptReplicas(Block block) {
@ -2313,6 +2545,8 @@ public class BlockManager {
public void removeBlockFromMap(Block block) { public void removeBlockFromMap(Block block) {
blocksMap.removeBlock(block); blocksMap.removeBlock(block);
// If block is removed from blocksMap remove it from corruptReplicasMap
corruptReplicas.removeFromCorruptReplicasMap(block);
} }
public int getCapacity() { public int getCapacity() {

View File

@ -29,7 +29,7 @@ import org.apache.hadoop.hdfs.util.LightWeightGSet;
* block's metadata currently includes INode it belongs to and * block's metadata currently includes INode it belongs to and
* the datanodes that store the block. * the datanodes that store the block.
*/ */
public class BlocksMap { class BlocksMap {
private static class NodeIterator implements Iterator<DatanodeDescriptor> { private static class NodeIterator implements Iterator<DatanodeDescriptor> {
private BlockInfo blockInfo; private BlockInfo blockInfo;
private int nextIdx = 0; private int nextIdx = 0;
@ -101,7 +101,7 @@ public class BlocksMap {
/** /**
* Add block b belonging to the specified file inode to the map. * Add block b belonging to the specified file inode to the map.
*/ */
public BlockInfo addINode(BlockInfo b, INodeFile iNode) { BlockInfo addINode(BlockInfo b, INodeFile iNode) {
BlockInfo info = blocks.get(b); BlockInfo info = blocks.get(b);
if (info != b) { if (info != b) {
info = b; info = b;
@ -137,7 +137,7 @@ public class BlocksMap {
* Searches for the block in the BlocksMap and * Searches for the block in the BlocksMap and
* returns Iterator that iterates through the nodes the block belongs to. * returns Iterator that iterates through the nodes the block belongs to.
*/ */
public Iterator<DatanodeDescriptor> nodeIterator(Block b) { Iterator<DatanodeDescriptor> nodeIterator(Block b) {
return nodeIterator(blocks.get(b)); return nodeIterator(blocks.get(b));
} }
@ -182,27 +182,6 @@ public class BlocksMap {
Iterable<BlockInfo> getBlocks() { Iterable<BlockInfo> getBlocks() {
return blocks; return blocks;
} }
/**
* Check if the block exists in map
*/
public boolean contains(Block block) {
return blocks.contains(block);
}
/**
* Check if the replica at the given datanode exists in map
*/
boolean contains(Block block, DatanodeDescriptor datanode) {
BlockInfo info = blocks.get(block);
if (info == null)
return false;
if (-1 == info.findDatanode(datanode))
return false;
return true;
}
/** Get the capacity of the HashMap that stores blocks */ /** Get the capacity of the HashMap that stores blocks */
int getCapacity() { int getCapacity() {

View File

@ -34,6 +34,7 @@ import java.util.TreeMap;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -405,7 +406,7 @@ public class DatanodeManager {
* @param nodeList * @param nodeList
* , array list of live or dead nodes. * , array list of live or dead nodes.
*/ */
public void removeDecomNodeFromList(final List<DatanodeDescriptor> nodeList) { private void removeDecomNodeFromList(final List<DatanodeDescriptor> nodeList) {
// If the include list is empty, any nodes are welcomed and it does not // If the include list is empty, any nodes are welcomed and it does not
// make sense to exclude any nodes from the cluster. Therefore, no remove. // make sense to exclude any nodes from the cluster. Therefore, no remove.
if (hostsReader.getHosts().isEmpty()) { if (hostsReader.getHosts().isEmpty()) {
@ -563,7 +564,7 @@ public class DatanodeManager {
nodeReg.getInfoPort(), nodeReg.getInfoPort(),
nodeReg.getIpcPort()); nodeReg.getIpcPort());
nodeReg.updateRegInfo(dnReg); nodeReg.updateRegInfo(dnReg);
nodeReg.exportedKeys = namesystem.getBlockManager().getBlockKeys(); nodeReg.exportedKeys = blockManager.getBlockKeys();
NameNode.stateChangeLog.info("BLOCK* NameSystem.registerDatanode: " NameNode.stateChangeLog.info("BLOCK* NameSystem.registerDatanode: "
+ "node registration from " + nodeReg.getName() + "node registration from " + nodeReg.getName()
@ -710,16 +711,59 @@ public class DatanodeManager {
return numDead; return numDead;
} }
/** @return list of datanodes where decommissioning is in progress. */
public List<DatanodeDescriptor> getDecommissioningNodes() {
namesystem.readLock();
try {
final List<DatanodeDescriptor> decommissioningNodes
= new ArrayList<DatanodeDescriptor>();
final List<DatanodeDescriptor> results = getDatanodeListForReport(
DatanodeReportType.LIVE);
for(DatanodeDescriptor node : results) {
if (node.isDecommissionInProgress()) {
decommissioningNodes.add(node);
}
}
return decommissioningNodes;
} finally {
namesystem.readUnlock();
}
}
/** Fetch live and dead datanodes. */ /** Fetch live and dead datanodes. */
public void fetchDatanodess(final List<DatanodeDescriptor> live, public void fetchDatanodes(final List<DatanodeDescriptor> live,
final List<DatanodeDescriptor> dead) { final List<DatanodeDescriptor> dead, final boolean removeDecommissionNode) {
final List<DatanodeDescriptor> results = if (live == null && dead == null) {
getDatanodeListForReport(DatanodeReportType.ALL); throw new HadoopIllegalArgumentException("Both live and dead lists are null");
for(DatanodeDescriptor node : results) { }
if (isDatanodeDead(node))
dead.add(node); namesystem.readLock();
else try {
live.add(node); final List<DatanodeDescriptor> results =
getDatanodeListForReport(DatanodeReportType.ALL);
for(DatanodeDescriptor node : results) {
if (isDatanodeDead(node)) {
if (dead != null) {
dead.add(node);
}
} else {
if (live != null) {
live.add(node);
}
}
}
} finally {
namesystem.readUnlock();
}
if (removeDecommissionNode) {
if (live != null) {
removeDecomNodeFromList(live);
}
if (dead != null) {
removeDecomNodeFromList(dead);
}
} }
} }
@ -847,7 +891,7 @@ public class DatanodeManager {
blockPoolId, blks)); blockPoolId, blks));
} }
namesystem.addKeyUpdateCommand(cmds, nodeinfo); blockManager.addKeyUpdateCommand(cmds, nodeinfo);
// check for balancer bandwidth update // check for balancer bandwidth update
if (nodeinfo.getBalancerBandwidth() > 0) { if (nodeinfo.getBalancerBandwidth() > 0) {

View File

@ -26,11 +26,11 @@ import java.net.InetSocketAddress;
import java.net.Socket; import java.net.Socket;
import java.net.URL; import java.net.URL;
import java.net.URLEncoder; import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.TreeSet; import java.util.TreeSet;
import javax.servlet.ServletContext; import javax.servlet.ServletContext;
@ -249,7 +249,7 @@ public class JspHelper {
out.print("</tbody></table>"); out.print("</tbody></table>");
} }
public static void sortNodeList(ArrayList<DatanodeDescriptor> nodes, public static void sortNodeList(final List<DatanodeDescriptor> nodes,
String field, String order) { String field, String order) {
class NodeComapare implements Comparator<DatanodeDescriptor> { class NodeComapare implements Comparator<DatanodeDescriptor> {

View File

@ -444,8 +444,6 @@ public class FSDirectory implements Closeable {
// modify file-> block and blocksMap // modify file-> block and blocksMap
fileNode.removeLastBlock(block); fileNode.removeLastBlock(block);
getBlockManager().removeBlockFromMap(block); getBlockManager().removeBlockFromMap(block);
// If block is removed from blocksMap remove it from corruptReplicasMap
getBlockManager().removeFromCorruptReplicasMap(block);
// write modified block locations to log // write modified block locations to log
fsImage.getEditLog().logOpenFile(path, fileNode); fsImage.getEditLog().logOpenFile(path, fileNode);
@ -809,7 +807,7 @@ public class FSDirectory implements Closeable {
* @return array of file blocks * @return array of file blocks
* @throws QuotaExceededException * @throws QuotaExceededException
*/ */
Block[] setReplication(String src, short replication, int[] oldReplication) Block[] setReplication(String src, short replication, short[] oldReplication)
throws QuotaExceededException, UnresolvedLinkException { throws QuotaExceededException, UnresolvedLinkException {
waitForReady(); waitForReady();
Block[] fileBlocks = null; Block[] fileBlocks = null;
@ -826,14 +824,10 @@ public class FSDirectory implements Closeable {
Block[] unprotectedSetReplication(String src, Block[] unprotectedSetReplication(String src,
short replication, short replication,
int[] oldReplication short[] oldReplication
) throws QuotaExceededException, ) throws QuotaExceededException,
UnresolvedLinkException { UnresolvedLinkException {
assert hasWriteLock(); assert hasWriteLock();
if (oldReplication == null) {
oldReplication = new int[1];
}
oldReplication[0] = -1;
INode[] inodes = rootDir.getExistingPathINodes(src, true); INode[] inodes = rootDir.getExistingPathINodes(src, true);
INode inode = inodes[inodes.length - 1]; INode inode = inodes[inodes.length - 1];
@ -845,14 +839,17 @@ public class FSDirectory implements Closeable {
return null; return null;
} }
INodeFile fileNode = (INodeFile)inode; INodeFile fileNode = (INodeFile)inode;
oldReplication[0] = fileNode.getReplication(); final short oldRepl = fileNode.getReplication();
// check disk quota // check disk quota
long dsDelta = (replication - oldReplication[0]) * long dsDelta = (replication - oldRepl) * (fileNode.diskspaceConsumed()/oldRepl);
(fileNode.diskspaceConsumed()/oldReplication[0]);
updateCount(inodes, inodes.length-1, 0, dsDelta, true); updateCount(inodes, inodes.length-1, 0, dsDelta, true);
fileNode.setReplication(replication); fileNode.setReplication(replication);
if (oldReplication != null) {
oldReplication[0] = oldRepl;
}
return fileNode.getBlocks(); return fileNode.getBlocks();
} }
@ -2075,8 +2072,9 @@ public class FSDirectory implements Closeable {
size = fileNode.computeFileSize(true); size = fileNode.computeFileSize(true);
replication = fileNode.getReplication(); replication = fileNode.getReplication();
blocksize = fileNode.getPreferredBlockSize(); blocksize = fileNode.getPreferredBlockSize();
loc = getFSNamesystem().getBlockLocationsInternal( loc = getFSNamesystem().getBlockManager().createLocatedBlocks(
fileNode, 0L, size, false); fileNode.getBlocks(), fileNode.computeFileSize(false),
fileNode.isUnderConstruction(), 0L, size, false);
if (loc==null) { if (loc==null) {
loc = new LocatedBlocks(); loc = new LocatedBlocks();
} }

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now; import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.DataInputStream;
import java.io.File; import java.io.File;
import java.io.FilterInputStream; import java.io.FilterInputStream;
import java.io.IOException; import java.io.IOException;
@ -144,8 +143,8 @@ public class FSEditLogLoader {
// versions > 0 support per file replication // versions > 0 support per file replication
// get name and replication // get name and replication
short replication final short replication = fsNamesys.getBlockManager(
= fsNamesys.adjustReplication(addCloseOp.replication); ).adjustReplication(addCloseOp.replication);
long blockSize = addCloseOp.blockSize; long blockSize = addCloseOp.blockSize;
BlockInfo blocks[] = new BlockInfo[addCloseOp.blocks.length]; BlockInfo blocks[] = new BlockInfo[addCloseOp.blocks.length];
@ -218,8 +217,8 @@ public class FSEditLogLoader {
} }
case OP_SET_REPLICATION: { case OP_SET_REPLICATION: {
SetReplicationOp setReplicationOp = (SetReplicationOp)op; SetReplicationOp setReplicationOp = (SetReplicationOp)op;
short replication short replication = fsNamesys.getBlockManager().adjustReplication(
= fsNamesys.adjustReplication(setReplicationOp.replication); setReplicationOp.replication);
fsDir.unprotectedSetReplication(setReplicationOp.path, fsDir.unprotectedSetReplication(setReplicationOp.path,
replication, null); replication, null);
break; break;

View File

@ -330,7 +330,7 @@ class FSImageFormat {
int imgVersion = getLayoutVersion(); int imgVersion = getLayoutVersion();
short replication = in.readShort(); short replication = in.readShort();
replication = namesystem.adjustReplication(replication); replication = namesystem.getBlockManager().adjustReplication(replication);
modificationTime = in.readLong(); modificationTime = in.readLong();
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imgVersion)) { if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imgVersion)) {
atime = in.readLong(); atime = in.readLong();

View File

@ -39,10 +39,8 @@ import java.util.Date;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -74,7 +72,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -88,12 +85,12 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException; import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure; import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
@ -110,11 +107,9 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand; import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand; import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
@ -394,7 +389,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
return this.fsLock.getReadHoldCount() > 0; return this.fsLock.getReadHoldCount() > 0;
} }
boolean hasReadOrWriteLock() { public boolean hasReadOrWriteLock() {
return hasReadLock() || hasWriteLock(); return hasReadLock() || hasWriteLock();
} }
@ -534,14 +529,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
long totalInodes = this.dir.totalInodes(); long totalInodes = this.dir.totalInodes();
long totalBlocks = this.getBlocksTotal(); long totalBlocks = this.getBlocksTotal();
out.println(totalInodes + " files and directories, " + totalBlocks
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); + " blocks = " + (totalInodes + totalBlocks) + " total");
ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
this.DFSNodesStatus(live, dead); final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
String str = totalInodes + " files and directories, " + totalBlocks blockManager.getDatanodeManager().fetchDatanodes(live, dead, false);
+ " blocks = " + (totalInodes + totalBlocks) + " total";
out.println(str);
out.println("Live Datanodes: "+live.size()); out.println("Live Datanodes: "+live.size());
out.println("Dead Datanodes: "+dead.size()); out.println("Dead Datanodes: "+dead.size());
blockManager.metaSave(out); blockManager.metaSave(out);
@ -750,7 +743,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
} }
dir.setTimes(src, inode, -1, now, false); dir.setTimes(src, inode, -1, now, false);
} }
return getBlockLocationsInternal(inode, offset, length, needBlockToken); return blockManager.createLocatedBlocks(inode.getBlocks(),
inode.computeFileSize(false), inode.isUnderConstruction(),
offset, length, needBlockToken);
} finally { } finally {
if (attempt == 0) { if (attempt == 0) {
readUnlock(); readUnlock();
@ -761,44 +756,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
} }
return null; // can never reach here return null; // can never reach here
} }
LocatedBlocks getBlockLocationsInternal(INodeFile inode,
long offset, long length, boolean needBlockToken)
throws IOException {
assert hasReadOrWriteLock();
final BlockInfo[] blocks = inode.getBlocks();
if (LOG.isDebugEnabled()) {
LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
}
if (blocks == null) {
return null;
}
if (blocks.length == 0) {
return new LocatedBlocks(0, inode.isUnderConstruction(),
Collections.<LocatedBlock>emptyList(), null, false);
} else {
final long n = inode.computeFileSize(false);
final List<LocatedBlock> locatedblocks = blockManager.getBlockLocations(
blocks, offset, length, Integer.MAX_VALUE);
final BlockInfo last = inode.getLastBlock();
if (LOG.isDebugEnabled()) {
LOG.debug("last = " + last);
}
LocatedBlock lastBlock = last.isComplete() ? blockManager
.getBlockLocation(last, n - last.getNumBytes()) : blockManager
.getBlockLocation(last, n);
if (blockManager.isBlockTokenEnabled() && needBlockToken) {
blockManager.setBlockTokens(locatedblocks);
blockManager.setBlockToken(lastBlock);
}
return new LocatedBlocks(n, inode.isUnderConstruction(), locatedblocks,
lastBlock, last.isComplete());
}
}
/** /**
* Moves all the blocks from srcs and appends them to trg * Moves all the blocks from srcs and appends them to trg
@ -960,7 +917,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
* The access time is precise upto an hour. The transaction, if needed, is * The access time is precise upto an hour. The transaction, if needed, is
* written to the edits log but is not flushed. * written to the edits log but is not flushed.
*/ */
public void setTimes(String src, long mtime, long atime) void setTimes(String src, long mtime, long atime)
throws IOException, UnresolvedLinkException { throws IOException, UnresolvedLinkException {
if (!isAccessTimeSupported() && atime != -1) { if (!isAccessTimeSupported() && atime != -1) {
throw new IOException("Access time for hdfs is not configured. " + throw new IOException("Access time for hdfs is not configured. " +
@ -1060,60 +1017,37 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
* @return true if successful; * @return true if successful;
* false if file does not exist or is a directory * false if file does not exist or is a directory
*/ */
public boolean setReplication(String src, short replication) boolean setReplication(final String src, final short replication
throws IOException, UnresolvedLinkException { ) throws IOException {
boolean status = false; blockManager.verifyReplication(src, replication, null);
final boolean isFile;
writeLock(); writeLock();
try { try {
if (isInSafeMode()) { if (isInSafeMode()) {
throw new SafeModeException("Cannot set replication for " + src, safeMode); throw new SafeModeException("Cannot set replication for " + src, safeMode);
} }
status = setReplicationInternal(src, replication); if (isPermissionEnabled) {
checkPathAccess(src, FsAction.WRITE);
}
final short[] oldReplication = new short[1];
final Block[] blocks = dir.setReplication(src, replication, oldReplication);
isFile = blocks != null;
if (isFile) {
blockManager.setReplication(oldReplication[0], replication, src, blocks);
}
} finally { } finally {
writeUnlock(); writeUnlock();
} }
getEditLog().logSync(); getEditLog().logSync();
if (status && auditLog.isInfoEnabled() && isExternalInvocation()) { if (isFile && auditLog.isInfoEnabled() && isExternalInvocation()) {
logAuditEvent(UserGroupInformation.getCurrentUser(), logAuditEvent(UserGroupInformation.getCurrentUser(),
Server.getRemoteIp(), Server.getRemoteIp(),
"setReplication", src, null, null); "setReplication", src, null, null);
} }
return status; return isFile;
}
private boolean setReplicationInternal(String src,
short replication) throws AccessControlException, QuotaExceededException,
SafeModeException, UnresolvedLinkException, IOException {
assert hasWriteLock();
blockManager.verifyReplication(src, replication, null);
if (isPermissionEnabled) {
checkPathAccess(src, FsAction.WRITE);
}
int[] oldReplication = new int[1];
Block[] fileBlocks;
fileBlocks = dir.setReplication(src, replication, oldReplication);
if (fileBlocks == null) // file not found or is a directory
return false;
int oldRepl = oldReplication[0];
if (oldRepl == replication) // the same replication
return true;
// update needReplication priority queues
for(int idx = 0; idx < fileBlocks.length; idx++)
blockManager.updateNeededReplications(fileBlocks[idx], 0, replication-oldRepl);
if (oldRepl > replication) {
// old replication > the new one; need to remove copies
LOG.info("Reducing replication for file " + src
+ ". New replication is " + replication);
for(int idx = 0; idx < fileBlocks.length; idx++)
blockManager.processOverReplicatedBlock(fileBlocks[idx], replication, null, null);
} else { // replication factor is increased
LOG.info("Increasing replication for file " + src
+ ". New replication is " + replication);
}
return true;
} }
long getPreferredBlockSize(String filename) long getPreferredBlockSize(String filename)
@ -1287,9 +1221,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
LocatedBlock lb = LocatedBlock lb =
blockManager.convertLastBlockToUnderConstruction(cons); blockManager.convertLastBlockToUnderConstruction(cons);
if (lb != null && blockManager.isBlockTokenEnabled()) { if (lb != null) {
lb.setBlockToken(blockManager.getBlockTokenSecretManager().generateToken(lb.getBlock(), blockManager.setBlockToken(lb, AccessMode.WRITE);
EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE)));
} }
return lb; return lb;
} else { } else {
@ -1456,7 +1389,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
try { try {
lb = startFileInternal(src, null, holder, clientMachine, lb = startFileInternal(src, null, holder, clientMachine,
EnumSet.of(CreateFlag.APPEND), EnumSet.of(CreateFlag.APPEND),
false, (short)blockManager.maxReplication, (long)0); false, blockManager.maxReplication, (long)0);
} finally { } finally {
writeUnlock(); writeUnlock();
} }
@ -1577,10 +1510,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
// Create next block // Create next block
LocatedBlock b = new LocatedBlock(getExtendedBlock(newBlock), targets, fileLength); LocatedBlock b = new LocatedBlock(getExtendedBlock(newBlock), targets, fileLength);
if (blockManager.isBlockTokenEnabled()) { blockManager.setBlockToken(b, BlockTokenSecretManager.AccessMode.WRITE);
b.setBlockToken(blockManager.getBlockTokenSecretManager().generateToken(b.getBlock(),
EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE)));
}
return b; return b;
} }
@ -1626,17 +1556,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true, ).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true,
excludes, preferredblocksize); excludes, preferredblocksize);
final LocatedBlock lb = new LocatedBlock(blk, targets); final LocatedBlock lb = new LocatedBlock(blk, targets);
if (blockManager.isBlockTokenEnabled()) { blockManager.setBlockToken(lb, AccessMode.COPY);
lb.setBlockToken(blockManager.getBlockTokenSecretManager().generateToken(lb.getBlock(),
EnumSet.of(BlockTokenSecretManager.AccessMode.COPY)));
}
return lb; return lb;
} }
/** /**
* The client would like to let go of the given block * The client would like to let go of the given block
*/ */
public boolean abandonBlock(ExtendedBlock b, String src, String holder) boolean abandonBlock(ExtendedBlock b, String src, String holder)
throws LeaseExpiredException, FileNotFoundException, throws LeaseExpiredException, FileNotFoundException,
UnresolvedLinkException, IOException { UnresolvedLinkException, IOException {
writeLock(); writeLock();
@ -1821,23 +1748,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
} }
} }
/**
* Mark the block belonging to datanode as corrupt
* @param blk Block to be marked as corrupt
* @param dn Datanode which holds the corrupt replica
*/
public void markBlockAsCorrupt(ExtendedBlock blk, DatanodeInfo dn)
throws IOException {
writeLock();
try {
blockManager.findAndMarkBlockAsCorrupt(blk.getLocalBlock(), dn);
} finally {
writeUnlock();
}
}
//////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////
// Here's how to handle block-copy failure during client write: // Here's how to handle block-copy failure during client write:
// -- As usual, the client's write should result in a streaming // -- As usual, the client's write should result in a streaming
@ -2621,16 +2531,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
} }
} }
public void addKeyUpdateCommand(final List<DatanodeCommand> cmds,
final DatanodeDescriptor nodeinfo) {
// check access key update
if (blockManager.isBlockTokenEnabled() && nodeinfo.needKeyUpdate) {
cmds.add(new KeyUpdateCommand(blockManager.getBlockTokenSecretManager().exportKeys()));
nodeinfo.needKeyUpdate = false;
}
}
/** /**
* Returns whether or not there were available resources at the last check of * Returns whether or not there were available resources at the last check of
* resources. * resources.
@ -2688,179 +2588,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
FSEditLog getEditLog() { FSEditLog getEditLog() {
return getFSImage().getEditLog(); return getFSImage().getEditLog();
} }
/**
* The given node is reporting all its blocks. Use this info to
* update the (machine-->blocklist) and (block-->machinelist) tables.
*/
void processReport(DatanodeID nodeID, String poolId,
BlockListAsLongs newReport) throws IOException {
long startTime, endTime;
writeLock();
startTime = now(); //after acquiring write lock
try {
final DatanodeDescriptor node = blockManager.getDatanodeManager(
).getDatanode(nodeID);
if (node == null || !node.isAlive) {
throw new IOException("ProcessReport from dead or unregistered node: "
+ nodeID.getName());
}
// To minimize startup time, we discard any second (or later) block reports
// that we receive while still in startup phase.
if (isInStartupSafeMode() && node.numBlocks() > 0) {
NameNode.stateChangeLog.info("BLOCK* NameSystem.processReport: "
+ "discarded non-initial block report from " + nodeID.getName()
+ " because namenode still in startup phase");
return;
}
blockManager.processReport(node, newReport);
} finally {
endTime = now();
writeUnlock();
}
// Log the block report processing stats from Namenode perspective
NameNode.getNameNodeMetrics().addBlockReport((int) (endTime - startTime));
NameNode.stateChangeLog.info("BLOCK* NameSystem.processReport: from "
+ nodeID.getName() + ", blocks: " + newReport.getNumberOfBlocks()
+ ", processing time: " + (endTime - startTime) + " msecs");
}
/**
* We want "replication" replicates for the block, but we now have too many.
* In this method, copy enough nodes from 'srcNodes' into 'dstNodes' such that:
*
* srcNodes.size() - dstNodes.size() == replication
*
* We pick node that make sure that replicas are spread across racks and
* also try hard to pick one with least free space.
* The algorithm is first to pick a node with least free space from nodes
* that are on a rack holding more than one replicas of the block.
* So removing such a replica won't remove a rack.
* If no such a node is available,
* then pick a node with least free space
*/
public void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess,
Block b, short replication,
DatanodeDescriptor addedNode,
DatanodeDescriptor delNodeHint,
BlockPlacementPolicy replicator) {
assert hasWriteLock();
// first form a rack to datanodes map and
INodeFile inode = blockManager.getINode(b);
HashMap<String, ArrayList<DatanodeDescriptor>> rackMap =
new HashMap<String, ArrayList<DatanodeDescriptor>>();
for (Iterator<DatanodeDescriptor> iter = nonExcess.iterator();
iter.hasNext();) {
DatanodeDescriptor node = iter.next();
String rackName = node.getNetworkLocation();
ArrayList<DatanodeDescriptor> datanodeList = rackMap.get(rackName);
if(datanodeList==null) {
datanodeList = new ArrayList<DatanodeDescriptor>();
}
datanodeList.add(node);
rackMap.put(rackName, datanodeList);
}
// split nodes into two sets
// priSet contains nodes on rack with more than one replica
// remains contains the remaining nodes
ArrayList<DatanodeDescriptor> priSet = new ArrayList<DatanodeDescriptor>();
ArrayList<DatanodeDescriptor> remains = new ArrayList<DatanodeDescriptor>();
for( Iterator<Entry<String, ArrayList<DatanodeDescriptor>>> iter =
rackMap.entrySet().iterator(); iter.hasNext(); ) {
Entry<String, ArrayList<DatanodeDescriptor>> rackEntry = iter.next();
ArrayList<DatanodeDescriptor> datanodeList = rackEntry.getValue();
if( datanodeList.size() == 1 ) {
remains.add(datanodeList.get(0));
} else {
priSet.addAll(datanodeList);
}
}
// pick one node to delete that favors the delete hint
// otherwise pick one with least space from priSet if it is not empty
// otherwise one node with least space from remains
boolean firstOne = true;
while (nonExcess.size() - replication > 0) {
DatanodeInfo cur = null;
// check if we can del delNodeHint
if (firstOne && delNodeHint !=null && nonExcess.contains(delNodeHint) &&
(priSet.contains(delNodeHint) || (addedNode != null && !priSet.contains(addedNode))) ) {
cur = delNodeHint;
} else { // regular excessive replica removal
cur = replicator.chooseReplicaToDelete(inode, b, replication, priSet, remains);
}
firstOne = false;
// adjust rackmap, priSet, and remains
String rack = cur.getNetworkLocation();
ArrayList<DatanodeDescriptor> datanodes = rackMap.get(rack);
datanodes.remove(cur);
if(datanodes.isEmpty()) {
rackMap.remove(rack);
}
if( priSet.remove(cur) ) {
if (datanodes.size() == 1) {
priSet.remove(datanodes.get(0));
remains.add(datanodes.get(0));
}
} else {
remains.remove(cur);
}
nonExcess.remove(cur);
blockManager.addToExcessReplicate(cur, b);
//
// The 'excessblocks' tracks blocks until we get confirmation
// that the datanode has deleted them; the only way we remove them
// is when we get a "removeBlock" message.
//
// The 'invalidate' list is used to inform the datanode the block
// should be deleted. Items are removed from the invalidate list
// upon giving instructions to the namenode.
//
blockManager.addToInvalidates(b, cur);
NameNode.stateChangeLog.info("BLOCK* NameSystem.chooseExcessReplicates: "
+"("+cur.getName()+", "+b+") is added to recentInvalidateSets");
}
}
/**
* The given node is reporting that it received a certain block.
*/
public void blockReceived(DatanodeID nodeID,
String poolId,
Block block,
String delHint
) throws IOException {
writeLock();
try {
final DatanodeDescriptor node = blockManager.getDatanodeManager(
).getDatanode(nodeID);
if (node == null || !node.isAlive) {
NameNode.stateChangeLog.warn("BLOCK* NameSystem.blockReceived: " + block
+ " is received from dead or unregistered node " + nodeID.getName());
throw new IOException(
"Got blockReceived message from unregistered or dead node " + block);
}
if (NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("BLOCK* NameSystem.blockReceived: "
+block+" is received from " + nodeID.getName());
}
blockManager.addBlock(node, block, delHint);
} finally {
writeUnlock();
}
}
private void checkBlock(ExtendedBlock block) throws IOException { private void checkBlock(ExtendedBlock block) throws IOException {
if (block != null && !this.blockPoolId.equals(block.getBlockPoolId())) { if (block != null && !this.blockPoolId.equals(block.getBlockPoolId())) {
@ -3009,43 +2737,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
} }
} }
/**
*/
public void DFSNodesStatus(ArrayList<DatanodeDescriptor> live,
ArrayList<DatanodeDescriptor> dead) {
readLock();
try {
getBlockManager().getDatanodeManager().fetchDatanodess(live, dead);
} finally {
readUnlock();
}
}
public Date getStartTime() { public Date getStartTime() {
return new Date(systemStart); return new Date(systemStart);
} }
short getMaxReplication() { return (short)blockManager.maxReplication; }
short getMinReplication() { return (short)blockManager.minReplication; }
short getDefaultReplication() { return (short)blockManager.defaultReplication; }
/**
* Clamp the specified replication between the minimum and maximum
* replication levels for this namesystem.
*/
short adjustReplication(short replication) {
short minReplication = getMinReplication();
if (replication < minReplication) {
replication = minReplication;
}
short maxReplication = getMaxReplication();
if (replication > maxReplication) {
replication = maxReplication;
}
return replication;
}
/** /**
* Rereads the config to get hosts and exclude list file names. * Rereads the config to get hosts and exclude list file names.
* Rereads the files to update the hosts and exclude lists. It * Rereads the files to update the hosts and exclude lists. It
@ -3740,10 +3435,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
writeUnlock(); writeUnlock();
} }
} }
public RemoteEditLogManifest getEditLogManifest(long sinceTxId) throws IOException {
return getEditLog().getEditLogManifest(sinceTxId);
}
NamenodeCommand startCheckpoint( NamenodeCommand startCheckpoint(
NamenodeRegistration bnReg, // backup node NamenodeRegistration bnReg, // backup node
@ -3968,7 +3659,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
/** /**
* shutdown FSNamesystem * shutdown FSNamesystem
*/ */
public void shutdown() { void shutdown() {
if (mbeanName != null) if (mbeanName != null)
MBeans.unregister(mbeanName); MBeans.unregister(mbeanName);
} }
@ -4069,10 +3760,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
// get a new generation stamp and an access token // get a new generation stamp and an access token
block.setGenerationStamp(nextGenerationStamp()); block.setGenerationStamp(nextGenerationStamp());
locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]); locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
if (blockManager.isBlockTokenEnabled()) { blockManager.setBlockToken(locatedBlock, AccessMode.WRITE);
locatedBlock.setBlockToken(blockManager.getBlockTokenSecretManager().generateToken(
block, EnumSet.of(BlockTokenSecretManager.AccessMode.WRITE)));
}
} finally { } finally {
writeUnlock(); writeUnlock();
} }
@ -4273,26 +3961,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
return blockManager.numCorruptReplicas(blk); return blockManager.numCorruptReplicas(blk);
} }
/**
* Return a range of corrupt replica block ids. Up to numExpectedBlocks
* blocks starting at the next block after startingBlockId are returned
* (fewer if numExpectedBlocks blocks are unavailable). If startingBlockId
* is null, up to numExpectedBlocks blocks are returned from the beginning.
* If startingBlockId cannot be found, null is returned.
*
* @param numExpectedBlocks Number of block ids to return.
* 0 <= numExpectedBlocks <= 100
* @param startingBlockId Block id from which to start. If null, start at
* beginning.
* @return Up to numExpectedBlocks blocks from startingBlockId if it exists
*
*/
long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
Long startingBlockId) {
return blockManager.getCorruptReplicaBlockIds(numExpectedBlocks,
startingBlockId);
}
static class CorruptFileBlockInfo { static class CorruptFileBlockInfo {
String path; String path;
Block block; Block block;
@ -4354,28 +4022,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
} }
} }
/**
* @return list of datanodes where decommissioning is in progress
*/
public ArrayList<DatanodeDescriptor> getDecommissioningNodes() {
readLock();
try {
ArrayList<DatanodeDescriptor> decommissioningNodes =
new ArrayList<DatanodeDescriptor>();
final List<DatanodeDescriptor> results = getBlockManager(
).getDatanodeManager().getDatanodeListForReport(DatanodeReportType.LIVE);
for (Iterator<DatanodeDescriptor> it = results.iterator(); it.hasNext();) {
DatanodeDescriptor node = it.next();
if (node.isDecommissionInProgress()) {
decommissioningNodes.add(node);
}
}
return decommissioningNodes;
} finally {
readUnlock();
}
}
/** /**
* Create delegation token secret manager * Create delegation token secret manager
*/ */
@ -4406,7 +4052,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
* @return Token<DelegationTokenIdentifier> * @return Token<DelegationTokenIdentifier>
* @throws IOException * @throws IOException
*/ */
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
throws IOException { throws IOException {
Token<DelegationTokenIdentifier> token; Token<DelegationTokenIdentifier> token;
writeLock(); writeLock();
@ -4686,13 +4332,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
public String getLiveNodes() { public String getLiveNodes() {
final Map<String, Map<String,Object>> info = final Map<String, Map<String,Object>> info =
new HashMap<String, Map<String,Object>>(); new HashMap<String, Map<String,Object>>();
final ArrayList<DatanodeDescriptor> liveNodeList = final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
new ArrayList<DatanodeDescriptor>(); blockManager.getDatanodeManager().fetchDatanodes(live, null, true);
final ArrayList<DatanodeDescriptor> deadNodeList = for (DatanodeDescriptor node : live) {
new ArrayList<DatanodeDescriptor>();
DFSNodesStatus(liveNodeList, deadNodeList);
removeDecomNodeFromList(liveNodeList);
for (DatanodeDescriptor node : liveNodeList) {
final Map<String, Object> innerinfo = new HashMap<String, Object>(); final Map<String, Object> innerinfo = new HashMap<String, Object>();
innerinfo.put("lastContact", getLastContact(node)); innerinfo.put("lastContact", getLastContact(node));
innerinfo.put("usedSpace", getDfsUsed(node)); innerinfo.put("usedSpace", getDfsUsed(node));
@ -4710,14 +4352,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
public String getDeadNodes() { public String getDeadNodes() {
final Map<String, Map<String, Object>> info = final Map<String, Map<String, Object>> info =
new HashMap<String, Map<String, Object>>(); new HashMap<String, Map<String, Object>>();
final ArrayList<DatanodeDescriptor> liveNodeList = final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
new ArrayList<DatanodeDescriptor>(); blockManager.getDatanodeManager().fetchDatanodes(null, dead, true);
final ArrayList<DatanodeDescriptor> deadNodeList = for (DatanodeDescriptor node : dead) {
new ArrayList<DatanodeDescriptor>();
// we need to call DFSNodeStatus to filter out the dead data nodes
DFSNodesStatus(liveNodeList, deadNodeList);
removeDecomNodeFromList(deadNodeList);
for (DatanodeDescriptor node : deadNodeList) {
final Map<String, Object> innerinfo = new HashMap<String, Object>(); final Map<String, Object> innerinfo = new HashMap<String, Object>();
innerinfo.put("lastContact", getLastContact(node)); innerinfo.put("lastContact", getLastContact(node));
innerinfo.put("decommissioned", node.isDecommissioned()); innerinfo.put("decommissioned", node.isDecommissioned());
@ -4734,8 +4371,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
public String getDecomNodes() { public String getDecomNodes() {
final Map<String, Map<String, Object>> info = final Map<String, Map<String, Object>> info =
new HashMap<String, Map<String, Object>>(); new HashMap<String, Map<String, Object>>();
final ArrayList<DatanodeDescriptor> decomNodeList = final List<DatanodeDescriptor> decomNodeList = blockManager.getDatanodeManager(
this.getDecommissioningNodes(); ).getDecommissioningNodes();
for (DatanodeDescriptor node : decomNodeList) { for (DatanodeDescriptor node : decomNodeList) {
final Map<String, Object> innerinfo = new HashMap<String, Object>(); final Map<String, Object> innerinfo = new HashMap<String, Object>();
innerinfo.put("underReplicatedBlocks", node.decommissioningStatus innerinfo.put("underReplicatedBlocks", node.decommissioningStatus
@ -4771,18 +4408,4 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
public BlockManager getBlockManager() { public BlockManager getBlockManager() {
return blockManager; return blockManager;
} }
void removeDecomNodeFromList(List<DatanodeDescriptor> nodeList) {
getBlockManager().getDatanodeManager().removeDecomNodeFromList(nodeList);
}
/**
* Tell all datanodes to use a new, non-persistent bandwidth value for
* dfs.datanode.balance.bandwidthPerSec.
* @param bandwidth Blanacer bandwidth in bytes per second for all datanodes.
* @throws IOException
*/
public void setBalancerBandwidth(long bandwidth) throws IOException {
getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
}
} }

View File

@ -30,6 +30,7 @@ import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
/** /**
@ -59,13 +60,12 @@ public class FsckServlet extends DfsServlet {
NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context); NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
final FSNamesystem namesystem = nn.getNamesystem(); final FSNamesystem namesystem = nn.getNamesystem();
final BlockManager bm = namesystem.getBlockManager();
final int totalDatanodes = final int totalDatanodes =
namesystem.getNumberOfDatanodes(DatanodeReportType.LIVE); namesystem.getNumberOfDatanodes(DatanodeReportType.LIVE);
final short minReplication = namesystem.getMinReplication();
new NamenodeFsck(conf, nn, new NamenodeFsck(conf, nn,
NamenodeJspHelper.getNetworkTopology(nn), pmap, out, bm.getDatanodeManager().getNetworkTopology(), pmap, out,
totalDatanodes, minReplication, remoteAddress).fsck(); totalDatanodes, bm.minReplication, remoteAddress).fsck();
return null; return null;
} }

View File

@ -855,7 +855,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
DatanodeInfo[] nodes = blocks[i].getLocations(); DatanodeInfo[] nodes = blocks[i].getLocations();
for (int j = 0; j < nodes.length; j++) { for (int j = 0; j < nodes.length; j++) {
DatanodeInfo dn = nodes[j]; DatanodeInfo dn = nodes[j];
namesystem.markBlockAsCorrupt(blk, dn); namesystem.getBlockManager().findAndMarkBlockAsCorrupt(blk, dn);
} }
} }
} }
@ -1055,7 +1055,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
@Override @Override
public RemoteEditLogManifest getEditLogManifest(long sinceTxId) public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
throws IOException { throws IOException {
return namesystem.getEditLogManifest(sinceTxId); return namesystem.getEditLog().getEditLogManifest(sinceTxId);
} }
@Override // ClientProtocol @Override // ClientProtocol
@ -1096,8 +1096,9 @@ public class NameNode implements NamenodeProtocols, FSConstants {
* @param bandwidth Blanacer bandwidth in bytes per second for all datanodes. * @param bandwidth Blanacer bandwidth in bytes per second for all datanodes.
* @throws IOException * @throws IOException
*/ */
@Override // ClientProtocol
public void setBalancerBandwidth(long bandwidth) throws IOException { public void setBalancerBandwidth(long bandwidth) throws IOException {
namesystem.setBalancerBandwidth(bandwidth); namesystem.getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
} }
@Override // ClientProtocol @Override // ClientProtocol
@ -1195,7 +1196,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
+ " blocks"); + " blocks");
} }
namesystem.processReport(nodeReg, poolId, blist); namesystem.getBlockManager().processReport(nodeReg, poolId, blist);
if (getFSImage().isUpgradeFinalized()) if (getFSImage().isUpgradeFinalized())
return new DatanodeCommand.Finalize(poolId); return new DatanodeCommand.Finalize(poolId);
return null; return null;
@ -1210,7 +1211,8 @@ public class NameNode implements NamenodeProtocols, FSConstants {
+"from "+nodeReg.getName()+" "+blocks.length+" blocks."); +"from "+nodeReg.getName()+" "+blocks.length+" blocks.");
} }
for (int i = 0; i < blocks.length; i++) { for (int i = 0; i < blocks.length; i++) {
namesystem.blockReceived(nodeReg, poolId, blocks[i], delHints[i]); namesystem.getBlockManager().blockReceived(
nodeReg, poolId, blocks[i], delHints[i]);
} }
} }

View File

@ -42,13 +42,14 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction; import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport; import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.NodeBase; import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
@ -229,14 +230,10 @@ class NamenodeJspHelper {
void generateHealthReport(JspWriter out, NameNode nn, void generateHealthReport(JspWriter out, NameNode nn,
HttpServletRequest request) throws IOException { HttpServletRequest request) throws IOException {
FSNamesystem fsn = nn.getNamesystem(); FSNamesystem fsn = nn.getNamesystem();
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
fsn.DFSNodesStatus(live, dead); final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
// If a data node has been first included in the include list, dm.fetchDatanodes(live, dead, true);
// then decommissioned, then removed from both include and exclude list.
// We make the web console to "forget" this node by not displaying it.
fsn.removeDecomNodeFromList(live);
fsn.removeDecomNodeFromList(dead);
int liveDecommissioned = 0; int liveDecommissioned = 0;
for (DatanodeDescriptor d : live) { for (DatanodeDescriptor d : live) {
@ -248,8 +245,7 @@ class NamenodeJspHelper {
deadDecommissioned += d.isDecommissioned() ? 1 : 0; deadDecommissioned += d.isDecommissioned() ? 1 : 0;
} }
ArrayList<DatanodeDescriptor> decommissioning = fsn final List<DatanodeDescriptor> decommissioning = dm.getDecommissioningNodes();
.getDecommissioningNodes();
sorterField = request.getParameter("sorter/field"); sorterField = request.getParameter("sorter/field");
sorterOrder = request.getParameter("sorter/order"); sorterOrder = request.getParameter("sorter/order");
@ -370,15 +366,10 @@ class NamenodeJspHelper {
return token == null ? null : token.encodeToUrlString(); return token == null ? null : token.encodeToUrlString();
} }
/** @return the network topology. */
static NetworkTopology getNetworkTopology(final NameNode namenode) {
return namenode.getNamesystem().getBlockManager().getDatanodeManager(
).getNetworkTopology();
}
/** @return a randomly chosen datanode. */ /** @return a randomly chosen datanode. */
static DatanodeDescriptor getRandomDatanode(final NameNode namenode) { static DatanodeDescriptor getRandomDatanode(final NameNode namenode) {
return (DatanodeDescriptor)getNetworkTopology(namenode).chooseRandom( return (DatanodeDescriptor)namenode.getNamesystem().getBlockManager(
).getDatanodeManager().getNetworkTopology().chooseRandom(
NodeBase.ROOT); NodeBase.ROOT);
} }
@ -564,12 +555,14 @@ class NamenodeJspHelper {
void generateNodesList(ServletContext context, JspWriter out, void generateNodesList(ServletContext context, JspWriter out,
HttpServletRequest request) throws IOException { HttpServletRequest request) throws IOException {
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context); final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
nn.getNamesystem().DFSNodesStatus(live, dead); final FSNamesystem ns = nn.getNamesystem();
nn.getNamesystem().removeDecomNodeFromList(live); final DatanodeManager dm = ns.getBlockManager().getDatanodeManager();
nn.getNamesystem().removeDecomNodeFromList(dead);
final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
dm.fetchDatanodes(live, dead, true);
InetSocketAddress nnSocketAddress = (InetSocketAddress) context InetSocketAddress nnSocketAddress = (InetSocketAddress) context
.getAttribute(NameNodeHttpServer.NAMENODE_ADDRESS_ATTRIBUTE_KEY); .getAttribute(NameNodeHttpServer.NAMENODE_ADDRESS_ATTRIBUTE_KEY);
String nnaddr = nnSocketAddress.getAddress().getHostAddress() + ":" String nnaddr = nnSocketAddress.getAddress().getHostAddress() + ":"
@ -678,8 +671,7 @@ class NamenodeJspHelper {
} }
} else if (whatNodes.equals("DECOMMISSIONING")) { } else if (whatNodes.equals("DECOMMISSIONING")) {
// Decommissioning Nodes // Decommissioning Nodes
ArrayList<DatanodeDescriptor> decommissioning = nn.getNamesystem() final List<DatanodeDescriptor> decommissioning = dm.getDecommissioningNodes();
.getDecommissioningNodes();
out.print("<br> <a name=\"DecommissioningNodes\" id=\"title\"> " out.print("<br> <a name=\"DecommissioningNodes\" id=\"title\"> "
+ " Decommissioning Datanodes : " + decommissioning.size() + " Decommissioning Datanodes : " + decommissioning.size()
+ "</a><br><br>\n"); + "</a><br><br>\n");
@ -715,16 +707,17 @@ class NamenodeJspHelper {
static class XMLBlockInfo { static class XMLBlockInfo {
final Block block; final Block block;
final INodeFile inode; final INodeFile inode;
final FSNamesystem fsn; final BlockManager blockManager;
public XMLBlockInfo(FSNamesystem fsn, Long blockId) { XMLBlockInfo(FSNamesystem fsn, Long blockId) {
this.fsn = fsn; this.blockManager = fsn.getBlockManager();
if (blockId == null) { if (blockId == null) {
this.block = null; this.block = null;
this.inode = null; this.inode = null;
} else { } else {
this.block = new Block(blockId); this.block = new Block(blockId);
this.inode = fsn.getBlockManager().getINode(block); this.inode = blockManager.getINode(block);
} }
} }
@ -798,31 +791,25 @@ class NamenodeJspHelper {
} }
doc.startTag("replicas"); doc.startTag("replicas");
for(final Iterator<DatanodeDescriptor> it = blockManager.datanodeIterator(block);
if (fsn.getBlockManager().blocksMap.contains(block)) { it.hasNext(); ) {
Iterator<DatanodeDescriptor> it = doc.startTag("replica");
fsn.getBlockManager().blocksMap.nodeIterator(block);
while (it.hasNext()) { DatanodeDescriptor dd = it.next();
doc.startTag("replica");
DatanodeDescriptor dd = it.next(); doc.startTag("host_name");
doc.pcdata(dd.getHostName());
doc.endTag();
doc.startTag("host_name"); boolean isCorrupt = blockManager.getCorruptReplicaBlockIds(0,
doc.pcdata(dd.getHostName()); block.getBlockId()) != null;
doc.endTag();
doc.startTag("is_corrupt");
boolean isCorrupt = fsn.getCorruptReplicaBlockIds(0, doc.pcdata(""+isCorrupt);
block.getBlockId()) != null; doc.endTag();
doc.startTag("is_corrupt"); doc.endTag(); // </replica>
doc.pcdata(""+isCorrupt); }
doc.endTag();
doc.endTag(); // </replica>
}
}
doc.endTag(); // </replicas> doc.endTag(); // </replicas>
} }
@ -834,14 +821,14 @@ class NamenodeJspHelper {
// utility class used in corrupt_replicas_xml.jsp // utility class used in corrupt_replicas_xml.jsp
static class XMLCorruptBlockInfo { static class XMLCorruptBlockInfo {
final FSNamesystem fsn;
final Configuration conf; final Configuration conf;
final Long startingBlockId; final Long startingBlockId;
final int numCorruptBlocks; final int numCorruptBlocks;
final BlockManager blockManager;
public XMLCorruptBlockInfo(FSNamesystem fsn, Configuration conf, XMLCorruptBlockInfo(FSNamesystem fsn, Configuration conf,
int numCorruptBlocks, Long startingBlockId) { int numCorruptBlocks, Long startingBlockId) {
this.fsn = fsn; this.blockManager = fsn.getBlockManager();
this.conf = conf; this.conf = conf;
this.numCorruptBlocks = numCorruptBlocks; this.numCorruptBlocks = numCorruptBlocks;
this.startingBlockId = startingBlockId; this.startingBlockId = startingBlockId;
@ -864,17 +851,16 @@ class NamenodeJspHelper {
doc.endTag(); doc.endTag();
doc.startTag("num_missing_blocks"); doc.startTag("num_missing_blocks");
doc.pcdata(""+fsn.getMissingBlocksCount()); doc.pcdata(""+blockManager.getMissingBlocksCount());
doc.endTag(); doc.endTag();
doc.startTag("num_corrupt_replica_blocks"); doc.startTag("num_corrupt_replica_blocks");
doc.pcdata(""+fsn.getCorruptReplicaBlocks()); doc.pcdata(""+blockManager.getCorruptReplicaBlocksCount());
doc.endTag(); doc.endTag();
doc.startTag("corrupt_replica_block_ids"); doc.startTag("corrupt_replica_block_ids");
long[] corruptBlockIds final long[] corruptBlockIds = blockManager.getCorruptReplicaBlockIds(
= fsn.getCorruptReplicaBlockIds(numCorruptBlocks, numCorruptBlocks, startingBlockId);
startingBlockId);
if (corruptBlockIds != null) { if (corruptBlockIds != null) {
for (Long blockId: corruptBlockIds) { for (Long blockId: corruptBlockIds) {
doc.startTag("block_id"); doc.startTag("block_id");

View File

@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw; import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
@ -375,10 +376,9 @@ public class DFSTestUtil {
/* /*
* Return the total capacity of all live DNs. * Return the total capacity of all live DNs.
*/ */
public static long getLiveDatanodeCapacity(FSNamesystem ns) { public static long getLiveDatanodeCapacity(DatanodeManager dm) {
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); dm.fetchDatanodes(live, null, false);
ns.DFSNodesStatus(live, dead);
long capacity = 0; long capacity = 0;
for (final DatanodeDescriptor dn : live) { for (final DatanodeDescriptor dn : live) {
capacity += dn.getCapacity(); capacity += dn.getCapacity();
@ -389,21 +389,20 @@ public class DFSTestUtil {
/* /*
* Return the capacity of the given live DN. * Return the capacity of the given live DN.
*/ */
public static long getDatanodeCapacity(FSNamesystem ns, int index) { public static long getDatanodeCapacity(DatanodeManager dm, int index) {
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); dm.fetchDatanodes(live, null, false);
ns.DFSNodesStatus(live, dead);
return live.get(index).getCapacity(); return live.get(index).getCapacity();
} }
/* /*
* Wait for the given # live/dead DNs, total capacity, and # vol failures. * Wait for the given # live/dead DNs, total capacity, and # vol failures.
*/ */
public static void waitForDatanodeStatus(FSNamesystem ns, int expectedLive, public static void waitForDatanodeStatus(DatanodeManager dm, int expectedLive,
int expectedDead, long expectedVolFails, long expectedTotalCapacity, int expectedDead, long expectedVolFails, long expectedTotalCapacity,
long timeout) throws InterruptedException, TimeoutException { long timeout) throws InterruptedException, TimeoutException {
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
final int ATTEMPTS = 10; final int ATTEMPTS = 10;
int count = 0; int count = 0;
long currTotalCapacity = 0; long currTotalCapacity = 0;
@ -413,7 +412,7 @@ public class DFSTestUtil {
Thread.sleep(timeout); Thread.sleep(timeout);
live.clear(); live.clear();
dead.clear(); dead.clear();
ns.DFSNodesStatus(live, dead); dm.fetchDatanodes(live, dead, false);
currTotalCapacity = 0; currTotalCapacity = 0;
volFails = 0; volFails = 0;
for (final DatanodeDescriptor dd : live) { for (final DatanodeDescriptor dd : live) {

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
/** /**
* This class tests DatanodeDescriptor.getBlocksScheduled() at the * This class tests DatanodeDescriptor.getBlocksScheduled() at the
@ -50,7 +51,9 @@ public class TestBlocksScheduledCounter extends TestCase {
((DFSOutputStream)(out.getWrappedStream())).hflush(); ((DFSOutputStream)(out.getWrappedStream())).hflush();
ArrayList<DatanodeDescriptor> dnList = new ArrayList<DatanodeDescriptor>(); ArrayList<DatanodeDescriptor> dnList = new ArrayList<DatanodeDescriptor>();
cluster.getNamesystem().DFSNodesStatus(dnList, dnList); final DatanodeManager dm = cluster.getNamesystem().getBlockManager(
).getDatanodeManager();
dm.fetchDatanodes(dnList, dnList, false);
DatanodeDescriptor dn = dnList.get(0); DatanodeDescriptor dn = dnList.get(0);
assertEquals(1, dn.getBlocksScheduled()); assertEquals(1, dn.getBlocksScheduled());

View File

@ -146,8 +146,8 @@ public class TestFileCorruption extends TestCase {
// report corrupted block by the third datanode // report corrupted block by the third datanode
DatanodeRegistration dnR = DatanodeRegistration dnR =
DataNodeTestUtils.getDNRegistrationForBP(dataNode, blk.getBlockPoolId()); DataNodeTestUtils.getDNRegistrationForBP(dataNode, blk.getBlockPoolId());
cluster.getNamesystem().markBlockAsCorrupt(blk, cluster.getNamesystem().getBlockManager().findAndMarkBlockAsCorrupt(
new DatanodeInfo(dnR)); blk, new DatanodeInfo(dnR));
// open the file // open the file
fs.open(FILE_PATH); fs.open(FILE_PATH);

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.junit.Test; import org.junit.Test;
@ -133,7 +134,7 @@ public class TestBlocksWithNotEnoughRacks {
DFSTestUtil.waitForReplication(cluster, b, 1, REPLICATION_FACTOR, 0); DFSTestUtil.waitForReplication(cluster, b, 1, REPLICATION_FACTOR, 0);
REPLICATION_FACTOR = 2; REPLICATION_FACTOR = 2;
ns.setReplication("/testFile", REPLICATION_FACTOR); NameNodeAdapter.setReplication(ns, "/testFile", REPLICATION_FACTOR);
DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0); DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0);
} finally { } finally {
cluster.shutdown(); cluster.shutdown();
@ -172,7 +173,7 @@ public class TestBlocksWithNotEnoughRacks {
String newRacks[] = {"/rack2", "/rack2"}; String newRacks[] = {"/rack2", "/rack2"};
cluster.startDataNodes(conf, 2, true, null, newRacks); cluster.startDataNodes(conf, 2, true, null, newRacks);
REPLICATION_FACTOR = 5; REPLICATION_FACTOR = 5;
ns.setReplication("/testFile", REPLICATION_FACTOR); NameNodeAdapter.setReplication(ns, "/testFile", REPLICATION_FACTOR);
DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0); DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0);
} finally { } finally {
@ -258,7 +259,7 @@ public class TestBlocksWithNotEnoughRacks {
// was not the one that lived on the rack with only one replica, // was not the one that lived on the rack with only one replica,
// ie we should still have 2 racks after reducing the repl factor. // ie we should still have 2 racks after reducing the repl factor.
REPLICATION_FACTOR = 2; REPLICATION_FACTOR = 2;
ns.setReplication("/testFile", REPLICATION_FACTOR); NameNodeAdapter.setReplication(ns, "/testFile", REPLICATION_FACTOR);
DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0); DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0);
} finally { } finally {

View File

@ -33,10 +33,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
import org.apache.hadoop.hdfs.TestDatanodeBlockScanner; import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.HeartbeatManager;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
public class TestOverReplicatedBlocks extends TestCase { public class TestOverReplicatedBlocks extends TestCase {
/** Test processOverReplicatedBlock can handle corrupt replicas fine. /** Test processOverReplicatedBlock can handle corrupt replicas fine.
@ -100,7 +99,7 @@ public class TestOverReplicatedBlocks extends TestCase {
} }
// decrease the replication factor to 1; // decrease the replication factor to 1;
namesystem.setReplication(fileName.toString(), (short)1); NameNodeAdapter.setReplication(namesystem, fileName.toString(), (short)1);
// corrupt one won't be chosen to be excess one // corrupt one won't be chosen to be excess one
// without 4910 the number of live replicas would be 0: block gets lost // without 4910 the number of live replicas would be 0: block gets lost

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
@ -113,10 +113,11 @@ public class TestDataNodeVolumeFailureReporting {
* heartbeat their capacities. * heartbeat their capacities.
*/ */
Thread.sleep(WAIT_FOR_HEARTBEATS); Thread.sleep(WAIT_FOR_HEARTBEATS);
FSNamesystem ns = cluster.getNamesystem(); final DatanodeManager dm = cluster.getNamesystem().getBlockManager(
).getDatanodeManager();
long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(ns); final long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(dm);
long dnCapacity = DFSTestUtil.getDatanodeCapacity(ns, 0); long dnCapacity = DFSTestUtil.getDatanodeCapacity(dm, 0);
File dn1Vol1 = new File(dataDir, "data"+(2*0+1)); File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
File dn2Vol1 = new File(dataDir, "data"+(2*1+1)); File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
@ -160,7 +161,7 @@ public class TestDataNodeVolumeFailureReporting {
assert (WAIT_FOR_HEARTBEATS * 10) > WAIT_FOR_DEATH; assert (WAIT_FOR_HEARTBEATS * 10) > WAIT_FOR_DEATH;
// Eventually the NN should report two volume failures // Eventually the NN should report two volume failures
DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 2, DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 2,
origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS); origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
/* /*
@ -177,10 +178,10 @@ public class TestDataNodeVolumeFailureReporting {
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
ns.DFSNodesStatus(live, dead); dm.fetchDatanodes(live, dead, false);
live.clear(); live.clear();
dead.clear(); dead.clear();
ns.DFSNodesStatus(live, dead); dm.fetchDatanodes(live, dead, false);
assertEquals("DN3 should have 1 failed volume", assertEquals("DN3 should have 1 failed volume",
1, live.get(2).getVolumeFailures()); 1, live.get(2).getVolumeFailures());
@ -189,8 +190,8 @@ public class TestDataNodeVolumeFailureReporting {
* total capacity should be down by three volumes (assuming the host * total capacity should be down by three volumes (assuming the host
* did not grow or shrink the data volume while the test was running). * did not grow or shrink the data volume while the test was running).
*/ */
dnCapacity = DFSTestUtil.getDatanodeCapacity(ns, 0); dnCapacity = DFSTestUtil.getDatanodeCapacity(dm, 0);
DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 3, DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 3,
origCapacity - (3*dnCapacity), WAIT_FOR_HEARTBEATS); origCapacity - (3*dnCapacity), WAIT_FOR_HEARTBEATS);
/* /*
@ -212,7 +213,7 @@ public class TestDataNodeVolumeFailureReporting {
getMetrics(dns.get(2).getMetrics().name())); getMetrics(dns.get(2).getMetrics().name()));
// The NN considers the DN dead // The NN considers the DN dead
DFSTestUtil.waitForDatanodeStatus(ns, 2, 1, 2, DFSTestUtil.waitForDatanodeStatus(dm, 2, 1, 2,
origCapacity - (4*dnCapacity), WAIT_FOR_HEARTBEATS); origCapacity - (4*dnCapacity), WAIT_FOR_HEARTBEATS);
/* /*
@ -236,7 +237,7 @@ public class TestDataNodeVolumeFailureReporting {
* and that the volume failure count should be reported as zero by * and that the volume failure count should be reported as zero by
* both the metrics and the NN. * both the metrics and the NN.
*/ */
DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 0, origCapacity, DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 0, origCapacity,
WAIT_FOR_HEARTBEATS); WAIT_FOR_HEARTBEATS);
} }
@ -251,9 +252,10 @@ public class TestDataNodeVolumeFailureReporting {
cluster.startDataNodes(conf, 2, true, null, null); cluster.startDataNodes(conf, 2, true, null, null);
cluster.waitActive(); cluster.waitActive();
FSNamesystem ns = cluster.getNamesystem(); final DatanodeManager dm = cluster.getNamesystem().getBlockManager(
long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(ns); ).getDatanodeManager();
long dnCapacity = DFSTestUtil.getDatanodeCapacity(ns, 0); long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(dm);
long dnCapacity = DFSTestUtil.getDatanodeCapacity(dm, 0);
// Fail the first volume on both datanodes (we have to keep the // Fail the first volume on both datanodes (we have to keep the
// third healthy so one node in the pipeline will not fail). // third healthy so one node in the pipeline will not fail).
@ -267,13 +269,13 @@ public class TestDataNodeVolumeFailureReporting {
DFSTestUtil.waitReplication(fs, file1, (short)2); DFSTestUtil.waitReplication(fs, file1, (short)2);
// The NN reports two volumes failures // The NN reports two volumes failures
DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 2, DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 2,
origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS); origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
// After restarting the NN it still see the two failures // After restarting the NN it still see the two failures
cluster.restartNameNode(0); cluster.restartNameNode(0);
cluster.waitActive(); cluster.waitActive();
DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 2, DFSTestUtil.waitForDatanodeStatus(dm, 3, 0, 2,
origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS); origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
} }
} }

View File

@ -17,29 +17,30 @@
*/ */
package org.apache.hadoop.hdfs.server.datanode; package org.apache.hadoop.hdfs.server.datanode;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assume.assumeTrue;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger; import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*;
import static org.junit.Assume.assumeTrue;
/** /**
* Test the ability of a DN to tolerate volume failures. * Test the ability of a DN to tolerate volume failures.
@ -154,9 +155,10 @@ public class TestDataNodeVolumeFailureToleration {
conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 0); conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 0);
cluster.startDataNodes(conf, 2, true, null, null); cluster.startDataNodes(conf, 2, true, null, null);
cluster.waitActive(); cluster.waitActive();
FSNamesystem ns = cluster.getNamesystem(); final DatanodeManager dm = cluster.getNamesystem().getBlockManager(
long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(ns); ).getDatanodeManager();
long dnCapacity = DFSTestUtil.getDatanodeCapacity(ns, 0); long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(dm);
long dnCapacity = DFSTestUtil.getDatanodeCapacity(dm, 0);
// Fail a volume on the 2nd DN // Fail a volume on the 2nd DN
File dn2Vol1 = new File(dataDir, "data"+(2*1+1)); File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
@ -168,7 +170,7 @@ public class TestDataNodeVolumeFailureToleration {
DFSTestUtil.waitReplication(fs, file1, (short)2); DFSTestUtil.waitReplication(fs, file1, (short)2);
// Check that this single failure caused a DN to die. // Check that this single failure caused a DN to die.
DFSTestUtil.waitForDatanodeStatus(ns, 2, 1, 0, DFSTestUtil.waitForDatanodeStatus(dm, 2, 1, 0,
origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS); origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
// If we restore the volume we should still only be able to get // If we restore the volume we should still only be able to get

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@ -52,6 +51,11 @@ public class NameNodeAdapter {
public static Server getRpcServer(NameNode namenode) { public static Server getRpcServer(NameNode namenode) {
return namenode.server; return namenode.server;
} }
public static boolean setReplication(final FSNamesystem ns,
final String src, final short replication) throws IOException {
return ns.setReplication(src, replication);
}
public static String getLeaseHolderForPath(NameNode namenode, String path) { public static String getLeaseHolderForPath(NameNode namenode, String path) {
return namenode.getNamesystem().leaseManager.getLeaseByPath(path).getHolder(); return namenode.getNamesystem().leaseManager.getLeaseByPath(path).getHolder();

View File

@ -24,6 +24,7 @@ import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
import java.util.Random; import java.util.Random;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -37,6 +38,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -199,13 +201,13 @@ public class TestDecommissioningStatus {
Thread.sleep(5000); Thread.sleep(5000);
FSNamesystem fsn = cluster.getNamesystem(); FSNamesystem fsn = cluster.getNamesystem();
final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
for (int iteration = 0; iteration < numDatanodes; iteration++) { for (int iteration = 0; iteration < numDatanodes; iteration++) {
String downnode = decommissionNode(fsn, conf, client, localFileSys, String downnode = decommissionNode(fsn, conf, client, localFileSys,
iteration); iteration);
decommissionedNodes.add(downnode); decommissionedNodes.add(downnode);
Thread.sleep(5000); Thread.sleep(5000);
ArrayList<DatanodeDescriptor> decommissioningNodes = fsn final List<DatanodeDescriptor> decommissioningNodes = dm.getDecommissioningNodes();
.getDecommissioningNodes();
if (iteration == 0) { if (iteration == 0) {
assertEquals(decommissioningNodes.size(), 1); assertEquals(decommissioningNodes.size(), 1);
DatanodeDescriptor decommNode = decommissioningNodes.get(0); DatanodeDescriptor decommNode = decommissioningNodes.get(0);

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.File; import java.io.File;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List;
import junit.framework.TestCase; import junit.framework.TestCase;
@ -32,6 +33,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@ -59,11 +61,13 @@ public class TestNamenodeCapacityReport extends TestCase {
cluster.waitActive(); cluster.waitActive();
final FSNamesystem namesystem = cluster.getNamesystem(); final FSNamesystem namesystem = cluster.getNamesystem();
final DatanodeManager dm = cluster.getNamesystem().getBlockManager(
).getDatanodeManager();
// Ensure the data reported for each data node is right // Ensure the data reported for each data node is right
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>(); final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>(); final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
namesystem.DFSNodesStatus(live, dead); dm.fetchDatanodes(live, dead, false);
assertTrue(live.size() == 1); assertTrue(live.size() == 1);

View File

@ -75,6 +75,7 @@ public class TestNameNodeMetrics extends TestCase {
private DistributedFileSystem fs; private DistributedFileSystem fs;
private Random rand = new Random(); private Random rand = new Random();
private FSNamesystem namesystem; private FSNamesystem namesystem;
private BlockManager bm;
private static Path getTestPath(String fileName) { private static Path getTestPath(String fileName) {
return new Path(TEST_ROOT_DIR_PATH, fileName); return new Path(TEST_ROOT_DIR_PATH, fileName);
@ -85,6 +86,7 @@ public class TestNameNodeMetrics extends TestCase {
cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(DATANODE_COUNT).build(); cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(DATANODE_COUNT).build();
cluster.waitActive(); cluster.waitActive();
namesystem = cluster.getNamesystem(); namesystem = cluster.getNamesystem();
bm = namesystem.getBlockManager();
fs = (DistributedFileSystem) cluster.getFileSystem(); fs = (DistributedFileSystem) cluster.getFileSystem();
} }
@ -167,7 +169,7 @@ public class TestNameNodeMetrics extends TestCase {
// Corrupt first replica of the block // Corrupt first replica of the block
LocatedBlock block = NameNodeAdapter.getBlockLocations( LocatedBlock block = NameNodeAdapter.getBlockLocations(
cluster.getNameNode(), file.toString(), 0, 1).get(0); cluster.getNameNode(), file.toString(), 0, 1).get(0);
namesystem.markBlockAsCorrupt(block.getBlock(), block.getLocations()[0]); bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0]);
updateMetrics(); updateMetrics();
MetricsRecordBuilder rb = getMetrics(NS_METRICS); MetricsRecordBuilder rb = getMetrics(NS_METRICS);
assertGauge("CorruptBlocks", 1L, rb); assertGauge("CorruptBlocks", 1L, rb);
@ -188,7 +190,7 @@ public class TestNameNodeMetrics extends TestCase {
Path file = getTestPath("testExcessBlocks"); Path file = getTestPath("testExcessBlocks");
createFile(file, 100, (short)2); createFile(file, 100, (short)2);
long totalBlocks = 1; long totalBlocks = 1;
namesystem.setReplication(file.toString(), (short)1); NameNodeAdapter.setReplication(namesystem, file.toString(), (short)1);
updateMetrics(); updateMetrics();
MetricsRecordBuilder rb = getMetrics(NS_METRICS); MetricsRecordBuilder rb = getMetrics(NS_METRICS);
assertGauge("ExcessBlocks", totalBlocks, rb); assertGauge("ExcessBlocks", totalBlocks, rb);
@ -204,7 +206,7 @@ public class TestNameNodeMetrics extends TestCase {
// Corrupt the only replica of the block to result in a missing block // Corrupt the only replica of the block to result in a missing block
LocatedBlock block = NameNodeAdapter.getBlockLocations( LocatedBlock block = NameNodeAdapter.getBlockLocations(
cluster.getNameNode(), file.toString(), 0, 1).get(0); cluster.getNameNode(), file.toString(), 0, 1).get(0);
namesystem.markBlockAsCorrupt(block.getBlock(), block.getLocations()[0]); bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0]);
updateMetrics(); updateMetrics();
MetricsRecordBuilder rb = getMetrics(NS_METRICS); MetricsRecordBuilder rb = getMetrics(NS_METRICS);
assertGauge("UnderReplicatedBlocks", 1L, rb); assertGauge("UnderReplicatedBlocks", 1L, rb);

View File

@ -455,7 +455,7 @@ public class TestNNLeaseRecovery {
fsn.leaseManager.addLease("mock-lease", file.toString()); fsn.leaseManager.addLease("mock-lease", file.toString());
if (setStoredBlock) { if (setStoredBlock) {
when(b1.getINode()).thenReturn(iNFmock); when(b1.getINode()).thenReturn(iNFmock);
fsn.getBlockManager().blocksMap.addINode(b1, iNFmock); fsn.getBlockManager().addINode(b1, iNFmock);
} }
when(fsDir.getFileINode(anyString())).thenReturn(iNFmock); when(fsDir.getFileINode(anyString())).thenReturn(iNFmock);