Skip to content

Commit

Permalink
Re-introduce "kv,migration: rm code handling legacy raft truncated st…
Browse files Browse the repository at this point in the history
…ate"

This reverts commit ef1dd6f. #70432
reverted #69887, as temporary stop-gap until we release the first 21.2
beta. See the discussion over on #70432 for why we want to queue up this
revert to the original revert; this should only be merged after #69826
lands.

Release note: None
  • Loading branch information
irfansharif committed Sep 21, 2021
1 parent af8d186 commit cfd9d5a
Show file tree
Hide file tree
Showing 43 changed files with 452 additions and 2,100 deletions.
8 changes: 1 addition & 7 deletions pkg/cli/debug_check_store.go
Original file line number Diff line number Diff line change
Expand Up @@ -252,7 +252,7 @@ func checkStoreRaftState(
return err
}
getReplicaInfo(rangeID).committedIndex = hs.Commit
case bytes.Equal(suffix, keys.LocalRaftTruncatedStateLegacySuffix):
case bytes.Equal(suffix, keys.LocalRaftTruncatedStateSuffix):
var trunc roachpb.RaftTruncatedState
if err := kv.Value.GetProto(&trunc); err != nil {
return err
Expand All @@ -264,12 +264,6 @@ func checkStoreRaftState(
return err
}
getReplicaInfo(rangeID).appliedIndex = state.RaftAppliedIndex
case bytes.Equal(suffix, keys.LocalRaftAppliedIndexLegacySuffix):
idx, err := kv.Value.GetInt()
if err != nil {
return err
}
getReplicaInfo(rangeID).appliedIndex = uint64(idx)
case bytes.Equal(suffix, keys.LocalRaftLogSuffix):
_, index, err := encoding.DecodeUint64Ascending(detail)
if err != nil {
Expand Down
42 changes: 0 additions & 42 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,32 +156,6 @@ const (

// v21.1 versions.
//
// replacedTruncatedAndRangeAppliedStateMigration stands in for
// TruncatedAndRangeAppliedStateMigration which was re-introduced after the
// migration job was introduced. This is necessary because the jobs
// infrastructure used to run this migration in v21.1 and its later alphas
// was introduced after this version was first introduced. Later code in the
// release relies on the job to run the migration but the job relies on
// its startup migrations having been run. Versions associated with long
// running migrations must follow deletedLongRunningMigrations.
replacedTruncatedAndRangeAppliedStateMigration
// replacedPostTruncatedAndRangeAppliedStateMigration is like the above
// version. See its comment.
replacedPostTruncatedAndRangeAppliedStateMigration
// TruncatedAndRangeAppliedStateMigration is part of the migration to stop
// using the legacy truncated state within KV. After the migration, we'll be
// using the unreplicated truncated state and the RangeAppliedState on all
// ranges. Callers that wish to assert on there no longer being any legacy
// will be able to do so after PostTruncatedAndRangeAppliedStateMigration is
// active. This lets remove any holdover code handling the possibility of
// replicated truncated state in 21.2.
//
// TODO(irfansharif): Do the above in 21.2.
TruncatedAndRangeAppliedStateMigration
// PostTruncatedAndRangeAppliedStateMigration is used to purge all replicas
// using the replicated legacy TruncatedState. It's also used in asserting
// that no replicated truncated state representation is found.
PostTruncatedAndRangeAppliedStateMigration
// V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases.
//
// TODO(irfansharif): This can be removed as part of #69828 (bumping the min
Expand Down Expand Up @@ -307,22 +281,6 @@ const (
// to be added (i.e., when cutting the final release candidate).
var versionsSingleton = keyedVersions{
// v21.1 versions.
{
Key: replacedTruncatedAndRangeAppliedStateMigration,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 14},
},
{
Key: replacedPostTruncatedAndRangeAppliedStateMigration,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 16},
},
{
Key: TruncatedAndRangeAppliedStateMigration,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 22},
},
{
Key: PostTruncatedAndRangeAppliedStateMigration,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 24},
},
{
// V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases.
Key: V21_1,
Expand Down
76 changes: 36 additions & 40 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

11 changes: 3 additions & 8 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,17 +80,12 @@ var (
// LocalRangeAppliedStateSuffix is the suffix for the range applied state
// key.
LocalRangeAppliedStateSuffix = []byte("rask")
// LocalRaftAppliedIndexLegacySuffix is the suffix for the raft applied index.
LocalRaftAppliedIndexLegacySuffix = []byte("rfta")
// LocalRaftTruncatedStateLegacySuffix is the suffix for the legacy
// RaftTruncatedState. See VersionUnreplicatedRaftTruncatedState.
// LocalRaftTruncatedStateSuffix is the suffix for the
// RaftTruncatedState.
// Note: This suffix is also used for unreplicated Range-ID keys.
LocalRaftTruncatedStateLegacySuffix = []byte("rftt")
LocalRaftTruncatedStateSuffix = []byte("rftt")
// LocalRangeLeaseSuffix is the suffix for a range lease.
LocalRangeLeaseSuffix = []byte("rll-")
// LocalLeaseAppliedIndexLegacySuffix is the suffix for the applied lease
// index.
LocalLeaseAppliedIndexLegacySuffix = []byte("rlla")
// LocalRangePriorReadSummarySuffix is the suffix for a range's prior read
// summary.
LocalRangePriorReadSummarySuffix = []byte("rprs")
Expand Down
16 changes: 6 additions & 10 deletions pkg/keys/doc.go
Original file line number Diff line number Diff line change
Expand Up @@ -185,16 +185,12 @@ var _ = [...]interface{}{
// range as a whole. Though they are replicated, they are unaddressable.
// Typical examples are MVCC stats and the abort span. They all share
// `LocalRangeIDPrefix` and `LocalRangeIDReplicatedInfix`.
AbortSpanKey, // "abc-"
RangeGCThresholdKey, // "lgc-"
RangeAppliedStateKey, // "rask"
RaftAppliedIndexLegacyKey, // "rfta"
RaftTruncatedStateLegacyKey, // "rftt"
RangeLeaseKey, // "rll-"
LeaseAppliedIndexLegacyKey, // "rlla"
RangePriorReadSummaryKey, // "rprs"
RangeVersionKey, // "rver"
RangeStatsLegacyKey, // "stat"
AbortSpanKey, // "abc-"
RangeGCThresholdKey, // "lgc-"
RangeAppliedStateKey, // "rask"
RangeLeaseKey, // "rll-"
RangePriorReadSummaryKey, // "rprs"
RangeVersionKey, // "rver"

// 2. Unreplicated range-ID local keys: These contain metadata that
// pertain to just one replica of a range. They are unreplicated and
Expand Down
57 changes: 1 addition & 56 deletions pkg/keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,34 +230,10 @@ func DecodeAbortSpanKey(key roachpb.Key, dest []byte) (uuid.UUID, error) {
}

// RangeAppliedStateKey returns a system-local key for the range applied state key.
// This key has subsumed the responsibility of the following three keys:
// - RaftAppliedIndexLegacyKey
// - LeaseAppliedIndexLegacyKey
// - RangeStatsLegacyKey
func RangeAppliedStateKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangeAppliedStateKey()
}

// RaftAppliedIndexLegacyKey returns a system-local key for a raft applied index.
// The key is no longer written to. Its responsibility has been subsumed by the
// RangeAppliedStateKey.
func RaftAppliedIndexLegacyKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RaftAppliedIndexLegacyKey()
}

// LeaseAppliedIndexLegacyKey returns a system-local key for a lease applied index.
// The key is no longer written to. Its responsibility has been subsumed by the
// RangeAppliedStateKey.
func LeaseAppliedIndexLegacyKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).LeaseAppliedIndexLegacyKey()
}

