Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
84420: sql/logictests: don't fail parent test when using retry r=knz a=stevendanna

testutils.SucceedsSoon calls Fatal() on the passed testing.T.  Here,
we were calling SucceedsSoon with the root T, which in the case of a
subtest resulted in this error showing up in our logs

    testing.go:1169: test executed panic(nil) or runtime.Goexit:
    subtest may have called FailNow on a parent test

This moves to using SucceedsSoonError so that we can process errors
using the same formatting that we use elsewhere.

Release note: None

85120: roachpb: make range/replica descriptor fields non-nullable r=pavelkalinnikov a=erikgrinaker

This patch makes all fields in range/replica descriptors non-nullable,
fixing a long-standing TODO.

Additionally, this fixes a set of bugs where code would use regular
comparison operators (e.g. `==`) to compare replica descriptors, which
with nullable pointer fields would compare the memory locations rather
than the values. One practical consequence of this was that the
DistSender would fail to use a new leaseholder with a non-`VOTER` type
(e.g.  `VOTER_INCOMING`), instead continuing to try other replicas
before backing off. However, there are further issues surrounding this
bug and it will be addressed separately in a way that is more amenable
to backporting.

The preparatory work for this was done in ea720e3.

Touches #85060.
Touches #38308.
Touches #38465.

Release note: None

85352: opt: revert data race fix r=mgartner a=mgartner

This commit reverts #37972. We no longer lazily build filter props and
share them across multiple threads.

Release note: None

