Skip to content

Commit

Permalink
kv*: make RangeDescriptor.Generation non-nullable
Browse files Browse the repository at this point in the history
This field used to be nullable because the proto's encoding of old
protos without the field had to be stable when unmarshalled/marshalled.
Since #38302, that's no longer needed, and so the field can now be as
nature intended.

Release note: None
  • Loading branch information
andreimatei committed May 15, 2020
1 parent 47cae95 commit 7499244
Show file tree
Hide file tree
Showing 14 changed files with 160 additions and 207 deletions.
3 changes: 0 additions & 3 deletions c-deps/libroach/protos/roachpb/metadata.pb.cc

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

2 changes: 0 additions & 2 deletions c-deps/libroach/protos/roachpb/metadata.pb.h

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

3 changes: 1 addition & 2 deletions pkg/kv/kvclient/kvcoord/dist_sender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -919,8 +919,7 @@ func TestEvictOnFirstRangeGossip(t *testing.T) {
}
}
// Tweak the descriptor so that the gossip callback will be invoked.
desc.Generation = new(int64)
*desc.Generation = 1
desc.Generation = 1
if err := g.AddInfoProto(gossip.KeyFirstRangeDescriptor, &desc, 0); err != nil {
t.Fatal(err)
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvclient/kvcoord/range_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ func makeLookupRequestKey(
// two cases, we can use the generation of the previous descriptor.
if prevDesc != nil && prevDesc.GetGenerationComparable() {
ret.WriteString(":")
ret.WriteString(strconv.FormatInt(prevDesc.GetGeneration(), 10))
ret.WriteString(strconv.FormatInt(prevDesc.Generation, 10))
}
return ret.String()
}
Expand Down Expand Up @@ -547,7 +547,7 @@ func (rdc *RangeDescriptorCache) evictCachedRangeDescriptorLocked(
// doing it again (which would prompt another expensive lookup).
if seenDesc != nil {
if seenDesc.GetGenerationComparable() && cachedDesc.GetGenerationComparable() {
if seenDesc.GetGeneration() != cachedDesc.GetGeneration() {
if seenDesc.Generation != cachedDesc.Generation {
return nil
}
} else if !seenDesc.GetGenerationComparable() && !cachedDesc.GetGenerationComparable() {
Expand Down Expand Up @@ -684,7 +684,7 @@ func (rdc *RangeDescriptorCache) clearOverlappingCachedRangeDescriptors(
// the end key is inclusive.
if cached.StartKey.Less(desc.EndKey) && !cached.EndKey.Less(desc.EndKey) {
if desc.GetGenerationComparable() && cached.GetGenerationComparable() {
if desc.GetGeneration() <= cached.GetGeneration() {
if desc.Generation <= cached.Generation {
// Generations are comparable and a newer descriptor already exists in
// cache.
continueWithInsert = false
Expand Down Expand Up @@ -718,7 +718,7 @@ func (rdc *RangeDescriptorCache) clearOverlappingCachedRangeDescriptors(
if desc.GetGenerationComparable() && descriptor.GetGenerationComparable() {
// If generations are comparable, then check generations to see if we
// evict.
if desc.GetGeneration() <= descriptor.GetGeneration() {
if desc.Generation <= descriptor.Generation {
continueWithInsert = false
} else {
entriesToEvict = append(entriesToEvict, e)
Expand Down
10 changes: 5 additions & 5 deletions pkg/kv/kvclient/kvcoord/range_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1166,32 +1166,32 @@ func TestRangeCacheGeneration(t *testing.T) {
descAM1 := &roachpb.RangeDescriptor{
StartKey: roachpb.RKey("a"),
EndKey: roachpb.RKey("m"),
Generation: proto.Int64(1),
Generation: 1,
GenerationComparable: proto.Bool(true),
}
descMZ3 := &roachpb.RangeDescriptor{
StartKey: roachpb.RKey("m"),
EndKey: roachpb.RKey("z"),
Generation: proto.Int64(3),
Generation: 3,
GenerationComparable: proto.Bool(true),
}

descBY0 := &roachpb.RangeDescriptor{
StartKey: roachpb.RKey("b"),
EndKey: roachpb.RKey("y"),
Generation: proto.Int64(0),
Generation: 0,
GenerationComparable: proto.Bool(true),
}
descBY2 := &roachpb.RangeDescriptor{
StartKey: roachpb.RKey("b"),
EndKey: roachpb.RKey("y"),
Generation: proto.Int64(2),
Generation: 2,
GenerationComparable: proto.Bool(true),
}
descBY4 := &roachpb.RangeDescriptor{
StartKey: roachpb.RKey("b"),
EndKey: roachpb.RKey("y"),
Generation: proto.Int64(4),
Generation: 4,
GenerationComparable: proto.Bool(true),
}
descBYIncomparable := &roachpb.RangeDescriptor{
Expand Down
22 changes: 11 additions & 11 deletions pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ func TestStoreRangeMergeTwoEmptyRanges(t *testing.T) {

// The LHS has been split once and merged once, so it should have received
// two generation bumps.
if e, a := int64(2), lhsRepl.Desc().GetGeneration(); e != a {
if e, a := int64(2), lhsRepl.Desc().Generation; e != a {
t.Fatalf("expected LHS to have generation %d, but got %d", e, a)
}
}
Expand Down Expand Up @@ -810,14 +810,14 @@ func TestStoreRangeSplitMergeGeneration(t *testing.T) {
assert.NoError(t, err)
leftRepl := store.LookupReplica(keys.MustAddr(leftKey))
assert.NotNil(t, leftRepl)
preSplitGen := leftRepl.Desc().GetGeneration()
preSplitGen := leftRepl.Desc().Generation
leftDesc, rightDesc, err := s.SplitRange(rightKey)
assert.NoError(t, err)

// Split should increment the LHS' generation and also propagate the result
// to the RHS.
assert.Equal(t, preSplitGen+1, leftDesc.GetGeneration())
assert.Equal(t, preSplitGen+1, rightDesc.GetGeneration())
assert.Equal(t, preSplitGen+1, leftDesc.Generation)
assert.Equal(t, preSplitGen+1, rightDesc.Generation)

if rhsHasHigherGen {
// Split the RHS again to increment its generation once more, so that
Expand All @@ -827,7 +827,7 @@ func TestStoreRangeSplitMergeGeneration(t *testing.T) {
//
rightDesc, _, err = s.SplitRange(rightKey.Next())
assert.NoError(t, err)
assert.Equal(t, preSplitGen+2, rightDesc.GetGeneration())
assert.Equal(t, preSplitGen+2, rightDesc.Generation)
} else {
// Split and merge the LHS to increment the generation (it ends up
// being incremented by two). Note that leftKey.Next() is still in
Expand All @@ -837,12 +837,12 @@ func TestStoreRangeSplitMergeGeneration(t *testing.T) {
// |--left@4---||---right@2---|
var tmpRightDesc roachpb.RangeDescriptor
leftDesc, tmpRightDesc, err = s.SplitRange(leftKey.Next())
assert.Equal(t, preSplitGen+2, leftDesc.GetGeneration())
assert.Equal(t, preSplitGen+2, tmpRightDesc.GetGeneration())
assert.Equal(t, preSplitGen+2, leftDesc.Generation)
assert.Equal(t, preSplitGen+2, tmpRightDesc.Generation)
assert.NoError(t, err)
leftDesc, err = s.MergeRanges(leftKey)
assert.NoError(t, err)
assert.Equal(t, preSplitGen+3, leftDesc.GetGeneration())
assert.Equal(t, preSplitGen+3, leftDesc.Generation)
}

// Make sure the split/merge shenanigans above didn't get the range
Expand All @@ -855,12 +855,12 @@ func TestStoreRangeSplitMergeGeneration(t *testing.T) {
mergedDesc, err := s.MergeRanges(leftKey)
assert.NoError(t, err)

maxPreMergeGen := leftDesc.GetGeneration()
if rhsGen := rightDesc.GetGeneration(); rhsGen > maxPreMergeGen {
maxPreMergeGen := leftDesc.Generation
if rhsGen := rightDesc.Generation; rhsGen > maxPreMergeGen {
maxPreMergeGen = rhsGen
}

assert.Equal(t, maxPreMergeGen+1, mergedDesc.GetGeneration())
assert.Equal(t, maxPreMergeGen+1, mergedDesc.Generation)
assert.Equal(t, leftDesc.RangeID, mergedDesc.RangeID)
})
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1678,17 +1678,17 @@ func TestChangeReplicasGeneration(t *testing.T) {
t.Fatal(err)
}

oldGeneration := repl.Desc().GetGeneration()
oldGeneration := repl.Desc().Generation
chgs := roachpb.MakeReplicationChanges(roachpb.ADD_REPLICA, roachpb.ReplicationTarget{
NodeID: mtc.idents[1].NodeID,
StoreID: mtc.idents[1].StoreID,
})
if _, err := repl.ChangeReplicas(context.Background(), repl.Desc(), kvserver.SnapshotRequest_REBALANCE, storagepb.ReasonRangeUnderReplicated, "", chgs); err != nil {
t.Fatalf("unexpected error: %v", err)
}
assert.EqualValues(t, repl.Desc().GetGeneration(), oldGeneration+2)
assert.EqualValues(t, repl.Desc().Generation, oldGeneration+2)

oldGeneration = repl.Desc().GetGeneration()
oldGeneration = repl.Desc().Generation
oldDesc := repl.Desc()
chgs[0].ChangeType = roachpb.REMOVE_REPLICA
newDesc, err := repl.ChangeReplicas(context.Background(), oldDesc, kvserver.SnapshotRequest_REBALANCE, storagepb.ReasonRangeOverReplicated, "", chgs)
Expand All @@ -1699,7 +1699,7 @@ func TestChangeReplicasGeneration(t *testing.T) {
// +1 for entering joint config due to demotion
// +1 for transitioning out of joint config
// +1 for removing learner
assert.EqualValues(t, repl.Desc().GetGeneration(), oldGeneration+3, "\nold: %+v\nnew: %+v", oldDesc, newDesc)
assert.EqualValues(t, repl.Desc().Generation, oldGeneration+3, "\nold: %+v\nnew: %+v", oldDesc, newDesc)
}

func TestSystemZoneConfigs(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -644,7 +644,7 @@ func (r *Replica) AdminMerge(
updatedLeftDesc := *origLeftDesc
// lhs.Generation = max(rhs.Generation, lhs.Generation)+1.
// See the comment on the Generation field for why generation are useful.
if updatedLeftDesc.GetGeneration() < rightDesc.GetGeneration() {
if updatedLeftDesc.Generation < rightDesc.Generation {
updatedLeftDesc.Generation = rightDesc.Generation
}
updatedLeftDesc.IncrementGeneration()
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/range_lookup.go
Original file line number Diff line number Diff line change
Expand Up @@ -219,7 +219,7 @@ func RangeLookup(
// Since we support scanning non-transactionally, it's possible that
// we pick up both the pre- and post-split descriptor for a range.
if desc.GetGenerationComparable() && matchingRanges[0].GetGenerationComparable() {
if desc.GetGeneration() > matchingRanges[0].GetGeneration() {
if desc.Generation > matchingRanges[0].Generation {
// If both generations are comparable, we take the range
// descriptor with the newer generation.
matchingRanges[0] = *desc
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/range_lookup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,21 +33,21 @@ func TestRangeLookupRaceSplits(t *testing.T) {
RangeID: 1,
StartKey: roachpb.RKey("j"),
EndKey: roachpb.RKey("p"),
Generation: proto.Int64(0),
Generation: 0,
GenerationComparable: proto.Bool(true),
}
desc1AfterSplit := roachpb.RangeDescriptor{
RangeID: 1,
StartKey: roachpb.RKey("j"),
EndKey: roachpb.RKey("m"),
Generation: proto.Int64(1),
Generation: 1,
GenerationComparable: proto.Bool(true),
}
desc2AfterSplit := roachpb.RangeDescriptor{
RangeID: 2,
StartKey: roachpb.RKey("m"),
EndKey: roachpb.RKey("p"),
Generation: proto.Int64(1),
Generation: 1,
GenerationComparable: proto.Bool(true),
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/roachpb/data_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1674,7 +1674,7 @@ func TestChangeReplicasTrigger_String(t *testing.T) {
repl3,
},
NextReplicaID: 10,
Generation: proto.Int64(5),
Generation: 5,
GenerationComparable: proto.Bool(true),
},
}
Expand Down
17 changes: 3 additions & 14 deletions pkg/roachpb/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/errors"
"github.com/gogo/protobuf/proto"
)

// NodeID is a custom type for a cockroach node ID. (not a raft node ID)
Expand Down Expand Up @@ -238,20 +237,10 @@ func (r *RangeDescriptor) IsInitialized() bool {
return len(r.EndKey) != 0
}

// GetGeneration returns the generation of this RangeDescriptor.
func (r *RangeDescriptor) GetGeneration() int64 {
if r.Generation != nil {
return *r.Generation
}
return 0
}

// IncrementGeneration increments the generation of this RangeDescriptor.
// This method mutates the receiver; do not call it with shared RangeDescriptors.
func (r *RangeDescriptor) IncrementGeneration() {
// Create a new *int64 for the new generation. We permit shallow copies of
// RangeDescriptors, so we need to be careful not to mutate the
// potentially-shared generation counter.
r.Generation = proto.Int64(r.GetGeneration() + 1)
r.Generation++
}

// GetGenerationComparable returns if the generation of this RangeDescriptor is comparable.
Expand Down Expand Up @@ -320,7 +309,7 @@ func (r RangeDescriptor) String() string {
} else {
buf.WriteString("<no replicas>")
}
fmt.Fprintf(&buf, ", next=%d, gen=%d", r.NextReplicaID, r.GetGeneration())
fmt.Fprintf(&buf, ", next=%d, gen=%d", r.NextReplicaID, r.Generation)
if !r.GetGenerationComparable() {
buf.WriteString("?")
}
Expand Down
Loading

0 comments on commit 7499244

Please sign in to comment.