Skip to content

Commit

Permalink
keys: rename RangeLastGCKey to RangeGCThresholdKey
Browse files Browse the repository at this point in the history
The key `RangeLastGCKey` was not actually used to store the last GC
timestamp, but rather the last GC threshold. This commit renames the key
to the more appropriate `RangeGCThresholdKey`.

Release note: None
  • Loading branch information
erikgrinaker committed Jun 1, 2021
1 parent 7762f00 commit a6e4b3e
Show file tree
Hide file tree
Showing 15 changed files with 30 additions and 31 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/storageccl/export.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ func declareKeysExport(
latchSpans, lockSpans *spanset.SpanSet,
) {
batcheval.DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLastGCKey(header.RangeID)})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeGCThresholdKey(header.RangeID)})
}

// evalExport dumps the requested keys into files of non-overlapping key ranges
Expand Down
5 changes: 3 additions & 2 deletions pkg/keys/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,8 +77,9 @@ var (
LocalAbortSpanSuffix = []byte("abc-")
// localRangeFrozenStatusSuffix is DEPRECATED and remains to prevent reuse.
localRangeFrozenStatusSuffix = []byte("fzn-")
// LocalRangeLastGCSuffix is the suffix for the last GC.
LocalRangeLastGCSuffix = []byte("lgc-")
// LocalRangeGCThresholdSuffix is the suffix for the GC threshold. It keeps
// the lgc- ("last GC") representation for backwards compatibility.
LocalRangeGCThresholdSuffix = []byte("lgc-")
// LocalRangeAppliedStateSuffix is the suffix for the range applied state
// key.
LocalRangeAppliedStateSuffix = []byte("rask")
Expand Down
2 changes: 1 addition & 1 deletion pkg/keys/doc.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,7 @@ var _ = [...]interface{}{
// Typical examples are MVCC stats and the abort span. They all share
// `LocalRangeIDPrefix` and `LocalRangeIDReplicatedInfix`.
AbortSpanKey, // "abc-"
RangeLastGCKey, // "lgc-"
RangeGCThresholdKey, // "lgc-"
RangeAppliedStateKey, // "rask"
RaftAppliedIndexLegacyKey, // "rfta"
RaftTruncatedStateLegacyKey, // "rftt"
Expand Down
14 changes: 6 additions & 8 deletions pkg/keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -284,12 +284,10 @@ func RangeStatsLegacyKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangeStatsLegacyKey()
}

// RangeLastGCKey returns a system-local key for last used GC threshold on the
// RangeGCThresholdKey returns a system-local key for last used GC threshold on the
// user keyspace. Reads and writes <= this timestamp will not be served.
//
// TODO(tschottdorf): should be renamed to RangeGCThresholdKey.
func RangeLastGCKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangeLastGCKey()
func RangeGCThresholdKey(rangeID roachpb.RangeID) roachpb.Key {
return MakeRangeIDPrefixBuf(rangeID).RangeGCThresholdKey()
}

// RangeVersionKey returns a system-local for the range version.
Expand Down Expand Up @@ -971,9 +969,9 @@ func (b RangeIDPrefixBuf) RangeStatsLegacyKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRangeStatsLegacySuffix...)
}

// RangeLastGCKey returns a system-local key for the last GC.
func (b RangeIDPrefixBuf) RangeLastGCKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRangeLastGCSuffix...)
// RangeGCThresholdKey returns a system-local key for the GC threshold.
func (b RangeIDPrefixBuf) RangeGCThresholdKey() roachpb.Key {
return append(b.replicatedPrefix(), LocalRangeGCThresholdSuffix...)
}

