anmolanmol1234 commented on code in PR #8137:
URL: https://github.com/apache/hadoop/pull/8137#discussion_r2650700026


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/MetricsBucket.java:
##########
@@ -0,0 +1,196 @@
+/**
+ * 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.fs.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.fs.azurebfs.utils.SimpleRateLimiter;
+
+/**
+ * MetricsBucket holds metrics for multiple AbfsClients and
+ * dispatches them in batches, respecting rate limits.
+ */
+final class MetricsBucket {
+
+  // Rate limiter to control the rate of dispatching metrics.
+  private final SimpleRateLimiter rateLimiter;
+
+  // Buffer to hold metrics before sending.
+  private final AtomicReference<ConcurrentLinkedQueue<String>> metricsBuffer =
+      new AtomicReference<>(new ConcurrentLinkedQueue<>());
+
+  // Set of registered AbfsClients.
+  private final Set<AbfsClient> clients =
+      ConcurrentHashMap.newKeySet();
+
+  // Maximum size of metrics header in characters.
+  private static final long MAX_HEADER_SIZE = 1024;
+
+  // Constructor
+  MetricsBucket(SimpleRateLimiter rateLimiter) {
+    this.rateLimiter = rateLimiter;
+  }
+
+  /**
+   * Register a new AbfsClient.
+   * @param client the AbfsClient to register
+   */
+  public void registerClient(AbfsClient client) {
+    if (client != null) {
+      clients.add(client);
+    }
+  }
+
+  /**
+   * Deregister an AbfsClient. If this is the last client, drain and send
+   * any remaining metrics.
+   * @param client the AbfsClient to deregister
+   * @return true if the client was deregistered, false otherwise
+   */
+  public boolean deregisterClient(AbfsClient client) {
+    if (client == null) {
+      return false;
+    }
+    ConcurrentLinkedQueue<String> batchToSend = null;
+    boolean isLastClient = false;
+
+    synchronized (this) {
+      if (!clients.contains(client)) {
+        return false;
+      }
+
+      if (clients.size() == 1) {
+        // This client is the last one — drain metrics now
+        batchToSend = metricsBuffer.getAndSet(new ConcurrentLinkedQueue<>());
+        isLastClient = true;
+      }
+
+      clients.remove(client);
+    }
+    if (isLastClient) {
+      sendMetrics(client, batchToSend);
+    }
+    return true;
+  }
+
+  /**
+   * Add a metric to the buffer.
+   * @param metric the metric to add
+   */
+  void addRequest(String metric) {
+    if (metric != null) {
+      metricsBuffer.get().add(metric);
+    }
+  }
+
+  /**
+   * Drain the metrics buffer and send if there are registered clients.
+   */
+  public void drainAndSendIfReady() {
+    AbfsClient client;
+
+    synchronized (this) {
+      if (clients.isEmpty()) {
+        return;
+      }
+      client = clients.iterator().next();
+    }
+
+    ConcurrentLinkedQueue<String> batch = metricsBuffer.getAndSet(
+        new ConcurrentLinkedQueue<>());
+    if (batch.isEmpty()) {
+      return;
+    }
+
+    sendMetrics(client, batch);
+  }
+
+  // Send metrics outside synchronized block
+  private void sendMetrics(AbfsClient client,
+      ConcurrentLinkedQueue<String> batchToSend) {
+    // Send outside synchronized block
+    if (client != null && batchToSend != null && !batchToSend.isEmpty()) {
+      for (String chunk : splitListBySize(batchToSend, MAX_HEADER_SIZE)) {
+        rateLimiter.acquire(); // Rate limiting
+        try {
+          client.getMetricCall(chunk);
+        } catch (IOException ignored) {}
+      }
+    }
+  }
+
+  // Check if there are no registered clients
+  public boolean isEmpty() {
+    return clients.isEmpty();
+  }
+
+  /**
+   * Split the list of metrics into chunks that fit within maxChars.
+   * Each metric is wrapped in square brackets and separated by colons.
+   */
+  private static List<String> splitListBySize(
+      ConcurrentLinkedQueue<String> items, long maxChars) {
+
+    if (items.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    List<String> result = new ArrayList<>();
+    StringBuilder sb = new StringBuilder();
+
+    for (String s : new ArrayList<>(items)) {

Review Comment:
   No need of copy here, you can directly iterate as for (String s: items)



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