Skip to content

Conversation

@vinkal-chudgar
Copy link
Contributor

@vinkal-chudgar vinkal-chudgar commented Oct 22, 2025

Fixes #24797

Motivation

In CI, the test backlogsAgeMetricsNoPreciseWithoutBacklogQuota fails intermittently with:

java.lang.ArrayIndexOutOfBoundsException: Index 0 out of bounds for length 0
    at org.apache.pulsar.broker.service.BacklogQuotaManagerTest.getQuotaCheckCount(BacklogQuotaManagerTest.java:960)
    at org.apache.pulsar.broker.service.BacklogQuotaManagerTest.waitForQuotaCheckToRunTwice(BacklogQuotaManagerTest.java:937)

Source: BacklogQuotaManagerTest.xml, failure section.

What the test currently does
The test reads the histogram counter _count for the quota check duration:

// BacklogQuotaManagerTest.java
private long getQuotaCheckCount() {
    Metrics metrics = prometheusMetricsClient.getMetrics();
    return (long) metrics.findByNameAndLabels(
            "pulsar_storage_backlog_quota_check_duration_seconds_count",
            "cluster", CLUSTER_NAME)
        .get(0).value;   // indexes the first sample unconditionally
}

private void waitForQuotaCheckToRunTwice() {
    final long initial = getQuotaCheckCount();
    Awaitility.await()
            .pollInterval(1, SECONDS)
            .atMost(TIME_TO_CHECK_BACKLOG_QUOTA * 3, SECONDS)   // TIME_TO_CHECK_BACKLOG_QUOTA = 2
            .until(() -> getQuotaCheckCount() > initial + 1);
}

What actually happened in the failing run
During the first scrape performed by the test, the Prometheus servlet failed while generating /metrics, which resulted in no metric families being emitted for that scrape.

Evidence from BacklogQuotaManagerTest.xml:

  • The servlet logs the failure, then the access log shows HTTP 500 for /metrics
2025-09-27T20:17:07,196 - ERROR - [prometheus-stats-4342-1:PulsarPrometheusMetricsServlet] - Failed to generate metrics
2025-09-27T20:17:07,197 - INFO  - [pulsar-web-4341-8:RequestLog] - 127.0.0.1 - - [27/Sep/2025:20:17:07 +0000] "GET /metrics/ HTTP/1.1" 500 0 "-" "Apache-HttpClient/4.5.13 (Java/21.0.8)" 7
  • The stack for the failure shows a collector throwing DisabledMockException during servlet rendering
Caused by: org.mockito.exceptions.misusing.DisabledMockException: Mock accessed after inline mocks were cleared
	at org.apache.pulsar.broker.PulsarService.getConfig(PulsarService.java:215) ~[classes/:4.2.0-SNAPSHOT]
	at org.apache.pulsar.broker.service.BrokerService.getPendingLookupRequest(BrokerService.java:472) ~[classes/:?]
	at org.apache.pulsar.broker.stats.prometheus.metrics.ObserverGauge.collect(ObserverGauge.java:96) ~[classes/:?]
	at io.prometheus.client.Collector.collect(Collector.java:45) ~[simpleclient-0.16.0.jar:?]
	at io.prometheus.client.CollectorRegistry$MetricFamilySamplesEnumeration.findNextElement(CollectorRegistry.java:204) ~[simpleclient-0.16.0.jar:?]

This explains why the client saw an empty match list for the histogram family during that scrape. The current test code then performs .get(0) on an empty list which throws the ArrayIndexOutOfBoundsException shown in the failure section.

Why a collector exception makes /metrics empty for that scrape

  • Pulsar exposes metrics through a servlet that iterates the CollectorRegistry to build the response. If a collector throws while the servlet is rendering, the servlet logs the error and the request produces HTTP 500. This behavior is directly visible in the cited log lines above.

  • The failing collector in the stack is ObserverGauge.collect. The ObserverGauge implementation calls each child supplier while collecting:

