|
@@ -18,18 +18,13 @@
|
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.HashSet;
|
|
|
-import java.util.IdentityHashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
-import java.util.Set;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
-import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
@@ -57,10 +52,10 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
* and removes them from the list.
|
|
|
* It is synchronized by the heartbeat manager lock.
|
|
|
*/
|
|
|
- private final List<DatanodeDescriptor> datanodes = new ArrayList<DatanodeDescriptor>();
|
|
|
+ private final List<DatanodeDescriptor> datanodes = new ArrayList<>();
|
|
|
|
|
|
/** Statistics, which are synchronized by the heartbeat manager lock. */
|
|
|
- private final Stats stats = new Stats();
|
|
|
+ private final DatanodeStats stats = new DatanodeStats();
|
|
|
|
|
|
/** The time period to check for expired datanodes */
|
|
|
private final long heartbeatRecheckInterval;
|
|
@@ -96,7 +91,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void activate(Configuration conf) {
|
|
|
+ void activate() {
|
|
|
heartbeatThread.start();
|
|
|
}
|
|
|
|
|
@@ -105,7 +100,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
try {
|
|
|
// This will no effect if the thread hasn't yet been started.
|
|
|
heartbeatThread.join(3000);
|
|
|
- } catch (InterruptedException e) {
|
|
|
+ } catch (InterruptedException ignored) {
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -114,74 +109,69 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized long getCapacityTotal() {
|
|
|
- return stats.capacityTotal;
|
|
|
+ public long getCapacityTotal() {
|
|
|
+ return stats.getCapacityTotal();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized long getCapacityUsed() {
|
|
|
- return stats.capacityUsed;
|
|
|
+ public long getCapacityUsed() {
|
|
|
+ return stats.getCapacityUsed();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized float getCapacityUsedPercent() {
|
|
|
- return DFSUtilClient.getPercentUsed(stats.capacityUsed, stats.capacityTotal);
|
|
|
+ public float getCapacityUsedPercent() {
|
|
|
+ return stats.getCapacityUsedPercent();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized long getCapacityRemaining() {
|
|
|
- return stats.capacityRemaining;
|
|
|
+ public long getCapacityRemaining() {
|
|
|
+ return stats.getCapacityRemaining();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized float getCapacityRemainingPercent() {
|
|
|
- return DFSUtilClient.getPercentRemaining(stats.capacityRemaining,
|
|
|
- stats.capacityTotal);
|
|
|
+ public float getCapacityRemainingPercent() {
|
|
|
+ return stats.getCapacityRemainingPercent();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized long getBlockPoolUsed() {
|
|
|
- return stats.blockPoolUsed;
|
|
|
+ public long getBlockPoolUsed() {
|
|
|
+ return stats.getBlockPoolUsed();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized float getPercentBlockPoolUsed() {
|
|
|
- return DFSUtilClient.getPercentUsed(stats.blockPoolUsed,
|
|
|
- stats.capacityTotal);
|
|
|
+ public float getPercentBlockPoolUsed() {
|
|
|
+ return stats.getPercentBlockPoolUsed();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized long getCapacityUsedNonDFS() {
|
|
|
- final long nonDFSUsed = stats.capacityTotal
|
|
|
- - stats.capacityRemaining - stats.capacityUsed;
|
|
|
- return nonDFSUsed < 0L? 0L : nonDFSUsed;
|
|
|
+ public long getCapacityUsedNonDFS() {
|
|
|
+ return stats.getCapacityUsedNonDFS();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized int getXceiverCount() {
|
|
|
- return stats.xceiverCount;
|
|
|
+ public int getXceiverCount() {
|
|
|
+ return stats.getXceiverCount();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized int getInServiceXceiverCount() {
|
|
|
- return stats.nodesInServiceXceiverCount;
|
|
|
+ public int getInServiceXceiverCount() {
|
|
|
+ return stats.getNodesInServiceXceiverCount();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized int getNumDatanodesInService() {
|
|
|
- return stats.nodesInService;
|
|
|
+ public int getNumDatanodesInService() {
|
|
|
+ return stats.getNodesInService();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized long getCacheCapacity() {
|
|
|
- return stats.cacheCapacity;
|
|
|
+ public long getCacheCapacity() {
|
|
|
+ return stats.getCacheCapacity();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized long getCacheUsed() {
|
|
|
- return stats.cacheUsed;
|
|
|
+ public long getCacheUsed() {
|
|
|
+ return stats.getCacheUsed();
|
|
|
}
|
|
|
-
|
|
|
|
|
|
@Override
|
|
|
public synchronized long[] getStats() {
|
|
@@ -195,17 +185,17 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized int getExpiredHeartbeats() {
|
|
|
- return stats.expiredHeartbeats;
|
|
|
+ public int getExpiredHeartbeats() {
|
|
|
+ return stats.getExpiredHeartbeats();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public Map<StorageType, StorageTypeStats> getStorageTypeStats() {
|
|
|
- return stats.statsMap.get();
|
|
|
+ public Map<StorageType, StorageTypeStats> getStorageTypeStats() {
|
|
|
+ return stats.getStatsMap();
|
|
|
}
|
|
|
|
|
|
synchronized void register(final DatanodeDescriptor d) {
|
|
|
- if (!d.isAlive) {
|
|
|
+ if (!d.isAlive()) {
|
|
|
addDatanode(d);
|
|
|
|
|
|
//update its timestamp
|
|
@@ -221,14 +211,14 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
// update in-service node count
|
|
|
stats.add(d);
|
|
|
datanodes.add(d);
|
|
|
- d.isAlive = true;
|
|
|
+ d.setAlive(true);
|
|
|
}
|
|
|
|
|
|
synchronized void removeDatanode(DatanodeDescriptor node) {
|
|
|
- if (node.isAlive) {
|
|
|
+ if (node.isAlive()) {
|
|
|
stats.subtract(node);
|
|
|
datanodes.remove(node);
|
|
|
- node.isAlive = false;
|
|
|
+ node.setAlive(false);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -243,7 +233,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
}
|
|
|
|
|
|
synchronized void startDecommission(final DatanodeDescriptor node) {
|
|
|
- if (!node.isAlive) {
|
|
|
+ if (!node.isAlive()) {
|
|
|
LOG.info("Dead node {} is decommissioned immediately.", node);
|
|
|
node.setDecommissioned();
|
|
|
} else {
|
|
@@ -255,8 +245,8 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
|
|
|
synchronized void stopDecommission(final DatanodeDescriptor node) {
|
|
|
LOG.info("Stopping decommissioning of {} node {}",
|
|
|
- node.isAlive ? "live" : "dead", node);
|
|
|
- if (!node.isAlive) {
|
|
|
+ node.isAlive() ? "live" : "dead", node);
|
|
|
+ if (!node.isAlive()) {
|
|
|
node.stopDecommission();
|
|
|
} else {
|
|
|
stats.subtract(node);
|
|
@@ -302,6 +292,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
* B. Remove all blocks in PendingDataNodeMessages for the failed storage
|
|
|
* when we remove all blocks from BlocksMap for that storage.
|
|
|
*/
|
|
|
+ @VisibleForTesting
|
|
|
void heartbeatCheck() {
|
|
|
final DatanodeManager dm = blockManager.getDatanodeManager();
|
|
|
// It's OK to check safe mode w/o taking the lock here, we re-check
|
|
@@ -354,16 +345,14 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
}
|
|
|
|
|
|
allAlive = dead == null && failedStorage == null;
|
|
|
+ if (!allAlive && namesystem.isInStartupSafeMode()) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
if (dead != null) {
|
|
|
// acquire the fsnamesystem lock, and then remove the dead node.
|
|
|
namesystem.writeLock();
|
|
|
try {
|
|
|
- if (namesystem.isInStartupSafeMode()) {
|
|
|
- return;
|
|
|
- }
|
|
|
- synchronized(this) {
|
|
|
- dm.removeDeadDatanode(dead);
|
|
|
- }
|
|
|
+ dm.removeDeadDatanode(dead);
|
|
|
} finally {
|
|
|
namesystem.writeUnlock();
|
|
|
}
|
|
@@ -372,12 +361,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
// acquire the fsnamesystem lock, and remove blocks on the storage.
|
|
|
namesystem.writeLock();
|
|
|
try {
|
|
|
- if (namesystem.isInStartupSafeMode()) {
|
|
|
- return;
|
|
|
- }
|
|
|
- synchronized(this) {
|
|
|
- blockManager.removeBlocksAssociatedTo(failedStorage);
|
|
|
- }
|
|
|
+ blockManager.removeBlocksAssociatedTo(failedStorage);
|
|
|
} finally {
|
|
|
namesystem.writeUnlock();
|
|
|
}
|
|
@@ -385,7 +369,6 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
/** Periodically check heartbeat and update block key */
|
|
|
private class Monitor implements Runnable {
|
|
|
private long lastHeartbeatCheck;
|
|
@@ -404,7 +387,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
if (blockManager.shouldUpdateBlockKey(now - lastBlockKeyUpdate)) {
|
|
|
synchronized(HeartbeatManager.this) {
|
|
|
for(DatanodeDescriptor d : datanodes) {
|
|
|
- d.needKeyUpdate = true;
|
|
|
+ d.setNeedKeyUpdate(true);
|
|
|
}
|
|
|
}
|
|
|
lastBlockKeyUpdate = now;
|
|
@@ -414,7 +397,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
}
|
|
|
try {
|
|
|
Thread.sleep(5000); // 5 seconds
|
|
|
- } catch (InterruptedException ie) {
|
|
|
+ } catch (InterruptedException ignored) {
|
|
|
}
|
|
|
// avoid declaring nodes dead for another cycle if a GC pause lasts
|
|
|
// longer than the node recheck interval
|
|
@@ -425,143 +408,4 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /** Datanode statistics.
|
|
|
- * For decommissioning/decommissioned nodes, only used capacity is counted.
|
|
|
- */
|
|
|
- private static class Stats {
|
|
|
-
|
|
|
- private final StorageTypeStatsMap statsMap = new StorageTypeStatsMap();
|
|
|
-
|
|
|
- private long capacityTotal = 0L;
|
|
|
- private long capacityUsed = 0L;
|
|
|
- private long capacityRemaining = 0L;
|
|
|
- private long blockPoolUsed = 0L;
|
|
|
- private int xceiverCount = 0;
|
|
|
- private long cacheCapacity = 0L;
|
|
|
- private long cacheUsed = 0L;
|
|
|
-
|
|
|
- private int nodesInService = 0;
|
|
|
- private int nodesInServiceXceiverCount = 0;
|
|
|
-
|
|
|
- private int expiredHeartbeats = 0;
|
|
|
-
|
|
|
- private void add(final DatanodeDescriptor node) {
|
|
|
- capacityUsed += node.getDfsUsed();
|
|
|
- blockPoolUsed += node.getBlockPoolUsed();
|
|
|
- xceiverCount += node.getXceiverCount();
|
|
|
- if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
|
|
|
- nodesInService++;
|
|
|
- nodesInServiceXceiverCount += node.getXceiverCount();
|
|
|
- capacityTotal += node.getCapacity();
|
|
|
- capacityRemaining += node.getRemaining();
|
|
|
- } else {
|
|
|
- capacityTotal += node.getDfsUsed();
|
|
|
- }
|
|
|
- cacheCapacity += node.getCacheCapacity();
|
|
|
- cacheUsed += node.getCacheUsed();
|
|
|
- Set<StorageType> storageTypes = new HashSet<>();
|
|
|
- for (DatanodeStorageInfo storageInfo : node.getStorageInfos()) {
|
|
|
- statsMap.addStorage(storageInfo, node);
|
|
|
- storageTypes.add(storageInfo.getStorageType());
|
|
|
- }
|
|
|
- for (StorageType storageType : storageTypes) {
|
|
|
- statsMap.addNode(storageType, node);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void subtract(final DatanodeDescriptor node) {
|
|
|
- capacityUsed -= node.getDfsUsed();
|
|
|
- blockPoolUsed -= node.getBlockPoolUsed();
|
|
|
- xceiverCount -= node.getXceiverCount();
|
|
|
- if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
|
|
|
- nodesInService--;
|
|
|
- nodesInServiceXceiverCount -= node.getXceiverCount();
|
|
|
- capacityTotal -= node.getCapacity();
|
|
|
- capacityRemaining -= node.getRemaining();
|
|
|
- } else {
|
|
|
- capacityTotal -= node.getDfsUsed();
|
|
|
- }
|
|
|
- cacheCapacity -= node.getCacheCapacity();
|
|
|
- cacheUsed -= node.getCacheUsed();
|
|
|
- Set<StorageType> storageTypes = new HashSet<>();
|
|
|
- for (DatanodeStorageInfo storageInfo : node.getStorageInfos()) {
|
|
|
- statsMap.subtractStorage(storageInfo, node);
|
|
|
- storageTypes.add(storageInfo.getStorageType());
|
|
|
- }
|
|
|
- for (StorageType storageType : storageTypes) {
|
|
|
- statsMap.subtractNode(storageType, node);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /** Increment expired heartbeat counter. */
|
|
|
- private void incrExpiredHeartbeats() {
|
|
|
- expiredHeartbeats++;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /** StorageType specific statistics.
|
|
|
- * For decommissioning/decommissioned nodes, only used capacity is counted.
|
|
|
- */
|
|
|
-
|
|
|
- static final class StorageTypeStatsMap {
|
|
|
-
|
|
|
- private Map<StorageType, StorageTypeStats> storageTypeStatsMap =
|
|
|
- new IdentityHashMap<>();
|
|
|
-
|
|
|
- private StorageTypeStatsMap() {}
|
|
|
-
|
|
|
- private StorageTypeStatsMap(StorageTypeStatsMap other) {
|
|
|
- storageTypeStatsMap =
|
|
|
- new IdentityHashMap<>(other.storageTypeStatsMap);
|
|
|
- for (Map.Entry<StorageType, StorageTypeStats> entry :
|
|
|
- storageTypeStatsMap.entrySet()) {
|
|
|
- entry.setValue(new StorageTypeStats(entry.getValue()));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private Map<StorageType, StorageTypeStats> get() {
|
|
|
- return Collections.unmodifiableMap(storageTypeStatsMap);
|
|
|
- }
|
|
|
-
|
|
|
- private void addNode(StorageType storageType,
|
|
|
- final DatanodeDescriptor node) {
|
|
|
- StorageTypeStats storageTypeStats =
|
|
|
- storageTypeStatsMap.get(storageType);
|
|
|
- if (storageTypeStats == null) {
|
|
|
- storageTypeStats = new StorageTypeStats();
|
|
|
- storageTypeStatsMap.put(storageType, storageTypeStats);
|
|
|
- }
|
|
|
- storageTypeStats.addNode(node);
|
|
|
- }
|
|
|
-
|
|
|
- private void addStorage(final DatanodeStorageInfo info,
|
|
|
- final DatanodeDescriptor node) {
|
|
|
- StorageTypeStats storageTypeStats =
|
|
|
- storageTypeStatsMap.get(info.getStorageType());
|
|
|
- if (storageTypeStats == null) {
|
|
|
- storageTypeStats = new StorageTypeStats();
|
|
|
- storageTypeStatsMap.put(info.getStorageType(), storageTypeStats);
|
|
|
- }
|
|
|
- storageTypeStats.addStorage(info, node);
|
|
|
- }
|
|
|
-
|
|
|
- private void subtractStorage(final DatanodeStorageInfo info,
|
|
|
- final DatanodeDescriptor node) {
|
|
|
- StorageTypeStats storageTypeStats =
|
|
|
- storageTypeStatsMap.get(info.getStorageType());
|
|
|
- if (storageTypeStats != null) {
|
|
|
- storageTypeStats.subtractStorage(info, node);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void subtractNode(StorageType storageType,
|
|
|
- final DatanodeDescriptor node) {
|
|
|
- StorageTypeStats storageTypeStats =
|
|
|
- storageTypeStatsMap.get(storageType);
|
|
|
- if (storageTypeStats != null) {
|
|
|
- storageTypeStats.subtractNode(node);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
}
|