Skip to content

Commit 1e2f0e2

Browse files
craig[bot]andyyang890wenyihu6
committed
150855: span: add MultiFrontier data structure r=aerfrei,asg0451 a=andyyang890 Informs #148110 --- **span: make frontierHeap type-safe** This patch makes `frontierHeap` type-safe by replacing the usage of the go standard library heap package with the cockroach copy of the heap package that uses generics. Release note: None --- **heaputil: add package with helper functions for working with heaps** This patch adds a new `heaputil` containing helper functions for working with heaps. As of now, it only contains a single `Valid` function which returns whether a heap is a valid heap based on the min-heap invariant defined by `heap.Interface`. The reasoning for creating a new package instead of reusing the existing `heap` package is that that package is intended to be a type-safe copy of the standard library package with the same name. Release note: None --- **span: add MultiFrontier data structure** This patch adds a new `MultiFrontier` data structure that can be used when we want a frontier that's partitioned into sub-frontiers under the hood based on a custom partitioner function. Release note: None 151166: kvserver: move LoadBasedRebalancingMode to kvserverbase r=sumeerbhola a=wenyihu6 Previously, the cluster setting LoadBasedRebalancingMode was in the kvserver package. This would create an import cycle when future commits introduce allocator sync. Allocator sync (in the mmaprototypehelpers package) needs to check if mma is enabled by accessing this setting in kvserver, while kvserver depends on mmaprototypehelpers to initialize allocator sync on every node. To resolve this, this commit moves the setting out of kvserver to kvserverbase, breaking the dependency cycle. This also improves clarity, as the setting fits better in kvserverbase alongside other cluster settings, rather than in the already large kvserver package. Epic: none Release note: none Co-authored-by: Andy Yang <[email protected]> Co-authored-by: wenyihu6 <[email protected]>
3 parents 9f9a049 + 594289b + bbd2bb4 commit 1e2f0e2

28 files changed

+966
-87
lines changed

