-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
stopper.go
630 lines (558 loc) · 18.5 KB
/
stopper.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
// Copyright 2014 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package stop
import (
"context"
"fmt"
"net/http"
"runtime/debug"
"sync"
"sync/atomic"
"testing"
"time"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
)
func init() {
leaktest.PrintLeakedStoppers = PrintLeakedStoppers
}
// ErrThrottled is returned from RunAsyncTaskEx in the event that there
// is no more capacity for async tasks, as limited by the semaphore.
var ErrThrottled = errors.New("throttled on async limiting semaphore")
// ErrUnavailable indicates that the server is quiescing and is unable to
// process new work.
var ErrUnavailable = &roachpb.NodeUnavailableError{}
func register(s *Stopper) {
trackedStoppers.Lock()
trackedStoppers.stoppers = append(trackedStoppers.stoppers,
stopperWithStack{s: s, createdAt: string(debug.Stack())})
trackedStoppers.Unlock()
}
func unregister(s *Stopper) {
trackedStoppers.Lock()
defer trackedStoppers.Unlock()
sl := trackedStoppers.stoppers
for i, tracked := range sl {
if tracked.s == s {
trackedStoppers.stoppers = append(sl[:i], sl[i+1:]...)
return
}
}
panic("attempt to unregister untracked stopper")
}
type stopperWithStack struct {
s *Stopper
createdAt string // stack from NewStopper()
}
var trackedStoppers struct {
syncutil.Mutex
stoppers []stopperWithStack
}
// HandleDebug responds with the list of stopper tasks actively running.
func HandleDebug(w http.ResponseWriter, r *http.Request) {
w.Header().Set("Content-Type", "text/plain; charset=utf-8")
trackedStoppers.Lock()
defer trackedStoppers.Unlock()
for _, ss := range trackedStoppers.stoppers {
s := ss.s
fmt.Fprintf(w, "%p: %d tasks\n", s, s.NumTasks())
}
}
// PrintLeakedStoppers prints (using `t`) the creation site of each Stopper
// for which `.Stop()` has not yet been called.
func PrintLeakedStoppers(t testing.TB) {
trackedStoppers.Lock()
defer trackedStoppers.Unlock()
for _, tracked := range trackedStoppers.stoppers {
t.Errorf("leaked stopper, created at:\n%s", tracked.createdAt)
}
}
// Closer is an interface for objects to attach to the stopper to
// be closed once the stopper completes.
type Closer interface {
Close()
}
// CloserFn is type that allows any function to be a Closer.
type CloserFn func()
// Close implements the Closer interface.
func (f CloserFn) Close() {
f()
}
// A Stopper provides control over the lifecycle of goroutines started
// through it via its RunTask, RunAsyncTask, and other similar methods.
//
// When Stop is invoked, the Stopper
//
// - it invokes Quiesce, which causes the Stopper to refuse new work
// (that is, its Run* family of methods starts returning ErrUnavailable),
// closes the channel returned by ShouldQuiesce, and blocks until
// until no more tasks are tracked, then
// - it runs all of the methods supplied to AddCloser, then
// - closes the IsStopped channel.
//
// When ErrUnavailable is returned from a task, the caller needs
// to handle it appropriately by terminating any work that it had
// hoped to defer to the task (which is guaranteed to never have been
// invoked). A simple example of this can be seen in the below snippet:
//
// var wg sync.WaitGroup
// wg.Add(1)
// if err := s.RunAsyncTask("foo", func(ctx context.Context) {
// defer wg.Done()
// }); err != nil {
// // Task never ran.
// wg.Done()
// }
//
// To ensure that tasks that do get started are sensitive to Quiesce,
// they need to observe the ShouldQuiesce channel similar to how they
// are expected to observe context cancellation:
//
// func x() {
// select {
// case <-s.ShouldQuiesce:
// return
// case <-ctx.Done():
// return
// case <-someChan:
// // Do work.
// }
// }
//
// TODO(tbg): many improvements here are possible:
// - propagate quiescing via context cancellation
// - better API around refused tasks
// - all the other things mentioned in:
// https://github.com/cockroachdb/cockroach/issues/58164
type Stopper struct {
quiescer chan struct{} // Closed when quiescing
stopped chan struct{} // Closed when stopped completely
onPanic func(interface{}) // called with recover() on panic on any goroutine
tracer *tracing.Tracer // tracer used to create spans for tasks
mu struct {
syncutil.RWMutex
// _numTasks is the number of active tasks. It is incremented atomically via
// addTask() under the read lock for task acquisition. We need the read lock
// to ensure task creation is prohibited atomically with the quiescing or
// stopping bools set below. When simply reading or decrementing the number
// of tasks, the lock is not necessary.
_numTasks int32
// quiescing and stopping are set in Quiesce and Stop (which calls
// Quiesce). When either is set, no new tasks are allowed and closers
// should execute immediately.
quiescing, stopping bool
closers []Closer
// idAlloc is incremented atomically under the read lock when adding a
// context to be canceled.
idAlloc int64 // allocates index into qCancels
qCancels sync.Map
}
}
// An Option can be passed to NewStopper.
type Option interface {
apply(*Stopper)
}
type optionPanicHandler func(interface{})
var _ Option = optionPanicHandler(nil)
func (oph optionPanicHandler) apply(stopper *Stopper) {
stopper.onPanic = oph
}
// OnPanic is an option which lets the Stopper recover from all panics using
// the provided panic handler.
//
// When Stop() is invoked during stack unwinding, OnPanic is also invoked, but
// Stop() may not have carried out its duties.
func OnPanic(handler func(interface{})) Option {
return optionPanicHandler(handler)
}
type withTracer struct {
tr *tracing.Tracer
}
var _ Option = withTracer{}
func (o withTracer) apply(stopper *Stopper) {
stopper.tracer = o.tr
}
// WithTracer is an option for NewStopper() supplying the Tracer to use for
// creating spans for tasks. Note that for tasks asking for a child span, the
// parent's tracer is used instead of this one.
func WithTracer(t *tracing.Tracer) Option {
return withTracer{t}
}
// NewStopper returns an instance of Stopper.
func NewStopper(options ...Option) *Stopper {
s := &Stopper{
quiescer: make(chan struct{}),
stopped: make(chan struct{}),
}
for _, opt := range options {
opt.apply(s)
}
if s.tracer == nil {
s.tracer = tracing.NewTracer()
}
register(s)
return s
}
// Recover is used internally by Stopper to provide a hook for recovery of
// panics on goroutines started by the Stopper. It can also be invoked
// explicitly (via "defer s.Recover()") on goroutines that are created outside
// of Stopper.
func (s *Stopper) Recover(ctx context.Context) {
if r := recover(); r != nil {
if s.onPanic != nil {
s.onPanic(r)
return
}
logcrash.ReportPanicWithGlobalSettings(ctx, r, 1)
panic(r)
}
}
func (s *Stopper) addTask(delta int32) (updated int32) {
return atomic.AddInt32(&s.mu._numTasks, delta)
}
// refuseRLocked returns true if the stopper refuses new tasks. This
// means that the stopper is either quiescing or stopping.
func (s *Stopper) refuseRLocked() bool {
return s.mu.stopping || s.mu.quiescing
}
// AddCloser adds an object to close after the stopper has been stopped.
//
// WARNING: memory resources acquired by this method will stay around for
// the lifetime of the Stopper. Use with care to avoid leaking memory.
//
// A closer that is added after Stop has already been called will be
// called immediately.
func (s *Stopper) AddCloser(c Closer) {
s.mu.Lock()
defer s.mu.Unlock()
if s.refuseRLocked() {
c.Close()
return
}
s.mu.closers = append(s.mu.closers, c)
}
// WithCancelOnQuiesce returns a child context which is canceled when the
// returned cancel function is called or when the Stopper begins to quiesce,
// whichever happens first.
//
// Canceling this context releases resources associated with it, so code should
// call cancel as soon as the operations running in this Context complete.
func (s *Stopper) WithCancelOnQuiesce(ctx context.Context) (context.Context, func()) {
var cancel func()
ctx, cancel = context.WithCancel(ctx)
s.mu.RLock()
defer s.mu.RUnlock()
if s.refuseRLocked() {
cancel()
return ctx, func() {}
}
id := atomic.AddInt64(&s.mu.idAlloc, 1)
s.mu.qCancels.Store(id, cancel)
return ctx, func() {
cancel()
s.mu.qCancels.Delete(id)
}
}
// RunTask adds one to the count of tasks left to quiesce in the system.
// Any worker which is a "first mover" when starting tasks must call this method
// before starting work on a new task. First movers include goroutines launched
// to do periodic work and the kv/db.go gateway which accepts external client
// requests.
//
// taskName is used as the "operation" field of the span opened for this task
// and is visible in traces. It's also part of reports printed by stoppers
// waiting to stop. The convention is
// <package name>.<struct name>: <succinct description of the task's action>
//
// Returns an error to indicate that the system is currently quiescing and
// function f was not called.
func (s *Stopper) RunTask(ctx context.Context, taskName string, f func(context.Context)) error {
if !s.runPrelude() {
return ErrUnavailable
}
// Call f.
defer s.Recover(ctx)
defer s.runPostlude()
f(ctx)
return nil
}
// RunTaskWithErr is like RunTask(), but takes in a callback that can return an
// error. The error is returned to the caller.
func (s *Stopper) RunTaskWithErr(
ctx context.Context, taskName string, f func(context.Context) error,
) error {
if !s.runPrelude() {
return ErrUnavailable
}
// Call f.
defer s.Recover(ctx)
defer s.runPostlude()
return f(ctx)
}
// RunAsyncTask is like RunTask, except the callback is run in a goroutine. The
// method doesn't block for the callback to finish execution.
//
// See also RunAsyncTaskEx for a version with more options.
func (s *Stopper) RunAsyncTask(
ctx context.Context, taskName string, f func(context.Context),
) error {
return s.RunAsyncTaskEx(ctx,
TaskOpts{
TaskName: taskName,
SpanOpt: FollowsFromSpan,
Sem: nil,
WaitForSem: false,
},
f)
}
// SpanOption specifies the type of tracing span that a task will run in.
type SpanOption int
const (
// FollowsFromSpan makes the task run in a span that's not included in the
// caller's recording (if any). For external tracers, the task's span will
// still reference the caller's span through a FollowsFrom relationship. If
// the caller doesn't have a span, then the task will execute in a root span.
//
// Use this when the caller will not wait for the task to finish, but a
// relationship between the caller and the task might still be useful to
// visualize in a trace collector.
FollowsFromSpan SpanOption = iota
// ChildSpan makes the task run in a span that's a child of the caller's span
// (if any). The child is included in the parent's recording. For external
// tracers, the child references the parent through a ChildOf relationship.
// If the caller doesn't have a span, then the task will execute in a root
// span.
//
// ChildSpan has consequences on memory usage: the memory lifetime of
// the task's span becomes tied to the lifetime of the parent. Generally
// ChildSpan should be used when the parent usually waits for the task to
// complete, and the parent is not a long-running process.
ChildSpan
// SterileRootSpan makes the task run in a root span that doesn't get any
// children. Anybody trying to create a child of the task's span will get a
// root span. This is suitable for long-running tasks: connecting children to
// these tasks would lead to infinitely-long traces, and connecting the
// long-running task to its parent is also problematic because of the
// different lifetimes.
SterileRootSpan
)
// TaskOpts groups the task execution options for RunAsyncTaskEx.
type TaskOpts struct {
// TaskName is a human-readable name for the operation. Used as the name of
// the tracing span.
TaskName string
// SpanOpt controls the kind of span that the task will run in.
SpanOpt SpanOption
// If set, Sem is used as a semaphore limiting the concurrency (each task has
// weight 1).
//
// It is the caller's responsibility to ensure that Sem is closed when the
// stopper is quiesced. For quotapools which live for the lifetime of the
// stopper, it is generally best to register the sem with the stopper using
// AddCloser.
Sem *quotapool.IntPool
// If Sem is not nil, WaitForSem specifies whether the call blocks or not when
// the semaphore is full. If true, the call blocks until the semaphore is
// available in order to push back on callers that may be trying to create
// many tasks. If false, returns immediately with an error if the semaphore is
// not available.
WaitForSem bool
}
// RunAsyncTaskEx is like RunTask, except the callback f is run in a goroutine.
// The call doesn't block for the callback to finish execution.
func (s *Stopper) RunAsyncTaskEx(ctx context.Context, opt TaskOpts, f func(context.Context)) error {
var alloc *quotapool.IntAlloc
taskStarted := false
if opt.Sem != nil {
// Wait for permission to run from the semaphore.
var err error
if opt.WaitForSem {
alloc, err = opt.Sem.Acquire(ctx, 1)
} else {
alloc, err = opt.Sem.TryAcquire(ctx, 1)
}
if errors.Is(err, quotapool.ErrNotEnoughQuota) {
err = ErrThrottled
} else if quotapool.HasErrClosed(err) {
err = ErrUnavailable
}
if err != nil {
return err
}
defer func() {
// If the task is started, the alloc will be released async.
if !taskStarted {
alloc.Release()
}
}()
// Check for canceled context: it's possible to get the semaphore even
// if the context is canceled.
if ctx.Err() != nil {
return ctx.Err()
}
}
if !s.runPrelude() {
return ErrUnavailable
}
// If the caller has a span, the task gets a child span.
//
// Note that we have to create the child in this parent goroutine; we can't
// defer the creation to the spawned async goroutine since the parent span
// might get Finish()ed by then. However, we'll update the child'd goroutine
// ID.
var sp *tracing.Span
switch opt.SpanOpt {
case FollowsFromSpan:
ctx, sp = tracing.EnsureForkSpan(ctx, s.tracer, opt.TaskName)
case ChildSpan:
ctx, sp = tracing.EnsureChildSpan(ctx, s.tracer, opt.TaskName)
case SterileRootSpan:
ctx, sp = s.tracer.StartSpanCtx(ctx, opt.TaskName, tracing.WithSterile())
default:
panic(fmt.Sprintf("unsupported SpanOption: %v", opt.SpanOpt))
}
// Call f on another goroutine.
taskStarted = true // Another goroutine now takes ownership of the alloc, if any.
go func() {
defer s.Recover(ctx)
defer s.runPostlude()
if sp != nil {
defer sp.Finish()
sp.UpdateGoroutineIDToCurrent()
}
if alloc != nil {
defer alloc.Release()
}
f(ctx)
}()
return nil
}
func (s *Stopper) runPrelude() bool {
s.mu.RLock()
defer s.mu.RUnlock()
if s.refuseRLocked() {
return false
}
// NB: we run this under the read lock to ensure that `refuseRLocked()` cannot
// change until the task is registered. If we didn't do this, we'd run the
// risk of starting a task after a successful call to Stop().
s.addTask(1)
return true
}
func (s *Stopper) runPostlude() {
s.addTask(-1)
}
// NumTasks returns the number of active tasks.
func (s *Stopper) NumTasks() int {
return int(atomic.LoadInt32(&s.mu._numTasks))
}
// Stop signals all live workers to stop and then waits for each to
// confirm it has stopped.
//
// Stop is idempotent; concurrent calls will block on each other.
func (s *Stopper) Stop(ctx context.Context) {
s.mu.Lock()
stopCalled := s.mu.stopping
s.mu.stopping = true
s.mu.Unlock()
if stopCalled {
// Wait for the concurrent Stop() to complete.
<-s.stopped
return
}
defer func() {
s.Recover(ctx)
unregister(s)
close(s.stopped)
}()
// Don't bother doing stuff cleanly if we're panicking, that would likely
// block. Instead, best effort only. This cleans up the stack traces,
// avoids stalls and helps some tests in `./cli` finish cleanly (where
// panics happen on purpose).
if r := recover(); r != nil {
go s.Quiesce(ctx)
s.mu.Lock()
for _, c := range s.mu.closers {
go c.Close()
}
s.mu.Unlock()
panic(r)
}
s.Quiesce(ctx)
// Run the closers without holding s.mu. There's no concern around new
// closers being added; we've marked this stopper as `stopping` above, so
// any attempts to do so will be refused.
for _, c := range s.mu.closers {
c.Close()
}
}
// ShouldQuiesce returns a channel which will be closed when Stop() has been
// invoked and outstanding tasks should begin to quiesce.
func (s *Stopper) ShouldQuiesce() <-chan struct{} {
if s == nil {
// A nil stopper will never signal ShouldQuiesce, but will also never panic.
return nil
}
return s.quiescer
}
// IsStopped returns a channel which will be closed after Stop() has
// been invoked to full completion, meaning all workers have completed
// and all closers have been closed.
func (s *Stopper) IsStopped() <-chan struct{} {
if s == nil {
return nil
}
return s.stopped
}
// Quiesce moves the stopper to state quiescing and waits until all
// tasks complete. This is used from Stop() and unittests.
func (s *Stopper) Quiesce(ctx context.Context) {
defer time.AfterFunc(5*time.Second, func() {
log.Infof(ctx, "quiescing...")
}).Stop()
defer s.Recover(ctx)
func() {
s.mu.Lock()
defer s.mu.Unlock()
if !s.mu.quiescing {
s.mu.quiescing = true
close(s.quiescer)
}
s.mu.qCancels.Range(func(k, v interface{}) (wantMore bool) {
cancel := v.(func())
cancel()
s.mu.qCancels.Delete(k)
return true
})
}()
for s.NumTasks() > 0 {
time.Sleep(5 * time.Millisecond)
}
}
// SetTracer sets the tracer to be used for task spans. This cannot be called
// concurrently with starting tasks.
//
// Note that for tasks asking for a child span, the parent's tracer is used
// instead of this one.
//
// When possible, prefer supplying the tracer to the ctor through WithTracer.
func (s *Stopper) SetTracer(tr *tracing.Tracer) {
s.tracer = tr
}
// Tracer returns the Tracer that the Stopper will use for tasks.
func (s *Stopper) Tracer() *tracing.Tracer {
return s.tracer
}