Context
In multi-raft deployments — the AMQP quorum-queue use case the library targets, and any other workload with many groups per node — each Raft group currently sends its own AppendEntries heartbeat to every peer on every heartbeat interval. The default heartbeat period is 2 ticks × 50ms = 100ms.
For a typical AMQP quorum-queue deployment (clusters are capped at around 5 nodes, but each node can host hundreds of queues = hundreds of Raft groups), heartbeat traffic dominates the small-message path. A node that is leader of 100 co-resident groups sends 100 × 4 = 400 heartbeat messages per peer-fanout per 100ms — most of them carrying no log entries, all of them competing for the same shared TCP connection.
This is wasteful in three ways:
- Message volume. Liveness signals scale as O(groups × peers) per interval. Most heartbeats carry no payload.
- Per-message wire and syscall overhead. Each heartbeat is a separate
Message struct serialized and written to the shared TCP connection.
- Head-of-line blocking pressure. The shared per-peer TCP connection (see
ARCHITECTURE.md §6 "Single TCP connection per peer pair") means heartbeats compete with replication payloads on the wire. Heartbeats are exactly the messages you most want to slip past large entries — if they're delayed, leases expire and unnecessary elections happen.
Proposed approach
Aggregate all heartbeats from this node to a given peer into a single batched RPC, sent once per heartbeat interval. The batch carries a list of (group_id, prev_log_index, prev_log_term, commit_index) tuples. The receiver dispatches per-group internally.
This is roughly what Redpanda does for the same problem.
Implementation sketch:
- New
MessageType::HeartbeatBatch (or extend an existing message with a batch payload).
- Transport-layer aggregation: instead of each
Node independently emitting heartbeat AppendEntries, the transport (or a thin coordinator above it) collects per-group heartbeat state once per interval and emits one batched message per peer.
- Receiver decodes the batch and synthesizes per-group
step() calls so the protocol code in Node doesn't change.
Why this matters
- For single-group or low-group-count clusters, this is essentially a no-op — the win is proportional to the number of co-resident groups.
- For the AMQP quorum-queue case (many queues = many groups), this is one of the biggest available structural wins for transport throughput and tail latency.
- It is largely orthogonal to other transport changes — composes with future multi-connection splits or body-out-of-log decisions.
Trade-offs
- New message type or extension to AppendEntries. Adds a code path on both sides. The aggregated heartbeat is logically distinct from a per-group AppendEntries with no entries.
- Coordination at the transport layer. Real heartbeats come from per-group leadership state; batching forces the transport (or an adapter layer) to gather per-group state before the wire write. Care is needed not to leak protocol concerns into the transport.
- Doesn't help non-heartbeat messages. Actual replication payloads still go per-group. This is a heartbeat-specific optimization, not a general transport rewrite.
- Must preserve term-bump semantics. A heartbeat carries the leader's current term; aggregated heartbeats must still cause a follower to step down or update term per group, and the receiver dispatch must preserve message ordering relative to in-flight per-group AppendEntries.
Out of scope
- Linearizable reads, snapshots, log compaction (separate gaps — see
ARCHITECTURE.md §6.5).
Reference
See ARCHITECTURE.md — sections "Single TCP connection per peer pair" and "What is intentionally not optimized (yet)" — for the broader transport design context.
Context
In multi-raft deployments — the AMQP quorum-queue use case the library targets, and any other workload with many groups per node — each Raft group currently sends its own AppendEntries heartbeat to every peer on every heartbeat interval. The default heartbeat period is 2 ticks × 50ms = 100ms.
For a typical AMQP quorum-queue deployment (clusters are capped at around 5 nodes, but each node can host hundreds of queues = hundreds of Raft groups), heartbeat traffic dominates the small-message path. A node that is leader of 100 co-resident groups sends 100 × 4 = 400 heartbeat messages per peer-fanout per 100ms — most of them carrying no log entries, all of them competing for the same shared TCP connection.
This is wasteful in three ways:
Messagestruct serialized and written to the shared TCP connection.ARCHITECTURE.md§6 "Single TCP connection per peer pair") means heartbeats compete with replication payloads on the wire. Heartbeats are exactly the messages you most want to slip past large entries — if they're delayed, leases expire and unnecessary elections happen.Proposed approach
Aggregate all heartbeats from this node to a given peer into a single batched RPC, sent once per heartbeat interval. The batch carries a list of
(group_id, prev_log_index, prev_log_term, commit_index)tuples. The receiver dispatches per-group internally.This is roughly what Redpanda does for the same problem.
Implementation sketch:
MessageType::HeartbeatBatch(or extend an existing message with a batch payload).Nodeindependently emitting heartbeat AppendEntries, the transport (or a thin coordinator above it) collects per-group heartbeat state once per interval and emits one batched message per peer.step()calls so the protocol code inNodedoesn't change.Why this matters
Trade-offs
Out of scope
ARCHITECTURE.md§6.5).Reference
See
ARCHITECTURE.md— sections "Single TCP connection per peer pair" and "What is intentionally not optimized (yet)" — for the broader transport design context.