// ObserverGauge.collect()
samples.add(new MetricFamilySamples.Sample(fullname, labelNames, c.getKey(),
                    child.supplier.get().doubleValue()));  // supplier exception propagates
  • The supplier in question dereferences pulsar.getConfig() inside BrokerService.getPendingLookupRequest():
     private int getPendingLookupRequest() {
         return pulsar.getConfig().getMaxConcurrentLookupRequest() - lookupRequestSemaphore.get().availablePermits();
     }
    

The stack shows PulsarService.getConfig at the top of the cause chain. The exception message confirms the inline mock had been cleared at that time, which is consistent with DisabledMockException.

Conclusion
The failure was caused by indexing the first element without verifying that the _count sample was present.. In this CI run, the servlet returned HTTP 500 while the test scraped. The client received no samples for that scrape which made the list empty. The subsequent .get(0) caused ArrayIndexOutOfBoundsException.

Modifications

Scope: test-only changes in BacklogQuotaManagerTest. No broker or metrics code is changed.

1. Make the metric read method tolerant of empty scrapes

getQuotaCheckCount() now returns OptionalLong. It returns OptionalLong.empty() when a scrape has no sample for the expected name and label, for example when the servlet returns 500 or when the series has not yet appeared. This removes the .get(0) crash.

    private static final String QUOTA_CHECK_COUNT = "pulsar_storage_backlog_quota_check_duration_seconds_count";
    
    private java.util.OptionalLong getQuotaCheckCount() {
        try {
            final Metrics metrics = prometheusMetricsClient.getMetrics();
            final java.util.List<Metric> matches =
                    metrics.findByNameAndLabels(QUOTA_CHECK_COUNT, "cluster", CLUSTER_NAME);
            if (matches.isEmpty()) {
                // No metric sample for this name and labels in this scrape.
                return java.util.OptionalLong.empty();
            }
            return java.util.OptionalLong.of((long) matches.get(0).value);
        } catch (Exception e) {
            // Scrape failed or the metrics client threw, treat as not available now.
            return java.util.OptionalLong.empty();
        }
    }

2. Capture the baseline only after a successful scrape, then require two checks
waitForQuotaCheckToRunTwice() continues to poll every 1 second and uses the same overall bound TIME_TO_CHECK_BACKLOG_QUOTA * 3. It still asserts two checks after a baseline.
The change is when the baseline is taken: the method now waits until /metrics returns a sample for the series, records that value as the baseline, then requires the counter to increase by two.

    private void waitForQuotaCheckToRunTwice() {
        final long[] baselineCount = new long[1];
        final boolean[] baselineCaptured = new boolean[1];

        Awaitility.await()
                .pollInterval(1, SECONDS)
                .atMost(TIME_TO_CHECK_BACKLOG_QUOTA * 3, SECONDS)
                .until(() -> {
                    final java.util.OptionalLong countOpt = getQuotaCheckCount();

                    // If /metrics is not returning the metric yet, keep waiting.
                    // Don't take the baseline until a successful scrape shows the metric.
                    if (countOpt.isEmpty()) {
                        return false;
                    }

                    // First successful scrape: capture baseline, then ask for two more checks.
                    final long observedCount = countOpt.getAsLong();
                    if (!baselineCaptured[0]) {
                        baselineCount[0] = observedCount;
                        baselineCaptured[0] = true;
                        return false;
                    }

                    return observedCount > baselineCount[0] + 1;
                });
    }

Justification for this change

  • In the failing CI run, /metrics servlet returned HTTP 500 while the test was scraping. When /metrics servlet returns 500, the client returns no data for that scrape. If the test takes its baseline during such an outage, it can record a value like 0 even if the counter had already advanced. On the next successful scrape, the value might already be greater than baseline + 1, which can produce a false pass by counting increments that occurred before the baseline. The fix avoids taking a baseline during an outage by waiting for the first successful scrape that shows the sample, then requiring two increments after that point. This keeps the original intent intact and removes the flake.

  • This change does not mask real failures. If the servlet continues to fail due to DisabledMockException and the sample never becomes visible, the await times out within the same overall bound, and the test fails deterministically rather than crashing or falsely passing. The failing CI case showed exactly this failure mode at scrape time.

