|
@@ -365,22 +365,27 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
// precision of access times.
|
|
// precision of access times.
|
|
private long accessTimePrecision = 0;
|
|
private long accessTimePrecision = 0;
|
|
private String nameNodeHostName;
|
|
private String nameNodeHostName;
|
|
-
|
|
|
|
- /** Whether or not to check stale DataNodes for read/write */
|
|
|
|
- private boolean checkForStaleDataNodes;
|
|
|
|
/** The interval for judging stale DataNodes for read/write */
|
|
/** The interval for judging stale DataNodes for read/write */
|
|
private long staleInterval;
|
|
private long staleInterval;
|
|
- /** Whether or not to avoid using stale DataNodes for writing */
|
|
|
|
- private volatile boolean avoidStaleDataNodesForWrite;
|
|
|
|
- private boolean initialAvoidWriteStaleNodes;
|
|
|
|
- /** The number of stale DataNodes */
|
|
|
|
- private volatile int numStaleNodes = 0;
|
|
|
|
|
|
+ /** Whether or not to avoid using stale DataNodes for reading */
|
|
|
|
+ private boolean avoidStaleDataNodesForRead;
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Whether or not to avoid using stale DataNodes for writing.
|
|
|
|
+ * Note that, even if this is configured, the policy may be
|
|
|
|
+ * temporarily disabled when a high percentage of the nodes
|
|
|
|
+ * are marked as stale.
|
|
|
|
+ */
|
|
|
|
+ private boolean avoidStaleDataNodesForWrite;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* When the ratio of stale datanodes reaches this number, stop avoiding
|
|
* When the ratio of stale datanodes reaches this number, stop avoiding
|
|
* writing to stale datanodes, i.e., continue using stale nodes for writing.
|
|
* writing to stale datanodes, i.e., continue using stale nodes for writing.
|
|
*/
|
|
*/
|
|
private float ratioUseStaleDataNodesForWrite;
|
|
private float ratioUseStaleDataNodesForWrite;
|
|
-
|
|
|
|
|
|
+ /** The number of stale DataNodes */
|
|
|
|
+ private volatile int numStaleNodes = 0;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* FSNamesystem constructor.
|
|
* FSNamesystem constructor.
|
|
*/
|
|
*/
|
|
@@ -593,15 +598,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
+ " min(s)");
|
|
+ " min(s)");
|
|
|
|
|
|
// set the value of stale interval based on configuration
|
|
// set the value of stale interval based on configuration
|
|
- checkForStaleDataNodes = conf.getBoolean(
|
|
|
|
- DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY,
|
|
|
|
- DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_DEFAULT);
|
|
|
|
- staleInterval = getStaleIntervalFromConf(conf, heartbeatExpireInterval);
|
|
|
|
- avoidStaleDataNodesForWrite = getAvoidStaleForWriteFromConf(conf,
|
|
|
|
- checkForStaleDataNodes);
|
|
|
|
- initialAvoidWriteStaleNodes = avoidStaleDataNodesForWrite;
|
|
|
|
- ratioUseStaleDataNodesForWrite =
|
|
|
|
|
|
+ this.avoidStaleDataNodesForRead = conf.getBoolean(
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY,
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT);
|
|
|
|
+ this.avoidStaleDataNodesForWrite = conf.getBoolean(
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY,
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_DEFAULT);
|
|
|
|
+ this.staleInterval = getStaleIntervalFromConf(conf, heartbeatExpireInterval);
|
|
|
|
+ this.ratioUseStaleDataNodesForWrite =
|
|
getRatioUseStaleNodesForWriteFromConf(conf);
|
|
getRatioUseStaleNodesForWriteFromConf(conf);
|
|
|
|
+
|
|
|
|
+ if (avoidStaleDataNodesForWrite && staleInterval < heartbeatRecheckInterval) {
|
|
|
|
+ this.heartbeatRecheckInterval = staleInterval;
|
|
|
|
+ LOG.info("Setting heartbeat recheck interval to " + staleInterval
|
|
|
|
+ + " since " + DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY
|
|
|
|
+ + " is less than "
|
|
|
|
+ + DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
private static float getRatioUseStaleNodesForWriteFromConf(Configuration conf) {
|
|
private static float getRatioUseStaleNodesForWriteFromConf(Configuration conf) {
|
|
@@ -658,22 +671,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
return staleInterval;
|
|
return staleInterval;
|
|
}
|
|
}
|
|
|
|
|
|
- static boolean getAvoidStaleForWriteFromConf(Configuration conf,
|
|
|
|
- boolean checkForStale) {
|
|
|
|
- boolean avoid = conf.getBoolean(
|
|
|
|
- DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY,
|
|
|
|
- DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_DEFAULT);
|
|
|
|
- boolean avoidStaleDataNodesForWrite = checkForStale && avoid;
|
|
|
|
- if (!checkForStale && avoid) {
|
|
|
|
- LOG.warn("Cannot set "
|
|
|
|
- + DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY
|
|
|
|
- + " as false while setting "
|
|
|
|
- + DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY
|
|
|
|
- + " as true");
|
|
|
|
- }
|
|
|
|
- return avoidStaleDataNodesForWrite;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Return the default path permission when upgrading from releases with no
|
|
* Return the default path permission when upgrading from releases with no
|
|
* permissions (<=0.15) to releases with permissions (>=0.16)
|
|
* permissions (<=0.15) to releases with permissions (>=0.16)
|
|
@@ -1047,13 +1044,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
}
|
|
}
|
|
|
|
|
|
DFSUtil.StaleComparator comparator = null;
|
|
DFSUtil.StaleComparator comparator = null;
|
|
- if (checkForStaleDataNodes) {
|
|
|
|
|
|
+ if (avoidStaleDataNodesForRead) {
|
|
comparator = new DFSUtil.StaleComparator(staleInterval);
|
|
comparator = new DFSUtil.StaleComparator(staleInterval);
|
|
}
|
|
}
|
|
// Note: the last block is also included and sorted
|
|
// Note: the last block is also included and sorted
|
|
for (LocatedBlock b : blocks.getLocatedBlocks()) {
|
|
for (LocatedBlock b : blocks.getLocatedBlocks()) {
|
|
clusterMap.pseudoSortByDistance(client, b.getLocations());
|
|
clusterMap.pseudoSortByDistance(client, b.getLocations());
|
|
- if (checkForStaleDataNodes) {
|
|
|
|
|
|
+ if (avoidStaleDataNodesForRead) {
|
|
Arrays.sort(b.getLocations(), comparator);
|
|
Arrays.sort(b.getLocations(), comparator);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -3753,29 +3750,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
if (dead == null && isDatanodeDead(nodeInfo)) {
|
|
if (dead == null && isDatanodeDead(nodeInfo)) {
|
|
foundDead = true;
|
|
foundDead = true;
|
|
dead = nodeInfo;
|
|
dead = nodeInfo;
|
|
- if (!this.checkForStaleDataNodes) {
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
}
|
|
}
|
|
- if (this.checkForStaleDataNodes
|
|
|
|
- && nodeInfo.isStale(this.staleInterval)) {
|
|
|
|
|
|
+ if (nodeInfo.isStale(this.staleInterval)) {
|
|
numOfStaleNodes++;
|
|
numOfStaleNodes++;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- // Change whether to avoid using stale datanodes for writing
|
|
|
|
- // based on proportion of stale datanodes
|
|
|
|
- if (this.checkForStaleDataNodes) {
|
|
|
|
- this.numStaleNodes = numOfStaleNodes;
|
|
|
|
- if (numOfStaleNodes > heartbeats.size()
|
|
|
|
- * this.ratioUseStaleDataNodesForWrite) {
|
|
|
|
- this.avoidStaleDataNodesForWrite = false;
|
|
|
|
- } else {
|
|
|
|
- if (this.initialAvoidWriteStaleNodes) {
|
|
|
|
- this.avoidStaleDataNodesForWrite = true;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ setNumStaleNodes(numOfStaleNodes);
|
|
}
|
|
}
|
|
|
|
|
|
// acquire the fsnamesystem lock, and then remove the dead node.
|
|
// acquire the fsnamesystem lock, and then remove the dead node.
|
|
@@ -6409,41 +6390,38 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
|
|
public String toString() {
|
|
public String toString() {
|
|
return getClass().getSimpleName() + ": " + host2DataNodeMap;
|
|
return getClass().getSimpleName() + ": " + host2DataNodeMap;
|
|
}
|
|
}
|
|
-
|
|
|
|
- /**
|
|
|
|
- * @return Return the current number of stale DataNodes (detected by
|
|
|
|
- * HeartbeatMonitor).
|
|
|
|
- */
|
|
|
|
- public int getNumStaleNodes() {
|
|
|
|
- return this.numStaleNodes;
|
|
|
|
- }
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
- * @return whether or not to avoid writing to stale datanodes
|
|
|
|
|
|
+ * Set the number of stale DataNodes, based on DataNodes' heartbeats.
|
|
|
|
+ *
|
|
|
|
+ * @param numStaleNodes
|
|
|
|
+ * The number of stale DataNodes to be set.
|
|
*/
|
|
*/
|
|
- @Override // FSClusterStats
|
|
|
|
- public boolean isAvoidingStaleDataNodesForWrite() {
|
|
|
|
- return avoidStaleDataNodesForWrite;
|
|
|
|
|
|
+ void setNumStaleNodes(int numStaleNodes) {
|
|
|
|
+ this.numStaleNodes = numStaleNodes;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * @return The interval used to judge whether or not a DataNode is stale
|
|
|
|
|
|
+ * @return Return the current number of stale DataNodes (detected by
|
|
|
|
+ * HeartbeatMonitor).
|
|
*/
|
|
*/
|
|
- public long getStaleInterval() {
|
|
|
|
- return this.staleInterval;
|
|
|
|
|
|
+ public int getNumStaleNodes() {
|
|
|
|
+ return this.numStaleNodes;
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Set the value of {@link DatanodeManager#avoidStaleDataNodesForWrite}. The
|
|
|
|
- * HeartbeatManager disable avoidStaleDataNodesForWrite when more than half of
|
|
|
|
- * the DataNodes are marked as stale.
|
|
|
|
|
|
+ /*
|
|
|
|
+ * Whether stale datanodes should be avoided as targets on the write path.
|
|
|
|
+ * The result of this function may change if the number of stale datanodes
|
|
|
|
+ * eclipses a configurable threshold.
|
|
*
|
|
*
|
|
- * @param avoidStaleDataNodesForWrite
|
|
|
|
- * The value to set to
|
|
|
|
- * {@link DatanodeManager#avoidStaleDataNodesForWrite}
|
|
|
|
- */
|
|
|
|
- void setAvoidStaleDataNodesForWrite(boolean avoidStaleDataNodesForWrite) {
|
|
|
|
- this.avoidStaleDataNodesForWrite = avoidStaleDataNodesForWrite;
|
|
|
|
|
|
+ * @return whether stale datanodes should be avoided on the write path
|
|
|
|
+ */
|
|
|
|
+ public boolean shouldAvoidStaleDataNodesForWrite() {
|
|
|
|
+ // If # stale exceeds maximum staleness ratio, disable stale
|
|
|
|
+ // datanode avoidance on the write path
|
|
|
|
+ return avoidStaleDataNodesForWrite &&
|
|
|
|
+ (numStaleNodes <= heartbeats.size()
|
|
|
|
+ * ratioUseStaleDataNodesForWrite);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|