|
@@ -18,9 +18,15 @@
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.Collections;
|
|
|
|
+import java.util.HashSet;
|
|
|
|
+import java.util.IdentityHashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
+import java.util.Set;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.fs.StorageType;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
@@ -189,6 +195,11 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
return stats.expiredHeartbeats;
|
|
return stats.expiredHeartbeats;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public Map<StorageType, StorageTypeStats> getStorageTypeStats() {
|
|
|
|
+ return stats.statsMap.get();
|
|
|
|
+ }
|
|
|
|
+
|
|
synchronized void register(final DatanodeDescriptor d) {
|
|
synchronized void register(final DatanodeDescriptor d) {
|
|
if (!d.isAlive) {
|
|
if (!d.isAlive) {
|
|
addDatanode(d);
|
|
addDatanode(d);
|
|
@@ -393,6 +404,9 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
* For decommissioning/decommissioned nodes, only used capacity is counted.
|
|
* For decommissioning/decommissioned nodes, only used capacity is counted.
|
|
*/
|
|
*/
|
|
private static class Stats {
|
|
private static class Stats {
|
|
|
|
+
|
|
|
|
+ private final StorageTypeStatsMap statsMap = new StorageTypeStatsMap();
|
|
|
|
+
|
|
private long capacityTotal = 0L;
|
|
private long capacityTotal = 0L;
|
|
private long capacityUsed = 0L;
|
|
private long capacityUsed = 0L;
|
|
private long capacityRemaining = 0L;
|
|
private long capacityRemaining = 0L;
|
|
@@ -420,6 +434,14 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
}
|
|
}
|
|
cacheCapacity += node.getCacheCapacity();
|
|
cacheCapacity += node.getCacheCapacity();
|
|
cacheUsed += node.getCacheUsed();
|
|
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) {
|
|
private void subtract(final DatanodeDescriptor node) {
|
|
@@ -436,6 +458,14 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
}
|
|
}
|
|
cacheCapacity -= node.getCacheCapacity();
|
|
cacheCapacity -= node.getCacheCapacity();
|
|
cacheUsed -= node.getCacheUsed();
|
|
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. */
|
|
/** Increment expired heartbeat counter. */
|
|
@@ -443,5 +473,69 @@ class HeartbeatManager implements DatanodeStatistics {
|
|
expiredHeartbeats++;
|
|
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);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+}
|