Improve sendsync performance#151
Open
omnibs wants to merge 5 commits into
Open
Conversation
$ KAFKA_BROKER_ADDRESSES=localhost:9092 BENCHMARK_MESSAGE_COUNT=200 BENCHMARK_WARMUP=20 cabal run sync-write-benchmark -fsync-write-benchmark # Without the fix Benchmarking sync writes: warmup=20 count=200 topic=nri-kafka-sync-benchmark Warming up... Measuring 200 sends... count=200 min=100.1ms avg=101.1ms p50=101.1ms p95=101.4ms p99=102.1ms max=102.1ms # With the fix Benchmarking sync writes: warmup=20 count=200 topic=nri-kafka-sync-benchmark Warming up... Measuring 200 sends... count=200 min=5.6ms avg=6.3ms p50=6.2ms p95=6.9ms p99=8.6ms max=8.8ms
Contributor
There was a problem hiding this comment.
Pull request overview
This PR reduces Kafka.sendSync latency by changing the producer event-polling loop to drain librdkafka delivery-report events promptly, and adds a small benchmark executable to measure per-message sync-write latency.
Changes:
- Replaces the
pollEventsloop’s fixed 100msthreadDelay+ empty-batch trick withProducer.flushProducerto wakesendSyncwaiters as soon as delivery reports arrive. - Adds a
sync-write-benchmarkexecutable (behind a Cabal flag) to benchmark sequentialsendSynclatency. - Updates Cabal metadata (
package.yaml+ generated.cabal) to include the new flag/executable.
Reviewed changes
Copilot reviewed 5 out of 5 changed files in this pull request and generated 1 comment.
| File | Description |
|---|---|
nri-kafka/src/Kafka.hs |
Switches the background event loop to use flushProducer and documents the rationale/shutdown behavior. |
nri-kafka/scripts/sync-write-benchmark/Main.hs |
New benchmark program for measuring sendSync per-message latency. |
nri-kafka/package.yaml |
Adds a flag-gated sync-write-benchmark executable definition. |
nri-kafka/nri-kafka.cabal |
Regenerated Cabal file reflecting the new flag/executable. |
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
Comment on lines
+111
to
+116
| formatStats :: [Data.Word.Word64] -> String | ||
| formatStats samples = | ||
| let sorted = Data.List.sort samples | ||
| n = Prelude.length sorted | ||
| total = Prelude.sum sorted | ||
| avg = nsToMs (total `Prelude.div` fromIntegral n) |
we're running out of space
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Context
Our synchronous writes to Kafka have been slow since forever, and we used to think that was on Kafka itself: high throughput but high latency.
Turns out it was a simple mistake on our side.
Synchronous sends are synchronous as much as Erlang/Elixir's synchronous messaging is synchronous: it's a wrapper around an async send that synchronously waits on an async ack.
When a kafka broker acks a message, librdkafka's I/O thread enqueues a delivery-report event onto an internal queue. That event sits there until some thread calls
rd_kafka_poll, which drains the queue and dispatches callbacks. Those callbacks free up pendingsendSynccalls.The issue
In our codebase,
pollEventsis what triggersrd_kafka_poll, and we used to have it like this:That 100ms delay means
sendSyncmight take up to 100ms to be waken up after an ack comes back from the broker.The fix
We could have simply reduced the delay, but there's another function called
flushProducer, which does something similar to what we did, but in a better way.flushProducerhangs on a call tord_kafka_pollfor up to 100ms. If no events come, it times out, and you can call it again. If an event comes, it return immediately.The 100ms timeout might seem superfluous, but ceding control back to Haskell allows us to respond to async exceptions and terminate our producers cleanly.
hw-kafka's own example producer uses this function.Benchmarks
We created a simple benchmark script here to test sync sends with and without the fix.
Without the fix
With the fix
This opens the possibility to adopt synchronous Kafka writes more broadly.
New issue
We uncovered a new issue while digging into this: we're not handling delivery failures in
sendSync.We just wrote a
known-issues.mddocumenting future work.