Skip to content

Commit

Permalink
storage: don't send log truncations through Raft
Browse files Browse the repository at this point in the history
Sending log truncations through Raft is inefficient: the Raft log is not itself
part of the replicated state. Instead, we only replicate the TruncatedState and,
as a side effect, ClearRange() the affected key range.

This is an individual performance optimization whose impact we should measure;
anecdotally it always looked like we were doing a lot of work for truncations
during a write-heavy workload; this should alleviate this somewhat).

It also removes one migration concern for cockroachdb#16809, see
cockroachdb#16809 (comment).
  • Loading branch information
tbg committed Jul 11, 2017
1 parent 5caec41 commit a237057
Show file tree
Hide file tree
Showing 2 changed files with 63 additions and 17 deletions.
36 changes: 25 additions & 11 deletions pkg/storage/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -1883,17 +1883,31 @@ func evalTruncateLog(
if err != nil {
return EvalResult{}, err
}
start := keys.RaftLogKey(cArgs.EvalCtx.RangeID(), 0)
end := keys.RaftLogKey(cArgs.EvalCtx.RangeID(), args.Index)
var diff enginepb.MVCCStats
// Passing zero timestamp to MVCCDeleteRange is equivalent to a ranged clear
// but it also computes stats. Note that any sideloaded payloads that may be
// removed by this truncation don't matter; they're not tracked in the raft
// log delta.
if _, _, _, err := engine.MVCCDeleteRange(ctx, batch, &diff, start, end, math.MaxInt64, /* max */
hlc.Timestamp{}, nil /* txn */, false /* returnKeys */); err != nil {
return EvalResult{}, err

// We start at index zero because it's always possible that a previous
// truncation did not clean up entries made obsolete by the previous
// truncation.
start := engine.MakeMVCCMetadataKey(keys.RaftLogKey(cArgs.EvalCtx.RangeID(), 0))
end := engine.MakeMVCCMetadataKey(keys.RaftLogKey(cArgs.EvalCtx.RangeID(), args.Index-1).PrefixEnd())
// Compute the stats delta that were to occur should the log entries be
// purged. Compute this by passing zero timestamp to MVCCDeleteRange,
// which is equivalent to a ranged clear but it also computes stats.
// Pass a throwaway batch since we don't actually want to delete through
// Raft - that is expensive, and the removed entries are not part of
// consistent replicated state, so we just have followers clean them up
// as a side effect. They may miss one in the event of an ill-timed
// crash, but that's OK since the next truncation will get everything.
//
// Note that any sideloaded payloads that may be removed by this
// truncation don't matter; they're not tracked in the raft log delta.
iter := batch.NewIterator(false /* !prefix */)
defer iter.Close()
// We can pass zero as nowNanos because we're only interested in SysBytes.
ms, err := iter.ComputeStats(start, end, 0 /* nowNanos */)
if err != nil {
return EvalResult{}, errors.Wrap(err, "could not compute stats of Raft log freed by truncation")
}
ms.SysBytes = -ms.SysBytes // simulate the deletion

tState := &roachpb.RaftTruncatedState{
Index: args.Index - 1,
Expand All @@ -1902,7 +1916,7 @@ func evalTruncateLog(

var pd EvalResult
pd.Replicated.State.TruncatedState = tState
pd.Replicated.RaftLogDelta = &diff.SysBytes
pd.Replicated.RaftLogDelta = &ms.SysBytes

return pd, makeReplicaStateLoader(cArgs.EvalCtx.RangeID()).setTruncatedState(ctx, batch, cArgs.Stats, tState)
}
Expand Down
44 changes: 38 additions & 6 deletions pkg/storage/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -737,18 +737,50 @@ func (r *Replica) handleReplicatedEvalResult(

if newTruncState := rResult.State.TruncatedState; newTruncState != nil {
rResult.State.TruncatedState = nil // for assertion

r.mu.Lock()
r.mu.state.TruncatedState = newTruncState
r.mu.Unlock()

// TODO(tschottdorf): everything below doesn't need to be on this
// goroutine. Worth moving out -- truncations are frequent and missing
// one of the side effects below doesn't matter. Need to be careful
// about the interaction with `evalTruncateLog` though, which computes
// some stats based on the log entries it sees. Also, sideloaded storage
// needs to hold the raft mu. Perhaps it should just get its own mutex
// (which is usually held together with raftMu, except when accessing
// the storage for a truncation). Or, even better, make use of the fact
// that all we need to synchronize is disk i/o, and there is no overlap
// between files *removed* during truncation and those active in Raft.

// Truncate the Raft log.
{
start := engine.MakeMVCCMetadataKey(keys.RaftLogKey(r.RangeID, 0))
end := engine.MakeMVCCMetadataKey(
keys.RaftLogKey(r.RangeID, newTruncState.Index).PrefixEnd(),
)
iter := r.store.engine.NewIterator(false /* !prefix */)
// TODO(petermattis): add a comment explaining why using ClearRange() below
// is ill-advised.
if err := r.store.engine.ClearIterRange(iter, start, end); err != nil {
log.Errorf(ctx, "unable to clear truncated Raft entries for %+v: %s", newTruncState, err)
// Intentionally not returning.
}
iter.Close()
}

// Clear any entries in the Raft log entry cache for this range up
// to and including the most recently truncated index.
r.store.raftEntryCache.clearTo(r.RangeID, newTruncState.Index+1)
log.Eventf(ctx, "truncating sideloaded storage up to (and including) index %d", newTruncState.Index)
if err := r.raftMu.sideloaded.TruncateTo(ctx, newTruncState.Index+1); err != nil {
// We don't *have* to remove these entries for correctness. Log a
// loud error, but keep humming along.
log.Errorf(ctx, "while removing sideloaded files during log truncation: %s", err)
}

// Truncate the sideloaded storage.
log.Eventf(ctx, "truncating sideloaded storage up to (and including) index %d", newTruncState.Index)
if err := r.raftMu.sideloaded.TruncateTo(ctx, newTruncState.Index+1); err != nil {
// We don't *have* to remove these entries for correctness. Log a
// loud error, but keep humming along.
log.Errorf(ctx, "while removing sideloaded files during log truncation: %s", err)
// Intentionally not returning.

}

if newThresh := rResult.State.GCThreshold; newThresh != (hlc.Timestamp{}) {
Expand Down

0 comments on commit a237057

Please sign in to comment.