Co-authored-by: Steven Danna <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
4 people committed Aug 1, 2022
4 parents 94b9091 + 7069942 + acdf42a + 831e6e7 commit 590049f
Show file tree
Hide file tree
Showing 41 changed files with 221 additions and 355 deletions.
2 changes: 1 addition & 1 deletion docs/generated/swagger/spec.json
Original file line number Diff line number Diff line change
Expand Up @@ -1322,7 +1322,7 @@
"x-go-package": "github.com/cockroachdb/cockroach/pkg/server/serverpb"
},
"ReplicaDescriptor": {
"description": "ReplicaDescriptor describes a replica location by node ID\n(corresponds to a host:port via lookup on gossip network) and store\nID (identifies the device).\nTODO(jeffreyxiao): All nullable fields in ReplicaDescriptor can be made\nnon-nullable if #38302 is guaranteed to be on all nodes (I.E. 20.1).",
"description": "ReplicaDescriptor describes a replica location by node ID\n(corresponds to a host:port via lookup on gossip network) and store\nID (identifies the device).",
"type": "object",
"properties": {
"node_id": {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -660,7 +660,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
require.Equal(t, roachpb.StoreID(1), entry.Lease().Replica.StoreID)
require.Equal(t, []roachpb.ReplicaDescriptor{
{NodeID: 1, StoreID: 1, ReplicaID: 1},
{NodeID: 3, StoreID: 3, ReplicaID: 3, Type: roachpb.ReplicaTypeNonVoter()},
{NodeID: 3, StoreID: 3, ReplicaID: 3, Type: roachpb.NON_VOTER},
}, entry.Desc().Replicas().Descriptors())

// Make a note of the follower reads metric on n3. We'll check that it was
Expand Down
3 changes: 1 addition & 2 deletions pkg/cli/debug_recover_loss_of_quorum_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -273,7 +273,6 @@ func createIntentOnRangeDescriptor(
func TestJsonSerialization(t *testing.T) {
defer leaktest.AfterTest(t)()

rt := roachpb.VOTER_INCOMING
nr := loqrecoverypb.NodeReplicaInfo{
Replicas: []loqrecoverypb.ReplicaInfo{
{
Expand All @@ -288,7 +287,7 @@ func TestJsonSerialization(t *testing.T) {
NodeID: 1,
StoreID: 2,
ReplicaID: 3,
Type: &rt,
Type: roachpb.VOTER_INCOMING,
},
},
NextReplicaID: 4,
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/dist_sender_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -585,7 +585,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) {

recognizedLeaseHolder := testUserRangeDescriptor3Replicas.Replicas().VoterDescriptors()[1]
recognizedLeaseHolderIncoming := testUserRangeDescriptor3Replicas.Replicas().VoterDescriptors()[2]
recognizedLeaseHolderIncoming.Type = roachpb.ReplicaTypeVoterIncoming()
recognizedLeaseHolderIncoming.Type = roachpb.VOTER_INCOMING
unrecognizedLeaseHolder := roachpb.ReplicaDescriptor{
NodeID: 99,
StoreID: 999,
Expand Down Expand Up @@ -712,7 +712,7 @@ func TestRetryOnNotLeaseHolderError(t *testing.T) {
// initial range descriptor, not the one returned in the NLHE, i.e.
// it won't have the non-nil type.
expRetryReplica := *tc.expLeaseholder
expRetryReplica.Type = nil
expRetryReplica.Type = 0
require.Equal(t, expRetryReplica, retryReplica)
} else {
require.Nil(t, rng.Lease())
Expand Down
6 changes: 2 additions & 4 deletions pkg/kv/kvclient/kvcoord/replica_slice_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,8 +74,7 @@ func TestNewReplicaSlice(t *testing.T) {
require.Equal(t, 3, rs.Len())

// Check that learners are not included.
typLearner := roachpb.LEARNER
rd.InternalReplicas[2].Type = &typLearner
rd.InternalReplicas[2].Type = roachpb.LEARNER
rs, err = NewReplicaSlice(ctx, ns, rd, nil, OnlyPotentialLeaseholders)
require.NoError(t, err)
require.Equal(t, 2, rs.Len())
Expand All @@ -84,8 +83,7 @@ func TestNewReplicaSlice(t *testing.T) {
require.Equal(t, 2, rs.Len())

// Check that non-voters are included iff we ask for them to be.
typNonVoter := roachpb.NON_VOTER
rd.InternalReplicas[2].Type = &typNonVoter
rd.InternalReplicas[2].Type = roachpb.NON_VOTER
rs, err = NewReplicaSlice(ctx, ns, rd, nil, AllExtantReplicas)
require.NoError(t, err)
require.Equal(t, 3, rs.Len())
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -290,7 +290,7 @@ func TestRangeSplitsStickyBit(t *testing.T) {
if err != nil {
t.Fatal(err)
}
if desc.GetStickyBit().IsEmpty() {
if desc.StickyBit.IsEmpty() {
t.Fatal("Sticky bit not set after splitting")
}

Expand All @@ -309,7 +309,7 @@ func TestRangeSplitsStickyBit(t *testing.T) {
if err != nil {
t.Fatal(err)
}
if desc.GetStickyBit().IsEmpty() {
if desc.StickyBit.IsEmpty() {
t.Fatal("Sticky bit not set after splitting")
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvcoord/transport_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func TestTransportMoveToFront(t *testing.T) {
rd2 := roachpb.ReplicaDescriptor{NodeID: 2, StoreID: 2, ReplicaID: 2}
rd3 := roachpb.ReplicaDescriptor{NodeID: 3, StoreID: 3, ReplicaID: 3}
rd3Incoming := roachpb.ReplicaDescriptor{NodeID: 3, StoreID: 3, ReplicaID: 3,
Type: roachpb.ReplicaTypeVoterIncoming()}
Type: roachpb.VOTER_INCOMING}
gt := grpcTransport{replicas: []roachpb.ReplicaDescriptor{rd1, rd2, rd3}}

verifyOrder := func(replicas []roachpb.ReplicaDescriptor) {
Expand Down
31 changes: 15 additions & 16 deletions pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5678,7 +5678,7 @@ func TestAllocatorComputeAction(t *testing.T) {
StoreID: 4,
NodeID: 4,
ReplicaID: 4,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
},
},
Expand Down Expand Up @@ -5754,7 +5754,7 @@ func TestAllocatorComputeAction(t *testing.T) {
StoreID: 4,
NodeID: 4,
ReplicaID: 4,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
},
},
Expand Down Expand Up @@ -6032,13 +6032,13 @@ func TestAllocatorComputeAction(t *testing.T) {
StoreID: 4,
NodeID: 4,
ReplicaID: 4,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
{
StoreID: 6,
NodeID: 6,
ReplicaID: 6,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
},
},
Expand Down Expand Up @@ -6090,7 +6090,7 @@ func TestAllocatorComputeAction(t *testing.T) {
StoreID: 6,
NodeID: 6,
ReplicaID: 6,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
},
},
Expand All @@ -6113,13 +6113,13 @@ func TestAllocatorComputeAction(t *testing.T) {
StoreID: 2,
NodeID: 2,
ReplicaID: 2,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
{
StoreID: 6,
NodeID: 6,
ReplicaID: 6,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
},
},
Expand All @@ -6143,13 +6143,13 @@ func TestAllocatorComputeAction(t *testing.T) {
StoreID: 2,
NodeID: 2,
ReplicaID: 2,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
{
StoreID: 3,
NodeID: 3,
ReplicaID: 3,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
},
},
Expand Down Expand Up @@ -6646,19 +6646,19 @@ func TestAllocatorComputeActionDecommission(t *testing.T) {
StoreID: 4,
NodeID: 4,
ReplicaID: 4,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
{
StoreID: 6,
NodeID: 6,
ReplicaID: 6,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
{
StoreID: 7,
NodeID: 7,
ReplicaID: 7,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
},
},
Expand All @@ -6683,13 +6683,13 @@ func TestAllocatorComputeActionDecommission(t *testing.T) {
StoreID: 4,
NodeID: 4,
ReplicaID: 4,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
{
StoreID: 6,
NodeID: 6,
ReplicaID: 6,
Type: roachpb.ReplicaTypeNonVoter(),
Type: roachpb.NON_VOTER,
},
},
},
Expand Down Expand Up @@ -6719,7 +6719,6 @@ func TestAllocatorRemoveLearner(t *testing.T) {
defer log.Scope(t).Close(t)

conf := roachpb.SpanConfig{NumReplicas: 3}
learnerType := roachpb.LEARNER
rangeWithLearnerDesc := roachpb.RangeDescriptor{
InternalReplicas: []roachpb.ReplicaDescriptor{
{
Expand All @@ -6731,7 +6730,7 @@ func TestAllocatorRemoveLearner(t *testing.T) {
StoreID: 2,
NodeID: 2,
ReplicaID: 2,
Type: &learnerType,
Type: roachpb.LEARNER,
},
},
}
Expand Down
6 changes: 1 addition & 5 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -707,11 +707,7 @@ func RunCommitTrigger(
}
if sbt := ct.GetStickyBitTrigger(); sbt != nil {
newDesc := *rec.Desc()
if !sbt.StickyBit.IsEmpty() {
newDesc.StickyBit = &sbt.StickyBit
} else {
newDesc.StickyBit = nil
}
newDesc.StickyBit = sbt.StickyBit
var res result.Result
res.Replicated.State = &kvserverpb.ReplicaState{
Desc: &newDesc,
Expand Down
12 changes: 6 additions & 6 deletions pkg/kv/kvserver/batcheval/cmd_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,8 +122,8 @@ func TestLeaseCommandLearnerReplica(t *testing.T) {
ctx := context.Background()
const voterStoreID, learnerStoreID roachpb.StoreID = 1, 2
replicas := []roachpb.ReplicaDescriptor{
{NodeID: 1, StoreID: voterStoreID, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 1},
{NodeID: 2, StoreID: learnerStoreID, Type: roachpb.ReplicaTypeLearner(), ReplicaID: 2},
{NodeID: 1, StoreID: voterStoreID, Type: roachpb.VOTER_FULL, ReplicaID: 1},
{NodeID: 2, StoreID: learnerStoreID, Type: roachpb.LEARNER, ReplicaID: 2},
}
desc := roachpb.RangeDescriptor{}
desc.SetReplicas(roachpb.MakeReplicaSet(replicas))
Expand Down Expand Up @@ -183,8 +183,8 @@ func TestLeaseTransferForwardsStartTime(t *testing.T) {
defer batch.Close()

replicas := []roachpb.ReplicaDescriptor{
{NodeID: 1, StoreID: 1, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 1},
{NodeID: 2, StoreID: 2, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 2},
{NodeID: 1, StoreID: 1, Type: roachpb.VOTER_FULL, ReplicaID: 1},
{NodeID: 2, StoreID: 2, Type: roachpb.VOTER_FULL, ReplicaID: 2},
}
desc := roachpb.RangeDescriptor{}
desc.SetReplicas(roachpb.MakeReplicaSet(replicas))
Expand Down Expand Up @@ -301,15 +301,15 @@ func TestCheckCanReceiveLease(t *testing.T) {
t.Run(tc.leaseholderType.String(), func(t *testing.T) {
repDesc := roachpb.ReplicaDescriptor{
ReplicaID: 1,
Type: &tc.leaseholderType,
Type: tc.leaseholderType,
}
rngDesc := roachpb.RangeDescriptor{
InternalReplicas: []roachpb.ReplicaDescriptor{repDesc},
}
if tc.anotherReplicaType != none {
anotherDesc := roachpb.ReplicaDescriptor{
ReplicaID: 2,
Type: &tc.anotherReplicaType,
Type: tc.anotherReplicaType,
}
rngDesc.InternalReplicas = append(rngDesc.InternalReplicas, anotherDesc)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_split_burst_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ func setupSplitBurstTest(t *testing.T, delay time.Duration) *splitBurstTest {
if args.Split == nil || delay == 0 {
return 0, nil
}
if args.Split.RightDesc.GetStickyBit() != magicStickyBit {
if args.Split.RightDesc.StickyBit != magicStickyBit {
return 0, nil
}
select {
Expand Down
9 changes: 4 additions & 5 deletions pkg/kv/kvserver/loqrecovery/recovery_env_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,10 +105,10 @@ type descriptorViewWrapper struct {
// for the purpose of yaml representation and to avoid leaking test
// specific code into production.
type replicaDescriptorView struct {
NodeID roachpb.NodeID `yaml:"NodeID"`
StoreID roachpb.StoreID `yaml:"StoreID"`
ReplicaID roachpb.ReplicaID `yaml:"ReplicaID"`
ReplicaType *roachpb.ReplicaType `yaml:"ReplicaType,omitempty"`
NodeID roachpb.NodeID `yaml:"NodeID"`
StoreID roachpb.StoreID `yaml:"StoreID"`
ReplicaID roachpb.ReplicaID `yaml:"ReplicaID"`
ReplicaType roachpb.ReplicaType `yaml:"ReplicaType,omitempty"`
}

func (r replicaDescriptorView) asReplicaDescriptor() roachpb.ReplicaDescriptor {
Expand Down Expand Up @@ -273,7 +273,6 @@ func buildReplicaDescriptorFromTestData(
InternalReplicas: replicas,
NextReplicaID: maxReplicaID + 1,
Generation: replica.Generation,
StickyBit: nil,
}
lease := roachpb.Lease{
Start: clock.Now().Add(5*time.Minute.Nanoseconds(), 0).UnsafeToClockTimestamp(),
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/merge_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ func (mq *mergeQueue) process(

// Range was manually split and not expired, so skip merging.
now := mq.store.Clock().NowAsClockTimestamp()
if now.ToTimestamp().Less(rhsDesc.GetStickyBit()) {
if now.ToTimestamp().Less(rhsDesc.StickyBit) {
log.VEventf(ctx, 2, "skipping merge: ranges were manually split and sticky bit was not expired")
// TODO(jeffreyxiao): Consider returning a purgatory error to avoid
// repeatedly processing ranges that cannot be merged.
Expand Down Expand Up @@ -309,7 +309,7 @@ func (mq *mergeQueue) process(
// Defensive sanity check that the ranges involved only have either VOTER_FULL
// and NON_VOTER replicas.
for i := range leftRepls {
if typ := leftRepls[i].GetType(); !(typ == roachpb.VOTER_FULL || typ == roachpb.NON_VOTER) {
if typ := leftRepls[i].Type; !(typ == roachpb.VOTER_FULL || typ == roachpb.NON_VOTER) {
return false,
errors.AssertionFailedf(
`cannot merge because lhs is either in a joint state or has learner replicas: %v`,
Expand Down Expand Up @@ -370,7 +370,7 @@ func (mq *mergeQueue) process(
rightRepls = rhsDesc.Replicas().Descriptors()
}
for i := range rightRepls {
if typ := rightRepls[i].GetType(); !(typ == roachpb.VOTER_FULL || typ == roachpb.NON_VOTER) {
if typ := rightRepls[i].Type; !(typ == roachpb.VOTER_FULL || typ == roachpb.NON_VOTER) {
log.Infof(ctx, "RHS Type: %s", typ)
return false,
errors.AssertionFailedf(
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/raft_snapshot_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ func (rq *raftSnapshotQueue) processRaftSnapshot(
}
snapType := kvserverpb.SnapshotRequest_VIA_SNAPSHOT_QUEUE

if typ := repDesc.GetType(); typ == roachpb.LEARNER || typ == roachpb.NON_VOTER {
if typ := repDesc.Type; typ == roachpb.LEARNER || typ == roachpb.NON_VOTER {
if fn := repl.store.cfg.TestingKnobs.RaftSnapshotQueueSkipReplica; fn != nil && fn() {
return false, nil
}
Expand Down
Loading

0 comments on commit 590049f

Please sign in to comment.