Verifying this change

  • Make sure that the change passes the CI checks.

Reproduction snippet for reviewers
This section is optional and not part of the PR. It demonstrates the original crash locally. Do not commit any temporary edits from this section.

Prerequisite
You are on a commit before the fix, so the two helper methods already have their original implementations shown below. If you are on the PR branch that contains the fix, temporarily replace the fixed helpers with these originals before running this reproducer, then restore the fixed helpers after you finish.

1. Verify the original helpers in BacklogQuotaManagerTest.java

// Original implementations for local reproduction only.
// Do not commit these definitions.
private void waitForQuotaCheckToRunTwice() {
    final long initialQuotaCheckCount = getQuotaCheckCount();
    Awaitility.await()
            .pollInterval(1, SECONDS)
            .atMost(TIME_TO_CHECK_BACKLOG_QUOTA * 3, SECONDS)
            .until(() -> getQuotaCheckCount() > initialQuotaCheckCount + 1);
}

private long getQuotaCheckCount() {
    Metrics metrics = prometheusMetricsClient.getMetrics();
    return (long) metrics.findByNameAndLabels(
                    "pulsar_storage_backlog_quota_check_duration_seconds_count",
                    "cluster", CLUSTER_NAME)
            .get(0).value; // indexes into an empty list during a failing scrape
}

2. Add a temporary test that forces one failing scrape
It forces one failing /metrics scrape by registering a short-lived ObserverGauge whose supplier throws exactly once. The first scrape fails and returns no samples, which makes the original getQuotaCheckCount() index into an empty list and throw ArrayIndexOutOfBoundsException. The original getQuotaCheckCount() then indexes an empty list and causes ArrayIndexOutOfBoundsException. The gauge is labeled so it participates in metrics output and is unregistered in finally

// Reviewer-only local reproducer. Do not commit. Run just this method locally
@Test(timeOut = 60000)
    public void backlogQuotaMetricsReproOriginalCrash() {
        final java.util.concurrent.atomic.AtomicBoolean throwOnce =
                new java.util.concurrent.atomic.AtomicBoolean(true);

        final ObserverGauge observerGauge = ObserverGauge.build("pulsar_test_fail_once",
                        "Fail exactly one scrape")
                .labelNames("cluster")
                .supplier(() -> {
                    if (throwOnce.getAndSet(false)) {
                        throw new RuntimeException("fail_once");
                    }
                    return 0;
                })
                .register();

        try {
            observerGauge.labels(CLUSTER_NAME);
            waitForQuotaCheckToRunTwice();
            org.testng.Assert.fail("Expected ArrayIndexOutOfBoundsException from original code");
        } finally {
            io.prometheus.client.CollectorRegistry.defaultRegistry.unregister(observerGauge);
        }
    }

3. Run only this test.
./mvnw -pl pulsar-broker -Dtest=BacklogQuotaManagerTest#backlogQuotaMetricsReproOriginalCrash -DforkCount=1 -DreuseForks=false -Dsurefire.printSummary=true -Dtestng.verbose=2 test

4. Expected result
the test throws ArrayIndexOutOfBoundsException, proving the crash is reproducible when a scrape returns no samples.

5. Cleanup
Delete the temporary test method.

Personal CI Results

Tested in Personal CI fork: vinkal-chudgar#2

Status: All checks have passed (47 successful checks, 3 skipped)

Does this pull request potentially affect one of the following parts:

If the box was checked, please highlight the changes

  • Dependencies (add or upgrade a dependency)
  • The public API
  • The schema
  • The default values of configurations
  • The threading model
  • The binary protocol
  • The REST endpoints
  • The admin CLI options
  • The metrics
  • Anything that affects deployment

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

Matching PR in forked repository

PR in forked repository: vinkal-chudgar#2

…tBacklogQuota handle empty /metrics scrape