pkg/BUILD.bazel

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -727,6 +727,7 @@ ALL_TESTS = [
727727
"//pkg/util/cloudinfo:cloudinfo_test",
728728
"//pkg/util/collatedstring:collatedstring_test",
729729
"//pkg/util/container/heap:heap_test",
730+
"//pkg/util/container/heaputil:heaputil_test",
730731
"//pkg/util/container/list:list_test",
731732
"//pkg/util/container/ring:ring_test",
732733
"//pkg/util/ctxgroup:ctxgroup_test",
@@ -2572,6 +2573,8 @@ GO_TARGETS = [
25722573
"//pkg/util/collatedstring:collatedstring_test",
25732574
"//pkg/util/container/heap:heap",
25742575
"//pkg/util/container/heap:heap_test",
2576+
"//pkg/util/container/heaputil:heaputil",
2577+
"//pkg/util/container/heaputil:heaputil_test",
25752578
"//pkg/util/container/list:list",
25762579
"//pkg/util/container/list:list_test",
25772580
"//pkg/util/container/ring:ring",

pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -844,7 +844,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
844844

845845
// Further down, we'll set up the test to pin the lease to store 1. Turn off
846846
// load based rebalancing to make sure it doesn't move.
847-
kvserver.LoadBasedRebalancingMode.Override(ctx, &settings.SV, kvserver.LBRebalancingOff)
847+
kvserverbase.LoadBasedRebalancingMode.Override(ctx, &settings.SV, kvserverbase.LBRebalancingOff)
848848

849849
n1 := sqlutils.MakeSQLRunner(tc.Conns[0])
850850
n1.Exec(t, `CREATE DATABASE t`)

pkg/kv/kvserver/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -223,7 +223,6 @@ go_library(
223223
"//pkg/util/encoding",
224224
"//pkg/util/envutil",
225225
"//pkg/util/errorutil",
226-
"//pkg/util/errorutil/unimplemented",
227226
"//pkg/util/growstack",
228227
"//pkg/util/grpcutil",
229228
"//pkg/util/grunning",

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,10 +10,10 @@ go_library(
1010
visibility = ["//visibility:public"],
1111
deps = [
1212
"//pkg/kv/kvpb",
13-
"//pkg/kv/kvserver",
1413
"//pkg/kv/kvserver/allocator",
1514
"//pkg/kv/kvserver/allocator/mmaprototype",
1615
"//pkg/kv/kvserver/allocator/storepool",
16+
"//pkg/kv/kvserver/kvserverbase",
1717
"//pkg/roachpb",
1818
"//pkg/settings/cluster",
1919
"//pkg/util/log",

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

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

1212
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
13-
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
1413
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator"
1514
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/mmaprototype"
1615
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool"
16+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
1717
"github.com/cockroachdb/cockroach/pkg/roachpb"
1818
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1919
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -129,7 +129,7 @@ func (as *AllocatorSync) NonMMAPreTransferLease(
129129
log.Infof(ctx, "registering external lease transfer change: usage=%v changes=%v",
130130
usage, replicaChanges)
131131
var changeIDs []mmaprototype.ChangeID
132-
if kvserver.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserver.LBRebalancingMultiMetric {
132+
if kvserverbase.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserverbase.LBRebalancingMultiMetric {
133133
changeIDs = as.mmAllocator.RegisterExternalChanges(replicaChanges[:])
134134
if changeIDs == nil {
135135
log.Info(ctx, "mma did not track lease transfer, skipping")
@@ -220,7 +220,7 @@ func (as *AllocatorSync) NonMMAPreChangeReplicas(
220220
log.Infof(ctx, "registering external replica change: chgs=%v usage=%v changes=%v",
221221
changes, usage, replicaChanges)
222222
var changeIDs []mmaprototype.ChangeID
223-
if kvserver.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserver.LBRebalancingMultiMetric {
223+
if kvserverbase.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserverbase.LBRebalancingMultiMetric {
224224
changeIDs = as.mmAllocator.RegisterExternalChanges(replicaChanges)
225225
if changeIDs == nil {
226226
log.Info(ctx, "cluster does not have a range for the external replica change, skipping")
@@ -347,7 +347,7 @@ func (as *AllocatorSync) PostApply(ctx context.Context, syncChangeID SyncChangeI
347347
}
348348
delete(as.mu.trackedChanges, syncChangeID)
349349
}()
350-
if kvserver.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserver.LBRebalancingMultiMetric {
350+
if kvserverbase.LoadBasedRebalancingMode.Get(&as.st.SV) == kvserverbase.LBRebalancingMultiMetric {
351351
if changeIDs := tracked.changeIDs; changeIDs != nil {
352352
log.Infof(ctx, "PostApply: tracked=%v change_ids=%v success: %v", tracked, changeIDs, success)
353353
as.updateMetrics(success, tracked.typ, tracked.author)

pkg/kv/kvserver/asim/mmaintegration/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -9,13 +9,13 @@ go_library(
99
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/mmaintegration",
1010
visibility = ["//visibility:public"],
1111
deps = [
12-
"//pkg/kv/kvserver",
1312
"//pkg/kv/kvserver/allocator",
1413
"//pkg/kv/kvserver/allocator/mmaprototype",
1514
"//pkg/kv/kvserver/allocator/mmaprototypehelpers",
1615
"//pkg/kv/kvserver/asim/config",
1716
"//pkg/kv/kvserver/asim/op",
1817
"//pkg/kv/kvserver/asim/state",
18+
"//pkg/kv/kvserver/kvserverbase",
1919
"//pkg/roachpb",
2020
"//pkg/util/log",
2121
"@com_github_cockroachdb_logtags//:logtags",

pkg/kv/kvserver/asim/mmaintegration/mma_store_rebalancer.go

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

13-
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
1413
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator"
1514
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/mmaprototype"
1615
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/mmaprototypehelpers"
1716
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config"
1817
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/op"
1918
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state"
19+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
2020
"github.com/cockroachdb/cockroach/pkg/roachpb"
2121
"github.com/cockroachdb/cockroach/pkg/util/log"
2222
"github.com/cockroachdb/logtags"
@@ -97,7 +97,7 @@ func (msr *MMAStoreRebalancer) Tick(ctx context.Context, tick time.Time, s state
9797
return
9898
}
9999

100-
if kvserver.LoadBasedRebalancingMode.Get(&msr.settings.ST.SV) != kvserver.LBRebalancingMultiMetric {
100+
if kvserverbase.LoadBasedRebalancingMode.Get(&msr.settings.ST.SV) != kvserverbase.LBRebalancingMultiMetric {
101101
// When the store rebalancer isn't set to use the multi-metric mode, the
102102
// legacy store rebalancer is used.
103103
return

pkg/kv/kvserver/asim/state/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ go_library(
3030
"//pkg/kv/kvserver/allocator/storepool",
3131
"//pkg/kv/kvserver/asim/config",
3232
"//pkg/kv/kvserver/asim/workload",
33+
"//pkg/kv/kvserver/kvserverbase",
3334
"//pkg/kv/kvserver/liveness/livenesspb",
3435
"//pkg/kv/kvserver/load",
3536
"//pkg/kv/kvserver/split",

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import (
2727
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool"
2828
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config"
2929
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/workload"
30+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
3031
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb"
3132
"github.com/cockroachdb/cockroach/pkg/raft"
3233
"github.com/cockroachdb/cockroach/pkg/raft/raftpb"
@@ -1397,7 +1398,7 @@ func (s *state) RegisterConfigChangeListener(listener ConfigChangeListener) {
13971398
func (s *state) SetClusterSetting(Key string, Value interface{}) {
13981399
switch Key {
13991400
case "LBRebalancingMode":
1400-
kvserver.LoadBasedRebalancingMode.Override(context.Background(), &s.settings.ST.SV, kvserver.LBRebalancingMode(Value.(int64)))
1401+
kvserverbase.LoadBasedRebalancingMode.Override(context.Background(), &s.settings.ST.SV, kvserverbase.LBRebalancingMode(Value.(int64)))
14011402
default:
14021403
panic("other cluster settings not supported")
14031404
}

pkg/kv/kvserver/asim/storerebalancer/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ go_library(
2020
"//pkg/kv/kvserver/asim/op",
2121
"//pkg/kv/kvserver/asim/state",
2222
"//pkg/kv/kvserver/kvflowcontrol/rac2",
23+
"//pkg/kv/kvserver/kvserverbase",
2324
"//pkg/raft",
2425
"//pkg/roachpb",
2526
"//pkg/util/hlc",

0 commit comments

Comments
 (0)