// RangeVersionKey returns a system-local key for the range version.
Expand Down
2 changes: 1 addition & 1 deletion pkg/keys/printer.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,7 @@ var (
{name: "RangeLease", suffix: LocalRangeLeaseSuffix},
{name: "RangePriorReadSummary", suffix: LocalRangePriorReadSummarySuffix},
{name: "RangeStats", suffix: LocalRangeStatsLegacySuffix},
{name: "RangeLastGC", suffix: LocalRangeLastGCSuffix},
{name: "RangeGCThreshold", suffix: LocalRangeGCThresholdSuffix},
{name: "RangeVersion", suffix: LocalRangeVersionSuffix},
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/keys/printer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ func TestPrettyPrint(t *testing.T) {
{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.RangeLastGCKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeLastGC", revertSupportUnknown},
{keys.RangeGCThresholdKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeGCThreshold", revertSupportUnknown},
{keys.RangeVersionKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeVersion", revertSupportUnknown},

{keys.RaftHardStateKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RaftHardState", revertSupportUnknown},
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ func declareKeysGC(
// request first to bump the thresholds, and then another one that actually does work
// but can avoid declaring these keys below.
if !gcr.Threshold.IsEmpty() {
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLastGCKey(rs.GetRangeID())})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeGCThresholdKey(rs.GetRangeID())})
}
// Needed for Range bounds checks in calls to EvalContext.ContainsKey.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_revert_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func declareKeysRevertRange(
// We look up the range descriptor key to check whether the span
// is equal to the entire range for fast stats updating.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLastGCKey(rs.GetRangeID())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeGCThresholdKey(rs.GetRangeID())})
}

// isEmptyKeyTimeRange checks if the span has no writes in (since,until].
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/debug_print.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,7 +274,7 @@ func tryRangeIDKey(kv storage.MVCCKeyValue) (string, error) {
case bytes.Equal(suffix, keys.LocalAbortSpanSuffix):
msg = &roachpb.AbortSpanEntry{}

case bytes.Equal(suffix, keys.LocalRangeLastGCSuffix):
case bytes.Equal(suffix, keys.LocalRangeGCThresholdSuffix):
msg = &hlc.Timestamp{}

case bytes.Equal(suffix, keys.LocalRangeVersionSuffix):
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/kvserverpb/state.pb.go

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

2 changes: 1 addition & 1 deletion pkg/kv/kvserver/kvserverpb/state.proto
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ message ReplicaState {
roachpb.Lease lease = 4;
// The truncation state of the Raft log.
roachpb.RaftTruncatedState truncated_state = 5;
// gcThreshold is the GC threshold of the Range, typically updated when keys
// GCThreshold is the GC threshold of the Range, typically updated when keys
// are garbage collected. Reads and writes at timestamps <= this time will
// not be served.
util.hlc.Timestamp gc_threshold = 6 [(gogoproto.customname) = "GCThreshold"];
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/rditer/replica_data_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ func createRangeData(
}{
{keys.AbortSpanKey(desc.RangeID, testTxnID), ts0},
{keys.AbortSpanKey(desc.RangeID, testTxnID2), ts0},
{keys.RangeLastGCKey(desc.RangeID), ts0},
{keys.RangeGCThresholdKey(desc.RangeID), ts0},
{keys.RangeAppliedStateKey(desc.RangeID), ts0},
{keys.RaftAppliedIndexLegacyKey(desc.RangeID), ts0},
{keys.RaftTruncatedStateLegacyKey(desc.RangeID), ts0},
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_eval_context_span.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord(
// not be served.
func (rec SpanSetReplicaEvalContext) GetGCThreshold() hlc.Timestamp {
rec.ss.AssertAllowed(spanset.SpanReadOnly,
roachpb.Span{Key: keys.RangeLastGCKey(rec.GetRangeID())},
roachpb.Span{Key: keys.RangeGCThresholdKey(rec.GetRangeID())},
)
return rec.i.GetGCThreshold()
}
Expand Down
16 changes: 8 additions & 8 deletions pkg/kv/kvserver/spanset/spanset_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,11 +29,11 @@ func TestSpanSetGetSpansScope(t *testing.T) {

var ss SpanSet
ss.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("a")})
ss.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: keys.RangeLastGCKey(1)})
ss.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: keys.RangeGCThresholdKey(1)})
ss.AddNonMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")})

exp := []Span{
{Span: roachpb.Span{Key: keys.RangeLastGCKey(1)}},
{Span: roachpb.Span{Key: keys.RangeGCThresholdKey(1)}},
}
if act := ss.GetSpans(SpanReadOnly, SpanLocal); !reflect.DeepEqual(act, exp) {
t.Errorf("get local spans: got %v, expected %v", act, exp)
Expand Down Expand Up @@ -70,7 +70,7 @@ func TestSpanSetIterate(t *testing.T) {
spA := roachpb.Span{Key: roachpb.Key("a")}
spRO := roachpb.Span{Key: roachpb.Key("r"), EndKey: roachpb.Key("o")}
spRW := roachpb.Span{Key: roachpb.Key("r"), EndKey: roachpb.Key("w")}
spLocal := roachpb.Span{Key: keys.RangeLastGCKey(1)}
spLocal := roachpb.Span{Key: keys.RangeGCThresholdKey(1)}

ss := new(SpanSet)
ss.AddNonMVCC(SpanReadOnly, spLocal)
Expand Down Expand Up @@ -104,7 +104,7 @@ func TestSpanSetMerge(t *testing.T) {
spBC := roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("c")}
spCE := roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("e")}
spBE := roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("e")}
spLocal := roachpb.Span{Key: keys.RangeLastGCKey(1)}
spLocal := roachpb.Span{Key: keys.RangeGCThresholdKey(1)}

var ss SpanSet
ss.AddNonMVCC(SpanReadOnly, spLocal)
Expand Down Expand Up @@ -205,7 +205,7 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) {
ss.AddMVCC(SpanReadOnly, roachpb.Span{Key: roachpb.Key("g")}, hlc.Timestamp{WallTime: 2})
ss.AddMVCC(SpanReadWrite, roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("o")}, hlc.Timestamp{WallTime: 2})
ss.AddMVCC(SpanReadWrite, roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 2})
ss.AddNonMVCC(SpanReadWrite, roachpb.Span{Key: keys.RangeLastGCKey(1)})
ss.AddNonMVCC(SpanReadWrite, roachpb.Span{Key: keys.RangeGCThresholdKey(1)})

