|
@@ -27,22 +27,22 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Queue;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
-
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
|
|
|
import org.apache.hadoop.hdfs.util.LightWeightHashSet;
|
|
|
import org.apache.hadoop.util.IntrusiveCollection;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+
|
|
|
/**
|
|
|
* This class extends the DatanodeInfo class with ephemeral information (eg
|
|
|
* health, capacity, what blocks are associated with the Datanode) that is
|
|
@@ -192,6 +192,15 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|
|
/** A set of blocks to be invalidated by this datanode */
|
|
|
private LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
|
|
|
|
|
|
+ /* Variables for maintaining number of blocks scheduled to be written to
|
|
|
+ * this storage. This count is approximate and might be slightly bigger
|
|
|
+ * in case of errors (e.g. datanode does not report if an error occurs
|
|
|
+ * while writing the block).
|
|
|
+ */
|
|
|
+ private int currApproxBlocksScheduled = 0;
|
|
|
+ private int prevApproxBlocksScheduled = 0;
|
|
|
+ private long lastBlocksScheduledRollTime = 0;
|
|
|
+ private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
|
|
|
private int volumeFailures = 0;
|
|
|
|
|
|
/**
|
|
@@ -342,7 +351,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|
|
for (StorageReport report : reports) {
|
|
|
DatanodeStorageInfo storage = storageMap.get(report.getStorageID());
|
|
|
if (storage != null) {
|
|
|
- storage.receivedHeartbeat(report, getLastUpdate());
|
|
|
+ storage.receivedHeartbeat(report);
|
|
|
totalCapacity += report.getCapacity();
|
|
|
totalRemaining += report.getRemaining();
|
|
|
totalBlockPoolUsed += report.getBlockPoolUsed();
|
|
@@ -354,6 +363,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|
|
LOG.warn("Unrecognized storage ID " + report.getStorageID());
|
|
|
}
|
|
|
}
|
|
|
+ rollBlocksScheduled(getLastUpdate());
|
|
|
|
|
|
// Update total metrics for the node.
|
|
|
setCapacity(totalCapacity);
|
|
@@ -481,11 +491,31 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|
|
* to this datanode.
|
|
|
*/
|
|
|
public int getBlocksScheduled() {
|
|
|
- int n = 0;
|
|
|
- for(DatanodeStorageInfo storage : getStorageInfos()) {
|
|
|
- n += storage.getBlocksScheduled();
|
|
|
+ return currApproxBlocksScheduled + prevApproxBlocksScheduled;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Increment the number of blocks scheduled. */
|
|
|
+ void incrementBlocksScheduled() {
|
|
|
+ currApproxBlocksScheduled++;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Decrement the number of blocks scheduled. */
|
|
|
+ void decrementBlocksScheduled() {
|
|
|
+ if (prevApproxBlocksScheduled > 0) {
|
|
|
+ prevApproxBlocksScheduled--;
|
|
|
+ } else if (currApproxBlocksScheduled > 0) {
|
|
|
+ currApproxBlocksScheduled--;
|
|
|
+ }
|
|
|
+ // its ok if both counters are zero.
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Adjusts curr and prev number of blocks scheduled every few minutes. */
|
|
|
+ private void rollBlocksScheduled(long now) {
|
|
|
+ if (now - lastBlocksScheduledRollTime > BLOCKS_SCHEDULED_ROLL_INTERVAL) {
|
|
|
+ prevApproxBlocksScheduled = currApproxBlocksScheduled;
|
|
|
+ currApproxBlocksScheduled = 0;
|
|
|
+ lastBlocksScheduledRollTime = now;
|
|
|
}
|
|
|
- return n;
|
|
|
}
|
|
|
|
|
|
@Override
|