Skip to content

kvserver: prevent truncating log entries needed for entry application #70109

@tbg

Description

@tbg

Describe the problem

  • We do not seem to have appropriate safeguards in place that prevent log
    truncations that remove committed-but-unapplied entries from the log.

To Reproduce

I have not reproduced this, nor is there evidence that it has occurred in
the wild. This is based on observations in #66843.

Expected behavior

Log truncations should only remove entries that are no longer required. Our
log truncation has evolved over time. Originally, the truncated state index
(i.e. the last truncated log position) was part of the replicated state,
meaning that all replicas kept the same tail of the log around (as a function
of their applied index). This was loosened several releases ago to free us from
the requirement that the raft log be included in raft snapshots.

Log truncations are and were always triggered by the Raft leader through a queue:

// Is this the raft leader? We only perform log truncation on the raft leader
// which has the up to date info on followers.
if raftStatus.RaftState != raft.StateLeader {
return truncateDecision{}, nil
}

and the main concern is not cutting followers off from being able to obtain
the entire raft log. No thought was given to the fact that followers may not
have applied their acked portion of the raft log; in fact the raft leader
does not learn this information from the followers. In the short term, we
should allow followers to adjust the requested truncation to avoid deleting
entries that yet need to be applied.

We should generally transition off this mechanism and truncate the log in a
purely local fashion. This should also include performance optimizations;
at present, we block the raft handle loop and perform regular MVCC deletions
in a loop:

for idx := oldTruncatedState.Index + 1; idx <= newTruncatedState.Index; idx++ {
// NB: RangeIDPrefixBufs have sufficient capacity (32 bytes) to
// avoid allocating when constructing Raft log keys (16 bytes).
unsafeKey := prefixBuf.RaftLogKey(idx)
if err := readWriter.ClearUnversioned(unsafeKey); err != nil {
return false, errors.Wrapf(err, "unable to clear truncated Raft entries for %+v", newTruncatedState)
}
}

However, only the update to the truncated state itself needs to be part of
the raft cycle. The actual deletion can be carried out asynchronously, and
additionally could benefit from using a combination of SingleDelete and
pebble range deletions (or even, gasp, range single deletions, but let's
not go there).

Metadata

Metadata

Assignees

No one assigned

    Labels

    C-bugCode not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.T-kvKV Team

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions