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:
parent
8dc420ba36
commit
371f4a5905
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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() {
|
||||||
|
|
|
@ -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() {
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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> {
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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]);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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");
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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);
|
||||||
|
|
Loading…
Reference in New Issue