net/tstun, wgengine/magicsock: enable vectorized I/O on Linux (#6663)

This commit updates the wireguard-go dependency and implements the
necessary changes to the tun.Device and conn.Bind implementations to
support passing vectors of packets in tailscaled. This significantly
improves throughput performance on Linux.

Updates #414

Signed-off-by: Jordan Whited <jordan@tailscale.com>
Signed-off-by: James Tucker <james@tailscale.com>
Co-authored-by: James Tucker <james@tailscale.com>
This commit is contained in:
Jordan Whited
2022-12-08 17:58:14 -08:00
committed by GitHub
parent 389238fe4a
commit 76389d8baf
14 changed files with 777 additions and 298 deletions

View File

@@ -34,21 +34,22 @@ func (t *fakeTUN) Close() error {
return nil
}
func (t *fakeTUN) Read(out []byte, offset int) (int, error) {
func (t *fakeTUN) Read(out [][]byte, sizes []int, offset int) (int, error) {
<-t.closechan
return 0, io.EOF
}
func (t *fakeTUN) Write(b []byte, n int) (int, error) {
func (t *fakeTUN) Write(b [][]byte, n int) (int, error) {
select {
case <-t.closechan:
return 0, ErrClosed
default:
}
return len(b), nil
return 1, nil
}
func (t *fakeTUN) Flush() error { return nil }
func (t *fakeTUN) MTU() (int, error) { return 1500, nil }
func (t *fakeTUN) Name() (string, error) { return "FakeTUN", nil }
func (t *fakeTUN) Events() chan tun.Event { return t.evchan }
func (t *fakeTUN) Flush() error { return nil }
func (t *fakeTUN) MTU() (int, error) { return 1500, nil }
func (t *fakeTUN) Name() (string, error) { return "FakeTUN", nil }
func (t *fakeTUN) Events() <-chan tun.Event { return t.evchan }
func (t *fakeTUN) BatchSize() int { return 1 }

View File

@@ -12,6 +12,7 @@ import (
"net/netip"
"os"
"os/exec"
"sync"
"github.com/insomniacslk/dhcp/dhcpv4"
"golang.org/x/sys/unix"
@@ -23,6 +24,7 @@ import (
"tailscale.com/net/netaddr"
"tailscale.com/net/packet"
"tailscale.com/types/ipproto"
"tailscale.com/util/multierr"
)
// TODO: this was randomly generated once. Maybe do it per process start? But
@@ -69,13 +71,7 @@ func openDevice(fd int, tapName, bridgeName string) (tun.Device, error) {
}
}
// Also sets non-blocking I/O on fd when creating tun.Device.
dev, _, err := tun.CreateUnmonitoredTUNFromFD(fd) // TODO: MTU
if err != nil {
return nil, err
}
return dev, nil
return newTAPDevice(fd, tapName)
}
type etherType [2]byte
@@ -168,7 +164,8 @@ func (t *Wrapper) handleTAPFrame(ethBuf []byte) bool {
copy(res.HardwareAddressTarget(), req.HardwareAddressSender())
copy(res.ProtocolAddressTarget(), req.ProtocolAddressSender())
n, err := t.tdev.Write(buf, 0)
// TODO(raggi): reduce allocs!
n, err := t.tdev.Write([][]byte{buf}, 0)
if tapDebug {
t.logf("tap: wrote ARP reply %v, %v", n, err)
}
@@ -252,7 +249,9 @@ func (t *Wrapper) handleDHCPRequest(ethBuf []byte) bool {
netip.AddrPortFrom(netaddr.IPv4(100, 100, 100, 100), 67), // src
netip.AddrPortFrom(netaddr.IPv4(255, 255, 255, 255), 68), // dst
)
n, err := t.tdev.Write(pkt, 0)
// TODO(raggi): reduce allocs!
n, err := t.tdev.Write([][]byte{pkt}, 0)
if tapDebug {
t.logf("tap: wrote DHCP OFFER %v, %v", n, err)
}
@@ -279,7 +278,8 @@ func (t *Wrapper) handleDHCPRequest(ethBuf []byte) bool {
netip.AddrPortFrom(netaddr.IPv4(100, 100, 100, 100), 67), // src
netip.AddrPortFrom(netaddr.IPv4(255, 255, 255, 255), 68), // dst
)
n, err := t.tdev.Write(pkt, 0)
// TODO(raggi): reduce allocs!
n, err := t.tdev.Write([][]byte{pkt}, 0)
if tapDebug {
t.logf("tap: wrote DHCP ACK %v, %v", n, err)
}
@@ -346,21 +346,108 @@ func (t *Wrapper) destMAC() [6]byte {
return t.destMACAtomic.Load()
}
func (t *Wrapper) tapWrite(buf []byte, offset int) (int, error) {
if offset < ethernetFrameSize {
return 0, fmt.Errorf("[unexpected] weird offset %d for TAP write", offset)
func newTAPDevice(fd int, tapName string) (tun.Device, error) {
err := unix.SetNonblock(fd, true)
if err != nil {
return nil, err
}
eth := buf[offset-ethernetFrameSize:]
dst := t.destMAC()
copy(eth[:6], dst[:])
copy(eth[6:12], ourMAC[:])
et := etherTypeIPv4
if buf[offset]>>4 == 6 {
et = etherTypeIPv6
file := os.NewFile(uintptr(fd), "/dev/tap")
d := &tapDevice{
file: file,
events: make(chan tun.Event),
name: tapName,
}
eth[12], eth[13] = et[0], et[1]
if tapDebug {
t.logf("tap: tapWrite off=%v % x", offset, buf)
}
return t.tdev.Write(buf, offset-ethernetFrameSize)
return d, nil
}
var (
_ setWrapperer = &tapDevice{}
)
type tapDevice struct {
file *os.File
events chan tun.Event
name string
wrapper *Wrapper
closeOnce sync.Once
}
func (t *tapDevice) setWrapper(wrapper *Wrapper) {
t.wrapper = wrapper
}
func (t *tapDevice) File() *os.File {
return t.file
}
func (t *tapDevice) Name() (string, error) {
return t.name, nil
}
func (t *tapDevice) Read(buffs [][]byte, sizes []int, offset int) (int, error) {
n, err := t.file.Read(buffs[0][offset:])
if err != nil {
return 0, err
}
sizes[0] = n
return 1, nil
}
func (t *tapDevice) Write(buffs [][]byte, offset int) (int, error) {
errs := make([]error, 0)
wrote := 0
for _, buff := range buffs {
if offset < ethernetFrameSize {
errs = append(errs, fmt.Errorf("[unexpected] weird offset %d for TAP write", offset))
return 0, multierr.New(errs...)
}
eth := buff[offset-ethernetFrameSize:]
dst := t.wrapper.destMAC()
copy(eth[:6], dst[:])
copy(eth[6:12], ourMAC[:])
et := etherTypeIPv4
if buff[offset]>>4 == 6 {
et = etherTypeIPv6
}
eth[12], eth[13] = et[0], et[1]
if tapDebug {
t.wrapper.logf("tap: tapWrite off=%v % x", offset, buff)
}
_, err := t.file.Write(buff[offset-ethernetFrameSize:])
if err != nil {
errs = append(errs, err)
} else {
wrote++
}
}
return wrote, multierr.New(errs...)
}
func (t *tapDevice) MTU() (int, error) {
ifr, err := unix.NewIfreq(t.name)
if err != nil {
return 0, err
}
err = unix.IoctlIfreq(int(t.file.Fd()), unix.SIOCGIFMTU, ifr)
if err != nil {
return 0, err
}
return int(ifr.Uint32()), nil
}
func (t *tapDevice) Events() <-chan tun.Event {
return t.events
}
func (t *tapDevice) Close() error {
var err error
t.closeOnce.Do(func() {
close(t.events)
err = t.file.Close()
})
return err
}
func (t *tapDevice) BatchSize() int {
return 1
}

View File

@@ -6,5 +6,4 @@
package tstun
func (*Wrapper) handleTAPFrame([]byte) bool { panic("unreachable") }
func (*Wrapper) tapWrite([]byte, int) (int, error) { panic("unreachable") }
func (*Wrapper) handleTAPFrame([]byte) bool { panic("unreachable") }

View File

@@ -25,6 +25,7 @@ var createTAP func(tapName, bridgeName string) (tun.Device, error)
// New returns a tun.Device for the requested device name, along with
// the OS-dependent name that was allocated to the device.
func New(logf logger.Logf, tunName string) (tun.Device, string, error) {
var disableTUNOffload = envknob.Bool("TS_DISABLE_TUN_OFFLOAD")
var dev tun.Device
var err error
if strings.HasPrefix(tunName, "tap:") {
@@ -51,6 +52,11 @@ func New(logf logger.Logf, tunName string) (tun.Device, string, error) {
tunMTU = mtu
}
dev, err = tun.CreateTUN(tunName, tunMTU)
if err == nil && disableTUNOffload {
if do, ok := dev.(tun.DisableOffloader); ok {
do.DisableOffload()
}
}
}
if err != nil {
return nil, "", err

View File

@@ -88,25 +88,31 @@ type Wrapper struct {
destMACAtomic syncs.AtomicValue[[6]byte]
discoKey syncs.AtomicValue[key.DiscoPublic]
// buffer stores the oldest unconsumed packet from tdev.
// It is made a static buffer in order to avoid allocations.
buffer [maxBufferSize]byte
// bufferConsumedMu protects bufferConsumed from concurrent sends and closes.
// It does not prevent send-after-close, only data races.
// vectorBuffer stores the oldest unconsumed packet vector from tdev. It is
// allocated in wrap() and the underlying arrays should never grow.
vectorBuffer [][]byte
// bufferConsumedMu protects bufferConsumed from concurrent sends, closes,
// and send-after-close (by way of bufferConsumedClosed).
bufferConsumedMu sync.Mutex
// bufferConsumed synchronizes access to buffer (shared by Read and poll).
// bufferConsumedClosed is true when bufferConsumed has been closed. This is
// read by bufferConsumed writers to prevent send-after-close.
bufferConsumedClosed bool
// bufferConsumed synchronizes access to vectorBuffer (shared by Read() and
// pollVector()).
//
// Close closes bufferConsumed. There may be outstanding sends to bufferConsumed
// when that happens; we catch any resulting panics.
// This lets us avoid expensive multi-case selects.
// Close closes bufferConsumed and sets bufferConsumedClosed to true.
bufferConsumed chan struct{}
// closed signals poll (by closing) when the device is closed.
closed chan struct{}
// outboundMu protects outbound from concurrent sends and closes.
// It does not prevent send-after-close, only data races.
// outboundMu protects outbound and vectorOutbound from concurrent sends,
// closes, and send-after-close (by way of outboundClosed).
outboundMu sync.Mutex
// outbound is the queue by which packets leave the TUN device.
// outboundClosed is true when outbound or vectorOutbound have been closed.
// This is read by outbound and vectorOutbound writers to prevent
// send-after-close.
outboundClosed bool
// vectorOutbound is the queue by which packets leave the TUN device.
//
// The directions are relative to the network, not the device:
// inbound packets arrive via UDP and are written into the TUN device;
@@ -115,12 +121,10 @@ type Wrapper struct {
// the other direction must wait on a WireGuard goroutine to poll it.
//
// Empty reads are skipped by WireGuard, so it is always legal
// to discard an empty packet instead of sending it through t.outbound.
// to discard an empty packet instead of sending it through vectorOutbound.
//
// Close closes outbound. There may be outstanding sends to outbound
// when that happens; we catch any resulting panics.
// This lets us avoid expensive multi-case selects.
outbound chan tunReadResult
// Close closes vectorOutbound and sets outboundClosed to true.
vectorOutbound chan tunVectorReadResult
// eventsUpDown yields up and down tun.Events that arrive on a Wrapper's events channel.
eventsUpDown chan tun.Event
@@ -172,19 +176,30 @@ type Wrapper struct {
stats atomic.Pointer[connstats.Statistics]
}
// tunReadResult is the result of a TUN read, or an injected result pretending to be a TUN read.
// The data is not interpreted in the usual way for a Read method.
// See the comment in the middle of Wrap.Read.
type tunReadResult struct {
// Only one of err, packet or data should be set, and are read in that order
// of precedence.
err error
// tunInjectedRead is an injected packet pretending to be a tun.Read().
type tunInjectedRead struct {
// Only one of packet or data should be set, and are read in that order of
// precedence.
packet *stack.PacketBuffer
data []byte
}
// injected is set if the read result was generated internally, and contained packets should not
// pass through filters.
injected bool
// tunVectorReadResult is the result of a tun.Read(), or an injected packet
// pretending to be a tun.Read().
type tunVectorReadResult struct {
// Only one of err, data, or injected should be set, and are read in that
// order of precedence.
err error
data [][]byte
injected tunInjectedRead
dataOffset int
}
type setWrapperer interface {
// setWrapper enables the underlying TUN/TAP to have access to the Wrapper.
// It MUST be called only once during initialization, other usage is unsafe.
setWrapper(*Wrapper)
}
func WrapTAP(logf logger.Logf, tdev tun.Device) *Wrapper {
@@ -197,7 +212,7 @@ func Wrap(logf logger.Logf, tdev tun.Device) *Wrapper {
func wrap(logf logger.Logf, tdev tun.Device, isTAP bool) *Wrapper {
logf = logger.WithPrefix(logf, "tstun: ")
tun := &Wrapper{
w := &Wrapper{
logf: logf,
limitedLogf: logger.RateLimitedFn(logf, 1*time.Minute, 2, 10),
isTAP: isTAP,
@@ -206,21 +221,30 @@ func wrap(logf logger.Logf, tdev tun.Device, isTAP bool) *Wrapper {
// a goroutine should not block when setting it, even with no listeners.
bufferConsumed: make(chan struct{}, 1),
closed: make(chan struct{}),
// outbound can be unbuffered; the buffer is an optimization.
outbound: make(chan tunReadResult, 1),
eventsUpDown: make(chan tun.Event),
eventsOther: make(chan tun.Event),
// vectorOutbound can be unbuffered; the buffer is an optimization.
vectorOutbound: make(chan tunVectorReadResult, 1),
eventsUpDown: make(chan tun.Event),
eventsOther: make(chan tun.Event),
// TODO(dmytro): (highly rate-limited) hexdumps should happen on unknown packets.
filterFlags: filter.LogAccepts | filter.LogDrops,
}
go tun.poll()
go tun.pumpEvents()
// The buffer starts out consumed.
tun.bufferConsumed <- struct{}{}
tun.noteActivity()
w.vectorBuffer = make([][]byte, tdev.BatchSize())
for i := range w.vectorBuffer {
w.vectorBuffer[i] = make([]byte, maxBufferSize)
}
go w.pollVector()
return tun
go w.pumpEvents()
// The buffer starts out consumed.
w.bufferConsumed <- struct{}{}
w.noteActivity()
if sw, ok := w.tdev.(setWrapperer); ok {
sw.setWrapper(w)
}
return w
}
// SetDestIPActivityFuncs sets a map of funcs to run per packet
@@ -261,10 +285,12 @@ func (t *Wrapper) Close() error {
t.closeOnce.Do(func() {
close(t.closed)
t.bufferConsumedMu.Lock()
t.bufferConsumedClosed = true
close(t.bufferConsumed)
t.bufferConsumedMu.Unlock()
t.outboundMu.Lock()
close(t.outbound)
t.outboundClosed = true
close(t.vectorOutbound)
t.outboundMu.Unlock()
err = t.tdev.Close()
})
@@ -323,7 +349,7 @@ func (t *Wrapper) EventsUpDown() chan tun.Event {
// Events returns a TUN event channel that contains all non-Up, non-Down events.
// It is named Events because it is the set of events that we want to expose to wireguard-go,
// and Events is the name specified by the wireguard-go tun.Device interface.
func (t *Wrapper) Events() chan tun.Event {
func (t *Wrapper) Events() <-chan tun.Event {
return t.eventsOther
}
@@ -331,10 +357,6 @@ func (t *Wrapper) File() *os.File {
return t.tdev.File()
}
func (t *Wrapper) Flush() error {
return t.tdev.Flush()
}
func (t *Wrapper) MTU() (int, error) {
return t.tdev.MTU()
}
@@ -343,94 +365,95 @@ func (t *Wrapper) Name() (string, error) {
return t.tdev.Name()
}
// allowSendOnClosedChannel suppresses panics due to sending on a closed channel.
// This allows us to avoid synchronization between poll and Close.
// Such synchronization (particularly multi-case selects) is too expensive
// for code like poll or Read that is on the hot path of every packet.
// If this makes you sad or angry, you may want to join our
// weekly Go Performance Delinquents Anonymous meetings on Monday nights.
func allowSendOnClosedChannel() {
r := recover()
if r == nil {
return
}
e, _ := r.(error)
if e != nil && e.Error() == "send on closed channel" {
return
}
panic(r)
}
const ethernetFrameSize = 14 // 2 six byte MACs, 2 bytes ethertype
// poll polls t.tdev.Read, placing the oldest unconsumed packet into t.buffer.
// This is needed because t.tdev.Read in general may block (it does on Windows),
// so packets may be stuck in t.outbound if t.Read called t.tdev.Read directly.
func (t *Wrapper) poll() {
// pollVector polls t.tdev.Read(), placing the oldest unconsumed packet vector
// into t.vectorBuffer. This is needed because t.tdev.Read() in general may
// block (it does on Windows), so packets may be stuck in t.vectorOutbound if
// t.Read() called t.tdev.Read() directly.
func (t *Wrapper) pollVector() {
sizes := make([]int, len(t.vectorBuffer))
readOffset := PacketStartOffset
if t.isTAP {
readOffset = PacketStartOffset - ethernetFrameSize
}
for range t.bufferConsumed {
DoRead:
for i := range t.vectorBuffer {
t.vectorBuffer[i] = t.vectorBuffer[i][:cap(t.vectorBuffer[i])]
}
var n int
var err error
// Read may use memory in t.buffer before PacketStartOffset for mandatory headers.
// This is the rationale behind the tun.Wrapper.{Read,Write} interfaces
// and the reason t.buffer has size MaxMessageSize and not MaxContentSize.
// In principle, read errors are not fatal (but wireguard-go disagrees).
// We loop here until we get a non-empty (or failed) read.
// We don't need this loop for correctness,
// but wireguard-go will skip an empty read,
// so we might as well avoid the send through t.outbound.
for n == 0 && err == nil {
if t.isClosed() {
return
}
if t.isTAP {
n, err = t.tdev.Read(t.buffer[:], PacketStartOffset-ethernetFrameSize)
if tapDebug {
s := fmt.Sprintf("% x", t.buffer[:])
for strings.HasSuffix(s, " 00") {
s = strings.TrimSuffix(s, " 00")
}
t.logf("TAP read %v, %v: %s", n, err, s)
n, err = t.tdev.Read(t.vectorBuffer[:], sizes, readOffset)
if t.isTAP && tapDebug {
s := fmt.Sprintf("% x", t.vectorBuffer[0][:])
for strings.HasSuffix(s, " 00") {
s = strings.TrimSuffix(s, " 00")
}
} else {
n, err = t.tdev.Read(t.buffer[:], PacketStartOffset)
t.logf("TAP read %v, %v: %s", n, err, s)
}
}
for i := range sizes[:n] {
t.vectorBuffer[i] = t.vectorBuffer[i][:readOffset+sizes[i]]
}
if t.isTAP {
if err == nil {
ethernetFrame := t.buffer[PacketStartOffset-ethernetFrameSize:][:n]
ethernetFrame := t.vectorBuffer[0][readOffset:]
if t.handleTAPFrame(ethernetFrame) {
goto DoRead
}
}
// Fall through. We got an IP packet.
if n >= ethernetFrameSize {
n -= ethernetFrameSize
if sizes[0] >= ethernetFrameSize {
t.vectorBuffer[0] = t.vectorBuffer[0][:readOffset+sizes[0]-ethernetFrameSize]
}
if tapDebug {
t.logf("tap regular frame: %x", t.buffer[PacketStartOffset:PacketStartOffset+n])
t.logf("tap regular frame: %x", t.vectorBuffer[0][PacketStartOffset:PacketStartOffset+sizes[0]])
}
}
t.sendOutbound(tunReadResult{data: t.buffer[PacketStartOffset : PacketStartOffset+n], err: err})
t.sendVectorOutbound(tunVectorReadResult{
data: t.vectorBuffer[:n],
dataOffset: PacketStartOffset,
err: err,
})
}
}
// sendBufferConsumed does t.bufferConsumed <- struct{}{}.
// It protects against any panics or data races that that send could cause.
func (t *Wrapper) sendBufferConsumed() {
defer allowSendOnClosedChannel()
t.bufferConsumedMu.Lock()
defer t.bufferConsumedMu.Unlock()
if t.bufferConsumedClosed {
return
}
t.bufferConsumed <- struct{}{}
}
// sendOutbound does t.outboundMu <- r.
// It protects against any panics or data races that that send could cause.
func (t *Wrapper) sendOutbound(r tunReadResult) {
defer allowSendOnClosedChannel()
// injectOutbound does t.vectorOutbound <- r
func (t *Wrapper) injectOutbound(r tunInjectedRead) {
t.outboundMu.Lock()
defer t.outboundMu.Unlock()
t.outbound <- r
if t.outboundClosed {
return
}
t.vectorOutbound <- tunVectorReadResult{
injected: r,
}
}
// sendVectorOutbound does t.vectorOutbound <- r.
func (t *Wrapper) sendVectorOutbound(r tunVectorReadResult) {
t.outboundMu.Lock()
defer t.outboundMu.Unlock()
if t.outboundClosed {
return
}
t.vectorOutbound <- r
}
var (
@@ -514,34 +537,79 @@ func (t *Wrapper) IdleDuration() time.Duration {
return mono.Since(t.lastActivityAtomic.LoadAtomic())
}
func (t *Wrapper) Read(buf []byte, offset int) (int, error) {
res, ok := <-t.outbound
func (t *Wrapper) Read(buffs [][]byte, sizes []int, offset int) (int, error) {
res, ok := <-t.vectorOutbound
if !ok {
// Wrapper is closed.
return 0, io.EOF
}
if res.err != nil {
return 0, res.err
}
if res.data == nil {
n, err := t.injectedRead(res.injected, buffs[0], offset)
sizes[0] = n
if err != nil && n == 0 {
return 0, err
}
return 1, err
}
metricPacketOut.Add(int64(len(res.data)))
var buffsPos int
for _, data := range res.data {
p := parsedPacketPool.Get().(*packet.Parsed)
defer parsedPacketPool.Put(p)
p.Decode(data[res.dataOffset:])
if m := t.destIPActivity.Load(); m != nil {
if fn := m[p.Dst.Addr()]; fn != nil {
fn()
}
}
if !t.disableFilter {
response := t.filterOut(p)
if response != filter.Accept {
metricPacketOutDrop.Add(1)
continue
}
}
n := copy(buffs[buffsPos][offset:], data[res.dataOffset:])
if n != len(data)-res.dataOffset {
panic(fmt.Sprintf("short copy: %d != %d", n, len(data)-res.dataOffset))
}
sizes[buffsPos] = n
if stats := t.stats.Load(); stats != nil {
stats.UpdateTxVirtual(data[res.dataOffset:])
}
buffsPos++
}
// t.vectorBuffer has a fixed location in memory.
// TODO(raggi): add an explicit field and possibly method to the tunVectorReadResult
// to signal when sendBufferConsumed should be called.
if &res.data[0] == &t.vectorBuffer[0] {
// We are done with t.buffer. Let poll() re-use it.
t.sendBufferConsumed()
}
t.noteActivity()
return buffsPos, nil
}
// injectedRead handles injected reads, which bypass filters.
func (t *Wrapper) injectedRead(res tunInjectedRead, buf []byte, offset int) (int, error) {
metricPacketOut.Add(1)
var n int
if res.packet != nil {
n = copy(buf[offset:], res.packet.NetworkHeader().Slice())
n += copy(buf[offset+n:], res.packet.TransportHeader().Slice())
n += copy(buf[offset+n:], res.packet.Data().AsRange().ToSlice())
res.packet.DecRef()
} else {
n = copy(buf[offset:], res.data)
// t.buffer has a fixed location in memory.
if &res.data[0] == &t.buffer[PacketStartOffset] {
// We are done with t.buffer. Let poll re-use it.
t.sendBufferConsumed()
}
}
p := parsedPacketPool.Get().(*packet.Parsed)
@@ -554,16 +622,6 @@ func (t *Wrapper) Read(buf []byte, offset int) (int, error) {
}
}
// Do not filter injected packets.
if !res.injected && !t.disableFilter {
response := t.filterOut(p)
if response != filter.Accept {
metricPacketOutDrop.Add(1)
// WireGuard considers read errors fatal; pretend nothing was read
return 0, nil
}
}
if stats := t.stats.Load(); stats != nil {
stats.UpdateTxVirtual(buf[offset:][:n])
}
@@ -668,42 +726,40 @@ func (t *Wrapper) filterIn(buf []byte) filter.Response {
return filter.Accept
}
// Write accepts an incoming packet. The packet begins at buf[offset:],
// Write accepts incoming packets. The packets begins at buffs[:][offset:],
// like wireguard-go/tun.Device.Write.
func (t *Wrapper) Write(buf []byte, offset int) (int, error) {
metricPacketIn.Add(1)
func (t *Wrapper) Write(buffs [][]byte, offset int) (int, error) {
metricPacketIn.Add(int64(len(buffs)))
i := 0
if !t.disableFilter {
if t.filterIn(buf[offset:]) != filter.Accept {
metricPacketInDrop.Add(1)
// If we're not accepting the packet, lie to wireguard-go and pretend
// that everything is okay with a nil error, so wireguard-go
// doesn't log about this Write "failure".
//
// We return len(buf), but the ill-defined wireguard-go/tun.Device.Write
// method doesn't specify how the offset affects the return value.
// In fact, the Linux implementation does one of two different things depending
// on how the /dev/net/tun was created. But fortunately the wireguard-go
// code ignores the int return and only looks at the error:
//
// device/receive.go: _, err = device.tun.device.Write(....)
//
// TODO(bradfitz): fix upstream interface docs, implementation.
return len(buf), nil
for _, buff := range buffs {
if t.filterIn(buff[offset:]) != filter.Accept {
metricPacketInDrop.Add(1)
} else {
buffs[i] = buff
i++
}
}
} else {
i = len(buffs)
}
buffs = buffs[:i]
t.noteActivity()
return t.tdevWrite(buf, offset)
if len(buffs) > 0 {
t.noteActivity()
_, err := t.tdevWrite(buffs, offset)
return len(buffs), err
}
return 0, nil
}
func (t *Wrapper) tdevWrite(buf []byte, offset int) (int, error) {
func (t *Wrapper) tdevWrite(buffs [][]byte, offset int) (int, error) {
if stats := t.stats.Load(); stats != nil {
stats.UpdateRxVirtual(buf[offset:])
for i := range buffs {
stats.UpdateRxVirtual((buffs)[i][offset:])
}
}
if t.isTAP {
return t.tapWrite(buf, offset)
}
return t.tdev.Write(buf, offset)
return t.tdev.Write(buffs, offset)
}
func (t *Wrapper) GetFilter() *filter.Filter {
@@ -755,7 +811,7 @@ func (t *Wrapper) InjectInboundDirect(buf []byte, offset int) error {
}
// Write to the underlying device to skip filters.
_, err := t.tdevWrite(buf, offset)
_, err := t.tdevWrite([][]byte{buf}, offset) // TODO(jwhited): alloc?
return err
}
@@ -813,7 +869,7 @@ func (t *Wrapper) InjectOutbound(packet []byte) error {
if len(packet) == 0 {
return nil
}
t.sendOutbound(tunReadResult{data: packet, injected: true})
t.injectOutbound(tunInjectedRead{data: packet})
return nil
}
@@ -830,10 +886,14 @@ func (t *Wrapper) InjectOutboundPacketBuffer(packet *stack.PacketBuffer) error {
packet.DecRef()
return nil
}
t.sendOutbound(tunReadResult{packet: packet, injected: true})
t.injectOutbound(tunInjectedRead{packet: packet})
return nil
}
func (t *Wrapper) BatchSize() int {
return t.tdev.BatchSize()
}
// Unwrap returns the underlying tun.Device.
func (t *Wrapper) Unwrap() tun.Device {
return t.tdev

View File

@@ -208,16 +208,24 @@ func TestReadAndInject(t *testing.T) {
var buf [MaxPacketSize]byte
var seen = make(map[string]bool)
sizes := make([]int, 1)
// We expect the same packets back, in no particular order.
for i := 0; i < len(written)+len(injected); i++ {
n, err := tun.Read(buf[:], 0)
packet := buf[:]
buffs := [][]byte{packet}
numPackets, err := tun.Read(buffs, sizes, 0)
if err != nil {
t.Errorf("read %d: error: %v", i, err)
}
if n != size {
t.Errorf("read %d: got size %d; want %d", i, n, size)
if numPackets != 1 {
t.Fatalf("read %d packets, expected %d", numPackets, 1)
}
got := string(buf[:n])
packet = packet[:sizes[0]]
packetLen := len(packet)
if packetLen != size {
t.Errorf("read %d: got size %d; want %d", i, packetLen, size)
}
got := string(packet)
t.Logf("read %d: got %s", i, got)
seen[got] = true
}
@@ -245,13 +253,10 @@ func TestWriteAndInject(t *testing.T) {
go func() {
for _, packet := range written {
payload := []byte(packet)
n, err := tun.Write(payload, 0)
_, err := tun.Write([][]byte{payload}, 0)
if err != nil {
t.Errorf("%s: error: %v", packet, err)
}
if n != size {
t.Errorf("%s: got size %d; want %d", packet, n, size)
}
}
}()
@@ -339,6 +344,7 @@ func TestFilter(t *testing.T) {
var n int
var err error
var filtered bool
sizes := make([]int, 1)
tunStats, _ := stats.Extract()
if len(tunStats) > 0 {
@@ -352,11 +358,11 @@ func TestFilter(t *testing.T) {
// If it stays zero, nothing made it through
// to the wrapped TUN.
tun.lastActivityAtomic.StoreAtomic(0)
_, err = tun.Write(tt.data, 0)
_, err = tun.Write([][]byte{tt.data}, 0)
filtered = tun.lastActivityAtomic.LoadAtomic() == 0
} else {
chtun.Outbound <- tt.data
n, err = tun.Read(buf[:], 0)
n, err = tun.Read([][]byte{buf[:]}, sizes, 0)
// In the read direction, errors are fatal, so we return n = 0 instead.
filtered = (n == 0)
}
@@ -400,7 +406,7 @@ func TestAllocs(t *testing.T) {
ftun, tun := newFakeTUN(t.Logf, false)
defer tun.Close()
buf := []byte{0x00}
buf := [][]byte{[]byte{0x00}}
err := tstest.MinAllocsPerRun(t, 0, func() {
_, err := ftun.Write(buf, 0)
if err != nil {
@@ -417,7 +423,7 @@ func TestAllocs(t *testing.T) {
func TestClose(t *testing.T) {
ftun, tun := newFakeTUN(t.Logf, false)
data := udp4("1.2.3.4", "5.6.7.8", 98, 98)
data := [][]byte{udp4("1.2.3.4", "5.6.7.8", 98, 98)}
_, err := ftun.Write(data, 0)
if err != nil {
t.Error(err)
@@ -435,7 +441,7 @@ func BenchmarkWrite(b *testing.B) {
ftun, tun := newFakeTUN(b.Logf, true)
defer tun.Close()
packet := udp4("5.6.7.8", "1.2.3.4", 89, 89)
packet := [][]byte{udp4("5.6.7.8", "1.2.3.4", 89, 89)}
for i := 0; i < b.N; i++ {
_, err := ftun.Write(packet, 0)
if err != nil {