// RaftTruncatedStateLegacyKey returns a system-local key for a RaftTruncatedState.
// See VersionUnreplicatedRaftTruncatedState.
func RaftTruncatedStateLegacyKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RaftTruncatedStateLegacyKey()
}

// RangeLeaseKey returns a system-local key for a range lease.
func RangeLeaseKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangeLeaseKey()
Expand All @@ -269,13 +245,6 @@ func RangePriorReadSummaryKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangePriorReadSummaryKey()
}

// RangeStatsLegacyKey returns the key for accessing the MVCCStats struct for
// the specified Range ID. The key is no longer written to. Its responsibility
// has been subsumed by the RangeAppliedStateKey.
func RangeStatsLegacyKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangeStatsLegacyKey()
}

// RangeGCThresholdKey returns a system-local key for last used GC threshold on the
// user keyspace. Reads and writes <= this timestamp will not be served.
func RangeGCThresholdKey(rangeID roachpb.RangeID) roachpb.Key {
Expand Down Expand Up @@ -943,23 +912,6 @@ func (b RangeIDPrefixBuf) RangeAppliedStateKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRangeAppliedStateSuffix...)
}

// RaftAppliedIndexLegacyKey returns a system-local key for a raft applied index.
// See comment on RaftAppliedIndexLegacyKey function.
func (b RangeIDPrefixBuf) RaftAppliedIndexLegacyKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRaftAppliedIndexLegacySuffix...)
}

