Skip to content

Commit

Permalink
kvserver: cleanup store.go to remove functions exposed because cfg wa…
Browse files Browse the repository at this point in the history
…s private

A lot of tests required things from StoreConfig, which was previously
private on the Store. Now that #59059 exposed the StoreConfig,
we can clean up some of this and remove the extra getters added
to extract things from the config for example NodeLiveness, Raft settings etc.

Release note: None
  • Loading branch information
lunevalex committed Jan 21, 2021
1 parent 3b6a17d commit e4c8933
Show file tree
Hide file tree
Showing 4 changed files with 9 additions and 29 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ func TestGossipNodeLivenessOnLeaseChange(t *testing.T) {
// liveness are not triggering gossiping.
for _, s := range tc.Servers {
pErr := s.Stores().VisitStores(func(store *kvserver.Store) error {
store.NodeLiveness().PauseHeartbeatLoopForTest()
store.GetStoreConfig().NodeLiveness.PauseHeartbeatLoopForTest()
return nil
})
if pErr != nil {
Expand Down
10 changes: 5 additions & 5 deletions pkg/kv/kvserver/client_raft_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1152,7 +1152,7 @@ func TestRequestsOnLaggingReplica(t *testing.T) {

// Stop the heartbeats so that n1's lease can expire.
log.Infof(ctx, "test: suspending heartbeats for n1")
resumeN1Heartbeats := partitionStore.NodeLiveness().PauseAllHeartbeatsForTest()
resumeN1Heartbeats := partitionStore.GetStoreConfig().NodeLiveness.PauseAllHeartbeatsForTest()

// Wait until another replica campaigns and becomes leader, replacing the
// partitioned one.
Expand Down Expand Up @@ -2455,7 +2455,7 @@ func TestRaftHeartbeats(t *testing.T) {
store := tc.GetFirstStoreFromServer(t, int(leaderRepl.StoreID()-1))

// Wait for several ticks to elapse.
ticksToWait := 2 * store.RaftElectionTimeoutTicks()
ticksToWait := 2 * store.GetStoreConfig().RaftElectionTimeoutTicks
ticks := store.Metrics().RaftTicks.Count
for targetTicks := ticks() + int64(ticksToWait); ticks() < targetTicks; {
time.Sleep(time.Millisecond)
Expand Down Expand Up @@ -2508,7 +2508,7 @@ func TestReportUnreachableHeartbeats(t *testing.T) {
t.Fatal(err)
}

ticksToWait := 2 * leaderStore.RaftElectionTimeoutTicks()
ticksToWait := 2 * leaderStore.GetStoreConfig().RaftElectionTimeoutTicks
ticks := leaderStore.Metrics().RaftTicks.Count
for targetTicks := ticks() + int64(ticksToWait); ticks() < targetTicks; {
time.Sleep(time.Millisecond)
Expand Down Expand Up @@ -2563,7 +2563,7 @@ outer:
tc.RemoveVotersOrFatal(t, key, tc.Target(leaderIdx))
cb := tc.Servers[replicaIdx].RaftTransport().GetCircuitBreaker(tc.Target(replicaIdx).NodeID, rpc.DefaultClass)
cb.Break()
time.Sleep(tc.GetFirstStoreFromServer(t, replicaIdx).CoalescedHeartbeatsInterval())
time.Sleep(tc.GetFirstStoreFromServer(t, replicaIdx).GetStoreConfig().CoalescedHeartbeatsInterval)
cb.Reset()
tc.AddVotersOrFatal(t, key, tc.Target(leaderIdx))
continue outer
Expand Down Expand Up @@ -4063,7 +4063,7 @@ func TestRangeQuiescence(t *testing.T) {
// Wait for a bunch of ticks to occur which will allow the follower time to
// campaign.
ticks := tc.GetFirstStoreFromServer(t, followerIdx).Metrics().RaftTicks.Count
for targetTicks := ticks() + int64(2*tc.GetFirstStoreFromServer(t, followerIdx).RaftElectionTimeoutTicks()); ticks() < targetTicks; {
for targetTicks := ticks() + int64(2*tc.GetFirstStoreFromServer(t, followerIdx).GetStoreConfig().RaftElectionTimeoutTicks); ticks() < targetTicks; {
time.Sleep(time.Millisecond)
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/closed_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -691,10 +691,10 @@ func forceLeaseTransferOnSubsumedRange(
}
return nil
})
restartHeartbeats := oldLeaseholderStore.NodeLiveness().PauseAllHeartbeatsForTest()
restartHeartbeats := oldLeaseholderStore.GetStoreConfig().NodeLiveness.PauseAllHeartbeatsForTest()
defer restartHeartbeats()
log.Infof(ctx, "test: paused RHS rightLeaseholder's liveness heartbeats")
time.Sleep(oldLeaseholderStore.NodeLiveness().GetLivenessThreshold())
time.Sleep(oldLeaseholderStore.GetStoreConfig().NodeLiveness.GetLivenessThreshold())

// Send a read request from one of the followers of RHS so that it notices
// that the current rightLeaseholder has stopped heartbeating. This will prompt
Expand Down Expand Up @@ -954,7 +954,7 @@ func getCurrentMaxClosed(
attempts := 0
for attempts == 0 || timeutil.Now().Before(deadline) {
attempts++
store.ClosedTimestamp().Storage.VisitDescending(target.NodeID, func(entry ctpb.Entry) (done bool) {
store.GetStoreConfig().ClosedTimestamp.Storage.VisitDescending(target.NodeID, func(entry ctpb.Entry) (done bool) {
if _, ok := entry.MLAI[desc.RangeID]; ok {
maxClosed = entry
return true
Expand Down
20 changes: 0 additions & 20 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -806,16 +806,6 @@ func (sc *StoreConfig) LeaseExpiration() int64 {
return 2 * (sc.RangeLeaseActiveDuration() + maxOffset).Nanoseconds()
}

// RaftElectionTimeoutTicks exposed for testing.
func (s *Store) RaftElectionTimeoutTicks() int {
return s.cfg.RaftElectionTimeoutTicks
}

// CoalescedHeartbeatsInterval exposed for testing.
func (s *Store) CoalescedHeartbeatsInterval() time.Duration {
return s.cfg.CoalescedHeartbeatsInterval
}

// NewStore returns a new instance of a store.
func NewStore(
ctx context.Context, cfg StoreConfig, eng storage.Engine, nodeDesc *roachpb.NodeDescriptor,
Expand Down Expand Up @@ -2330,16 +2320,6 @@ func (s *Store) Stopper() *stop.Stopper { return s.stopper }
// TestingKnobs accessor.
func (s *Store) TestingKnobs() *StoreTestingKnobs { return &s.cfg.TestingKnobs }

// ClosedTimestamp accessor.
func (s *Store) ClosedTimestamp() *container.Container {
return s.cfg.ClosedTimestamp
}

// NodeLiveness accessor.
func (s *Store) NodeLiveness() *liveness.NodeLiveness {
return s.cfg.NodeLiveness
}

// IsDraining accessor.
func (s *Store) IsDraining() bool {
return s.draining.Load().(bool)
Expand Down

0 comments on commit e4c8933

Please sign in to comment.