|
@@ -172,6 +172,7 @@ class FSNamesystem implements FSConstants {
|
|
|
Daemon hbthread = null; // HeartbeatMonitor thread
|
|
|
Daemon lmthread = null; // LeaseMonitor thread
|
|
|
Daemon smmthread = null; // SafeModeMonitor thread
|
|
|
+ Daemon replthread = null; // Replication thread
|
|
|
boolean fsRunning = true;
|
|
|
long systemStart = 0;
|
|
|
|
|
@@ -186,6 +187,10 @@ class FSNamesystem implements FSConstants {
|
|
|
// 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;
|
|
|
+ static int replIndex = 0; // last datanode used for replication work
|
|
|
+ static int REPL_WORK_PER_ITERATION = 32; // max percent datanodes per iteration
|
|
|
|
|
|
public static FSNamesystem fsNamesystemObject;
|
|
|
private String localMachine;
|
|
@@ -228,6 +233,7 @@ class FSNamesystem implements FSConstants {
|
|
|
this.heartbeatRecheckInterval = 5 * 60 * 1000; // 5 minutes
|
|
|
this.heartbeatExpireInterval = 2 * heartbeatRecheckInterval +
|
|
|
10 * heartbeatInterval;
|
|
|
+ this.replicationRecheckInterval = 3 * 1000; // 3 second
|
|
|
|
|
|
this.localMachine = hostname;
|
|
|
this.port = port;
|
|
@@ -237,8 +243,10 @@ class FSNamesystem implements FSConstants {
|
|
|
setBlockTotal();
|
|
|
this.hbthread = new Daemon(new HeartbeatMonitor());
|
|
|
this.lmthread = new Daemon(new LeaseMonitor());
|
|
|
+ this.replthread = new Daemon(new ReplicationMonitor());
|
|
|
hbthread.start();
|
|
|
lmthread.start();
|
|
|
+ replthread.start();
|
|
|
this.systemStart = now();
|
|
|
this.startTime = new Date(systemStart);
|
|
|
|
|
@@ -280,6 +288,7 @@ class FSNamesystem implements FSConstants {
|
|
|
try {
|
|
|
infoServer.stop();
|
|
|
hbthread.join(3000);
|
|
|
+ replthread.join(3000);
|
|
|
} catch (InterruptedException ie) {
|
|
|
} finally {
|
|
|
// using finally to ensure we also wait for lease daemon
|
|
@@ -435,6 +444,14 @@ class FSNamesystem implements FSConstants {
|
|
|
int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
|
|
|
int curPri = getPriority(block, curReplicas, curExpectedReplicas);
|
|
|
int oldPri = getPriority(block, oldReplicas, oldExpectedReplicas);
|
|
|
+ NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " +
|
|
|
+ block +
|
|
|
+ " curReplicas " + curReplicas +
|
|
|
+ " curExpectedReplicas " + curExpectedReplicas +
|
|
|
+ " oldReplicas " + oldReplicas +
|
|
|
+ " oldExpectedReplicas " + oldExpectedReplicas +
|
|
|
+ " curPri " + curPri +
|
|
|
+ " oldPri " + oldPri);
|
|
|
if( oldPri != LEVEL && oldPri != curPri ) {
|
|
|
remove(block, oldPri);
|
|
|
}
|
|
@@ -1575,7 +1592,10 @@ class FSNamesystem implements FSConstants {
|
|
|
public boolean gotHeartbeat( DatanodeID nodeID,
|
|
|
long capacity,
|
|
|
long remaining,
|
|
|
- int xceiverCount
|
|
|
+ int xceiverCount,
|
|
|
+ int xmitsInProgress,
|
|
|
+ Object[] xferResults,
|
|
|
+ Object deleteList[]
|
|
|
) throws IOException {
|
|
|
synchronized (heartbeats) {
|
|
|
synchronized (datanodeMap) {
|
|
@@ -1595,6 +1615,16 @@ class FSNamesystem implements FSConstants {
|
|
|
updateStats(nodeinfo, false);
|
|
|
nodeinfo.updateHeartbeat(capacity, remaining, xceiverCount);
|
|
|
updateStats(nodeinfo, true);
|
|
|
+ //
|
|
|
+ // Extract pending replication work or block invalidation
|
|
|
+ // work from the datanode descriptor
|
|
|
+ //
|
|
|
+ nodeinfo.getReplicationSets(this.maxReplicationStreams -
|
|
|
+ xmitsInProgress, xferResults);
|
|
|
+ if (xferResults[0] == null) {
|
|
|
+ nodeinfo.getInvalidateBlocks(FSConstants.BLOCK_INVALIDATE_CHUNK,
|
|
|
+ deleteList);
|
|
|
+ }
|
|
|
return false;
|
|
|
}
|
|
|
}
|
|
@@ -1633,6 +1663,130 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Periodically calls computeReplicationWork().
|
|
|
+ */
|
|
|
+ class ReplicationMonitor implements Runnable {
|
|
|
+ public void run() {
|
|
|
+ while (fsRunning) {
|
|
|
+ try {
|
|
|
+ computeDatanodeWork();
|
|
|
+ Thread.sleep(replicationRecheckInterval);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ } catch (IOException ie) {
|
|
|
+ LOG.warn("ReplicationMonitor thread received exception. " + ie);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Look at a few datanodes and compute any replication work that
|
|
|
+ * can be scheduled on them. The datanode will be infomed of this
|
|
|
+ * work at the next heartbeat.
|
|
|
+ */
|
|
|
+ void computeDatanodeWork() throws IOException {
|
|
|
+ int numiter = 0;
|
|
|
+ int foundwork = 0;
|
|
|
+ int hsize = 0;
|
|
|
+
|
|
|
+ while (true) {
|
|
|
+ DatanodeDescriptor node = null;
|
|
|
+
|
|
|
+ //
|
|
|
+ // pick the datanode that was the last one in the
|
|
|
+ // previous invocation of this method.
|
|
|
+ //
|
|
|
+ synchronized (heartbeats) {
|
|
|
+ hsize = heartbeats.size();
|
|
|
+ if (numiter++ >= hsize) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ if (replIndex >= hsize) {
|
|
|
+ replIndex = 0;
|
|
|
+ }
|
|
|
+ node = heartbeats.get(replIndex);
|
|
|
+ replIndex++;
|
|
|
+ }
|
|
|
+
|
|
|
+ //
|
|
|
+ // Is there replication work to be computed for this datanode?
|
|
|
+ //
|
|
|
+ int precomputed = node.getNumberOfBlocksToBeReplicated();
|
|
|
+ int needed = this.maxReplicationStreams - precomputed;
|
|
|
+ boolean doReplication = false;
|
|
|
+ boolean doInvalidation = false;
|
|
|
+ if (needed > 0) {
|
|
|
+ //
|
|
|
+ // Compute replication work and store work into the datanode
|
|
|
+ //
|
|
|
+ Object replsets[] = pendingTransfers(node, needed);
|
|
|
+ if (replsets != null) {
|
|
|
+ doReplication = true;
|
|
|
+ addBlocksToBeReplicated(node, (Block[])replsets[0],
|
|
|
+ (DatanodeDescriptor[][])replsets[1]);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (!doReplication) {
|
|
|
+ //
|
|
|
+ // Determine if block deletion is pending for this datanode
|
|
|
+ //
|
|
|
+ Block blocklist[] = blocksToInvalidate(node);
|
|
|
+ if (blocklist != null) {
|
|
|
+ doInvalidation = true;
|
|
|
+ addBlocksToBeInvalidated(node, blocklist);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (doReplication || doInvalidation) {
|
|
|
+ //
|
|
|
+ // If we have already computed work for a predefined
|
|
|
+ // number of datanodes in this iteration, then relax
|
|
|
+ //
|
|
|
+ if (foundwork > ((hsize * REPL_WORK_PER_ITERATION)/100)) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ foundwork++;
|
|
|
+ } else {
|
|
|
+ //
|
|
|
+ // See if the decommissioned node has finished moving all
|
|
|
+ // its datablocks to another replica. This is a loose
|
|
|
+ // heuristic to determine when a decommission is really over.
|
|
|
+ //
|
|
|
+ checkDecommissionState(node);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add more replication work for this datanode.
|
|
|
+ */
|
|
|
+ synchronized void addBlocksToBeReplicated(DatanodeDescriptor node,
|
|
|
+ Block[] blocklist,
|
|
|
+ DatanodeDescriptor[][] targets)
|
|
|
+ throws IOException {
|
|
|
+ //
|
|
|
+ // Find the datanode with the FSNamesystem lock held.
|
|
|
+ //
|
|
|
+ DatanodeDescriptor n = getDatanode(node);
|
|
|
+ if (n != null) {
|
|
|
+ n.addBlocksToBeReplicated(blocklist, targets);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add more block invalidation work for this datanode.
|
|
|
+ */
|
|
|
+ synchronized void addBlocksToBeInvalidated(DatanodeDescriptor node,
|
|
|
+ Block[] blocklist) throws IOException {
|
|
|
+ //
|
|
|
+ // Find the datanode with the FSNamesystem lock held.
|
|
|
+ //
|
|
|
+ DatanodeDescriptor n = getDatanode(node);
|
|
|
+ if (n != null) {
|
|
|
+ n.addBlocksToBeInvalidated(blocklist);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* remove a datanode descriptor
|
|
|
* @param nodeID datanode ID
|
|
@@ -2405,7 +2559,7 @@ class FSNamesystem implements FSConstants {
|
|
|
*
|
|
|
*/
|
|
|
public synchronized Object[] pendingTransfers(DatanodeID srcNode,
|
|
|
- int xmitsInProgress) {
|
|
|
+ int needed) {
|
|
|
// Ask datanodes to perform block replication
|
|
|
// only if safe mode is off.
|
|
|
if( isInSafeMode() )
|
|
@@ -2413,7 +2567,6 @@ class FSNamesystem implements FSConstants {
|
|
|
|
|
|
synchronized (neededReplications) {
|
|
|
Object results[] = null;
|
|
|
- int scheduledXfers = 0;
|
|
|
|
|
|
if (neededReplications.size() > 0) {
|
|
|
//
|
|
@@ -2426,13 +2579,9 @@ class FSNamesystem implements FSConstants {
|
|
|
List<DatanodeDescriptor[]> replicateTargetSets;
|
|
|
replicateTargetSets = new ArrayList<DatanodeDescriptor[]>();
|
|
|
for (Iterator<Block> it = neededReplications.iterator(); it.hasNext();) {
|
|
|
- //
|
|
|
- // We can only reply with 'maxXfers' or fewer blocks
|
|
|
- //
|
|
|
- if (scheduledXfers >= this.maxReplicationStreams - xmitsInProgress) {
|
|
|
+ if (needed <= 0) {
|
|
|
break;
|
|
|
}
|
|
|
-
|
|
|
Block block = it.next();
|
|
|
long blockSize = block.getNumBytes();
|
|
|
FSDirectory.INode fileINode = dir.getFileByBlock(block);
|
|
@@ -2453,7 +2602,7 @@ class FSNamesystem implements FSConstants {
|
|
|
int numCurrentReplica = nodes.size();
|
|
|
DatanodeDescriptor targets[] = replicator.chooseTarget(
|
|
|
Math.min( fileINode.getReplication() - numCurrentReplica,
|
|
|
- this.maxReplicationStreams - xmitsInProgress),
|
|
|
+ needed),
|
|
|
datanodeMap.get(srcNode.getStorageID()),
|
|
|
nodes, null, blockSize);
|
|
|
if (targets.length > 0) {
|
|
@@ -2461,7 +2610,7 @@ class FSNamesystem implements FSConstants {
|
|
|
replicateBlocks.add(block);
|
|
|
numCurrentReplicas.add(new Integer(numCurrentReplica));
|
|
|
replicateTargetSets.add(targets);
|
|
|
- scheduledXfers += targets.length;
|
|
|
+ needed -= targets.length;
|
|
|
}
|
|
|
}
|
|
|
}
|