Signed-off-by: Vinkal Chudgar <[email protected]>
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Oct 22, 2025
@vinkal-chudgar vinkal-chudgar marked this pull request as ready for review October 22, 2025 14:00
@Technoboy- Technoboy- added this to the 4.2.0 milestone Oct 23, 2025
@vinkal-chudgar
Copy link
Contributor Author

@Technoboy- Thanks for the review and approval. CI shows two unrelated test failures that match known issues.

1. CI - Unit - Brokers - Broker Group 1

Failing test: org.apache.pulsar.broker.service.ReplicatorTest.testResumptionAfterBacklogRelaxed
Known issue: #24789

2. CI - Unit - Brokers - Client Impl

Failing test: org.apache.pulsar.client.api.PatternConsumerBackPressureTest.testInfiniteGetThousandsTopics
Known issue: #24827

This PR is test only and modifies BacklogQuotaManagerTest in the broker module. No production code or client code is touched.
Could you please rerun the failed jobs, or let me know if any action is needed on my side.

@Technoboy-
Copy link
Contributor

/pulsarbot rerun-failure-checks

2 similar comments
@Technoboy-
Copy link
Contributor

/pulsarbot rerun-failure-checks

@Technoboy-
Copy link
Contributor

/pulsarbot rerun-failure-checks

@Technoboy- Technoboy- closed this Oct 27, 2025
@Technoboy- Technoboy- reopened this Oct 27, 2025
@codecov-commenter
Copy link

codecov-commenter commented Oct 27, 2025

Codecov Report

✅ All modified and coverable lines are covered by tests.
✅ Project coverage is 74.25%. Comparing base (678db6b) to head (1c9414d).
⚠️ Report is 3 commits behind head on master.

Additional details and impacted files

Impacted file tree graph

@@             Coverage Diff              @@
##             master   #24887      +/-   ##
============================================
- Coverage     74.34%   74.25%   -0.10%     
+ Complexity    33496    33422      -74     
============================================
  Files          1913     1913              
  Lines        149315   149496     +181     
  Branches      17331    17391      +60     
============================================
- Hits         111012   111008       -4     
- Misses        29475    29646     +171     
- Partials       8828     8842      +14     
Flag Coverage Δ
inttests 26.71% <ø> (-0.40%) ⬇️
systests 22.83% <ø> (-0.19%) ⬇️
unittests 73.73% <ø> (-0.13%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.
see 94 files with indirect coverage changes

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.
  • 📦 JS Bundle Analysis: Save yourself from yourself by tracking and limiting bundle sizes in JS merges.

@Technoboy- Technoboy- merged commit ba8468e into apache:master Oct 27, 2025
286 of 303 checks passed
manas-ctds pushed a commit to datastax/pulsar that referenced this pull request Oct 28, 2025
…tBacklogQuota handle empty /metrics scrape (apache#24887)

Signed-off-by: Vinkal Chudgar <[email protected]>
(cherry picked from commit ba8468e)
lhotari pushed a commit that referenced this pull request Oct 28, 2025
…tBacklogQuota handle empty /metrics scrape (#24887)

Signed-off-by: Vinkal Chudgar <[email protected]>
(cherry picked from commit ba8468e)
lhotari pushed a commit that referenced this pull request Oct 28, 2025
…tBacklogQuota handle empty /metrics scrape (#24887)

Signed-off-by: Vinkal Chudgar <[email protected]>
(cherry picked from commit ba8468e)
manas-ctds pushed a commit to datastax/pulsar that referenced this pull request Oct 29, 2025
…tBacklogQuota handle empty /metrics scrape (apache#24887)

Signed-off-by: Vinkal Chudgar <[email protected]>
(cherry picked from commit ba8468e)
(cherry picked from commit 42e02a8)
srinath-ctds pushed a commit to datastax/pulsar that referenced this pull request Nov 6, 2025
…tBacklogQuota handle empty /metrics scrape (apache#24887)

Signed-off-by: Vinkal Chudgar <[email protected]>
(cherry picked from commit ba8468e)
(cherry picked from commit 42e02a8)
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.

Flaky-test: BacklogQuotaManagerTest.backlogsAgeMetricsNoPreciseWithoutBacklogQuota

4 participants