Skip to content

Commit 0c343c6

Browse files
craig[bot]wenyihu6
andcommitted
Merge #151351
151351: kvserver: remove `ApplyImpact` from `AllocationOp` r=sumeerbhola a=wenyihu6 Epic: CRDB-25222 Release note: none --- **kvserver: update local store pool directly with AllocationChangeReplicasOp** Previously, `change.Op.ApplyImpact` was used to update the replicateQueue's local store pool state after a successful change replica call to reflect the result of applying the operation. But in practice only `AllocationChangeReplicasOp` used it - other ops update the store pool directly. This commit updates `AllocationChangeReplicasOp` to also update the local store pool directly after a successful `repl.changeReplicasImpl`. In a follow-up, we'll remove `ApplyImpact` from the `AllocationOp` interface entirely, since it adds an unnecessary layer of indirection without providing much value. --- **kvserver: remove `ApplyImpact` from `AllocationOp`** This commit removes `ApplyImpact` from the `AllocationOp` interface entirely, since it adds an unnecessary layer of indirection without providing much value. Co-authored-by: wenyihu6 <[email protected]>
2 parents 0e4fdea + b9a794e commit 0c343c6

File tree

3 files changed

+14
-35
lines changed

3 files changed

+14
-35
lines changed

pkg/kv/kvserver/allocator/plan/op.go

Lines changed: 2 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,6 @@ package plan
88
import (
99
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
1010
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator"
11-
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool"
1211
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
1312
"github.com/cockroachdb/cockroach/pkg/roachpb"
1413
)
@@ -18,9 +17,6 @@ import (
1817
//
1918
// TODO(kvoli): Add AllocationRelocateRangeOp.
2019
type AllocationOp interface {
21-
// ApplyImpact updates the given storepool to reflect the result of
22-
// applying this operation.
23-
ApplyImpact(storepool storepool.AllocatorStorePool)
2420
// LHBeingRemoved returns true when the leaseholder is will be removed if
2521
// this operation succeeds, otherwise false.
2622
LHBeingRemoved() bool
@@ -43,12 +39,6 @@ func (o AllocationTransferLeaseOp) LHBeingRemoved() bool {
4339
return true
4440
}
4541

46-
func (o AllocationTransferLeaseOp) ApplyImpact(storepool storepool.AllocatorStorePool) {
47-
// TODO(kvoli): Currently the local storepool is updated directly in the
48-
// lease transfer call, rather than in this function. Move the storepool
49-
// tracking from rq.TransferLease to this function once #89771 is merged.
50-
}
51-
5242
// AllocationChangeReplicasOp represents an operation to execute a change
5343
// replicas txn.
5444
type AllocationChangeReplicasOp struct {
@@ -73,14 +63,6 @@ func (o AllocationChangeReplicasOp) LHBeingRemoved() bool {
7363
return false
7464
}
7565

76-
// applyEstimatedImpact updates the given storepool to reflect the result
77-
// of applying this operation.
78-
func (o AllocationChangeReplicasOp) ApplyImpact(storepool storepool.AllocatorStorePool) {
79-
for _, chg := range o.Chgs {
80-
storepool.UpdateLocalStoreAfterRebalance(chg.Target.StoreID, o.Usage, chg.ChangeType)
81-
}
82-
}
83-
8466
// AllocationFinalizeAtomicReplicationOp represents an operation to finalize an
8567
// atomic change replicas operation and remove any remaining learners.
8668
type AllocationFinalizeAtomicReplicationOp struct{}
@@ -89,13 +71,11 @@ var _ AllocationOp = &AllocationFinalizeAtomicReplicationOp{}
8971

9072
// TODO(kvoli): This always returns false, however it is possible that the LH
9173
// may have been removed here.
92-
func (o AllocationFinalizeAtomicReplicationOp) LHBeingRemoved() bool { return false }
93-
func (o AllocationFinalizeAtomicReplicationOp) ApplyImpact(storepool storepool.AllocatorStorePool) {}
74+
func (o AllocationFinalizeAtomicReplicationOp) LHBeingRemoved() bool { return false }
9475

9576
// AllocationNoop represents no operation.
9677
type AllocationNoop struct{}
9778

9879
var _ AllocationOp = &AllocationNoop{}
9980

100-
func (o AllocationNoop) LHBeingRemoved() bool { return false }
101-
func (o AllocationNoop) ApplyImpact(storepool storepool.AllocatorStorePool) {}
81+
func (o AllocationNoop) LHBeingRemoved() bool { return false }

pkg/kv/kvserver/lease_queue.go

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -140,12 +140,6 @@ func (lq *leaseQueue) process(
140140
return false, errors.Wrapf(err, "%s: unable to transfer lease to s%d", repl, transferOp.Target)
141141
}
142142

143-
// TODO(wenyihu6): Initially, change.Op.ApplyImpact was used here. This was
144-
// a problem since AllocationTransferLeaseOp.ApplyImpact was left
145-
// unimplemented. We should either implement
146-
// AllocationTransferLeaseOp.ApplyImpact correctly or remove the use of
147-
// ApplyImpact entirely. The replicate queue does not have this issue since
148-
// it uses rq.TransferLease, which updates the local store pool directly.
149143
lq.storePool.UpdateLocalStoresAfterLeaseTransfer(
150144
transferOp.Source.StoreID, transferOp.Target.StoreID, transferOp.Usage)
151145
}

pkg/kv/kvserver/replicate_queue.go

Lines changed: 12 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -816,6 +816,7 @@ func (rq *replicateQueue) applyChange(
816816
replica,
817817
op.Chgs,
818818
replica.Desc(),
819+
op.Usage,
819820
op.AllocatorPriority,
820821
op.Reason,
821822
op.Details,
@@ -914,10 +915,6 @@ func (rq *replicateQueue) processOneChange(
914915
return false, maybeAnnotateDecommissionErr(err, change.Action)
915916
}
916917

917-
// Update the local storepool state to reflect the successful application
918-
// of the change.
919-
change.Op.ApplyImpact(rq.storePool)
920-
921918
// Requeue the replica if it meets the criteria in ShouldRequeue.
922919
return ShouldRequeue(ctx, change, conf), nil
923920
}
@@ -1060,18 +1057,26 @@ func (rq *replicateQueue) changeReplicas(
10601057
repl *Replica,
10611058
chgs kvpb.ReplicationChanges,
10621059
desc *roachpb.RangeDescriptor,
1060+
rangeUsageInfo allocator.RangeUsageInfo,
10631061
allocatorPriority float64,
10641062
reason kvserverpb.RangeLogEventReason,
10651063
details string,
10661064
) error {
10671065
// NB: this calls the impl rather than ChangeReplicas because
10681066
// the latter traps tests that try to call it while the replication
10691067
// queue is active.
1070-
_, err := repl.changeReplicasImpl(
1068+
if _, err := repl.changeReplicasImpl(
10711069
ctx, desc, kvserverpb.SnapshotRequest_REPLICATE_QUEUE, allocatorPriority, reason,
10721070
details, chgs,
1073-
)
1074-
return err
1071+
); err != nil {
1072+
return err
1073+
}
1074+
// On success, update local store pool to reflect the result of applying the
1075+
// operations.
1076+
for _, chg := range chgs {
1077+
rq.storePool.UpdateLocalStoreAfterRebalance(chg.Target.StoreID, rangeUsageInfo, chg.ChangeType)
1078+
}
1079+
return nil
10751080
}
10761081

10771082
func (*replicateQueue) postProcessScheduled(

0 commit comments

Comments
 (0)