mirror of
https://github.com/tailscale/tailscale.git
synced 2025-10-09 08:01:31 +00:00
net/connstats: invert network logging data flow (#6272)
Previously, tstun.Wrapper and magicsock.Conn managed their own statistics data structure and relied on an external call to Extract to extract (and reset) the statistics. This makes it difficult to ensure a maximum size on the statistics as the caller has no introspection into whether the number of unique connections is getting too large. Invert the control flow such that a *connstats.Statistics is registered with tstun.Wrapper and magicsock.Conn. Methods on non-nil *connstats.Statistics are called for every packet. This allows the implementation of connstats.Statistics (in the future) to better control when it needs to flush to ensure bounds on maximum sizes. The value registered into tstun.Wrapper and magicsock.Conn could be an interface, but that has two performance detriments: 1. Method calls on interface values are more expensive since they must go through a virtual method dispatch. 2. The implementation would need a sync.Mutex to protect the statistics value instead of using an atomic.Pointer. Given that methods on constats.Statistics are called for every packet, we want reduce the CPU cost on this hot path. Signed-off-by: Joe Tsai <joetsai@digital-static.net>
This commit is contained in:
@@ -37,6 +37,7 @@ import (
|
||||
"tailscale.com/health"
|
||||
"tailscale.com/ipn/ipnstate"
|
||||
"tailscale.com/logtail/backoff"
|
||||
"tailscale.com/net/connstats"
|
||||
"tailscale.com/net/dnscache"
|
||||
"tailscale.com/net/interfaces"
|
||||
"tailscale.com/net/netaddr"
|
||||
@@ -52,7 +53,6 @@ import (
|
||||
"tailscale.com/tstime/mono"
|
||||
"tailscale.com/types/key"
|
||||
"tailscale.com/types/logger"
|
||||
"tailscale.com/types/netlogtype"
|
||||
"tailscale.com/types/netmap"
|
||||
"tailscale.com/types/nettype"
|
||||
"tailscale.com/util/clientmetric"
|
||||
@@ -337,19 +337,7 @@ type Conn struct {
|
||||
port atomic.Uint32
|
||||
|
||||
// stats maintains per-connection counters.
|
||||
// See SetStatisticsEnabled and ExtractStatistics for details.
|
||||
stats struct {
|
||||
enabled atomic.Bool
|
||||
|
||||
// TODO(joetsai): A per-Conn map of connections is easiest to implement.
|
||||
// Since every packet occurs within the context of an endpoint,
|
||||
// we could track the counts within the endpoint itself,
|
||||
// and then merge the results when ExtractStatistics is called.
|
||||
// That would avoid a map lookup for every packet.
|
||||
|
||||
mu sync.Mutex
|
||||
m map[netlogtype.Connection]netlogtype.Counts
|
||||
}
|
||||
stats atomic.Pointer[connstats.Statistics]
|
||||
|
||||
// ============================================================
|
||||
// mu guards all following fields; see userspaceEngine lock
|
||||
@@ -1754,8 +1742,8 @@ func (c *Conn) receiveIP(b []byte, ipp netip.AddrPort, cache *ippEndpointCache,
|
||||
ep = de
|
||||
}
|
||||
ep.noteRecvActivity()
|
||||
if c.stats.enabled.Load() {
|
||||
c.updateStats(ep.nodeAddr, ipp, netlogtype.Counts{RxPackets: 1, RxBytes: uint64(len(b))})
|
||||
if stats := c.stats.Load(); stats != nil {
|
||||
stats.UpdateRxPhysical(ep.nodeAddr, ipp, len(b))
|
||||
}
|
||||
return ep, true
|
||||
}
|
||||
@@ -1812,8 +1800,8 @@ func (c *Conn) processDERPReadResult(dm derpReadResult, b []byte) (n int, ep *en
|
||||
}
|
||||
|
||||
ep.noteRecvActivity()
|
||||
if c.stats.enabled.Load() {
|
||||
c.updateStats(ep.nodeAddr, ipp, netlogtype.Counts{RxPackets: 1, RxBytes: uint64(dm.n)})
|
||||
if stats := c.stats.Load(); stats != nil {
|
||||
stats.UpdateRxPhysical(ep.nodeAddr, ipp, dm.n)
|
||||
}
|
||||
return n, ep
|
||||
}
|
||||
@@ -3306,37 +3294,10 @@ func (c *Conn) UpdateStatus(sb *ipnstate.StatusBuilder) {
|
||||
})
|
||||
}
|
||||
|
||||
// updateStats updates the statistics counters with the src, dst, and cnts.
|
||||
// It is the caller's responsibility to check whether logging is enabled.
|
||||
func (c *Conn) updateStats(src netip.Addr, dst netip.AddrPort, cnts netlogtype.Counts) {
|
||||
conn := netlogtype.Connection{Src: netip.AddrPortFrom(src, 0), Dst: dst}
|
||||
c.stats.mu.Lock()
|
||||
defer c.stats.mu.Unlock()
|
||||
mak.Set(&c.stats.m, conn, c.stats.m[conn].Add(cnts))
|
||||
}
|
||||
|
||||
// SetStatisticsEnabled enables per-connection packet counters.
|
||||
// Disabling statistics gathering does not reset the counters.
|
||||
// ExtractStatistics must be called to reset the counters and
|
||||
// be periodically called while enabled to avoid unbounded memory use.
|
||||
func (c *Conn) SetStatisticsEnabled(enable bool) {
|
||||
c.stats.enabled.Store(enable)
|
||||
}
|
||||
|
||||
// ExtractStatistics extracts and resets the counters for all active connections.
|
||||
// It must be called periodically otherwise the memory used is unbounded.
|
||||
//
|
||||
// The source is always a peer's tailscale IP address,
|
||||
// while the destination is the peer's physical IP address and port.
|
||||
// As a special case, packets routed through DERP use a destination address
|
||||
// of 127.3.3.40 with the port being the DERP region.
|
||||
// This node's tailscale IP address never appears in the returned map.
|
||||
func (c *Conn) ExtractStatistics() map[netlogtype.Connection]netlogtype.Counts {
|
||||
c.stats.mu.Lock()
|
||||
defer c.stats.mu.Unlock()
|
||||
m := c.stats.m
|
||||
c.stats.m = nil
|
||||
return m
|
||||
// SetStatistics specifies a per-connection statistics aggregator.
|
||||
// Nil may be specified to disable statistics gathering.
|
||||
func (c *Conn) SetStatistics(stats *connstats.Statistics) {
|
||||
c.stats.Store(stats)
|
||||
}
|
||||
|
||||
func ippDebugString(ua netip.AddrPort) string {
|
||||
@@ -3701,14 +3662,14 @@ func (de *endpoint) send(b []byte) error {
|
||||
var err error
|
||||
if udpAddr.IsValid() {
|
||||
_, err = de.c.sendAddr(udpAddr, de.publicKey, b)
|
||||
if err == nil && de.c.stats.enabled.Load() {
|
||||
de.c.updateStats(de.nodeAddr, udpAddr, netlogtype.Counts{TxPackets: 1, TxBytes: uint64(len(b))})
|
||||
if stats := de.c.stats.Load(); err == nil && stats != nil {
|
||||
stats.UpdateTxPhysical(de.nodeAddr, udpAddr, len(b))
|
||||
}
|
||||
}
|
||||
if derpAddr.IsValid() {
|
||||
if ok, _ := de.c.sendAddr(derpAddr, de.publicKey, b); ok {
|
||||
if de.c.stats.enabled.Load() {
|
||||
de.c.updateStats(de.nodeAddr, derpAddr, netlogtype.Counts{TxPackets: 1, TxBytes: uint64(len(b))})
|
||||
if stats := de.c.stats.Load(); stats != nil {
|
||||
stats.UpdateTxPhysical(de.nodeAddr, derpAddr, len(b))
|
||||
}
|
||||
if err != nil {
|
||||
// UDP failed but DERP worked, so good enough:
|
||||
|
@@ -35,6 +35,7 @@ import (
|
||||
"tailscale.com/derp/derphttp"
|
||||
"tailscale.com/disco"
|
||||
"tailscale.com/ipn/ipnstate"
|
||||
"tailscale.com/net/connstats"
|
||||
"tailscale.com/net/netaddr"
|
||||
"tailscale.com/net/stun/stuntest"
|
||||
"tailscale.com/net/tstun"
|
||||
@@ -133,6 +134,7 @@ func runDERPAndStun(t *testing.T, logf logger.Logf, l nettype.PacketListener, st
|
||||
type magicStack struct {
|
||||
privateKey key.NodePrivate
|
||||
epCh chan []tailcfg.Endpoint // endpoint updates produced by this peer
|
||||
stats connstats.Statistics // per-connection statistics
|
||||
conn *Conn // the magicsock itself
|
||||
tun *tuntest.ChannelTUN // TUN device to send/receive packets
|
||||
tsTun *tstun.Wrapper // wrapped tun that implements filtering and wgengine hooks
|
||||
@@ -1047,11 +1049,11 @@ func testTwoDevicePing(t *testing.T, d *devices) {
|
||||
}
|
||||
}
|
||||
|
||||
m1.conn.SetStatisticsEnabled(true)
|
||||
m2.conn.SetStatisticsEnabled(true)
|
||||
m1.conn.SetStatistics(&m1.stats)
|
||||
m2.conn.SetStatistics(&m2.stats)
|
||||
|
||||
checkStats := func(t *testing.T, m *magicStack, wantConns []netlogtype.Connection) {
|
||||
stats := m.conn.ExtractStatistics()
|
||||
_, stats := m.stats.Extract()
|
||||
for _, conn := range wantConns {
|
||||
if _, ok := stats[conn]; ok {
|
||||
return
|
||||
|
@@ -20,6 +20,7 @@ import (
|
||||
"golang.org/x/sync/errgroup"
|
||||
"tailscale.com/logpolicy"
|
||||
"tailscale.com/logtail"
|
||||
"tailscale.com/net/connstats"
|
||||
"tailscale.com/net/tsaddr"
|
||||
"tailscale.com/smallzstd"
|
||||
"tailscale.com/tailcfg"
|
||||
@@ -34,14 +35,12 @@ const pollPeriod = 5 * time.Second
|
||||
// *tstun.Wrapper implements this interface.
|
||||
// *magicsock.Conn implements this interface.
|
||||
type Device interface {
|
||||
SetStatisticsEnabled(bool)
|
||||
ExtractStatistics() map[netlogtype.Connection]netlogtype.Counts
|
||||
SetStatistics(*connstats.Statistics)
|
||||
}
|
||||
|
||||
type noopDevice struct{}
|
||||
|
||||
func (noopDevice) SetStatisticsEnabled(bool) {}
|
||||
func (noopDevice) ExtractStatistics() map[netlogtype.Connection]netlogtype.Counts { return nil }
|
||||
func (noopDevice) SetStatistics(*connstats.Statistics) {}
|
||||
|
||||
// Logger logs statistics about every connection.
|
||||
// At present, it only logs connections within a tailscale network.
|
||||
@@ -130,16 +129,15 @@ func (nl *Logger) Startup(nodeID tailcfg.StableNodeID, nodeLogID, domainLogID lo
|
||||
}, log.Printf)
|
||||
nl.logger = logger
|
||||
|
||||
stats := new(connstats.Statistics)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
nl.cancel = cancel
|
||||
nl.group.Go(func() error {
|
||||
tun.SetStatisticsEnabled(true)
|
||||
defer tun.SetStatisticsEnabled(false)
|
||||
tun.ExtractStatistics() // clear out any stale statistics
|
||||
tun.SetStatistics(stats)
|
||||
defer tun.SetStatistics(nil)
|
||||
|
||||
sock.SetStatisticsEnabled(true)
|
||||
defer sock.SetStatisticsEnabled(false)
|
||||
sock.ExtractStatistics() // clear out any stale statistics
|
||||
sock.SetStatistics(stats)
|
||||
defer sock.SetStatistics(nil)
|
||||
|
||||
start := time.Now()
|
||||
ticker := time.NewTicker(pollPeriod)
|
||||
@@ -147,22 +145,20 @@ func (nl *Logger) Startup(nodeID tailcfg.StableNodeID, nodeLogID, domainLogID lo
|
||||
var end time.Time
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
tun.SetStatisticsEnabled(false)
|
||||
end = time.Now()
|
||||
case end = <-ticker.C:
|
||||
}
|
||||
|
||||
// NOTE: tunStats and sockStats will always be slightly out-of-sync.
|
||||
// NOTE: connstats and sockStats will always be slightly out-of-sync.
|
||||
// It is impossible to have an atomic snapshot of statistics
|
||||
// at both layers without a global mutex that spans all layers.
|
||||
tunStats := tun.ExtractStatistics()
|
||||
sockStats := sock.ExtractStatistics()
|
||||
if len(tunStats)+len(sockStats) > 0 {
|
||||
connstats, sockStats := stats.Extract()
|
||||
if len(connstats)+len(sockStats) > 0 {
|
||||
nl.mu.Lock()
|
||||
addrs := nl.addrs
|
||||
prefixes := nl.prefixes
|
||||
nl.mu.Unlock()
|
||||
recordStatistics(logger, nodeID, start, end, tunStats, sockStats, addrs, prefixes)
|
||||
recordStatistics(logger, nodeID, start, end, connstats, sockStats, addrs, prefixes)
|
||||
}
|
||||
|
||||
if ctx.Err() != nil {
|
||||
@@ -175,7 +171,7 @@ func (nl *Logger) Startup(nodeID tailcfg.StableNodeID, nodeLogID, domainLogID lo
|
||||
return nil
|
||||
}
|
||||
|
||||
func recordStatistics(logger *logtail.Logger, nodeID tailcfg.StableNodeID, start, end time.Time, tunStats, sockStats map[netlogtype.Connection]netlogtype.Counts, addrs map[netip.Addr]bool, prefixes map[netip.Prefix]bool) {
|
||||
func recordStatistics(logger *logtail.Logger, nodeID tailcfg.StableNodeID, start, end time.Time, connstats, sockStats map[netlogtype.Connection]netlogtype.Counts, addrs map[netip.Addr]bool, prefixes map[netip.Prefix]bool) {
|
||||
m := netlogtype.Message{NodeID: nodeID, Start: start.UTC(), End: end.UTC()}
|
||||
|
||||
classifyAddr := func(a netip.Addr) (isTailscale, withinRoute bool) {
|
||||
@@ -194,7 +190,7 @@ func recordStatistics(logger *logtail.Logger, nodeID tailcfg.StableNodeID, start
|
||||
}
|
||||
|
||||
exitTraffic := make(map[netlogtype.Connection]netlogtype.Counts)
|
||||
for conn, cnts := range tunStats {
|
||||
for conn, cnts := range connstats {
|
||||
srcIsTailscaleIP, srcWithinSubnet := classifyAddr(conn.Src.Addr())
|
||||
dstIsTailscaleIP, dstWithinSubnet := classifyAddr(conn.Dst.Addr())
|
||||
switch {
|
||||
|
@@ -1,66 +0,0 @@
|
||||
// Copyright (c) 2022 Tailscale Inc & AUTHORS All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package netlog
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/http"
|
||||
"testing"
|
||||
|
||||
qt "github.com/frankban/quicktest"
|
||||
"tailscale.com/logtail"
|
||||
"tailscale.com/tstest"
|
||||
"tailscale.com/types/netlogtype"
|
||||
"tailscale.com/util/must"
|
||||
"tailscale.com/wgengine/router"
|
||||
)
|
||||
|
||||
func init() {
|
||||
testClient = &http.Client{Transport: &roundTripper}
|
||||
}
|
||||
|
||||
var roundTripper roundTripperFunc
|
||||
|
||||
type roundTripperFunc struct {
|
||||
F func(*http.Request) (*http.Response, error)
|
||||
}
|
||||
|
||||
func (f roundTripperFunc) RoundTrip(r *http.Request) (*http.Response, error) {
|
||||
return f.F(r)
|
||||
}
|
||||
|
||||
type fakeDevice struct {
|
||||
toggled int // even => disabled, odd => enabled
|
||||
}
|
||||
|
||||
func (d *fakeDevice) SetStatisticsEnabled(enable bool) {
|
||||
if enabled := d.toggled%2 == 1; enabled != enable {
|
||||
d.toggled++
|
||||
}
|
||||
|
||||
}
|
||||
func (fakeDevice) ExtractStatistics() map[netlogtype.Connection]netlogtype.Counts {
|
||||
// TODO(dsnet): Add a test that verifies that statistics are correctly
|
||||
// extracted from the device and uploaded. Unfortunately,
|
||||
// we can't reliably run this test until we fix http://go/oss/5856.
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestResourceCheck(t *testing.T) {
|
||||
roundTripper.F = func(r *http.Request) (*http.Response, error) {
|
||||
return &http.Response{StatusCode: 200}, nil
|
||||
}
|
||||
|
||||
c := qt.New(t)
|
||||
tstest.ResourceCheck(t)
|
||||
var l Logger
|
||||
var d fakeDevice
|
||||
for i := 0; i < 10; i++ {
|
||||
must.Do(l.Startup("", logtail.PrivateID{}, logtail.PrivateID{}, &d, nil))
|
||||
l.ReconfigRoutes(&router.Config{})
|
||||
must.Do(l.Shutdown(context.Background()))
|
||||
c.Assert(d.toggled, qt.Equals, 2*(i+1))
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user