Kafka Broker Internals — Segments, Indexes, and Log Compaction

Kafka Broker Internals — Segments, Indexes, and Log Compaction

The Kafka producer's batching and zero-copy tricks explain why sending data is fast. But the broker side is where Kafka's durability story lives — and it is built entirely on a deliberate approach to how data is laid out on disk.

This post covers the broker's physical storage model: how log segments work, how the sparse index makes random reads fast without loading data into memory, how time-based indexing supports timestamp lookups, and how compaction and retention clean up data over time.


The Partition as a Log

Every Kafka topic is divided into partitions. Each partition on a broker is a directory on disk:

/kafka-logs/
  my-topic-0/
  my-topic-1/
  my-topic-2/

Inside each partition directory, data is stored in log segments — a set of files that together form a logical append-only log. Each segment has a base offset, and its files share that offset as their name:

my-topic-0/
  00000000000000000000.log      ← data file
  00000000000000000000.index    ← sparse offset index
  00000000000000000000.timeindex ← sparse time index
  00000000000000001500.log
  00000000000000001500.index
  00000000000000001500.timeindex

The number in the filename is the base offset — the offset of the first record in that segment. When a consumer asks for offset 1800, the broker knows it must be in the segment starting at 1500 (and not 0) without reading any data files.


Log Segments

A segment is the unit of allocation, retention, and deletion. The broker always appends to the active segment — the newest one. When the active segment reaches a size or time threshold, it is closed and a new one is opened.

Two broker-level configuration parameters control this:

Parameter Default Effect
log.segment.bytes 1 GB Roll a new segment when this size is reached
log.roll.ms 7 days Roll a new segment after this time, even if not full

Closed segments are immutable — they are never modified. This is what makes the sparse index possible (more on that shortly) and what makes zero-copy safe: the OS can serve the bytes directly from the page cache without worrying about concurrent writes.


The Sparse Offset Index

A naive implementation would keep a mapping of every offset → byte position in the .log file. For a segment with millions of records, that index would be enormous.

Kafka instead uses a sparse index: it records the byte position of roughly every N bytes (controlled by log.index.interval.bytes, default 4 KB). The index entry is a pair:

relative_offset (4 bytes) | file_position (4 bytes)

The relative offset is relative to the base offset of the segment, which saves space (4 bytes vs 8 bytes).

When a consumer requests offset O:

  1. Binary search the .index file for the largest entry ≤ O
  2. Seek to that byte position in the .log file
  3. Scan forward through individual records until offset O is found

Because the index covers every ~4 KB of data, the scan in step 3 is bounded and fast. The index file itself is small enough that the OS keeps it in the page cache permanently.

.index (sparse)          .log (dense)
┌──────────┬──────┐      ┌──────────────────┐
│ rel:   0 │  0   │ ──▶  │ offset 1500 ...  │
│ rel:  50 │ 4096 │ ──▶  │ offset 1550 ...  │
│ rel: 100 │ 8192 │ ──▶  │ offset 1600 ...  │
└──────────┴──────┘      └──────────────────┘

The index is memory-mapped (mmap). Kafka maps it directly into the process address space, so binary search is a series of pointer dereferences — no system calls, no copying.


The Time Index

Consumers can also seek by timestamp — "give me all records after 14:00 on Tuesday". The .timeindex file supports this:

timestamp (8 bytes) | relative_offset (4 bytes)

Each entry records the maximum timestamp seen up to that offset in the segment. To seek to a timestamp T:

  1. Binary search .timeindex for the first entry where timestamp ≥ T
  2. Use the corresponding relative offset to look up the byte position in .index
  3. Seek to that position in .log and scan forward

Like the offset index, the time index is sparse and memory-mapped.


Physical Layout: What Each File Contains

The .log file stores RecordBatch structures back-to-back. Each batch has a header (base offset, length, partition leader epoch, magic byte, CRC, attributes, timestamps, producer ID, sequence number) followed by the compressed record data.

.log
┌─────────────────────────────────────────┐
│ RecordBatch (offset 1500–1549)          │
│   header: baseOffset=1500, crc, ...     │
│   records: [r0, r1, ..., r49] (snappy)  │
├─────────────────────────────────────────┤
│ RecordBatch (offset 1550–1599)          │
│   ...                                   │
└─────────────────────────────────────────┘

