Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

MINOR: Fix a race and add JMH bench for HdrHistogram #17184

Merged

Conversation

dimitarndimitrov
Copy link
Contributor

About

Addresses the race described here and adds a JMH benchmark comparing HdrHistogram with Yammer's histogram.

  • A sample result from a local run on my development machine:
Benchmark                                        Mode  Cnt    Score     Error  Units
HistogramBenchmark.runner                        avgt    5  410.811 ±  95.204  ns/op
HistogramBenchmark.runner:read_HdrHistogram      avgt    5   50.839 ±  11.850  ns/op
HistogramBenchmark.runner:read_YammerHistogram   avgt    5   51.206 ±  11.789  ns/op
HistogramBenchmark.runner:write_HdrHistogram     avgt    5  314.767 ±  93.978  ns/op
HistogramBenchmark.runner:write_YammerHistogram  avgt    5  986.574 ± 214.357  ns/op

Testing

I haven't included a test reproducing the race, because on my development machine I wasn't able to see consistent reproducibility without more sophisticated testing tools like jcstress or Byteman/BMUnit. The race is easily reproducible manually while debugging the test.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@dimitarndimitrov
Copy link
Contributor Author

because on my development machine I wasn't able to see consistent reproducibility without more sophisticated testing tools like jcstress or Byteman/BMUnit.

After a bunch of experiments, I did arrive at this test, which on my development machine fails >50% without this change and passes 100% (in more than 500 attempts) with this change:

@Test
public void testLatestHistogramRace() throws InterruptedException {
    long maxSnapshotAgeMs = 10L;
    long now = System.currentTimeMillis();
    HdrHistogram hdrHistogram = new HdrHistogram(maxSnapshotAgeMs, MAX_VALUE, 3);
    for (int i = 1; i < 10000; i++) {
        int numEvents = 10;
        for (int j = 0; j < numEvents; j++) {
            hdrHistogram.record(i);
        }
        AtomicLong t1Counter = new AtomicLong();
        AtomicLong t2Counter = new AtomicLong();
        final long moreThanMaxAge = now + maxSnapshotAgeMs + 1;
        now = moreThanMaxAge;
        Thread t1 = new Thread(() -> t1Counter.set(hdrHistogram.count(moreThanMaxAge)));
        Thread t2 = new Thread(() -> t2Counter.set(hdrHistogram.count(moreThanMaxAge)));
        t1.start();
        t2.start();
        t1.join();
        t2.join();
        long t1Count = t1Counter.get();
        long t2Count = t2Counter.get();
        assertTrue(
            numEvents == t1Count && numEvents == t2Count,
            String.format("Expected %d events in both threads, got %d in T1 and %d in T2",
                numEvents, t1Count, t2Count));
    }
}

It does rely on numbers and it takes >2 sec locally when it succeeds, so I am inclined not to include it. Let me know if you prefer otherwise. I did spend some time trying to make the iteration body smaller by using a Phaser and reusing the threads, but that made the test less readable and didn't help with reproducibility, hence why I'm posting this version.

Comment on lines 63 to 64
private volatile Histogram yammerHistogram;
private volatile HdrHistogram hdrHistogram;
Copy link
Contributor Author

Choose a reason for hiding this comment

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

nit: On a second look these might not need to be volatile as the @Setup method probably happens-before the @Benchmark methods. The accesses through the volatile references are really cheap compared to what's being measured though, so it might also be OK to leave them as-is.

@dajac dajac added the KIP-848 The Next Generation of the Consumer Rebalance Protocol label Sep 13, 2024
Copy link
Contributor

@jeffkbkim jeffkbkim left a comment

Choose a reason for hiding this comment

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

Thanks for the PR! left some comments

