Skip to content

Commit 048b879

Browse files
committed
mmaprototypehelpers: skip mma coordination when mma is disabled
Previously, non-mma components registered external changes with mma via allocator sync even when mma was disabled. This led to logs like `did not register external changes: due to range does not exist in cluster state`. Without mma being enabled, mma can’t register external changes because it never learns about the ranges via processStoreLeaseholderMsg in the first place, and this external change would get dropped. This commit changes allocator sync to skip coordinating with the mma allocator when mma is disabled. Note that we changes still go via allocator sync even when mma is disabled to update the local store pool correctly. Epic: none Release note: none
1 parent fb90c8d commit 048b879

File tree

3 files changed

+31
-15
lines changed

3 files changed

+31
-15
lines changed

pkg/kv/kvserver/allocator/mmaprototypehelpers/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,10 +10,12 @@ go_library(
1010
visibility = ["//visibility:public"],
1111
deps = [
1212
"//pkg/kv/kvpb",
13+
"//pkg/kv/kvserver",
1314
"//pkg/kv/kvserver/allocator",
1415
"//pkg/kv/kvserver/allocator/mmaprototype",
1516
"//pkg/kv/kvserver/allocator/storepool",
1617
"//pkg/roachpb",
18+
"//pkg/settings/cluster",
1719
"//pkg/util/log",
1820
"//pkg/util/syncutil",
1921
"//pkg/util/timeutil",

pkg/kv/kvserver/allocator/mmaprototypehelpers/kvserver_mma_integration.go

Lines changed: 27 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -10,10 +10,12 @@ import (
1010
"fmt"
1111

1212
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
13+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
1314
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator"
1415
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/mmaprototype"
1516
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool"
1617
"github.com/cockroachdb/cockroach/pkg/roachpb"
18+
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1719
"github.com/cockroachdb/cockroach/pkg/util/log"
1820
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
1921
)
@@ -62,17 +64,21 @@ func (s Author) External() bool {
6264
type AllocatorSync struct {
6365
sp *storepool.StorePool
6466
mmAllocator mmaprototype.Allocator
67+
st *cluster.Settings
6568
mu struct {
6669
syncutil.Mutex
6770
changeSeqGen SyncChangeID
6871
trackedChanges map[SyncChangeID]trackedAllocatorChange
6972
}
7073
}
7174

72-
func NewAllocatorSync(sp *storepool.StorePool, mmAllocator mmaprototype.Allocator) *AllocatorSync {
75+
func NewAllocatorSync(
76+
sp *storepool.StorePool, mmAllocator mmaprototype.Allocator, st *cluster.Settings,
77+
) *AllocatorSync {
7378
as := &AllocatorSync{
7479
sp: sp,
7580
mmAllocator: mmAllocator,
81+
st: st,
7682
}
7783
as.mu.trackedChanges = make(map[SyncChangeID]trackedAllocatorChange)
7884
return as
@@ -122,9 +128,12 @@ func (as *AllocatorSync) NonMMAPreTransferLease(
122128
)
123129
log.Infof(ctx, "registering external lease transfer change: usage=%v changes=%v",
124130
usage, replicaChanges)
125-
changeIDs := as.mmAllocator.RegisterExternalChanges(replicaChanges[:])
126-
if changeIDs == nil {
127-
log.Info(ctx, "mma did not track lease transfer, skipping")
131+
var changeIDs []mmaprototype.ChangeID
132+
if kvserver.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserver.LBRebalancingMultiMetric {
133+
changeIDs = as.mmAllocator.RegisterExternalChanges(replicaChanges[:])
134+
if changeIDs == nil {
135+
log.Info(ctx, "mma did not track lease transfer, skipping")
136+
}
128137
}
129138
trackedChange := trackedAllocatorChange{
130139
typ: AllocatorChangeTypeLeaseTransfer,
@@ -210,9 +219,12 @@ func (as *AllocatorSync) NonMMAPreChangeReplicas(
210219

211220
log.Infof(ctx, "registering external replica change: chgs=%v usage=%v changes=%v",
212221
changes, usage, replicaChanges)
213-
changeIDs := as.mmAllocator.RegisterExternalChanges(replicaChanges)
214-
if changeIDs == nil {
215-
log.Info(ctx, "cluster does not have a range for the external replica change, skipping")
222+
var changeIDs []mmaprototype.ChangeID
223+
if kvserver.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserver.LBRebalancingMultiMetric {
224+
changeIDs = as.mmAllocator.RegisterExternalChanges(replicaChanges)
225+
if changeIDs == nil {
226+
log.Info(ctx, "cluster does not have a range for the external replica change, skipping")
227+
}
216228
}
217229
trackedChange := trackedAllocatorChange{
218230
typ: AllocatorChangeTypeChangeReplicas,
@@ -335,12 +347,14 @@ func (as *AllocatorSync) PostApply(ctx context.Context, syncChangeID SyncChangeI
335347
}
336348
delete(as.mu.trackedChanges, syncChangeID)
337349
}()
338-
if changeIDs := tracked.changeIDs; changeIDs != nil {
339-
log.Infof(ctx, "PostApply: tracked=%v change_ids=%v success: %v", tracked, changeIDs, success)
340-
as.updateMetrics(success, tracked.typ, tracked.author)
341-
as.mmAllocator.AdjustPendingChangesDisposition(changeIDs, success)
342-
} else {
343-
log.Infof(ctx, "PostApply: tracked=%v no change_ids success: %v", tracked, success)
350+
if kvserver.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserver.LBRebalancingMultiMetric {
351+
if changeIDs := tracked.changeIDs; changeIDs != nil {
352+
log.Infof(ctx, "PostApply: tracked=%v change_ids=%v success: %v", tracked, changeIDs, success)
353+
as.updateMetrics(success, tracked.typ, tracked.author)
354+
as.mmAllocator.AdjustPendingChangesDisposition(changeIDs, success)
355+
} else {
356+
log.Infof(ctx, "PostApply: tracked=%v no change_ids success: %v", tracked, success)
357+
}
344358
}
345359
as.updateMetrics(success, tracked.typ, tracked.author)
346360
if !success {

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -430,9 +430,9 @@ func (s *state) AddNode() Node {
430430
nodeID: nodeID,
431431
desc: roachpb.NodeDescriptor{NodeID: roachpb.NodeID(nodeID)},
432432
stores: []StoreID{},
433-
storepool: sp,
434433
mmAllocator: mmAllocator,
435-
as: mmaprototypehelpers.NewAllocatorSync(sp, mmAllocator),
434+
storepool: sp,
435+
as: mmaprototypehelpers.NewAllocatorSync(sp, mmAllocator, s.settings.ST),
436436
}
437437
s.nodes[nodeID] = node
438438
s.SetNodeLiveness(nodeID, livenesspb.NodeLivenessStatus_LIVE)

0 commit comments

Comments
 (0)