Skip to content

kvserver: limit MVCC scan in Raft Entries retrieval #66682

@erikgrinaker

Description

@erikgrinaker

In a recent customer escalation, we saw a newly elected Raft leader spend so long to send 100 log entries to catch up a follower that it missed its heartbeat. The log entries were found to be about 132 kb each, and we limit these messages to be a maximum of 32 kb (but at least 1 entry), so responses were being sent with a single entry each. It seemed to be taking 60 ms to retrieve a single entry, totalling 6s for 100 log entries. The log was also huge, at ~900,000 entries, with the current commit index at 1500.

Digging into the Raft code, we find that raft.maybeSendAppend() calls int raftLog.entries() to fetch entries to send to the follower. Notice how it limits this solely by maxsize (32 kb), otherwise using the entire Raft log (lastIndex()+1) as an upper bound:

https://github.com/cockroachdb/vendored/blob/310ea3cd09b55055b290f0a4cf9991a91725797e/go.etcd.io/etcd/raft/v3/log.go#L287-L292

This in turn gets passed into Storage.Entries():

https://github.com/cockroachdb/vendored/blob/310ea3cd09b55055b290f0a4cf9991a91725797e/go.etcd.io/etcd/raft/v3/log.go#L350

Our implementation of Entries(), however, does not pass the max size bound to the MVCC iterator, only the key bounds, instead relying on the passed scan function to error when the size limit is exceeded:

_, err := storage.MVCCIterate(
ctx, reader,
keys.RaftLogKey(rangeID, lo),
keys.RaftLogKey(rangeID, hi),
hlc.Timestamp{},
storage.MVCCScanOptions{},
scanFunc,
)

The problem is that MVCCIterate will actually fetch 1000 entries before calling scanFunc:

cockroach/pkg/storage/mvcc.go

Lines 2662 to 2666 in 8f5231d

const maxKeysPerScan = 1000
opts := opts
opts.MaxKeys = maxKeysPerScan
res, err := mvccScanToKvs(
ctx, iter, key, endKey, timestamp, opts)

cockroach/pkg/storage/mvcc.go

Lines 2679 to 2686 in 8f5231d

for i := range res.KVs {
if err := f(res.KVs[i]); err != nil {
if iterutil.Done(err) {
return intents, nil
}
return nil, err
}
}

Since we're using the entire Raft log as an upper bound, not passing in a MVCCScanOptions.MaxSize for the iterator, and only doing size limiting in the passed scanFunc, this means that every time we fetch a single log entry we're actually fetching and decoding 1000 entries, each 132 kb large.

The simplest fix here is probably to just pass in the max size from Raft in MVCCScanOptions.

/cc @cockroachdb/kv

Metadata

Metadata

Assignees

No one assigned

    Labels

    A-kv-replicationRelating to Raft, consensus, and coordination.C-bugCode not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.C-performancePerf of queries or internals. Solution not expected to change functional behavior.N-followupNeeds followup.O-postmortemOriginated from a Postmortem action item.T-kvKV Team

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions