HDFS-2112. Move ReplicationMonitor to block management. Contributed by Uma Maheswara Rao G
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1149771 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ad3f694261
commit
89537b7710
|
@ -590,6 +590,9 @@ Trunk (unreleased changes)
|
|||
HDFS-2116. Use Mokito in TestStreamFile and TestByteRangeInputStream.
|
||||
(Plamen Jeliazkov via shv)
|
||||
|
||||
HDFS-2112. Move ReplicationMonitor to block management. (Uma Maheswara
|
||||
Rao G via szetszwo)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
|
||||
|
|
|
@ -42,9 +42,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
|
||||
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.ReplicaState;
|
||||
|
@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|||
import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.net.Node;
|
||||
import org.apache.hadoop.util.Daemon;
|
||||
|
||||
/**
|
||||
* Keeps information related to the blocks stored in the Hadoop cluster.
|
||||
|
@ -101,6 +102,9 @@ public class BlockManager {
|
|||
return excessBlocksCount;
|
||||
}
|
||||
|
||||
/**replicationRecheckInterval is how often namenode checks for new replication work*/
|
||||
private final long replicationRecheckInterval;
|
||||
|
||||
/**
|
||||
* Mapping: Block -> { INode, datanodes, self ref }
|
||||
* Updated only in response to client-sent information.
|
||||
|
@ -109,6 +113,9 @@ public class BlockManager {
|
|||
|
||||
private final DatanodeManager datanodeManager;
|
||||
|
||||
/** Replication thread. */
|
||||
final Daemon replicationThread = new Daemon(new ReplicationMonitor());
|
||||
|
||||
/** Store blocks -> datanodedescriptor(s) map of corrupt replicas */
|
||||
final CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
|
||||
|
||||
|
@ -162,7 +169,6 @@ public class BlockManager {
|
|||
public BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
|
||||
namesystem = fsn;
|
||||
datanodeManager = new DatanodeManager(fsn, conf);
|
||||
|
||||
blocksMap = new BlocksMap(DEFAULT_MAP_LOAD_FACTOR);
|
||||
blockplacement = BlockPlacementPolicy.getInstance(
|
||||
conf, namesystem, datanodeManager.getNetworkTopology());
|
||||
|
@ -198,22 +204,29 @@ public class BlockManager {
|
|||
DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
|
||||
this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null ? false
|
||||
: true;
|
||||
|
||||
this.replicationRecheckInterval =
|
||||
conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
|
||||
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000L;
|
||||
FSNamesystem.LOG.info("defaultReplication = " + defaultReplication);
|
||||
FSNamesystem.LOG.info("maxReplication = " + maxReplication);
|
||||
FSNamesystem.LOG.info("minReplication = " + minReplication);
|
||||
FSNamesystem.LOG.info("maxReplicationStreams = " + maxReplicationStreams);
|
||||
FSNamesystem.LOG.info("shouldCheckForEnoughRacks = " + shouldCheckForEnoughRacks);
|
||||
FSNamesystem.LOG.info("replicationRecheckInterval = " + replicationRecheckInterval);
|
||||
}
|
||||
|
||||
public void activate(Configuration conf) {
|
||||
pendingReplications.start();
|
||||
datanodeManager.activate(conf);
|
||||
this.replicationThread.start();
|
||||
}
|
||||
|
||||
public void close() {
|
||||
if (pendingReplications != null) pendingReplications.stop();
|
||||
blocksMap.close();
|
||||
datanodeManager.close();
|
||||
if (replicationThread != null) replicationThread.interrupt();
|
||||
}
|
||||
|
||||
/** @return the datanodeManager */
|
||||
|
@ -2248,4 +2261,72 @@ public class BlockManager {
|
|||
processOverReplicatedBlocksOnReCommission(node);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Periodically calls computeReplicationWork().
|
||||
*/
|
||||
private class ReplicationMonitor implements Runnable {
|
||||
static final int INVALIDATE_WORK_PCT_PER_ITERATION = 32;
|
||||
static final float REPLICATION_WORK_MULTIPLIER_PER_ITERATION = 2;
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
while (namesystem.isRunning()) {
|
||||
try {
|
||||
computeDatanodeWork();
|
||||
processPendingReplications();
|
||||
Thread.sleep(replicationRecheckInterval);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("ReplicationMonitor thread received InterruptedException.", ie);
|
||||
break;
|
||||
} catch (IOException ie) {
|
||||
LOG.warn("ReplicationMonitor thread received exception. " , ie);
|
||||
} catch (Throwable t) {
|
||||
LOG.warn("ReplicationMonitor thread received Runtime exception. ", t);
|
||||
Runtime.getRuntime().exit(-1);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Compute block replication and block invalidation work that can be scheduled
|
||||
* on data-nodes. The datanode will be informed of this work at the next
|
||||
* heartbeat.
|
||||
*
|
||||
* @return number of blocks scheduled for replication or removal.
|
||||
* @throws IOException
|
||||
*/
|
||||
int computeDatanodeWork() throws IOException {
|
||||
int workFound = 0;
|
||||
int blocksToProcess = 0;
|
||||
int nodesToProcess = 0;
|
||||
// Blocks should not be replicated or removed if in safe mode.
|
||||
// It's OK to check safe mode here w/o holding lock, in the worst
|
||||
// case extra replications will be scheduled, and these will get
|
||||
// fixed up later.
|
||||
if (namesystem.isInSafeMode())
|
||||
return workFound;
|
||||
|
||||
synchronized (namesystem.heartbeats) {
|
||||
blocksToProcess = (int) (namesystem.heartbeats.size() * ReplicationMonitor.REPLICATION_WORK_MULTIPLIER_PER_ITERATION);
|
||||
nodesToProcess = (int) Math.ceil((double) namesystem.heartbeats.size()
|
||||
* ReplicationMonitor.INVALIDATE_WORK_PCT_PER_ITERATION / 100);
|
||||
}
|
||||
|
||||
workFound = this.computeReplicationWork(blocksToProcess);
|
||||
|
||||
// Update FSNamesystemMetrics counters
|
||||
namesystem.writeLock();
|
||||
try {
|
||||
this.updateState();
|
||||
this.scheduledReplicationBlocksCount = workFound;
|
||||
} finally {
|
||||
namesystem.writeUnlock();
|
||||
}
|
||||
workFound += this.computeInvalidateWork(nodesToProcess);
|
||||
return workFound;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -35,11 +35,11 @@ class DecommissionManager {
|
|||
static final Log LOG = LogFactory.getLog(DecommissionManager.class);
|
||||
|
||||
private final FSNamesystem fsnamesystem;
|
||||
private final BlockManager blockmanager;
|
||||
private final BlockManager blockManager;
|
||||
|
||||
DecommissionManager(FSNamesystem namesystem) {
|
||||
this.fsnamesystem = namesystem;
|
||||
this.blockmanager = fsnamesystem.getBlockManager();
|
||||
this.blockManager = fsnamesystem.getBlockManager();
|
||||
}
|
||||
|
||||
/** Periodically check decommission status. */
|
||||
|
@ -90,7 +90,7 @@ class DecommissionManager {
|
|||
|
||||
if (d.isDecommissionInProgress()) {
|
||||
try {
|
||||
blockmanager.checkDecommissionStateInternal(d);
|
||||
blockManager.checkDecommissionStateInternal(d);
|
||||
} catch(Exception e) {
|
||||
LOG.warn("entry=" + entry, e);
|
||||
}
|
||||
|
|
|
@ -154,7 +154,7 @@ public class FSDirectory implements Closeable {
|
|||
}
|
||||
|
||||
private BlockManager getBlockManager() {
|
||||
return getFSNamesystem().blockManager;
|
||||
return getFSNamesystem().getBlockManager();
|
||||
}
|
||||
|
||||
void loadFSImage(Collection<URI> dataDirs,
|
||||
|
|
|
@ -233,7 +233,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|||
// Stores the correct file name hierarchy
|
||||
//
|
||||
public FSDirectory dir;
|
||||
BlockManager blockManager;
|
||||
private BlockManager blockManager;
|
||||
|
||||
// Block pool ID used by this namenode
|
||||
String blockPoolId;
|
||||
|
@ -280,7 +280,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|||
Daemon hbthread = null; // HeartbeatMonitor thread
|
||||
public Daemon lmthread = null; // LeaseMonitor thread
|
||||
Daemon smmthread = null; // SafeModeMonitor thread
|
||||
public Daemon replthread = null; // Replication thread
|
||||
|
||||
Daemon nnrmthread = null; // NamenodeResourceMonitor thread
|
||||
|
||||
private volatile boolean hasResourcesAvailable = false;
|
||||
|
@ -292,8 +292,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|||
// heartbeatExpireInterval is how long namenode waits for datanode to report
|
||||
// heartbeat
|
||||
private long heartbeatExpireInterval;
|
||||
//replicationRecheckInterval is how often namenode checks for new replication work
|
||||
private long replicationRecheckInterval;
|
||||
|
||||
//resourceRecheckInterval is how often namenode checks for the disk space availability
|
||||
private long resourceRecheckInterval;
|
||||
|
@ -387,10 +385,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|||
blockManager.activate(conf);
|
||||
this.hbthread = new Daemon(new HeartbeatMonitor());
|
||||
this.lmthread = new Daemon(leaseManager.new Monitor());
|
||||
this.replthread = new Daemon(new ReplicationMonitor());
|
||||
|
||||
hbthread.start();
|
||||
lmthread.start();
|
||||
replthread.start();
|
||||
|
||||
this.nnrmthread = new Daemon(new NameNodeResourceMonitor());
|
||||
nnrmthread.start();
|
||||
|
@ -524,9 +521,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|||
DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT); // 5 minutes
|
||||
this.heartbeatExpireInterval = 2 * heartbeatRecheckInterval +
|
||||
10 * heartbeatInterval;
|
||||
this.replicationRecheckInterval =
|
||||
conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
|
||||
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000L;
|
||||
|
||||
this.serverDefaults = new FsServerDefaults(
|
||||
conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE),
|
||||
conf.getInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BYTES_PER_CHECKSUM),
|
||||
|
@ -595,7 +590,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|||
try {
|
||||
if (blockManager != null) blockManager.close();
|
||||
if (hbthread != null) hbthread.interrupt();
|
||||
if (replthread != null) replthread.interrupt();
|
||||
if (smmthread != null) smmthread.interrupt();
|
||||
if (dtSecretManager != null) dtSecretManager.stopThreads();
|
||||
if (nnrmthread != null) nnrmthread.interrupt();
|
||||
|
@ -3009,76 +3003,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Periodically calls computeReplicationWork().
|
||||
*/
|
||||
class ReplicationMonitor implements Runnable {
|
||||
static final int INVALIDATE_WORK_PCT_PER_ITERATION = 32;
|
||||
static final float REPLICATION_WORK_MULTIPLIER_PER_ITERATION = 2;
|
||||
public void run() {
|
||||
while (fsRunning) {
|
||||
try {
|
||||
computeDatanodeWork();
|
||||
blockManager.processPendingReplications();
|
||||
Thread.sleep(replicationRecheckInterval);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("ReplicationMonitor thread received InterruptedException." + ie);
|
||||
break;
|
||||
} catch (IOException ie) {
|
||||
LOG.warn("ReplicationMonitor thread received exception. " + ie);
|
||||
} catch (Throwable t) {
|
||||
LOG.warn("ReplicationMonitor thread received Runtime exception. " + t);
|
||||
Runtime.getRuntime().exit(-1);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/////////////////////////////////////////////////////////
|
||||
//
|
||||
// These methods are called by the Namenode system, to see
|
||||
// if there is any work for registered datanodes.
|
||||
//
|
||||
/////////////////////////////////////////////////////////
|
||||
/**
|
||||
* Compute block replication and block invalidation work
|
||||
* that can be scheduled on data-nodes.
|
||||
* The datanode will be informed of this work at the next heartbeat.
|
||||
*
|
||||
* @return number of blocks scheduled for replication or removal.
|
||||
* @throws IOException
|
||||
*/
|
||||
public int computeDatanodeWork() throws IOException {
|
||||
int workFound = 0;
|
||||
int blocksToProcess = 0;
|
||||
int nodesToProcess = 0;
|
||||
// Blocks should not be replicated or removed if in safe mode.
|
||||
// It's OK to check safe mode here w/o holding lock, in the worst
|
||||
// case extra replications will be scheduled, and these will get
|
||||
// fixed up later.
|
||||
if (isInSafeMode())
|
||||
return workFound;
|
||||
|
||||
synchronized (heartbeats) {
|
||||
blocksToProcess = (int)(heartbeats.size()
|
||||
* ReplicationMonitor.REPLICATION_WORK_MULTIPLIER_PER_ITERATION);
|
||||
nodesToProcess = (int)Math.ceil((double)heartbeats.size()
|
||||
* ReplicationMonitor.INVALIDATE_WORK_PCT_PER_ITERATION / 100);
|
||||
}
|
||||
|
||||
workFound = blockManager.computeReplicationWork(blocksToProcess);
|
||||
|
||||
// Update FSNamesystemMetrics counters
|
||||
writeLock();
|
||||
try {
|
||||
blockManager.updateState();
|
||||
blockManager.scheduledReplicationBlocksCount = workFound;
|
||||
} finally {
|
||||
writeUnlock();
|
||||
}
|
||||
workFound += blockManager.computeInvalidateWork(nodesToProcess);
|
||||
return workFound;
|
||||
}
|
||||
|
||||
public void setNodeReplicationLimit(int limit) {
|
||||
blockManager.maxReplicationStreams = limit;
|
||||
|
|
|
@ -724,7 +724,7 @@ class NamenodeJspHelper {
|
|||
this.inode = null;
|
||||
} else {
|
||||
this.block = new Block(blockId);
|
||||
this.inode = fsn.blockManager.getINode(block);
|
||||
this.inode = fsn.getBlockManager().getINode(block);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -799,9 +799,9 @@ class NamenodeJspHelper {
|
|||
|
||||
doc.startTag("replicas");
|
||||
|
||||
if (fsn.blockManager.blocksMap.contains(block)) {
|
||||
if (fsn.getBlockManager().blocksMap.contains(block)) {
|
||||
Iterator<DatanodeDescriptor> it =
|
||||
fsn.blockManager.blocksMap.nodeIterator(block);
|
||||
fsn.getBlockManager().blocksMap.nodeIterator(block);
|
||||
|
||||
while (it.hasNext()) {
|
||||
doc.startTag("replica");
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.server.blockmanagement;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
|
@ -24,7 +25,7 @@ import java.util.Set;
|
|||
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.util.Daemon;
|
||||
|
||||
public class BlockManagerTestUtil {
|
||||
/**
|
||||
|
@ -48,12 +49,12 @@ public class BlockManagerTestUtil {
|
|||
* decommissioning/decommissioned nodes are not counted. corrupt replicas
|
||||
* are also ignored
|
||||
*/
|
||||
private static int getNumberOfRacks(final BlockManager blockmanager,
|
||||
private static int getNumberOfRacks(final BlockManager blockManager,
|
||||
final Block b) {
|
||||
final Set<String> rackSet = new HashSet<String>(0);
|
||||
final Collection<DatanodeDescriptor> corruptNodes =
|
||||
blockmanager.corruptReplicas.getNodes(b);
|
||||
for (Iterator<DatanodeDescriptor> it = blockmanager.blocksMap.nodeIterator(b);
|
||||
getCorruptReplicas(blockManager).getNodes(b);
|
||||
for (Iterator<DatanodeDescriptor> it = blockManager.blocksMap.nodeIterator(b);
|
||||
it.hasNext();) {
|
||||
DatanodeDescriptor cur = it.next();
|
||||
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
|
||||
|
@ -68,4 +69,33 @@ public class BlockManagerTestUtil {
|
|||
return rackSet.size();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param blockManager
|
||||
* @return replication monitor thread instance from block manager.
|
||||
*/
|
||||
public static Daemon getReplicationThread(final BlockManager blockManager)
|
||||
{
|
||||
return blockManager.replicationThread;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param blockManager
|
||||
* @return corruptReplicas from block manager
|
||||
*/
|
||||
public static CorruptReplicasMap getCorruptReplicas(final BlockManager blockManager){
|
||||
return blockManager.corruptReplicas;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* @param blockManager
|
||||
* @return computed block replication and block invalidation work that can be
|
||||
* scheduled on data-nodes.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static int getComputedDatanodeWork(final BlockManager blockManager) throws IOException
|
||||
{
|
||||
return blockManager.computeDatanodeWork();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
|
|||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
|
@ -220,7 +221,8 @@ public class TestBlockReport {
|
|||
cluster.getNameNode().blockReport(dnR, poolId,
|
||||
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
||||
|
||||
cluster.getNamesystem().computeDatanodeWork();
|
||||
BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
|
||||
.getBlockManager());
|
||||
|
||||
printStats();
|
||||
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
|
@ -191,7 +192,7 @@ public class TestDataNodeVolumeFailure {
|
|||
// now check the number of under-replicated blocks
|
||||
FSNamesystem fsn = cluster.getNamesystem();
|
||||
// force update of all the metric counts by calling computeDatanodeWork
|
||||
fsn.computeDatanodeWork();
|
||||
BlockManagerTestUtil.getComputedDatanodeWork(fsn.getBlockManager());
|
||||
// get all the counts
|
||||
long underRepl = fsn.getUnderReplicatedBlocks();
|
||||
long pendRepl = fsn.getPendingReplicationBlocks();
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.FSConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
||||
|
@ -1111,9 +1112,11 @@ public class NNThroughputBenchmark {
|
|||
// start data-nodes; create a bunch of files; generate block reports.
|
||||
blockReportObject.generateInputs(ignore);
|
||||
// stop replication monitor
|
||||
namesystem.replthread.interrupt();
|
||||
BlockManagerTestUtil.getReplicationThread(namesystem.getBlockManager())
|
||||
.interrupt();
|
||||
try {
|
||||
namesystem.replthread.join();
|
||||
BlockManagerTestUtil.getReplicationThread(namesystem.getBlockManager())
|
||||
.join();
|
||||
} catch(InterruptedException ei) {
|
||||
return;
|
||||
}
|
||||
|
@ -1156,7 +1159,8 @@ public class NNThroughputBenchmark {
|
|||
assert daemonId < numThreads : "Wrong daemonId.";
|
||||
long start = System.currentTimeMillis();
|
||||
// compute data-node work
|
||||
int work = nameNode.getNamesystem().computeDatanodeWork();
|
||||
int work = BlockManagerTestUtil.getComputedDatanodeWork(nameNode
|
||||
.getNamesystem().getBlockManager());
|
||||
long end = System.currentTimeMillis();
|
||||
numPendingBlocks += work;
|
||||
if(work == 0)
|
||||
|
|
|
@ -50,7 +50,7 @@ public class NameNodeAdapter {
|
|||
* @param namenode to proxy the invocation to
|
||||
*/
|
||||
public static void refreshBlockCounts(NameNode namenode) {
|
||||
namenode.getNamesystem().blockManager.updateState();
|
||||
namenode.getNamesystem().getBlockManager().updateState();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -100,7 +100,7 @@ public class TestBlockUnderConstruction {
|
|||
assertTrue("Block is not complete: " + curBlock,
|
||||
curBlock.isComplete());
|
||||
assertTrue("Block is not in BlocksMap: " + curBlock,
|
||||
ns.blockManager.getStoredBlock(curBlock) == curBlock);
|
||||
ns.getBlockManager().getStoredBlock(curBlock) == curBlock);
|
||||
}
|
||||
|
||||
// the penultimate block is either complete or
|
||||
|
@ -115,7 +115,7 @@ public class TestBlockUnderConstruction {
|
|||
(curBlock.getBlockUCState() ==
|
||||
BlockUCState.COMMITTED)));
|
||||
assertTrue("Block is not in BlocksMap: " + curBlock,
|
||||
ns.blockManager.getStoredBlock(curBlock) == curBlock);
|
||||
ns.getBlockManager().getStoredBlock(curBlock) == curBlock);
|
||||
}
|
||||
|
||||
// The last block is complete if the file is closed.
|
||||
|
@ -126,7 +126,7 @@ public class TestBlockUnderConstruction {
|
|||
curBlock.isComplete());
|
||||
}
|
||||
assertTrue("Block is not in BlocksMap: " + curBlock,
|
||||
ns.blockManager.getStoredBlock(curBlock) == curBlock);
|
||||
ns.getBlockManager().getStoredBlock(curBlock) == curBlock);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -51,23 +51,23 @@ public class TestComputeInvalidateWork extends TestCase {
|
|||
for(int j=0; j<3*namesystem.blockInvalidateLimit+1; j++) {
|
||||
Block block = new Block(i*(namesystem.blockInvalidateLimit+1)+j, 0,
|
||||
GenerationStamp.FIRST_VALID_STAMP);
|
||||
namesystem.blockManager.addToInvalidates(block, nodes[i]);
|
||||
namesystem.getBlockManager().addToInvalidates(block, nodes[i]);
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(namesystem.blockInvalidateLimit*NUM_OF_DATANODES,
|
||||
namesystem.blockManager.computeInvalidateWork(NUM_OF_DATANODES+1));
|
||||
namesystem.getBlockManager().computeInvalidateWork(NUM_OF_DATANODES+1));
|
||||
assertEquals(namesystem.blockInvalidateLimit*NUM_OF_DATANODES,
|
||||
namesystem.blockManager.computeInvalidateWork(NUM_OF_DATANODES));
|
||||
namesystem.getBlockManager().computeInvalidateWork(NUM_OF_DATANODES));
|
||||
assertEquals(namesystem.blockInvalidateLimit*(NUM_OF_DATANODES-1),
|
||||
namesystem.blockManager.computeInvalidateWork(NUM_OF_DATANODES-1));
|
||||
int workCount = namesystem.blockManager.computeInvalidateWork(1);
|
||||
namesystem.getBlockManager().computeInvalidateWork(NUM_OF_DATANODES-1));
|
||||
int workCount = namesystem.getBlockManager().computeInvalidateWork(1);
|
||||
if (workCount == 1) {
|
||||
assertEquals(namesystem.blockInvalidateLimit+1,
|
||||
namesystem.blockManager.computeInvalidateWork(2));
|
||||
namesystem.getBlockManager().computeInvalidateWork(2));
|
||||
} else {
|
||||
assertEquals(workCount, namesystem.blockInvalidateLimit);
|
||||
assertEquals(2, namesystem.blockManager.computeInvalidateWork(2));
|
||||
assertEquals(2, namesystem.getBlockManager().computeInvalidateWork(2));
|
||||
}
|
||||
} finally {
|
||||
namesystem.writeUnlock();
|
||||
|
|
|
@ -78,7 +78,7 @@ public class TestLargeDirectoryDelete {
|
|||
Assert.assertNotNull("No Namenode in cluster", mc.getNameNode());
|
||||
FSNamesystem namesystem = mc.getNamesystem();
|
||||
Assert.assertNotNull("Null Namesystem in cluster", namesystem);
|
||||
Assert.assertNotNull("Null Namesystem.blockmanager", namesystem.blockManager);
|
||||
Assert.assertNotNull("Null Namesystem.blockmanager", namesystem.getBlockManager());
|
||||
return (int) namesystem.getBlocksTotal();
|
||||
}
|
||||
|
||||
|
|
|
@ -102,12 +102,12 @@ public class TestNodeCount extends TestCase {
|
|||
}
|
||||
|
||||
// find out a non-excess node
|
||||
Iterator<DatanodeDescriptor> iter = namesystem.blockManager.blocksMap
|
||||
Iterator<DatanodeDescriptor> iter = namesystem.getBlockManager().blocksMap
|
||||
.nodeIterator(block.getLocalBlock());
|
||||
DatanodeDescriptor nonExcessDN = null;
|
||||
while (iter.hasNext()) {
|
||||
DatanodeDescriptor dn = iter.next();
|
||||
Collection<Block> blocks = namesystem.blockManager.excessReplicateMap.get(dn.getStorageID());
|
||||
Collection<Block> blocks = namesystem.getBlockManager().excessReplicateMap.get(dn.getStorageID());
|
||||
if (blocks == null || !blocks.contains(block) ) {
|
||||
nonExcessDN = dn;
|
||||
break;
|
||||
|
@ -184,7 +184,7 @@ public class TestNodeCount extends TestCase {
|
|||
namesystem.readLock();
|
||||
try {
|
||||
lastBlock = block;
|
||||
lastNum = namesystem.blockManager.countNodes(block);
|
||||
lastNum = namesystem.getBlockManager().countNodes(block);
|
||||
return lastNum;
|
||||
}
|
||||
finally {
|
||||
|
|
|
@ -100,7 +100,7 @@ public class TestOverReplicatedBlocks extends TestCase {
|
|||
|
||||
// corrupt one won't be chosen to be excess one
|
||||
// without 4910 the number of live replicas would be 0: block gets lost
|
||||
assertEquals(1, namesystem.blockManager.countNodes(block.getLocalBlock())
|
||||
assertEquals(1, namesystem.getBlockManager().countNodes(block.getLocalBlock())
|
||||
.liveReplicas());
|
||||
}
|
||||
} finally {
|
||||
|
|
|
@ -455,7 +455,7 @@ public class TestNNLeaseRecovery {
|
|||
fsn.leaseManager.addLease("mock-lease", file.toString());
|
||||
if (setStoredBlock) {
|
||||
when(b1.getINode()).thenReturn(iNFmock);
|
||||
fsn.blockManager.blocksMap.addINode(b1, iNFmock);
|
||||
fsn.getBlockManager().blocksMap.addINode(b1, iNFmock);
|
||||
}
|
||||
|
||||
when(fsDir.getFileINode(anyString())).thenReturn(iNFmock);
|
||||
|
|
Loading…
Reference in New Issue