syncs: add Mutex/RWMutex alias/wrappers for future mutex debugging

Updates #17852

Change-Id: I477340fb8e40686870e981ade11cd61597c34a20
Signed-off-by: Brad Fitzpatrick <bradfitz@tailscale.com>
This commit is contained in:
Brad Fitzpatrick
2025-11-15 09:40:57 -08:00
committed by Brad Fitzpatrick
parent 3a41c0c585
commit 99b06eac49
55 changed files with 145 additions and 94 deletions

View File

@@ -16,9 +16,9 @@ import (
"net/netip"
"slices"
"strings"
"sync"
"time"
"tailscale.com/syncs"
"tailscale.com/types/appctype"
"tailscale.com/types/logger"
"tailscale.com/types/views"
@@ -139,7 +139,7 @@ type AppConnector struct {
hasStoredRoutes bool
// mu guards the fields that follow
mu sync.Mutex
mu syncs.Mutex
// domains is a map of lower case domain names with no trailing dot, to an
// ordered list of resolved IP addresses.

View File

@@ -38,6 +38,7 @@ import (
"tailscale.com/net/udprelay/status"
"tailscale.com/paths"
"tailscale.com/safesocket"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/types/appctype"
"tailscale.com/types/dnstype"
@@ -1363,7 +1364,7 @@ type IPNBusWatcher struct {
httpRes *http.Response
dec *json.Decoder
mu sync.Mutex
mu syncs.Mutex
closed bool
}

View File

@@ -18,6 +18,7 @@ import (
"golang.org/x/crypto/blake2s"
chp "golang.org/x/crypto/chacha20poly1305"
"tailscale.com/syncs"
"tailscale.com/types/key"
)
@@ -48,7 +49,7 @@ type Conn struct {
// rxState is all the Conn state that Read uses.
type rxState struct {
sync.Mutex
syncs.Mutex
cipher cipher.AEAD
nonce nonce
buf *maxMsgBuffer // or nil when reads exhausted

View File

@@ -23,7 +23,6 @@ import (
"runtime"
"slices"
"strings"
"sync"
"sync/atomic"
"time"
@@ -44,6 +43,7 @@ import (
"tailscale.com/net/netx"
"tailscale.com/net/tlsdial"
"tailscale.com/net/tsdial"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/tka"
"tailscale.com/tstime"
@@ -92,7 +92,7 @@ type Direct struct {
dialPlan ControlDialPlanner // can be nil
mu sync.Mutex // mutex guards the following fields
mu syncs.Mutex // mutex guards the following fields
serverLegacyKey key.MachinePublic // original ("legacy") nacl crypto_box-based public key; only used for signRegisterRequest on Windows now
serverNoiseKey key.MachinePublic

View File

@@ -177,7 +177,7 @@ type Server struct {
verifyClientsURL string
verifyClientsURLFailOpen bool
mu sync.Mutex
mu syncs.Mutex
closed bool
netConns map[derp.Conn]chan struct{} // chan is closed when conn closes
clients map[key.NodePublic]*clientSet

View File

@@ -28,19 +28,19 @@ import (
"slices"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"tailscale.com/feature/buildfeatures"
"tailscale.com/kube/kubetypes"
"tailscale.com/syncs"
"tailscale.com/types/opt"
"tailscale.com/version"
"tailscale.com/version/distro"
)
var (
mu sync.Mutex
mu syncs.Mutex
// +checklocks:mu
set = map[string]string{}
// +checklocks:mu

View File

@@ -9,7 +9,6 @@ import (
"encoding/json"
"fmt"
"net/http"
"sync"
"tailscale.com/disco"
"tailscale.com/feature"
@@ -19,6 +18,7 @@ import (
"tailscale.com/net/udprelay"
"tailscale.com/net/udprelay/endpoint"
"tailscale.com/net/udprelay/status"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/types/key"
"tailscale.com/types/logger"
@@ -95,7 +95,7 @@ type extension struct {
ec *eventbus.Client
respPub *eventbus.Publisher[magicsock.UDPRelayAllocResp]
mu sync.Mutex // guards the following fields
mu syncs.Mutex // guards the following fields
shutdown bool // true if Shutdown() has been called
rs relayServer // nil when disabled
port *int // ipn.Prefs.RelayServerPort, nil if disabled

View File

@@ -20,6 +20,7 @@ import (
"tailscale.com/envknob"
"tailscale.com/feature/buildfeatures"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/tstime"
"tailscale.com/types/opt"
@@ -30,7 +31,7 @@ import (
)
var (
mu sync.Mutex
mu syncs.Mutex
debugHandler map[string]http.Handler
)

View File

@@ -7,7 +7,6 @@ import (
"context"
"errors"
"fmt"
"sync"
"time"
"tailscale.com/control/controlclient"
@@ -15,6 +14,7 @@ import (
"tailscale.com/ipn"
"tailscale.com/ipn/ipnauth"
"tailscale.com/ipn/ipnext"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/types/lazy"
"tailscale.com/types/logger"
@@ -40,7 +40,7 @@ type extension struct {
store lazy.SyncValue[LogStore]
// mu protects all following fields.
mu sync.Mutex
mu syncs.Mutex
// logger is the current audit logger, or nil if it is not set up,
// such as before the first control client is created, or after
// a profile change and before the new control client is created.

View File

@@ -30,7 +30,6 @@ import (
"runtime"
"slices"
"strings"
"sync"
"time"
"tailscale.com/atomicfile"
@@ -42,6 +41,7 @@ import (
"tailscale.com/ipn/store"
"tailscale.com/ipn/store/mem"
"tailscale.com/net/bakedroots"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/tempfork/acme"
"tailscale.com/types/logger"
@@ -60,9 +60,9 @@ var (
// acmeMu guards all ACME operations, so concurrent requests
// for certs don't slam ACME. The first will go through and
// populate the on-disk cache and the rest should use that.
acmeMu sync.Mutex
acmeMu syncs.Mutex
renewMu sync.Mutex // lock order: acmeMu before renewMu
renewMu syncs.Mutex // lock order: acmeMu before renewMu
renewCertAt = map[string]time.Time{}
)

View File

@@ -247,7 +247,7 @@ type LocalBackend struct {
extHost *ExtensionHost
// The mutex protects the following elements.
mu sync.Mutex
mu syncs.Mutex
// currentNodeAtomic is the current node context. It is always non-nil.
// It must be re-created when [LocalBackend] switches to a different profile/node
@@ -329,14 +329,14 @@ type LocalBackend struct {
//
// tkaSyncLock MUST be taken before mu (or inversely, mu must not be held
// at the moment that tkaSyncLock is taken).
tkaSyncLock sync.Mutex
tkaSyncLock syncs.Mutex
clock tstime.Clock
// Last ClientVersion received in MapResponse, guarded by mu.
lastClientVersion *tailcfg.ClientVersion
// lastNotifiedDriveSharesMu guards lastNotifiedDriveShares
lastNotifiedDriveSharesMu sync.Mutex
lastNotifiedDriveSharesMu syncs.Mutex
// lastNotifiedDriveShares keeps track of the last set of shares that we
// notified about.

View File

@@ -16,6 +16,7 @@ import (
"tailscale.com/ipn"
"tailscale.com/net/dns"
"tailscale.com/net/tsaddr"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/types/dnstype"
"tailscale.com/types/key"
@@ -82,7 +83,7 @@ type nodeBackend struct {
derpMapViewPub *eventbus.Publisher[tailcfg.DERPMapView]
// TODO(nickkhyl): maybe use sync.RWMutex?
mu sync.Mutex // protects the following fields
mu syncs.Mutex // protects the following fields
shutdownOnce sync.Once // guards calling [nodeBackend.shutdown]
readyCh chan struct{} // closed by [nodeBackend.ready]; nil after shutdown

View File

@@ -9,7 +9,8 @@ import (
"bytes"
"errors"
"fmt"
"sync"
"tailscale.com/syncs"
)
type Buffer interface {
@@ -36,7 +37,7 @@ type memBuffer struct {
next []byte
pending chan qentry
dropMu sync.Mutex
dropMu syncs.Mutex
dropCount int
}

View File

@@ -11,7 +11,6 @@ import (
"io"
"slices"
"strings"
"sync"
"tailscale.com/syncs"
)
@@ -41,7 +40,7 @@ type LabelMap struct {
Label string
expvar.Map
// shardedIntMu orders the initialization of new shardedint keys
shardedIntMu sync.Mutex
shardedIntMu syncs.Mutex
}
// SetInt64 sets the *Int value stored under the given map key.

View File

@@ -18,6 +18,7 @@ import (
"time"
"tailscale.com/net/netmon"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/types/logger"
)
@@ -32,7 +33,7 @@ type Detector struct {
// currIfIndex is the index of the interface that is currently being used by the httpClient.
currIfIndex int
// mu guards currIfIndex.
mu sync.Mutex
mu syncs.Mutex
// logf is the logger used for logging messages. If it is nil, log.Printf is used.
logf logger.Logf
}

View File

@@ -16,7 +16,6 @@ import (
"slices"
"sort"
"strings"
"sync"
"syscall"
"time"
@@ -27,6 +26,7 @@ import (
"tailscale.com/control/controlknobs"
"tailscale.com/envknob"
"tailscale.com/health"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/util/dnsname"
"tailscale.com/util/syspolicy/pkey"
@@ -51,7 +51,7 @@ type windowsManager struct {
unregisterPolicyChangeCb func() // called when the manager is closing
mu sync.Mutex
mu syncs.Mutex
closing bool
}

View File

@@ -8,12 +8,12 @@ import (
"html"
"net/http"
"strconv"
"sync"
"sync/atomic"
"time"
"tailscale.com/feature/buildfeatures"
"tailscale.com/health"
"tailscale.com/syncs"
)
func init() {
@@ -39,7 +39,7 @@ func init() {
var fwdLogAtomic atomic.Pointer[fwdLog]
type fwdLog struct {
mu sync.Mutex
mu syncs.Mutex
pos int // ent[pos] is next entry
ent []fwdLogEntry
}

View File

@@ -37,6 +37,7 @@ import (
"tailscale.com/net/netx"
"tailscale.com/net/sockstats"
"tailscale.com/net/tsdial"
"tailscale.com/syncs"
"tailscale.com/types/dnstype"
"tailscale.com/types/logger"
"tailscale.com/types/nettype"
@@ -231,7 +232,7 @@ type forwarder struct {
ctx context.Context // good until Close
ctxCancel context.CancelFunc // closes ctx
mu sync.Mutex // guards following
mu syncs.Mutex // guards following
dohClient map[string]*http.Client // urlBase -> client

View File

@@ -214,7 +214,7 @@ type Resolver struct {
closed chan struct{}
// mu guards the following fields from being updated while used.
mu sync.Mutex
mu syncs.Mutex
localDomains []dnsname.FQDN
hostToIP map[dnsname.FQDN][]netip.Addr
ipToHost map[netip.Addr]dnsname.FQDN

View File

@@ -20,6 +20,7 @@ import (
"tailscale.com/envknob"
"tailscale.com/net/netx"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/util/cloudenv"
"tailscale.com/util/singleflight"
@@ -97,7 +98,7 @@ type Resolver struct {
sf singleflight.Group[string, ipRes]
mu sync.Mutex
mu syncs.Mutex
ipCache map[string]ipCacheEntry
}
@@ -474,7 +475,7 @@ type dialCall struct {
d *dialer
network, address, host, port string
mu sync.Mutex // lock ordering: dialer.mu, then dialCall.mu
mu syncs.Mutex // lock ordering: dialer.mu, then dialCall.mu
fails map[netip.Addr]error // set of IPs that failed to dial thus far
}

View File

@@ -12,9 +12,9 @@ import (
"fmt"
"net"
"net/netip"
"sync"
"tailscale.com/net/netx"
"tailscale.com/syncs"
)
var _ netx.Network = (*Network)(nil)
@@ -26,7 +26,7 @@ var _ netx.Network = (*Network)(nil)
//
// Its zero value is a valid [netx.Network] implementation.
type Network struct {
mu sync.Mutex
mu syncs.Mutex
lns map[string]*Listener // address -> listener
}

View File

@@ -235,7 +235,7 @@ type Client struct {
testEnoughRegions int
testCaptivePortalDelay time.Duration
mu sync.Mutex // guards following
mu syncs.Mutex // guards following
nextFull bool // do a full region scan, even if last != nil
prev map[time.Time]*Report // some previous reports
last *Report // most recent report
@@ -597,7 +597,7 @@ type reportState struct {
stopProbeCh chan struct{}
waitPortMap sync.WaitGroup
mu sync.Mutex
mu syncs.Mutex
report *Report // to be returned by GetReport
inFlight map[stun.TxID]func(netip.AddrPort) // called without c.mu held
gotEP4 netip.AddrPort

View File

@@ -7,12 +7,12 @@ import (
"fmt"
"net"
"strings"
"sync"
"syscall"
"unsafe"
"golang.org/x/net/route"
"golang.org/x/sys/unix"
"tailscale.com/syncs"
"tailscale.com/util/mak"
)
@@ -26,7 +26,7 @@ func parseRoutingTable(rib []byte) ([]route.Message, error) {
}
var ifNames struct {
sync.Mutex
syncs.Mutex
m map[int]string // ifindex => name
}

View File

@@ -15,6 +15,7 @@ import (
"time"
"tailscale.com/feature/buildfeatures"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/util/clientmetric"
"tailscale.com/util/eventbus"
@@ -65,7 +66,7 @@ type Monitor struct {
// and not change at runtime.
tsIfName string // tailscale interface name, if known/set ("tailscale0", "utun3", ...)
mu sync.Mutex // guards all following fields
mu syncs.Mutex // guards all following fields
cbs set.HandleSet[ChangeFunc]
ifState *State
gwValid bool // whether gw and gwSelfIP are valid

View File

@@ -8,7 +8,8 @@ import (
"bufio"
"io"
"net"
"sync"
"tailscale.com/syncs"
)
// NewOneConnListener returns a net.Listener that returns c on its
@@ -29,7 +30,7 @@ func NewOneConnListener(c net.Conn, addr net.Addr) net.Listener {
type oneConnListener struct {
addr net.Addr
mu sync.Mutex
mu syncs.Mutex
conn net.Conn
}

View File

@@ -23,6 +23,7 @@ import (
"golang.org/x/net/icmp"
"golang.org/x/net/ipv4"
"golang.org/x/net/ipv6"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/util/mak"
)
@@ -64,7 +65,7 @@ type Pinger struct {
wg sync.WaitGroup
// Following fields protected by mu
mu sync.Mutex
mu syncs.Mutex
// conns is a map of "type" to net.PacketConn, type is either
// "ip4:icmp" or "ip6:icmp"
conns map[string]net.PacketConn

View File

@@ -14,7 +14,6 @@ import (
"net/http"
"net/netip"
"slices"
"sync"
"sync/atomic"
"time"
@@ -123,7 +122,7 @@ type Client struct {
testPxPPort uint16 // if non-zero, pxpPort to use for tests
testUPnPPort uint16 // if non-zero, uPnPPort to use for tests
mu sync.Mutex // guards following, and all fields thereof
mu syncs.Mutex // guards following, and all fields thereof
// runningCreate is whether we're currently working on creating
// a port mapping (whether GetCachedMappingOrStartCreatingOne kicked

View File

@@ -10,12 +10,12 @@ import (
"fmt"
"net"
"strings"
"sync"
"sync/atomic"
"syscall"
"time"
"tailscale.com/net/netmon"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/util/clientmetric"
"tailscale.com/version"
@@ -40,7 +40,7 @@ var sockStats = struct {
// mu protects fields in this group (but not the fields within
// sockStatCounters). It should not be held in the per-read/write
// callbacks.
mu sync.Mutex
mu syncs.Mutex
countersByLabel map[Label]*sockStatCounters
knownInterfaces map[int]string // interface index -> name
usedInterfaces map[int]int // set of interface indexes

View File

@@ -27,6 +27,7 @@ import (
"tailscale.com/net/netns"
"tailscale.com/net/netx"
"tailscale.com/net/tsaddr"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/types/netmap"
"tailscale.com/util/clientmetric"
@@ -86,7 +87,7 @@ type Dialer struct {
routes atomic.Pointer[bart.Table[bool]] // or nil if UserDial should not use routes. `true` indicates routes that point into the Tailscale interface
mu sync.Mutex
mu syncs.Mutex
closed bool
dns dnsMap
tunName string // tun device name

View File

@@ -12,11 +12,11 @@ import (
"math"
"net"
"os"
"sync"
"sync/atomic"
"time"
"github.com/coder/websocket"
"tailscale.com/syncs"
)
// NetConn converts a *websocket.Conn into a net.Conn.
@@ -102,7 +102,7 @@ type netConn struct {
reading atomic.Bool
afterReadDeadline atomic.Bool
readMu sync.Mutex
readMu syncs.Mutex
// eofed is true if the reader should return io.EOF from the Read call.
//
// +checklocks:readMu

View File

@@ -9,9 +9,9 @@ import (
"fmt"
"net/netip"
"strings"
"sync"
"time"
"tailscale.com/syncs"
"tailscale.com/util/mak"
)
@@ -22,7 +22,7 @@ import (
// ask tailscaled (via the LocalAPI WhoIs method) the Tailscale identity that a
// given localhost:port corresponds to.
type Mapper struct {
mu sync.Mutex
mu syncs.Mutex
// m holds the mapping from localhost IP:ports to Tailscale IPs. It is
// keyed first by the protocol ("tcp" or "udp"), then by the IP:port.

View File

@@ -8,7 +8,7 @@ import (
)
// AssertLocked panics if m is not locked.
func AssertLocked(m *sync.Mutex) {
func AssertLocked(m *Mutex) {
if m.TryLock() {
m.Unlock()
panic("mutex is not locked")
@@ -16,7 +16,7 @@ func AssertLocked(m *sync.Mutex) {
}
// AssertRLocked panics if rw is not locked for reading or writing.
func AssertRLocked(rw *sync.RWMutex) {
func AssertRLocked(rw *RWMutex) {
if rw.TryLock() {
rw.Unlock()
panic("mutex is not locked")

18
syncs/mutex.go Normal file
View File

@@ -0,0 +1,18 @@
// Copyright (c) Tailscale Inc & AUTHORS
// SPDX-License-Identifier: BSD-3-Clause
//go:build !ts_mutex_debug
package syncs
import "sync"
// Mutex is an alias for sync.Mutex.
//
// It's only not a sync.Mutex when built with the ts_mutex_debug build tag.
type Mutex = sync.Mutex
// RWMutex is an alias for sync.RWMutex.
//
// It's only not a sync.RWMutex when built with the ts_mutex_debug build tag.
type RWMutex = sync.RWMutex

18
syncs/mutex_debug.go Normal file
View File

@@ -0,0 +1,18 @@
// Copyright (c) Tailscale Inc & AUTHORS
// SPDX-License-Identifier: BSD-3-Clause
//go:build ts_mutex_debug
package syncs
import "sync"
type Mutex struct {
sync.Mutex
}
type RWMutex struct {
sync.RWMutex
}
// TODO(bradfitz): actually track stuff when in debug mode.

View File

@@ -1,13 +1,14 @@
// Copyright (c) Tailscale Inc & AUTHORS
// SPDX-License-Identifier: BSD-3-Clause
package syncs
package syncs_test
import (
"expvar"
"sync"
"testing"
. "tailscale.com/syncs"
"tailscale.com/tstest"
)

View File

@@ -8,8 +8,8 @@ import (
"log"
"reflect"
"slices"
"sync"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/util/set"
)
@@ -34,12 +34,12 @@ type Bus struct {
routeDebug hook[RoutedEvent]
logf logger.Logf
topicsMu sync.Mutex
topicsMu syncs.Mutex
topics map[reflect.Type][]*subscribeState
// Used for introspection/debugging only, not in the normal event
// publishing path.
clientsMu sync.Mutex
clientsMu syncs.Mutex
clients set.Set[*Client]
}
@@ -306,7 +306,7 @@ func (w *worker) StopAndWait() {
type stopFlag struct {
// guards the lazy construction of stopped, and the value of
// alreadyStopped.
mu sync.Mutex
mu syncs.Mutex
stopped chan struct{}
alreadyStopped bool
}

View File

@@ -5,8 +5,8 @@ package eventbus
import (
"reflect"
"sync"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/util/set"
)
@@ -22,7 +22,7 @@ type Client struct {
bus *Bus
publishDebug hook[PublishedEvent]
mu sync.Mutex
mu syncs.Mutex
pub set.Set[publisher]
sub *subscribeState // Lazily created on first subscribe
stop stopFlag // signaled on Close

View File

@@ -11,10 +11,10 @@ import (
"runtime"
"slices"
"strings"
"sync"
"sync/atomic"
"time"
"tailscale.com/syncs"
"tailscale.com/types/logger"
)
@@ -147,7 +147,7 @@ func (d *Debugger) SubscribeTypes(client *Client) []reflect.Type {
// A hook collects hook functions that can be run as a group.
type hook[T any] struct {
sync.Mutex
syncs.Mutex
fns []hookFn[T]
}

View File

@@ -8,9 +8,9 @@ import (
"fmt"
"reflect"
"runtime"
"sync"
"time"
"tailscale.com/syncs"
"tailscale.com/types/logger"
"tailscale.com/util/cibuild"
)
@@ -51,7 +51,7 @@ type subscribeState struct {
snapshot chan chan []DeliveredEvent
debug hook[DeliveredEvent]
outputsMu sync.Mutex
outputsMu syncs.Mutex
outputs map[reflect.Type]subscriber
}

View File

@@ -7,11 +7,12 @@ package execqueue
import (
"context"
"errors"
"sync"
"tailscale.com/syncs"
)
type ExecQueue struct {
mu sync.Mutex
mu syncs.Mutex
ctx context.Context // context.Background + closed on Shutdown
cancel context.CancelFunc // closes ctx
closed bool

View File

@@ -7,9 +7,9 @@ package expvarx
import (
"encoding/json"
"expvar"
"sync"
"time"
"tailscale.com/syncs"
"tailscale.com/types/lazy"
)
@@ -20,7 +20,7 @@ type SafeFunc struct {
limit time.Duration
onSlow func(time.Duration, any)
mu sync.Mutex
mu syncs.Mutex
inflight *lazy.SyncValue[any]
}

View File

@@ -4,9 +4,9 @@
package goroutines
import (
"sync"
"sync/atomic"
"tailscale.com/syncs"
"tailscale.com/util/set"
)
@@ -15,7 +15,7 @@ type Tracker struct {
started atomic.Int64 // counter
running atomic.Int64 // gauge
mu sync.Mutex
mu syncs.Mutex
onDone set.HandleSet[func()]
}

View File

@@ -8,9 +8,9 @@ import (
"fmt"
"html"
"io"
"sync"
"time"
"tailscale.com/syncs"
"tailscale.com/util/lru"
)
@@ -75,7 +75,7 @@ type Limiter[K comparable] struct {
// perpetually in debt and cannot proceed at all.
Overdraft int64
mu sync.Mutex
mu syncs.Mutex
cache *lru.Cache[K, *bucket]
}

View File

@@ -4,7 +4,7 @@
// Package ringlog contains a limited-size concurrency-safe generic ring log.
package ringlog
import "sync"
import "tailscale.com/syncs"
// New creates a new [RingLog] containing at most max items.
func New[T any](max int) *RingLog[T] {
@@ -15,7 +15,7 @@ func New[T any](max int) *RingLog[T] {
// RingLog is a concurrency-safe fixed size log window containing entries of [T].
type RingLog[T any] struct {
mu sync.Mutex
mu syncs.Mutex
pos int
buf []T
max int

View File

@@ -9,6 +9,7 @@ import (
"sync"
"time"
"tailscale.com/syncs"
"tailscale.com/util/set"
"tailscale.com/util/syspolicy/internal/loggerx"
"tailscale.com/util/syspolicy/pkey"
@@ -70,7 +71,7 @@ func (c PolicyChange) HasChangedAnyOf(keys ...pkey.Key) bool {
// policyChangeCallbacks are the callbacks to invoke when the effective policy changes.
// It is safe for concurrent use.
type policyChangeCallbacks struct {
mu sync.Mutex
mu syncs.Mutex
cbs set.HandleSet[PolicyChangeCallback]
}

View File

@@ -7,10 +7,10 @@ import (
"errors"
"fmt"
"slices"
"sync"
"sync/atomic"
"time"
"tailscale.com/syncs"
"tailscale.com/util/syspolicy/internal/loggerx"
"tailscale.com/util/syspolicy/setting"
"tailscale.com/util/testenv"
@@ -58,7 +58,7 @@ type Policy struct {
changeCallbacks policyChangeCallbacks
mu sync.Mutex
mu syncs.Mutex
watcherStarted bool // whether [Policy.watchReload] was started
sources source.ReadableSources
closing bool // whether [Policy.Close] was called (even if we're still closing)

View File

@@ -10,7 +10,6 @@ import (
"errors"
"fmt"
"slices"
"sync"
"tailscale.com/syncs"
"tailscale.com/util/slicesx"
@@ -20,7 +19,7 @@ import (
)
var (
policyMu sync.Mutex // protects [policySources] and [effectivePolicies]
policyMu syncs.Mutex // protects [policySources] and [effectivePolicies]
policySources []*source.Source // all registered policy sources
effectivePolicies []*Policy // all active (non-closed) effective policies returned by [PolicyFor]

View File

@@ -11,9 +11,9 @@ import (
"fmt"
"slices"
"strings"
"sync"
"time"
"tailscale.com/syncs"
"tailscale.com/types/lazy"
"tailscale.com/util/syspolicy/internal"
"tailscale.com/util/syspolicy/pkey"
@@ -215,7 +215,7 @@ type DefinitionMap map[pkey.Key]*Definition
var (
definitions lazy.SyncValue[DefinitionMap]
definitionsMu sync.Mutex
definitionsMu syncs.Mutex
definitionsList []*Definition
definitionsUsed bool
)

View File

@@ -10,11 +10,13 @@ import (
"sync"
"syscall"
"time"
"tailscale.com/syncs"
)
// blockForeverConn is a net.PacketConn whose reads block until it is closed.
type blockForeverConn struct {
mu sync.Mutex
mu syncs.Mutex
cond *sync.Cond
closed bool
}

View File

@@ -17,7 +17,6 @@ import (
"reflect"
"runtime"
"slices"
"sync"
"sync/atomic"
"time"
@@ -28,6 +27,7 @@ import (
"tailscale.com/net/packet"
"tailscale.com/net/stun"
"tailscale.com/net/tstun"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/tstime/mono"
"tailscale.com/types/key"
@@ -73,7 +73,7 @@ type endpoint struct {
disco atomic.Pointer[endpointDisco] // if the peer supports disco, the key and short string
// mu protects all following fields.
mu sync.Mutex // Lock ordering: Conn.mu, then endpoint.mu
mu syncs.Mutex // Lock ordering: Conn.mu, then endpoint.mu
heartBeatTimer *time.Timer // nil when idle
lastSendExt mono.Time // last time there were outgoing packets sent to this peer from an external trigger (e.g. wireguard-go or disco pingCLI)

View File

@@ -6,9 +6,9 @@ package magicsock
import (
"net/netip"
"slices"
"sync"
"time"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/tempfork/heap"
"tailscale.com/util/mak"
@@ -107,7 +107,7 @@ func (eh endpointHeap) Min() *endpointTrackerEntry {
//
// See tailscale/tailscale#7877 for more information.
type endpointTracker struct {
mu sync.Mutex
mu syncs.Mutex
endpoints map[netip.Addr]*endpointHeap
}

View File

@@ -286,7 +286,7 @@ type Conn struct {
// mu guards all following fields; see userspaceEngine lock
// ordering rules against the engine. For derphttp, mu must
// be held before derphttp.Client.mu.
mu sync.Mutex
mu syncs.Mutex
muCond *sync.Cond
onlyTCP443 atomic.Bool

View File

@@ -8,7 +8,6 @@ import (
"fmt"
"net"
"net/netip"
"sync"
"sync/atomic"
"syscall"
@@ -16,6 +15,7 @@ import (
"tailscale.com/net/batching"
"tailscale.com/net/netaddr"
"tailscale.com/net/packet"
"tailscale.com/syncs"
"tailscale.com/types/nettype"
)
@@ -31,7 +31,7 @@ type RebindingUDPConn struct {
// Neither is expected to be nil, sockets are bound on creation.
pconnAtomic atomic.Pointer[nettype.PacketConn]
mu sync.Mutex // held while changing pconn (and pconnAtomic)
mu syncs.Mutex // held while changing pconn (and pconnAtomic)
pconn nettype.PacketConn
port uint16
}

View File

@@ -15,6 +15,7 @@ import (
"tailscale.com/net/packet"
"tailscale.com/net/stun"
udprelay "tailscale.com/net/udprelay/endpoint"
"tailscale.com/syncs"
"tailscale.com/tailcfg"
"tailscale.com/tstime"
"tailscale.com/types/key"
@@ -58,7 +59,7 @@ type relayManager struct {
getServersCh chan chan set.Set[candidatePeerRelay]
derpHomeChangeCh chan derpHomeChangeEvent
discoInfoMu sync.Mutex // guards the following field
discoInfoMu syncs.Mutex // guards the following field
discoInfoByServerDisco map[key.DiscoPublic]*relayHandshakeDiscoInfo
// runLoopStoppedCh is written to by runLoop() upon return, enabling event

View File

@@ -15,7 +15,6 @@ import (
"log"
"net/http"
"net/netip"
"sync"
"time"
"tailscale.com/health"
@@ -24,6 +23,7 @@ import (
"tailscale.com/net/netmon"
"tailscale.com/net/sockstats"
"tailscale.com/net/tsaddr"
"tailscale.com/syncs"
"tailscale.com/types/ipproto"
"tailscale.com/types/logger"
"tailscale.com/types/logid"
@@ -57,7 +57,7 @@ func (noopDevice) SetConnectionCounter(netlogfunc.ConnectionCounter) {}
// unless the Tailnet administrator opts-into explicit logging.
// The zero value is ready for use.
type Logger struct {
mu sync.Mutex // protects all fields below
mu syncs.Mutex // protects all fields below
logf logger.Logf
// shutdownLocked shuts down the logger.