|
@@ -22,18 +22,18 @@ package org.apache.hadoop.hdds.scm.node;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
|
|
|
-import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.
|
|
|
+ StorageContainerDatanodeProtocolProtos.SCMStorageReport;
|
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
|
import org.apache.hadoop.metrics2.util.MBeans;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
+import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import javax.management.ObjectName;
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.List;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.UUID;
|
|
|
+import java.io.IOException;
|
|
|
+import java.util.*;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
@@ -52,16 +52,15 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
|
|
|
private final double warningUtilizationThreshold;
|
|
|
private final double criticalUtilizationThreshold;
|
|
|
|
|
|
- private final Map<UUID, SCMNodeStat> scmNodeStorageStatMap;
|
|
|
+ private final Map<UUID, Set<StorageLocationReport>> scmNodeStorageReportMap;
|
|
|
// NodeStorageInfo MXBean
|
|
|
private ObjectName scmNodeStorageInfoBean;
|
|
|
- // Aggregated node stats
|
|
|
- private SCMNodeStat clusterStat;
|
|
|
/**
|
|
|
- * constructs the scmNodeStorageStatMap object
|
|
|
+ * constructs the scmNodeStorageReportMap object
|
|
|
*/
|
|
|
public SCMNodeStorageStatMap(OzoneConfiguration conf) {
|
|
|
- scmNodeStorageStatMap = new ConcurrentHashMap<>();
|
|
|
+ // scmNodeStorageReportMap = new ConcurrentHashMap<>();
|
|
|
+ scmNodeStorageReportMap = new ConcurrentHashMap<>();
|
|
|
warningUtilizationThreshold = conf.getDouble(
|
|
|
OzoneConfigKeys.
|
|
|
HDDS_DATANODE_STORAGE_UTILIZATION_WARNING_THRESHOLD,
|
|
@@ -72,7 +71,6 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
|
|
|
HDDS_DATANODE_STORAGE_UTILIZATION_CRITICAL_THRESHOLD,
|
|
|
OzoneConfigKeys.
|
|
|
HDDS_DATANODE_STORAGE_UTILIZATION_CRITICAL_THRESHOLD_DEFAULT);
|
|
|
- clusterStat = new SCMNodeStat();
|
|
|
}
|
|
|
|
|
|
public enum UtilizationThreshold {
|
|
@@ -81,20 +79,22 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
|
|
|
|
|
|
/**
|
|
|
* Returns true if this a datanode that is already tracked by
|
|
|
- * scmNodeStorageStatMap.
|
|
|
+ * scmNodeStorageReportMap.
|
|
|
*
|
|
|
* @param datanodeID - UUID of the Datanode.
|
|
|
* @return True if this is tracked, false if this map does not know about it.
|
|
|
*/
|
|
|
public boolean isKnownDatanode(UUID datanodeID) {
|
|
|
Preconditions.checkNotNull(datanodeID);
|
|
|
- return scmNodeStorageStatMap.containsKey(datanodeID);
|
|
|
+ return scmNodeStorageReportMap.containsKey(datanodeID);
|
|
|
}
|
|
|
|
|
|
public List<UUID> getDatanodeList(
|
|
|
UtilizationThreshold threshold) {
|
|
|
- return scmNodeStorageStatMap.entrySet().stream()
|
|
|
- .filter(entry -> (isThresholdReached(threshold, entry.getValue())))
|
|
|
+ return scmNodeStorageReportMap.entrySet().stream().filter(
|
|
|
+ entry -> (isThresholdReached(threshold,
|
|
|
+ getScmUsedratio(getUsedSpace(entry.getKey()),
|
|
|
+ getCapacity(entry.getKey())))))
|
|
|
.map(Map.Entry::getKey)
|
|
|
.collect(Collectors.toList());
|
|
|
}
|
|
@@ -105,19 +105,19 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
|
|
|
* Insert a new datanode into Node2Container Map.
|
|
|
*
|
|
|
* @param datanodeID -- Datanode UUID
|
|
|
- * @param stat - scmNode stat for the Datanode.
|
|
|
+ * @param report - set if StorageReports.
|
|
|
*/
|
|
|
- public void insertNewDatanode(UUID datanodeID, SCMNodeStat stat)
|
|
|
+ public void insertNewDatanode(UUID datanodeID, Set<StorageLocationReport> report)
|
|
|
throws SCMException {
|
|
|
- Preconditions.checkNotNull(stat);
|
|
|
+ Preconditions.checkNotNull(report);
|
|
|
+ Preconditions.checkState(report.size() != 0);
|
|
|
Preconditions.checkNotNull(datanodeID);
|
|
|
- synchronized (scmNodeStorageStatMap) {
|
|
|
+ synchronized (scmNodeStorageReportMap) {
|
|
|
if (isKnownDatanode(datanodeID)) {
|
|
|
throw new SCMException("Node already exists in the map",
|
|
|
DUPLICATE_DATANODE);
|
|
|
}
|
|
|
- scmNodeStorageStatMap.put(datanodeID, stat);
|
|
|
- clusterStat.add(stat);
|
|
|
+ scmNodeStorageReportMap.putIfAbsent(datanodeID, report);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -138,72 +138,103 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
|
|
|
* Updates the Container list of an existing DN.
|
|
|
*
|
|
|
* @param datanodeID - UUID of DN.
|
|
|
- * @param stat - scmNode stat for the Datanode.
|
|
|
+ * @param report - set of Storage Reports for the Datanode.
|
|
|
* @throws SCMException - if we don't know about this datanode, for new DN
|
|
|
* use insertNewDatanode.
|
|
|
*/
|
|
|
- public void updateDatanodeMap(UUID datanodeID, SCMNodeStat stat)
|
|
|
+ public void updateDatanodeMap(UUID datanodeID, Set<StorageLocationReport> report)
|
|
|
throws SCMException {
|
|
|
Preconditions.checkNotNull(datanodeID);
|
|
|
- Preconditions.checkNotNull(stat);
|
|
|
- synchronized (scmNodeStorageStatMap) {
|
|
|
- if (!scmNodeStorageStatMap.containsKey(datanodeID)) {
|
|
|
+ Preconditions.checkNotNull(report);
|
|
|
+ Preconditions.checkState(report.size() != 0);
|
|
|
+ synchronized (scmNodeStorageReportMap) {
|
|
|
+ if (!scmNodeStorageReportMap.containsKey(datanodeID)) {
|
|
|
throw new SCMException("No such datanode", NO_SUCH_DATANODE);
|
|
|
}
|
|
|
- SCMNodeStat removed = scmNodeStorageStatMap.get(datanodeID);
|
|
|
- clusterStat.subtract(removed);
|
|
|
- scmNodeStorageStatMap.put(datanodeID, stat);
|
|
|
- clusterStat.add(stat);
|
|
|
+ scmNodeStorageReportMap.put(datanodeID, report);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public NodeReportStatus processNodeReport(UUID datanodeID,
|
|
|
+ public StorageReportResult processNodeReport(UUID datanodeID,
|
|
|
StorageContainerDatanodeProtocolProtos.SCMNodeReport nodeReport)
|
|
|
- throws SCMException {
|
|
|
+ throws IOException {
|
|
|
Preconditions.checkNotNull(datanodeID);
|
|
|
Preconditions.checkNotNull(nodeReport);
|
|
|
+
|
|
|
long totalCapacity = 0;
|
|
|
long totalRemaining = 0;
|
|
|
long totalScmUsed = 0;
|
|
|
- List<StorageContainerDatanodeProtocolProtos.SCMStorageReport>
|
|
|
+ Set<StorageLocationReport> storagReportSet = new HashSet<>();
|
|
|
+ Set<StorageLocationReport> fullVolumeSet = new HashSet<>();
|
|
|
+ Set<StorageLocationReport> failedVolumeSet = new HashSet<>();
|
|
|
+ List<SCMStorageReport>
|
|
|
storageReports = nodeReport.getStorageReportList();
|
|
|
- for (StorageContainerDatanodeProtocolProtos.SCMStorageReport report : storageReports) {
|
|
|
+ for (SCMStorageReport report : storageReports) {
|
|
|
+ StorageLocationReport storageReport =
|
|
|
+ StorageLocationReport.getFromProtobuf(report);
|
|
|
+ storagReportSet.add(storageReport);
|
|
|
+ if (report.hasFailed() && report.getFailed()) {
|
|
|
+ failedVolumeSet.add(storageReport);
|
|
|
+ } else if (isThresholdReached(UtilizationThreshold.CRITICAL,
|
|
|
+ getScmUsedratio(report.getScmUsed(), report.getCapacity()))) {
|
|
|
+ fullVolumeSet.add(storageReport);
|
|
|
+ }
|
|
|
totalCapacity += report.getCapacity();
|
|
|
totalRemaining += report.getRemaining();
|
|
|
totalScmUsed += report.getScmUsed();
|
|
|
}
|
|
|
- SCMNodeStat stat = scmNodeStorageStatMap.get(datanodeID);
|
|
|
- if (stat == null) {
|
|
|
- stat = new SCMNodeStat();
|
|
|
- stat.set(totalCapacity, totalScmUsed, totalRemaining);
|
|
|
- insertNewDatanode(datanodeID, stat);
|
|
|
+
|
|
|
+ if (!isKnownDatanode(datanodeID)) {
|
|
|
+ insertNewDatanode(datanodeID, storagReportSet);
|
|
|
} else {
|
|
|
- stat.set(totalCapacity, totalScmUsed, totalRemaining);
|
|
|
- updateDatanodeMap(datanodeID, stat);
|
|
|
+ updateDatanodeMap(datanodeID, storagReportSet);
|
|
|
}
|
|
|
- if (isThresholdReached(UtilizationThreshold.CRITICAL, stat)) {
|
|
|
+ if (isThresholdReached(UtilizationThreshold.CRITICAL,
|
|
|
+ getScmUsedratio(totalScmUsed, totalCapacity))) {
|
|
|
LOG.warn("Datanode {} is out of storage space. Capacity: {}, Used: {}",
|
|
|
- datanodeID, stat.getCapacity().get(), stat.getScmUsed().get());
|
|
|
- return NodeReportStatus.DATANODE_OUT_OF_SPACE;
|
|
|
- } else {
|
|
|
- if (isThresholdReached(UtilizationThreshold.WARN, stat)) {
|
|
|
- LOG.warn("Datanode {} is low on storage space. Capacity: {}, Used: {}",
|
|
|
- datanodeID, stat.getCapacity().get(), stat.getScmUsed().get());
|
|
|
- }
|
|
|
- return NodeReportStatus.ALL_IS_WELL;
|
|
|
+ datanodeID, totalCapacity, totalScmUsed);
|
|
|
+ return StorageReportResult.ReportResultBuilder.newBuilder()
|
|
|
+ .setStatus(ReportStatus.DATANODE_OUT_OF_SPACE)
|
|
|
+ .setFullVolumeSet(fullVolumeSet).setFailedVolumeSet(failedVolumeSet)
|
|
|
+ .build();
|
|
|
+ }
|
|
|
+ if (isThresholdReached(UtilizationThreshold.WARN,
|
|
|
+ getScmUsedratio(totalScmUsed, totalCapacity))) {
|
|
|
+ LOG.warn("Datanode {} is low on storage space. Capacity: {}, Used: {}",
|
|
|
+ datanodeID, totalCapacity, totalScmUsed);
|
|
|
}
|
|
|
+
|
|
|
+ if (failedVolumeSet.isEmpty() && !fullVolumeSet.isEmpty()) {
|
|
|
+ return StorageReportResult.ReportResultBuilder.newBuilder()
|
|
|
+ .setStatus(ReportStatus.STORAGE_OUT_OF_SPACE)
|
|
|
+ .setFullVolumeSet(fullVolumeSet).build();
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!failedVolumeSet.isEmpty() && fullVolumeSet.isEmpty()) {
|
|
|
+ return StorageReportResult.ReportResultBuilder.newBuilder()
|
|
|
+ .setStatus(ReportStatus.FAILED_STORAGE)
|
|
|
+ .setFailedVolumeSet(failedVolumeSet).build();
|
|
|
+ }
|
|
|
+ if (!failedVolumeSet.isEmpty() && !fullVolumeSet.isEmpty()) {
|
|
|
+ return StorageReportResult.ReportResultBuilder.newBuilder()
|
|
|
+ .setStatus(ReportStatus.FAILED_AND_OUT_OF_SPACE_STORAGE)
|
|
|
+ .setFailedVolumeSet(failedVolumeSet).setFullVolumeSet(fullVolumeSet)
|
|
|
+ .build();
|
|
|
+ }
|
|
|
+ return StorageReportResult.ReportResultBuilder.newBuilder()
|
|
|
+ .setStatus(ReportStatus.ALL_IS_WELL).build();
|
|
|
}
|
|
|
|
|
|
private boolean isThresholdReached(UtilizationThreshold threshold,
|
|
|
- SCMNodeStat stat) {
|
|
|
+ double scmUsedratio) {
|
|
|
switch (threshold) {
|
|
|
case NORMAL:
|
|
|
- return stat.getScmUsedratio() < warningUtilizationThreshold;
|
|
|
+ return scmUsedratio < warningUtilizationThreshold;
|
|
|
case WARN:
|
|
|
- return stat.getScmUsedratio() >= warningUtilizationThreshold &&
|
|
|
- stat.getScmUsedratio() < criticalUtilizationThreshold;
|
|
|
+ return scmUsedratio >= warningUtilizationThreshold
|
|
|
+ && scmUsedratio < criticalUtilizationThreshold;
|
|
|
case CRITICAL:
|
|
|
- return stat.getScmUsedratio() >= criticalUtilizationThreshold;
|
|
|
+ return scmUsedratio >= criticalUtilizationThreshold;
|
|
|
default:
|
|
|
throw new RuntimeException("Unknown UtilizationThreshold value");
|
|
|
}
|
|
@@ -211,67 +242,120 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
|
|
|
|
|
|
@Override
|
|
|
public long getCapacity(UUID dnId) {
|
|
|
- return scmNodeStorageStatMap.get(dnId).getCapacity().get();
|
|
|
+ long capacity = 0;
|
|
|
+ Set<StorageLocationReport> reportSet = scmNodeStorageReportMap.get(dnId);
|
|
|
+ for (StorageLocationReport report : reportSet) {
|
|
|
+ capacity += report.getCapacity();
|
|
|
+ }
|
|
|
+ return capacity;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getRemainingSpace(UUID dnId) {
|
|
|
- return scmNodeStorageStatMap.get(dnId).getRemaining().get();
|
|
|
+ long remaining = 0;
|
|
|
+ Set<StorageLocationReport> reportSet = scmNodeStorageReportMap.get(dnId);
|
|
|
+ for (StorageLocationReport report : reportSet) {
|
|
|
+ remaining += report.getRemaining();
|
|
|
+ }
|
|
|
+ return remaining;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getUsedSpace(UUID dnId) {
|
|
|
- return scmNodeStorageStatMap.get(dnId).getScmUsed().get();
|
|
|
+ long scmUsed = 0;
|
|
|
+ Set<StorageLocationReport> reportSet = scmNodeStorageReportMap.get(dnId);
|
|
|
+ for (StorageLocationReport report : reportSet) {
|
|
|
+ scmUsed += report.getScmUsed();
|
|
|
+ }
|
|
|
+ return scmUsed;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getTotalCapacity() {
|
|
|
- return clusterStat.getCapacity().get();
|
|
|
+ long capacity = 0;
|
|
|
+ Set<UUID> dnIdSet = scmNodeStorageReportMap.keySet();
|
|
|
+ for (UUID id : dnIdSet) {
|
|
|
+ capacity += getCapacity(id);
|
|
|
+ }
|
|
|
+ return capacity;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getTotalSpaceUsed() {
|
|
|
- return clusterStat.getScmUsed().get();
|
|
|
+ long scmUsed = 0;
|
|
|
+ Set<UUID> dnIdSet = scmNodeStorageReportMap.keySet();
|
|
|
+ for (UUID id : dnIdSet) {
|
|
|
+ scmUsed += getUsedSpace(id);
|
|
|
+ }
|
|
|
+ return scmUsed;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public long getTotalFreeSpace() {
|
|
|
- return clusterStat.getRemaining().get();
|
|
|
+ long remaining = 0;
|
|
|
+ Set<UUID> dnIdSet = scmNodeStorageReportMap.keySet();
|
|
|
+ for (UUID id : dnIdSet) {
|
|
|
+ remaining += getRemainingSpace(id);
|
|
|
+ }
|
|
|
+ return remaining;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * removes the dataNode from scmNodeStorageStatMap
|
|
|
+ * removes the dataNode from scmNodeStorageReportMap
|
|
|
* @param datanodeID
|
|
|
* @throws SCMException in case the dataNode is not found in the map.
|
|
|
*/
|
|
|
public void removeDatanode(UUID datanodeID) throws SCMException {
|
|
|
Preconditions.checkNotNull(datanodeID);
|
|
|
- synchronized (scmNodeStorageStatMap) {
|
|
|
- if (!scmNodeStorageStatMap.containsKey(datanodeID)) {
|
|
|
+ synchronized (scmNodeStorageReportMap) {
|
|
|
+ if (!scmNodeStorageReportMap.containsKey(datanodeID)) {
|
|
|
throw new SCMException("No such datanode", NO_SUCH_DATANODE);
|
|
|
}
|
|
|
- SCMNodeStat stat = scmNodeStorageStatMap.remove(datanodeID);
|
|
|
- clusterStat.subtract(stat);
|
|
|
+ scmNodeStorageReportMap.remove(datanodeID);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Gets the SCMNodeStat for the datanode
|
|
|
+ * Returns the set of storage volumes for a Datanode.
|
|
|
* @param datanodeID
|
|
|
- * @return SCMNodeStat
|
|
|
+ * @return set of storage volumes.
|
|
|
*/
|
|
|
|
|
|
- SCMNodeStat getNodeStat(UUID datanodeID) {
|
|
|
- return scmNodeStorageStatMap.get(datanodeID);
|
|
|
+ @Override
|
|
|
+ public Set<StorageLocationReport> getStorageVolumes(UUID datanodeID) {
|
|
|
+ return scmNodeStorageReportMap.get(datanodeID);
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Truncate to 4 digits since uncontrolled precision is some times
|
|
|
+ * counter intuitive to what users expect.
|
|
|
+ * @param value - double.
|
|
|
+ * @return double.
|
|
|
+ */
|
|
|
+ private double truncateDecimals(double value) {
|
|
|
+ final int multiplier = 10000;
|
|
|
+ return (double) ((long) (value * multiplier)) / multiplier;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * get the scmUsed ratio
|
|
|
+ */
|
|
|
+ public double getScmUsedratio(long scmUsed, long capacity) {
|
|
|
+ double scmUsedRatio =
|
|
|
+ truncateDecimals (scmUsed / (double) capacity);
|
|
|
+ return scmUsedRatio;
|
|
|
+ }
|
|
|
/**
|
|
|
* Results possible from processing a Node report by
|
|
|
* Node2ContainerMapper.
|
|
|
*/
|
|
|
- public enum NodeReportStatus {
|
|
|
+ public enum ReportStatus {
|
|
|
ALL_IS_WELL,
|
|
|
- DATANODE_OUT_OF_SPACE
|
|
|
+ DATANODE_OUT_OF_SPACE,
|
|
|
+ STORAGE_OUT_OF_SPACE,
|
|
|
+ FAILED_STORAGE,
|
|
|
+ FAILED_AND_OUT_OF_SPACE_STORAGE
|
|
|
}
|
|
|
|
|
|
-}
|
|
|
+}
|