Skip to content

Commit ac400e7

Browse files
craig[bot]wenyihu6
andcommitted
Merge #150462
150462: kvserver: pass ReplicationTarget for TransferLease r=tbg a=wenyihu6 **asim: add node id for simulator replica** Previously, the simulator replica interface did not include a NodeID() method. Future changes to replica_rankings.go will require this as part of the candidate replica interface. This commit updates the simulator replica to implement NodeID() method. Epic: none Release note: none --- **kvserver: pass ReplicationTarget for TransferLease** Previously, the replicate queue’s TransferLease used store IDs to represent the source and target. This will change in upcoming mma work, which requires full replication targets as arguments. This change also brings it in line with RangeRebalancer.RelocateRange for consistency. Epic: none Release note: none Co-authored-by: wenyihu6 <[email protected]>
2 parents 72657a7 + 34d2849 commit ac400e7

File tree

6 files changed

+40
-11
lines changed

6 files changed

+40
-11
lines changed

pkg/kv/kvserver/asim/state/impl.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -628,6 +628,7 @@ func (s *state) addReplica(
628628

629629
desc := rng.desc.AddReplica(roachpb.NodeID(nodeID), roachpb.StoreID(storeID), rtype)
630630
replica := &replica{
631+
nodeID: nodeID,
631632
replicaID: ReplicaID(desc.ReplicaID),
632633
storeID: storeID,
633634
rangeID: rangeID,
@@ -1607,6 +1608,7 @@ func (r *rng) Size() int64 {
16071608
// replica is an implementation of the Replica interface.
16081609
type replica struct {
16091610
replicaID ReplicaID
1611+
nodeID NodeID
16101612
storeID StoreID
16111613
rangeID RangeID
16121614
desc roachpb.ReplicaDescriptor
@@ -1618,6 +1620,11 @@ func (r *replica) ReplicaID() ReplicaID {
16181620
return r.replicaID
16191621
}
16201622

1623+
// NodeID returns the ID of the node this replica is on.
1624+
func (r *replica) NodeID() NodeID {
1625+
return r.nodeID
1626+
}
1627+
16211628
// StoreID returns the ID of the store this replica is on.
16221629
func (r *replica) StoreID() StoreID {
16231630
return r.storeID

pkg/kv/kvserver/asim/state/state.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -270,6 +270,8 @@ type Range interface {
270270
type Replica interface {
271271
// ReplicaID returns the ID of this replica.
272272
ReplicaID() ReplicaID
273+
// NodeID returns the ID of the node this replica is on.
274+
NodeID() NodeID
273275
// StoreID returns the ID of the store this replica is on.
274276
StoreID() StoreID
275277
// Descriptor returns the descriptor for this replica.

pkg/kv/kvserver/asim/storerebalancer/candidate_replica.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,11 @@ func (sr *simulatorReplica) OwnsValidLease(context.Context, hlc.ClockTimestamp)
4949
return sr.repl.HoldsLease()
5050
}
5151

52+
// NodeID returns the Replica's NodeID.
53+
func (sr *simulatorReplica) NodeID() roachpb.NodeID {
54+
return roachpb.NodeID(sr.repl.NodeID())
55+
}
56+
5257
// StoreID returns the Replica's StoreID.
5358
func (sr *simulatorReplica) StoreID() roachpb.StoreID {
5459
return roachpb.StoreID(sr.repl.StoreID())

pkg/kv/kvserver/replica_rankings.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,8 @@ type CandidateReplica interface {
3131
// OwnsValidLease returns whether this replica is the current valid
3232
// leaseholder.
3333
OwnsValidLease(context.Context, hlc.ClockTimestamp) bool
34+
// NodeID returns the Replica's NodeID.
35+
NodeID() roachpb.NodeID
3436
// StoreID returns the Replica's StoreID.
3537
StoreID() roachpb.StoreID
3638
// GetRangeID returns the Range ID.

pkg/kv/kvserver/replicate_queue.go

Lines changed: 16 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -809,7 +809,7 @@ func (rq *replicateQueue) applyChange(
809809
case plan.AllocationFinalizeAtomicReplicationOp:
810810
err = rq.finalizeAtomicReplication(ctx, replica)
811811
case plan.AllocationTransferLeaseOp:
812-
err = rq.TransferLease(ctx, replica, op.Source.StoreID, op.Target.StoreID, op.Usage)
812+
err = rq.TransferLease(ctx, replica, op.Source, op.Target, op.Usage)
813813
case plan.AllocationChangeReplicasOp:
814814
err = rq.changeReplicas(
815815
ctx,
@@ -949,7 +949,7 @@ func (rq *replicateQueue) shedLease(
949949
rangeUsageInfo := repl.RangeUsageInfo()
950950
// Learner replicas aren't allowed to become the leaseholder or raft leader,
951951
// so only consider the `VoterDescriptors` replicas.
952-
target := rq.allocator.TransferLeaseTarget(
952+
targetDesc := rq.allocator.TransferLeaseTarget(
953953
ctx,
954954
rq.storePool,
955955
desc,
@@ -960,11 +960,18 @@ func (rq *replicateQueue) shedLease(
960960
false, /* forceDecisionWithoutStats */
961961
opts,
962962
)
963-
if target == (roachpb.ReplicaDescriptor{}) {
963+
if targetDesc == (roachpb.ReplicaDescriptor{}) {
964964
return allocator.NoSuitableTarget, nil
965965
}
966-
967-
if err := rq.TransferLease(ctx, repl, repl.store.StoreID(), target.StoreID, rangeUsageInfo); err != nil {
966+
source := roachpb.ReplicationTarget{
967+
NodeID: repl.NodeID(),
968+
StoreID: repl.StoreID(),
969+
}
970+
target := roachpb.ReplicationTarget{
971+
NodeID: targetDesc.NodeID,
972+
StoreID: targetDesc.StoreID,
973+
}
974+
if err := rq.TransferLease(ctx, repl, source, target, rangeUsageInfo); err != nil {
968975
return allocator.TransferErr, err
969976
}
970977
return allocator.TransferOK, nil
@@ -990,7 +997,7 @@ type RangeRebalancer interface {
990997
TransferLease(
991998
ctx context.Context,
992999
rlm ReplicaLeaseMover,
993-
source, target roachpb.StoreID,
1000+
source, target roachpb.ReplicationTarget,
9941001
rangeUsageInfo allocator.RangeUsageInfo,
9951002
) error
9961003

@@ -1019,16 +1026,16 @@ func (rq *replicateQueue) finalizeAtomicReplication(ctx context.Context, repl *R
10191026
func (rq *replicateQueue) TransferLease(
10201027
ctx context.Context,
10211028
rlm ReplicaLeaseMover,
1022-
source, target roachpb.StoreID,
1029+
source, target roachpb.ReplicationTarget,
10231030
rangeUsageInfo allocator.RangeUsageInfo,
10241031
) error {
10251032
rq.metrics.TransferLeaseCount.Inc(1)
10261033
log.KvDistribution.Infof(ctx, "transferring lease to s%d", target)
1027-
if err := rlm.AdminTransferLease(ctx, target, false /* bypassSafetyChecks */); err != nil {
1034+
if err := rlm.AdminTransferLease(ctx, target.StoreID, false /* bypassSafetyChecks */); err != nil {
10281035
return errors.Wrapf(err, "%s: unable to transfer lease to s%d", rlm, target)
10291036
}
10301037

1031-
rq.storePool.UpdateLocalStoresAfterLeaseTransfer(source, target, rangeUsageInfo)
1038+
rq.storePool.UpdateLocalStoresAfterLeaseTransfer(source.StoreID, target.StoreID, rangeUsageInfo)
10321039
return nil
10331040
}
10341041

pkg/kv/kvserver/store_rebalancer.go

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -581,8 +581,14 @@ func (sr *StoreRebalancer) applyLeaseRebalance(
581581
return sr.rr.TransferLease(
582582
ctx,
583583
candidateReplica,
584-
candidateReplica.StoreID(),
585-
target.StoreID,
584+
roachpb.ReplicationTarget{
585+
NodeID: candidateReplica.NodeID(),
586+
StoreID: candidateReplica.StoreID(),
587+
},
588+
roachpb.ReplicationTarget{
589+
NodeID: target.NodeID,
590+
StoreID: target.StoreID,
591+
},
586592
candidateReplica.RangeUsageInfo(),
587593
)
588594
}); err != nil {

0 commit comments

Comments
 (0)