// LeaseAppliedIndexLegacyKey returns a system-local key for a lease applied index.
// See comment on LeaseAppliedIndexLegacyKey function.
func (b RangeIDPrefixBuf) LeaseAppliedIndexLegacyKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalLeaseAppliedIndexLegacySuffix...)
}

// RaftTruncatedStateLegacyKey returns a system-local key for a RaftTruncatedState.
func (b RangeIDPrefixBuf) RaftTruncatedStateLegacyKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRaftTruncatedStateLegacySuffix...)
}

// RangeLeaseKey returns a system-local key for a range lease.
func (b RangeIDPrefixBuf) RangeLeaseKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRangeLeaseSuffix...)
Expand All @@ -971,13 +923,6 @@ func (b RangeIDPrefixBuf) RangePriorReadSummaryKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRangePriorReadSummarySuffix...)
}

// RangeStatsLegacyKey returns the key for accessing the MVCCStats struct
// for the specified Range ID.
// See comment on RangeStatsLegacyKey function.
func (b RangeIDPrefixBuf) RangeStatsLegacyKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRangeStatsLegacySuffix...)
}

// RangeGCThresholdKey returns a system-local key for the GC threshold.
func (b RangeIDPrefixBuf) RangeGCThresholdKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRangeGCThresholdSuffix...)
Expand All @@ -995,7 +940,7 @@ func (b RangeIDPrefixBuf) RangeTombstoneKey() roachpb.Key {

// RaftTruncatedStateKey returns a system-local key for a RaftTruncatedState.
func (b RangeIDPrefixBuf) RaftTruncatedStateKey() roachpb.Key {
return append(b.unreplicatedPrefix(), LocalRaftTruncatedStateLegacySuffix...)
return append(b.unreplicatedPrefix(), LocalRaftTruncatedStateSuffix...)
}

// RaftHardStateKey returns a system-local key for a Raft HardState.
Expand Down
3 changes: 0 additions & 3 deletions pkg/keys/keys_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,10 +156,7 @@ func TestKeyAddressError(t *testing.T) {
"local range ID key .* is not addressable": {
AbortSpanKey(0, uuid.MakeV4()),
RangeTombstoneKey(0),
RaftAppliedIndexLegacyKey(0),
RaftTruncatedStateLegacyKey(0),
RangeLeaseKey(0),
RangeStatsLegacyKey(0),
RaftHardStateKey(0),
RaftLogPrefix(0),
RaftLogKey(0, 0),
Expand Down
4 changes: 1 addition & 3 deletions pkg/keys/printer.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,13 +160,11 @@ var (
{name: "RangeTombstone", suffix: LocalRangeTombstoneSuffix},
{name: "RaftHardState", suffix: LocalRaftHardStateSuffix},
{name: "RangeAppliedState", suffix: LocalRangeAppliedStateSuffix},
{name: "RaftAppliedIndex", suffix: LocalRaftAppliedIndexLegacySuffix},
{name: "LeaseAppliedIndex", suffix: LocalLeaseAppliedIndexLegacySuffix},
{name: "RaftLog", suffix: LocalRaftLogSuffix,
ppFunc: raftLogKeyPrint,
psFunc: raftLogKeyParse,
},
{name: "RaftTruncatedState", suffix: LocalRaftTruncatedStateLegacySuffix},
{name: "RaftTruncatedState", suffix: LocalRaftTruncatedStateSuffix},
{name: "RangeLastReplicaGCTimestamp", suffix: LocalRangeLastReplicaGCTimestampSuffix},
{name: "RangeLease", suffix: LocalRangeLeaseSuffix},
{name: "RangePriorReadSummary", suffix: LocalRangePriorReadSummarySuffix},
Expand Down
4 changes: 0 additions & 4 deletions pkg/keys/printer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,13 +69,9 @@ func TestPrettyPrint(t *testing.T) {

{keys.AbortSpanKey(roachpb.RangeID(1000001), txnID), fmt.Sprintf(`/Local/RangeID/1000001/r/AbortSpan/%q`, txnID), revertSupportUnknown},
{keys.RangeAppliedStateKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeAppliedState", revertSupportUnknown},
{keys.RaftAppliedIndexLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RaftAppliedIndex", revertSupportUnknown},
{keys.LeaseAppliedIndexLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/LeaseAppliedIndex", revertSupportUnknown},
{keys.RaftTruncatedStateLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RaftTruncatedState", revertSupportUnknown},
{keys.RaftTruncatedStateKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RaftTruncatedState", revertSupportUnknown},
{keys.RangeLeaseKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeLease", revertSupportUnknown},
{keys.RangePriorReadSummaryKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangePriorReadSummary", revertSupportUnknown},
{keys.RangeStatsLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeStats", revertSupportUnknown},
{keys.RangeGCThresholdKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeGCThreshold", revertSupportUnknown},
{keys.RangeVersionKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeVersion", revertSupportUnknown},

Expand Down
35 changes: 5 additions & 30 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -982,34 +982,6 @@ func splitTriggerHelper(
log.VEventf(ctx, 1, "LHS's GCThreshold of split is not set")
}

// We're about to write the initial state for the replica. We migrated
// the formerly replicated truncated state into unreplicated keyspace
// in 19.1, but this range may still be using the replicated version
// and we need to make a decision about what to use for the RHS that
// is consistent across the followers: do for the RHS what the LHS
// does: if the LHS has the legacy key, initialize the RHS with a
// legacy key as well.
//
// See VersionUnreplicatedRaftTruncatedState.
truncStateType := stateloader.TruncatedStateUnreplicated
if found, err := storage.MVCCGetProto(
ctx,
batch,
keys.RaftTruncatedStateLegacyKey(rec.GetRangeID()),
hlc.Timestamp{},
nil,
storage.MVCCGetOptions{},
); err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load legacy truncated state")
} else if found {
truncStateType = stateloader.TruncatedStateLegacyReplicated
}

replicaVersion, err := sl.LoadVersion(ctx, batch)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load GCThreshold")
}

// Writing the initial state is subtle since this also seeds the Raft
// group. It becomes more subtle due to proposer-evaluated Raft.
//
Expand Down Expand Up @@ -1039,10 +1011,13 @@ func splitTriggerHelper(
// HardState via a call to synthesizeRaftState. Here, we only call
// writeInitialReplicaState which essentially writes a ReplicaState
// only.

replicaVersion, err := sl.LoadVersion(ctx, batch)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load replica version")
}
*h.AbsPostSplitRight(), err = stateloader.WriteInitialReplicaState(
ctx, batch, *h.AbsPostSplitRight(), split.RightDesc, rightLease,
*gcThreshold, truncStateType, replicaVersion,
*gcThreshold, replicaVersion,
)
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to write initial Replica state")
Expand Down
Loading

0 comments on commit cfd9d5a

Please sign in to comment.