Skip to content

Commit f5cac1e

Browse files
craig[bot]kvoli
andcommitted
Merge #93945
93945: asim: rework gossip component r=alextalks a=kvoli This series of commits reworks the gossip component in `asim` to more accurately reflect reality. See individual commit messages. part of #83990 Release note: None Co-authored-by: Austen McClernon <austen@cockroachlabs.com>
2 parents f27fa9d + b54947d commit f5cac1e

33 files changed

+1267
-771
lines changed

pkg/BUILD.bazel

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -185,6 +185,7 @@ ALL_TESTS = [
185185
"//pkg/kv/kvserver/allocator/allocatorimpl:allocatorimpl_test",
186186
"//pkg/kv/kvserver/allocator/storepool:storepool_test",
187187
"//pkg/kv/kvserver/apply:apply_test",
188+
"//pkg/kv/kvserver/asim/gossip:gossip_test",
188189
"//pkg/kv/kvserver/asim/op:op_test",
189190
"//pkg/kv/kvserver/asim/queue:queue_test",
190191
"//pkg/kv/kvserver/asim/state:state_test",
@@ -1157,6 +1158,8 @@ GO_TARGETS = [
11571158
"//pkg/kv/kvserver/apply:apply",
11581159
"//pkg/kv/kvserver/apply:apply_test",
11591160
"//pkg/kv/kvserver/asim/config:config",
1161+
"//pkg/kv/kvserver/asim/gossip:gossip",
1162+
"//pkg/kv/kvserver/asim/gossip:gossip_test",
11601163
"//pkg/kv/kvserver/asim/op:op",
11611164
"//pkg/kv/kvserver/asim/op:op_test",
11621165
"//pkg/kv/kvserver/asim/queue:queue",
@@ -2547,6 +2550,7 @@ GET_X_DATA_TARGETS = [
25472550
"//pkg/kv/kvserver/apply:get_x_data",
25482551
"//pkg/kv/kvserver/asim:get_x_data",
25492552
"//pkg/kv/kvserver/asim/config:get_x_data",
2553+
"//pkg/kv/kvserver/asim/gossip:get_x_data",
25502554
"//pkg/kv/kvserver/asim/op:get_x_data",
25512555
"//pkg/kv/kvserver/asim/queue:get_x_data",
25522556
"//pkg/kv/kvserver/asim/state:get_x_data",

pkg/kv/kvserver/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -80,6 +80,7 @@ go_library(
8080
"storage_engine_client.go",
8181
"store.go",
8282
"store_create_replica.go",
83+
"store_gossip.go",
8384
"store_init.go",
8485
"store_merge.go",
8586
"store_raft.go",
@@ -312,6 +313,7 @@ go_test(
312313
"split_queue_test.go",
313314
"split_trigger_helper_test.go",
314315
"stats_test.go",
316+
"store_gossip_test.go",
315317
"store_pool_test.go",
316318
"store_raft_test.go",
317319
"store_rebalancer_test.go",

pkg/kv/kvserver/asim/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,12 +12,12 @@ go_library(
1212
visibility = ["//visibility:public"],
1313
deps = [
1414
"//pkg/kv/kvserver/asim/config",
15+
"//pkg/kv/kvserver/asim/gossip",
1516
"//pkg/kv/kvserver/asim/op",
1617
"//pkg/kv/kvserver/asim/queue",
1718
"//pkg/kv/kvserver/asim/state",
1819
"//pkg/kv/kvserver/asim/storerebalancer",
1920
"//pkg/kv/kvserver/asim/workload",
20-
"//pkg/roachpb",
2121
"//pkg/util/encoding/csv",
2222
"//pkg/util/log",
2323
],

pkg/kv/kvserver/asim/asim.go

Lines changed: 10 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -15,12 +15,12 @@ import (
1515
"time"
1616

1717
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config"
18+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gossip"
1819
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/op"
1920
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/queue"
2021
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state"
2122
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/storerebalancer"
2223
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/workload"
23-
"github.com/cockroachdb/cockroach/pkg/roachpb"
2424
"github.com/cockroachdb/cockroach/pkg/util/log"
2525
)
2626

@@ -56,7 +56,7 @@ type Simulator struct {
5656

5757
state state.State
5858
changer state.Changer
59-
exchange state.Exchange
59+
gossip gossip.Gossip
6060
shuffler func(n int, swap func(i, j int))
6161

6262
metrics *MetricsTracker
@@ -68,7 +68,6 @@ func NewSimulator(
6868
interval, bgInterval time.Duration,
6969
wgs []workload.Generator,
7070
initialState state.State,
71-
exchange state.Exchange,
7271
changer state.Changer,
7372
settings *config.SimulationSettings,
7473
metrics *MetricsTracker,
@@ -138,7 +137,7 @@ func NewSimulator(
138137
controllers: controllers,
139138
srs: srs,
140139
pacers: pacers,
141-
exchange: exchange,
140+
gossip: gossip.NewGossip(initialState, settings),
142141
metrics: metrics,
143142
shuffler: state.NewShuffler(settings.Seed),
144143
}
@@ -174,17 +173,17 @@ func (s *Simulator) RunSim(ctx context.Context) {
174173
break
175174
}
176175

176+
// Update the store clocks with the current tick time.
177+
s.tickStoreClocks(tick)
178+
177179
// Update the state with generated load.
178180
s.tickWorkload(ctx, tick)
179181

180182
// Update pending state changes.
181183
s.tickStateChanges(ctx, tick)
182184

183185
// Update each allocators view of the stores in the cluster.
184-
s.tickStateExchange(tick)
185-
186-
// Update the store clocks with the current tick time.
187-
s.tickStoreClocks(tick)
186+
s.tickGossip(ctx, tick)
188187

189188
// Done with config and load updates, the state is ready for the
190189
// allocators.
@@ -235,19 +234,11 @@ func (s *Simulator) tickStateChanges(ctx context.Context, tick time.Time) {
235234
}
236235
}
237236

238-
// tickStateExchange puts the current tick store descriptors into the state
237+
// tickGossip puts the current tick store descriptors into the state
239238
// exchange. It then updates the exchanged descriptors for each store's store
240239
// pool.
241-
func (s *Simulator) tickStateExchange(tick time.Time) {
242-
if s.bgLastTick.Add(s.bgInterval).After(tick) {
243-
return
244-
}
245-
storeDescriptors := s.state.StoreDescriptors()
246-
s.exchange.Put(tick, storeDescriptors...)
247-
for _, store := range s.state.Stores() {
248-
storeID := store.StoreID()
249-
s.state.UpdateStorePool(storeID, s.exchange.Get(tick, roachpb.StoreID(storeID)))
250-
}
240+
func (s *Simulator) tickGossip(ctx context.Context, tick time.Time) {
241+
s.gossip.Tick(ctx, tick, s.state)
251242
}
252243

253244
func (s *Simulator) tickStoreClocks(tick time.Time) {

pkg/kv/kvserver/asim/asim_test.go

Lines changed: 10 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -38,11 +38,10 @@ func TestRunAllocatorSimulator(t *testing.T) {
3838
rwg := make([]workload.Generator, 1)
3939
rwg[0] = testCreateWorkloadGenerator(start, 1, 10)
4040
m := asim.NewMetricsTracker(os.Stdout)
41-
exchange := state.NewFixedDelayExhange(start, settings.StateExchangeInterval, settings.StateExchangeDelay)
4241
changer := state.NewReplicaChanger()
4342
s := state.LoadConfig(state.ComplexConfig)
4443

45-
sim := asim.NewSimulator(start, end, interval, interval, rwg, s, exchange, changer, settings, m)
44+
sim := asim.NewSimulator(start, end, interval, interval, rwg, s, changer, settings, m)
4645
sim.RunSim(ctx)
4746
}
4847

@@ -67,16 +66,6 @@ func testCreateWorkloadGenerator(start time.Time, stores int, keySpan int64) wor
6766
)
6867
}
6968

70-
// testPreGossipStores populates the state exchange with the existing state.
71-
// This is done at the time given, which should be before the test start time
72-
// minus the gossip delay and interval. This alleviates a cold start, where the
73-
// allocator for each store does not have information to make a decision for
74-
// the ranges it holds leases for.
75-
func testPreGossipStores(s state.State, exchange state.Exchange, at time.Time) {
76-
storeDescriptors := s.StoreDescriptors()
77-
exchange.Put(at, storeDescriptors...)
78-
}
79-
8069
// TestAllocatorSimulatorSpeed tests that the simulation runs at a rate of at
8170
// least 1.67 simulated minutes per wall clock second (1:100) for a 32 node
8271
// cluster, with 32000 replicas. The workload is generating 16000 keys per
@@ -96,7 +85,6 @@ func TestAllocatorSimulatorSpeed(t *testing.T) {
9685
end := start.Add(5 * time.Minute)
9786
bgInterval := 10 * time.Second
9887
interval := 2 * time.Second
99-
preGossipStart := start.Add(-settings.StateExchangeInterval - settings.StateExchangeDelay)
10088

10189
stores := 32
10290
replsPerRange := 3
@@ -112,7 +100,6 @@ func TestAllocatorSimulatorSpeed(t *testing.T) {
112100
sample := func() int64 {
113101
rwg := make([]workload.Generator, 1)
114102
rwg[0] = testCreateWorkloadGenerator(start, stores, int64(keyspace))
115-
exchange := state.NewFixedDelayExhange(preGossipStart, settings.StateExchangeInterval, settings.StateExchangeDelay)
116103
changer := state.NewReplicaChanger()
117104
m := asim.NewMetricsTracker() // no output
118105
replicaDistribution := make([]float64, stores)
@@ -129,8 +116,7 @@ func TestAllocatorSimulatorSpeed(t *testing.T) {
129116
}
130117

131118
s := state.NewTestStateReplDistribution(replicaDistribution, ranges, replsPerRange, keyspace)
132-
testPreGossipStores(s, exchange, preGossipStart)
133-
sim := asim.NewSimulator(start, end, interval, bgInterval, rwg, s, exchange, changer, settings, m)
119+
sim := asim.NewSimulator(start, end, interval, bgInterval, rwg, s, changer, settings, m)
134120

135121
startTime := timeutil.Now()
136122
sim.RunSim(ctx)
@@ -169,7 +155,6 @@ func TestAllocatorSimulatorDeterministic(t *testing.T) {
169155
end := start.Add(15 * time.Minute)
170156
bgInterval := 10 * time.Second
171157
interval := 2 * time.Second
172-
preGossipStart := start.Add(-settings.StateExchangeInterval - settings.StateExchangeDelay)
173158

174159
stores := 7
175160
replsPerRange := 3
@@ -187,7 +172,6 @@ func TestAllocatorSimulatorDeterministic(t *testing.T) {
187172
for run := 0; run < runs; run++ {
188173
rwg := make([]workload.Generator, 1)
189174
rwg[0] = testCreateWorkloadGenerator(start, stores, int64(keyspace))
190-
exchange := state.NewFixedDelayExhange(preGossipStart, settings.StateExchangeInterval, settings.StateExchangeDelay)
191175
changer := state.NewReplicaChanger()
192176
m := asim.NewMetricsTracker() // no output
193177
replicaDistribution := make([]float64, stores)
@@ -204,12 +188,17 @@ func TestAllocatorSimulatorDeterministic(t *testing.T) {
204188
}
205189

206190
s := state.NewTestStateReplDistribution(replicaDistribution, ranges, replsPerRange, keyspace)
207-
testPreGossipStores(s, exchange, preGossipStart)
208-
sim := asim.NewSimulator(start, end, interval, bgInterval, rwg, s, exchange, changer, settings, m)
191+
sim := asim.NewSimulator(start, end, interval, bgInterval, rwg, s, changer, settings, m)
209192

210193
ctx := context.Background()
211194
sim.RunSim(ctx)
212-
descs := s.StoreDescriptors()
195+
196+
storeRefs := s.Stores()
197+
storeIDs := make([]state.StoreID, len(storeRefs))
198+
for i, store := range storeRefs {
199+
storeIDs[i] = store.StoreID()
200+
}
201+
descs := s.StoreDescriptors(false /* cached */, storeIDs...)
213202

214203
if run == 0 {
215204
refRun = descs
Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,39 @@
1+
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
2+
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
3+
4+
go_library(
5+
name = "gossip",
6+
srcs = [
7+
"exchange.go",
8+
"gossip.go",
9+
],
10+
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gossip",
11+
visibility = ["//visibility:public"],
12+
deps = [
13+
"//pkg/kv/kvserver",
14+
"//pkg/kv/kvserver/allocator/storepool",
15+
"//pkg/kv/kvserver/asim/config",
16+
"//pkg/kv/kvserver/asim/state",
17+
"//pkg/roachpb",
18+
"//pkg/util/protoutil",
19+
],
20+
)
21+
22+
go_test(
23+
name = "gossip_test",
24+
srcs = [
25+
"exchange_test.go",
26+
"gossip_test.go",
27+
],
28+
args = ["-test.timeout=295s"],
29+
embed = [":gossip"],
30+
deps = [
31+
"//pkg/kv/kvserver/allocator/storepool",
32+
"//pkg/kv/kvserver/asim/config",
33+
"//pkg/kv/kvserver/asim/state",
34+
"//pkg/roachpb",
35+
"@com_github_stretchr_testify//require",
36+
],
37+
)
38+
39+
get_x_data(name = "get_x_data")
Lines changed: 64 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,64 @@
1+
// Copyright 2022 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the Business Source License
4+
// included in the file licenses/BSL.txt.
5+
//
6+
// As of the Change Date specified in that file, in accordance with
7+
// the Business Source License, use of this software will be governed
8+
// by the Apache License, Version 2.0, included in the file
9+
// licenses/APL.txt.
10+
11+
package gossip
12+
13+
import (
14+
"sort"
15+
"time"
16+
17+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool"
18+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config"
19+
"github.com/cockroachdb/cockroach/pkg/roachpb"
20+
)
21+
22+
// exchangeInfo contains the information of a gossiped store descriptor.
23+
type exchangeInfo struct {
24+
created time.Time
25+
desc roachpb.StoreDescriptor
26+
}
27+
28+
// fixedDelayExchange simulates a gossip exchange network with a symmetric
29+
// fixed delay between all connected clients.
30+
type fixedDelayExchange struct {
31+
pending []exchangeInfo
32+
settings *config.SimulationSettings
33+
}
34+
35+
// put adds the given descriptors at the current tick into the exchange
36+
// network.
37+
func (u *fixedDelayExchange) put(tick time.Time, descs ...roachpb.StoreDescriptor) {
38+
for _, desc := range descs {
39+
u.pending = append(u.pending, exchangeInfo{created: tick, desc: desc})
40+
}
41+
}
42+
43+
// updates returns back exchanged infos, wrapped as store details that have
44+
// completed between the last tick update was called and the tick given.
45+
func (u *fixedDelayExchange) updates(tick time.Time) []*storepool.StoreDetail {
46+
sort.Slice(u.pending, func(i, j int) bool { return u.pending[i].created.Before(u.pending[j].created) })
47+
ready := []*storepool.StoreDetail{}
48+
i := 0
49+
for ; i < len(u.pending) && !tick.Before(u.pending[i].created.Add(u.settings.StateExchangeDelay)); i++ {
50+
ready = append(ready, makeStoreDetail(&u.pending[i].desc, u.pending[i].created))
51+
}
52+
u.pending = u.pending[i:]
53+
return ready
54+
}
55+
56+
// makeStoreDetail wraps a store descriptor into a storepool StoreDetail at the
57+
// given tick.
58+
func makeStoreDetail(desc *roachpb.StoreDescriptor, tick time.Time) *storepool.StoreDetail {
59+
return &storepool.StoreDetail{
60+
Desc: desc,
61+
LastUpdatedTime: tick,
62+
LastAvailable: tick,
63+
}
64+
}
Lines changed: 51 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,51 @@
1+
// Copyright 2022 The Cockroach Authors.
2+
//
3+
// Use of this software is governed by the Business Source License
4+
// included in the file licenses/BSL.txt.
5+
//
6+
// As of the Change Date specified in that file, in accordance with
7+
// the Business Source License, use of this software will be governed
8+
// by the Apache License, Version 2.0, included in the file
9+
// licenses/APL.txt.
10+
11+
package gossip
12+
13+
import (
14+
"testing"
15+
16+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config"
17+
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state"
18+
"github.com/cockroachdb/cockroach/pkg/roachpb"
19+
"github.com/stretchr/testify/require"
20+
)
21+
22+
func TestFixedDelayExchange(t *testing.T) {
23+
makeStoresFn := func(stores []int32) []roachpb.StoreDescriptor {
24+
descriptors := make([]roachpb.StoreDescriptor, len(stores))
25+
for i := range stores {
26+
descriptors[i] = roachpb.StoreDescriptor{StoreID: roachpb.StoreID(stores[i])}
27+
28+
}
29+
return descriptors
30+
}
31+
32+
settings := config.DefaultSimulationSettings()
33+
tick := state.TestingStartTime()
34+
exchange := fixedDelayExchange{pending: []exchangeInfo{}, settings: settings}
35+
36+
// There should be no updates initially.
37+
require.Len(t, exchange.updates(tick), 0)
38+
39+
// Put an update at the current tick.
40+
exchange.put(tick, makeStoresFn([]int32{1, 2, 3})...)
41+
require.Len(t, exchange.pending, 3)
42+
43+
// There should be no updates until after the tick + state exchange delay.
44+
halfTick := tick.Add(settings.StateExchangeDelay / 2)
45+
require.Len(t, exchange.updates(halfTick), 0)
46+
47+
// Update the tick to be >= tick + delay, there should be three updates.
48+
tick = tick.Add(settings.StateExchangeDelay)
49+
require.Len(t, exchange.updates(tick), 3)
50+
require.Len(t, exchange.pending, 0)
51+
}

0 commit comments

Comments
 (0)