diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 266aede2d050..3e5731f50672 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -925,22 +924,11 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch( func (b *replicaAppBatch) stageTrivialReplicatedEvalResult( ctx context.Context, cmd *replicatedCmd, ) { - raftAppliedIndex := cmd.ent.Index - if raftAppliedIndex == 0 { + if cmd.ent.Index == 0 { log.Fatalf(ctx, "raft entry with index 0") } - b.state.RaftAppliedIndex = raftAppliedIndex - rs := cmd.decodedRaftEntry.replicatedResult().State - // We are post migration or this replicatedCmd is doing the migration. - if b.state.RaftAppliedIndexTerm > 0 || (rs != nil && - rs.RaftAppliedIndexTerm == stateloader.RaftLogTermSignalForAddRaftAppliedIndexTermMigration) { - // Once we populate b.state.RaftAppliedIndexTerm it will flow into the - // persisted RangeAppliedState and into the in-memory representation in - // Replica.mu.state. The latter is used to initialize b.state, so future - // calls to this method will see that the migration has already happened - // and will continue to populate the term. - b.state.RaftAppliedIndexTerm = cmd.ent.Term - } + b.state.RaftAppliedIndex = cmd.ent.Index + b.state.RaftAppliedIndexTerm = cmd.ent.Term if leaseAppliedIndex := cmd.leaseIndex; leaseAppliedIndex != 0 { b.state.LeaseAppliedIndex = leaseAppliedIndex @@ -1001,8 +989,6 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error { r := b.r r.mu.Lock() r.mu.state.RaftAppliedIndex = b.state.RaftAppliedIndex - // RaftAppliedIndexTerm will be non-zero only when the - // AddRaftAppliedIndexTermMigration has happened. r.mu.state.RaftAppliedIndexTerm = b.state.RaftAppliedIndexTerm r.mu.state.LeaseAppliedIndex = b.state.LeaseAppliedIndex diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 6e7f6c8ae174..a9b01cabf7c4 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -414,15 +414,10 @@ func (r *Replica) GetLeaseAppliedIndex() uint64 { // Snapshot method. func (r *replicaRaftStorage) Snapshot() (raftpb.Snapshot, error) { r.mu.AssertHeld() - appliedIndex := r.mu.state.RaftAppliedIndex - term, err := r.Term(appliedIndex) - if err != nil { - return raftpb.Snapshot{}, err - } return raftpb.Snapshot{ Metadata: raftpb.SnapshotMetadata{ - Index: appliedIndex, - Term: term, + Index: r.mu.state.RaftAppliedIndex, + Term: r.mu.state.RaftAppliedIndexTerm, }, }, nil } @@ -605,17 +600,6 @@ func snapshot( return OutgoingSnapshot{}, err } - term, err := term(ctx, rsl, snap, rangeID, eCache, state.RaftAppliedIndex) - // If we've migrated to populating RaftAppliedIndexTerm, check that the term - // from the two sources are equal. - if state.RaftAppliedIndexTerm != 0 && term != state.RaftAppliedIndexTerm { - return OutgoingSnapshot{}, - errors.AssertionFailedf("unequal terms %d != %d", term, state.RaftAppliedIndexTerm) - } - if err != nil { - return OutgoingSnapshot{}, errors.Wrapf(err, "failed to fetch term of %d", state.RaftAppliedIndex) - } - return OutgoingSnapshot{ RaftEntryCache: eCache, WithSideloaded: withSideloaded, @@ -626,7 +610,7 @@ func snapshot( Data: snapUUID.GetBytes(), Metadata: raftpb.SnapshotMetadata{ Index: state.RaftAppliedIndex, - Term: term, + Term: state.RaftAppliedIndexTerm, // Synthesize our raftpb.ConfState from desc. ConfState: desc.Replicas().ConfState(), }, @@ -979,9 +963,7 @@ func (r *Replica) applySnapshot( log.Fatalf(ctx, "snapshot RaftAppliedIndex %d doesn't match its metadata index %d", state.RaftAppliedIndex, nonemptySnap.Metadata.Index) } - // If we've migrated to populating RaftAppliedIndexTerm, check that the term - // from the two sources are equal. - if state.RaftAppliedIndexTerm != 0 && state.RaftAppliedIndexTerm != nonemptySnap.Metadata.Term { + if state.RaftAppliedIndexTerm != nonemptySnap.Metadata.Term { log.Fatalf(ctx, "snapshot RaftAppliedIndexTerm %d doesn't match its metadata term %d", state.RaftAppliedIndexTerm, nonemptySnap.Metadata.Term) }