mirror of
https://github.com/tailscale/tailscale.git
synced 2025-05-06 07:37:38 +00:00
net/connstats: enforce maximum number of connections (#6760)
The Tailscale logging service has a hard limit on the maximum log message size that can be accepted. We want to ensure that netlog messages never exceed this limit otherwise a client cannot transmit logs. Move the goroutine for periodically dumping netlog messages from wgengine/netlog to net/connstats. This allows net/connstats to manage when it dumps messages, either based on time or by size. Updates tailscale/corp#8427 Signed-off-by: Joe Tsai <joetsai@digital-static.net>
This commit is contained in:
parent
651e0d8aad
commit
d9df023e6f
@ -7,9 +7,12 @@
|
|||||||
package connstats
|
package connstats
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"context"
|
||||||
"net/netip"
|
"net/netip"
|
||||||
"sync"
|
"sync"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"golang.org/x/sync/errgroup"
|
||||||
"tailscale.com/net/packet"
|
"tailscale.com/net/packet"
|
||||||
"tailscale.com/types/netlogtype"
|
"tailscale.com/types/netlogtype"
|
||||||
)
|
)
|
||||||
@ -18,11 +21,64 @@ import (
|
|||||||
// All methods are safe for concurrent use.
|
// All methods are safe for concurrent use.
|
||||||
// The zero value is ready for use.
|
// The zero value is ready for use.
|
||||||
type Statistics struct {
|
type Statistics struct {
|
||||||
mu sync.Mutex
|
maxConns int // immutable once set
|
||||||
|
|
||||||
|
mu sync.Mutex
|
||||||
|
connCnts
|
||||||
|
|
||||||
|
connCntsCh chan connCnts
|
||||||
|
shutdownCtx context.Context
|
||||||
|
shutdown context.CancelFunc
|
||||||
|
group errgroup.Group
|
||||||
|
}
|
||||||
|
|
||||||
|
type connCnts struct {
|
||||||
|
start time.Time
|
||||||
|
end time.Time
|
||||||
virtual map[netlogtype.Connection]netlogtype.Counts
|
virtual map[netlogtype.Connection]netlogtype.Counts
|
||||||
physical map[netlogtype.Connection]netlogtype.Counts
|
physical map[netlogtype.Connection]netlogtype.Counts
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// NewStatistics creates a data structure for tracking connection statistics
|
||||||
|
// that periodically dumps the virtual and physical connection counts
|
||||||
|
// depending on whether the maxPeriod or maxConns is exceeded.
|
||||||
|
// The dump function is called from a single goroutine.
|
||||||
|
// Shutdown must be called to cleanup resources.
|
||||||
|
func NewStatistics(maxPeriod time.Duration, maxConns int, dump func(start, end time.Time, virtual, physical map[netlogtype.Connection]netlogtype.Counts)) *Statistics {
|
||||||
|
s := &Statistics{maxConns: maxConns}
|
||||||
|
s.connCntsCh = make(chan connCnts, 256)
|
||||||
|
s.shutdownCtx, s.shutdown = context.WithCancel(context.Background())
|
||||||
|
s.group.Go(func() error {
|
||||||
|
// TODO(joetsai): Using a ticker is problematic on mobile platforms
|
||||||
|
// where waking up a process every maxPeriod when there is no activity
|
||||||
|
// is a drain on battery life. Switch this instead to instead use
|
||||||
|
// a time.Timer that is triggered upon network activity.
|
||||||
|
ticker := new(time.Ticker)
|
||||||
|
if maxPeriod > 0 {
|
||||||
|
ticker := time.NewTicker(maxPeriod)
|
||||||
|
defer ticker.Stop()
|
||||||
|
}
|
||||||
|
|
||||||
|
for {
|
||||||
|
var cc connCnts
|
||||||
|
select {
|
||||||
|
case cc = <-s.connCntsCh:
|
||||||
|
case <-ticker.C:
|
||||||
|
cc = s.extract()
|
||||||
|
case <-s.shutdownCtx.Done():
|
||||||
|
cc = s.extract()
|
||||||
|
}
|
||||||
|
if len(cc.virtual)+len(cc.physical) > 0 && dump != nil {
|
||||||
|
dump(cc.start, cc.end, cc.virtual, cc.physical)
|
||||||
|
}
|
||||||
|
if s.shutdownCtx.Err() != nil {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
return s
|
||||||
|
}
|
||||||
|
|
||||||
// UpdateTxVirtual updates the counters for a transmitted IP packet
|
// UpdateTxVirtual updates the counters for a transmitted IP packet
|
||||||
// The source and destination of the packet directly correspond with
|
// The source and destination of the packet directly correspond with
|
||||||
// the source and destination in netlogtype.Connection.
|
// the source and destination in netlogtype.Connection.
|
||||||
@ -47,10 +103,10 @@ func (s *Statistics) updateVirtual(b []byte, receive bool) {
|
|||||||
|
|
||||||
s.mu.Lock()
|
s.mu.Lock()
|
||||||
defer s.mu.Unlock()
|
defer s.mu.Unlock()
|
||||||
if s.virtual == nil {
|
cnts, found := s.virtual[conn]
|
||||||
s.virtual = make(map[netlogtype.Connection]netlogtype.Counts)
|
if !found && !s.preInsertConn() {
|
||||||
|
return
|
||||||
}
|
}
|
||||||
cnts := s.virtual[conn]
|
|
||||||
if receive {
|
if receive {
|
||||||
cnts.RxPackets++
|
cnts.RxPackets++
|
||||||
cnts.RxBytes += uint64(len(b))
|
cnts.RxBytes += uint64(len(b))
|
||||||
@ -82,10 +138,10 @@ func (s *Statistics) updatePhysical(src netip.Addr, dst netip.AddrPort, n int, r
|
|||||||
|
|
||||||
s.mu.Lock()
|
s.mu.Lock()
|
||||||
defer s.mu.Unlock()
|
defer s.mu.Unlock()
|
||||||
if s.physical == nil {
|
cnts, found := s.physical[conn]
|
||||||
s.physical = make(map[netlogtype.Connection]netlogtype.Counts)
|
if !found && !s.preInsertConn() {
|
||||||
|
return
|
||||||
}
|
}
|
||||||
cnts := s.physical[conn]
|
|
||||||
if receive {
|
if receive {
|
||||||
cnts.RxPackets++
|
cnts.RxPackets++
|
||||||
cnts.RxBytes += uint64(n)
|
cnts.RxBytes += uint64(n)
|
||||||
@ -96,14 +152,57 @@ func (s *Statistics) updatePhysical(src netip.Addr, dst netip.AddrPort, n int, r
|
|||||||
s.physical[conn] = cnts
|
s.physical[conn] = cnts
|
||||||
}
|
}
|
||||||
|
|
||||||
// Extract extracts and resets the counters for all active connections.
|
// preInsertConn updates the maps to handle insertion of a new connection.
|
||||||
// It must be called periodically otherwise the memory used is unbounded.
|
// It reports false if insertion is not allowed (i.e., after shutdown).
|
||||||
func (s *Statistics) Extract() (virtual, physical map[netlogtype.Connection]netlogtype.Counts) {
|
func (s *Statistics) preInsertConn() bool {
|
||||||
|
// Check whether insertion of a new connection will exceed maxConns.
|
||||||
|
if len(s.virtual)+len(s.physical) == s.maxConns && s.maxConns > 0 {
|
||||||
|
// Extract the current statistics and send it to the serializer.
|
||||||
|
// Avoid blocking the network packet handling path.
|
||||||
|
select {
|
||||||
|
case s.connCntsCh <- s.extractLocked():
|
||||||
|
default:
|
||||||
|
// TODO(joetsai): Log that we are dropping an entire connCounts.
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Initialize the maps if nil.
|
||||||
|
if s.virtual == nil && s.physical == nil {
|
||||||
|
s.start = time.Now().UTC()
|
||||||
|
s.virtual = make(map[netlogtype.Connection]netlogtype.Counts)
|
||||||
|
s.physical = make(map[netlogtype.Connection]netlogtype.Counts)
|
||||||
|
}
|
||||||
|
|
||||||
|
return s.shutdownCtx.Err() == nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *Statistics) extract() connCnts {
|
||||||
s.mu.Lock()
|
s.mu.Lock()
|
||||||
defer s.mu.Unlock()
|
defer s.mu.Unlock()
|
||||||
virtual = s.virtual
|
return s.extractLocked()
|
||||||
s.virtual = make(map[netlogtype.Connection]netlogtype.Counts)
|
}
|
||||||
physical = s.physical
|
|
||||||
s.physical = make(map[netlogtype.Connection]netlogtype.Counts)
|
func (s *Statistics) extractLocked() connCnts {
|
||||||
return virtual, physical
|
if len(s.virtual)+len(s.physical) == 0 {
|
||||||
|
return connCnts{}
|
||||||
|
}
|
||||||
|
s.end = time.Now().UTC()
|
||||||
|
cc := s.connCnts
|
||||||
|
s.connCnts = connCnts{}
|
||||||
|
return cc
|
||||||
|
}
|
||||||
|
|
||||||
|
// TestExtract synchronously extracts the current network statistics map
|
||||||
|
// and resets the counters. This should only be used for testing purposes.
|
||||||
|
func (s *Statistics) TestExtract() (virtual, physical map[netlogtype.Connection]netlogtype.Counts) {
|
||||||
|
cc := s.extract()
|
||||||
|
return cc.virtual, cc.physical
|
||||||
|
}
|
||||||
|
|
||||||
|
// Shutdown performs a final flush of statistics.
|
||||||
|
// Statistics for any subsequent calls to Update will be dropped.
|
||||||
|
// It is safe to call Shutdown concurrently and repeatedly.
|
||||||
|
func (s *Statistics) Shutdown(context.Context) error {
|
||||||
|
s.shutdown()
|
||||||
|
return s.group.Wait()
|
||||||
}
|
}
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
package connstats
|
package connstats
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"context"
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"fmt"
|
"fmt"
|
||||||
"math/rand"
|
"math/rand"
|
||||||
@ -47,7 +48,20 @@ func testPacketV4(proto ipproto.Proto, srcAddr, dstAddr [4]byte, srcPort, dstPor
|
|||||||
func TestConcurrent(t *testing.T) {
|
func TestConcurrent(t *testing.T) {
|
||||||
c := qt.New(t)
|
c := qt.New(t)
|
||||||
|
|
||||||
var stats Statistics
|
const maxPeriod = 10 * time.Millisecond
|
||||||
|
const maxConns = 10
|
||||||
|
virtualAggregate := make(map[netlogtype.Connection]netlogtype.Counts)
|
||||||
|
stats := NewStatistics(maxPeriod, maxConns, func(start, end time.Time, virtual, physical map[netlogtype.Connection]netlogtype.Counts) {
|
||||||
|
c.Assert(start.IsZero(), qt.IsFalse)
|
||||||
|
c.Assert(end.IsZero(), qt.IsFalse)
|
||||||
|
c.Assert(end.Before(start), qt.IsFalse)
|
||||||
|
c.Assert(len(virtual) > 0 && len(virtual) <= maxConns, qt.IsTrue)
|
||||||
|
c.Assert(len(physical) == 0, qt.IsTrue)
|
||||||
|
for conn, cnts := range virtual {
|
||||||
|
virtualAggregate[conn] = virtualAggregate[conn].Add(cnts)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
defer stats.Shutdown(context.Background())
|
||||||
var wants []map[netlogtype.Connection]netlogtype.Counts
|
var wants []map[netlogtype.Connection]netlogtype.Counts
|
||||||
gots := make([]map[netlogtype.Connection]netlogtype.Counts, runtime.NumCPU())
|
gots := make([]map[netlogtype.Connection]netlogtype.Counts, runtime.NumCPU())
|
||||||
var group sync.WaitGroup
|
var group sync.WaitGroup
|
||||||
@ -95,14 +109,9 @@ func TestConcurrent(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}(i)
|
}(i)
|
||||||
}
|
}
|
||||||
for range gots {
|
|
||||||
virtual, _ := stats.Extract()
|
|
||||||
wants = append(wants, virtual)
|
|
||||||
time.Sleep(time.Millisecond)
|
|
||||||
}
|
|
||||||
group.Wait()
|
group.Wait()
|
||||||
virtual, _ := stats.Extract()
|
c.Assert(stats.Shutdown(context.Background()), qt.IsNil)
|
||||||
wants = append(wants, virtual)
|
wants = append(wants, virtualAggregate)
|
||||||
|
|
||||||
got := make(map[netlogtype.Connection]netlogtype.Counts)
|
got := make(map[netlogtype.Connection]netlogtype.Counts)
|
||||||
want := make(map[netlogtype.Connection]netlogtype.Counts)
|
want := make(map[netlogtype.Connection]netlogtype.Counts)
|
||||||
@ -126,7 +135,7 @@ func Benchmark(b *testing.B) {
|
|||||||
b.ResetTimer()
|
b.ResetTimer()
|
||||||
b.ReportAllocs()
|
b.ReportAllocs()
|
||||||
for i := 0; i < b.N; i++ {
|
for i := 0; i < b.N; i++ {
|
||||||
var s Statistics
|
s := NewStatistics(0, 0, nil)
|
||||||
for j := 0; j < 1e3; j++ {
|
for j := 0; j < 1e3; j++ {
|
||||||
s.UpdateTxVirtual(p)
|
s.UpdateTxVirtual(p)
|
||||||
}
|
}
|
||||||
@ -137,7 +146,7 @@ func Benchmark(b *testing.B) {
|
|||||||
b.ResetTimer()
|
b.ResetTimer()
|
||||||
b.ReportAllocs()
|
b.ReportAllocs()
|
||||||
for i := 0; i < b.N; i++ {
|
for i := 0; i < b.N; i++ {
|
||||||
var s Statistics
|
s := NewStatistics(0, 0, nil)
|
||||||
for j := 0; j < 1e3; j++ {
|
for j := 0; j < 1e3; j++ {
|
||||||
binary.BigEndian.PutUint32(p[20:], uint32(j)) // unique port combination
|
binary.BigEndian.PutUint32(p[20:], uint32(j)) // unique port combination
|
||||||
s.UpdateTxVirtual(p)
|
s.UpdateTxVirtual(p)
|
||||||
@ -149,7 +158,7 @@ func Benchmark(b *testing.B) {
|
|||||||
b.ResetTimer()
|
b.ResetTimer()
|
||||||
b.ReportAllocs()
|
b.ReportAllocs()
|
||||||
for i := 0; i < b.N; i++ {
|
for i := 0; i < b.N; i++ {
|
||||||
var s Statistics
|
s := NewStatistics(0, 0, nil)
|
||||||
var group sync.WaitGroup
|
var group sync.WaitGroup
|
||||||
for j := 0; j < runtime.NumCPU(); j++ {
|
for j := 0; j < runtime.NumCPU(); j++ {
|
||||||
group.Add(1)
|
group.Add(1)
|
||||||
@ -171,7 +180,7 @@ func Benchmark(b *testing.B) {
|
|||||||
b.ResetTimer()
|
b.ResetTimer()
|
||||||
b.ReportAllocs()
|
b.ReportAllocs()
|
||||||
for i := 0; i < b.N; i++ {
|
for i := 0; i < b.N; i++ {
|
||||||
var s Statistics
|
s := NewStatistics(0, 0, nil)
|
||||||
var group sync.WaitGroup
|
var group sync.WaitGroup
|
||||||
for j := 0; j < runtime.NumCPU(); j++ {
|
for j := 0; j < runtime.NumCPU(); j++ {
|
||||||
group.Add(1)
|
group.Add(1)
|
||||||
|
@ -6,15 +6,17 @@ package tstun
|
|||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"context"
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"fmt"
|
"fmt"
|
||||||
"net/netip"
|
"net/netip"
|
||||||
"reflect"
|
|
||||||
"strconv"
|
"strconv"
|
||||||
"strings"
|
"strings"
|
||||||
"testing"
|
"testing"
|
||||||
"unsafe"
|
"unsafe"
|
||||||
|
|
||||||
|
"github.com/google/go-cmp/cmp"
|
||||||
|
"github.com/google/go-cmp/cmp/cmpopts"
|
||||||
"github.com/tailscale/wireguard-go/tun/tuntest"
|
"github.com/tailscale/wireguard-go/tun/tuntest"
|
||||||
"go4.org/mem"
|
"go4.org/mem"
|
||||||
"go4.org/netipx"
|
"go4.org/netipx"
|
||||||
@ -337,7 +339,8 @@ func TestFilter(t *testing.T) {
|
|||||||
}()
|
}()
|
||||||
|
|
||||||
var buf [MaxPacketSize]byte
|
var buf [MaxPacketSize]byte
|
||||||
stats := new(connstats.Statistics)
|
stats := connstats.NewStatistics(0, 0, nil)
|
||||||
|
defer stats.Shutdown(context.Background())
|
||||||
tun.SetStatistics(stats)
|
tun.SetStatistics(stats)
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
@ -346,7 +349,7 @@ func TestFilter(t *testing.T) {
|
|||||||
var filtered bool
|
var filtered bool
|
||||||
sizes := make([]int, 1)
|
sizes := make([]int, 1)
|
||||||
|
|
||||||
tunStats, _ := stats.Extract()
|
tunStats, _ := stats.TestExtract()
|
||||||
if len(tunStats) > 0 {
|
if len(tunStats) > 0 {
|
||||||
t.Errorf("connstats.Statistics.Extract = %v, want {}", stats)
|
t.Errorf("connstats.Statistics.Extract = %v, want {}", stats)
|
||||||
}
|
}
|
||||||
@ -381,7 +384,7 @@ func TestFilter(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
got, _ := stats.Extract()
|
got, _ := stats.TestExtract()
|
||||||
want := map[netlogtype.Connection]netlogtype.Counts{}
|
want := map[netlogtype.Connection]netlogtype.Counts{}
|
||||||
if !tt.drop {
|
if !tt.drop {
|
||||||
var p packet.Parsed
|
var p packet.Parsed
|
||||||
@ -395,8 +398,8 @@ func TestFilter(t *testing.T) {
|
|||||||
want[conn] = netlogtype.Counts{TxPackets: 1, TxBytes: uint64(len(tt.data))}
|
want[conn] = netlogtype.Counts{TxPackets: 1, TxBytes: uint64(len(tt.data))}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if !reflect.DeepEqual(got, want) {
|
if diff := cmp.Diff(got, want, cmpopts.EquateEmpty()); diff != "" {
|
||||||
t.Errorf("tun.ExtractStatistics = %v, want %v", got, want)
|
t.Errorf("stats.TestExtract (-got +want):\n%s", diff)
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
@ -30,6 +30,15 @@ type Message struct {
|
|||||||
}
|
}
|
||||||
|
|
||||||
const (
|
const (
|
||||||
|
messageJSON = `{"nodeId":"n0123456789abcdefCNTRL",` + maxJSONTimeRange + `,` + minJSONTraffic + `}`
|
||||||
|
maxJSONTimeRange = `"start":` + maxJSONRFC3339 + `,"end":` + maxJSONRFC3339
|
||||||
|
maxJSONRFC3339 = `"0001-01-01T00:00:00.000000000Z"`
|
||||||
|
minJSONTraffic = `"virtualTraffic":{},"subnetTraffic":{},"exitTraffic":{},"physicalTraffic":{}`
|
||||||
|
|
||||||
|
// MaxMessageJSONSize is the overhead size of Message when it is
|
||||||
|
// serialized as JSON assuming that each traffic map is populated.
|
||||||
|
MaxMessageJSONSize = len(messageJSON)
|
||||||
|
|
||||||
maxJSONConnCounts = `{` + maxJSONConn + `,` + maxJSONCounts + `}`
|
maxJSONConnCounts = `{` + maxJSONConn + `,` + maxJSONCounts + `}`
|
||||||
maxJSONConn = `"proto":` + maxJSONProto + `,"src":` + maxJSONAddrPort + `,"dst":` + maxJSONAddrPort
|
maxJSONConn = `"proto":` + maxJSONProto + `,"src":` + maxJSONAddrPort + `,"dst":` + maxJSONAddrPort
|
||||||
maxJSONProto = `255`
|
maxJSONProto = `255`
|
||||||
|
@ -135,7 +135,7 @@ func runDERPAndStun(t *testing.T, logf logger.Logf, l nettype.PacketListener, st
|
|||||||
type magicStack struct {
|
type magicStack struct {
|
||||||
privateKey key.NodePrivate
|
privateKey key.NodePrivate
|
||||||
epCh chan []tailcfg.Endpoint // endpoint updates produced by this peer
|
epCh chan []tailcfg.Endpoint // endpoint updates produced by this peer
|
||||||
stats connstats.Statistics // per-connection statistics
|
stats *connstats.Statistics // per-connection statistics
|
||||||
conn *Conn // the magicsock itself
|
conn *Conn // the magicsock itself
|
||||||
tun *tuntest.ChannelTUN // TUN device to send/receive packets
|
tun *tuntest.ChannelTUN // TUN device to send/receive packets
|
||||||
tsTun *tstun.Wrapper // wrapped tun that implements filtering and wgengine hooks
|
tsTun *tstun.Wrapper // wrapped tun that implements filtering and wgengine hooks
|
||||||
@ -1053,11 +1053,15 @@ func testTwoDevicePing(t *testing.T, d *devices) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
m1.conn.SetStatistics(&m1.stats)
|
m1.stats = connstats.NewStatistics(0, 0, nil)
|
||||||
m2.conn.SetStatistics(&m2.stats)
|
defer m1.stats.Shutdown(context.Background())
|
||||||
|
m1.conn.SetStatistics(m1.stats)
|
||||||
|
m2.stats = connstats.NewStatistics(0, 0, nil)
|
||||||
|
defer m2.stats.Shutdown(context.Background())
|
||||||
|
m2.conn.SetStatistics(m2.stats)
|
||||||
|
|
||||||
checkStats := func(t *testing.T, m *magicStack, wantConns []netlogtype.Connection) {
|
checkStats := func(t *testing.T, m *magicStack, wantConns []netlogtype.Connection) {
|
||||||
_, stats := m.stats.Extract()
|
_, stats := m.stats.TestExtract()
|
||||||
for _, conn := range wantConns {
|
for _, conn := range wantConns {
|
||||||
if _, ok := stats[conn]; ok {
|
if _, ok := stats[conn]; ok {
|
||||||
return
|
return
|
||||||
|
@ -17,7 +17,6 @@ import (
|
|||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"golang.org/x/sync/errgroup"
|
|
||||||
"tailscale.com/logpolicy"
|
"tailscale.com/logpolicy"
|
||||||
"tailscale.com/logtail"
|
"tailscale.com/logtail"
|
||||||
"tailscale.com/net/connstats"
|
"tailscale.com/net/connstats"
|
||||||
@ -25,6 +24,7 @@ import (
|
|||||||
"tailscale.com/smallzstd"
|
"tailscale.com/smallzstd"
|
||||||
"tailscale.com/tailcfg"
|
"tailscale.com/tailcfg"
|
||||||
"tailscale.com/types/netlogtype"
|
"tailscale.com/types/netlogtype"
|
||||||
|
"tailscale.com/util/multierr"
|
||||||
"tailscale.com/wgengine/router"
|
"tailscale.com/wgengine/router"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -32,8 +32,7 @@ import (
|
|||||||
const pollPeriod = 5 * time.Second
|
const pollPeriod = 5 * time.Second
|
||||||
|
|
||||||
// Device is an abstraction over a tunnel device or a magic socket.
|
// Device is an abstraction over a tunnel device or a magic socket.
|
||||||
// *tstun.Wrapper implements this interface.
|
// Both *tstun.Wrapper and *magicsock.Conn implement this interface.
|
||||||
// *magicsock.Conn implements this interface.
|
|
||||||
type Device interface {
|
type Device interface {
|
||||||
SetStatistics(*connstats.Statistics)
|
SetStatistics(*connstats.Statistics)
|
||||||
}
|
}
|
||||||
@ -47,15 +46,15 @@ func (noopDevice) SetStatistics(*connstats.Statistics) {}
|
|||||||
// Exit node traffic is not logged for privacy reasons.
|
// Exit node traffic is not logged for privacy reasons.
|
||||||
// The zero value is ready for use.
|
// The zero value is ready for use.
|
||||||
type Logger struct {
|
type Logger struct {
|
||||||
mu sync.Mutex
|
mu sync.Mutex // protects all fields below
|
||||||
|
|
||||||
logger *logtail.Logger
|
logger *logtail.Logger
|
||||||
|
stats *connstats.Statistics
|
||||||
|
tun Device
|
||||||
|
sock Device
|
||||||
|
|
||||||
addrs map[netip.Addr]bool
|
addrs map[netip.Addr]bool
|
||||||
prefixes map[netip.Prefix]bool
|
prefixes map[netip.Prefix]bool
|
||||||
|
|
||||||
group errgroup.Group
|
|
||||||
cancel context.CancelFunc
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Running reports whether the logger is running.
|
// Running reports whether the logger is running.
|
||||||
@ -97,18 +96,13 @@ func (nl *Logger) Startup(nodeID tailcfg.StableNodeID, nodeLogID, domainLogID lo
|
|||||||
if nl.logger != nil {
|
if nl.logger != nil {
|
||||||
return fmt.Errorf("network logger already running for %v", nl.logger.PrivateID().Public())
|
return fmt.Errorf("network logger already running for %v", nl.logger.PrivateID().Public())
|
||||||
}
|
}
|
||||||
if tun == nil {
|
|
||||||
tun = noopDevice{}
|
|
||||||
}
|
|
||||||
if sock == nil {
|
|
||||||
sock = noopDevice{}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
// Startup a log stream to Tailscale's logging service.
|
||||||
httpc := &http.Client{Transport: logpolicy.NewLogtailTransport(logtail.DefaultHost)}
|
httpc := &http.Client{Transport: logpolicy.NewLogtailTransport(logtail.DefaultHost)}
|
||||||
if testClient != nil {
|
if testClient != nil {
|
||||||
httpc = testClient
|
httpc = testClient
|
||||||
}
|
}
|
||||||
logger := logtail.NewLogger(logtail.Config{
|
nl.logger = logtail.NewLogger(logtail.Config{
|
||||||
Collection: "tailtraffic.log.tailscale.io",
|
Collection: "tailtraffic.log.tailscale.io",
|
||||||
PrivateID: nodeLogID,
|
PrivateID: nodeLogID,
|
||||||
CopyPrivateID: domainLogID,
|
CopyPrivateID: domainLogID,
|
||||||
@ -127,47 +121,34 @@ func (nl *Logger) Startup(nodeID tailcfg.StableNodeID, nodeLogID, domainLogID lo
|
|||||||
IncludeProcID: true,
|
IncludeProcID: true,
|
||||||
IncludeProcSequence: true,
|
IncludeProcSequence: true,
|
||||||
}, log.Printf)
|
}, log.Printf)
|
||||||
nl.logger = logger
|
|
||||||
|
|
||||||
stats := new(connstats.Statistics)
|
// Startup a data structure to track per-connection statistics.
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
// There is a maximum size for individual log messages that logtail
|
||||||
nl.cancel = cancel
|
// can upload to the Tailscale log service, so stay below this limit.
|
||||||
nl.group.Go(func() error {
|
const maxLogSize = 256 << 10
|
||||||
tun.SetStatistics(stats)
|
const maxConns = (maxLogSize - netlogtype.MaxMessageJSONSize) / netlogtype.MaxConnectionCountsJSONSize
|
||||||
defer tun.SetStatistics(nil)
|
nl.stats = connstats.NewStatistics(pollPeriod, maxConns, func(start, end time.Time, virtual, physical map[netlogtype.Connection]netlogtype.Counts) {
|
||||||
|
nl.mu.Lock()
|
||||||
sock.SetStatistics(stats)
|
addrs := nl.addrs
|
||||||
defer sock.SetStatistics(nil)
|
prefixes := nl.prefixes
|
||||||
|
nl.mu.Unlock()
|
||||||
start := time.Now()
|
recordStatistics(nl.logger, nodeID, start, end, virtual, physical, addrs, prefixes)
|
||||||
ticker := time.NewTicker(pollPeriod)
|
|
||||||
for {
|
|
||||||
var end time.Time
|
|
||||||
select {
|
|
||||||
case <-ctx.Done():
|
|
||||||
end = time.Now()
|
|
||||||
case end = <-ticker.C:
|
|
||||||
}
|
|
||||||
|
|
||||||
// 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.
|
|
||||||
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, connstats, sockStats, addrs, prefixes)
|
|
||||||
}
|
|
||||||
|
|
||||||
if ctx.Err() != nil {
|
|
||||||
break
|
|
||||||
}
|
|
||||||
start = end.Add(time.Nanosecond)
|
|
||||||
}
|
|
||||||
return nil
|
|
||||||
})
|
})
|
||||||
|
|
||||||
|
// Register the connection tracker into the TUN device.
|
||||||
|
if tun == nil {
|
||||||
|
tun = noopDevice{}
|
||||||
|
}
|
||||||
|
nl.tun = tun
|
||||||
|
nl.tun.SetStatistics(nl.stats)
|
||||||
|
|
||||||
|
// Register the connection tracker into magicsock.
|
||||||
|
if sock == nil {
|
||||||
|
sock = noopDevice{}
|
||||||
|
}
|
||||||
|
nl.sock = sock
|
||||||
|
nl.sock.SetStatistics(nl.stats)
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -222,21 +203,8 @@ func recordStatistics(logger *logtail.Logger, nodeID tailcfg.StableNodeID, start
|
|||||||
}
|
}
|
||||||
|
|
||||||
if len(m.VirtualTraffic)+len(m.SubnetTraffic)+len(m.ExitTraffic)+len(m.PhysicalTraffic) > 0 {
|
if len(m.VirtualTraffic)+len(m.SubnetTraffic)+len(m.ExitTraffic)+len(m.PhysicalTraffic) > 0 {
|
||||||
// TODO(joetsai): Place a hard limit on the size of a network log message.
|
|
||||||
// The log server rejects any payloads above a certain size, so logging
|
|
||||||
// a message that large would cause logtail to be stuck forever trying
|
|
||||||
// and failing to upload the same excessively large payload.
|
|
||||||
//
|
|
||||||
// We should figure out the behavior for handling this. We could split
|
|
||||||
// the message apart so that there are multiple chunks with the same window,
|
|
||||||
// We could also consider reducing the granularity of the data
|
|
||||||
// by dropping port numbers.
|
|
||||||
const maxSize = 256 << 10
|
|
||||||
if b, err := json.Marshal(m); err != nil {
|
if b, err := json.Marshal(m); err != nil {
|
||||||
logger.Logf("json.Marshal error: %v", err)
|
logger.Logf("json.Marshal error: %v", err)
|
||||||
} else if len(b) > maxSize {
|
|
||||||
logger.Logf("JSON body too large: %dB (virtual:%d subnet:%d exit:%d physical:%d)",
|
|
||||||
len(b), len(m.VirtualTraffic), len(m.SubnetTraffic), len(m.ExitTraffic), len(m.PhysicalTraffic))
|
|
||||||
} else {
|
} else {
|
||||||
logger.Logf("%s", b)
|
logger.Logf("%s", b)
|
||||||
}
|
}
|
||||||
@ -285,15 +253,23 @@ func (nl *Logger) Shutdown(ctx context.Context) error {
|
|||||||
if nl.logger == nil {
|
if nl.logger == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
nl.cancel()
|
|
||||||
nl.mu.Unlock()
|
|
||||||
nl.group.Wait() // do not hold lock while waiting
|
|
||||||
nl.mu.Lock()
|
|
||||||
err := nl.logger.Shutdown(ctx)
|
|
||||||
|
|
||||||
|
// Shutdown in reverse order of Startup.
|
||||||
|
// Do not hold lock while shutting down since this may flush one last time.
|
||||||
|
nl.mu.Unlock()
|
||||||
|
nl.sock.SetStatistics(nil)
|
||||||
|
nl.tun.SetStatistics(nil)
|
||||||
|
err1 := nl.stats.Shutdown(ctx)
|
||||||
|
err2 := nl.logger.Shutdown(ctx)
|
||||||
|
nl.mu.Lock()
|
||||||
|
|
||||||
|
// Purge state.
|
||||||
nl.logger = nil
|
nl.logger = nil
|
||||||
|
nl.stats = nil
|
||||||
|
nl.tun = nil
|
||||||
|
nl.sock = nil
|
||||||
nl.addrs = nil
|
nl.addrs = nil
|
||||||
nl.prefixes = nil
|
nl.prefixes = nil
|
||||||
nl.cancel = nil
|
|
||||||
return err
|
return multierr.New(err1, err2)
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user