Skip to content

Commit

Permalink
Merge #119595
Browse files Browse the repository at this point in the history
119595: timeutil: prevent illegal sharing of Timers r=yuzefovich a=yuzefovich

**timeutil: prevent illegal sharing of Timers**

Currently, whenever `Timer.Stop` is called, we unconditionally put that timer into `timerPool`. This works well assuming that the contract of `Stop` is satisfied - namely that the stopped timer can no longer be used.

However, we have at least one case where this contract is violated (fixed in the following commit), and in such a scenario it is possible for multiple users to access the same timer object which can lead to an undefined behavior (we've seen a deadlock on a test server shutdown).

This commit hardens the timer code to prevent a class of such issues by putting the timer into the pool on `Stop` only if the timer originally came from the pool.

Release note: None

**stmtdiagnostics: fix incorrect usage of timeutil.Timer**

The contract of `timeutil.Timer.Stop` is such that the timer can no longer be used, and it was violated in `Registry.poll`. This now fixed by allocating a fresh timer after each `Stop` call.

Fixes: #119593.

Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
craig[bot] and yuzefovich committed Feb 27, 2024
2 parents 609ac44 + 196dfd5 commit 2ae2550
Show file tree
Hide file tree
Showing 3 changed files with 81 additions and 4 deletions.
12 changes: 9 additions & 3 deletions pkg/sql/stmtdiagnostics/statement_diagnostics.go
Original file line number Diff line number Diff line change
Expand Up @@ -157,25 +157,31 @@ func (r *Registry) Start(ctx context.Context, stopper *stop.Stopper) {

func (r *Registry) poll(ctx context.Context) {
var (
timer timeutil.Timer
timer *timeutil.Timer
// We need to store timer.C reference separately because timer.Stop()
// (called when polling is disabled) puts timer into the pool and
// prohibits further usage of stored timer.C.
timerC = timer.C
timerC <-chan time.Time
lastPoll time.Time
deadline time.Time
pollIntervalChanged = make(chan struct{}, 1)
maybeResetTimer = func() {
if interval := pollingInterval.Get(&r.st.SV); interval == 0 {
// Setting the interval to zero stops the polling.
timer.Stop()
if timer != nil {
timer.Stop()
timer = nil
}
// nil out the channel so that it'd block forever in the loop
// below (until the polling interval is changed).
timerC = nil
} else {
newDeadline := lastPoll.Add(interval)
if deadline.IsZero() || !deadline.Equal(newDeadline) {
deadline = newDeadline
if timer == nil {
timer = timeutil.NewTimer()
}
timer.Reset(timeutil.Until(deadline))
timerC = timer.C
}
Expand Down
10 changes: 9 additions & 1 deletion pkg/util/timeutil/timer.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,11 +55,16 @@ type Timer struct {
// the timer has been initialized (via Reset).
C <-chan time.Time
Read bool
// fromPool indicates whether this Timer came from timerPool. If false, then
// it won't be put into the timerPool on Stop.
fromPool bool
}

// NewTimer allocates a new timer.
func NewTimer() *Timer {
return timerPool.Get().(*Timer)
t := timerPool.Get().(*Timer)
t.fromPool = true
return t
}

// Reset changes the timer to expire after duration d and returns
Expand Down Expand Up @@ -102,6 +107,9 @@ func (t *Timer) Stop() bool {
timeTimerPool.Put(t.timer)
}
}
if !t.fromPool {
return res
}
*t = Timer{}
timerPool.Put(t)
return res
Expand Down
63 changes: 63 additions & 0 deletions pkg/util/timeutil/timer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,14 @@
package timeutil

import (
"context"
"fmt"
"math/rand"
"sync"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

const timeStep = 10 * time.Millisecond
Expand Down Expand Up @@ -139,3 +144,61 @@ func TestTimerMakesProgressInLoop(t *testing.T) {
timer.Read = true
}
}

// TestIllegalTimerShare is a regression test for sharing the same Timer between
// multiple users when it was originally allocated on the stack of one of them
// but then later was put into timerPool on Stop() (see #119593).
func TestIllegalTimerShare(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), time.Second)
defer cancel()

resetTimer := func(t *Timer, rng *rand.Rand) {
t.Reset(time.Duration(rng.Intn(100)+1) * time.Nanosecond)
}

var wg sync.WaitGroup
// Simulate a pattern of usage of the stack-allocated Timer that is being
// stopped each time when the Timer fires.
fromStack := func() {
defer wg.Done()
rng, _ := randutil.NewTestRand()
var t Timer
defer t.Stop()
resetTimer(&t, rng)
for {
select {
case <-ctx.Done():
return
case <-t.C:
t.Read = true
t.Stop()
resetTimer(&t, rng)
}
}
}
// Simulate the most common pattern where the Timer is taken from the
// timerPool, fires repeatedly, and then is stopped in a defer.
fromPool := func() {
defer wg.Done()
rng, _ := randutil.NewTestRand()
t := NewTimer()
defer t.Stop()
resetTimer(t, rng)
for {
select {
case <-ctx.Done():
return
case <-t.C:
t.Read = true
resetTimer(t, rng)
}
}
}
// Spin up a few goroutines per each access pattern.
wg.Add(6)
for i := 0; i < 3; i++ {
go fromStack()
go fromPool()
}
wg.Wait()
}

0 comments on commit 2ae2550

Please sign in to comment.