Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ public StorageDistributionEndpoint(OzoneStorageContainerManager reconSCM,
public Response getStorageDistribution() {
try {
List<DatanodeStorageReport> nodeStorageReports = collectDatanodeReports();
GlobalStorageReport globalStorageReport = calculateGlobalStorageReport();
GlobalStorageReport globalStorageReport = calculateGlobalStorageReport(nodeStorageReports);
OpenKeyBytesInfo totalOpenKeySize;
try {
totalOpenKeySize = calculateOpenKeySizes();
Expand Down Expand Up @@ -221,28 +221,37 @@ public Response downloadDataNodeStorageDistribution() {
return ReconUtils.downloadCsv("datanode_storage_and_pending_deletion_stats.csv", headers, data, columns);
}

private GlobalStorageReport calculateGlobalStorageReport() {
GlobalStorageReport.Builder globalStorageBuilder = GlobalStorageReport.newBuilder();
try {
SCMNodeStat stats = nodeManager.getStats();
if (stats == null) {
LOG.warn("Node manager stats are null, returning default values");
return globalStorageBuilder.build();
}

return globalStorageBuilder
.setTotalOzoneCapacity(stats.getCapacity() != null ? stats.getCapacity().get() : 0L)
.setTotalReservedSpace(stats.getReserved() != null ? stats.getReserved().get() : 0L)
.setTotalOzoneFreeSpace(stats.getRemaining() != null ? stats.getRemaining().get() : 0L)
.setTotalOzoneUsedSpace(stats.getScmUsed() != null ? stats.getScmUsed().get() : 0L)
.setTotalOzoneCommittedSpace(stats.getCommitted() != null ? stats.getCommitted().get() : 0L)
.setTotalMinimumFreeSpace(stats.getFreeSpaceToSpare() != null ? stats.getFreeSpaceToSpare().get() : 0L)
.build();

} catch (Exception e) {
LOG.error("Error calculating global storage report", e);
return globalStorageBuilder.build();
/**
* Aggregates the global storage report by summing fields from the already-collected
* per-DN list. This guarantees that {@code globalStorage} and {@code dataNodeUsage}
* always reflect the same in-memory snapshot, eliminating the race window that existed
* when a separate {@code nodeManager.getStats()} call was made after the DN list was read.
*/
private GlobalStorageReport calculateGlobalStorageReport(List<DatanodeStorageReport> reports) {
long capacity = 0L;
long used = 0L;
long remaining = 0L;
long committed = 0L;
long minFreeSpace = 0L;
long reserved = 0L;

for (DatanodeStorageReport report : reports) {
capacity += report.getCapacity();
used += report.getUsed();
remaining += report.getRemaining();
committed += report.getCommitted();
minFreeSpace += report.getMinimumFreeSpace();
reserved += report.getReserved();
}

return GlobalStorageReport.newBuilder()
.setTotalOzoneCapacity(capacity)
.setTotalReservedSpace(reserved)
.setTotalOzoneFreeSpace(remaining)
.setTotalOzoneUsedSpace(used)
.setTotalOzoneCommittedSpace(committed)
.setTotalMinimumFreeSpace(minFreeSpace)
.build();
}

private Map<String, Long> calculateNamespaceMetrics(OpenKeyBytesInfo totalOpenKeySize) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,8 @@
import org.apache.hadoop.hdds.fs.SpaceUsageSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
import org.apache.hadoop.hdds.scm.node.NodeStatus;
import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
import org.apache.hadoop.ozone.recon.api.types.DUResponse;
import org.apache.hadoop.ozone.recon.api.types.DataNodeMetricsServiceResponse;
Expand Down Expand Up @@ -225,7 +225,7 @@ private List<String> mockStorageDistributionData(int numNodes) throws Exception
.build();
pendingDeletionMetrics.add(new DatanodePendingDeletionMetrics(hostName,
uuid.toString(), PENDING_DELETION_SIZE));
dataNodes.add(new DatanodeInfo(datanode, null, null));
dataNodes.add(new DatanodeInfo(datanode, NodeStatus.inServiceHealthy(), null));
when(nodeManager.getNodeStat(datanode))
.thenReturn(new SCMNodeMetric(OZONE_CAPACITY, OZONE_USED, OZONE_REMAINING, COMMITTED,
MIN_FREE_SPACE, RESERVED));
Expand All @@ -248,14 +248,6 @@ private List<String> mockStorageDistributionData(int numNodes) throws Exception

}
when(nodeManager.getAllNodes()).thenReturn(dataNodes);
when(nodeManager.getStats())
.thenReturn(new SCMNodeStat(
OZONE_CAPACITY * numNodes,
OZONE_USED * numNodes,
OZONE_REMAINING * numNodes,
COMMITTED * numNodes,
MIN_FREE_SPACE * numNodes,
RESERVED * numNodes));


Map<String, Long> pendingSizes = new HashMap<>();
Expand Down