|
@@ -49,37 +49,47 @@ import com.google.common.base.Preconditions;
|
|
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
|
|
|
/**
|
|
|
- * Manages datanode decommissioning. A background monitor thread
|
|
|
- * periodically checks the status of datanodes that are in-progress of
|
|
|
- * decommissioning.
|
|
|
+ * Manages decommissioning and maintenance state for DataNodes. A background
|
|
|
+ * monitor thread periodically checks the status of DataNodes that are
|
|
|
+ * decommissioning or entering maintenance state.
|
|
|
* <p/>
|
|
|
- * A datanode can be decommissioned in a few situations:
|
|
|
+ * A DataNode can be decommissioned in a few situations:
|
|
|
* <ul>
|
|
|
* <li>If a DN is dead, it is decommissioned immediately.</li>
|
|
|
- * <li>If a DN is alive, it is decommissioned after all of its blocks
|
|
|
- * are sufficiently replicated. Merely under-replicated blocks do not
|
|
|
- * block decommissioning as long as they are above a replication
|
|
|
+ * <li>If a DN is alive, it is decommissioned after all of its blocks
|
|
|
+ * are sufficiently replicated. Merely under-replicated blocks do not
|
|
|
+ * block decommissioning as long as they are above a replication
|
|
|
* threshold.</li>
|
|
|
* </ul>
|
|
|
- * In the second case, the datanode transitions to a
|
|
|
- * decommission-in-progress state and is tracked by the monitor thread. The
|
|
|
- * monitor periodically scans through the list of insufficiently replicated
|
|
|
- * blocks on these datanodes to
|
|
|
- * determine if they can be decommissioned. The monitor also prunes this list
|
|
|
- * as blocks become replicated, so monitor scans will become more efficient
|
|
|
+ * In the second case, the DataNode transitions to a DECOMMISSION_INPROGRESS
|
|
|
+ * state and is tracked by the monitor thread. The monitor periodically scans
|
|
|
+ * through the list of insufficiently replicated blocks on these DataNodes to
|
|
|
+ * determine if they can be DECOMMISSIONED. The monitor also prunes this list
|
|
|
+ * as blocks become replicated, so monitor scans will become more efficient
|
|
|
* over time.
|
|
|
* <p/>
|
|
|
- * Decommission-in-progress nodes that become dead do not progress to
|
|
|
- * decommissioned until they become live again. This prevents potential
|
|
|
+ * DECOMMISSION_INPROGRESS nodes that become dead do not progress to
|
|
|
+ * DECOMMISSIONED until they become live again. This prevents potential
|
|
|
* durability loss for singly-replicated blocks (see HDFS-6791).
|
|
|
* <p/>
|
|
|
+ * DataNodes can also be put under maintenance state for any short duration
|
|
|
+ * maintenance operations. Unlike decommissioning, blocks are not always
|
|
|
+ * re-replicated for the DataNodes to enter maintenance state. When the
|
|
|
+ * blocks are replicated at least dfs.namenode.maintenance.replication.min,
|
|
|
+ * DataNodes transition to IN_MAINTENANCE state. Otherwise, just like
|
|
|
+ * decommissioning, DataNodes transition to ENTERING_MAINTENANCE state and
|
|
|
+ * wait for the blocks to be sufficiently replicated and then transition to
|
|
|
+ * IN_MAINTENANCE state. The block replication factor is relaxed for a maximum
|
|
|
+ * of maintenance expiry time. When DataNodes don't transition or join the
|
|
|
+ * cluster back by expiry time, blocks are re-replicated just as in
|
|
|
+ * decommissioning case as to avoid read or write performance degradation.
|
|
|
+ * <p/>
|
|
|
* This class depends on the FSNamesystem lock for synchronization.
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
-public class DecommissionManager {
|
|
|
- private static final Logger LOG = LoggerFactory.getLogger(DecommissionManager
|
|
|
- .class);
|
|
|
-
|
|
|
+public class DatanodeAdminManager {
|
|
|
+ private static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(DatanodeAdminManager.class);
|
|
|
private final Namesystem namesystem;
|
|
|
private final BlockManager blockManager;
|
|
|
private final HeartbeatManager hbManager;
|
|
@@ -97,7 +107,7 @@ public class DecommissionManager {
|
|
|
* the node from being marked as decommissioned. During a monitor tick, this
|
|
|
* list is pruned as blocks becomes replicated.
|
|
|
* <p/>
|
|
|
- * Note also that the reference to the list of under-replicated blocks
|
|
|
+ * Note also that the reference to the list of under-replicated blocks
|
|
|
* will be null on initial add
|
|
|
* <p/>
|
|
|
* However, this map can become out-of-date since it is not updated by block
|
|
@@ -113,24 +123,23 @@ public class DecommissionManager {
|
|
|
* outOfServiceNodeBlocks. Additional nodes wait in pendingNodes.
|
|
|
*/
|
|
|
private final Queue<DatanodeDescriptor> pendingNodes;
|
|
|
-
|
|
|
private Monitor monitor = null;
|
|
|
|
|
|
- DecommissionManager(final Namesystem namesystem,
|
|
|
+ DatanodeAdminManager(final Namesystem namesystem,
|
|
|
final BlockManager blockManager, final HeartbeatManager hbManager) {
|
|
|
this.namesystem = namesystem;
|
|
|
this.blockManager = blockManager;
|
|
|
this.hbManager = hbManager;
|
|
|
|
|
|
executor = Executors.newScheduledThreadPool(1,
|
|
|
- new ThreadFactoryBuilder().setNameFormat("DecommissionMonitor-%d")
|
|
|
+ new ThreadFactoryBuilder().setNameFormat("DatanodeAdminMonitor-%d")
|
|
|
.setDaemon(true).build());
|
|
|
outOfServiceNodeBlocks = new TreeMap<>();
|
|
|
pendingNodes = new LinkedList<>();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Start the decommission monitor thread.
|
|
|
+ * Start the DataNode admin monitor thread.
|
|
|
* @param conf
|
|
|
*/
|
|
|
void activate(Configuration conf) {
|
|
@@ -151,7 +160,7 @@ public class DecommissionManager {
|
|
|
if (strNodes != null) {
|
|
|
LOG.warn("Deprecated configuration key {} will be ignored.",
|
|
|
deprecatedKey);
|
|
|
- LOG.warn("Please update your configuration to use {} instead.",
|
|
|
+ LOG.warn("Please update your configuration to use {} instead.",
|
|
|
DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
|
|
|
}
|
|
|
|
|
@@ -161,7 +170,8 @@ public class DecommissionManager {
|
|
|
|
|
|
final int maxConcurrentTrackedNodes = conf.getInt(
|
|
|
DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES,
|
|
|
- DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES_DEFAULT);
|
|
|
+ DFSConfigKeys
|
|
|
+ .DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES_DEFAULT);
|
|
|
checkArgument(maxConcurrentTrackedNodes >= 0, "Cannot set a negative " +
|
|
|
"value for "
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES);
|
|
@@ -170,14 +180,14 @@ public class DecommissionManager {
|
|
|
executor.scheduleAtFixedRate(monitor, intervalSecs, intervalSecs,
|
|
|
TimeUnit.SECONDS);
|
|
|
|
|
|
- LOG.debug("Activating DecommissionManager with interval {} seconds, " +
|
|
|
+ LOG.debug("Activating DatanodeAdminManager with interval {} seconds, " +
|
|
|
"{} max blocks per interval, " +
|
|
|
"{} max concurrently tracked nodes.", intervalSecs,
|
|
|
blocksPerInterval, maxConcurrentTrackedNodes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Stop the decommission monitor thread, waiting briefly for it to terminate.
|
|
|
+ * Stop the admin monitor thread, waiting briefly for it to terminate.
|
|
|
*/
|
|
|
void close() {
|
|
|
executor.shutdownNow();
|
|
@@ -187,7 +197,7 @@ public class DecommissionManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Start decommissioning the specified datanode.
|
|
|
+ * Start decommissioning the specified datanode.
|
|
|
* @param node
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
@@ -211,7 +221,7 @@ public class DecommissionManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Stop decommissioning the specified datanode.
|
|
|
+ * Stop decommissioning the specified datanode.
|
|
|
* @param node
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
@@ -224,7 +234,7 @@ public class DecommissionManager {
|
|
|
if (node.isAlive()) {
|
|
|
blockManager.processExtraRedundancyBlocksOnInService(node);
|
|
|
}
|
|
|
- // Remove from tracking in DecommissionManager
|
|
|
+ // Remove from tracking in DatanodeAdminManager
|
|
|
pendingNodes.remove(node);
|
|
|
outOfServiceNodeBlocks.remove(node);
|
|
|
} else {
|
|
@@ -303,7 +313,7 @@ public class DecommissionManager {
|
|
|
blockManager.processExtraRedundancyBlocksOnInService(node);
|
|
|
}
|
|
|
|
|
|
- // Remove from tracking in DecommissionManager
|
|
|
+ // Remove from tracking in DatanodeAdminManager
|
|
|
pendingNodes.remove(node);
|
|
|
outOfServiceNodeBlocks.remove(node);
|
|
|
} else {
|
|
@@ -324,8 +334,9 @@ public class DecommissionManager {
|
|
|
|
|
|
/**
|
|
|
* Checks whether a block is sufficiently replicated/stored for
|
|
|
- * decommissioning. For replicated blocks or striped blocks, full-strength
|
|
|
- * replication or storage is not always necessary, hence "sufficient".
|
|
|
+ * DECOMMISSION_INPROGRESS or ENTERING_MAINTENANCE datanodes. For replicated
|
|
|
+ * blocks or striped blocks, full-strength replication or storage is not
|
|
|
+ * always necessary, hence "sufficient".
|
|
|
* @return true if sufficient, else false.
|
|
|
*/
|
|
|
private boolean isSufficient(BlockInfo block, BlockCollection bc,
|
|
@@ -416,9 +427,10 @@ public class DecommissionManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Checks to see if DNs have finished decommissioning.
|
|
|
+ * Checks to see if datanodes have finished DECOMMISSION_INPROGRESS or
|
|
|
+ * ENTERING_MAINTENANCE state.
|
|
|
* <p/>
|
|
|
- * Since this is done while holding the namesystem lock,
|
|
|
+ * Since this is done while holding the namesystem lock,
|
|
|
* the amount of work per monitor tick is limited.
|
|
|
*/
|
|
|
private class Monitor implements Runnable {
|
|
@@ -440,15 +452,15 @@ public class DecommissionManager {
|
|
|
*/
|
|
|
private int numBlocksCheckedPerLock = 0;
|
|
|
/**
|
|
|
- * The number of nodes that have been checked on this tick. Used for
|
|
|
+ * The number of nodes that have been checked on this tick. Used for
|
|
|
* statistics.
|
|
|
*/
|
|
|
private int numNodesChecked = 0;
|
|
|
/**
|
|
|
- * The last datanode in outOfServiceNodeBlocks that we've processed
|
|
|
+ * The last datanode in outOfServiceNodeBlocks that we've processed.
|
|
|
*/
|
|
|
- private DatanodeDescriptor iterkey = new DatanodeDescriptor(new
|
|
|
- DatanodeID("", "", "", 0, 0, 0, 0));
|
|
|
+ private DatanodeDescriptor iterkey = new DatanodeDescriptor(
|
|
|
+ new DatanodeID("", "", "", 0, 0, 0, 0));
|
|
|
|
|
|
Monitor(int numBlocksPerCheck, int maxConcurrentTrackedNodes) {
|
|
|
this.numBlocksPerCheck = numBlocksPerCheck;
|
|
@@ -463,8 +475,8 @@ public class DecommissionManager {
|
|
|
@Override
|
|
|
public void run() {
|
|
|
if (!namesystem.isRunning()) {
|
|
|
- LOG.info("Namesystem is not running, skipping decommissioning checks"
|
|
|
- + ".");
|
|
|
+ LOG.info("Namesystem is not running, skipping " +
|
|
|
+ "decommissioning/maintenance checks.");
|
|
|
return;
|
|
|
}
|
|
|
// Reset the checked count at beginning of each iteration
|
|
@@ -486,7 +498,7 @@ public class DecommissionManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Pop datanodes off the pending list and into decomNodeBlocks,
|
|
|
+ * Pop datanodes off the pending list and into decomNodeBlocks,
|
|
|
* subject to the maxConcurrentTrackedNodes limit.
|
|
|
*/
|
|
|
private void processPendingNodes() {
|
|
@@ -522,8 +534,8 @@ public class DecommissionManager {
|
|
|
continue;
|
|
|
}
|
|
|
if (blocks == null) {
|
|
|
- // This is a newly added datanode, run through its list to schedule
|
|
|
- // under-replicated blocks for replication and collect the blocks
|
|
|
+ // This is a newly added datanode, run through its list to schedule
|
|
|
+ // under-replicated blocks for replication and collect the blocks
|
|
|
// that are insufficiently replicated for further tracking
|
|
|
LOG.debug("Newly-added node {}, doing full scan to find " +
|
|
|
"insufficiently-replicated blocks.", dn);
|
|
@@ -531,26 +543,27 @@ public class DecommissionManager {
|
|
|
outOfServiceNodeBlocks.put(dn, blocks);
|
|
|
fullScan = true;
|
|
|
} else {
|
|
|
- // This is a known datanode, check if its # of insufficiently
|
|
|
- // replicated blocks has dropped to zero and if it can be decommed
|
|
|
+ // This is a known datanode, check if its # of insufficiently
|
|
|
+ // replicated blocks has dropped to zero and if it can move
|
|
|
+ // to the next state.
|
|
|
LOG.debug("Processing {} node {}", dn.getAdminState(), dn);
|
|
|
pruneReliableBlocks(dn, blocks);
|
|
|
}
|
|
|
if (blocks.size() == 0) {
|
|
|
if (!fullScan) {
|
|
|
- // If we didn't just do a full scan, need to re-check with the
|
|
|
+ // If we didn't just do a full scan, need to re-check with the
|
|
|
// full block map.
|
|
|
//
|
|
|
- // We've replicated all the known insufficiently replicated
|
|
|
- // blocks. Re-check with the full block map before finally
|
|
|
- // marking the datanode as decommissioned
|
|
|
+ // We've replicated all the known insufficiently replicated
|
|
|
+ // blocks. Re-check with the full block map before finally
|
|
|
+ // marking the datanode as DECOMMISSIONED or IN_MAINTENANCE.
|
|
|
LOG.debug("Node {} has finished replicating current set of "
|
|
|
+ "blocks, checking with the full block map.", dn);
|
|
|
blocks = handleInsufficientlyStored(dn);
|
|
|
outOfServiceNodeBlocks.put(dn, blocks);
|
|
|
}
|
|
|
- // If the full scan is clean AND the node liveness is okay,
|
|
|
- // we can finally mark as decommissioned.
|
|
|
+ // If the full scan is clean AND the node liveness is okay,
|
|
|
+ // we can finally mark as DECOMMISSIONED or IN_MAINTENANCE.
|
|
|
final boolean isHealthy =
|
|
|
blockManager.isNodeHealthyForDecommissionOrMaintenance(dn);
|
|
|
if (blocks.size() == 0 && isHealthy) {
|
|
@@ -580,7 +593,7 @@ public class DecommissionManager {
|
|
|
}
|
|
|
iterkey = dn;
|
|
|
}
|
|
|
- // Remove the datanodes that are decommissioned or in service after
|
|
|
+ // Remove the datanodes that are DECOMMISSIONED or in service after
|
|
|
// maintenance expiration.
|
|
|
for (DatanodeDescriptor dn : toRemove) {
|
|
|
Preconditions.checkState(dn.isDecommissioned() || dn.isInService(),
|
|
@@ -598,9 +611,9 @@ public class DecommissionManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Returns a list of blocks on a datanode that are insufficiently replicated
|
|
|
- * or require recovery, i.e. requiring recovery and should prevent
|
|
|
- * decommission.
|
|
|
+ * Returns a list of blocks on a datanode that are insufficiently
|
|
|
+ * replicated or require recovery, i.e. requiring recovery and
|
|
|
+ * should prevent decommission or maintenance.
|
|
|
* <p/>
|
|
|
* As part of this, it also schedules replication/recovery work.
|
|
|
*
|
|
@@ -615,9 +628,10 @@ public class DecommissionManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Used while checking if decommission-in-progress datanodes can be marked
|
|
|
- * as decommissioned. Combines shared logic of
|
|
|
- * pruneReliableBlocks and handleInsufficientlyStored.
|
|
|
+ * Used while checking if DECOMMISSION_INPROGRESS datanodes can be
|
|
|
+ * marked as DECOMMISSIONED or ENTERING_MAINTENANCE datanodes can be
|
|
|
+ * marked as IN_MAINTENANCE. Combines shared logic of pruneReliableBlocks
|
|
|
+ * and handleInsufficientlyStored.
|
|
|
*
|
|
|
* @param datanode Datanode
|
|
|
* @param it Iterator over the blocks on the
|
|
@@ -652,7 +666,7 @@ public class DecommissionManager {
|
|
|
// configured per-iteration-limit.
|
|
|
namesystem.writeUnlock();
|
|
|
try {
|
|
|
- LOG.debug("Yielded lock during decommission check");
|
|
|
+ LOG.debug("Yielded lock during decommission/maintenance check");
|
|
|
Thread.sleep(0, 500);
|
|
|
} catch (InterruptedException ignored) {
|
|
|
return;
|
|
@@ -682,8 +696,8 @@ public class DecommissionManager {
|
|
|
final NumberReplicas num = blockManager.countNodes(block);
|
|
|
final int liveReplicas = num.liveReplicas();
|
|
|
|
|
|
- // Schedule low redundancy blocks for reconstruction if not already
|
|
|
- // pending
|
|
|
+ // Schedule low redundancy blocks for reconstruction
|
|
|
+ // if not already pending.
|
|
|
boolean isDecommission = datanode.isDecommissionInProgress();
|
|
|
boolean neededReconstruction = isDecommission ?
|
|
|
blockManager.isNeededReconstruction(block, num) :
|
|
@@ -701,7 +715,8 @@ public class DecommissionManager {
|
|
|
}
|
|
|
|
|
|
// Even if the block is without sufficient redundancy,
|
|
|
- // it doesn't block decommission if has sufficient redundancy
|
|
|
+ // it might not block decommission/maintenance if it
|
|
|
+ // has sufficient redundancy.
|
|
|
if (isSufficient(block, bc, num, isDecommission)) {
|
|
|
if (pruneReliableBlocks) {
|
|
|
it.remove();
|