Skip to content

Instantly share code, notes, and snippets.

@InKolev
Last active March 14, 2026 15:59
Show Gist options
  • Select an option

  • Save InKolev/e67c9ae0dadc4313f6cf2d14b8bf625a to your computer and use it in GitHub Desktop.

Select an option

Save InKolev/e67c9ae0dadc4313f6cf2d14b8bf625a to your computer and use it in GitHub Desktop.

librdkafka Performance Tuning Guide

Target version: librdkafka v2.13.x (latest stable as of March 2026) Applies to: All language bindings built on librdkafka - confluent-kafka-python, confluent-kafka-go, confluent-kafka-dotnet, node-rdkafka, Karafka/rdkafka-ruby, and others.


1. Building Kafka Performance from the Hardware Up

Before touching a single configuration property, ensure the underlying infrastructure matches the workload profile.

Performance Goal Primary Hardware Lever Notes
Higher message retention Disk capacity More storage lets you keep data longer without aggressive log compaction.
Higher message throughput Network bandwidth Kafka is I/O-bound at scale; 10 GbE or bonded NICs remove the bottleneck.
Higher producer performance Disk I/O speed (NVMe/SSD) Broker-side fsync and log segment writes are the limiting factor.
Higher consumer performance Memory (page cache) The OS page cache serves reads directly from RAM if data is recent.
Lower end-to-end latency All of the above + low-latency NIC Latency is a system-wide property — no single upgrade fixes it alone.

Cloud-specific considerations: On cloud VMs, network bandwidth is often the first thing to saturate. Dedicated host types with guaranteed bandwidth (e.g., AWS i3en, GCP n2-highmem) and EBS-optimized instances materially improve broker throughput.


2. Understanding the Producer

2.1 Batch Lifecycle

A batch is dispatched to the broker when any one of the following becomes true:

  1. batch.num.messages is reached (message-count trigger).
  2. batch.size is reached (byte-size trigger).
  3. linger.ms timer expires (time-based trigger).
  4. Another batch destined for the same broker is already ready (piggybacking).
  5. flush() or close() is called by the application.

More batching produces better compression ratios and higher throughput, at the cost of added latency.

2.2 Why linger.ms > 0 Outperforms Zero-Wait

A small deliberate delay at the sender often reduces observed end-to-end latency. Larger batches mean fewer requests in the broker's serial processing pipeline, which reduces head-of-line blocking. Kafka 4.0 formalized this by changing the default linger.ms from 0 to 5 ms. In librdkafka, the equivalent queue.buffering.max.ms (alias linger.ms) defaults to 5. Even for latency-sensitive workloads, values of 510 ms usually outperform zero-wait.


3. Critical Producer Configurations

3.1 Batching

Property Description Default Tuning Guidance
batch.num.messages Max messages per MessageSet. 10000 Increase for high-throughput workloads (50000–100000).
batch.size Max total bytes per batch. 1000000 (≈1 MB) Increase to 2–4 MB for large messages or very high throughput.
linger.ms / queue.buffering.max.ms Max time to buffer before sending. 5 ms 5–100 ms for throughput; 0–5 ms for latency-sensitive systems.
queue.buffering.max.messages Max total messages on the producer queue. 100000 Size to accommodate bursts without blocking produce().
queue.buffering.max.kbytes Max total size of queued messages (KB). 1048576 (1 GB) Whichever limit (messages or kbytes) hits first wins.

batch.num.messages and batch.size control per-partition batch boundaries. queue.buffering.max.messages and queue.buffering.max.kbytes cap the total memory footprint across all partitions.

3.2 Compression

Property Description Default
compression.codec / compression.type Codec for compressing message sets. none

Compression is typically the dominant CPU cost of produce(). The choice of codec has a dramatic impact on both throughput and CPU utilization.

