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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/JmxServiceProviderImpl.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.spi.impl;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URLEncoder;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.inject.Singleton;
+import org.apache.hadoop.ozone.recon.metrics.Metric;
+import org.apache.hadoop.ozone.recon.spi.MetricsServiceProvider;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.concurrent.FutureCallback;
+import org.apache.http.impl.nio.client.CloseableHttpAsyncClient;
+import org.apache.http.impl.nio.client.HttpAsyncClients;
+import org.apache.http.impl.nio.conn.PoolingNHttpClientConnectionManager;
+import org.apache.http.impl.nio.reactor.DefaultConnectingIOReactor;
+import org.apache.http.impl.nio.reactor.IOReactorConfig;
+import org.apache.http.nio.reactor.ConnectingIOReactor;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of the Prometheus Metrics Service provider.
+ */
+@Singleton
+public class JmxServiceProviderImpl implements MetricsServiceProvider {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(JmxServiceProviderImpl.class);
+  private final CloseableHttpAsyncClient asyncHttpClient;
+  private final ObjectMapper objectMapper;
+
+  public static final String JMX_INSTANT_QUERY_API = "qry";
+  private static final int DEFAULT_TIMEOUT_MS = 60000;
+  private static final int MAX_TOTAL_CONNECTIONS = 100;
+  private static final int MAX_CONNECTIONS_PER_ROUTE = 10;
+
+  public JmxServiceProviderImpl() {
+    this.asyncHttpClient = createAsyncHttpClient();
+    this.asyncHttpClient.start();

Review Comment:
   Constructors shouldn't start background operations. If constructor fails 
after start(), cleanup doesn't happen. 100+ threads created (IO threads + 
connection pool) even if never used. So better use lazy initialization.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/JmxServiceProviderImpl.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.spi.impl;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URLEncoder;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.inject.Singleton;
+import org.apache.hadoop.ozone.recon.metrics.Metric;
+import org.apache.hadoop.ozone.recon.spi.MetricsServiceProvider;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.concurrent.FutureCallback;
+import org.apache.http.impl.nio.client.CloseableHttpAsyncClient;
+import org.apache.http.impl.nio.client.HttpAsyncClients;
+import org.apache.http.impl.nio.conn.PoolingNHttpClientConnectionManager;
+import org.apache.http.impl.nio.reactor.DefaultConnectingIOReactor;
+import org.apache.http.impl.nio.reactor.IOReactorConfig;
+import org.apache.http.nio.reactor.ConnectingIOReactor;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of the Prometheus Metrics Service provider.

Review Comment:
   Fix the comment here.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/JmxServiceProviderImpl.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.spi.impl;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URLEncoder;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.inject.Singleton;
+import org.apache.hadoop.ozone.recon.metrics.Metric;
+import org.apache.hadoop.ozone.recon.spi.MetricsServiceProvider;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.concurrent.FutureCallback;
+import org.apache.http.impl.nio.client.CloseableHttpAsyncClient;
+import org.apache.http.impl.nio.client.HttpAsyncClients;
+import org.apache.http.impl.nio.conn.PoolingNHttpClientConnectionManager;
+import org.apache.http.impl.nio.reactor.DefaultConnectingIOReactor;
+import org.apache.http.impl.nio.reactor.IOReactorConfig;
+import org.apache.http.nio.reactor.ConnectingIOReactor;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of the Prometheus Metrics Service provider.
+ */
+@Singleton
+public class JmxServiceProviderImpl implements MetricsServiceProvider {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(JmxServiceProviderImpl.class);
+  private final CloseableHttpAsyncClient asyncHttpClient;
+  private final ObjectMapper objectMapper;
+
+  public static final String JMX_INSTANT_QUERY_API = "qry";
+  private static final int DEFAULT_TIMEOUT_MS = 60000;
+  private static final int MAX_TOTAL_CONNECTIONS = 100;
+  private static final int MAX_CONNECTIONS_PER_ROUTE = 10;
+
+  public JmxServiceProviderImpl() {
+    this.asyncHttpClient = createAsyncHttpClient();
+    this.asyncHttpClient.start();
+    this.objectMapper = new ObjectMapper();
+  }
+
+  private CloseableHttpAsyncClient createAsyncHttpClient() {
+    try {
+      // Configure IO reactor for non-blocking I/O
+      IOReactorConfig ioReactorConfig = IOReactorConfig.custom()
+              .setConnectTimeout(DEFAULT_TIMEOUT_MS)
+              .setSoTimeout(DEFAULT_TIMEOUT_MS)
+              .setIoThreadCount(Runtime.getRuntime().availableProcessors())
+              .build();
+
+      ConnectingIOReactor ioReactor = new 
DefaultConnectingIOReactor(ioReactorConfig);
+      PoolingNHttpClientConnectionManager connManager =
+              new PoolingNHttpClientConnectionManager(ioReactor);
+      connManager.setMaxTotal(MAX_TOTAL_CONNECTIONS);
+      connManager.setDefaultMaxPerRoute(MAX_CONNECTIONS_PER_ROUTE);
+
+      RequestConfig requestConfig = RequestConfig.custom()
+              .setConnectTimeout(DEFAULT_TIMEOUT_MS)
+              .setSocketTimeout(DEFAULT_TIMEOUT_MS)
+              .setConnectionRequestTimeout(DEFAULT_TIMEOUT_MS)
+              .build();
+
+      return HttpAsyncClients.custom()
+              .setConnectionManager(connManager)
+              .setDefaultRequestConfig(requestConfig)
+              .build();
+
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create async HTTP client", e);
+    }
+  }
+
+  /**
+   * Returns {@link HttpURLConnection} after querying Metrics endpoint for the
+   * given metric.
+   *
+   * @param api         api.
+   * @param queryString query string with metric name and other filters.
+   * @return HttpURLConnection
+   */
+  @Override
+  public HttpURLConnection getMetricsResponse(String api, String queryString) {
+    return null;
+  }
+
+  @Override
+  public List<Metric> getMetricsInstant(String queryString) {
+    return Collections.emptyList();
+  }
+
+  /**
+   * Returns a list of {@link Metric} for the given instant query.
+   *
+   * @param queryString query string with metric name and other filters.
+   * @return List of Json map of metrics response.
+   */
+  @Override
+  public CompletableFuture<List<Map<String, Object>>> getMetricsAsync(String 
jmxEndpoint, String queryString) {
+    // Remove the trailing slash from endpoint url.
+    if (jmxEndpoint != null && jmxEndpoint.endsWith("/")) {
+      jmxEndpoint = jmxEndpoint.substring(0, jmxEndpoint.length() - 1);
+    }
+    return getMetrics(jmxEndpoint, queryString);
+  }
+
+  @Override
+  public void shutdown() {
+    try {
+      LOG.info("Shutting down async HTTP client...");
+      asyncHttpClient.close();
+    } catch (IOException e) {
+      LOG.error("Error shutting down async HTTP client", e);
+    }
+  }
+
+  /**
+   * Returns a list of {@link Metric} for the given api and query string.
+   *
+   * @param jmxEndpoint endpoint for the jmx server
+   * @param queryString query string with metric name and other filters.
+   * @return List of Json map of metrics response.
+   */
+  private CompletableFuture<List<Map<String, Object>>> getMetrics(String 
jmxEndpoint, String queryString) {
+    CompletableFuture<List<Map<String, Object>>> future = new 
CompletableFuture<>();
+
+    try {
+      String url = String.format(
+          "%s?%s=%s",
+          jmxEndpoint,
+          JMX_INSTANT_QUERY_API,
+          URLEncoder.encode(queryString, "UTF-8"));
+      HttpGet request = new HttpGet(url);
+
+      asyncHttpClient.execute(request, new FutureCallback<HttpResponse>() {
+        @Override
+        public void completed(HttpResponse response) {
+          try {
+            int statusCode = response.getStatusLine().getStatusCode();
+            if (statusCode >= 200 && statusCode < 300) {
+              String responseBody = EntityUtils.toString(response.getEntity());
+              Map<String, Object> jsonResponse = 
objectMapper.readValue(responseBody, Map.class);
+              List<Map<String, Object>> beans = (List<Map<String, Object>>) 
jsonResponse.get("beans");
+              future.complete(beans);
+            } else {
+              future.completeExceptionally(new IOException("HTTP error code: " 
+ statusCode));
+            }
+          } catch (Exception e) {
+            future.completeExceptionally(e);
+          }
+        }
+
+        @Override
+        public void failed(Exception ex) {
+          LOG.debug("HTTP request failed for {}: {}", url, ex.getMessage());

Review Comment:
   I feel, this may not be very bulk, so connection failures to datanodes 
should not be logged at DEBUG level. We can log at warn level atleast and log 
complete error trace instead of just message.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/DataNodeMetricsService.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.recon.ReconServerConfigKeys.OZONE_RECON_DN_METRICS_COLLECTION_MINIMUM_API_DELAY;
+import static 
org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_DN_METRICS_COLLECTION_MINIMUM_API_DELAY_DEFAULT;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+import javax.annotation.PreDestroy;
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
+import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
+import org.apache.hadoop.hdds.server.http.HttpConfig;
+import org.apache.hadoop.ozone.recon.MetricsServiceProviderFactory;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.DataNodeMetricsServiceResponse;
+import org.apache.hadoop.ozone.recon.api.types.DatanodePendingDeletionMetrics;
+import org.apache.hadoop.ozone.recon.scm.ReconNodeManager;
+import org.apache.hadoop.ozone.recon.spi.MetricsServiceProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Service for collecting and managing DataNode metrics asynchronously.
+ * Uses non-blocking HTTP client for efficient concurrent metric collection.
+ */
+@Singleton
+public class DataNodeMetricsService {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DataNodeMetricsService.class);
+  private static final String BEAN_NAME = 
"Hadoop:service=HddsDatanode,name=BlockDeletingService";
+  private static final String KEY_NAME = "TotalPendingBlockBytes";
+  private static final int DEFAULT_TIMEOUT = 60;
+
+  private final ReconNodeManager reconNodeManager;
+  private final boolean httpsEnabled;
+  private final long minimumApiDelayMs;
+  private final MetricsServiceProvider asyncServiceProvider;
+  // Immutable state holder for thread-safe access
+  private final AtomicReference<MetricsState> currentState =
+      new AtomicReference<>(new MetricsState());
+
+  @Inject
+  public DataNodeMetricsService(
+      OzoneStorageContainerManager reconSCM,
+      OzoneConfiguration config,
+      MetricsServiceProviderFactory metricsServiceProviderFactory) {
+
+    this.reconNodeManager = (ReconNodeManager) reconSCM.getScmNodeManager();
+    this.httpsEnabled = HttpConfig.getHttpPolicy(config).isHttpsEnabled();
+    this.minimumApiDelayMs = config.getTimeDuration(
+        OZONE_RECON_DN_METRICS_COLLECTION_MINIMUM_API_DELAY,
+        OZONE_RECON_DN_METRICS_COLLECTION_MINIMUM_API_DELAY_DEFAULT,
+        TimeUnit.MILLISECONDS);
+    this.asyncServiceProvider = 
metricsServiceProviderFactory.getJmxMetricsServiceProvider();
+  }
+
+  /**
+   * Starts an asynchronous metrics collection task if one is not already 
running.
+   * Returns immediately - use getCollectedMetrics() to check status and 
results.
+   */
+  public synchronized void startTask() {
+    MetricsState state = currentState.get();
+    // Check if already running
+    if (state.status == MetricCollectionStatus.IN_PROGRESS) {
+      LOG.warn("Metrics collection task is already in progress");
+      return;
+    }
+    // Check rate limit
+    if (state.lastCollectionEndTime > System.currentTimeMillis() - 
minimumApiDelayMs) {
+      LOG.info("Skipping metrics collection due to rate limit ({}ms)", 
minimumApiDelayMs);
+      return;
+    }
+    Set<DatanodeDetails> nodes = reconNodeManager.getNodeStats().keySet();
+    if (nodes.isEmpty()) {
+      LOG.warn("No datanodes found to query");
+      currentState.set(new MetricsState(
+          MetricCollectionStatus.SUCCEEDED,
+          new ArrayList<>(),
+          0L, 0, 0,
+          System.currentTimeMillis()
+      ));
+      return;
+    }
+    LOG.info("Starting async metrics collection for {} datanodes", 
nodes.size());
+    // Update state to IN_PROGRESS
+    currentState.set(new MetricsState(
+        MetricCollectionStatus.IN_PROGRESS,
+        new ArrayList<>(),
+        0L, 0, 0,
+        state.lastCollectionEndTime
+    ));
+    collectMetricsAsync(nodes);
+  }
+
+  /**
+   * Collects metrics from all datanodes asynchronously using 
CompletableFuture.
+   */
+  private void collectMetricsAsync(Set<DatanodeDetails> nodes) {
+    int totalNodes = nodes.size();
+
+    // Create futures for all nodes
+    List<CompletableFuture<DatanodePendingDeletionMetrics>> futures = 
nodes.stream()
+        .map(this::collectFromSingleDataNode)
+        .collect(Collectors.toList());
+
+    // Combine all futures
+    CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
+        .handle((result, throwable) -> {
+          // Collect all completed results (including failures marked as -1)
+          List<DatanodePendingDeletionMetrics> allResults = new ArrayList<>();
+          long totalPendingDeletion = 0L;
+          int failedCount = 0;
+
+          for (CompletableFuture<DatanodePendingDeletionMetrics> future : 
futures) {
+            try {
+              // getNow with default will return the result if completed, or 
default if not
+              DatanodePendingDeletionMetrics metrics = 
future.get(DEFAULT_TIMEOUT, TimeUnit.SECONDS);

Review Comment:
   This timeout is of no use and doesn't work as intended. `allOf()` waits for 
ALL futures to complete before invoking `handle()`. By the time 
`future.get(60s)` is called, all futures are already complete, so the timeout 
parameter is ignored. So, If any single future hangs indefinitely, then 
`allOf() `waits forever. `handle()` callback is never invoked. 
`future.get(timeout) `line is never reached. State stays as `IN_PROGRESS` 
permanently. So it is a permanent service degradation, One stuck network 
connection can permanently disable the entire metrics collection feature.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/JmxServiceProviderImpl.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.spi.impl;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URLEncoder;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.inject.Singleton;
+import org.apache.hadoop.ozone.recon.metrics.Metric;
+import org.apache.hadoop.ozone.recon.spi.MetricsServiceProvider;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.concurrent.FutureCallback;
+import org.apache.http.impl.nio.client.CloseableHttpAsyncClient;
+import org.apache.http.impl.nio.client.HttpAsyncClients;
+import org.apache.http.impl.nio.conn.PoolingNHttpClientConnectionManager;
+import org.apache.http.impl.nio.reactor.DefaultConnectingIOReactor;
+import org.apache.http.impl.nio.reactor.IOReactorConfig;
+import org.apache.http.nio.reactor.ConnectingIOReactor;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of the Prometheus Metrics Service provider.
+ */
+@Singleton
+public class JmxServiceProviderImpl implements MetricsServiceProvider {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(JmxServiceProviderImpl.class);
+  private final CloseableHttpAsyncClient asyncHttpClient;
+  private final ObjectMapper objectMapper;
+
+  public static final String JMX_INSTANT_QUERY_API = "qry";
+  private static final int DEFAULT_TIMEOUT_MS = 60000;
+  private static final int MAX_TOTAL_CONNECTIONS = 100;
+  private static final int MAX_CONNECTIONS_PER_ROUTE = 10;
+
+  public JmxServiceProviderImpl() {
+    this.asyncHttpClient = createAsyncHttpClient();
+    this.asyncHttpClient.start();
+    this.objectMapper = new ObjectMapper();
+  }
+
+  private CloseableHttpAsyncClient createAsyncHttpClient() {
+    try {
+      // Configure IO reactor for non-blocking I/O
+      IOReactorConfig ioReactorConfig = IOReactorConfig.custom()
+              .setConnectTimeout(DEFAULT_TIMEOUT_MS)
+              .setSoTimeout(DEFAULT_TIMEOUT_MS)
+              .setIoThreadCount(Runtime.getRuntime().availableProcessors())
+              .build();
+
+      ConnectingIOReactor ioReactor = new 
DefaultConnectingIOReactor(ioReactorConfig);
+      PoolingNHttpClientConnectionManager connManager =
+              new PoolingNHttpClientConnectionManager(ioReactor);
+      connManager.setMaxTotal(MAX_TOTAL_CONNECTIONS);
+      connManager.setDefaultMaxPerRoute(MAX_CONNECTIONS_PER_ROUTE);
+
+      RequestConfig requestConfig = RequestConfig.custom()
+              .setConnectTimeout(DEFAULT_TIMEOUT_MS)
+              .setSocketTimeout(DEFAULT_TIMEOUT_MS)
+              .setConnectionRequestTimeout(DEFAULT_TIMEOUT_MS)
+              .build();
+
+      return HttpAsyncClients.custom()

Review Comment:
   We are building jmx url based on http or https enabled. We should add SSL 
configuration options also because in real production cluster with certificate, 
could be an issue.. Have you tested this in secure cluster ?
   
   
   ```suggestion
         SSLContext sslContext = ...; // From configuration
   return HttpAsyncClients.custom()
     .setSSLContext(sslContext)
     .setSSLHostnameVerifier(new DefaultHostnameVerifier())  // Or configurable
     .setConnectionManager(connManager)
     .build();
   ```



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java:
##########
@@ -847,4 +847,34 @@ public static String constructObjectPathWithPrefix(long... 
ids) {
     }
     return pathBuilder.toString();
   }
+
+  public static Map<String, Object> getMetricsData(List<Map<String, Object>> 
metrics, String beanName) {
+    if (metrics == null || StringUtils.isEmpty(beanName)) {
+      return null;
+    }
+    for (Map<String, Object> item :metrics) {
+      if (item.get("name").equals(beanName)) {

Review Comment:
   ```suggestion
         if (beanName.equals(item.get("name"))) {
     return item;
   }
   ```



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/JmxServiceProviderImpl.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.spi.impl;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URLEncoder;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.inject.Singleton;
+import org.apache.hadoop.ozone.recon.metrics.Metric;
+import org.apache.hadoop.ozone.recon.spi.MetricsServiceProvider;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.concurrent.FutureCallback;
+import org.apache.http.impl.nio.client.CloseableHttpAsyncClient;
+import org.apache.http.impl.nio.client.HttpAsyncClients;
+import org.apache.http.impl.nio.conn.PoolingNHttpClientConnectionManager;
+import org.apache.http.impl.nio.reactor.DefaultConnectingIOReactor;
+import org.apache.http.impl.nio.reactor.IOReactorConfig;
+import org.apache.http.nio.reactor.ConnectingIOReactor;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of the Prometheus Metrics Service provider.
+ */
+@Singleton
+public class JmxServiceProviderImpl implements MetricsServiceProvider {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(JmxServiceProviderImpl.class);
+  private final CloseableHttpAsyncClient asyncHttpClient;
+  private final ObjectMapper objectMapper;
+
+  public static final String JMX_INSTANT_QUERY_API = "qry";
+  private static final int DEFAULT_TIMEOUT_MS = 60000;
+  private static final int MAX_TOTAL_CONNECTIONS = 100;

Review Comment:
   Make these configurable. May be too few for 1000-node cluster (bottleneck)



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/JmxServiceProviderImpl.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.spi.impl;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URLEncoder;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.inject.Singleton;
+import org.apache.hadoop.ozone.recon.metrics.Metric;
+import org.apache.hadoop.ozone.recon.spi.MetricsServiceProvider;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.concurrent.FutureCallback;
+import org.apache.http.impl.nio.client.CloseableHttpAsyncClient;
+import org.apache.http.impl.nio.client.HttpAsyncClients;
+import org.apache.http.impl.nio.conn.PoolingNHttpClientConnectionManager;
+import org.apache.http.impl.nio.reactor.DefaultConnectingIOReactor;
+import org.apache.http.impl.nio.reactor.IOReactorConfig;
+import org.apache.http.nio.reactor.ConnectingIOReactor;
+import org.apache.http.util.EntityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of the Prometheus Metrics Service provider.
+ */
+@Singleton
+public class JmxServiceProviderImpl implements MetricsServiceProvider {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(JmxServiceProviderImpl.class);
+  private final CloseableHttpAsyncClient asyncHttpClient;
+  private final ObjectMapper objectMapper;
+
+  public static final String JMX_INSTANT_QUERY_API = "qry";
+  private static final int DEFAULT_TIMEOUT_MS = 60000;
+  private static final int MAX_TOTAL_CONNECTIONS = 100;
+  private static final int MAX_CONNECTIONS_PER_ROUTE = 10;
+
+  public JmxServiceProviderImpl() {
+    this.asyncHttpClient = createAsyncHttpClient();
+    this.asyncHttpClient.start();
+    this.objectMapper = new ObjectMapper();
+  }
+
+  private CloseableHttpAsyncClient createAsyncHttpClient() {
+    try {
+      // Configure IO reactor for non-blocking I/O
+      IOReactorConfig ioReactorConfig = IOReactorConfig.custom()
+              .setConnectTimeout(DEFAULT_TIMEOUT_MS)
+              .setSoTimeout(DEFAULT_TIMEOUT_MS)
+              .setIoThreadCount(Runtime.getRuntime().availableProcessors())
+              .build();
+
+      ConnectingIOReactor ioReactor = new 
DefaultConnectingIOReactor(ioReactorConfig);
+      PoolingNHttpClientConnectionManager connManager =
+              new PoolingNHttpClientConnectionManager(ioReactor);
+      connManager.setMaxTotal(MAX_TOTAL_CONNECTIONS);
+      connManager.setDefaultMaxPerRoute(MAX_CONNECTIONS_PER_ROUTE);
+
+      RequestConfig requestConfig = RequestConfig.custom()
+              .setConnectTimeout(DEFAULT_TIMEOUT_MS)
+              .setSocketTimeout(DEFAULT_TIMEOUT_MS)
+              .setConnectionRequestTimeout(DEFAULT_TIMEOUT_MS)
+              .build();
+
+      return HttpAsyncClients.custom()
+              .setConnectionManager(connManager)
+              .setDefaultRequestConfig(requestConfig)
+              .build();
+
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create async HTTP client", e);
+    }
+  }
+
+  /**
+   * Returns {@link HttpURLConnection} after querying Metrics endpoint for the
+   * given metric.
+   *
+   * @param api         api.
+   * @param queryString query string with metric name and other filters.
+   * @return HttpURLConnection
+   */
+  @Override
+  public HttpURLConnection getMetricsResponse(String api, String queryString) {
+    return null;
+  }
+
+  @Override
+  public List<Metric> getMetricsInstant(String queryString) {
+    return Collections.emptyList();
+  }
+
+  /**
+   * Returns a list of {@link Metric} for the given instant query.
+   *
+   * @param queryString query string with metric name and other filters.
+   * @return List of Json map of metrics response.
+   */
+  @Override
+  public CompletableFuture<List<Map<String, Object>>> getMetricsAsync(String 
jmxEndpoint, String queryString) {
+    // Remove the trailing slash from endpoint url.
+    if (jmxEndpoint != null && jmxEndpoint.endsWith("/")) {
+      jmxEndpoint = jmxEndpoint.substring(0, jmxEndpoint.length() - 1);
+    }
+    return getMetrics(jmxEndpoint, queryString);
+  }
+
+  @Override
+  public void shutdown() {
+    try {
+      LOG.info("Shutting down async HTTP client...");
+      asyncHttpClient.close();
+    } catch (IOException e) {
+      LOG.error("Error shutting down async HTTP client", e);
+    }
+  }
+
+  /**
+   * Returns a list of {@link Metric} for the given api and query string.
+   *
+   * @param jmxEndpoint endpoint for the jmx server
+   * @param queryString query string with metric name and other filters.
+   * @return List of Json map of metrics response.
+   */
+  private CompletableFuture<List<Map<String, Object>>> getMetrics(String 
jmxEndpoint, String queryString) {
+    CompletableFuture<List<Map<String, Object>>> future = new 
CompletableFuture<>();
+
+    try {
+      String url = String.format(
+          "%s?%s=%s",
+          jmxEndpoint,
+          JMX_INSTANT_QUERY_API,
+          URLEncoder.encode(queryString, "UTF-8"));
+      HttpGet request = new HttpGet(url);
+
+      asyncHttpClient.execute(request, new FutureCallback<HttpResponse>() {
+        @Override
+        public void completed(HttpResponse response) {
+          try {
+            int statusCode = response.getStatusLine().getStatusCode();
+            if (statusCode >= 200 && statusCode < 300) {
+              String responseBody = EntityUtils.toString(response.getEntity());
+              Map<String, Object> jsonResponse = 
objectMapper.readValue(responseBody, Map.class);
+              List<Map<String, Object>> beans = (List<Map<String, Object>>) 
jsonResponse.get("beans");

Review Comment:
   No check if "beans" key exists in response. NPE if beans is null
   
   ```suggestion
   if (beansObj instanceof List) {
     future.complete((List<Map<String, Object>>) beansObj);
   } else {
     future.completeExceptionally(new IOException("Invalid JMX response: 
'beans' not found or not a list"));
   }
   ```



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/DatanodePendingDeletionMetrics.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.types;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Represents pending deletion metrics for a datanode.
+ * This class encapsulates information about blocks pending deletion on a 
specific datanode.
+ */
+public class DatanodePendingDeletionMetrics {
+
+  @JsonProperty("hostName")
+  private final String hostName;
+
+  @JsonProperty("datanodeUuid")
+  private final String datanodeUuid;
+
+  @JsonProperty("pendingBlockSize")
+  private final long pendingBlockSize;

Review Comment:
   Since we are adding failed nodes metrics also in API response, we should 
provide proper mechanism for API client to filter out if needed. Can we also 
add explicit failed: true boolean field here ?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PendingDeletionEndpoint.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
+import org.apache.hadoop.ozone.recon.api.types.DataNodeMetricsServiceResponse;
+import org.apache.hadoop.ozone.recon.api.types.ScmPendingDeletion;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * REST API endpoint that provides metrics and information related to
+ * pending deletions. It responds to requests on the "/pendingDeletion" path
+ * and produces application/json responses.
+ */
+@Path("/pendingDeletion")
+@Produces("application/json")
+@AdminOnly
+public class PendingDeletionEndpoint {
+  private static final Logger LOG = 
LoggerFactory.getLogger(PendingDeletionEndpoint.class);
+  private final ReconGlobalMetricsService reconGlobalMetricsService;
+  private final DataNodeMetricsService dataNodeMetricsService;
+  private final StorageContainerLocationProtocol scmClient;
+
+  @Inject
+  public PendingDeletionEndpoint(ReconGlobalMetricsService 
reconGlobalMetricsService,
+                                 DataNodeMetricsService dataNodeMetricsService,
+                                 StorageContainerLocationProtocol scmClient) {
+    this.reconGlobalMetricsService = reconGlobalMetricsService;
+    this.dataNodeMetricsService = dataNodeMetricsService;
+    this.scmClient = scmClient;
+  }
+
+  @GET
+  public Response getPendingDeletionByComponent(@QueryParam("component") 
String component) {
+    if (component == null || component.isEmpty()) {
+      return Response.status(Response.Status.BAD_REQUEST)
+          .entity("component query parameter is required").build();
+    }
+    final String normalizedComponent = component.trim().toLowerCase();
+    switch (normalizedComponent) {
+    case "dn":
+      return handleDataNodeMetrics();
+    case "scm":
+      return handleScmPendingDeletion();
+    case "om":
+      return handleOmPendingDeletion();
+    default:
+      return Response.status(Response.Status.BAD_REQUEST)
+          .entity("component query parameter must be one of dn, scm, 
om").build();
+    }
+  }
+
+  private Response handleDataNodeMetrics() {
+    DataNodeMetricsService.MetricCollectionStatus status = 
dataNodeMetricsService.getTaskStatus();
+    if (status != DataNodeMetricsService.MetricCollectionStatus.IN_PROGRESS) {
+      CompletableFuture.runAsync(dataNodeMetricsService::startTask);

Review Comment:
   This may be ok for our usecase when user will trigger endpoint on demand, 
but this code will create issues in stress test when multiple concurremt 
requests can hit at same time.  `CompletableFuture.runAsync(Runnable) `without 
an executor parameter uses `ForkJoinPool.commonPool()` by default. This is 
Java's shared thread pool used across the entire JVM. So ot can be easily 
create DOS attack situation and Other application tasks blocked.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/PendingDeletionEndpoint.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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 java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
+import org.apache.hadoop.ozone.recon.api.types.DataNodeMetricsServiceResponse;
+import org.apache.hadoop.ozone.recon.api.types.ScmPendingDeletion;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * REST API endpoint that provides metrics and information related to
+ * pending deletions. It responds to requests on the "/pendingDeletion" path
+ * and produces application/json responses.
+ */
+@Path("/pendingDeletion")
+@Produces("application/json")
+@AdminOnly
+public class PendingDeletionEndpoint {
+  private static final Logger LOG = 
LoggerFactory.getLogger(PendingDeletionEndpoint.class);
+  private final ReconGlobalMetricsService reconGlobalMetricsService;
+  private final DataNodeMetricsService dataNodeMetricsService;
+  private final StorageContainerLocationProtocol scmClient;
+
+  @Inject
+  public PendingDeletionEndpoint(ReconGlobalMetricsService 
reconGlobalMetricsService,
+                                 DataNodeMetricsService dataNodeMetricsService,
+                                 StorageContainerLocationProtocol scmClient) {
+    this.reconGlobalMetricsService = reconGlobalMetricsService;
+    this.dataNodeMetricsService = dataNodeMetricsService;
+    this.scmClient = scmClient;
+  }
+
+  @GET
+  public Response getPendingDeletionByComponent(@QueryParam("component") 
String component) {
+    if (component == null || component.isEmpty()) {
+      return Response.status(Response.Status.BAD_REQUEST)
+          .entity("component query parameter is required").build();
+    }
+    final String normalizedComponent = component.trim().toLowerCase();
+    switch (normalizedComponent) {
+    case "dn":
+      return handleDataNodeMetrics();
+    case "scm":
+      return handleScmPendingDeletion();
+    case "om":
+      return handleOmPendingDeletion();
+    default:
+      return Response.status(Response.Status.BAD_REQUEST)
+          .entity("component query parameter must be one of dn, scm, 
om").build();
+    }
+  }
+
+  private Response handleDataNodeMetrics() {
+    DataNodeMetricsService.MetricCollectionStatus status = 
dataNodeMetricsService.getTaskStatus();
+    if (status != DataNodeMetricsService.MetricCollectionStatus.IN_PROGRESS) {
+      CompletableFuture.runAsync(dataNodeMetricsService::startTask);
+    }
+    DataNodeMetricsServiceResponse response = 
dataNodeMetricsService.getCollectedMetrics();
+    if (response.getStatus() != 
DataNodeMetricsService.MetricCollectionStatus.SUCCEEDED) {
+      return Response.accepted(DataNodeMetricsServiceResponse.newBuilder()
+          
.setStatus(DataNodeMetricsService.MetricCollectionStatus.IN_PROGRESS).build())

Review Comment:
   Returns HTTP 202 (Accepted) for both NOT_STARTED and IN_PROGRESS states, but 
always reports status as IN_PROGRESS in response body. How about below?
   
   ```suggestion
             DataNodeMetricsServiceResponse actualResponse = 
dataNodeMetricsService.getCollectedMetrics();
   if (actualResponse.getStatus() != 
DataNodeMetricsService.MetricCollectionStatus.SUCCEEDED) {
     return Response.accepted(actualResponse).build();  // Return actual 
status, not hardcoded IN_PROGRESS
   }
   ```



-- 
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