Every Kafka Streams application with stateful operations uses RocksDB by default. Most teams discover this fact only after their production system starts exhibiting mysterious latency spikes, memory pressure, or disk I/O that saturates their SSDs.
I have debugged RocksDB-related Kafka Streams issues across multiple production deployments. The pattern is consistent: teams scale their Kafka infrastructure, tune their consumer configurations, optimize their topology, and still hit a wall. The wall is almost always RocksDB.
How We Got Here
RocksDB started at Facebook in 2012. The engineering team needed an embedded database that could handle their flash storage infrastructure. LevelDB existed but was optimized for hard drives. Facebook forked it and built RocksDB specifically for SSDs, with better parallelism and more tuning knobs.
The design goal was simple: fast writes on flash storage for workloads with high write volume and reasonable read latency requirements. Social media features like counters, timelines, and real-time analytics fit this pattern perfectly.
When Confluent built Kafka Streams in 2016, they needed an embedded state store. RocksDB was battle-tested at Facebook scale, had a permissive license, and offered the write performance that streaming workloads demand. It became the default.
The problem: RocksDB was designed for Facebook's infrastructure team with deep storage expertise. Kafka Streams is used by application developers who reasonably expect sensible defaults. The defaults are sensible for a single database instance. They fall apart when Kafka Streams creates hundreds of instances per node.
How RocksDB Works Under Your Kafka Streams App
Kafka Streams uses RocksDB as an embedded key-value store for all stateful operations: aggregations, joins, windowed computations, and custom state stores. Each task in your topology gets its own RocksDB instance. A single Kafka Streams application with 100 partitions and 3 stateful operators creates 300 separate RocksDB instances on a single node.
RocksDB is an LSM-tree (Log-Structured Merge-tree) database. Writes go to an in-memory buffer called a memtable. When the memtable fills, it flushes to disk as an immutable SSTable file. Background compaction merges these files to reclaim space and maintain read performance.
This architecture creates three categories of production problems that surface only under load.
The Memory You Cannot See
RocksDB memory consumption in Kafka Streams is not what you expect. The JVM heap you configure for your application is only part of the story.
Each RocksDB instance allocates:
- Block cache (default: 50MB per instance, shared across column families)
- Memtables (default: 64MB write buffer, typically 2-3 active per instance)
- Index and filter blocks (size depends on data volume)
- Table readers for open SST files
None of this memory is tracked by JVM heap metrics. It is native memory allocated via JNI. A Kafka Streams application reporting 2GB JVM heap usage can easily consume 8GB of actual RAM when you account for RocksDB.
I measured one production deployment where a Kafka Streams application configured with -Xmx4g was consistently OOM-killed by the kernel at 12GB RSS. The team had 150 RocksDB instances per node. The math: 150 instances × 64MB memtable × 2 active memtables = 19.2GB just for memtables, before counting block cache or anything else.
The default Kafka Streams configuration assumes you have infinite memory. You do not.
Your SSDs Are Dying Faster Than You Think
LSM-trees trade read performance for write performance. The cost is write amplification: the ratio of bytes written to disk versus bytes written by the application.
RocksDB's write amplification factor ranges from 10x to 30x depending on configuration and workload. A Kafka Streams application ingesting 100MB/s of state updates writes 1-3GB/s to disk after accounting for memtable flushes and compaction.
Production evidence from a windowed aggregation workload:
# iostat output during peak load Device: tps kB_read/s kB_wrtn/s await nvme0n1 45000 120000 2800000 4.2 # Application-level state update rate State updates: ~95,000/sec Average value size: 1.2KB Logical write rate: 114 MB/s Actual disk write rate: 2.73 GB/s Write amplification: 24x
This particular deployment burned through SSD endurance at 3x the expected rate. Enterprise NVMe drives rated for 3 DWPD (Drive Writes Per Day) were hitting their write limits in under a year.
The compaction strategy matters. Kafka Streams uses leveled compaction by default, which provides better read performance but higher write amplification compared to universal compaction.
When Compaction Goes Wrong
RocksDB compaction runs in background threads. When compaction falls behind ingestion rate, you get a compaction storm: multiple levels of the LSM-tree trigger compaction simultaneously, competing for disk I/O and CPU.
Symptoms in Kafka Streams:
- Consumer lag spikes with no apparent cause
- Processing latency P99 increases 10-100x while P50 stays stable
- Disk I/O wait spikes to 20-40% on otherwise healthy SSDs
- Memory pressure as compaction buffers compete with normal operation
I traced one incident where a Kafka Streams application processing 50,000 events/sec suddenly fell to 5,000 events/sec. The cause was compaction at L5 and L6 running simultaneously, consuming all available disk bandwidth. The application had been running fine for weeks because compaction had been keeping up. A traffic spike pushed it past the tipping point.
The default compaction configuration in Kafka Streams allocates 1 background thread for compaction. This is insufficient for any serious workload.
Rebalances Hurt More Than They Should
When a Kafka Streams instance restarts or partitions rebalance, state stores must be rebuilt from changelog topics. This is where RocksDB configuration directly impacts your recovery time objective.
Recovery involves:
- Reading all records from the changelog topic since the last checkpoint
- Deserializing each record
- Writing to RocksDB (triggering memtable flushes and compaction)
The bottleneck is almost always RocksDB write throughput during recovery. I measured recovery rates across different configurations:
Configuration Recovery Rate Time for 100GB --------------------------------------------------------------------------- Default Kafka Streams settings 15-25 MB/s 70-110 minutes Tuned memtable + write buffer 45-60 MB/s 28-37 minutes Tuned + increased compaction threads 80-120 MB/s 14-21 minutes
A 100GB state store with default settings takes over an hour to recover. During this time, that partition is unavailable for processing. If you have strict availability requirements, this is unacceptable.
Tuning Parameters That Actually Matter
Kafka Streams exposes RocksDB configuration through RocksDBConfigSetter. The parameters that matter most:
Block cache size: Controls memory used for caching frequently read data. Default 50MB is too small for most workloads. Set based on your read patterns and available memory.
options.setTableFormatConfig(
new BlockBasedTableConfig()
.setBlockCache(new LRUCache(256 * 1024 * 1024)) // 256MB
);
Write buffer configuration: Controls memtable size and count. Larger buffers reduce flush frequency but increase memory usage and recovery time after crashes.
options.setWriteBufferSize(128 * 1024 * 1024); // 128MB options.setMaxWriteBufferNumber(3); options.setMinWriteBufferNumberToMerge(2);
Compaction threads: Default of 1 is insufficient. Increase based on available CPU cores and disk bandwidth.
options.setIncreaseParallelism(4); options.setMaxBackgroundCompactions(4); options.setMaxBackgroundFlushes(2);
Compression: Default LZ4 compression is reasonable. For high-cardinality data, consider ZSTD for better compression ratios at the cost of CPU.
These are starting points, not solutions. The correct configuration depends on your specific workload characteristics: read/write ratio, key distribution, value sizes, and available hardware resources.
Can You Use Something Other Than RocksDB?
Yes, but your options have tradeoffs.
In-memory store: Kafka Streams includes a built-in in-memory state store. No disk I/O, no compaction, no native memory surprises. The catch: state must fit in heap, and you lose it on restart (full rebuild from changelog). Works well for small state or when you can tolerate longer recovery.
Stores.inMemoryKeyValueStore("my-store")
Custom StateStore implementation: Kafka Streams lets you implement StateStore interface with any backing storage. Teams have built stores backed by Redis, Apache Ignite, and various other databases. You gain operational familiarity with your chosen database but lose the simplicity of embedded storage. Network latency becomes a factor.
Alternative embedded databases: Starting with Kafka Streams 3.5, the state store abstraction is cleaner. Some teams experiment with LMDB (better read performance, worse write amplification) or SQLite (familiar, but not designed for this workload). Neither has the production track record of RocksDB in streaming contexts.
Remote state stores: For state exceeding local disk capacity, some architectures push state to external systems like Redis Cluster or ScyllaDB. This fundamentally changes the operational model and introduces network dependencies. It works, but you are no longer running a "simple" Kafka Streams app.
My recommendation: start with RocksDB and tune it properly. Switch only when you hit a specific limitation that tuning cannot address. Most teams never need to switch; they just need to configure RocksDB correctly.
What You Should Actually Monitor
Standard Kafka Streams metrics do not expose RocksDB internals. You need to instrument explicitly:
- RocksDB statistics via
Statisticsobject: compaction stats, stall conditions, cache hit rates - Native memory tracking: RSS minus JVM heap gives you RocksDB footprint
- Disk I/O correlation: match RocksDB compaction events to I/O spikes
- Recovery time tracking: measure time from rebalance trigger to processing resume
Without this visibility, you are debugging blind.
The Bigger Picture
RocksDB in Kafka Streams is not a bug. It is a design choice with specific tradeoffs. The problem is that most teams encounter these tradeoffs only after they have built their system around Kafka Streams.
If you are designing a new stateful streaming application, understand the state management implications before committing to an architecture. If you are debugging an existing Kafka Streams deployment with unexplained performance issues, start with RocksDB. The evidence is in the metrics you are not collecting.
I work with teams on Kafka Streams performance optimization, state store architecture, and streaming system design. If your stateful streaming workload is hitting limits you cannot diagnose, reach out.
If you need help with distributed systems, backend engineering, or data platforms, check my Services.