Codec Speed Ratio CPU Cost When to Use
none 1:1 Zero Tiny messages where compression overhead exceeds savings.
snappy Very fast Moderate Low General-purpose; great balance of speed and ratio.
lz4 Very fast Moderate Low Slightly better throughput than snappy in most benchmarks.
zstd Fast High Medium Best ratio; ideal when network or disk is the bottleneck. Requires broker ≥ 2.1 and librdkafka built with zstd support.
gzip Slow High High Legacy; rarely the best choice today.

Recommendation: Start with lz4 for general workloads. Use zstd when maximum compression ratio matters. Verify your build includes zstd support (builtin.features should list zstd). Static linking of zstd (≥ 1.2.1) further improves consumer decompression speed by encoding the original frame size.

3.3 Acknowledgements and Durability

Property Description Default
acks Acknowledgement level required from brokers. all
acks Throughput Latency Durability
0 Highest Lowest No guarantee — fire and forget.
1 Medium Medium Leader only — data loss possible if leader fails before replication.
all (-1) Lowest Highest Full ISR acknowledgement — strongest durability guarantee.

3.4 Idempotent Producer

Guarantees per-partition ordering and exactly-once delivery without duplicates, even under retries.

Property Description Default
enable.idempotence Enables the idempotent producer. false
max.in.flight.requests.per.connection Max unacknowledged requests per broker connection. 1000000

When enable.idempotence=true:

  • acks is forced to all.
  • max.in.flight is capped at 5 (protocol limit for idempotent producers).
  • Ordering is guaranteed per-partition, even with max.in.flight > 1.
  • Retries are handled internally with sequence numbers to prevent duplicates.

Throughput note: The cap of 5 in-flight requests applies per connection, not per partition. Unlike the Java client, librdkafka does not combine batches for multiple partitions into a single ProduceRequest. If throughput is critical and idempotency is not required, disabling it allows max.in.flight to remain uncapped. If you disable idempotence but need ordering, set max.in.flight.requests.per.connection=1.

Recommendation: Enable idempotence for all production workloads using acks=all. The throughput cost is modest and the durability gain is significant.

3.5 Message Delivery Timeout

Property Description Default
message.timeout.ms / delivery.timeout.ms Max time for a produced message to be delivered (including retries). 300000 (5 min)
message.send.max.retries / retries Max number of send retries. 2147483647
retry.backoff.ms Initial backoff between retries. 100
retry.backoff.max.ms Max backoff (exponential). 1000

Whichever limit (timeout or retry count) is hit first causes the message to fail with a delivery error.


4. Critical Consumer Configurations

4.1 Fetch Tuning

Property Description Default Tuning Guidance
queued.min.messages Min messages per topic+partition in the local consumer queue. 100000 Increase for high-throughput consumers; decrease if memory is constrained.
queued.max.messages.kbytes Max KB per topic+partition in the local queue. Higher priority than queued.min.messages. 65536 (64 MB) Cap per-partition memory usage.
fetch.wait.max.ms Max time the broker waits to fill fetch.min.bytes before responding. 500 Lower (50–100 ms) reduces latency; higher improves batching.
fetch.min.bytes Min bytes the broker accumulates before responding. 1 Increase (1024–65536) to reduce fetch request frequency.
fetch.message.max.bytes Initial max bytes per topic+partition per fetch. 1048576 (1 MB) Auto-increases if a message exceeds this value.
fetch.error.backoff.ms Delay before retrying after a fetch error. 500 Lower values recover faster; too low may spin-loop.
max.poll.interval.ms Max time between poll() calls before consumer is considered failed. 300000 (5 min) Increase for long-processing workloads; triggers rebalance if exceeded.

4.2 Offset Management

Property Description Default
enable.auto.commit Automatically commit offsets in the background. true
auto.commit.interval.ms Frequency of auto-commit (ms). 5000
enable.auto.offset.store Automatically store the offset of the last message provided to the application. true

Recommendation for at-least-once processing: Set enable.auto.offset.store=false and manually call offsets_store() after processing each message or batch. This prevents committing offsets for unprocessed messages.

4.3 KIP-848: Next-Generation Consumer Group Protocol

The new consumer group rebalance protocol (GA since librdkafka v2.12.0) eliminates stop-the-world rebalances. Requires Kafka 4.0+ (or Confluent Platform 8.0+ / Confluent Cloud).

Property Description Default
group.protocol Consumer group protocol: classic or consumer. classic
group.remote.assignor Server-side assignor: uniform, range, or null (broker decides). null

What it provides:

  • Partition assignment is calculated by the Group Coordinator (broker), not a client-side group leader.
  • Fully incremental — no stop-the-world rebalances.
  • Up to 20× faster rebalance times.
  • Assignments are distributed via heartbeats; no JoinGroup/SyncGroup phases.

Caveats:

  • Regex subscriptions use the broker's RE2/J engine (full-topic match). Patterns like ^topic must be ^topic.* to match topic-1, topic-2, etc.
  • Properties like partition.assignment.strategy and session.timeout.ms are not used under the new protocol.
  • Rack-aware assignment strategies are not yet supported.

Recommendation: Use group.protocol=consumer for all new deployments on Kafka 4.0+.


5. Network and Connection Tuning

Property Description Default Tuning Guidance
socket.send.buffer.bytes OS socket send buffer size (0 = system default). 0 Increase for high-throughput over high-latency links.
socket.receive.buffer.bytes OS socket receive buffer size (0 = system default). 0 Increase for consumers processing large volumes.
socket.keepalive.enable Enable TCP keepalive. false Set true for cloud/container deployments to survive idle connection reaping.
socket.nagle.disable Disable Nagle's algorithm (TCP_NODELAY). false Set true for latency-sensitive workloads.
connections.max.idle.ms Close idle broker connections after this time (0 = heuristic). 0 Azure deployments may benefit from an explicit value.
reconnect.backoff.ms Initial reconnect backoff (exponential). 100
reconnect.backoff.max.ms Max reconnect backoff. 10000
metadata.max.age.ms Max age of cached metadata before forced refresh. 900000 (15 min) Lower to 60000 for dynamic environments (autoscaling, containers).

6. Monitoring and Observability

Property Description Default
statistics.interval.ms Emit statistics JSON at this interval. 0 = disabled. 0
enable.metrics.push KIP-714 push-based telemetry to the broker. true

Register a stats callback (rd_kafka_conf_set_stats_cb) to receive a JSON blob with per-broker, per-topic, and per-partition metrics. Key metrics to monitor:

  • Producer: txmsgs, txbytes, msg_cnt and msg_size (queue depth), outbuf_cnt (requests waiting to send).
  • Consumer: rxmsgs, rxbytes, consumer_lag (offset lag per partition), fetchq_cnt (messages in fetch queue).
  • Broker-level: rtt (round-trip time), throttle (broker throttling), outbuf_cnt, waitresp_cnt (in-flight requests).

7. Quick-Reference Tuning Profiles

Maximum Throughput

# Producer
queue.buffering.max.ms=100
batch.num.messages=100000
batch.size=4194304
compression.codec=lz4
acks=1
enable.idempotence=false

# Consumer
group.protocol=consumer
queued.min.messages=1000000
fetch.min.bytes=65536
fetch.wait.max.ms=500

Minimum Latency

# Producer
queue.buffering.max.ms=0
batch.num.messages=1
acks=1
compression.codec=none

# Consumer
group.protocol=consumer
fetch.wait.max.ms=10
fetch.min.bytes=1
queued.min.messages=1

Balanced Production (Recommended Default)

# Producer
queue.buffering.max.ms=5
batch.size=1000000
compression.codec=lz4
acks=all
enable.idempotence=true

# Consumer
group.protocol=consumer
fetch.wait.max.ms=100
fetch.min.bytes=1024
enable.auto.offset.store=false

8. Further Reading

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