Comment on lines 77 to 85
Timestamped<Histogram> latest = timestampedHistogramSnapshot.get();
while (now - latest.timestamp > maxSnapshotAgeMs) {
Histogram currentSnapshot = recorder.getIntervalHistogram();
boolean updatedLatest = timestampedHistogramSnapshot.compareAndSet(
latest, new Timestamped<>(now, currentSnapshot));

latest = timestampedHistogramSnapshot.get();
if (updatedLatest) {
break;
Copy link
Contributor

Choose a reason for hiding this comment

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

I was able to reproduce the bug mentioned and it fails deterministically in the previous implementation

    @Test
    public void testConcurrentAccessDoesNotLosePreviousState() {
        long maxSnapshotAgeMs = 10L;
        CountDownLatch latch = new CountDownLatch(1);
        Recorder mockRecorder = new MockRecorder(latch);
        HdrHistogram hdrHistogram = new HdrHistogram(maxSnapshotAgeMs, MAX_VALUE, 3, mockRecorder);

        AtomicReference<org.HdrHistogram.Histogram> asyncHistogram = new AtomicReference<>();
        org.HdrHistogram.Histogram mainHistogram;

        CompletableFuture<Void> asyncFuture = CompletableFuture.runAsync(() -> {
            asyncHistogram.set(hdrHistogram.latestHistogram(11));
        });

        mainHistogram = hdrHistogram.latestHistogram(11);

        asyncFuture.join();
        assertNotSame(asyncHistogram.get(), mainHistogram);
    }

    private static class MockRecorder extends Recorder {
        private final CountDownLatch latch;

        private final AtomicInteger invocationCounter = new AtomicInteger(0);

        public MockRecorder(CountDownLatch latch) {
            super(3);
            this.latch = latch;
        }

        @Override
        public org.HdrHistogram.Histogram getIntervalHistogram() {
            if (invocationCounter.compareAndSet(0, 1)) {
                try {
                    latch.await();
                } catch (Exception e) {
                    e.printStackTrace();
                }
            } else {
                latch.countDown();
            }
            return super.getIntervalHistogram();
        }
    }

This test hangs in the new implementation because the call to getIntervalHistogram is synchronized on lock. Which makes sense since we want to ensure that the thread that calls getIntervalHistogram is the only thread that can proceed (and update the state). I wonder if we can modify this test to pass when it hangs for x seconds. What do you think?

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 some improvements to the test I mentioned here, and as a result it now runs much faster (well under a second on my development machine) and fails reliably without this change (failed 500 out of 500 attempts locally when run via Gradle).

After these improvements I prefer the aforementioned test over the one here, as the latter requires changes in the histogram code, is still not deterministically proving things if we are going to make it pass on a timeout, and is a bit harder to understand.

@Measurement(iterations = 5)
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.NANOSECONDS)
public class HistogramBenchmark {
Copy link
Contributor

Choose a reason for hiding this comment

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

To understand the benchmark:

  • @group annotation allows the read and writes to happen concurrently
  • Simulate "slower" histogram reads compared to writes

Is this correct?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@group annotation allows the read and writes to happen concurrently

The @Group annotation allows the reads and writes to happen concurrently (if in the same group), but also asymmetrically, with some threads doing reads and some doing writes.
Check out https://github.com/melix/jmh-gradle-example/blob/master/src/jmh/java/org/openjdk/jmh/samples/JMHSample_15_Asymmetric.java

Simulate "slower" histogram reads compared to writes

Not slower, just more rare than writes, which is what we would expect in the general case.

@Benchmark
@Group("runner")
@GroupThreads(3)
public void write_YammerHistogram() {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: is this snake-case naming convention used in jmh benchmarks?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In some, but apparently not in the ones in AK, so I'll fix this (e.g. making this writeYammerHistogram).

@airlock-confluentinc airlock-confluentinc bot force-pushed the hdrhistogram-latesthistogram-race-fix branch from 2a14f85 to f1e7954 Compare September 17, 2024 15:28
@chia7712 chia7712 merged commit f1e7954 into apache:trunk Sep 17, 2024
13 of 15 checks passed
@dimitarndimitrov
Copy link
Contributor Author

@chia7712 did you merge this intentionally? This was in the process of being rebased in addition to having some of its comments addressed. I don't think reopening is an option now, which means the history will be split between this and a subsequent PR, but this shouldn't be that big of an issue.

@chia7712
Copy link
Contributor

I did not merge it ... this (GitHub) bug happened before but I do have no idea why Github so hate me 😢

I guess GitHub get confused by the force-push from the bot "airlock-confluentinc"?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
KIP-848 The Next Generation of the Consumer Rebalance Protocol
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants