Skip to content

Conversation

@bhattmanish98
Copy link
Contributor

This PR introduces a centralized Aggregated Metrics Manager and defines the conditions under which aggregated metrics are emitted from individual file systems.

Key Changes

  1. Criteria for Emitting Aggregated Metrics
    Aggregated metrics are emitted based on the following conditions:
  • Time-based interval - Each file system periodically emits its collected metrics at a fixed interval. After emission, metric collection is reset.

  • Threshold-based emission - A scheduler runs at regular intervals to check whether the total number of operations has exceeded a configured threshold. This prevents the aggregated metrics string from growing too large to be safely sent as an HTTP request header. If the threshold is reached, the collected metrics are emitted immediately, and metric collection is reset.

  • Idle-period emission - If a file system remains idle for a configured duration, any accumulated metrics are emitted, and metric collection is reset.

  • File system close - When a file system is closed, all remaining collected metrics are emitted to ensure no data is lost.

  1. Centralized Metrics Management
    All file systems now push their aggregated metrics to a shared Aggregated Metrics Manager. This manager evaluates the configured emission criteria and determines whether metrics should be emitted immediately or deferred until a later time.
    This will also rate limit the number of metrics calls per second.

@hadoop-yetus

This comment was marked as outdated.

@hadoop-yetus

This comment was marked as outdated.

@hadoop-yetus

This comment was marked as outdated.

@hadoop-yetus

This comment was marked as outdated.

@hadoop-yetus

This comment was marked as outdated.

@bhattmanish98
Copy link
Contributor Author

============================================================
HNS-OAuth-DFS

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 904, Failures: 0, Errors: 0, Skipped: 220
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
HNS-SharedKey-DFS

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 4
[WARNING] Tests run: 907, Failures: 0, Errors: 0, Skipped: 166
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 10

============================================================
NonHNS-SharedKey-DFS

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 744, Failures: 0, Errors: 0, Skipped: 287
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 11

============================================================
AppendBlob-HNS-OAuth-DFS

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 904, Failures: 0, Errors: 0, Skipped: 231
[WARNING] Tests run: 135, Failures: 0, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
NonHNS-SharedKey-Blob

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 750, Failures: 0, Errors: 0, Skipped: 144
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 11

============================================================
NonHNS-OAuth-DFS

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 741, Failures: 0, Errors: 0, Skipped: 289
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
NonHNS-OAuth-Blob

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 747, Failures: 0, Errors: 0, Skipped: 156
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
AppendBlob-NonHNS-OAuth-Blob

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 742, Failures: 0, Errors: 0, Skipped: 202
[WARNING] Tests run: 135, Failures: 0, Errors: 0, Skipped: 4
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
HNS-Oauth-DFS-IngressBlob

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 778, Failures: 0, Errors: 0, Skipped: 229
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
NonHNS-OAuth-DFS-IngressBlob

[WARNING] Tests run: 235, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 739, Failures: 0, Errors: 0, Skipped: 286
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

@bhattmanish98 bhattmanish98 marked this pull request as ready for review December 19, 2025 14:12
abfsConfiguration.isBackoffRetryMetricsEnabled());
break;
case INTERNAL_FOOTER_METRIC_FORMAT:
initializeReadFooterMetrics();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

break missing here