var allowedRO = []struct {
span roachpb.Span
Expand All @@ -225,8 +225,8 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) {
{roachpb.Span{Key: roachpb.Key("s")}, hlc.Timestamp{WallTime: 1}},

// Local keys.
{roachpb.Span{Key: keys.RangeLastGCKey(1)}, hlc.Timestamp{}},
{roachpb.Span{Key: keys.RangeLastGCKey(1)}, hlc.Timestamp{WallTime: 1}},
{roachpb.Span{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{}},
{roachpb.Span{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{WallTime: 1}},
}
for _, tc := range allowedRO {
if err := ss.CheckAllowedAt(SpanReadOnly, tc.span, tc.ts); err != nil {
Expand Down Expand Up @@ -257,7 +257,7 @@ func TestSpanSetCheckAllowedAtTimestamps(t *testing.T) {
{roachpb.Span{Key: roachpb.Key("m"), EndKey: roachpb.Key("n")}, hlc.Timestamp{WallTime: 3}},

// Local keys.
{roachpb.Span{Key: keys.RangeLastGCKey(1)}, hlc.Timestamp{}},
{roachpb.Span{Key: keys.RangeGCThresholdKey(1)}, hlc.Timestamp{}},
}
for _, tc := range allowedRW {
if err := ss.CheckAllowedAt(SpanReadWrite, tc.span, tc.ts); err != nil {
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/stateloader/stateloader.go
Original file line number Diff line number Diff line change
Expand Up @@ -528,7 +528,7 @@ func (rsl StateLoader) LoadGCThreshold(
ctx context.Context, reader storage.Reader,
) (*hlc.Timestamp, error) {
var t hlc.Timestamp
_, err := storage.MVCCGetProto(ctx, reader, rsl.RangeLastGCKey(),
_, err := storage.MVCCGetProto(ctx, reader, rsl.RangeGCThresholdKey(),
hlc.Timestamp{}, &t, storage.MVCCGetOptions{})
return &t, err
}
Expand All @@ -544,7 +544,7 @@ func (rsl StateLoader) SetGCThreshold(
return errors.New("cannot persist nil GCThreshold")
}
return storage.MVCCPutProto(ctx, readWriter, ms,
rsl.RangeLastGCKey(), hlc.Timestamp{}, nil, threshold)
rsl.RangeGCThresholdKey(), hlc.Timestamp{}, nil, threshold)
}

// LoadVersion loads the replica version.
Expand Down

0 comments on commit a6e4b3e

Please sign in to comment.