devmadhuu commented on code in PR #9326:
URL: https://github.com/apache/ozone/pull/9326#discussion_r2554699989


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/StorageDistributionEndpoint.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.KEY_TABLE;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Response;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import 
org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary;
+import org.apache.hadoop.hdds.recon.ReconConfigKeys;
+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.protocol.StorageContainerLocationProtocol;
+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.DatanodeStorageReport;
+import org.apache.hadoop.ozone.recon.api.types.DeletionPendingBytesByComponent;
+import org.apache.hadoop.ozone.recon.api.types.GlobalNamespaceReport;
+import org.apache.hadoop.ozone.recon.api.types.GlobalStorageReport;
+import org.apache.hadoop.ozone.recon.api.types.KeyInsightInfoResponse;
+import 
org.apache.hadoop.ozone.recon.api.types.StorageCapacityDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.UsedSpaceBreakDown;
+import org.apache.hadoop.ozone.recon.scm.ReconNodeManager;
+import org.apache.hadoop.ozone.recon.spi.ReconGlobalStatsManager;
+import org.apache.hadoop.ozone.recon.tasks.GlobalStatsValue;
+import org.apache.hadoop.ozone.recon.tasks.JmxMetricsCollectorTask;
+import org.apache.hadoop.ozone.recon.tasks.JmxMetricsCollectorTaskResult;
+import org.apache.hadoop.ozone.recon.tasks.OmTableInsightTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This endpoint handles requests related to storage distribution across
+ * different datanodes in a Recon instance. It provides detailed reports
+ * on storage capacity, utilization, and associated metrics.
+ * <p>
+ * The data is aggregated from multiple sources, including node manager
+ * statistics, and is used to construct responses with information
+ * about global storage and namespace usage, storage usage breakdown,
+ * and deletion operations in progress.
+ * <p>
+ * An instance of {@link ReconNodeManager} is used to fetch detailed
+ * node-specific statistics required for generating the report.
+ */
+@Path("/storageDistribution")
+@Produces("application/json")
+@AdminOnly
+public class StorageDistributionEndpoint {
+  private final ReconNodeManager nodeManager;
+  private final NSSummaryEndpoint nsSummaryEndpoint;
+  private final StorageContainerLocationProtocol scmClient;
+  private static final Logger LOG = 
LoggerFactory.getLogger(StorageDistributionEndpoint.class);
+  private final Map<DatanodeDetails, Long> blockDeletionMetricsMap = new 
ConcurrentHashMap<>();
+  private final ReconGlobalStatsManager reconGlobalStatsManager;
+  private final ReconGlobalMetricsService reconGlobalMetricsService;
+  private final OzoneConfiguration conf;
+
+  @Inject
+  public StorageDistributionEndpoint(OzoneStorageContainerManager reconSCM,
+                                     NSSummaryEndpoint nsSummaryEndpoint,
+                                     ReconGlobalStatsManager 
reconGlobalStatsManager,
+                                     StorageContainerLocationProtocol 
scmClient,
+                                     ReconGlobalMetricsService 
reconGlobalMetricsService,
+                                     OzoneConfiguration conf) {
+    this.nodeManager = (ReconNodeManager) reconSCM.getScmNodeManager();
+    this.nsSummaryEndpoint = nsSummaryEndpoint;
+    this.scmClient = scmClient;
+    this.reconGlobalStatsManager = reconGlobalStatsManager;
+    this.reconGlobalMetricsService = reconGlobalMetricsService;
+    this.conf = conf;
+  }
+
+  @GET
+  public Response getStorageDistribution() {
+    try {
+      initializeBlockDeletionMetricsMap();
+      List<DatanodeStorageReport> nodeStorageReports = 
collectDatanodeReports();
+      GlobalStorageReport globalStorageReport = calculateGlobalStorageReport();
+
+      Map<String, Long> namespaceMetrics = new HashMap<>();
+      try {
+        namespaceMetrics = calculateNamespaceMetrics();
+      } catch (Exception e) {
+        LOG.error("Error calculating namespace metrics", e);
+        // Initialize with default values
+        namespaceMetrics.put("totalUsedNamespace", 0L);
+        namespaceMetrics.put("totalOpenKeySize", 0L);
+        namespaceMetrics.put("totalCommittedSize", 0L);
+        namespaceMetrics.put("pendingDirectorySize", 0L);
+        namespaceMetrics.put("pendingKeySize", 0L);
+        namespaceMetrics.put("totalKeys", 0L);
+      }
+
+      StorageCapacityDistributionResponse response = 
buildStorageDistributionResponse(
+              nodeStorageReports, globalStorageReport, namespaceMetrics);
+      return Response.ok(response).build();
+    } catch (Exception e) {
+      LOG.error("Error getting storage distribution", e);
+      return Response.status(Response.Status.INTERNAL_SERVER_ERROR)
+              .entity("Error retrieving storage distribution: " + 
e.getMessage())
+              .build();
+    }
+  }
+
+  private GlobalStorageReport calculateGlobalStorageReport() {
+    try {
+      SCMNodeStat stats = nodeManager.getStats();
+      if (stats == null) {
+        LOG.warn("Node manager stats are null, returning default values");
+        return new GlobalStorageReport(0L, 0L, 0L);
+      }
+
+      long scmUsed = stats.getScmUsed() != null ? stats.getScmUsed().get() : 
0L;
+      long remaining = stats.getRemaining() != null ? 
stats.getRemaining().get() : 0L;
+      long capacity = stats.getCapacity() != null ? stats.getCapacity().get() 
: 0L;
+
+      return new GlobalStorageReport(scmUsed, remaining, capacity);
+    } catch (Exception e) {
+      LOG.error("Error calculating global storage report", e);
+      return new GlobalStorageReport(0L, 0L, 0L);
+    }
+  }
+
+  private Map<String, Long> calculateNamespaceMetrics() throws IOException {
+    Map<String, Long> metrics = new HashMap<>();
+    Map<String, Long> totalPendingAtOmSide = calculatePendingSizes();
+    long totalOpenKeySize = calculateOpenKeySizes();
+    long totalCommittedSize = calculateCommittedSize();
+    long pendingDirectorySize = 
totalPendingAtOmSide.getOrDefault("pendingDirectorySize", 0L);
+    long pendingKeySize = totalPendingAtOmSide.getOrDefault("pendingKeySize", 
0L);
+    long totalUsedNamespace = pendingDirectorySize + pendingKeySize + 
totalOpenKeySize + totalCommittedSize;
+
+    long totalKeys = 0L;
+    // Keys from OBJECT_STORE buckets.
+    GlobalStatsValue keyRecord = reconGlobalStatsManager.getGlobalStatsValue(
+            OmTableInsightTask.getTableCountKeyFromTable(KEY_TABLE));
+    // Keys from FILE_SYSTEM_OPTIMIZED buckets
+    GlobalStatsValue fileRecord = reconGlobalStatsManager.getGlobalStatsValue(
+            OmTableInsightTask.getTableCountKeyFromTable(FILE_TABLE));
+    if (keyRecord != null) {
+      totalKeys += keyRecord.getValue();
+    }
+    if (fileRecord != null) {
+      totalKeys += fileRecord.getValue();
+    }
+
+    metrics.put("pendingDirectorySize", pendingDirectorySize);
+    metrics.put("pendingKeySize", pendingKeySize);
+    metrics.put("totalOpenKeySize", totalOpenKeySize);
+    metrics.put("totalCommittedSize", totalCommittedSize);
+    metrics.put("totalUsedNamespace", totalUsedNamespace);
+    metrics.put("totalKeys", totalKeys);
+    return metrics;
+  }
+
+  private StorageCapacityDistributionResponse buildStorageDistributionResponse(
+          List<DatanodeStorageReport> nodeStorageReports,
+          GlobalStorageReport storageMetrics,
+          Map<String, Long> namespaceMetrics) {
+    DeletedBlocksTransactionSummary scmSummary = null;
+    try {
+      scmSummary = scmClient.getDeletedBlockSummary();
+    } catch (IOException e) {
+      LOG.error("Failed to get deleted block summary from SCM", e);
+    }
+
+    long totalPendingAtDnSide = 0L;
+    try {
+      totalPendingAtDnSide = 
blockDeletionMetricsMap.values().stream().reduce(0L, Long::sum);
+    } catch (Exception e) {
+      LOG.error("Error calculating pending deletion metrics", e);
+    }
+
+    DeletionPendingBytesByComponent deletionPendingBytesByStage =
+            createDeletionPendingBytesByStage(
+                    namespaceMetrics.getOrDefault("pendingDirectorySize", 0L),
+                    namespaceMetrics.getOrDefault("pendingKeySize", 0L),
+                    scmSummary != null ? 
scmSummary.getTotalBlockReplicatedSize() : 0L,
+                    totalPendingAtDnSide);
+
+    // Safely get values from namespaceMetrics with null checks
+    Long totalUsedNamespace = namespaceMetrics.get("totalUsedNamespace");
+    Long totalOpenKeySize = namespaceMetrics.get("totalOpenKeySize");
+    Long totalCommittedSize = namespaceMetrics.get("totalCommittedSize");
+    Long totalKeys = namespaceMetrics.get("totalKeys");
+    Long totalContainerPreAllocated = nodeStorageReports.stream()
+        .map(DatanodeStorageReport::getCommitted)
+        .reduce(0L, Long::sum);
+
+    return StorageCapacityDistributionResponse.newBuilder()
+            .setDataNodeUsage(nodeStorageReports)
+            .setGlobalStorage(storageMetrics)
+            .setGlobalNamespace(new GlobalNamespaceReport(
+                    totalUsedNamespace != null ? totalUsedNamespace : 0L,
+                    totalKeys != null ? totalKeys : 0L))
+            .setUsedSpaceBreakDown(new UsedSpaceBreakDown(
+                    totalOpenKeySize != null ? totalOpenKeySize : 0L,
+                    totalCommittedSize != null ? totalCommittedSize : 0L, 
totalContainerPreAllocated,
+                    deletionPendingBytesByStage))
+            .build();
+  }
+
+  private List<DatanodeStorageReport> collectDatanodeReports() {
+    return nodeManager.getAllNodes().stream()
+        .map(this::getStorageReport)
+        .filter(Objects::nonNull) // Filter out null reports
+        .collect(Collectors.toList());
+  }
+
+  private Map<String, Long> calculatePendingSizes() {
+    Map<String, Long> result = new HashMap<>();
+    KeyInsightInfoResponse response = 
reconGlobalMetricsService.getPendingForDeletionDirInfo(-1, "");
+    Map<String, Long> pendingKeySize = 
reconGlobalMetricsService.getDeletedKeySummary();
+    result.put("pendingDirectorySize", response.getReplicatedDataSize());
+    result.put("pendingKeySize", 
pendingKeySize.getOrDefault("totalReplicatedDataSize", 0L));
+    return result;
+  }
+
+  private long calculateOpenKeySizes() {
+    Map<String, Long> openKeySummary = 
reconGlobalMetricsService.getOpenKeySummary();
+    Map<String, Long> openKeyMPUSummary = 
reconGlobalMetricsService.getMPUKeySummary();
+    long openKeyDataSize = 
openKeySummary.getOrDefault("totalReplicatedDataSize", 0L);
+    long totalMPUKeySize = 
openKeyMPUSummary.getOrDefault("totalReplicatedDataSize", 0L);
+    return openKeyDataSize + totalMPUKeySize;
+  }
+
+  private long calculateCommittedSize() {
+    try {
+      Response rootResponse = nsSummaryEndpoint.getDiskUsage("/", false, true, 
false);
+      if (rootResponse.getStatus() != Response.Status.OK.getStatusCode()) {
+        LOG.warn("Failed to get disk usage, status: {}", 
rootResponse.getStatus());
+        return 0L;
+      }
+      DUResponse duRootRes = (DUResponse) rootResponse.getEntity();
+      return duRootRes != null ? duRootRes.getSizeWithReplica() : 0L;
+    } catch (IOException e) {
+      LOG.error("IOException while calculating committed size", e);
+      return 0L;
+    }
+  }
+
+  private DeletionPendingBytesByComponent 
createDeletionPendingBytesByStage(long pendingDirectorySize,
+                                                                        long 
pendingKeySize,
+                                                                        long 
totalPendingAtScmSide,
+                                                                        long 
totalPendingAtDnSide) {
+    long totalPending = pendingDirectorySize + pendingKeySize + 
totalPendingAtScmSide + totalPendingAtDnSide;
+    Map<String, Map<String, Long>> stageItems = new HashMap<>();
+    Map<String, Long> omMap = new HashMap<>();
+    omMap.put("totalBytes", pendingDirectorySize + pendingKeySize);
+    omMap.put("pendingDirectoryBytes", pendingDirectorySize);
+    omMap.put("pendingKeyBytes", pendingKeySize);
+    Map<String, Long> scmMap = new HashMap<>();
+    scmMap.put("pendingBytes", totalPendingAtScmSide);
+    Map<String, Long> dnMap = new HashMap<>();
+    dnMap.put("pendingBytes", totalPendingAtDnSide);
+    stageItems.put("OM", omMap);
+    stageItems.put("SCM", scmMap);
+    stageItems.put("DN", dnMap);
+    return new DeletionPendingBytesByComponent(totalPending, stageItems);
+  }
+
+  private void initializeBlockDeletionMetricsMap() {
+    List<JmxMetricsCollectorTask> tasks = 
nodeManager.getNodeStats().keySet().stream().map(dn ->
+      new JmxMetricsCollectorTask(conf, dn, "HddsDatanode",
+          "BlockDeletingService", "TotalPendingBlockBytes"))
+        .collect(Collectors.toList());
+    try {
+      int threadPoolSize = conf.getInt(
+          ReconConfigKeys.OZONE_RECON_JMX_FETCH_THREAD_POOL_SIZE,
+          ReconConfigKeys.OZONE_RECON_JMX_FETCH_THREAD_POOL_SIZE_DEFAULT);
+      ExecutorService executor = Executors.newFixedThreadPool(threadPoolSize);
+      List<Future<JmxMetricsCollectorTaskResult>> results = 
executor.invokeAll(tasks);

Review Comment:
   Concerns below are :
   
     1.  Insufficient error handling - Silent failures, no differentiation 
between timeout/network/parse errors
     2. Missing observability - No metrics for success/failure rates, latency



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/JmxMetricsCollectorTask.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.Callable;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.recon.ReconConfigKeys;
+
+/**
+ * JmxMetricsCollectorTask is a Callable implementation that retrieves specific
+ * JMX metrics from a given DataNode's HTTP JMX endpoint. It fetches the 
metrics
+ * for a given service and metric name, parses the response JSON, and extracts
+ * the desired metric value.
+ * This task is primarily designed to collect metrics in a concurrent manner 
and
+ * return the results wrapped in a JmxMetricsCollectorTaskResult object, which
+ * contains the DataNode details and the fetched metric value.
+ */
+public class JmxMetricsCollectorTask implements 
Callable<JmxMetricsCollectorTaskResult> {
+
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+  private final OzoneConfiguration configuration;
+  private final DatanodeDetails datanodeDetails;
+  private final String componentName;
+  private final String serviceName;
+  private final String keyName;
+
+  public JmxMetricsCollectorTask(OzoneConfiguration configuration, 
DatanodeDetails datanodeDetails,
+                                 String componentName, String serviceName, 
String keyName) {
+    this.datanodeDetails = datanodeDetails;
+    this.componentName = componentName;
+    this.serviceName = serviceName;
+    this.keyName = keyName;
+    this.configuration = configuration;
+  }
+
+  @Override
+  public JmxMetricsCollectorTaskResult call() throws Exception {
+    return getMetricsFromDatanode();
+  }
+
+  private JmxMetricsCollectorTaskResult getMetricsFromDatanode() throws 
IOException {
+    if (datanodeDetails == null) {
+      throw new IOException("DataNode details are null");
+    }
+    HttpURLConnection connection = null;
+    try {
+      // Use standard Java HttpURLConnection (compatible with all HTTP 
implementations)
+      URL url = new URL(getJmxMetricsUrl());
+      connection = (HttpURLConnection) url.openConnection();

Review Comment:
   **Issue:** Better to use `HttpURLConnection` instead of 
`CloseableHttpClient` 
   If you use `HttpURLConnection`, then it has following problems:
   
   1. No connection pooling (creates new connection per request)
   2. Less robust timeout handling
   3. No automatic retry logic
   
   **Recommendation:** Switch to `CloseableHttpClient` used elsewhere in Ozone:
   
   ```
   try (CloseableHttpClient httpClient = HttpClients.custom()
           .setDefaultRequestConfig(RequestConfig.custom()
               .setConnectTimeout(connectTimeout)
               .setSocketTimeout(socketTimeout)
               .build())
           .build()) {
       // ... make request
   }
   ```



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/StorageDistributionEndpoint.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.KEY_TABLE;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Response;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import 
org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary;
+import org.apache.hadoop.hdds.recon.ReconConfigKeys;
+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.protocol.StorageContainerLocationProtocol;
+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.DatanodeStorageReport;
+import org.apache.hadoop.ozone.recon.api.types.DeletionPendingBytesByComponent;
+import org.apache.hadoop.ozone.recon.api.types.GlobalNamespaceReport;
+import org.apache.hadoop.ozone.recon.api.types.GlobalStorageReport;
+import org.apache.hadoop.ozone.recon.api.types.KeyInsightInfoResponse;
+import 
org.apache.hadoop.ozone.recon.api.types.StorageCapacityDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.UsedSpaceBreakDown;
+import org.apache.hadoop.ozone.recon.scm.ReconNodeManager;
+import org.apache.hadoop.ozone.recon.spi.ReconGlobalStatsManager;
+import org.apache.hadoop.ozone.recon.tasks.GlobalStatsValue;
+import org.apache.hadoop.ozone.recon.tasks.JmxMetricsCollectorTask;
+import org.apache.hadoop.ozone.recon.tasks.JmxMetricsCollectorTaskResult;
+import org.apache.hadoop.ozone.recon.tasks.OmTableInsightTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This endpoint handles requests related to storage distribution across
+ * different datanodes in a Recon instance. It provides detailed reports
+ * on storage capacity, utilization, and associated metrics.
+ * <p>
+ * The data is aggregated from multiple sources, including node manager
+ * statistics, and is used to construct responses with information
+ * about global storage and namespace usage, storage usage breakdown,
+ * and deletion operations in progress.
+ * <p>
+ * An instance of {@link ReconNodeManager} is used to fetch detailed
+ * node-specific statistics required for generating the report.
+ */
+@Path("/storageDistribution")
+@Produces("application/json")
+@AdminOnly
+public class StorageDistributionEndpoint {
+  private final ReconNodeManager nodeManager;
+  private final NSSummaryEndpoint nsSummaryEndpoint;
+  private final StorageContainerLocationProtocol scmClient;
+  private static final Logger LOG = 
LoggerFactory.getLogger(StorageDistributionEndpoint.class);
+  private final Map<DatanodeDetails, Long> blockDeletionMetricsMap = new 
ConcurrentHashMap<>();
+  private final ReconGlobalStatsManager reconGlobalStatsManager;
+  private final ReconGlobalMetricsService reconGlobalMetricsService;
+  private final OzoneConfiguration conf;
+
+  @Inject
+  public StorageDistributionEndpoint(OzoneStorageContainerManager reconSCM,
+                                     NSSummaryEndpoint nsSummaryEndpoint,
+                                     ReconGlobalStatsManager 
reconGlobalStatsManager,
+                                     StorageContainerLocationProtocol 
scmClient,
+                                     ReconGlobalMetricsService 
reconGlobalMetricsService,
+                                     OzoneConfiguration conf) {
+    this.nodeManager = (ReconNodeManager) reconSCM.getScmNodeManager();
+    this.nsSummaryEndpoint = nsSummaryEndpoint;
+    this.scmClient = scmClient;
+    this.reconGlobalStatsManager = reconGlobalStatsManager;
+    this.reconGlobalMetricsService = reconGlobalMetricsService;
+    this.conf = conf;
+  }
+
+  @GET
+  public Response getStorageDistribution() {
+    try {
+      initializeBlockDeletionMetricsMap();
+      List<DatanodeStorageReport> nodeStorageReports = 
collectDatanodeReports();
+      GlobalStorageReport globalStorageReport = calculateGlobalStorageReport();
+
+      Map<String, Long> namespaceMetrics = new HashMap<>();
+      try {
+        namespaceMetrics = calculateNamespaceMetrics();
+      } catch (Exception e) {
+        LOG.error("Error calculating namespace metrics", e);
+        // Initialize with default values
+        namespaceMetrics.put("totalUsedNamespace", 0L);
+        namespaceMetrics.put("totalOpenKeySize", 0L);
+        namespaceMetrics.put("totalCommittedSize", 0L);
+        namespaceMetrics.put("pendingDirectorySize", 0L);
+        namespaceMetrics.put("pendingKeySize", 0L);
+        namespaceMetrics.put("totalKeys", 0L);
+      }
+
+      StorageCapacityDistributionResponse response = 
buildStorageDistributionResponse(
+              nodeStorageReports, globalStorageReport, namespaceMetrics);
+      return Response.ok(response).build();
+    } catch (Exception e) {
+      LOG.error("Error getting storage distribution", e);
+      return Response.status(Response.Status.INTERNAL_SERVER_ERROR)
+              .entity("Error retrieving storage distribution: " + 
e.getMessage())
+              .build();
+    }
+  }
+
+  private GlobalStorageReport calculateGlobalStorageReport() {
+    try {
+      SCMNodeStat stats = nodeManager.getStats();
+      if (stats == null) {
+        LOG.warn("Node manager stats are null, returning default values");
+        return new GlobalStorageReport(0L, 0L, 0L);
+      }
+
+      long scmUsed = stats.getScmUsed() != null ? stats.getScmUsed().get() : 
0L;
+      long remaining = stats.getRemaining() != null ? 
stats.getRemaining().get() : 0L;
+      long capacity = stats.getCapacity() != null ? stats.getCapacity().get() 
: 0L;
+
+      return new GlobalStorageReport(scmUsed, remaining, capacity);
+    } catch (Exception e) {
+      LOG.error("Error calculating global storage report", e);
+      return new GlobalStorageReport(0L, 0L, 0L);
+    }
+  }
+
+  private Map<String, Long> calculateNamespaceMetrics() throws IOException {
+    Map<String, Long> metrics = new HashMap<>();
+    Map<String, Long> totalPendingAtOmSide = calculatePendingSizes();
+    long totalOpenKeySize = calculateOpenKeySizes();
+    long totalCommittedSize = calculateCommittedSize();
+    long pendingDirectorySize = 
totalPendingAtOmSide.getOrDefault("pendingDirectorySize", 0L);
+    long pendingKeySize = totalPendingAtOmSide.getOrDefault("pendingKeySize", 
0L);
+    long totalUsedNamespace = pendingDirectorySize + pendingKeySize + 
totalOpenKeySize + totalCommittedSize;
+
+    long totalKeys = 0L;
+    // Keys from OBJECT_STORE buckets.
+    GlobalStatsValue keyRecord = reconGlobalStatsManager.getGlobalStatsValue(
+            OmTableInsightTask.getTableCountKeyFromTable(KEY_TABLE));
+    // Keys from FILE_SYSTEM_OPTIMIZED buckets
+    GlobalStatsValue fileRecord = reconGlobalStatsManager.getGlobalStatsValue(
+            OmTableInsightTask.getTableCountKeyFromTable(FILE_TABLE));
+    if (keyRecord != null) {
+      totalKeys += keyRecord.getValue();
+    }
+    if (fileRecord != null) {
+      totalKeys += fileRecord.getValue();
+    }
+
+    metrics.put("pendingDirectorySize", pendingDirectorySize);
+    metrics.put("pendingKeySize", pendingKeySize);
+    metrics.put("totalOpenKeySize", totalOpenKeySize);
+    metrics.put("totalCommittedSize", totalCommittedSize);
+    metrics.put("totalUsedNamespace", totalUsedNamespace);
+    metrics.put("totalKeys", totalKeys);
+    return metrics;
+  }
+
+  private StorageCapacityDistributionResponse buildStorageDistributionResponse(
+          List<DatanodeStorageReport> nodeStorageReports,
+          GlobalStorageReport storageMetrics,
+          Map<String, Long> namespaceMetrics) {
+    DeletedBlocksTransactionSummary scmSummary = null;
+    try {
+      scmSummary = scmClient.getDeletedBlockSummary();
+    } catch (IOException e) {
+      LOG.error("Failed to get deleted block summary from SCM", e);
+    }
+
+    long totalPendingAtDnSide = 0L;
+    try {
+      totalPendingAtDnSide = 
blockDeletionMetricsMap.values().stream().reduce(0L, Long::sum);
+    } catch (Exception e) {
+      LOG.error("Error calculating pending deletion metrics", e);
+    }
+
+    DeletionPendingBytesByComponent deletionPendingBytesByStage =
+            createDeletionPendingBytesByStage(
+                    namespaceMetrics.getOrDefault("pendingDirectorySize", 0L),
+                    namespaceMetrics.getOrDefault("pendingKeySize", 0L),
+                    scmSummary != null ? 
scmSummary.getTotalBlockReplicatedSize() : 0L,
+                    totalPendingAtDnSide);
+
+    // Safely get values from namespaceMetrics with null checks
+    Long totalUsedNamespace = namespaceMetrics.get("totalUsedNamespace");
+    Long totalOpenKeySize = namespaceMetrics.get("totalOpenKeySize");
+    Long totalCommittedSize = namespaceMetrics.get("totalCommittedSize");
+    Long totalKeys = namespaceMetrics.get("totalKeys");
+    Long totalContainerPreAllocated = nodeStorageReports.stream()
+        .map(DatanodeStorageReport::getCommitted)
+        .reduce(0L, Long::sum);
+
+    return StorageCapacityDistributionResponse.newBuilder()
+            .setDataNodeUsage(nodeStorageReports)
+            .setGlobalStorage(storageMetrics)
+            .setGlobalNamespace(new GlobalNamespaceReport(
+                    totalUsedNamespace != null ? totalUsedNamespace : 0L,
+                    totalKeys != null ? totalKeys : 0L))
+            .setUsedSpaceBreakDown(new UsedSpaceBreakDown(
+                    totalOpenKeySize != null ? totalOpenKeySize : 0L,
+                    totalCommittedSize != null ? totalCommittedSize : 0L, 
totalContainerPreAllocated,
+                    deletionPendingBytesByStage))
+            .build();
+  }
+
+  private List<DatanodeStorageReport> collectDatanodeReports() {
+    return nodeManager.getAllNodes().stream()
+        .map(this::getStorageReport)
+        .filter(Objects::nonNull) // Filter out null reports
+        .collect(Collectors.toList());
+  }
+
+  private Map<String, Long> calculatePendingSizes() {
+    Map<String, Long> result = new HashMap<>();
+    KeyInsightInfoResponse response = 
reconGlobalMetricsService.getPendingForDeletionDirInfo(-1, "");
+    Map<String, Long> pendingKeySize = 
reconGlobalMetricsService.getDeletedKeySummary();
+    result.put("pendingDirectorySize", response.getReplicatedDataSize());
+    result.put("pendingKeySize", 
pendingKeySize.getOrDefault("totalReplicatedDataSize", 0L));
+    return result;
+  }
+
+  private long calculateOpenKeySizes() {
+    Map<String, Long> openKeySummary = 
reconGlobalMetricsService.getOpenKeySummary();
+    Map<String, Long> openKeyMPUSummary = 
reconGlobalMetricsService.getMPUKeySummary();
+    long openKeyDataSize = 
openKeySummary.getOrDefault("totalReplicatedDataSize", 0L);
+    long totalMPUKeySize = 
openKeyMPUSummary.getOrDefault("totalReplicatedDataSize", 0L);
+    return openKeyDataSize + totalMPUKeySize;
+  }
+
+  private long calculateCommittedSize() {
+    try {
+      Response rootResponse = nsSummaryEndpoint.getDiskUsage("/", false, true, 
false);
+      if (rootResponse.getStatus() != Response.Status.OK.getStatusCode()) {
+        LOG.warn("Failed to get disk usage, status: {}", 
rootResponse.getStatus());
+        return 0L;
+      }
+      DUResponse duRootRes = (DUResponse) rootResponse.getEntity();
+      return duRootRes != null ? duRootRes.getSizeWithReplica() : 0L;
+    } catch (IOException e) {
+      LOG.error("IOException while calculating committed size", e);
+      return 0L;
+    }
+  }
+
+  private DeletionPendingBytesByComponent 
createDeletionPendingBytesByStage(long pendingDirectorySize,
+                                                                        long 
pendingKeySize,
+                                                                        long 
totalPendingAtScmSide,
+                                                                        long 
totalPendingAtDnSide) {
+    long totalPending = pendingDirectorySize + pendingKeySize + 
totalPendingAtScmSide + totalPendingAtDnSide;
+    Map<String, Map<String, Long>> stageItems = new HashMap<>();
+    Map<String, Long> omMap = new HashMap<>();
+    omMap.put("totalBytes", pendingDirectorySize + pendingKeySize);
+    omMap.put("pendingDirectoryBytes", pendingDirectorySize);
+    omMap.put("pendingKeyBytes", pendingKeySize);
+    Map<String, Long> scmMap = new HashMap<>();
+    scmMap.put("pendingBytes", totalPendingAtScmSide);
+    Map<String, Long> dnMap = new HashMap<>();
+    dnMap.put("pendingBytes", totalPendingAtDnSide);
+    stageItems.put("OM", omMap);
+    stageItems.put("SCM", scmMap);
+    stageItems.put("DN", dnMap);
+    return new DeletionPendingBytesByComponent(totalPending, stageItems);
+  }
+
+  private void initializeBlockDeletionMetricsMap() {
+    List<JmxMetricsCollectorTask> tasks = 
nodeManager.getNodeStats().keySet().stream().map(dn ->
+      new JmxMetricsCollectorTask(conf, dn, "HddsDatanode",
+          "BlockDeletingService", "TotalPendingBlockBytes"))
+        .collect(Collectors.toList());
+    try {
+      int threadPoolSize = conf.getInt(
+          ReconConfigKeys.OZONE_RECON_JMX_FETCH_THREAD_POOL_SIZE,
+          ReconConfigKeys.OZONE_RECON_JMX_FETCH_THREAD_POOL_SIZE_DEFAULT);

Review Comment:
   This is not needed to be called in every API call request.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/StorageDistributionEndpoint.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.KEY_TABLE;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Response;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import 
org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary;
+import org.apache.hadoop.hdds.recon.ReconConfigKeys;
+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.protocol.StorageContainerLocationProtocol;
+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.DatanodeStorageReport;
+import org.apache.hadoop.ozone.recon.api.types.DeletionPendingBytesByComponent;
+import org.apache.hadoop.ozone.recon.api.types.GlobalNamespaceReport;
+import org.apache.hadoop.ozone.recon.api.types.GlobalStorageReport;
+import org.apache.hadoop.ozone.recon.api.types.KeyInsightInfoResponse;
+import 
org.apache.hadoop.ozone.recon.api.types.StorageCapacityDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.UsedSpaceBreakDown;
+import org.apache.hadoop.ozone.recon.scm.ReconNodeManager;
+import org.apache.hadoop.ozone.recon.spi.ReconGlobalStatsManager;
+import org.apache.hadoop.ozone.recon.tasks.GlobalStatsValue;
+import org.apache.hadoop.ozone.recon.tasks.JmxMetricsCollectorTask;
+import org.apache.hadoop.ozone.recon.tasks.JmxMetricsCollectorTaskResult;
+import org.apache.hadoop.ozone.recon.tasks.OmTableInsightTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This endpoint handles requests related to storage distribution across
+ * different datanodes in a Recon instance. It provides detailed reports
+ * on storage capacity, utilization, and associated metrics.
+ * <p>
+ * The data is aggregated from multiple sources, including node manager
+ * statistics, and is used to construct responses with information
+ * about global storage and namespace usage, storage usage breakdown,
+ * and deletion operations in progress.
+ * <p>
+ * An instance of {@link ReconNodeManager} is used to fetch detailed
+ * node-specific statistics required for generating the report.
+ */
+@Path("/storageDistribution")
+@Produces("application/json")
+@AdminOnly
+public class StorageDistributionEndpoint {
+  private final ReconNodeManager nodeManager;
+  private final NSSummaryEndpoint nsSummaryEndpoint;
+  private final StorageContainerLocationProtocol scmClient;
+  private static final Logger LOG = 
LoggerFactory.getLogger(StorageDistributionEndpoint.class);
+  private final Map<DatanodeDetails, Long> blockDeletionMetricsMap = new 
ConcurrentHashMap<>();
+  private final ReconGlobalStatsManager reconGlobalStatsManager;
+  private final ReconGlobalMetricsService reconGlobalMetricsService;
+  private final OzoneConfiguration conf;
+
+  @Inject
+  public StorageDistributionEndpoint(OzoneStorageContainerManager reconSCM,
+                                     NSSummaryEndpoint nsSummaryEndpoint,
+                                     ReconGlobalStatsManager 
reconGlobalStatsManager,
+                                     StorageContainerLocationProtocol 
scmClient,
+                                     ReconGlobalMetricsService 
reconGlobalMetricsService,
+                                     OzoneConfiguration conf) {
+    this.nodeManager = (ReconNodeManager) reconSCM.getScmNodeManager();
+    this.nsSummaryEndpoint = nsSummaryEndpoint;
+    this.scmClient = scmClient;
+    this.reconGlobalStatsManager = reconGlobalStatsManager;
+    this.reconGlobalMetricsService = reconGlobalMetricsService;
+    this.conf = conf;
+  }
+
+  @GET
+  public Response getStorageDistribution() {
+    try {
+      initializeBlockDeletionMetricsMap();
+      List<DatanodeStorageReport> nodeStorageReports = 
collectDatanodeReports();
+      GlobalStorageReport globalStorageReport = calculateGlobalStorageReport();
+
+      Map<String, Long> namespaceMetrics = new HashMap<>();
+      try {
+        namespaceMetrics = calculateNamespaceMetrics();
+      } catch (Exception e) {
+        LOG.error("Error calculating namespace metrics", e);
+        // Initialize with default values
+        namespaceMetrics.put("totalUsedNamespace", 0L);
+        namespaceMetrics.put("totalOpenKeySize", 0L);
+        namespaceMetrics.put("totalCommittedSize", 0L);
+        namespaceMetrics.put("pendingDirectorySize", 0L);
+        namespaceMetrics.put("pendingKeySize", 0L);
+        namespaceMetrics.put("totalKeys", 0L);
+      }
+
+      StorageCapacityDistributionResponse response = 
buildStorageDistributionResponse(
+              nodeStorageReports, globalStorageReport, namespaceMetrics);
+      return Response.ok(response).build();
+    } catch (Exception e) {
+      LOG.error("Error getting storage distribution", e);
+      return Response.status(Response.Status.INTERNAL_SERVER_ERROR)
+              .entity("Error retrieving storage distribution: " + 
e.getMessage())
+              .build();
+    }
+  }
+
+  private GlobalStorageReport calculateGlobalStorageReport() {
+    try {
+      SCMNodeStat stats = nodeManager.getStats();
+      if (stats == null) {
+        LOG.warn("Node manager stats are null, returning default values");
+        return new GlobalStorageReport(0L, 0L, 0L);
+      }
+
+      long scmUsed = stats.getScmUsed() != null ? stats.getScmUsed().get() : 
0L;
+      long remaining = stats.getRemaining() != null ? 
stats.getRemaining().get() : 0L;
+      long capacity = stats.getCapacity() != null ? stats.getCapacity().get() 
: 0L;
+
+      return new GlobalStorageReport(scmUsed, remaining, capacity);
+    } catch (Exception e) {
+      LOG.error("Error calculating global storage report", e);
+      return new GlobalStorageReport(0L, 0L, 0L);
+    }
+  }
+
+  private Map<String, Long> calculateNamespaceMetrics() throws IOException {
+    Map<String, Long> metrics = new HashMap<>();
+    Map<String, Long> totalPendingAtOmSide = calculatePendingSizes();
+    long totalOpenKeySize = calculateOpenKeySizes();
+    long totalCommittedSize = calculateCommittedSize();
+    long pendingDirectorySize = 
totalPendingAtOmSide.getOrDefault("pendingDirectorySize", 0L);
+    long pendingKeySize = totalPendingAtOmSide.getOrDefault("pendingKeySize", 
0L);
+    long totalUsedNamespace = pendingDirectorySize + pendingKeySize + 
totalOpenKeySize + totalCommittedSize;
+
+    long totalKeys = 0L;
+    // Keys from OBJECT_STORE buckets.
+    GlobalStatsValue keyRecord = reconGlobalStatsManager.getGlobalStatsValue(
+            OmTableInsightTask.getTableCountKeyFromTable(KEY_TABLE));
+    // Keys from FILE_SYSTEM_OPTIMIZED buckets
+    GlobalStatsValue fileRecord = reconGlobalStatsManager.getGlobalStatsValue(
+            OmTableInsightTask.getTableCountKeyFromTable(FILE_TABLE));
+    if (keyRecord != null) {
+      totalKeys += keyRecord.getValue();
+    }
+    if (fileRecord != null) {
+      totalKeys += fileRecord.getValue();
+    }
+
+    metrics.put("pendingDirectorySize", pendingDirectorySize);
+    metrics.put("pendingKeySize", pendingKeySize);
+    metrics.put("totalOpenKeySize", totalOpenKeySize);
+    metrics.put("totalCommittedSize", totalCommittedSize);
+    metrics.put("totalUsedNamespace", totalUsedNamespace);
+    metrics.put("totalKeys", totalKeys);
+    return metrics;
+  }
+
+  private StorageCapacityDistributionResponse buildStorageDistributionResponse(
+          List<DatanodeStorageReport> nodeStorageReports,
+          GlobalStorageReport storageMetrics,
+          Map<String, Long> namespaceMetrics) {
+    DeletedBlocksTransactionSummary scmSummary = null;
+    try {
+      scmSummary = scmClient.getDeletedBlockSummary();
+    } catch (IOException e) {
+      LOG.error("Failed to get deleted block summary from SCM", e);
+    }
+
+    long totalPendingAtDnSide = 0L;
+    try {
+      totalPendingAtDnSide = 
blockDeletionMetricsMap.values().stream().reduce(0L, Long::sum);
+    } catch (Exception e) {
+      LOG.error("Error calculating pending deletion metrics", e);
+    }
+
+    DeletionPendingBytesByComponent deletionPendingBytesByStage =
+            createDeletionPendingBytesByStage(
+                    namespaceMetrics.getOrDefault("pendingDirectorySize", 0L),
+                    namespaceMetrics.getOrDefault("pendingKeySize", 0L),
+                    scmSummary != null ? 
scmSummary.getTotalBlockReplicatedSize() : 0L,
+                    totalPendingAtDnSide);
+
+    // Safely get values from namespaceMetrics with null checks
+    Long totalUsedNamespace = namespaceMetrics.get("totalUsedNamespace");
+    Long totalOpenKeySize = namespaceMetrics.get("totalOpenKeySize");
+    Long totalCommittedSize = namespaceMetrics.get("totalCommittedSize");
+    Long totalKeys = namespaceMetrics.get("totalKeys");
+    Long totalContainerPreAllocated = nodeStorageReports.stream()
+        .map(DatanodeStorageReport::getCommitted)
+        .reduce(0L, Long::sum);
+
+    return StorageCapacityDistributionResponse.newBuilder()
+            .setDataNodeUsage(nodeStorageReports)
+            .setGlobalStorage(storageMetrics)
+            .setGlobalNamespace(new GlobalNamespaceReport(
+                    totalUsedNamespace != null ? totalUsedNamespace : 0L,
+                    totalKeys != null ? totalKeys : 0L))
+            .setUsedSpaceBreakDown(new UsedSpaceBreakDown(
+                    totalOpenKeySize != null ? totalOpenKeySize : 0L,
+                    totalCommittedSize != null ? totalCommittedSize : 0L, 
totalContainerPreAllocated,
+                    deletionPendingBytesByStage))
+            .build();
+  }
+
+  private List<DatanodeStorageReport> collectDatanodeReports() {
+    return nodeManager.getAllNodes().stream()
+        .map(this::getStorageReport)
+        .filter(Objects::nonNull) // Filter out null reports
+        .collect(Collectors.toList());
+  }
+
+  private Map<String, Long> calculatePendingSizes() {
+    Map<String, Long> result = new HashMap<>();
+    KeyInsightInfoResponse response = 
reconGlobalMetricsService.getPendingForDeletionDirInfo(-1, "");
+    Map<String, Long> pendingKeySize = 
reconGlobalMetricsService.getDeletedKeySummary();
+    result.put("pendingDirectorySize", response.getReplicatedDataSize());
+    result.put("pendingKeySize", 
pendingKeySize.getOrDefault("totalReplicatedDataSize", 0L));
+    return result;
+  }
+
+  private long calculateOpenKeySizes() {
+    Map<String, Long> openKeySummary = 
reconGlobalMetricsService.getOpenKeySummary();
+    Map<String, Long> openKeyMPUSummary = 
reconGlobalMetricsService.getMPUKeySummary();
+    long openKeyDataSize = 
openKeySummary.getOrDefault("totalReplicatedDataSize", 0L);
+    long totalMPUKeySize = 
openKeyMPUSummary.getOrDefault("totalReplicatedDataSize", 0L);
+    return openKeyDataSize + totalMPUKeySize;
+  }
+
+  private long calculateCommittedSize() {
+    try {
+      Response rootResponse = nsSummaryEndpoint.getDiskUsage("/", false, true, 
false);
+      if (rootResponse.getStatus() != Response.Status.OK.getStatusCode()) {
+        LOG.warn("Failed to get disk usage, status: {}", 
rootResponse.getStatus());
+        return 0L;
+      }
+      DUResponse duRootRes = (DUResponse) rootResponse.getEntity();
+      return duRootRes != null ? duRootRes.getSizeWithReplica() : 0L;
+    } catch (IOException e) {
+      LOG.error("IOException while calculating committed size", e);
+      return 0L;
+    }
+  }
+
+  private DeletionPendingBytesByComponent 
createDeletionPendingBytesByStage(long pendingDirectorySize,
+                                                                        long 
pendingKeySize,
+                                                                        long 
totalPendingAtScmSide,
+                                                                        long 
totalPendingAtDnSide) {
+    long totalPending = pendingDirectorySize + pendingKeySize + 
totalPendingAtScmSide + totalPendingAtDnSide;
+    Map<String, Map<String, Long>> stageItems = new HashMap<>();
+    Map<String, Long> omMap = new HashMap<>();
+    omMap.put("totalBytes", pendingDirectorySize + pendingKeySize);
+    omMap.put("pendingDirectoryBytes", pendingDirectorySize);
+    omMap.put("pendingKeyBytes", pendingKeySize);
+    Map<String, Long> scmMap = new HashMap<>();
+    scmMap.put("pendingBytes", totalPendingAtScmSide);
+    Map<String, Long> dnMap = new HashMap<>();
+    dnMap.put("pendingBytes", totalPendingAtDnSide);
+    stageItems.put("OM", omMap);
+    stageItems.put("SCM", scmMap);
+    stageItems.put("DN", dnMap);
+    return new DeletionPendingBytesByComponent(totalPending, stageItems);
+  }
+
+  private void initializeBlockDeletionMetricsMap() {
+    List<JmxMetricsCollectorTask> tasks = 
nodeManager.getNodeStats().keySet().stream().map(dn ->
+      new JmxMetricsCollectorTask(conf, dn, "HddsDatanode",
+          "BlockDeletingService", "TotalPendingBlockBytes"))
+        .collect(Collectors.toList());
+    try {
+      int threadPoolSize = conf.getInt(
+          ReconConfigKeys.OZONE_RECON_JMX_FETCH_THREAD_POOL_SIZE,
+          ReconConfigKeys.OZONE_RECON_JMX_FETCH_THREAD_POOL_SIZE_DEFAULT);
+      ExecutorService executor = Executors.newFixedThreadPool(threadPoolSize);

Review Comment:
   This is not needed to be called in every API call request.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/StorageDistributionEndpoint.java:
##########
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.api;
+
+import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.FILE_TABLE;
+import static org.apache.hadoop.ozone.om.codec.OMDBDefinition.KEY_TABLE;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Response;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import 
org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary;
+import org.apache.hadoop.hdds.recon.ReconConfigKeys;
+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.protocol.StorageContainerLocationProtocol;
+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.DatanodeStorageReport;
+import org.apache.hadoop.ozone.recon.api.types.DeletionPendingBytesByComponent;
+import org.apache.hadoop.ozone.recon.api.types.GlobalNamespaceReport;
+import org.apache.hadoop.ozone.recon.api.types.GlobalStorageReport;
+import org.apache.hadoop.ozone.recon.api.types.KeyInsightInfoResponse;
+import 
org.apache.hadoop.ozone.recon.api.types.StorageCapacityDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.UsedSpaceBreakDown;
+import org.apache.hadoop.ozone.recon.scm.ReconNodeManager;
+import org.apache.hadoop.ozone.recon.spi.ReconGlobalStatsManager;
+import org.apache.hadoop.ozone.recon.tasks.GlobalStatsValue;
+import org.apache.hadoop.ozone.recon.tasks.JmxMetricsCollectorTask;
+import org.apache.hadoop.ozone.recon.tasks.JmxMetricsCollectorTaskResult;
+import org.apache.hadoop.ozone.recon.tasks.OmTableInsightTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This endpoint handles requests related to storage distribution across
+ * different datanodes in a Recon instance. It provides detailed reports
+ * on storage capacity, utilization, and associated metrics.
+ * <p>
+ * The data is aggregated from multiple sources, including node manager
+ * statistics, and is used to construct responses with information
+ * about global storage and namespace usage, storage usage breakdown,
+ * and deletion operations in progress.
+ * <p>
+ * An instance of {@link ReconNodeManager} is used to fetch detailed
+ * node-specific statistics required for generating the report.
+ */
+@Path("/storageDistribution")
+@Produces("application/json")
+@AdminOnly
+public class StorageDistributionEndpoint {
+  private final ReconNodeManager nodeManager;
+  private final NSSummaryEndpoint nsSummaryEndpoint;
+  private final StorageContainerLocationProtocol scmClient;
+  private static final Logger LOG = 
LoggerFactory.getLogger(StorageDistributionEndpoint.class);
+  private final Map<DatanodeDetails, Long> blockDeletionMetricsMap = new 
ConcurrentHashMap<>();
+  private final ReconGlobalStatsManager reconGlobalStatsManager;
+  private final ReconGlobalMetricsService reconGlobalMetricsService;
+  private final OzoneConfiguration conf;
+
+  @Inject
+  public StorageDistributionEndpoint(OzoneStorageContainerManager reconSCM,
+                                     NSSummaryEndpoint nsSummaryEndpoint,
+                                     ReconGlobalStatsManager 
reconGlobalStatsManager,
+                                     StorageContainerLocationProtocol 
scmClient,
+                                     ReconGlobalMetricsService 
reconGlobalMetricsService,
+                                     OzoneConfiguration conf) {
+    this.nodeManager = (ReconNodeManager) reconSCM.getScmNodeManager();
+    this.nsSummaryEndpoint = nsSummaryEndpoint;
+    this.scmClient = scmClient;
+    this.reconGlobalStatsManager = reconGlobalStatsManager;
+    this.reconGlobalMetricsService = reconGlobalMetricsService;
+    this.conf = conf;
+  }
+
+  @GET
+  public Response getStorageDistribution() {
+    try {
+      initializeBlockDeletionMetricsMap();
+      List<DatanodeStorageReport> nodeStorageReports = 
collectDatanodeReports();
+      GlobalStorageReport globalStorageReport = calculateGlobalStorageReport();
+
+      Map<String, Long> namespaceMetrics = new HashMap<>();
+      try {
+        namespaceMetrics = calculateNamespaceMetrics();
+      } catch (Exception e) {
+        LOG.error("Error calculating namespace metrics", e);
+        // Initialize with default values
+        namespaceMetrics.put("totalUsedNamespace", 0L);
+        namespaceMetrics.put("totalOpenKeySize", 0L);
+        namespaceMetrics.put("totalCommittedSize", 0L);
+        namespaceMetrics.put("pendingDirectorySize", 0L);
+        namespaceMetrics.put("pendingKeySize", 0L);
+        namespaceMetrics.put("totalKeys", 0L);
+      }
+
+      StorageCapacityDistributionResponse response = 
buildStorageDistributionResponse(
+              nodeStorageReports, globalStorageReport, namespaceMetrics);
+      return Response.ok(response).build();
+    } catch (Exception e) {
+      LOG.error("Error getting storage distribution", e);
+      return Response.status(Response.Status.INTERNAL_SERVER_ERROR)
+              .entity("Error retrieving storage distribution: " + 
e.getMessage())
+              .build();
+    }
+  }
+
+  private GlobalStorageReport calculateGlobalStorageReport() {
+    try {
+      SCMNodeStat stats = nodeManager.getStats();
+      if (stats == null) {
+        LOG.warn("Node manager stats are null, returning default values");
+        return new GlobalStorageReport(0L, 0L, 0L);
+      }
+
+      long scmUsed = stats.getScmUsed() != null ? stats.getScmUsed().get() : 
0L;
+      long remaining = stats.getRemaining() != null ? 
stats.getRemaining().get() : 0L;
+      long capacity = stats.getCapacity() != null ? stats.getCapacity().get() 
: 0L;
+
+      return new GlobalStorageReport(scmUsed, remaining, capacity);
+    } catch (Exception e) {
+      LOG.error("Error calculating global storage report", e);
+      return new GlobalStorageReport(0L, 0L, 0L);
+    }
+  }
+
+  private Map<String, Long> calculateNamespaceMetrics() throws IOException {
+    Map<String, Long> metrics = new HashMap<>();
+    Map<String, Long> totalPendingAtOmSide = calculatePendingSizes();
+    long totalOpenKeySize = calculateOpenKeySizes();
+    long totalCommittedSize = calculateCommittedSize();
+    long pendingDirectorySize = 
totalPendingAtOmSide.getOrDefault("pendingDirectorySize", 0L);
+    long pendingKeySize = totalPendingAtOmSide.getOrDefault("pendingKeySize", 
0L);
+    long totalUsedNamespace = pendingDirectorySize + pendingKeySize + 
totalOpenKeySize + totalCommittedSize;
+
+    long totalKeys = 0L;
+    // Keys from OBJECT_STORE buckets.
+    GlobalStatsValue keyRecord = reconGlobalStatsManager.getGlobalStatsValue(
+            OmTableInsightTask.getTableCountKeyFromTable(KEY_TABLE));
+    // Keys from FILE_SYSTEM_OPTIMIZED buckets
+    GlobalStatsValue fileRecord = reconGlobalStatsManager.getGlobalStatsValue(
+            OmTableInsightTask.getTableCountKeyFromTable(FILE_TABLE));
+    if (keyRecord != null) {
+      totalKeys += keyRecord.getValue();
+    }
+    if (fileRecord != null) {
+      totalKeys += fileRecord.getValue();
+    }
+
+    metrics.put("pendingDirectorySize", pendingDirectorySize);
+    metrics.put("pendingKeySize", pendingKeySize);
+    metrics.put("totalOpenKeySize", totalOpenKeySize);
+    metrics.put("totalCommittedSize", totalCommittedSize);
+    metrics.put("totalUsedNamespace", totalUsedNamespace);
+    metrics.put("totalKeys", totalKeys);
+    return metrics;
+  }
+
+  private StorageCapacityDistributionResponse buildStorageDistributionResponse(
+          List<DatanodeStorageReport> nodeStorageReports,
+          GlobalStorageReport storageMetrics,
+          Map<String, Long> namespaceMetrics) {
+    DeletedBlocksTransactionSummary scmSummary = null;
+    try {
+      scmSummary = scmClient.getDeletedBlockSummary();
+    } catch (IOException e) {
+      LOG.error("Failed to get deleted block summary from SCM", e);
+    }
+
+    long totalPendingAtDnSide = 0L;
+    try {
+      totalPendingAtDnSide = 
blockDeletionMetricsMap.values().stream().reduce(0L, Long::sum);
+    } catch (Exception e) {
+      LOG.error("Error calculating pending deletion metrics", e);
+    }
+
+    DeletionPendingBytesByComponent deletionPendingBytesByStage =
+            createDeletionPendingBytesByStage(
+                    namespaceMetrics.getOrDefault("pendingDirectorySize", 0L),
+                    namespaceMetrics.getOrDefault("pendingKeySize", 0L),
+                    scmSummary != null ? 
scmSummary.getTotalBlockReplicatedSize() : 0L,
+                    totalPendingAtDnSide);
+
+    // Safely get values from namespaceMetrics with null checks
+    Long totalUsedNamespace = namespaceMetrics.get("totalUsedNamespace");
+    Long totalOpenKeySize = namespaceMetrics.get("totalOpenKeySize");
+    Long totalCommittedSize = namespaceMetrics.get("totalCommittedSize");
+    Long totalKeys = namespaceMetrics.get("totalKeys");
+    Long totalContainerPreAllocated = nodeStorageReports.stream()
+        .map(DatanodeStorageReport::getCommitted)
+        .reduce(0L, Long::sum);
+
+    return StorageCapacityDistributionResponse.newBuilder()
+            .setDataNodeUsage(nodeStorageReports)
+            .setGlobalStorage(storageMetrics)
+            .setGlobalNamespace(new GlobalNamespaceReport(
+                    totalUsedNamespace != null ? totalUsedNamespace : 0L,
+                    totalKeys != null ? totalKeys : 0L))
+            .setUsedSpaceBreakDown(new UsedSpaceBreakDown(
+                    totalOpenKeySize != null ? totalOpenKeySize : 0L,
+                    totalCommittedSize != null ? totalCommittedSize : 0L, 
totalContainerPreAllocated,
+                    deletionPendingBytesByStage))
+            .build();
+  }
+
+  private List<DatanodeStorageReport> collectDatanodeReports() {
+    return nodeManager.getAllNodes().stream()
+        .map(this::getStorageReport)
+        .filter(Objects::nonNull) // Filter out null reports
+        .collect(Collectors.toList());
+  }
+
+  private Map<String, Long> calculatePendingSizes() {
+    Map<String, Long> result = new HashMap<>();
+    KeyInsightInfoResponse response = 
reconGlobalMetricsService.getPendingForDeletionDirInfo(-1, "");
+    Map<String, Long> pendingKeySize = 
reconGlobalMetricsService.getDeletedKeySummary();
+    result.put("pendingDirectorySize", response.getReplicatedDataSize());
+    result.put("pendingKeySize", 
pendingKeySize.getOrDefault("totalReplicatedDataSize", 0L));
+    return result;
+  }
+
+  private long calculateOpenKeySizes() {
+    Map<String, Long> openKeySummary = 
reconGlobalMetricsService.getOpenKeySummary();
+    Map<String, Long> openKeyMPUSummary = 
reconGlobalMetricsService.getMPUKeySummary();
+    long openKeyDataSize = 
openKeySummary.getOrDefault("totalReplicatedDataSize", 0L);
+    long totalMPUKeySize = 
openKeyMPUSummary.getOrDefault("totalReplicatedDataSize", 0L);
+    return openKeyDataSize + totalMPUKeySize;
+  }
+
+  private long calculateCommittedSize() {
+    try {
+      Response rootResponse = nsSummaryEndpoint.getDiskUsage("/", false, true, 
false);
+      if (rootResponse.getStatus() != Response.Status.OK.getStatusCode()) {
+        LOG.warn("Failed to get disk usage, status: {}", 
rootResponse.getStatus());
+        return 0L;
+      }
+      DUResponse duRootRes = (DUResponse) rootResponse.getEntity();
+      return duRootRes != null ? duRootRes.getSizeWithReplica() : 0L;
+    } catch (IOException e) {
+      LOG.error("IOException while calculating committed size", e);
+      return 0L;
+    }
+  }
+
+  private DeletionPendingBytesByComponent 
createDeletionPendingBytesByStage(long pendingDirectorySize,
+                                                                        long 
pendingKeySize,
+                                                                        long 
totalPendingAtScmSide,
+                                                                        long 
totalPendingAtDnSide) {
+    long totalPending = pendingDirectorySize + pendingKeySize + 
totalPendingAtScmSide + totalPendingAtDnSide;
+    Map<String, Map<String, Long>> stageItems = new HashMap<>();
+    Map<String, Long> omMap = new HashMap<>();
+    omMap.put("totalBytes", pendingDirectorySize + pendingKeySize);
+    omMap.put("pendingDirectoryBytes", pendingDirectorySize);
+    omMap.put("pendingKeyBytes", pendingKeySize);
+    Map<String, Long> scmMap = new HashMap<>();
+    scmMap.put("pendingBytes", totalPendingAtScmSide);
+    Map<String, Long> dnMap = new HashMap<>();
+    dnMap.put("pendingBytes", totalPendingAtDnSide);
+    stageItems.put("OM", omMap);
+    stageItems.put("SCM", scmMap);
+    stageItems.put("DN", dnMap);
+    return new DeletionPendingBytesByComponent(totalPending, stageItems);
+  }
+
+  private void initializeBlockDeletionMetricsMap() {
+    List<JmxMetricsCollectorTask> tasks = 
nodeManager.getNodeStats().keySet().stream().map(dn ->
+      new JmxMetricsCollectorTask(conf, dn, "HddsDatanode",
+          "BlockDeletingService", "TotalPendingBlockBytes"))
+        .collect(Collectors.toList());
+    try {
+      int threadPoolSize = conf.getInt(
+          ReconConfigKeys.OZONE_RECON_JMX_FETCH_THREAD_POOL_SIZE,
+          ReconConfigKeys.OZONE_RECON_JMX_FETCH_THREAD_POOL_SIZE_DEFAULT);
+      ExecutorService executor = Executors.newFixedThreadPool(threadPoolSize);
+      List<Future<JmxMetricsCollectorTaskResult>> results = 
executor.invokeAll(tasks);

Review Comment:
   This solution is not scalable as mentioned earlier. In 1000 nodes cluster 
with just 10 threads in pool will take lot of time and API response will be 
very slow. Kindly implement the correct scalable solution. Also creating 
executor pool like this in every API request and not closing and releasing 
resources will create further issues:
   Every API call creates a new thread pool that's never shutdown. This will 
cause:
     - Thread exhaustion
     - Memory leaks
     - File descriptor leaks
     - Production outages
   
   Also above solution is not scalable , what if some DNs are slow, we don't 
have any mechanism to timeout ? If even timeout, will it do failure of whole 
API ? What is the way to handle ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to