diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 5a359d1a64cc..0f037790698c 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -482,52 +482,6 @@ func (r *Replica) applySnapshot( log.Infof(ctx, "applied %s (%s)", inSnap, logDetails) }(timeutil.Now()) - writeUnreplicatedSST := func(ctx context.Context, id storage.FullReplicaID, st *cluster.Settings, meta raftpb.SnapshotMetadata, hs raftpb.HardState, sl *logstore.StateLoader) (*storage.MemFile, bool, error) { - unreplicatedSSTFile := &storage.MemFile{} - unreplicatedSST := storage.MakeIngestionSSTWriter( - ctx, st, unreplicatedSSTFile, - ) - defer unreplicatedSST.Close() - - // Clearing the unreplicated state. - // - // NB: We do not expect to see range keys in the unreplicated state, so - // we don't drop a range tombstone across the range key space. - unreplicatedPrefixKey := keys.MakeRangeIDUnreplicatedPrefix(id.RangeID) - unreplicatedStart := unreplicatedPrefixKey - unreplicatedEnd := unreplicatedPrefixKey.PrefixEnd() - if err := unreplicatedSST.ClearRawRange( - unreplicatedStart, unreplicatedEnd, true /* pointKeys */, false, /* rangeKeys */ - ); err != nil { - return nil, false, errors.Wrapf(err, "error clearing range of unreplicated SST writer") - } - - // Update HardState. - if err := sl.SetHardState(ctx, &unreplicatedSST, hs); err != nil { - return nil, false, errors.Wrapf(err, "unable to write HardState to unreplicated SST writer") - } - // We've cleared all the raft state above, so we are forced to write the - // RaftReplicaID again here. - if err := sl.SetRaftReplicaID( - ctx, &unreplicatedSST, id.ReplicaID); err != nil { - return nil, false, errors.Wrapf(err, "unable to write RaftReplicaID to unreplicated SST writer") - } - - if err := sl.SetRaftTruncatedState( - ctx, &unreplicatedSST, - &roachpb.RaftTruncatedState{ - Index: meta.Index, - Term: meta.Term, - }, - ); err != nil { - return nil, false, errors.Wrapf(err, "unable to write TruncatedState to unreplicated SST writer") - } - - if err := unreplicatedSST.Finish(); err != nil { - return nil, false, err - } - return unreplicatedSSTFile, unreplicatedSST.DataSize > 0, nil - } unreplicatedSSTFile, nonempty, err := writeUnreplicatedSST( ctx, r.ID(), r.ClusterSettings(), nonemptySnap.Metadata, hs, &r.raftMu.stateLoader.StateLoader, ) @@ -704,6 +658,60 @@ func (r *Replica) applySnapshot( return nil } +func writeUnreplicatedSST( + ctx context.Context, + id storage.FullReplicaID, + st *cluster.Settings, + meta raftpb.SnapshotMetadata, + hs raftpb.HardState, + sl *logstore.StateLoader, +) (*storage.MemFile, bool, error) { + unreplicatedSSTFile := &storage.MemFile{} + unreplicatedSST := storage.MakeIngestionSSTWriter( + ctx, st, unreplicatedSSTFile, + ) + defer unreplicatedSST.Close() + + // Clearing the unreplicated state. + // + // NB: We do not expect to see range keys in the unreplicated state, so + // we don't drop a range tombstone across the range key space. + unreplicatedPrefixKey := keys.MakeRangeIDUnreplicatedPrefix(id.RangeID) + unreplicatedStart := unreplicatedPrefixKey + unreplicatedEnd := unreplicatedPrefixKey.PrefixEnd() + if err := unreplicatedSST.ClearRawRange( + unreplicatedStart, unreplicatedEnd, true /* pointKeys */, false, /* rangeKeys */ + ); err != nil { + return nil, false, errors.Wrapf(err, "error clearing range of unreplicated SST writer") + } + + // Update HardState. + if err := sl.SetHardState(ctx, &unreplicatedSST, hs); err != nil { + return nil, false, errors.Wrapf(err, "unable to write HardState to unreplicated SST writer") + } + // We've cleared all the raft state above, so we are forced to write the + // RaftReplicaID again here. + if err := sl.SetRaftReplicaID( + ctx, &unreplicatedSST, id.ReplicaID); err != nil { + return nil, false, errors.Wrapf(err, "unable to write RaftReplicaID to unreplicated SST writer") + } + + if err := sl.SetRaftTruncatedState( + ctx, &unreplicatedSST, + &roachpb.RaftTruncatedState{ + Index: meta.Index, + Term: meta.Term, + }, + ); err != nil { + return nil, false, errors.Wrapf(err, "unable to write TruncatedState to unreplicated SST writer") + } + + if err := unreplicatedSST.Finish(); err != nil { + return nil, false, err + } + return unreplicatedSSTFile, unreplicatedSST.DataSize > 0, nil +} + // clearSubsumedReplicaDiskData clears the on disk data of the subsumed // replicas by creating SSTs with range deletion tombstones. We have to be // careful here not to have overlapping ranges with the SSTs we have already