Producer Tuning and Idempotency
Most Kafka performance problems I've seen in the field weren't broker problems—they were producer problems. Teams copy a hello-world config, ship it to production, and wonder why throughput is half what they expected or why they're seeing duplicate messages under retry. The producer is where the performance and reliability story actually starts.
This post works through the four knobs that matter most: acknowledgment semantics, batching, compression, and idempotency. Each one has a real tradeoff, and the defaults are almost never right for production.
Acknowledgment Semantics: What acks Actually Means
acks is the single most consequential producer setting. It controls when the broker tells the producer "I got it."
acks=0— Fire and forget. No confirmation. Fastest, no durability.acks=1— Leader confirms write. Follower replication is async. Leader failure before replication = lost message.acks=all(or-1) — Leader waits for all in-sync replicas (ISR) to confirm. No message loss as long as at least one ISR survives.
at least 2 replicas must confirm
acks=all without min.insync.replicas set at the topic or broker level is a trap. If your ISR shrinks to one broker (the leader itself), acks=all still succeeds—you just confirmed to yourself. Set min.insync.replicas=2 to prevent phantom durability:
# Broker-level default (server.properties)
min.insync.replicas=2
default.replication.factor=3// Producer config
Properties props = new Properties();
props.put(ProducerConfig.ACKS_CONFIG, "all");
props.put(ProducerConfig.RETRIES_CONFIG, Integer.MAX_VALUE);
props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 5);
// With idempotence enabled, ordering is preserved even with retries
props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);Batching: The Throughput Multiplier
The Kafka producer buffers records into batches before sending. Larger batches mean fewer network round trips and better compression ratios—but higher latency. Two settings control this:
batch.size— Maximum batch size in bytes. Default is 16 KB. For high-throughput topics, 256 KB or 512 KB is common.linger.ms— How long to wait for a batch to fill before sending anyway. Default is 0 (send immediately). Setting this to 5–20 ms gives the producer time to accumulate records.
props.put(ProducerConfig.BATCH_SIZE_CONFIG, 262144); // 256 KB
props.put(ProducerConfig.LINGER_MS_CONFIG, 10); // wait up to 10ms
props.put(ProducerConfig.BUFFER_MEMORY_CONFIG, 67108864); // 64 MB total bufferfrom confluent_kafka import Producer
producer = Producer({
"bootstrap.servers": "broker1:9092,broker2:9092",
"acks": "all",
"batch.size": 262144,
"linger.ms": 10,
"compression.type": "lz4",
"enable.idempotence": True,
"retries": 2147483647,
"max.in.flight.requests.per.connection": 5,
})The interaction between linger.ms and batch.size: whichever threshold is hit first triggers a send. At high throughput batch.size fills before linger.ms expires. At low throughput linger.ms is what triggers the send.
Compression: Always On, Right Codec
Compression is almost always worth enabling. The CPU cost on producer and broker is low compared to the network and disk savings. At RF=3 with lz4, you can cut wire bytes by 40–70% on typical JSON payloads.
Codec comparison for Kafka workloads:
| Codec | Ratio | CPU Cost | Best For |
|---|---|---|---|
| none | 1.0× | none | Already-binary data |
| gzip | 3–5× | High | Archival, batch |
| snappy | 2–3× | Low | General use |
| lz4 | 2–4× | Very low | High-throughput, latency-sensitive |
| zstd | 3–6× | Medium | Best ratio at acceptable cost |
lz4 is the default choice for most real-time pipelines. zstd is worth evaluating if you're disk-constrained and can afford slightly more CPU.
props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "lz4");One gotcha: compression happens at the batch level, not per-record. A batch with one record compresses poorly. This is another reason linger.ms > 0 matters—more records per batch means better compression ratios.
Idempotent Producer: Exactly-Once at the Producer Level
Without idempotency, producer retries can create duplicates. The sequence: producer sends a batch, the broker appends it, the network drops the ack, the producer retries, the broker appends a duplicate. With enable.idempotence=true, the broker tracks a producer ID (PID) and sequence number per partition and deduplicates silently.
// Minimum correct config for idempotent producer
Properties props = new Properties();
props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
// These are set automatically when idempotence is enabled,
// but being explicit is clearer:
props.put(ProducerConfig.ACKS_CONFIG, "all");
props.put(ProducerConfig.RETRIES_CONFIG, Integer.MAX_VALUE);
props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 5);What idempotency does NOT cover:
- It's scoped to a single producer session. If the producer process restarts, it gets a new PID. Records written before the restart can still be duplicated on retry.
- It does not coordinate across multiple producer instances writing to the same topic.
- It does not span multiple topics (that requires transactions, covered in post 6).
Putting It Together: Production Template
public KafkaProducer<String, byte[]> buildProducer(String bootstrapServers) {
Properties props = new Properties();
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
StringSerializer.class.getName());
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
ByteArraySerializer.class.getName());
// Durability
props.put(ProducerConfig.ACKS_CONFIG, "all");
props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
// Throughput
props.put(ProducerConfig.BATCH_SIZE_CONFIG, 262144);
props.put(ProducerConfig.LINGER_MS_CONFIG, 10);
props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "lz4");
props.put(ProducerConfig.BUFFER_MEMORY_CONFIG, 67108864L);
// Resilience
props.put(ProducerConfig.RETRIES_CONFIG, Integer.MAX_VALUE);
props.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, 120000);
props.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, 30000);
props.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 60000);
return new KafkaProducer<>(props);
}Monitor these producer-side metrics in production:
| Metric | Healthy | Investigate |
|---|---|---|
record-error-rate |
0 | > 0 |
record-retry-rate |
< 0.1% | > 1% |
batch-size-avg |
Close to batch.size |
Much smaller (linger too low) |
compression-rate-avg |
< 0.6 for lz4 | > 0.9 (compression not working) |
buffer-available-bytes |
> 20% | < 10% (producer backpressure) |
Key Takeaways
acks=allwithoutmin.insync.replicas=2is a false guarantee—a shrunk ISR lets the leader ack to itself, and you lose the durability you thought you had.linger.ms=10is almost always better than the default 0—it costs 10ms of latency in exchange for larger batches, better compression, and meaningfully higher throughput.- lz4 compression is nearly free CPU-wise and typically saves 40–70% on wire bytes—turn it on by default; the only exception is already-compressed binary payloads.
- Idempotent producer eliminates retry-induced duplicates within a single producer session—it does not survive process restarts and does not coordinate across multiple producers.
retries=MAX_VALUEcombined withdelivery.timeout.msis safer than a fixed retry count—you set the deadline, not the number of attempts, and let the producer decide how many fits.- Batch fill rate is your throughput signal—if
batch-size-avgis tiny compared tobatch.size, your linger window is too short or your throughput is too low to justify the current batch size.