@Override
public String toString() {
String metric = "";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use empty string constant

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";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add jvadoc for all these with {@value} tag.

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";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Keep all the metric related configs name consistent with same prefix.
fs.azure.metrics....

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

metric account name, key and format have "metric", other configurations have "metrics". I have kept it intentionally. Do you want to change this?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. I think we should have common prefix for all the metric related configs

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

metricAccountKey)) {
int dotIndex = metricAccountName.indexOf(AbfsHttpConstants.DOT);
if (dotIndex <= 0) {
throw new InvalidUriException(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a test around this exception if not already there

final AbfsConfiguration abfsConfiguration,
final EncryptionContextProvider encryptionContextProvider,
final AbfsClientContext abfsClientContext,
final String fileSystemId,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think its better to pass it as a part of client context similar to other client related fields.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, make sense. Will do this change.

this.metricsEmitScheduler
= Executors.newSingleThreadScheduledExecutor();
// run every 1 minute to check the metrics count
this.metricsEmitScheduler.scheduleAtFixedRate(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are 2 separate schedulers being added here seems like.
Each client has its own scheduler and then the singleton metric manager class also has one?

Is this as per design?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, it is as per design, each file system will emit the metrics to manager class at regular interval if not closed. The singleton manager class will do actual API call to send those collected metrics.

if (isMetricCollectionEnabled && runningTimerTask != null) {
runningTimerTask.cancel();
timer.cancel();
if (isMetricCollectionEnabled()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If not already done, verify that after FS close all the threads are properly getting shutdown and no leak is there.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are few tests which already cover this. Will check if more test cases are needed to cover more scenario.

private static final Logger LOG = LoggerFactory.getLogger(
AbfsReadFooterMetrics.class);

private static final String FOOTER_LENGTH = "20";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add comment as to why 20?

operationType, failureReason, retryCount, retryPolicy.getAbbreviation(), retryInterval);
if (abfsBackoffMetrics != null) {
updateBackoffTimeMetrics(retryCount, sleepDuration);
updateBackoffTimeMetrics(retryCount, retryInterval);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why this change?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Earlier sleepDuration was wrongly passed in updateBackoffTimeMetrics, actually it is retryInterval which tells us the delay between two retries.

}
StringBuilder metricBuilder = new StringBuilder();
getRetryMetrics(metricBuilder);
if (isRetryMetricEnabled) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should always have retry metrics added as a part of these aggregate metrics

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I made this change as per the discussion we had with the team. We can discuss this offline and if agreed by all I will revert the change.

private static volatile AggregateMetricsManager instance;

// Rate limiter to control the rate of dispatching metrics.
private static volatile SimpleRateLimiter rateLimiter;
Copy link
Contributor

@anmolanmol1234 anmolanmol1234 Dec 26, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

rateLimiter is declared as static but is initialized in the constructor using permitsPerSecond.

This means the rate limiter ends up being global to the JVM, even though its value comes from instance-level configuration. In practice, whichever code initializes the manager first decides the rate, and any later calls to get() with different values are silently ignored.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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


boolean isRemoved = bucket.deregisterClient(abfsClient);

if (bucket.isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is a small race window between isEmpty() and remove(). Another thread may concurrently register a new client for the same account and reuse the bucket, but it can still be removed based on the earlier emptiness check. This makes the behavior timing-dependent and hard to reason about under concurrency.

You can use buckets.computeIfPresent() to perform the emptiness check and removal atomically, which avoids this race and keeps the map state consistent.

// Add shutdown hook to dispatch remaining metrics on JVM shutdown.
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
dispatchMetrics();
scheduler.shutdown();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't we wait for dispatch metrics to finish before scheduler is shutdown ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

});

// Schedule periodic dispatching of metrics.
this.scheduler.scheduleAtFixedRate(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Better to use scheduleWithFixedDelay as scheduleAtFixedRate can overlap executions.

* @param permitsPerSecond Rate limit for dispatching metrics.
* @return Singleton instance of AggregateMetricsManager.
*/
public static AggregateMetricsManager get(final long dispatchIntervalInMins,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

get method name is very generic, use better naming

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Renamed it to getInstance

@Override
public void initializeMetrics(MetricFormat metricFormat) {
public void initializeMetrics(final MetricFormat metricFormat,
final AbfsConfiguration abfsConfiguration) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of passing AbfsConfiguration here, better to only pass isRetryMetricsEnabled.
Whole object is not needed

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

if (abfsReadFooterMetrics == null) {
abfsReadFooterMetrics = new AbfsReadFooterMetrics();
} else {
//In case metrics is emitted based on total count, there could be a chance
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Typo, space after //


private static final Logger LOG = LoggerFactory.getLogger(
AbfsReadFooterMetrics.class);

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Empty line changes can be reverted.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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


/**
* Constructor to initialize the IOStatisticsStore with counters and mean statistics.
* Updates the file type based on the metrics collected.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Explain in javadoc what logic is used to infer file type

//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(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not able to get this.
How can the file type will be lost? It must be a part of AbfsReadFooterMetrics object only

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

/**
* Initialize the read footer metrics.
* In case the metrics are already initialized,
* create a new instance with the existing map.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why create a new instance?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

/**
* Metrics to client request id header.
*/
AV0("av0", 3);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add what 3 fields will be there in comment as added in other versions javadoc

checkPrerequisites();
}

private void checkPrerequisites(){
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are these moved somewhere else?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

: Stream.of(readFooterMetricsEnum.getName()))
.toArray(String[]::new);
private boolean haveEqualValues(String value) {
String[] parts = value.split("_");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add constant for underscore

long nextReadPos,
int len,
long contentLength) {
if (nextReadPos
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Condition should be in same line

}

// Check if there are no registered clients
public boolean isEmpty() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should be synchronized

StringBuilder sb = new StringBuilder();

for (String s : new ArrayList<>(items)) {
String wrapped = "[" + s + "]";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use constants for colon and brackets

/**
* Acquires a permit from the rate limiter, blocking until one is available.
*/
public synchronized void acquire() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method can block forever because it waits in an infinite loop while holding a synchronized lock, which can stall other threads and delay shutdown.
It also handles interrupts weakly and offers no timeout or non-blocking option, making it unsafe for critical paths.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Made the changes to make it time bounded wait.

"Metric collection should be enabled even if metric account is not set")
.isTrue();

Assertions.assertThat(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If its INTERNAL_METRIC_FORMAT it should have both backoff and footer metrics right ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

tracingContext.constructHeader(abfsHttpOperation, null,
EXPONENTIAL_RETRY_POLICY_ABBREVIATION);
assertThat(abfsHttpOperation.getClientRequestId())
.describedAs("ClientRequestId should be contains Backoff metrics")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: should be containing

public void testAggregatedMetricsManagerWithJVMExit0()
throws IOException, InterruptedException {
// -------------------------------
// Program 1 (kept exactly as you asked)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove this comment

private void runProgramAndCaptureOutput(String program,
boolean expectMetricsFlush, int expectedExitCode)
throws IOException, InterruptedException {
Path tempFile = Files.createTempFile("ShutdownTestProg", ".java");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These temporary files are never deleted

.start();

String compileOutput = readProcessOutput(javac);
javac.waitFor();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add timed wait like
if (!javac.waitFor(30, TimeUnit.SECONDS)) {
javac.destroyForcibly();
throw new AssertionError("javac timed out");
}

.start();

String output = readProcessOutput(javaProc);
int exitCode = javaProc.waitFor();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same as above

initializeReadFooterMetrics();
break;
default:
break;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: break is redundant

Copy link
Contributor Author

@bhattmanish98 bhattmanish98 Jan 6, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is as per best practice to keep break statement with default case.

case INTERNAL_METRIC_FORMAT:
abfsBackoffMetrics = new AbfsBackoffMetrics(
abfsConfiguration.isBackoffRetryMetricsEnabled());
initializeReadFooterMetrics();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we can do sm like
abfsReadFooterMetrics = new AbfsReadFooterMetrics(
abfsReadFooterMetrics == null ? null : abfsReadFooterMetrics.getFileTypeMetricsMap()
);

we would have a single constructor for AbfsReadFooterMetrics then

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Even then we may need two constructors as it is getting used in some test cases. But as you suggest I can simplify this if else statement in initializeReadFooterMetrics method.

* @throws IOException if URL is malformed.
*/
private void setMetricsUrl(String urlString) throws IOException {
metricUrl = UriUtils.changeUrlFromBlobToDfs(new URL(urlString));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why cant we send it for blob endpoints?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

GetFileSystemProperties on root is only supported for DFS endpoint.

*
* @param fileTypeMetricsMap the map to track file type metrics
*/
public AbfsReadFooterMetrics(Map<String, FileTypeMetrics> fileTypeMetricsMap) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we can try combining the 2 constructors from the above comment

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Better to keep these constructors separate otherwise from all other places we will need to pass empty map or null.

private final long intervalNanos;

// Next allowed time to acquire a permit in nanoseconds.
private long nextAllowedTime;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we have this as atomic/volatile?

Copy link
Contributor Author

@bhattmanish98 bhattmanish98 Jan 6, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is an overhead if we make this variable Atomic along with synchronized method where it is getting used.


import org.apache.hadoop.fs.azurebfs.contracts.exceptions.InvalidConfigurationValueException;

public final class SimpleRateLimiter {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: we can add some class definition here and if its local to JVM

ALL_ID_FORMAT, // <client-correlation-id>:<client-req-id>:<filesystem-id>
// :<primary-req-id>:<stream-id>:<hdfs-operation>:<retry-count>

AGGREGATED_METRICS_FORMAT; // <client-correlation-id>:<filesystem-id>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: we can have the comment in same line

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 15m 29s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 10 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 34m 22s trunk passed
+1 💚 compile 0m 45s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 compile 0m 45s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 checkstyle 0m 33s trunk passed
+1 💚 mvnsite 0m 52s trunk passed
+1 💚 javadoc 0m 44s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javadoc 0m 40s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
-1 ❌ spotbugs 1m 22s /branch-spotbugs-hadoop-tools_hadoop-azure-warnings.html hadoop-tools/hadoop-azure in trunk has 1 extant spotbugs warnings.
+1 💚 shadedclient 26m 24s branch has no errors when building and testing our client artifacts.
-0 ⚠️ patch 26m 46s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 38s the patch passed
+1 💚 compile 0m 37s the patch passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 37s the patch passed
+1 💚 compile 0m 37s the patch passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 37s the patch passed
+1 💚 blanks 0m 1s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 22s /results-checkstyle-hadoop-tools_hadoop-azure.txt hadoop-tools/hadoop-azure: The patch generated 9 new + 5 unchanged - 0 fixed = 14 total (was 5)
+1 💚 mvnsite 0m 39s the patch passed
-1 ❌ javadoc 0m 32s /patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-21.0.7+6-Ubuntu-0ubuntu120.04.txt hadoop-azure in the patch failed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04.
-1 ❌ javadoc 0m 29s /patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-17.0.15+6-Ubuntu-0ubuntu120.04.txt hadoop-azure in the patch failed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04.
+1 💚 spotbugs 1m 20s the patch passed
+1 💚 shadedclient 26m 31s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 3m 19s hadoop-azure in the patch passed.
+1 💚 asflicense 0m 35s The patch does not generate ASF License warnings.
119m 55s
Subsystem Report/Notes
Docker ClientAPI=1.52 ServerAPI=1.52 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/6/artifact/out/Dockerfile
GITHUB PR #8137
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux b0d87e2631ae 5.15.0-161-generic #171-Ubuntu SMP Sat Oct 11 08:17:01 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / c89af43
Default Java Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Multi-JDK versions /usr/lib/jvm/java-21-openjdk-amd64:Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-17-openjdk-amd64:Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/6/testReport/
Max. process+thread count 769 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/6/console
versions git=2.25.1 maven=3.9.11 spotbugs=4.9.7
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@bhattmanish98
Copy link
Contributor Author

============================================================
HNS-OAuth-DFS

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 906, Failures: 0, Errors: 0, Skipped: 220
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
HNS-SharedKey-DFS

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 4
[WARNING] Tests run: 909, Failures: 0, Errors: 0, Skipped: 166
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 10

============================================================
NonHNS-SharedKey-DFS

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 746, Failures: 0, Errors: 0, Skipped: 287
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 11

============================================================
AppendBlob-HNS-OAuth-DFS

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 906, Failures: 0, Errors: 0, Skipped: 231
[WARNING] Tests run: 135, Failures: 0, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
NonHNS-SharedKey-Blob

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 754, Failures: 0, Errors: 0, Skipped: 144
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 11

============================================================
NonHNS-OAuth-DFS

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 743, Failures: 0, Errors: 0, Skipped: 289
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
NonHNS-OAuth-Blob

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 751, Failures: 0, Errors: 0, Skipped: 156
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
AppendBlob-NonHNS-OAuth-Blob

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 746, Failures: 0, Errors: 0, Skipped: 202
[WARNING] Tests run: 135, Failures: 0, Errors: 0, Skipped: 4
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

============================================================
HNS-Oauth-DFS-IngressBlob

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 3
[WARNING] Tests run: 780, Failures: 0, Errors: 0, Skipped: 229
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 8
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 23

============================================================
NonHNS-OAuth-DFS-IngressBlob

[WARNING] Tests run: 237, Failures: 0, Errors: 0, Skipped: 10
[WARNING] Tests run: 743, Failures: 0, Errors: 0, Skipped: 286
[WARNING] Tests run: 158, Failures: 0, Errors: 0, Skipped: 9
[WARNING] Tests run: 271, Failures: 0, Errors: 0, Skipped: 24

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 9s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 10 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 35m 20s trunk passed
+1 💚 compile 0m 45s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 compile 0m 47s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 checkstyle 0m 33s trunk passed
+1 💚 mvnsite 0m 51s trunk passed
+1 💚 javadoc 0m 43s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javadoc 0m 40s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
-1 ❌ spotbugs 1m 23s /branch-spotbugs-hadoop-tools_hadoop-azure-warnings.html hadoop-tools/hadoop-azure in trunk has 1 extant spotbugs warnings.
+1 💚 shadedclient 26m 25s branch has no errors when building and testing our client artifacts.
-0 ⚠️ patch 26m 46s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 39s the patch passed
+1 💚 compile 0m 36s the patch passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 36s the patch passed
+1 💚 compile 0m 35s the patch passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 35s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 22s the patch passed
+1 💚 mvnsite 0m 41s the patch passed
-1 ❌ javadoc 0m 32s /patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-21.0.7+6-Ubuntu-0ubuntu120.04.txt hadoop-azure in the patch failed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04.
-1 ❌ javadoc 0m 30s /patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-17.0.15+6-Ubuntu-0ubuntu120.04.txt hadoop-azure in the patch failed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04.
+1 💚 spotbugs 1m 21s the patch passed
+1 💚 shadedclient 26m 23s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 3m 19s hadoop-azure in the patch passed.
+1 💚 asflicense 0m 35s The patch does not generate ASF License warnings.
106m 21s
Subsystem Report/Notes
Docker ClientAPI=1.52 ServerAPI=1.52 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/7/artifact/out/Dockerfile
GITHUB PR #8137
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux 81a41bf58fe6 5.15.0-161-generic #171-Ubuntu SMP Sat Oct 11 08:17:01 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 3f8af3d
Default Java Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Multi-JDK versions /usr/lib/jvm/java-21-openjdk-amd64:Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-17-openjdk-amd64:Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/7/testReport/
Max. process+thread count 640 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/7/console
versions git=2.25.1 maven=3.9.11 spotbugs=4.9.7
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 1m 11s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 10 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 34m 42s trunk passed
+1 💚 compile 0m 47s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 compile 0m 45s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 checkstyle 0m 34s trunk passed
+1 💚 mvnsite 0m 50s trunk passed
+1 💚 javadoc 0m 44s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javadoc 0m 39s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
-1 ❌ spotbugs 1m 20s /branch-spotbugs-hadoop-tools_hadoop-azure-warnings.html hadoop-tools/hadoop-azure in trunk has 1 extant spotbugs warnings.
+1 💚 shadedclient 26m 28s branch has no errors when building and testing our client artifacts.
-0 ⚠️ patch 26m 50s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 38s the patch passed
+1 💚 compile 0m 36s the patch passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 36s the patch passed
+1 💚 compile 0m 37s the patch passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 37s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 22s the patch passed
+1 💚 mvnsite 0m 40s the patch passed
-1 ❌ javadoc 0m 31s /patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-21.0.7+6-Ubuntu-0ubuntu120.04.txt hadoop-azure in the patch failed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04.
-1 ❌ javadoc 0m 30s /patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-17.0.15+6-Ubuntu-0ubuntu120.04.txt hadoop-azure in the patch failed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04.
+1 💚 spotbugs 1m 19s the patch passed
+1 💚 shadedclient 26m 12s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 3m 20s hadoop-azure in the patch passed.
+1 💚 asflicense 0m 33s The patch does not generate ASF License warnings.
105m 28s
Subsystem Report/Notes
Docker ClientAPI=1.52 ServerAPI=1.52 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/8/artifact/out/Dockerfile
GITHUB PR #8137
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux 60dde8514129 5.15.0-161-generic #171-Ubuntu SMP Sat Oct 11 08:17:01 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / fbd203f
Default Java Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Multi-JDK versions /usr/lib/jvm/java-21-openjdk-amd64:Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-17-openjdk-amd64:Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/8/testReport/
Max. process+thread count 641 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/8/console
versions git=2.25.1 maven=3.9.11 spotbugs=4.9.7
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

if (runningTimerTask != null) {
runningTimerTask.cancel();
}
if (timer != null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if (timer != null) {
timer.cancel();
timer.purge();
timer = null;
} #7852 as per this

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

timer.purge is redundant after timer.cancel
Will add timer = null.

* @param metricsData Metrics data to record.
*/
public void recordMetric(String accountName, String metricsData) {
if (StringUtils.isEmpty(accountName)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should have null check for metricsData as we are returning null from metrics manager

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

StringUtils.isEmpty() does both null and empty string check.

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 15m 24s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 10 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 35m 19s trunk passed
+1 💚 compile 0m 43s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 compile 0m 45s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 checkstyle 0m 33s trunk passed
+1 💚 mvnsite 0m 50s trunk passed
+1 💚 javadoc 0m 45s trunk passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javadoc 0m 40s trunk passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
-1 ❌ spotbugs 1m 23s /branch-spotbugs-hadoop-tools_hadoop-azure-warnings.html hadoop-tools/hadoop-azure in trunk has 1 extant spotbugs warnings.
+1 💚 shadedclient 26m 30s branch has no errors when building and testing our client artifacts.
-0 ⚠️ patch 26m 52s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 38s the patch passed
+1 💚 compile 0m 35s the patch passed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 35s the patch passed
+1 💚 compile 0m 37s the patch passed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
+1 💚 javac 0m 37s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 23s the patch passed
+1 💚 mvnsite 0m 40s the patch passed
-1 ❌ javadoc 0m 32s /patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-21.0.7+6-Ubuntu-0ubuntu120.04.txt hadoop-azure in the patch failed with JDK Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04.
-1 ❌ javadoc 0m 30s /patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-17.0.15+6-Ubuntu-0ubuntu120.04.txt hadoop-azure in the patch failed with JDK Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04.
+1 💚 spotbugs 1m 21s the patch passed
+1 💚 shadedclient 26m 20s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 3m 21s hadoop-azure in the patch passed.
+1 💚 asflicense 0m 36s The patch does not generate ASF License warnings.
120m 37s
Subsystem Report/Notes
Docker ClientAPI=1.52 ServerAPI=1.52 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/9/artifact/out/Dockerfile
GITHUB PR #8137
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux e23ba3e596d0 5.15.0-161-generic #171-Ubuntu SMP Sat Oct 11 08:17:01 UTC 2025 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 655dcb1
Default Java Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Multi-JDK versions /usr/lib/jvm/java-21-openjdk-amd64:Ubuntu-21.0.7+6-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-17-openjdk-amd64:Ubuntu-17.0.15+6-Ubuntu-0ubuntu120.04
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/9/testReport/
Max. process+thread count 631 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-8137/9/console
versions git=2.25.1 maven=3.9.11 spotbugs=4.9.7
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants