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


##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java:
##########
@@ -136,6 +136,13 @@ public final class FileSystemConfigurations {
   public static final boolean DEFAULT_ENABLE_AUTOTHROTTLING = false;
   public static final int DEFAULT_METRIC_IDLE_TIMEOUT_MS = 60_000;
   public static final int DEFAULT_METRIC_ANALYSIS_TIMEOUT_MS = 60_000;
+  public static final boolean DEFAULT_METRICS_COLLECTION_ENABLED = true;
+  public static final boolean DEFAULT_SHOULD_EMIT_METRICS_ON_IDLE_TIME = false;
+  public static final long DEFAULT_METRICS_EMIT_THRESHOLD = 100_000L;
+  public static final long DEFAULT_METRICS_EMIT_THRESHOLD_INTERVAL_SECS = 60;

Review Comment:
   As discussed, no changes required here.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsReadFooterMetrics.java:
##########
@@ -62,476 +63,536 @@
  * This class is responsible for tracking and updating metrics related to 
reading footers in files.
  */
 public class AbfsReadFooterMetrics extends AbstractAbfsStatisticsSource {
-    private static final Logger LOG = 
LoggerFactory.getLogger(AbfsReadFooterMetrics.class);
-    private static final String FOOTER_LENGTH = "20";
-    private static final List<FileType> FILE_TYPE_LIST =
-            Arrays.asList(FileType.values());
-    private final Map<String, FileTypeMetrics> fileTypeMetricsMap =
-            new ConcurrentHashMap<>();
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      AbfsReadFooterMetrics.class);
+

Review Comment:
   This line was added by the formatter we follow in OSS repo.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AggregateMetricsManager.java:
##########
@@ -0,0 +1,175 @@
+/**
+ * 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.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.StringUtils;
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.utils.SimpleRateLimiter;
+
+/**
+ * AggregateMetricsManager manages metrics collection and dispatching
+ * for multiple AbfsClients across different accounts.
+ */
+public final class AggregateMetricsManager {
+
+  // Singleton instance of AggregateMetricsManager.
+  private static volatile AggregateMetricsManager instance;
+
+  // Rate limiter to control the rate of dispatching metrics.
+  private static volatile SimpleRateLimiter rateLimiter;
+
+  // Map of account name to MetricsBucket.
+  private final ConcurrentHashMap<String, MetricsBucket> buckets =
+      new ConcurrentHashMap<>();
+
+  // Scheduler for periodic dispatching of metrics.
+  private final ScheduledExecutorService scheduler;
+
+  // Private constructor to enforce singleton pattern.
+  private AggregateMetricsManager(final long dispatchIntervalInMins,
+      final int permitsPerSecond) throws InvalidConfigurationValueException {
+
+    if (dispatchIntervalInMins <= 0) {
+      throw new InvalidConfigurationValueException(
+          "dispatchIntervalInMins must be > 0");
+    }
+
+    if (permitsPerSecond <= 0) {
+      throw new InvalidConfigurationValueException(
+          "permitsPerSecond must be > 0");
+    }
+
+    this.rateLimiter = new SimpleRateLimiter(permitsPerSecond);
+
+    // Initialize scheduler for periodic dispatching of metrics.
+    this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> {
+      Thread t = new Thread(r, "ABFS-Aggregated-Metrics-Dispatcher");
+      t.setDaemon(true);
+      return t;
+    });
+
+    // Schedule periodic dispatching of metrics.
+    this.scheduler.scheduleAtFixedRate(
+        this::dispatchMetrics,
+        dispatchIntervalInMins,
+        dispatchIntervalInMins,
+        TimeUnit.MINUTES);
+
+    // Add shutdown hook to dispatch remaining metrics on JVM shutdown.
+    Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+      dispatchMetrics();
+      scheduler.shutdown();
+    }));
+  }
+
+  /**
+   * Get the singleton instance of AggregateMetricsManager.
+   *
+   * @param dispatchIntervalInMins Interval in minutes for dispatching metrics.
+   * @param permitsPerSecond       Rate limit for dispatching metrics.
+   * @return Singleton instance of AggregateMetricsManager.
+   */
+  public static AggregateMetricsManager get(final long dispatchIntervalInMins,

Review Comment:
   Renamed it to getInstance



##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestAbfsClient.java:
##########
@@ -876,6 +894,308 @@ public void testIsNonEmptyDirectory() throws IOException {
         false, 1, true);
   }
 
+  /**
+   * Test to verify that in case metric account is not set,
+   * metric collection is enabled with default metric format
+   * and account url.
+   *
+   * @throws Exception in case of any failure
+   */
+  @Test
+  public void testMetricAccountFallback() throws Exception {
+    Configuration configuration = getRawConfiguration();
+    configuration.setBoolean(
+        AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, false);
+    configuration.setBoolean(FS_AZURE_ACCOUNT_IS_HNS_ENABLED, false);
+    configuration.unset(FS_AZURE_METRIC_ACCOUNT_NAME);
+    configuration.unset(FS_AZURE_METRIC_ACCOUNT_KEY);
+    configuration.unset(FS_AZURE_METRIC_FORMAT);
+    configuration.setBoolean(FS_AZURE_ALWAYS_USE_HTTPS, true);
+    final AzureBlobFileSystem fs = getFileSystem(configuration);
+    Assertions.assertThat(
+            fs.getAbfsStore().getAbfsConfiguration().getMetricFormat())
+        .describedAs(
+            "In case metric format is not set, metric format should "
+                + "be defaulted to internal metric format")
+        .isEqualTo(MetricFormat.INTERNAL_METRIC_FORMAT);
+
+    Assertions.assertThat(
+            fs.getAbfsStore().getClient().isMetricCollectionEnabled())
+        .describedAs(
+            "Metric collection should be enabled even if metric account is not 
set")
+        .isTrue();
+
+    Assertions.assertThat(

Review Comment:
   Yes, but there is no backend call it has done yet to make backoff counter or 
read footer counter nonzero, so no metrics will be emitted.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java:
##########
@@ -71,7 +71,14 @@ public final class ConfigurationKeys {
   public static final String FS_AZURE_ACCOUNT_KEY_PROPERTY_NAME = 
"fs.azure.account.key";
   public static final String FS_AZURE_METRIC_ACCOUNT_NAME = 
"fs.azure.metric.account.name";
   public static final String FS_AZURE_METRIC_ACCOUNT_KEY = 
"fs.azure.metric.account.key";
-  public static final String FS_AZURE_METRIC_URI = "fs.azure.metric.uri";
+  public static final String FS_AZURE_METRIC_FORMAT = "fs.azure.metric.format";

Review Comment:
   Fixed



##########
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) {

Review Comment:
   As discussed, this change is not needed as the current code guarantees the 
metrics emission correctly.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java:
##########
@@ -201,20 +201,39 @@ public void initializeWriteResourceUtilizationMetrics() {
 
 
   @Override
-  public void initializeMetrics(MetricFormat metricFormat) {
+  public void initializeMetrics(final MetricFormat metricFormat,
+      final AbfsConfiguration abfsConfiguration) {

Review Comment:
   The reason for sending the entire AbfsConfiguration was that we may need 
additional configuration values in the future; in that case, we can fetch them 
directly without having to change this again.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java:
##########
@@ -201,20 +201,39 @@ public void initializeWriteResourceUtilizationMetrics() {
 
 
   @Override
-  public void initializeMetrics(MetricFormat metricFormat) {
+  public void initializeMetrics(final MetricFormat metricFormat,
+      final AbfsConfiguration abfsConfiguration) {
     switch (metricFormat) {
-      case INTERNAL_BACKOFF_METRIC_FORMAT:
-        abfsBackoffMetrics = new AbfsBackoffMetrics();
-        break;
-      case INTERNAL_FOOTER_METRIC_FORMAT:
-        abfsReadFooterMetrics = new AbfsReadFooterMetrics();
-        break;
-      case INTERNAL_METRIC_FORMAT:
-        abfsBackoffMetrics = new AbfsBackoffMetrics();
-        abfsReadFooterMetrics = new AbfsReadFooterMetrics();
-        break;
-      default:
-        break;
+    case INTERNAL_BACKOFF_METRIC_FORMAT:
+      abfsBackoffMetrics = new AbfsBackoffMetrics(
+          abfsConfiguration.isBackoffRetryMetricsEnabled());
+      break;
+    case INTERNAL_FOOTER_METRIC_FORMAT:
+      initializeReadFooterMetrics();
+    case INTERNAL_METRIC_FORMAT:
+      abfsBackoffMetrics = new AbfsBackoffMetrics(
+          abfsConfiguration.isBackoffRetryMetricsEnabled());
+      initializeReadFooterMetrics();
+      break;
+    default:
+      break;
+    }
+  }
+
+  /**
+   * Initialize the read footer metrics.
+   * In case the metrics are already initialized,
+   * create a new instance with the existing map.

Review Comment:
   After the collected metrics are pushed to the backend, the counters are 
refreshed by creating new instances of the backoff and read-footer metrics 
rather than clearing each counter individually. The old instances are 
automatically garbage-collected.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AggregateMetricsManager.java:
##########
@@ -0,0 +1,175 @@
+/**
+ * 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.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.StringUtils;
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.utils.SimpleRateLimiter;
+
+/**
+ * AggregateMetricsManager manages metrics collection and dispatching
+ * for multiple AbfsClients across different accounts.
+ */
+public final class AggregateMetricsManager {
+
+  // Singleton instance of AggregateMetricsManager.
+  private static volatile AggregateMetricsManager instance;
+
+  // Rate limiter to control the rate of dispatching metrics.
+  private static volatile SimpleRateLimiter rateLimiter;
+
+  // Map of account name to MetricsBucket.
+  private final ConcurrentHashMap<String, MetricsBucket> buckets =
+      new ConcurrentHashMap<>();
+
+  // Scheduler for periodic dispatching of metrics.
+  private final ScheduledExecutorService scheduler;
+
+  // Private constructor to enforce singleton pattern.
+  private AggregateMetricsManager(final long dispatchIntervalInMins,
+      final int permitsPerSecond) throws InvalidConfigurationValueException {
+
+    if (dispatchIntervalInMins <= 0) {
+      throw new InvalidConfigurationValueException(
+          "dispatchIntervalInMins must be > 0");
+    }
+
+    if (permitsPerSecond <= 0) {
+      throw new InvalidConfigurationValueException(
+          "permitsPerSecond must be > 0");
+    }
+
+    this.rateLimiter = new SimpleRateLimiter(permitsPerSecond);
+
+    // Initialize scheduler for periodic dispatching of metrics.
+    this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> {
+      Thread t = new Thread(r, "ABFS-Aggregated-Metrics-Dispatcher");
+      t.setDaemon(true);
+      return t;
+    });
+
+    // Schedule periodic dispatching of metrics.
+    this.scheduler.scheduleAtFixedRate(
+        this::dispatchMetrics,
+        dispatchIntervalInMins,
+        dispatchIntervalInMins,
+        TimeUnit.MINUTES);
+
+    // Add shutdown hook to dispatch remaining metrics on JVM shutdown.
+    Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+      dispatchMetrics();
+      scheduler.shutdown();

Review Comment:
   Since it is a sequential operation so shutdown will happen only after 
dispatchMetrics is completed.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java:
##########
@@ -201,20 +201,39 @@ public void initializeWriteResourceUtilizationMetrics() {
 
 
   @Override
-  public void initializeMetrics(MetricFormat metricFormat) {
+  public void initializeMetrics(final MetricFormat metricFormat,
+      final AbfsConfiguration abfsConfiguration) {
     switch (metricFormat) {
-      case INTERNAL_BACKOFF_METRIC_FORMAT:
-        abfsBackoffMetrics = new AbfsBackoffMetrics();
-        break;
-      case INTERNAL_FOOTER_METRIC_FORMAT:
-        abfsReadFooterMetrics = new AbfsReadFooterMetrics();
-        break;
-      case INTERNAL_METRIC_FORMAT:
-        abfsBackoffMetrics = new AbfsBackoffMetrics();
-        abfsReadFooterMetrics = new AbfsReadFooterMetrics();
-        break;
-      default:
-        break;
+    case INTERNAL_BACKOFF_METRIC_FORMAT:
+      abfsBackoffMetrics = new AbfsBackoffMetrics(
+          abfsConfiguration.isBackoffRetryMetricsEnabled());
+      break;
+    case INTERNAL_FOOTER_METRIC_FORMAT:
+      initializeReadFooterMetrics();
+    case INTERNAL_METRIC_FORMAT:
+      abfsBackoffMetrics = new AbfsBackoffMetrics(
+          abfsConfiguration.isBackoffRetryMetricsEnabled());
+      initializeReadFooterMetrics();
+      break;
+    default:
+      break;
+    }
+  }
+
+  /**
+   * Initialize the read footer metrics.
+   * In case the metrics are already initialized,
+   * create a new instance with the existing map.
+   */
+  private void initializeReadFooterMetrics() {
+    if (abfsReadFooterMetrics == null) {
+      abfsReadFooterMetrics = new AbfsReadFooterMetrics();
+    } else {
+      //In case metrics is emitted based on total count, there could be a 
chance
+      // that file type for which we have calculated the type will be lost.
+      // To avoid that, creating a new instance with existing map.
+      abfsReadFooterMetrics = new AbfsReadFooterMetrics(

Review Comment:
   Here we are creating new object of AbfsReadFooterMetrics. Thats why we are 
copying the file type information to new object.



##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadFooterMetrics.java:
##########
@@ -57,32 +54,14 @@
 import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
 import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
 
-import static org.assertj.core.api.Assumptions.assumeThat;
-
 public class ITestAbfsReadFooterMetrics extends AbstractAbfsScaleTest {
 
-  public ITestAbfsReadFooterMetrics() throws Exception {
-    checkPrerequisites();
-  }
-
-  private void checkPrerequisites(){

Review Comment:
   Now, it is not required as irrespective of these configurations, we will the 
metrics in the main account.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/SimpleRateLimiter.java:
##########
@@ -0,0 +1,75 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.utils;
+
+import java.util.concurrent.locks.LockSupport;
+
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+
+public final class SimpleRateLimiter {
+
+  // Interval between permits in nanoseconds.
+  private final long intervalNanos;
+
+  // Next allowed time to acquire a permit in nanoseconds.
+  private long nextAllowedTime;
+
+  /** Number of nanoseconds in one second. */
+  private static final long NANOS_PER_SECOND = 1_000_000_000L;
+
+  /**
+   * Constructs a SimpleRateLimiter that allows the specified number of
+   * permits per second.
+   *
+   * @param permitsPerSecond Number of permits allowed per second.
+   * @throws InvalidConfigurationValueException if permitsPerSecond is
+   *                                            less than or equal to zero.
+   */
+  public SimpleRateLimiter(int permitsPerSecond)
+      throws InvalidConfigurationValueException {
+    if (permitsPerSecond <= 0) {
+      throw new InvalidConfigurationValueException(
+          "permitsPerSecond must be > 0");
+    }
+    this.intervalNanos = NANOS_PER_SECOND / permitsPerSecond;
+    this.nextAllowedTime = System.nanoTime();
+  }
+
+  /**
+   * Acquires a permit from the rate limiter, blocking until one is available.
+   */
+  public synchronized void acquire() {

Review Comment:
   Made the changes to make it time bounded wait.



##########
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AggregateMetricsManager.java:
##########
@@ -0,0 +1,175 @@
+/**
+ * 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.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.StringUtils;
+import 
org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;
+import org.apache.hadoop.fs.azurebfs.utils.SimpleRateLimiter;
+
+/**
+ * AggregateMetricsManager manages metrics collection and dispatching
+ * for multiple AbfsClients across different accounts.
+ */
+public final class AggregateMetricsManager {
+
+  // Singleton instance of AggregateMetricsManager.
+  private static volatile AggregateMetricsManager instance;
+
+  // Rate limiter to control the rate of dispatching metrics.
+  private static volatile SimpleRateLimiter rateLimiter;

Review Comment:
   Yes, this is what we want as well - Ratelimit on the JVM level so that there 
should be less number of metrics calls.



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