Skip to content

Commit

Permalink
storage: add fast-path for ticking quiesced/dormant replicas
Browse files Browse the repository at this point in the history
Add fast-path for ticking quiesced/dormant replicas which avoids going
through the Raft scheduler and avoids grabbing Replica.raftMu which can
be held for significant periods of time.

Fixes #17609
  • Loading branch information
petermattis committed Aug 14, 2017
1 parent cc645f3 commit 2186866
Show file tree
Hide file tree
Showing 2 changed files with 46 additions and 14 deletions.
36 changes: 26 additions & 10 deletions pkg/storage/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -3476,21 +3476,16 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
// tick the Raft group, returning any error and true if the raft group exists
// and false otherwise.
func (r *Replica) tick() (bool, error) {
r.raftMu.Lock()
defer r.raftMu.Unlock()
return r.tickRaftMuLocked()
}

// tickRaftMuLocked requires that raftMu is held, but not replicaMu.
func (r *Replica) tickRaftMuLocked() (bool, error) {
r.mu.Lock()
defer r.mu.Unlock()

r.unreachablesMu.Lock()
remotes := r.unreachablesMu.remotes
r.unreachablesMu.remotes = nil
r.unreachablesMu.Unlock()

r.raftMu.Lock()
defer r.raftMu.Unlock()
r.mu.Lock()
defer r.mu.Unlock()

// If the raft group is uninitialized, do not initialize raft groups on
// tick.
if r.mu.internalRaftGroup == nil {
Expand Down Expand Up @@ -3548,6 +3543,27 @@ func (r *Replica) tickRaftMuLocked() (bool, error) {
return true, nil
}

// maybeTickQuiesced attempts to tick a quiesced or dormant replica, returning
// true on success and false if the regular tick path must be taken
// (i.e. Replica.tick).
func (r *Replica) maybeTickQuiesced() bool {
var done bool
r.mu.Lock()
if r.mu.internalRaftGroup == nil {
done = true
} else if r.mu.quiescent {
done = true
if tickQuiesced {
// NB: It is safe to call TickQuiesced without holding Replica.raftMu
// because that method simply increments a counter without performing any
// other logic.
r.mu.internalRaftGroup.TickQuiesced()
}
}
r.mu.Unlock()
return done
}

// maybeQuiesceLocked checks to see if the replica is quiescable and initiates
// quiescence if it is. Returns true if the replica has been quiesced and false
// otherwise.
Expand Down
24 changes: 20 additions & 4 deletions pkg/storage/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -3560,11 +3560,12 @@ func (s *Store) processRaft(ctx context.Context) {

s.stopper.RunWorker(ctx, s.raftTickLoop)
s.stopper.RunWorker(ctx, s.coalescedHeartbeatsLoop)
s.stopper.AddCloser(stop.CloserFn(func() {
s.cfg.Transport.Stop(s.StoreID())
}))
}

func (s *Store) raftTickLoop(context.Context) {
defer s.cfg.Transport.Stop(s.StoreID())

func (s *Store) raftTickLoop(ctx context.Context) {
ticker := time.NewTicker(s.cfg.RaftTickInterval)
defer ticker.Stop()

Expand All @@ -3576,7 +3577,22 @@ func (s *Store) raftTickLoop(context.Context) {
rangeIDs = rangeIDs[:0]

s.mu.replicas.Range(func(k int64, v unsafe.Pointer) bool {
rangeIDs = append(rangeIDs, roachpb.RangeID(k))
// Fast-path handling of quiesced replicas. This avoids the overhead of
// queueing the replica on the Raft scheduler. This overhead is
// significant and there is overhead to filling the Raft scheduler with
// replicas to tick. A node with 3TB of disk might contain 50k+
// replicas. Filling the Raft scheduler with all of those replicas
// every tick interval can starve other Raft processing of cycles.
//
// Why do we bother to ever queue a Replica on the Raft scheduler for
// tick processing? Couldn't we just call Replica.tick() here? Yes, but
// then a single bad/slow Replica can disrupt tick processing for every
// Replica on the store which cascades into Raft elections and more
// disruption. Replica.maybeTickQuiesced only grabs short-duration
// locks and not locks that are held during disk I/O.
if !(*Replica)(v).maybeTickQuiesced() {
rangeIDs = append(rangeIDs, roachpb.RangeID(k))
}
return true
})

Expand Down

0 comments on commit 2186866

Please sign in to comment.