The broker never decompresses records to serve a fetch request. It reads the raw bytes from disk (or page cache) and ships them directly to the consumer. If the consumer uses the same compression codec, it receives the compressed batch and decompresses locally — zero broker CPU for reads.


Log Retention

By default Kafka retains data for a fixed period. Old segments are deleted once they are entirely outside the retention window.

# Time-based retention
log.retention.hours=168        # 7 days (default)

# Size-based retention (per partition)
log.retention.bytes=-1         # disabled by default

Kafka deletes at the segment granularity, not the record granularity. A segment is eligible for deletion only when its newest record is older than the retention window. This means the effective storage usage is slightly higher than the configured retention — the tail segment always stays until all its records age out.


Log Compaction

Retention deletes whole segments. Log compaction does something different: it keeps the last known value for each key and discards older records with the same key.

This is the mechanism that makes Kafka suitable as a changelog or event sourcing store — you can replay from the beginning and reconstruct current state without processing the full history.

How Compaction Works

Each partition's log is divided into two regions:

┌────────────────────────────┬─────────────────┐
│  clean (already compacted) │  dirty (new data)│
└────────────────────────────┴─────────────────┘

A background log cleaner thread scans the dirty region, builds an offset map (key → highest offset seen), and rewrites the dirty segments, keeping only the records that have the highest offset for their key. The resulting segments are merged into the clean region.

The Tombstone

To delete a key entirely, a producer sends a record with the same key and a null value (a tombstone). Compaction preserves tombstones long enough for all consumers to see the deletion, then removes them. The delete.retention.ms parameter (default 24 hours) controls how long tombstones are retained.

Configuration

# Enable compaction on the topic
log.cleanup.policy=compact

# Both compaction and time-based deletion
log.cleanup.policy=compact,delete
Parameter Default Effect
log.cleaner.min.compaction.lag.ms 0 Minimum time before a record can be compacted
log.cleaner.max.compaction.lag.ms MAX_LONG Maximum time a record stays in the dirty region
min.cleanable.dirty.ratio 0.5 Cleaner only runs when dirty/total > this ratio
delete.retention.ms 86 400 000 (24 h) How long tombstones are retained

Segment Merging During Compaction

Compaction does not just delete records — it also merges small segments into larger ones. After cleaning the dirty region, the cleaner joins adjacent segments that are below log.segment.bytes. This keeps the number of open file handles and index files manageable as records are removed and segments shrink.

The merge is a copy operation: the cleaner reads input segments sequentially and writes a new output segment. Because it is sequential I/O on immutable files, it is fast and does not interfere with normal read/write paths.


Important Tuning Parameters

Parameter Default Notes
log.segment.bytes 1 073 741 824 (1 GB) Larger segments = fewer files, longer time before retention can delete
log.roll.ms 604 800 000 (7 days) Maximum age of the active segment before rolling
log.index.interval.bytes 4 096 Denser index = faster seek, larger index file
log.index.size.max.bytes 10 485 760 (10 MB) Maximum size of offset/time index files
log.retention.hours 168 Time-based data retention
log.retention.bytes -1 (disabled) Size-based retention per partition
log.cleaner.threads 1 Background compaction threads
log.cleaner.io.max.bytes.per.second MAX_DOUBLE I/O rate limit for the cleaner

Summary

The Kafka broker's storage model is built around a small set of ideas that reinforce each other:

  • Append-only, immutable segments enable sequential I/O, safe zero-copy, and simple concurrent access
  • Sparse, memory-mapped indexes make offset and timestamp seeks fast without loading data into memory
  • Segment-granularity retention keeps deletion simple and fast — no record-level bookkeeping
  • Log compaction turns partitions into key-value change logs, enabling full-state replay from offset 0
  • The page cache ties it all together — reads that arrive shortly after writes almost never touch disk

Understanding this layout is the foundation for understanding everything else the broker does: replication copies segment bytes, consumers use the same index to seek efficiently, and compaction rewrites segments in place.