summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
Diffstat (limited to 'pkg')
-rw-r--r--pkg/abi/linux/fuse.go105
-rw-r--r--pkg/sentry/fsimpl/fuse/BUILD4
-rw-r--r--pkg/sentry/fsimpl/fuse/connection.go234
-rw-r--r--pkg/sentry/fsimpl/fuse/dev.go13
-rw-r--r--pkg/sentry/fsimpl/fuse/dev_test.go17
-rw-r--r--pkg/sentry/fsimpl/fuse/fusefs.go22
-rw-r--r--pkg/sentry/fsimpl/fuse/init.go166
-rw-r--r--pkg/sentry/platform/kvm/kvm_const_arm64.go11
-rw-r--r--pkg/sentry/platform/kvm/machine_arm64.go72
-rw-r--r--pkg/sentry/syscalls/linux/sys_file.go8
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/fd.go4
-rw-r--r--pkg/syserror/syserror.go1
-rw-r--r--pkg/tcpip/network/fragmentation/fragmentation.go70
-rw-r--r--pkg/tcpip/network/fragmentation/fragmentation_test.go105
-rw-r--r--pkg/tcpip/network/ipv4/ipv4.go6
-rw-r--r--pkg/tcpip/network/ipv4/ipv4_test.go53
-rw-r--r--pkg/tcpip/network/ipv6/ipv6.go2
-rw-r--r--pkg/tcpip/network/ipv6/ipv6_test.go109
-rw-r--r--pkg/tcpip/stack/BUILD25
-rw-r--r--pkg/tcpip/stack/fake_time_test.go209
-rw-r--r--pkg/tcpip/stack/ndp.go16
-rw-r--r--pkg/tcpip/stack/neighbor_cache.go335
-rw-r--r--pkg/tcpip/stack/neighbor_cache_test.go1752
-rw-r--r--pkg/tcpip/stack/neighbor_entry.go482
-rw-r--r--pkg/tcpip/stack/neighbor_entry_test.go2770
-rw-r--r--pkg/tcpip/stack/neighborstate_string.go44
-rw-r--r--pkg/tcpip/stack/nic.go33
-rw-r--r--pkg/tcpip/stack/nud.go466
-rw-r--r--pkg/tcpip/stack/nud_test.go795
-rw-r--r--pkg/tcpip/stack/route.go10
-rw-r--r--pkg/tcpip/stack/stack.go63
-rw-r--r--pkg/tcpip/stack/stack_test.go223
-rw-r--r--pkg/tcpip/tcpip.go26
-rw-r--r--pkg/tcpip/transport/tcp/protocol.go2
-rw-r--r--pkg/tcpip/transport/udp/endpoint.go8
-rw-r--r--pkg/tcpip/transport/udp/udp_test.go189
-rw-r--r--pkg/test/dockerutil/container.go16
37 files changed, 8312 insertions, 154 deletions
diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go
index d3ebbccc4..5c6ffe4a3 100644
--- a/pkg/abi/linux/fuse.go
+++ b/pkg/abi/linux/fuse.go
@@ -141,3 +141,108 @@ type FUSEWriteIn struct {
_ uint32
}
+
+// FUSE_INIT flags, consistent with the ones in include/uapi/linux/fuse.h.
+const (
+ FUSE_ASYNC_READ = 1 << 0
+ FUSE_POSIX_LOCKS = 1 << 1
+ FUSE_FILE_OPS = 1 << 2
+ FUSE_ATOMIC_O_TRUNC = 1 << 3
+ FUSE_EXPORT_SUPPORT = 1 << 4
+ FUSE_BIG_WRITES = 1 << 5
+ FUSE_DONT_MASK = 1 << 6
+ FUSE_SPLICE_WRITE = 1 << 7
+ FUSE_SPLICE_MOVE = 1 << 8
+ FUSE_SPLICE_READ = 1 << 9
+ FUSE_FLOCK_LOCKS = 1 << 10
+ FUSE_HAS_IOCTL_DIR = 1 << 11
+ FUSE_AUTO_INVAL_DATA = 1 << 12
+ FUSE_DO_READDIRPLUS = 1 << 13
+ FUSE_READDIRPLUS_AUTO = 1 << 14
+ FUSE_ASYNC_DIO = 1 << 15
+ FUSE_WRITEBACK_CACHE = 1 << 16
+ FUSE_NO_OPEN_SUPPORT = 1 << 17
+ FUSE_PARALLEL_DIROPS = 1 << 18
+ FUSE_HANDLE_KILLPRIV = 1 << 19
+ FUSE_POSIX_ACL = 1 << 20
+ FUSE_ABORT_ERROR = 1 << 21
+ FUSE_MAX_PAGES = 1 << 22
+ FUSE_CACHE_SYMLINKS = 1 << 23
+ FUSE_NO_OPENDIR_SUPPORT = 1 << 24
+ FUSE_EXPLICIT_INVAL_DATA = 1 << 25
+ FUSE_MAP_ALIGNMENT = 1 << 26
+)
+
+// currently supported FUSE protocol version numbers.
+const (
+ FUSE_KERNEL_VERSION = 7
+ FUSE_KERNEL_MINOR_VERSION = 31
+)
+
+// FUSEInitIn is the request sent by the kernel to the daemon,
+// to negotiate the version and flags.
+//
+// +marshal
+type FUSEInitIn struct {
+ // Major version supported by kernel.
+ Major uint32
+
+ // Minor version supported by the kernel.
+ Minor uint32
+
+ // MaxReadahead is the maximum number of bytes to read-ahead
+ // decided by the kernel.
+ MaxReadahead uint32
+
+ // Flags of this init request.
+ Flags uint32
+}
+
+// FUSEInitOut is the reply sent by the daemon to the kernel
+// for FUSEInitIn.
+//
+// +marshal
+type FUSEInitOut struct {
+ // Major version supported by daemon.
+ Major uint32
+
+ // Minor version supported by daemon.
+ Minor uint32
+
+ // MaxReadahead is the maximum number of bytes to read-ahead.
+ // Decided by the daemon, after receiving the value from kernel.
+ MaxReadahead uint32
+
+ // Flags of this init reply.
+ Flags uint32
+
+ // MaxBackground is the maximum number of pending background requests
+ // that the daemon wants.
+ MaxBackground uint16
+
+ // CongestionThreshold is the daemon-decided threshold for
+ // the number of the pending background requests.
+ CongestionThreshold uint16
+
+ // MaxWrite is the daemon's maximum size of a write buffer.
+ // Kernel adjusts it to the minimum (fuse/init.go:fuseMinMaxWrite).
+ // if the value from daemon is too small.
+ MaxWrite uint32
+
+ // TimeGran is the daemon's time granularity for mtime and ctime metadata.
+ // The unit is nanosecond.
+ // Value should be power of 10.
+ // 1 indicates full nanosecond granularity support.
+ TimeGran uint32
+
+ // MaxPages is the daemon's maximum number of pages for one write operation.
+ // Kernel adjusts it to the maximum (fuse/init.go:FUSE_MAX_MAX_PAGES).
+ // if the value from daemon is too large.
+ MaxPages uint16
+
+ // MapAlignment is an unknown field and not used by this package at this moment.
+ // Use as a placeholder to be consistent with the FUSE protocol.
+ MapAlignment uint16
+
+ _ [8]uint32
+}
diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD
index 67649e811..999111deb 100644
--- a/pkg/sentry/fsimpl/fuse/BUILD
+++ b/pkg/sentry/fsimpl/fuse/BUILD
@@ -21,6 +21,7 @@ go_library(
"connection.go",
"dev.go",
"fusefs.go",
+ "init.go",
"register.go",
"request_list.go",
],
@@ -44,14 +45,13 @@ go_library(
)
go_test(
- name = "dev_test",
+ name = "fuse_test",
size = "small",
srcs = ["dev_test.go"],
library = ":fuse",
deps = [
"//pkg/abi/linux",
"//pkg/sentry/fsimpl/testutil",
- "//pkg/sentry/fsimpl/tmpfs",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/vfs",
diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go
index f330da0bd..6df2728ab 100644
--- a/pkg/sentry/fsimpl/fuse/connection.go
+++ b/pkg/sentry/fsimpl/fuse/connection.go
@@ -17,6 +17,8 @@ package fuse
import (
"errors"
"fmt"
+ "sync"
+ "sync/atomic"
"syscall"
"gvisor.dev/gvisor/pkg/abi/linux"
@@ -25,18 +27,29 @@ import (
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/waiter"
"gvisor.dev/gvisor/tools/go_marshal/marshal"
)
-// MaxActiveRequestsDefault is the default setting controlling the upper bound
+// maxActiveRequestsDefault is the default setting controlling the upper bound
// on the number of active requests at any given time.
-const MaxActiveRequestsDefault = 10000
+const maxActiveRequestsDefault = 10000
-var (
- // Ordinary requests have even IDs, while interrupts IDs are odd.
- InitReqBit uint64 = 1
- ReqIDStep uint64 = 2
+// Ordinary requests have even IDs, while interrupts IDs are odd.
+// Used to increment the unique ID for each FUSE request.
+var reqIDStep uint64 = 2
+
+const (
+ // fuseDefaultMaxBackground is the default value for MaxBackground.
+ fuseDefaultMaxBackground = 12
+
+ // fuseDefaultCongestionThreshold is the default value for CongestionThreshold,
+ // and is 75% of the default maximum of MaxGround.
+ fuseDefaultCongestionThreshold = (fuseDefaultMaxBackground * 3 / 4)
+
+ // fuseDefaultMaxPagesPerReq is the default value for MaxPagesPerReq.
+ fuseDefaultMaxPagesPerReq = 32
)
// Request represents a FUSE operation request that hasn't been sent to the
@@ -61,17 +74,125 @@ type Response struct {
data []byte
}
-// Connection is the struct by which the sentry communicates with the FUSE server daemon.
-type Connection struct {
+// connection is the struct by which the sentry communicates with the FUSE server daemon.
+type connection struct {
fd *DeviceFD
- // MaxWrite is the daemon's maximum size of a write buffer.
- // This is negotiated during FUSE_INIT.
- MaxWrite uint32
+ // The following FUSE_INIT flags are currently unsupported by this implementation:
+ // - FUSE_ATOMIC_O_TRUNC: requires open(..., O_TRUNC)
+ // - FUSE_EXPORT_SUPPORT
+ // - FUSE_HANDLE_KILLPRIV
+ // - FUSE_POSIX_LOCKS: requires POSIX locks
+ // - FUSE_FLOCK_LOCKS: requires POSIX locks
+ // - FUSE_AUTO_INVAL_DATA: requires page caching eviction
+ // - FUSE_EXPLICIT_INVAL_DATA: requires page caching eviction
+ // - FUSE_DO_READDIRPLUS/FUSE_READDIRPLUS_AUTO: requires FUSE_READDIRPLUS implementation
+ // - FUSE_ASYNC_DIO
+ // - FUSE_POSIX_ACL: affects defaultPermissions, posixACL, xattr handler
+
+ // initialized after receiving FUSE_INIT reply.
+ // Until it's set, suspend sending FUSE requests.
+ // Use SetInitialized() and IsInitialized() for atomic access.
+ initialized int32
+
+ // initializedChan is used to block requests before initialization.
+ initializedChan chan struct{}
+
+ // blocked when there are too many outstading backgrounds requests (NumBackground == MaxBackground).
+ // TODO(gvisor.dev/issue/3185): update the numBackground accordingly; use a channel to block.
+ blocked bool
+
+ // connected (connection established) when a new FUSE file system is created.
+ // Set to false when:
+ // umount,
+ // connection abort,
+ // device release.
+ connected bool
+
+ // aborted via sysfs.
+ // TODO(gvisor.dev/issue/3185): abort all queued requests.
+ aborted bool
+
+ // connInitError if FUSE_INIT encountered error (major version mismatch).
+ // Only set in INIT.
+ connInitError bool
+
+ // connInitSuccess if FUSE_INIT is successful.
+ // Only set in INIT.
+ // Used for destory.
+ connInitSuccess bool
+
+ // TODO(gvisor.dev/issue/3185): All the queue logic are working in progress.
+
+ // NumberBackground is the number of requests in the background.
+ numBackground uint16
+
+ // congestionThreshold for NumBackground.
+ // Negotiated in FUSE_INIT.
+ congestionThreshold uint16
+
+ // maxBackground is the maximum number of NumBackground.
+ // Block connection when it is reached.
+ // Negotiated in FUSE_INIT.
+ maxBackground uint16
+
+ // numActiveBackground is the number of requests in background and has being marked as active.
+ numActiveBackground uint16
+
+ // numWating is the number of requests waiting for completion.
+ numWaiting uint32
+
+ // TODO(gvisor.dev/issue/3185): BgQueue
+ // some queue for background queued requests.
+
+ // bgLock protects:
+ // MaxBackground, CongestionThreshold, NumBackground,
+ // NumActiveBackground, BgQueue, Blocked.
+ bgLock sync.Mutex
+
+ // maxRead is the maximum size of a read buffer in in bytes.
+ maxRead uint32
+
+ // maxWrite is the maximum size of a write buffer in bytes.
+ // Negotiated in FUSE_INIT.
+ maxWrite uint32
+
+ // maxPages is the maximum number of pages for a single request to use.
+ // Negotiated in FUSE_INIT.
+ maxPages uint16
+
+ // minor version of the FUSE protocol.
+ // Negotiated and only set in INIT.
+ minor uint32
+
+ // asyncRead if read pages asynchronously.
+ // Negotiated and only set in INIT.
+ asyncRead bool
+
+ // abortErr is true if kernel need to return an unique read error after abort.
+ // Negotiated and only set in INIT.
+ abortErr bool
+
+ // writebackCache is true for write-back cache policy,
+ // false for write-through policy.
+ // Negotiated and only set in INIT.
+ writebackCache bool
+
+ // cacheSymlinks if filesystem needs to cache READLINK responses in page cache.
+ // Negotiated and only set in INIT.
+ cacheSymlinks bool
+
+ // bigWrites if doing multi-page cached writes.
+ // Negotiated and only set in INIT.
+ bigWrites bool
+
+ // dontMask if filestestem does not apply umask to creation modes.
+ // Negotiated in INIT.
+ dontMask bool
}
-// NewFUSEConnection creates a FUSE connection to fd
-func NewFUSEConnection(_ context.Context, fd *vfs.FileDescription, maxInFlightRequests uint64) (*Connection, error) {
+// newFUSEConnection creates a FUSE connection to fd.
+func newFUSEConnection(_ context.Context, fd *vfs.FileDescription, maxInFlightRequests uint64) (*connection, error) {
// Mark the device as ready so it can be used. /dev/fuse can only be used if the FD was used to
// mount a FUSE filesystem.
fuseFD := fd.Impl().(*DeviceFD)
@@ -84,16 +205,41 @@ func NewFUSEConnection(_ context.Context, fd *vfs.FileDescription, maxInFlightRe
fuseFD.fullQueueCh = make(chan struct{}, maxInFlightRequests)
fuseFD.writeCursor = 0
- return &Connection{
- fd: fuseFD,
+ return &connection{
+ fd: fuseFD,
+ maxBackground: fuseDefaultMaxBackground,
+ congestionThreshold: fuseDefaultCongestionThreshold,
+ maxPages: fuseDefaultMaxPagesPerReq,
+ initializedChan: make(chan struct{}),
+ connected: true,
}, nil
}
+// SetInitialized atomically sets the connection as initialized.
+func (conn *connection) SetInitialized() {
+ // Unblock the requests sent before INIT.
+ close(conn.initializedChan)
+
+ // Close the channel first to avoid the non-atomic situation
+ // where conn.initialized is true but there are
+ // tasks being blocked on the channel.
+ // And it prevents the newer tasks from gaining
+ // unnecessary higher chance to be issued before the blocked one.
+
+ atomic.StoreInt32(&(conn.initialized), int32(1))
+}
+
+// IsInitialized atomically check if the connection is initialized.
+// pairs with SetInitialized().
+func (conn *connection) Initialized() bool {
+ return atomic.LoadInt32(&(conn.initialized)) != 0
+}
+
// NewRequest creates a new request that can be sent to the FUSE server.
-func (conn *Connection) NewRequest(creds *auth.Credentials, pid uint32, ino uint64, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*Request, error) {
+func (conn *connection) NewRequest(creds *auth.Credentials, pid uint32, ino uint64, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*Request, error) {
conn.fd.mu.Lock()
defer conn.fd.mu.Unlock()
- conn.fd.nextOpID += linux.FUSEOpID(ReqIDStep)
+ conn.fd.nextOpID += linux.FUSEOpID(reqIDStep)
hdrLen := (*linux.FUSEHeaderIn)(nil).SizeBytes()
hdr := linux.FUSEHeaderIn{
@@ -118,13 +264,49 @@ func (conn *Connection) NewRequest(creds *auth.Credentials, pid uint32, ino uint
}
// Call makes a request to the server and blocks the invoking task until a
-// server responds with a response.
-// NOTE: If no task is provided then the Call will simply enqueue the request
-// and return a nil response. No blocking will happen in this case. Instead,
-// this is used to signify that the processing of this request will happen by
-// the kernel.Task that writes the response. See FUSE_INIT for such an
-// invocation.
-func (conn *Connection) Call(t *kernel.Task, r *Request) (*Response, error) {
+// server responds with a response. Task should never be nil.
+// Requests will not be sent before the connection is initialized.
+// For async tasks, use CallAsync().
+func (conn *connection) Call(t *kernel.Task, r *Request) (*Response, error) {
+ // Block requests sent before connection is initalized.
+ if !conn.Initialized() {
+ if err := t.Block(conn.initializedChan); err != nil {
+ return nil, err
+ }
+ }
+
+ return conn.call(t, r)
+}
+
+// CallAsync makes an async (aka background) request.
+// Those requests either do not expect a response (e.g. release) or
+// the response should be handled by others (e.g. init).
+// Return immediately unless the connection is blocked (before initialization).
+// Async call example: init, release, forget, aio, interrupt.
+// When the Request is FUSE_INIT, it will not be blocked before initialization.
+func (conn *connection) CallAsync(t *kernel.Task, r *Request) error {
+ // Block requests sent before connection is initalized.
+ if !conn.Initialized() && r.hdr.Opcode != linux.FUSE_INIT {
+ if err := t.Block(conn.initializedChan); err != nil {
+ return err
+ }
+ }
+
+ // This should be the only place that invokes call() with a nil task.
+ _, err := conn.call(nil, r)
+ return err
+}
+
+// call makes a call without blocking checks.
+func (conn *connection) call(t *kernel.Task, r *Request) (*Response, error) {
+ if !conn.connected {
+ return nil, syserror.ENOTCONN
+ }
+
+ if conn.connInitError {
+ return nil, syserror.ECONNREFUSED
+ }
+
fut, err := conn.callFuture(t, r)
if err != nil {
return nil, err
@@ -160,7 +342,7 @@ func (r *Response) UnmarshalPayload(m marshal.Marshallable) error {
// callFuture makes a request to the server and returns a future response.
// Call resolve() when the response needs to be fulfilled.
-func (conn *Connection) callFuture(t *kernel.Task, r *Request) (*futureResponse, error) {
+func (conn *connection) callFuture(t *kernel.Task, r *Request) (*futureResponse, error) {
conn.fd.mu.Lock()
defer conn.fd.mu.Unlock()
@@ -195,7 +377,7 @@ func (conn *Connection) callFuture(t *kernel.Task, r *Request) (*futureResponse,
}
// callFutureLocked makes a request to the server and returns a future response.
-func (conn *Connection) callFutureLocked(t *kernel.Task, r *Request) (*futureResponse, error) {
+func (conn *connection) callFutureLocked(t *kernel.Task, r *Request) (*futureResponse, error) {
conn.fd.queue.PushBack(r)
conn.fd.numActiveRequests += 1
fut := newFutureResponse(r.hdr.Opcode)
diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go
index f3443ac71..2225076bc 100644
--- a/pkg/sentry/fsimpl/fuse/dev.go
+++ b/pkg/sentry/fsimpl/fuse/dev.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
@@ -98,7 +99,9 @@ type DeviceFD struct {
}
// Release implements vfs.FileDescriptionImpl.Release.
-func (fd *DeviceFD) Release() {}
+func (fd *DeviceFD) Release() {
+ fd.fs.conn.connected = false
+}
// PRead implements vfs.FileDescriptionImpl.PRead.
func (fd *DeviceFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
@@ -124,7 +127,7 @@ func (fd *DeviceFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.R
minBuffSize := linux.FUSE_MIN_READ_BUFFER
inHdrLen := uint32((*linux.FUSEHeaderIn)(nil).SizeBytes())
writeHdrLen := uint32((*linux.FUSEWriteIn)(nil).SizeBytes())
- negotiatedMinBuffSize := inHdrLen + writeHdrLen + fd.fs.conn.MaxWrite
+ negotiatedMinBuffSize := inHdrLen + writeHdrLen + fd.fs.conn.maxWrite
if minBuffSize < negotiatedMinBuffSize {
minBuffSize = negotiatedMinBuffSize
}
@@ -385,9 +388,9 @@ func (fd *DeviceFD) sendError(ctx context.Context, errno int32, req *Request) er
// FUSE_INIT.
func (fd *DeviceFD) noReceiverAction(ctx context.Context, r *Response) error {
if r.opcode == linux.FUSE_INIT {
- // TODO: process init response here.
- // Maybe get the creds from the context?
- // creds := auth.CredentialsFromContext(ctx)
+ creds := auth.CredentialsFromContext(ctx)
+ rootUserNs := kernel.KernelFromContext(ctx).RootUserNamespace()
+ return fd.fs.conn.InitRecv(r, creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, rootUserNs))
}
return nil
diff --git a/pkg/sentry/fsimpl/fuse/dev_test.go b/pkg/sentry/fsimpl/fuse/dev_test.go
index fcd77832a..84c222ad6 100644
--- a/pkg/sentry/fsimpl/fuse/dev_test.go
+++ b/pkg/sentry/fsimpl/fuse/dev_test.go
@@ -22,7 +22,6 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/testutil"
- "gvisor.dev/gvisor/pkg/sentry/fsimpl/tmpfs"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/vfs"
@@ -60,25 +59,25 @@ func TestFUSECommunication(t *testing.T) {
Name: "SingleClientSingleServer",
NumClients: 1,
NumServers: 1,
- MaxActiveRequests: MaxActiveRequestsDefault,
+ MaxActiveRequests: maxActiveRequestsDefault,
},
{
Name: "SingleClientMultipleServers",
NumClients: 1,
NumServers: 10,
- MaxActiveRequests: MaxActiveRequestsDefault,
+ MaxActiveRequests: maxActiveRequestsDefault,
},
{
Name: "MultipleClientsSingleServer",
NumClients: 10,
NumServers: 1,
- MaxActiveRequests: MaxActiveRequestsDefault,
+ MaxActiveRequests: maxActiveRequestsDefault,
},
{
Name: "MultipleClientsMultipleServers",
NumClients: 10,
NumServers: 10,
- MaxActiveRequests: MaxActiveRequestsDefault,
+ MaxActiveRequests: maxActiveRequestsDefault,
},
{
Name: "RequestCapacityFull",
@@ -145,7 +144,7 @@ func TestFUSECommunication(t *testing.T) {
// CallTest makes a request to the server and blocks the invoking
// goroutine until a server responds with a response. Doesn't block
// a kernel.Task. Analogous to Connection.Call but used for testing.
-func CallTest(conn *Connection, t *kernel.Task, r *Request, i uint32) (*Response, error) {
+func CallTest(conn *connection, t *kernel.Task, r *Request, i uint32) (*Response, error) {
conn.fd.mu.Lock()
// Wait until we're certain that a new request can be processed.
@@ -214,7 +213,7 @@ func ReadTest(serverTask *kernel.Task, fd *vfs.FileDescription, inIOseq usermem.
// fuseClientRun emulates all the actions of a normal FUSE request. It creates
// a header, a payload, calls the server, waits for the response, and processes
// the response.
-func fuseClientRun(t *testing.T, s *testutil.System, k *kernel.Kernel, conn *Connection, creds *auth.Credentials, pid uint32, inode uint64, clientDone chan struct{}) {
+func fuseClientRun(t *testing.T, s *testutil.System, k *kernel.Kernel, conn *connection, creds *auth.Credentials, pid uint32, inode uint64, clientDone chan struct{}) {
defer func() { clientDone <- struct{}{} }()
tc := k.NewThreadGroup(nil, k.RootPIDNamespace(), kernel.NewSignalHandlers(), linux.SIGCHLD, k.GlobalInit().Limits())
@@ -343,7 +342,7 @@ func setup(t *testing.T) *testutil.System {
AllowUserMount: true,
})
- mntns, err := k.VFS().NewMountNamespace(ctx, creds, "", tmpfs.Name, &vfs.GetFilesystemOptions{})
+ mntns, err := k.VFS().NewMountNamespace(ctx, creds, "", "tmpfs", &vfs.GetFilesystemOptions{})
if err != nil {
t.Fatalf("NewMountNamespace(): %v", err)
}
@@ -353,7 +352,7 @@ func setup(t *testing.T) *testutil.System {
// newTestConnection creates a fuse connection that the sentry can communicate with
// and the FD for the server to communicate with.
-func newTestConnection(system *testutil.System, k *kernel.Kernel, maxActiveRequests uint64) (*Connection, *vfs.FileDescription, error) {
+func newTestConnection(system *testutil.System, k *kernel.Kernel, maxActiveRequests uint64) (*connection, *vfs.FileDescription, error) {
vfsObj := &vfs.VirtualFilesystem{}
fuseDev := &DeviceFD{}
diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go
index 911b6f7cb..200a93bbf 100644
--- a/pkg/sentry/fsimpl/fuse/fusefs.go
+++ b/pkg/sentry/fsimpl/fuse/fusefs.go
@@ -65,7 +65,7 @@ type filesystem struct {
// conn is used for communication between the FUSE server
// daemon and the sentry fusefs.
- conn *Connection
+ conn *connection
// opts is the options the fusefs is initialized with.
opts *filesystemOptions
@@ -140,7 +140,7 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
fsopts.rootMode = rootMode
// Set the maxInFlightRequests option.
- fsopts.maxActiveRequests = MaxActiveRequestsDefault
+ fsopts.maxActiveRequests = maxActiveRequestsDefault
// Check for unparsed options.
if len(mopts) != 0 {
@@ -157,8 +157,12 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
fs.VFSFilesystem().Init(vfsObj, &fsType, fs)
- // TODO: dispatch a FUSE_INIT request to the FUSE daemon server before
- // returning. Mount will not block on this dispatched request.
+ // Send a FUSE_INIT request to the FUSE daemon server before returning.
+ // This call is not blocking.
+ if err := fs.conn.InitSend(creds, uint32(kernelTask.ThreadID())); err != nil {
+ log.Warningf("%s.InitSend: failed with error: %v", fsType.Name(), err)
+ return nil, nil, err
+ }
// root is the fusefs root directory.
root := fs.newInode(creds, fsopts.rootMode)
@@ -173,7 +177,7 @@ func NewFUSEFilesystem(ctx context.Context, devMinor uint32, opts *filesystemOpt
opts: opts,
}
- conn, err := NewFUSEConnection(ctx, device, opts.maxActiveRequests)
+ conn, err := newFUSEConnection(ctx, device, opts.maxActiveRequests)
if err != nil {
log.Warningf("fuse.NewFUSEFilesystem: NewFUSEConnection failed with error: %v", err)
return nil, syserror.EINVAL
@@ -192,8 +196,8 @@ func (fs *filesystem) Release() {
fs.Filesystem.Release()
}
-// Inode implements kernfs.Inode.
-type Inode struct {
+// inode implements kernfs.Inode.
+type inode struct {
kernfs.InodeAttrs
kernfs.InodeNoDynamicLookup
kernfs.InodeNotSymlink
@@ -206,7 +210,7 @@ type Inode struct {
}
func (fs *filesystem) newInode(creds *auth.Credentials, mode linux.FileMode) *kernfs.Dentry {
- i := &Inode{}
+ i := &inode{}
i.InodeAttrs.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0755)
i.OrderedChildren.Init(kernfs.OrderedChildrenOptions{})
i.dentry.Init(i)
@@ -215,7 +219,7 @@ func (fs *filesystem) newInode(creds *auth.Credentials, mode linux.FileMode) *ke
}
// Open implements kernfs.Inode.Open.
-func (i *Inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
+func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
fd, err := kernfs.NewGenericDirectoryFD(rp.Mount(), vfsd, &i.OrderedChildren, &i.locks, &opts)
if err != nil {
return nil, err
diff --git a/pkg/sentry/fsimpl/fuse/init.go b/pkg/sentry/fsimpl/fuse/init.go
new file mode 100644
index 000000000..779c2bd3f
--- /dev/null
+++ b/pkg/sentry/fsimpl/fuse/init.go
@@ -0,0 +1,166 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package fuse
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+)
+
+// consts used by FUSE_INIT negotiation.
+const (
+ // fuseMaxMaxPages is the maximum value for MaxPages received in InitOut.
+ // Follow the same behavior as unix fuse implementation.
+ fuseMaxMaxPages = 256
+
+ // Maximum value for the time granularity for file time stamps, 1s.
+ // Follow the same behavior as unix fuse implementation.
+ fuseMaxTimeGranNs = 1000000000
+
+ // Minimum value for MaxWrite.
+ // Follow the same behavior as unix fuse implementation.
+ fuseMinMaxWrite = 4096
+
+ // Temporary default value for max readahead, 128kb.
+ fuseDefaultMaxReadahead = 131072
+
+ // The FUSE_INIT_IN flags sent to the daemon.
+ // TODO(gvisor.dev/issue/3199): complete the flags.
+ fuseDefaultInitFlags = linux.FUSE_MAX_PAGES
+)
+
+// Adjustable maximums for Connection's cogestion control parameters.
+// Used as the upperbound of the config values.
+// Currently we do not support adjustment to them.
+var (
+ MaxUserBackgroundRequest uint16 = fuseDefaultMaxBackground
+ MaxUserCongestionThreshold uint16 = fuseDefaultCongestionThreshold
+)
+
+// InitSend sends a FUSE_INIT request.
+func (conn *connection) InitSend(creds *auth.Credentials, pid uint32) error {
+ in := linux.FUSEInitIn{
+ Major: linux.FUSE_KERNEL_VERSION,
+ Minor: linux.FUSE_KERNEL_MINOR_VERSION,
+ // TODO(gvisor.dev/issue/3196): find appropriate way to calculate this
+ MaxReadahead: fuseDefaultMaxReadahead,
+ Flags: fuseDefaultInitFlags,
+ }
+
+ req, err := conn.NewRequest(creds, pid, 0, linux.FUSE_INIT, &in)
+ if err != nil {
+ return err
+ }
+
+ // Since there is no task to block on and FUSE_INIT is the request
+ // to unblock other requests, use nil.
+ return conn.CallAsync(nil, req)
+}
+
+// InitRecv receives a FUSE_INIT reply and process it.
+func (conn *connection) InitRecv(res *Response, hasSysAdminCap bool) error {
+ if err := res.Error(); err != nil {
+ return err
+ }
+
+ var out linux.FUSEInitOut
+ if err := res.UnmarshalPayload(&out); err != nil {
+ return err
+ }
+
+ return conn.initProcessReply(&out, hasSysAdminCap)
+}
+
+// Process the FUSE_INIT reply from the FUSE server.
+func (conn *connection) initProcessReply(out *linux.FUSEInitOut, hasSysAdminCap bool) error {
+ // No support for old major fuse versions.
+ if out.Major != linux.FUSE_KERNEL_VERSION {
+ conn.connInitError = true
+
+ // Set the connection as initialized and unblock the blocked requests
+ // (i.e. return error for them).
+ conn.SetInitialized()
+
+ return nil
+ }
+
+ // Start processing the reply.
+ conn.connInitSuccess = true
+ conn.minor = out.Minor
+
+ // No support for limits before minor version 13.
+ if out.Minor >= 13 {
+ conn.bgLock.Lock()
+
+ if out.MaxBackground > 0 {
+ conn.maxBackground = out.MaxBackground
+
+ if !hasSysAdminCap &&
+ conn.maxBackground > MaxUserBackgroundRequest {
+ conn.maxBackground = MaxUserBackgroundRequest
+ }
+ }
+
+ if out.CongestionThreshold > 0 {
+ conn.congestionThreshold = out.CongestionThreshold
+
+ if !hasSysAdminCap &&
+ conn.congestionThreshold > MaxUserCongestionThreshold {
+ conn.congestionThreshold = MaxUserCongestionThreshold
+ }
+ }
+
+ conn.bgLock.Unlock()
+ }
+
+ // No support for the following flags before minor version 6.
+ if out.Minor >= 6 {
+ conn.asyncRead = out.Flags&linux.FUSE_ASYNC_READ != 0
+ conn.bigWrites = out.Flags&linux.FUSE_BIG_WRITES != 0
+ conn.dontMask = out.Flags&linux.FUSE_DONT_MASK != 0
+ conn.writebackCache = out.Flags&linux.FUSE_WRITEBACK_CACHE != 0
+ conn.cacheSymlinks = out.Flags&linux.FUSE_CACHE_SYMLINKS != 0
+ conn.abortErr = out.Flags&linux.FUSE_ABORT_ERROR != 0
+
+ // TODO(gvisor.dev/issue/3195): figure out how to use TimeGran (0 < TimeGran <= fuseMaxTimeGranNs).
+
+ if out.Flags&linux.FUSE_MAX_PAGES != 0 {
+ maxPages := out.MaxPages
+ if maxPages < 1 {
+ maxPages = 1
+ }
+ if maxPages > fuseMaxMaxPages {
+ maxPages = fuseMaxMaxPages
+ }
+ conn.maxPages = maxPages
+ }
+ }
+
+ // No support for negotiating MaxWrite before minor version 5.
+ if out.Minor >= 5 {
+ conn.maxWrite = out.MaxWrite
+ } else {
+ conn.maxWrite = fuseMinMaxWrite
+ }
+ if conn.maxWrite < fuseMinMaxWrite {
+ conn.maxWrite = fuseMinMaxWrite
+ }
+
+ // Set connection as initialized and unblock the requests
+ // issued before init.
+ conn.SetInitialized()
+
+ return nil
+}
diff --git a/pkg/sentry/platform/kvm/kvm_const_arm64.go b/pkg/sentry/platform/kvm/kvm_const_arm64.go
index 6f0539c29..fdc599477 100644
--- a/pkg/sentry/platform/kvm/kvm_const_arm64.go
+++ b/pkg/sentry/platform/kvm/kvm_const_arm64.go
@@ -116,6 +116,17 @@ const (
// Arm64: Exception Syndrome Register EL1.
const (
+ _ESR_ELx_EC_SHIFT = 26
+ _ESR_ELx_EC_MASK = 0x3F << _ESR_ELx_EC_SHIFT
+
+ _ESR_ELx_EC_IMP_DEF = 0x1f
+ _ESR_ELx_EC_IABT_LOW = 0x20
+ _ESR_ELx_EC_IABT_CUR = 0x21
+ _ESR_ELx_EC_PC_ALIGN = 0x22
+
+ _ESR_ELx_CM = 1 << 8
+ _ESR_ELx_WNR = 1 << 6
+
_ESR_ELx_FSC = 0x3F
_ESR_SEGV_MAPERR_L0 = 0x4
diff --git a/pkg/sentry/platform/kvm/machine_arm64.go b/pkg/sentry/platform/kvm/machine_arm64.go
index f3bf973de..9db171af9 100644
--- a/pkg/sentry/platform/kvm/machine_arm64.go
+++ b/pkg/sentry/platform/kvm/machine_arm64.go
@@ -125,71 +125,59 @@ func nonCanonical(addr uint64, signal int32, info *arch.SignalInfo) (usermem.Acc
return usermem.NoAccess, platform.ErrContextSignal
}
+// isInstructionAbort returns true if it is an instruction abort.
+//
+//go:nosplit
+func isInstructionAbort(code uint64) bool {
+ value := (code & _ESR_ELx_EC_MASK) >> _ESR_ELx_EC_SHIFT
+ return value == _ESR_ELx_EC_IABT_LOW
+}
+
+// isWriteFault returns whether it is a write fault.
+//
+//go:nosplit
+func isWriteFault(code uint64) bool {
+ if isInstructionAbort(code) {
+ return false
+ }
+
+ return (code & _ESR_ELx_WNR) != 0
+}
+
// fault generates an appropriate fault return.
//
//go:nosplit
func (c *vCPU) fault(signal int32, info *arch.SignalInfo) (usermem.AccessType, error) {
+ bluepill(c) // Probably no-op, but may not be.
faultAddr := c.GetFaultAddr()
code, user := c.ErrorCode()
+ if !user {
+ // The last fault serviced by this CPU was not a user
+ // fault, so we can't reliably trust the faultAddr or
+ // the code provided here. We need to re-execute.
+ return usermem.NoAccess, platform.ErrContextInterrupt
+ }
+
// Reset the pointed SignalInfo.
*info = arch.SignalInfo{Signo: signal}
info.SetAddr(uint64(faultAddr))
- read := true
- write := false
- execute := true
-
ret := code & _ESR_ELx_FSC
switch ret {
case _ESR_SEGV_MAPERR_L0, _ESR_SEGV_MAPERR_L1, _ESR_SEGV_MAPERR_L2, _ESR_SEGV_MAPERR_L3:
info.Code = 1 //SEGV_MAPERR
- read = false
- write = true
- execute = false
case _ESR_SEGV_ACCERR_L1, _ESR_SEGV_ACCERR_L2, _ESR_SEGV_ACCERR_L3, _ESR_SEGV_PEMERR_L1, _ESR_SEGV_PEMERR_L2, _ESR_SEGV_PEMERR_L3:
info.Code = 2 // SEGV_ACCERR.
- read = true
- write = false
- execute = false
default:
info.Code = 2
}
- if !user {
- read = true
- write = false
- execute = true
-
- }
accessType := usermem.AccessType{
- Read: read,
- Write: write,
- Execute: execute,
+ Read: !isWriteFault(uint64(code)),
+ Write: isWriteFault(uint64(code)),
+ Execute: isInstructionAbort(uint64(code)),
}
return accessType, platform.ErrContextSignal
}
-
-// retryInGuest runs the given function in guest mode.
-//
-// If the function does not complete in guest mode (due to execution of a
-// system call due to a GC stall, for example), then it will be retried. The
-// given function must be idempotent as a result of the retry mechanism.
-func (m *machine) retryInGuest(fn func()) {
- c := m.Get()
- defer m.Put(c)
- for {
- c.ClearErrorCode() // See below.
- bluepill(c) // Force guest mode.
- fn() // Execute the given function.
- _, user := c.ErrorCode()
- if user {
- // If user is set, then we haven't bailed back to host
- // mode via a kernel exception or system call. We
- // consider the full function to have executed in guest
- // mode and we can return.
- break
- }
- }
-}
diff --git a/pkg/sentry/syscalls/linux/sys_file.go b/pkg/sentry/syscalls/linux/sys_file.go
index 2797c6a72..8cf6401e7 100644
--- a/pkg/sentry/syscalls/linux/sys_file.go
+++ b/pkg/sentry/syscalls/linux/sys_file.go
@@ -1057,7 +1057,7 @@ func Fcntl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall
case linux.F_SETOWN_EX:
addr := args[2].Pointer()
var owner linux.FOwnerEx
- n, err := t.CopyIn(addr, &owner)
+ _, err := t.CopyIn(addr, &owner)
if err != nil {
return 0, nil, err
}
@@ -1069,21 +1069,21 @@ func Fcntl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall
return 0, nil, syserror.ESRCH
}
a.SetOwnerTask(t, task)
- return uintptr(n), nil, nil
+ return 0, nil, nil
case linux.F_OWNER_PID:
tg := t.PIDNamespace().ThreadGroupWithID(kernel.ThreadID(owner.PID))
if tg == nil {
return 0, nil, syserror.ESRCH
}
a.SetOwnerThreadGroup(t, tg)
- return uintptr(n), nil, nil
+ return 0, nil, nil
case linux.F_OWNER_PGRP:
pg := t.PIDNamespace().ProcessGroupWithID(kernel.ProcessGroupID(owner.PID))
if pg == nil {
return 0, nil, syserror.ESRCH
}
a.SetOwnerProcessGroup(t, pg)
- return uintptr(n), nil, nil
+ return 0, nil, nil
default:
return 0, nil, syserror.EINVAL
}
diff --git a/pkg/sentry/syscalls/linux/vfs2/fd.go b/pkg/sentry/syscalls/linux/vfs2/fd.go
index 517394ba9..67f191551 100644
--- a/pkg/sentry/syscalls/linux/vfs2/fd.go
+++ b/pkg/sentry/syscalls/linux/vfs2/fd.go
@@ -185,11 +185,11 @@ func Fcntl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall
return 0, nil, err
case linux.F_SETOWN_EX:
var owner linux.FOwnerEx
- n, err := t.CopyIn(args[2].Pointer(), &owner)
+ _, err := t.CopyIn(args[2].Pointer(), &owner)
if err != nil {
return 0, nil, err
}
- return uintptr(n), nil, setAsyncOwner(t, file, owner.Type, owner.PID)
+ return 0, nil, setAsyncOwner(t, file, owner.Type, owner.PID)
case linux.F_GETPIPE_SZ:
pipefile, ok := file.Impl().(*pipe.VFSPipeFD)
if !ok {
diff --git a/pkg/syserror/syserror.go b/pkg/syserror/syserror.go
index c73072c42..798e07b01 100644
--- a/pkg/syserror/syserror.go
+++ b/pkg/syserror/syserror.go
@@ -61,6 +61,7 @@ var (
ENOMEM = error(syscall.ENOMEM)
ENOSPC = error(syscall.ENOSPC)
ENOSYS = error(syscall.ENOSYS)
+ ENOTCONN = error(syscall.ENOTCONN)
ENOTDIR = error(syscall.ENOTDIR)
ENOTEMPTY = error(syscall.ENOTEMPTY)
ENOTSOCK = error(syscall.ENOTSOCK)
diff --git a/pkg/tcpip/network/fragmentation/fragmentation.go b/pkg/tcpip/network/fragmentation/fragmentation.go
index 2982450f8..ffbadb6e2 100644
--- a/pkg/tcpip/network/fragmentation/fragmentation.go
+++ b/pkg/tcpip/network/fragmentation/fragmentation.go
@@ -17,6 +17,7 @@
package fragmentation
import (
+ "errors"
"fmt"
"log"
"time"
@@ -25,20 +26,31 @@ import (
"gvisor.dev/gvisor/pkg/tcpip/buffer"
)
-// DefaultReassembleTimeout is based on the linux stack: net.ipv4.ipfrag_time.
-const DefaultReassembleTimeout = 30 * time.Second
+const (
+ // DefaultReassembleTimeout is based on the linux stack: net.ipv4.ipfrag_time.
+ DefaultReassembleTimeout = 30 * time.Second
-// HighFragThreshold is the threshold at which we start trimming old
-// fragmented packets. Linux uses a default value of 4 MB. See
-// net.ipv4.ipfrag_high_thresh for more information.
-const HighFragThreshold = 4 << 20 // 4MB
+ // HighFragThreshold is the threshold at which we start trimming old
+ // fragmented packets. Linux uses a default value of 4 MB. See
+ // net.ipv4.ipfrag_high_thresh for more information.
+ HighFragThreshold = 4 << 20 // 4MB
-// LowFragThreshold is the threshold we reach to when we start dropping
-// older fragmented packets. It's important that we keep enough room for newer
-// packets to be re-assembled. Hence, this needs to be lower than
-// HighFragThreshold enough. Linux uses a default value of 3 MB. See
-// net.ipv4.ipfrag_low_thresh for more information.
-const LowFragThreshold = 3 << 20 // 3MB
+ // LowFragThreshold is the threshold we reach to when we start dropping
+ // older fragmented packets. It's important that we keep enough room for newer
+ // packets to be re-assembled. Hence, this needs to be lower than
+ // HighFragThreshold enough. Linux uses a default value of 3 MB. See
+ // net.ipv4.ipfrag_low_thresh for more information.
+ LowFragThreshold = 3 << 20 // 3MB
+
+ // minBlockSize is the minimum block size for fragments.
+ minBlockSize = 1
+)
+
+var (
+ // ErrInvalidArgs indicates to the caller that that an invalid argument was
+ // provided.
+ ErrInvalidArgs = errors.New("invalid args")
+)
// Fragmentation is the main structure that other modules
// of the stack should use to implement IP Fragmentation.
@@ -50,10 +62,13 @@ type Fragmentation struct {
rList reassemblerList
size int
timeout time.Duration
+ blockSize uint16
}
// NewFragmentation creates a new Fragmentation.
//
+// blockSize specifies the fragment block size, in bytes.
+//
// highMemoryLimit specifies the limit on the memory consumed
// by the fragments stored by Fragmentation (overhead of internal data-structures
// is not accounted). Fragments are dropped when the limit is reached.
@@ -64,7 +79,7 @@ type Fragmentation struct {
// reassemblingTimeout specifies the maximum time allowed to reassemble a packet.
// Fragments are lazily evicted only when a new a packet with an
// already existing fragmentation-id arrives after the timeout.
-func NewFragmentation(highMemoryLimit, lowMemoryLimit int, reassemblingTimeout time.Duration) *Fragmentation {
+func NewFragmentation(blockSize uint16, highMemoryLimit, lowMemoryLimit int, reassemblingTimeout time.Duration) *Fragmentation {
if lowMemoryLimit >= highMemoryLimit {
lowMemoryLimit = highMemoryLimit
}
@@ -73,17 +88,46 @@ func NewFragmentation(highMemoryLimit, lowMemoryLimit int, reassemblingTimeout t
lowMemoryLimit = 0
}
+ if blockSize < minBlockSize {
+ blockSize = minBlockSize
+ }
+
return &Fragmentation{
reassemblers: make(map[uint32]*reassembler),
highLimit: highMemoryLimit,
lowLimit: lowMemoryLimit,
timeout: reassemblingTimeout,
+ blockSize: blockSize,
}
}
// Process processes an incoming fragment belonging to an ID and returns a
// complete packet when all the packets belonging to that ID have been received.
+//
+// [first, last] is the range of the fragment bytes.
+//
+// first must be a multiple of the block size f is configured with. The size
+// of the fragment data must be a multiple of the block size, unless there are
+// no fragments following this fragment (more set to false).
func (f *Fragmentation) Process(id uint32, first, last uint16, more bool, vv buffer.VectorisedView) (buffer.VectorisedView, bool, error) {
+ if first > last {
+ return buffer.VectorisedView{}, false, fmt.Errorf("first=%d is greater than last=%d: %w", first, last, ErrInvalidArgs)
+ }
+
+ if first%f.blockSize != 0 {
+ return buffer.VectorisedView{}, false, fmt.Errorf("first=%d is not a multiple of block size=%d: %w", first, f.blockSize, ErrInvalidArgs)
+ }
+
+ fragmentSize := last - first + 1
+ if more && fragmentSize%f.blockSize != 0 {
+ return buffer.VectorisedView{}, false, fmt.Errorf("fragment size=%d bytes is not a multiple of block size=%d on non-final fragment: %w", fragmentSize, f.blockSize, ErrInvalidArgs)
+ }
+
+ if l := vv.Size(); l < int(fragmentSize) {
+ return buffer.VectorisedView{}, false, fmt.Errorf("got fragment size=%d bytes less than the expected fragment size=%d bytes (first=%d last=%d): %w", l, fragmentSize, first, last, ErrInvalidArgs)
+ }
+ vv.CapLength(int(fragmentSize))
+
f.mu.Lock()
r, ok := f.reassemblers[id]
if ok && r.tooOld(f.timeout) {
diff --git a/pkg/tcpip/network/fragmentation/fragmentation_test.go b/pkg/tcpip/network/fragmentation/fragmentation_test.go
index 72c0f53be..ebc3232e5 100644
--- a/pkg/tcpip/network/fragmentation/fragmentation_test.go
+++ b/pkg/tcpip/network/fragmentation/fragmentation_test.go
@@ -15,6 +15,7 @@
package fragmentation
import (
+ "errors"
"reflect"
"testing"
"time"
@@ -81,7 +82,7 @@ var processTestCases = []struct {
func TestFragmentationProcess(t *testing.T) {
for _, c := range processTestCases {
t.Run(c.comment, func(t *testing.T) {
- f := NewFragmentation(1024, 512, DefaultReassembleTimeout)
+ f := NewFragmentation(minBlockSize, 1024, 512, DefaultReassembleTimeout)
for i, in := range c.in {
vv, done, err := f.Process(in.id, in.first, in.last, in.more, in.vv)
if err != nil {
@@ -110,7 +111,7 @@ func TestFragmentationProcess(t *testing.T) {
func TestReassemblingTimeout(t *testing.T) {
timeout := time.Millisecond
- f := NewFragmentation(1024, 512, timeout)
+ f := NewFragmentation(minBlockSize, 1024, 512, timeout)
// Send first fragment with id = 0, first = 0, last = 0, and more = true.
f.Process(0, 0, 0, true, vv(1, "0"))
// Sleep more than the timeout.
@@ -127,7 +128,7 @@ func TestReassemblingTimeout(t *testing.T) {
}
func TestMemoryLimits(t *testing.T) {
- f := NewFragmentation(3, 1, DefaultReassembleTimeout)
+ f := NewFragmentation(minBlockSize, 3, 1, DefaultReassembleTimeout)
// Send first fragment with id = 0.
f.Process(0, 0, 0, true, vv(1, "0"))
// Send first fragment with id = 1.
@@ -151,7 +152,7 @@ func TestMemoryLimits(t *testing.T) {
}
func TestMemoryLimitsIgnoresDuplicates(t *testing.T) {
- f := NewFragmentation(1, 0, DefaultReassembleTimeout)
+ f := NewFragmentation(minBlockSize, 1, 0, DefaultReassembleTimeout)
// Send first fragment with id = 0.
f.Process(0, 0, 0, true, vv(1, "0"))
// Send the same packet again.
@@ -163,3 +164,99 @@ func TestMemoryLimitsIgnoresDuplicates(t *testing.T) {
t.Errorf("Wrong size, duplicates are not handled correctly: got=%d, want=%d.", got, want)
}
}
+
+func TestErrors(t *testing.T) {
+ const fragID = 5
+
+ tests := []struct {
+ name string
+ blockSize uint16
+ first uint16
+ last uint16
+ more bool
+ data string
+ err error
+ }{
+ {
+ name: "exact block size without more",
+ blockSize: 2,
+ first: 2,
+ last: 3,
+ more: false,
+ data: "01",
+ },
+ {
+ name: "exact block size with more",
+ blockSize: 2,
+ first: 2,
+ last: 3,
+ more: true,
+ data: "01",
+ },
+ {
+ name: "exact block size with more and extra data",
+ blockSize: 2,
+ first: 2,
+ last: 3,
+ more: true,
+ data: "012",
+ },
+ {
+ name: "exact block size with more and too little data",
+ blockSize: 2,
+ first: 2,
+ last: 3,
+ more: true,
+ data: "0",
+ err: ErrInvalidArgs,
+ },
+ {
+ name: "not exact block size with more",
+ blockSize: 2,
+ first: 2,
+ last: 2,
+ more: true,
+ data: "0",
+ err: ErrInvalidArgs,
+ },
+ {
+ name: "not exact block size without more",
+ blockSize: 2,
+ first: 2,
+ last: 2,
+ more: false,
+ data: "0",
+ },
+ {
+ name: "first not a multiple of block size",
+ blockSize: 2,
+ first: 3,
+ last: 4,
+ more: true,
+ data: "01",
+ err: ErrInvalidArgs,
+ },
+ {
+ name: "first more than last",
+ blockSize: 2,
+ first: 4,
+ last: 3,
+ more: true,
+ data: "01",
+ err: ErrInvalidArgs,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ f := NewFragmentation(test.blockSize, HighFragThreshold, LowFragThreshold, DefaultReassembleTimeout)
+ _, done, err := f.Process(fragID, test.first, test.last, test.more, vv(len(test.data), test.data))
+ if !errors.Is(err, test.err) {
+ t.Errorf("got Proceess(%d, %d, %d, %t, %q) = (_, _, %v), want = (_, _, %v)", fragID, test.first, test.last, test.more, test.data, err, test.err)
+ }
+ if done {
+ t.Errorf("got Proceess(%d, %d, %d, %t, %q) = (_, true, _), want = (_, false, _)", fragID, test.first, test.last, test.more, test.data)
+ }
+ })
+ }
+}
diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go
index b1776e5ee..0b5a35cce 100644
--- a/pkg/tcpip/network/ipv4/ipv4.go
+++ b/pkg/tcpip/network/ipv4/ipv4.go
@@ -45,6 +45,10 @@ const (
// buckets is the number of identifier buckets.
buckets = 2048
+
+ // The size of a fragment block, in bytes, as per RFC 791 section 3.1,
+ // page 14.
+ fragmentblockSize = 8
)
type endpoint struct {
@@ -66,7 +70,7 @@ func (p *protocol) NewEndpoint(nicID tcpip.NICID, addrWithPrefix tcpip.AddressWi
prefixLen: addrWithPrefix.PrefixLen,
linkEP: linkEP,
dispatcher: dispatcher,
- fragmentation: fragmentation.NewFragmentation(fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout),
+ fragmentation: fragmentation.NewFragmentation(fragmentblockSize, fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout),
protocol: p,
stack: st,
}
diff --git a/pkg/tcpip/network/ipv4/ipv4_test.go b/pkg/tcpip/network/ipv4/ipv4_test.go
index 11e579c4b..4f82c45e2 100644
--- a/pkg/tcpip/network/ipv4/ipv4_test.go
+++ b/pkg/tcpip/network/ipv4/ipv4_test.go
@@ -519,6 +519,11 @@ func TestReceiveFragments(t *testing.T) {
// UDP header plus a payload of 0..256 in increments of 2.
ipv4Payload2 := udpGen(128, 2)
udpPayload2 := ipv4Payload2[header.UDPMinimumSize:]
+ // UDP header plus a payload of 0..256 in increments of 3.
+ // Used to test cases where the fragment blocks are not a multiple of
+ // the fragment block size of 8 (RFC 791 section 3.1 page 14).
+ ipv4Payload3 := udpGen(127, 3)
+ udpPayload3 := ipv4Payload3[header.UDPMinimumSize:]
type fragmentData struct {
id uint16
@@ -545,6 +550,18 @@ func TestReceiveFragments(t *testing.T) {
expectedPayloads: [][]byte{udpPayload1},
},
{
+ name: "No fragmentation with size not a multiple of fragment block size",
+ fragments: []fragmentData{
+ {
+ id: 1,
+ flags: 0,
+ fragmentOffset: 0,
+ payload: ipv4Payload3,
+ },
+ },
+ expectedPayloads: [][]byte{udpPayload3},
+ },
+ {
name: "More fragments without payload",
fragments: []fragmentData{
{
@@ -587,6 +604,42 @@ func TestReceiveFragments(t *testing.T) {
expectedPayloads: [][]byte{udpPayload1},
},
{
+ name: "Two fragments with last fragment size not a multiple of fragment block size",
+ fragments: []fragmentData{
+ {
+ id: 1,
+ flags: header.IPv4FlagMoreFragments,
+ fragmentOffset: 0,
+ payload: ipv4Payload3[:64],
+ },
+ {
+ id: 1,
+ flags: 0,
+ fragmentOffset: 64,
+ payload: ipv4Payload3[64:],
+ },
+ },
+ expectedPayloads: [][]byte{udpPayload3},
+ },
+ {
+ name: "Two fragments with first fragment size not a multiple of fragment block size",
+ fragments: []fragmentData{
+ {
+ id: 1,
+ flags: header.IPv4FlagMoreFragments,
+ fragmentOffset: 0,
+ payload: ipv4Payload3[:63],
+ },
+ {
+ id: 1,
+ flags: 0,
+ fragmentOffset: 63,
+ payload: ipv4Payload3[63:],
+ },
+ },
+ expectedPayloads: nil,
+ },
+ {
name: "Second fragment has MoreFlags set",
fragments: []fragmentData{
{
diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go
index 95fbcf2d1..5483ae4ee 100644
--- a/pkg/tcpip/network/ipv6/ipv6.go
+++ b/pkg/tcpip/network/ipv6/ipv6.go
@@ -467,7 +467,7 @@ func (p *protocol) NewEndpoint(nicID tcpip.NICID, addrWithPrefix tcpip.AddressWi
linkEP: linkEP,
linkAddrCache: linkAddrCache,
dispatcher: dispatcher,
- fragmentation: fragmentation.NewFragmentation(fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout),
+ fragmentation: fragmentation.NewFragmentation(header.IPv6FragmentExtHdrFragmentOffsetBytesPerUnit, fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout),
protocol: p,
}, nil
}
diff --git a/pkg/tcpip/network/ipv6/ipv6_test.go b/pkg/tcpip/network/ipv6/ipv6_test.go
index 213ff64f2..84bac14ff 100644
--- a/pkg/tcpip/network/ipv6/ipv6_test.go
+++ b/pkg/tcpip/network/ipv6/ipv6_test.go
@@ -678,13 +678,18 @@ type fragmentData struct {
}
func TestReceiveIPv6Fragments(t *testing.T) {
- const nicID = 1
- const udpPayload1Length = 256
- const udpPayload2Length = 128
- const fragmentExtHdrLen = 8
- // Note, not all routing extension headers will be 8 bytes but this test
- // uses 8 byte routing extension headers for most sub tests.
- const routingExtHdrLen = 8
+ const (
+ nicID = 1
+ udpPayload1Length = 256
+ udpPayload2Length = 128
+ // Used to test cases where the fragment blocks are not a multiple of
+ // the fragment block size of 8 (RFC 8200 section 4.5).
+ udpPayload3Length = 127
+ fragmentExtHdrLen = 8
+ // Note, not all routing extension headers will be 8 bytes but this test
+ // uses 8 byte routing extension headers for most sub tests.
+ routingExtHdrLen = 8
+ )
udpGen := func(payload []byte, multiplier uint8) buffer.View {
payloadLen := len(payload)
@@ -716,6 +721,10 @@ func TestReceiveIPv6Fragments(t *testing.T) {
udpPayload2 := udpPayload2Buf[:]
ipv6Payload2 := udpGen(udpPayload2, 2)
+ var udpPayload3Buf [udpPayload3Length]byte
+ udpPayload3 := udpPayload3Buf[:]
+ ipv6Payload3 := udpGen(udpPayload3, 3)
+
tests := []struct {
name string
expectedPayload []byte
@@ -751,6 +760,24 @@ func TestReceiveIPv6Fragments(t *testing.T) {
expectedPayloads: [][]byte{udpPayload1},
},
{
+ name: "Atomic fragment with size not a multiple of fragment block size",
+ fragments: []fragmentData{
+ {
+ nextHdr: fragmentExtHdrID,
+ data: buffer.NewVectorisedView(
+ fragmentExtHdrLen+len(ipv6Payload3),
+ []buffer.View{
+ // Fragment extension header.
+ buffer.View([]byte{uint8(header.UDPProtocolNumber), 0, 0, 0, 0, 0, 0, 0}),
+
+ ipv6Payload3,
+ },
+ ),
+ },
+ },
+ expectedPayloads: [][]byte{udpPayload3},
+ },
+ {
name: "Two fragments",
fragments: []fragmentData{
{
@@ -785,6 +812,74 @@ func TestReceiveIPv6Fragments(t *testing.T) {
expectedPayloads: [][]byte{udpPayload1},
},
{
+ name: "Two fragments with last fragment size not a multiple of fragment block size",
+ fragments: []fragmentData{
+ {
+ nextHdr: fragmentExtHdrID,
+ data: buffer.NewVectorisedView(
+ fragmentExtHdrLen+64,
+ []buffer.View{
+ // Fragment extension header.
+ //
+ // Fragment offset = 0, More = true, ID = 1
+ buffer.View([]byte{uint8(header.UDPProtocolNumber), 0, 0, 1, 0, 0, 0, 1}),
+
+ ipv6Payload3[:64],
+ },
+ ),
+ },
+ {
+ nextHdr: fragmentExtHdrID,
+ data: buffer.NewVectorisedView(
+ fragmentExtHdrLen+len(ipv6Payload3)-64,
+ []buffer.View{
+ // Fragment extension header.
+ //
+ // Fragment offset = 8, More = false, ID = 1
+ buffer.View([]byte{uint8(header.UDPProtocolNumber), 0, 0, 64, 0, 0, 0, 1}),
+
+ ipv6Payload3[64:],
+ },
+ ),
+ },
+ },
+ expectedPayloads: [][]byte{udpPayload3},
+ },
+ {
+ name: "Two fragments with first fragment size not a multiple of fragment block size",
+ fragments: []fragmentData{
+ {
+ nextHdr: fragmentExtHdrID,
+ data: buffer.NewVectorisedView(
+ fragmentExtHdrLen+63,
+ []buffer.View{
+ // Fragment extension header.
+ //
+ // Fragment offset = 0, More = true, ID = 1
+ buffer.View([]byte{uint8(header.UDPProtocolNumber), 0, 0, 1, 0, 0, 0, 1}),
+
+ ipv6Payload3[:63],
+ },
+ ),
+ },
+ {
+ nextHdr: fragmentExtHdrID,
+ data: buffer.NewVectorisedView(
+ fragmentExtHdrLen+len(ipv6Payload3)-63,
+ []buffer.View{
+ // Fragment extension header.
+ //
+ // Fragment offset = 8, More = false, ID = 1
+ buffer.View([]byte{uint8(header.UDPProtocolNumber), 0, 0, 64, 0, 0, 0, 1}),
+
+ ipv6Payload3[63:],
+ },
+ ),
+ },
+ },
+ expectedPayloads: nil,
+ },
+ {
name: "Two fragments with different IDs",
fragments: []fragmentData{
{
diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD
index 6b9a6b316..1c58bed2d 100644
--- a/pkg/tcpip/stack/BUILD
+++ b/pkg/tcpip/stack/BUILD
@@ -16,6 +16,18 @@ go_template_instance(
)
go_template_instance(
+ name = "neighbor_entry_list",
+ out = "neighbor_entry_list.go",
+ package = "stack",
+ prefix = "neighborEntry",
+ template = "//pkg/ilist:generic_list",
+ types = {
+ "Element": "*neighborEntry",
+ "Linker": "*neighborEntry",
+ },
+)
+
+go_template_instance(
name = "packet_buffer_list",
out = "packet_buffer_list.go",
package = "stack",
@@ -53,7 +65,12 @@ go_library(
"linkaddrcache.go",
"linkaddrentry_list.go",
"ndp.go",
+ "neighbor_cache.go",
+ "neighbor_entry.go",
+ "neighbor_entry_list.go",
+ "neighborstate_string.go",
"nic.go",
+ "nud.go",
"packet_buffer.go",
"packet_buffer_list.go",
"rand.go",
@@ -89,6 +106,7 @@ go_test(
size = "medium",
srcs = [
"ndp_test.go",
+ "nud_test.go",
"stack_test.go",
"transport_demuxer_test.go",
"transport_test.go",
@@ -110,6 +128,7 @@ go_test(
"//pkg/tcpip/transport/udp",
"//pkg/waiter",
"@com_github_google_go_cmp//cmp:go_default_library",
+ "@com_github_google_go_cmp//cmp/cmpopts:go_default_library",
],
)
@@ -117,8 +136,11 @@ go_test(
name = "stack_test",
size = "small",
srcs = [
+ "fake_time_test.go",
"forwarder_test.go",
"linkaddrcache_test.go",
+ "neighbor_cache_test.go",
+ "neighbor_entry_test.go",
"nic_test.go",
],
library = ":stack",
@@ -128,5 +150,8 @@ go_test(
"//pkg/tcpip",
"//pkg/tcpip/buffer",
"//pkg/tcpip/header",
+ "@com_github_dpjacques_clockwork//:go_default_library",
+ "@com_github_google_go_cmp//cmp:go_default_library",
+ "@com_github_google_go_cmp//cmp/cmpopts:go_default_library",
],
)
diff --git a/pkg/tcpip/stack/fake_time_test.go b/pkg/tcpip/stack/fake_time_test.go
new file mode 100644
index 000000000..92c8cb534
--- /dev/null
+++ b/pkg/tcpip/stack/fake_time_test.go
@@ -0,0 +1,209 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package stack
+
+import (
+ "container/heap"
+ "sync"
+ "time"
+
+ "github.com/dpjacques/clockwork"
+ "gvisor.dev/gvisor/pkg/tcpip"
+)
+
+type fakeClock struct {
+ clock clockwork.FakeClock
+
+ // mu protects the fields below.
+ mu sync.RWMutex
+
+ // times is min-heap of times. A heap is used for quick retrieval of the next
+ // upcoming time of scheduled work.
+ times *timeHeap
+
+ // waitGroups stores one WaitGroup for all work scheduled to execute at the
+ // same time via AfterFunc. This allows parallel execution of all functions
+ // passed to AfterFunc scheduled for the same time.
+ waitGroups map[time.Time]*sync.WaitGroup
+}
+
+func newFakeClock() *fakeClock {
+ return &fakeClock{
+ clock: clockwork.NewFakeClock(),
+ times: &timeHeap{},
+ waitGroups: make(map[time.Time]*sync.WaitGroup),
+ }
+}
+
+var _ tcpip.Clock = (*fakeClock)(nil)
+
+// NowNanoseconds implements tcpip.Clock.NowNanoseconds.
+func (fc *fakeClock) NowNanoseconds() int64 {
+ return fc.clock.Now().UnixNano()
+}
+
+// NowMonotonic implements tcpip.Clock.NowMonotonic.
+func (fc *fakeClock) NowMonotonic() int64 {
+ return fc.NowNanoseconds()
+}
+
+// AfterFunc implements tcpip.Clock.AfterFunc.
+func (fc *fakeClock) AfterFunc(d time.Duration, f func()) tcpip.Timer {
+ until := fc.clock.Now().Add(d)
+ wg := fc.addWait(until)
+ return &fakeTimer{
+ clock: fc,
+ until: until,
+ timer: fc.clock.AfterFunc(d, func() {
+ defer wg.Done()
+ f()
+ }),
+ }
+}
+
+// addWait adds an additional wait to the WaitGroup for parallel execution of
+// all work scheduled for t. Returns a reference to the WaitGroup modified.
+func (fc *fakeClock) addWait(t time.Time) *sync.WaitGroup {
+ fc.mu.RLock()
+ wg, ok := fc.waitGroups[t]
+ fc.mu.RUnlock()
+
+ if ok {
+ wg.Add(1)
+ return wg
+ }
+
+ fc.mu.Lock()
+ heap.Push(fc.times, t)
+ fc.mu.Unlock()
+
+ wg = &sync.WaitGroup{}
+ wg.Add(1)
+
+ fc.mu.Lock()
+ fc.waitGroups[t] = wg
+ fc.mu.Unlock()
+
+ return wg
+}
+
+// removeWait removes a wait from the WaitGroup for parallel execution of all
+// work scheduled for t.
+func (fc *fakeClock) removeWait(t time.Time) {
+ fc.mu.RLock()
+ defer fc.mu.RUnlock()
+
+ wg := fc.waitGroups[t]
+ wg.Done()
+}
+
+// advance executes all work that have been scheduled to execute within d from
+// the current fake time. Blocks until all work has completed execution.
+func (fc *fakeClock) advance(d time.Duration) {
+ // Block until all the work is done
+ until := fc.clock.Now().Add(d)
+ for {
+ fc.mu.Lock()
+ if fc.times.Len() == 0 {
+ fc.mu.Unlock()
+ return
+ }
+
+ t := heap.Pop(fc.times).(time.Time)
+ if t.After(until) {
+ // No work to do
+ heap.Push(fc.times, t)
+ fc.mu.Unlock()
+ return
+ }
+ fc.mu.Unlock()
+
+ diff := t.Sub(fc.clock.Now())
+ fc.clock.Advance(diff)
+
+ fc.mu.RLock()
+ wg := fc.waitGroups[t]
+ fc.mu.RUnlock()
+
+ wg.Wait()
+
+ fc.mu.Lock()
+ delete(fc.waitGroups, t)
+ fc.mu.Unlock()
+ }
+}
+
+type fakeTimer struct {
+ clock *fakeClock
+ timer clockwork.Timer
+
+ mu sync.RWMutex
+ until time.Time
+}
+
+var _ tcpip.Timer = (*fakeTimer)(nil)
+
+// Reset implements tcpip.Timer.Reset.
+func (ft *fakeTimer) Reset(d time.Duration) {
+ if !ft.timer.Reset(d) {
+ return
+ }
+
+ ft.mu.Lock()
+ defer ft.mu.Unlock()
+
+ ft.clock.removeWait(ft.until)
+ ft.until = ft.clock.clock.Now().Add(d)
+ ft.clock.addWait(ft.until)
+}
+
+// Stop implements tcpip.Timer.Stop.
+func (ft *fakeTimer) Stop() bool {
+ if !ft.timer.Stop() {
+ return false
+ }
+
+ ft.mu.RLock()
+ defer ft.mu.RUnlock()
+
+ ft.clock.removeWait(ft.until)
+ return true
+}
+
+type timeHeap []time.Time
+
+var _ heap.Interface = (*timeHeap)(nil)
+
+func (h timeHeap) Len() int {
+ return len(h)
+}
+
+func (h timeHeap) Less(i, j int) bool {
+ return h[i].Before(h[j])
+}
+
+func (h timeHeap) Swap(i, j int) {
+ h[i], h[j] = h[j], h[i]
+}
+
+func (h *timeHeap) Push(x interface{}) {
+ *h = append(*h, x.(time.Time))
+}
+
+func (h *timeHeap) Pop() interface{} {
+ last := (*h)[len(*h)-1]
+ *h = (*h)[:len(*h)-1]
+ return last
+}
diff --git a/pkg/tcpip/stack/ndp.go b/pkg/tcpip/stack/ndp.go
index 9dce11a97..5174e639c 100644
--- a/pkg/tcpip/stack/ndp.go
+++ b/pkg/tcpip/stack/ndp.go
@@ -33,12 +33,6 @@ const (
// Default = 1 (from RFC 4862 section 5.1)
defaultDupAddrDetectTransmits = 1
- // defaultRetransmitTimer is the default amount of time to wait between
- // sending NDP Neighbor solicitation messages.
- //
- // Default = 1s (from RFC 4861 section 10).
- defaultRetransmitTimer = time.Second
-
// defaultMaxRtrSolicitations is the default number of Router
// Solicitation messages to send when a NIC becomes enabled.
//
@@ -79,16 +73,6 @@ const (
// Default = true.
defaultAutoGenGlobalAddresses = true
- // minimumRetransmitTimer is the minimum amount of time to wait between
- // sending NDP Neighbor solicitation messages. Note, RFC 4861 does
- // not impose a minimum Retransmit Timer, but we do here to make sure
- // the messages are not sent all at once. We also come to this value
- // because in the RetransmitTimer field of a Router Advertisement, a
- // value of 0 means unspecified, so the smallest valid value is 1.
- // Note, the unit of the RetransmitTimer field in the Router
- // Advertisement is milliseconds.
- minimumRetransmitTimer = time.Millisecond
-
// minimumRtrSolicitationInterval is the minimum amount of time to wait
// between sending Router Solicitation messages. This limit is imposed
// to make sure that Router Solicitation messages are not sent all at
diff --git a/pkg/tcpip/stack/neighbor_cache.go b/pkg/tcpip/stack/neighbor_cache.go
new file mode 100644
index 000000000..1d37716c2
--- /dev/null
+++ b/pkg/tcpip/stack/neighbor_cache.go
@@ -0,0 +1,335 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package stack
+
+import (
+ "fmt"
+ "time"
+
+ "gvisor.dev/gvisor/pkg/sleep"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/tcpip"
+)
+
+const neighborCacheSize = 512 // max entries per interface
+
+// neighborCache maps IP addresses to link addresses. It uses the Least
+// Recently Used (LRU) eviction strategy to implement a bounded cache for
+// dynmically acquired entries. It contains the state machine and configuration
+// for running Neighbor Unreachability Detection (NUD).
+//
+// There are two types of entries in the neighbor cache:
+// 1. Dynamic entries are discovered automatically by neighbor discovery
+// protocols (e.g. ARP, NDP). These protocols will attempt to reconfirm
+// reachability with the device once the entry's state becomes Stale.
+// 2. Static entries are explicitly added by a user and have no expiration.
+// Their state is always Static. The amount of static entries stored in the
+// cache is unbounded.
+//
+// neighborCache implements NUDHandler.
+type neighborCache struct {
+ nic *NIC
+ state *NUDState
+
+ // mu protects the fields below.
+ mu sync.RWMutex
+
+ cache map[tcpip.Address]*neighborEntry
+ dynamic struct {
+ lru neighborEntryList
+
+ // count tracks the amount of dynamic entries in the cache. This is
+ // needed since static entries do not count towards the LRU cache
+ // eviction strategy.
+ count uint16
+ }
+}
+
+var _ NUDHandler = (*neighborCache)(nil)
+
+// getOrCreateEntry retrieves a cache entry associated with addr. The
+// returned entry is always refreshed in the cache (it is reachable via the
+// map, and its place is bumped in LRU).
+//
+// If a matching entry exists in the cache, it is returned. If no matching
+// entry exists and the cache is full, an existing entry is evicted via LRU,
+// reset to state incomplete, and returned. If no matching entry exists and the
+// cache is not full, a new entry with state incomplete is allocated and
+// returned.
+func (n *neighborCache) getOrCreateEntry(remoteAddr, localAddr tcpip.Address, linkRes LinkAddressResolver) *neighborEntry {
+ n.mu.Lock()
+ defer n.mu.Unlock()
+
+ if entry, ok := n.cache[remoteAddr]; ok {
+ entry.mu.RLock()
+ if entry.neigh.State != Static {
+ n.dynamic.lru.Remove(entry)
+ n.dynamic.lru.PushFront(entry)
+ }
+ entry.mu.RUnlock()
+ return entry
+ }
+
+ // The entry that needs to be created must be dynamic since all static
+ // entries are directly added to the cache via addStaticEntry.
+ entry := newNeighborEntry(n.nic, remoteAddr, localAddr, n.state, linkRes)
+ if n.dynamic.count == neighborCacheSize {
+ e := n.dynamic.lru.Back()
+ e.mu.Lock()
+
+ delete(n.cache, e.neigh.Addr)
+ n.dynamic.lru.Remove(e)
+ n.dynamic.count--
+
+ e.dispatchRemoveEventLocked()
+ e.setStateLocked(Unknown)
+ e.notifyWakersLocked()
+ e.mu.Unlock()
+ }
+ n.cache[remoteAddr] = entry
+ n.dynamic.lru.PushFront(entry)
+ n.dynamic.count++
+ return entry
+}
+
+// entry looks up the neighbor cache for translating address to link address
+// (e.g. IP -> MAC). If the LinkEndpoint requests address resolution and there
+// is a LinkAddressResolver registered with the network protocol, the cache
+// attempts to resolve the address and returns ErrWouldBlock. If a Waker is
+// provided, it will be notified when address resolution is complete (success
+// or not).
+//
+// If address resolution is required, ErrNoLinkAddress and a notification
+// channel is returned for the top level caller to block. Channel is closed
+// once address resolution is complete (success or not).
+func (n *neighborCache) entry(remoteAddr, localAddr tcpip.Address, linkRes LinkAddressResolver, w *sleep.Waker) (NeighborEntry, <-chan struct{}, *tcpip.Error) {
+ if linkRes != nil {
+ if linkAddr, ok := linkRes.ResolveStaticAddress(remoteAddr); ok {
+ e := NeighborEntry{
+ Addr: remoteAddr,
+ LocalAddr: localAddr,
+ LinkAddr: linkAddr,
+ State: Static,
+ UpdatedAt: time.Now(),
+ }
+ return e, nil, nil
+ }
+ }
+
+ entry := n.getOrCreateEntry(remoteAddr, localAddr, linkRes)
+ entry.mu.Lock()
+ defer entry.mu.Unlock()
+
+ switch s := entry.neigh.State; s {
+ case Reachable, Static:
+ return entry.neigh, nil, nil
+
+ case Unknown, Incomplete, Stale, Delay, Probe:
+ entry.addWakerLocked(w)
+
+ if entry.done == nil {
+ // Address resolution needs to be initiated.
+ if linkRes == nil {
+ return entry.neigh, nil, tcpip.ErrNoLinkAddress
+ }
+ entry.done = make(chan struct{})
+ }
+
+ entry.handlePacketQueuedLocked()
+ return entry.neigh, entry.done, tcpip.ErrWouldBlock
+
+ case Failed:
+ return entry.neigh, nil, tcpip.ErrNoLinkAddress
+
+ default:
+ panic(fmt.Sprintf("Invalid cache entry state: %s", s))
+ }
+}
+
+// removeWaker removes a waker that has been added when link resolution for
+// addr was requested.
+func (n *neighborCache) removeWaker(addr tcpip.Address, waker *sleep.Waker) {
+ n.mu.Lock()
+ if entry, ok := n.cache[addr]; ok {
+ delete(entry.wakers, waker)
+ }
+ n.mu.Unlock()
+}
+
+// entries returns all entries in the neighbor cache.
+func (n *neighborCache) entries() []NeighborEntry {
+ entries := make([]NeighborEntry, 0, len(n.cache))
+ n.mu.RLock()
+ for _, entry := range n.cache {
+ entry.mu.RLock()
+ entries = append(entries, entry.neigh)
+ entry.mu.RUnlock()
+ }
+ n.mu.RUnlock()
+ return entries
+}
+
+// addStaticEntry adds a static entry to the neighbor cache, mapping an IP
+// address to a link address. If a dynamic entry exists in the neighbor cache
+// with the same address, it will be replaced with this static entry. If a
+// static entry exists with the same address but different link address, it
+// will be updated with the new link address. If a static entry exists with the
+// same address and link address, nothing will happen.
+func (n *neighborCache) addStaticEntry(addr tcpip.Address, linkAddr tcpip.LinkAddress) {
+ n.mu.Lock()
+ defer n.mu.Unlock()
+
+ if entry, ok := n.cache[addr]; ok {
+ entry.mu.Lock()
+ if entry.neigh.State != Static {
+ // Dynamic entry found with the same address.
+ n.dynamic.lru.Remove(entry)
+ n.dynamic.count--
+ } else if entry.neigh.LinkAddr == linkAddr {
+ // Static entry found with the same address and link address.
+ entry.mu.Unlock()
+ return
+ } else {
+ // Static entry found with the same address but different link address.
+ entry.neigh.LinkAddr = linkAddr
+ entry.dispatchChangeEventLocked(entry.neigh.State)
+ entry.mu.Unlock()
+ return
+ }
+
+ // Notify that resolution has been interrupted, just in case the entry was
+ // in the Incomplete or Probe state.
+ entry.dispatchRemoveEventLocked()
+ entry.setStateLocked(Unknown)
+ entry.notifyWakersLocked()
+ entry.mu.Unlock()
+ }
+
+ entry := newStaticNeighborEntry(n.nic, addr, linkAddr, n.state)
+ n.cache[addr] = entry
+}
+
+// removeEntryLocked removes the specified entry from the neighbor cache.
+func (n *neighborCache) removeEntryLocked(entry *neighborEntry) {
+ if entry.neigh.State != Static {
+ n.dynamic.lru.Remove(entry)
+ n.dynamic.count--
+ }
+ if entry.neigh.State != Failed {
+ entry.dispatchRemoveEventLocked()
+ }
+ entry.setStateLocked(Unknown)
+ entry.notifyWakersLocked()
+
+ delete(n.cache, entry.neigh.Addr)
+}
+
+// removeEntry removes a dynamic or static entry by address from the neighbor
+// cache. Returns true if the entry was found and deleted.
+func (n *neighborCache) removeEntry(addr tcpip.Address) bool {
+ n.mu.Lock()
+ defer n.mu.Unlock()
+
+ entry, ok := n.cache[addr]
+ if !ok {
+ return false
+ }
+
+ entry.mu.Lock()
+ defer entry.mu.Unlock()
+
+ n.removeEntryLocked(entry)
+ return true
+}
+
+// clear removes all dynamic and static entries from the neighbor cache.
+func (n *neighborCache) clear() {
+ n.mu.Lock()
+ defer n.mu.Unlock()
+
+ for _, entry := range n.cache {
+ entry.mu.Lock()
+ entry.dispatchRemoveEventLocked()
+ entry.setStateLocked(Unknown)
+ entry.notifyWakersLocked()
+ entry.mu.Unlock()
+ }
+
+ n.dynamic.lru = neighborEntryList{}
+ n.cache = make(map[tcpip.Address]*neighborEntry)
+ n.dynamic.count = 0
+}
+
+// config returns the NUD configuration.
+func (n *neighborCache) config() NUDConfigurations {
+ return n.state.Config()
+}
+
+// setConfig changes the NUD configuration.
+//
+// If config contains invalid NUD configuration values, it will be fixed to
+// use default values for the erroneous values.
+func (n *neighborCache) setConfig(config NUDConfigurations) {
+ config.resetInvalidFields()
+ n.state.SetConfig(config)
+}
+
+// HandleProbe implements NUDHandler.HandleProbe by following the logic defined
+// in RFC 4861 section 7.2.3. Validation of the probe is expected to be handled
+// by the caller.
+func (n *neighborCache) HandleProbe(remoteAddr, localAddr tcpip.Address, protocol tcpip.NetworkProtocolNumber, remoteLinkAddr tcpip.LinkAddress) {
+ entry := n.getOrCreateEntry(remoteAddr, localAddr, nil)
+ entry.mu.Lock()
+ entry.handleProbeLocked(remoteLinkAddr)
+ entry.mu.Unlock()
+}
+
+// HandleConfirmation implements NUDHandler.HandleConfirmation by following the
+// logic defined in RFC 4861 section 7.2.5.
+//
+// TODO(gvisor.dev/issue/2277): To protect against ARP poisoning and other
+// attacks against NDP functions, Secure Neighbor Discovery (SEND) Protocol
+// should be deployed where preventing access to the broadcast segment might
+// not be possible. SEND uses RSA key pairs to produce cryptographically
+// generated addresses, as defined in RFC 3972, Cryptographically Generated
+// Addresses (CGA). This ensures that the claimed source of an NDP message is
+// the owner of the claimed address.
+func (n *neighborCache) HandleConfirmation(addr tcpip.Address, linkAddr tcpip.LinkAddress, flags ReachabilityConfirmationFlags) {
+ n.mu.RLock()
+ entry, ok := n.cache[addr]
+ n.mu.RUnlock()
+ if ok {
+ entry.mu.Lock()
+ entry.handleConfirmationLocked(linkAddr, flags)
+ entry.mu.Unlock()
+ }
+ // The confirmation SHOULD be silently discarded if the recipient did not
+ // initiate any communication with the target. This is indicated if there is
+ // no matching entry for the remote address.
+}
+
+// HandleUpperLevelConfirmation implements
+// NUDHandler.HandleUpperLevelConfirmation by following the logic defined in
+// RFC 4861 section 7.3.1.
+func (n *neighborCache) HandleUpperLevelConfirmation(addr tcpip.Address) {
+ n.mu.RLock()
+ entry, ok := n.cache[addr]
+ n.mu.RUnlock()
+ if ok {
+ entry.mu.Lock()
+ entry.handleUpperLevelConfirmationLocked()
+ entry.mu.Unlock()
+ }
+}
diff --git a/pkg/tcpip/stack/neighbor_cache_test.go b/pkg/tcpip/stack/neighbor_cache_test.go
new file mode 100644
index 000000000..4cb2c9c6b
--- /dev/null
+++ b/pkg/tcpip/stack/neighbor_cache_test.go
@@ -0,0 +1,1752 @@
+// Copyright 2019 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package stack
+
+import (
+ "bytes"
+ "encoding/binary"
+ "fmt"
+ "math"
+ "math/rand"
+ "strings"
+ "sync"
+ "sync/atomic"
+ "testing"
+ "time"
+
+ "github.com/google/go-cmp/cmp"
+ "github.com/google/go-cmp/cmp/cmpopts"
+ "gvisor.dev/gvisor/pkg/sleep"
+ "gvisor.dev/gvisor/pkg/tcpip"
+)
+
+const (
+ // entryStoreSize is the default number of entries that will be generated and
+ // added to the entry store. This number needs to be larger than the size of
+ // the neighbor cache to give ample opportunity for verifying behavior during
+ // cache overflows. Four times the size of the neighbor cache allows for
+ // three complete cache overflows.
+ entryStoreSize = 4 * neighborCacheSize
+
+ // typicalLatency is the typical latency for an ARP or NDP packet to travel
+ // to a router and back.
+ typicalLatency = time.Millisecond
+
+ // testEntryBroadcastAddr is a special address that indicates a packet should
+ // be sent to all nodes.
+ testEntryBroadcastAddr = tcpip.Address("broadcast")
+
+ // testEntryLocalAddr is the source address of neighbor probes.
+ testEntryLocalAddr = tcpip.Address("local_addr")
+
+ // testEntryBroadcastLinkAddr is a special link address sent back to
+ // multicast neighbor probes.
+ testEntryBroadcastLinkAddr = tcpip.LinkAddress("mac_broadcast")
+
+ // infiniteDuration indicates that a task will not occur in our lifetime.
+ infiniteDuration = time.Duration(math.MaxInt64)
+)
+
+// entryDiffOpts returns the options passed to cmp.Diff to compare neighbor
+// entries. The UpdatedAt field is ignored due to a lack of a deterministic
+// method to predict the time that an event will be dispatched.
+func entryDiffOpts() []cmp.Option {
+ return []cmp.Option{
+ cmpopts.IgnoreFields(NeighborEntry{}, "UpdatedAt"),
+ }
+}
+
+// entryDiffOptsWithSort is like entryDiffOpts but also includes an option to
+// sort slices of entries for cases where ordering must be ignored.
+func entryDiffOptsWithSort() []cmp.Option {
+ return []cmp.Option{
+ cmpopts.IgnoreFields(NeighborEntry{}, "UpdatedAt"),
+ cmpopts.SortSlices(func(a, b NeighborEntry) bool {
+ return strings.Compare(string(a.Addr), string(b.Addr)) < 0
+ }),
+ }
+}
+
+func newTestNeighborCache(nudDisp NUDDispatcher, config NUDConfigurations, clock tcpip.Clock) *neighborCache {
+ config.resetInvalidFields()
+ rng := rand.New(rand.NewSource(time.Now().UnixNano()))
+ return &neighborCache{
+ nic: &NIC{
+ stack: &Stack{
+ clock: clock,
+ nudDisp: nudDisp,
+ },
+ id: 1,
+ },
+ state: NewNUDState(config, rng),
+ cache: make(map[tcpip.Address]*neighborEntry, neighborCacheSize),
+ }
+}
+
+// testEntryStore contains a set of IP to NeighborEntry mappings.
+type testEntryStore struct {
+ mu sync.RWMutex
+ entriesMap map[tcpip.Address]NeighborEntry
+}
+
+func toAddress(i int) tcpip.Address {
+ buf := new(bytes.Buffer)
+ binary.Write(buf, binary.BigEndian, uint8(1))
+ binary.Write(buf, binary.BigEndian, uint8(0))
+ binary.Write(buf, binary.BigEndian, uint16(i))
+ return tcpip.Address(buf.String())
+}
+
+func toLinkAddress(i int) tcpip.LinkAddress {
+ buf := new(bytes.Buffer)
+ binary.Write(buf, binary.BigEndian, uint8(1))
+ binary.Write(buf, binary.BigEndian, uint8(0))
+ binary.Write(buf, binary.BigEndian, uint32(i))
+ return tcpip.LinkAddress(buf.String())
+}
+
+// newTestEntryStore returns a testEntryStore pre-populated with entries.
+func newTestEntryStore() *testEntryStore {
+ store := &testEntryStore{
+ entriesMap: make(map[tcpip.Address]NeighborEntry),
+ }
+ for i := 0; i < entryStoreSize; i++ {
+ addr := toAddress(i)
+ linkAddr := toLinkAddress(i)
+
+ store.entriesMap[addr] = NeighborEntry{
+ Addr: addr,
+ LocalAddr: testEntryLocalAddr,
+ LinkAddr: linkAddr,
+ }
+ }
+ return store
+}
+
+// size returns the number of entries in the store.
+func (s *testEntryStore) size() int {
+ s.mu.RLock()
+ defer s.mu.RUnlock()
+ return len(s.entriesMap)
+}
+
+// entry returns the entry at index i. Returns an empty entry and false if i is
+// out of bounds.
+func (s *testEntryStore) entry(i int) (NeighborEntry, bool) {
+ return s.entryByAddr(toAddress(i))
+}
+
+// entryByAddr returns the entry matching addr for situations when the index is
+// not available. Returns an empty entry and false if no entries match addr.
+func (s *testEntryStore) entryByAddr(addr tcpip.Address) (NeighborEntry, bool) {
+ s.mu.RLock()
+ defer s.mu.RUnlock()
+ entry, ok := s.entriesMap[addr]
+ return entry, ok
+}
+
+// entries returns all entries in the store.
+func (s *testEntryStore) entries() []NeighborEntry {
+ entries := make([]NeighborEntry, 0, len(s.entriesMap))
+ s.mu.RLock()
+ defer s.mu.RUnlock()
+ for i := 0; i < entryStoreSize; i++ {
+ addr := toAddress(i)
+ if entry, ok := s.entriesMap[addr]; ok {
+ entries = append(entries, entry)
+ }
+ }
+ return entries
+}
+
+// set modifies the link addresses of an entry.
+func (s *testEntryStore) set(i int, linkAddr tcpip.LinkAddress) {
+ addr := toAddress(i)
+ s.mu.Lock()
+ defer s.mu.Unlock()
+ if entry, ok := s.entriesMap[addr]; ok {
+ entry.LinkAddr = linkAddr
+ s.entriesMap[addr] = entry
+ }
+}
+
+// testNeighborResolver implements LinkAddressResolver to emulate sending a
+// neighbor probe.
+type testNeighborResolver struct {
+ clock tcpip.Clock
+ neigh *neighborCache
+ entries *testEntryStore
+ delay time.Duration
+ onLinkAddressRequest func()
+}
+
+var _ LinkAddressResolver = (*testNeighborResolver)(nil)
+
+func (r *testNeighborResolver) LinkAddressRequest(addr, localAddr tcpip.Address, linkAddr tcpip.LinkAddress, linkEP LinkEndpoint) *tcpip.Error {
+ // Delay handling the request to emulate network latency.
+ r.clock.AfterFunc(r.delay, func() {
+ r.fakeRequest(addr)
+ })
+
+ // Execute post address resolution action, if available.
+ if f := r.onLinkAddressRequest; f != nil {
+ f()
+ }
+ return nil
+}
+
+// fakeRequest emulates handling a response for a link address request.
+func (r *testNeighborResolver) fakeRequest(addr tcpip.Address) {
+ if entry, ok := r.entries.entryByAddr(addr); ok {
+ r.neigh.HandleConfirmation(addr, entry.LinkAddr, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ }
+}
+
+func (*testNeighborResolver) ResolveStaticAddress(addr tcpip.Address) (tcpip.LinkAddress, bool) {
+ if addr == testEntryBroadcastAddr {
+ return testEntryBroadcastLinkAddr, true
+ }
+ return "", false
+}
+
+func (*testNeighborResolver) LinkAddressProtocol() tcpip.NetworkProtocolNumber {
+ return 0
+}
+
+type entryEvent struct {
+ nicID tcpip.NICID
+ address tcpip.Address
+ linkAddr tcpip.LinkAddress
+ state NeighborState
+}
+
+func TestNeighborCacheGetConfig(t *testing.T) {
+ nudDisp := testNUDDispatcher{}
+ c := DefaultNUDConfigurations()
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, c, clock)
+
+ if got, want := neigh.config(), c; got != want {
+ t.Errorf("got neigh.config() = %+v, want = %+v", got, want)
+ }
+
+ // No events should have been dispatched.
+ nudDisp.mu.Lock()
+ defer nudDisp.mu.Unlock()
+ if diff := cmp.Diff(nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+func TestNeighborCacheSetConfig(t *testing.T) {
+ nudDisp := testNUDDispatcher{}
+ c := DefaultNUDConfigurations()
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, c, clock)
+
+ c.MinRandomFactor = 1
+ c.MaxRandomFactor = 1
+ neigh.setConfig(c)
+
+ if got, want := neigh.config(), c; got != want {
+ t.Errorf("got neigh.config() = %+v, want = %+v", got, want)
+ }
+
+ // No events should have been dispatched.
+ nudDisp.mu.Lock()
+ defer nudDisp.mu.Unlock()
+ if diff := cmp.Diff(nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+func TestNeighborCacheEntry(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, c, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+
+ clock.advance(typicalLatency)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...)
+ nudDisp.events = nil
+ nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != nil {
+ t.Fatalf("unexpected error from neigh.entry(%s, %s, _, nil): %s", entry.Addr, entry.LocalAddr, err)
+ }
+
+ // No more events should have been dispatched.
+ nudDisp.mu.Lock()
+ defer nudDisp.mu.Unlock()
+ if diff := cmp.Diff(nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+// TestNeighborCacheEntryNoLinkAddress verifies calling entry() without a
+// LinkAddressResolver returns ErrNoLinkAddress.
+func TestNeighborCacheEntryNoLinkAddress(t *testing.T) {
+ nudDisp := testNUDDispatcher{}
+ c := DefaultNUDConfigurations()
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, c, clock)
+ store := newTestEntryStore()
+
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, nil, nil)
+ if err != tcpip.ErrNoLinkAddress {
+ t.Errorf("got neigh.entry(%s, %s, nil, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrNoLinkAddress)
+ }
+
+ // No events should have been dispatched.
+ nudDisp.mu.Lock()
+ defer nudDisp.mu.Unlock()
+ if diff := cmp.Diff(nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+func TestNeighborCacheRemoveEntry(t *testing.T) {
+ config := DefaultNUDConfigurations()
+
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+
+ clock.advance(typicalLatency)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...)
+ nudDisp.events = nil
+ nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ neigh.removeEntry(entry.Addr)
+
+ {
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...)
+ nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrWouldBlock {
+ t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+}
+
+type testContext struct {
+ clock *fakeClock
+ neigh *neighborCache
+ store *testEntryStore
+ linkRes *testNeighborResolver
+ nudDisp *testNUDDispatcher
+}
+
+func newTestContext(c NUDConfigurations) testContext {
+ nudDisp := &testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(nudDisp, c, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ return testContext{
+ clock: clock,
+ neigh: neigh,
+ store: store,
+ linkRes: linkRes,
+ nudDisp: nudDisp,
+ }
+}
+
+type overflowOptions struct {
+ startAtEntryIndex int
+ wantStaticEntries []NeighborEntry
+}
+
+func (c *testContext) overflowCache(opts overflowOptions) error {
+ // Fill the neighbor cache to capacity to verify the LRU eviction strategy is
+ // working properly after the entry removal.
+ for i := opts.startAtEntryIndex; i < c.store.size(); i++ {
+ // Add a new entry
+ entry, ok := c.store.entry(i)
+ if !ok {
+ return fmt.Errorf("c.store.entry(%d) not found", i)
+ }
+ if _, _, err := c.neigh.entry(entry.Addr, entry.LocalAddr, c.linkRes, nil); err != tcpip.ErrWouldBlock {
+ return fmt.Errorf("got c.neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ c.clock.advance(c.neigh.config().RetransmitTimer)
+
+ var wantEvents []testEntryEventInfo
+
+ // When beyond the full capacity, the cache will evict an entry as per the
+ // LRU eviction strategy. Note that the number of static entries should not
+ // affect the total number of dynamic entries that can be added.
+ if i >= neighborCacheSize+opts.startAtEntryIndex {
+ removedEntry, ok := c.store.entry(i - neighborCacheSize)
+ if !ok {
+ return fmt.Errorf("store.entry(%d) not found", i-neighborCacheSize)
+ }
+ wantEvents = append(wantEvents, testEntryEventInfo{
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: removedEntry.Addr,
+ LinkAddr: removedEntry.LinkAddr,
+ State: Reachable,
+ })
+ }
+
+ wantEvents = append(wantEvents, testEntryEventInfo{
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ }, testEntryEventInfo{
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ })
+
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ return fmt.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ // Expect to find only the most recent entries. The order of entries reported
+ // by entries() is undeterministic, so entries have to be sorted before
+ // comparison.
+ wantUnsortedEntries := opts.wantStaticEntries
+ for i := c.store.size() - neighborCacheSize; i < c.store.size(); i++ {
+ entry, ok := c.store.entry(i)
+ if !ok {
+ return fmt.Errorf("c.store.entry(%d) not found", i)
+ }
+ wantEntry := NeighborEntry{
+ Addr: entry.Addr,
+ LocalAddr: entry.LocalAddr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ }
+ wantUnsortedEntries = append(wantUnsortedEntries, wantEntry)
+ }
+
+ if diff := cmp.Diff(c.neigh.entries(), wantUnsortedEntries, entryDiffOptsWithSort()...); diff != "" {
+ return fmt.Errorf("neighbor entries mismatch (-got, +want):\n%s", diff)
+ }
+
+ // No more events should have been dispatched.
+ c.nudDisp.mu.Lock()
+ defer c.nudDisp.mu.Unlock()
+ if diff := cmp.Diff(c.nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ return fmt.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ return nil
+}
+
+// TestNeighborCacheOverflow verifies that the LRU cache eviction strategy
+// respects the dynamic entry count.
+func TestNeighborCacheOverflow(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ // Stay in Reachable so the cache can overflow
+ config.BaseReachableTime = infiniteDuration
+ config.MinRandomFactor = 1
+ config.MaxRandomFactor = 1
+
+ c := newTestContext(config)
+ opts := overflowOptions{
+ startAtEntryIndex: 0,
+ }
+ if err := c.overflowCache(opts); err != nil {
+ t.Errorf("c.overflowCache(%+v): %s", opts, err)
+ }
+}
+
+// TestNeighborCacheRemoveEntryThenOverflow verifies that the LRU cache
+// eviction strategy respects the dynamic entry count when an entry is removed.
+func TestNeighborCacheRemoveEntryThenOverflow(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ // Stay in Reachable so the cache can overflow
+ config.BaseReachableTime = infiniteDuration
+ config.MinRandomFactor = 1
+ config.MaxRandomFactor = 1
+
+ c := newTestContext(config)
+
+ // Add a dynamic entry
+ entry, ok := c.store.entry(0)
+ if !ok {
+ t.Fatalf("c.store.entry(0) not found")
+ }
+ _, _, err := c.neigh.entry(entry.Addr, entry.LocalAddr, c.linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Errorf("got c.neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ c.clock.advance(c.neigh.config().RetransmitTimer)
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ // Remove the entry
+ c.neigh.removeEntry(entry.Addr)
+
+ {
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ opts := overflowOptions{
+ startAtEntryIndex: 0,
+ }
+ if err := c.overflowCache(opts); err != nil {
+ t.Errorf("c.overflowCache(%+v): %s", opts, err)
+ }
+}
+
+// TestNeighborCacheDuplicateStaticEntryWithSameLinkAddress verifies that
+// adding a duplicate static entry with the same link address does not dispatch
+// any events.
+func TestNeighborCacheDuplicateStaticEntryWithSameLinkAddress(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ c := newTestContext(config)
+
+ // Add a static entry
+ entry, ok := c.store.entry(0)
+ if !ok {
+ t.Fatalf("c.store.entry(0) not found")
+ }
+ staticLinkAddr := entry.LinkAddr + "static"
+ c.neigh.addStaticEntry(entry.Addr, staticLinkAddr)
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: staticLinkAddr,
+ State: Static,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ // Remove the static entry that was just added
+ c.neigh.addStaticEntry(entry.Addr, staticLinkAddr)
+
+ // No more events should have been dispatched.
+ c.nudDisp.mu.Lock()
+ defer c.nudDisp.mu.Unlock()
+ if diff := cmp.Diff(c.nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+// TestNeighborCacheDuplicateStaticEntryWithDifferentLinkAddress verifies that
+// adding a duplicate static entry with a different link address dispatches a
+// change event.
+func TestNeighborCacheDuplicateStaticEntryWithDifferentLinkAddress(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ c := newTestContext(config)
+
+ // Add a static entry
+ entry, ok := c.store.entry(0)
+ if !ok {
+ t.Fatalf("c.store.entry(0) not found")
+ }
+ staticLinkAddr := entry.LinkAddr + "static"
+ c.neigh.addStaticEntry(entry.Addr, staticLinkAddr)
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: staticLinkAddr,
+ State: Static,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ // Add a duplicate entry with a different link address
+ staticLinkAddr += "duplicate"
+ c.neigh.addStaticEntry(entry.Addr, staticLinkAddr)
+ {
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: staticLinkAddr,
+ State: Static,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ defer c.nudDisp.mu.Unlock()
+ if diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+}
+
+// TestNeighborCacheRemoveStaticEntryThenOverflow verifies that the LRU cache
+// eviction strategy respects the dynamic entry count when a static entry is
+// added then removed. In this case, the dynamic entry count shouldn't have
+// been touched.
+func TestNeighborCacheRemoveStaticEntryThenOverflow(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ // Stay in Reachable so the cache can overflow
+ config.BaseReachableTime = infiniteDuration
+ config.MinRandomFactor = 1
+ config.MaxRandomFactor = 1
+
+ c := newTestContext(config)
+
+ // Add a static entry
+ entry, ok := c.store.entry(0)
+ if !ok {
+ t.Fatalf("c.store.entry(0) not found")
+ }
+ staticLinkAddr := entry.LinkAddr + "static"
+ c.neigh.addStaticEntry(entry.Addr, staticLinkAddr)
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: staticLinkAddr,
+ State: Static,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ // Remove the static entry that was just added
+ c.neigh.removeEntry(entry.Addr)
+ {
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: staticLinkAddr,
+ State: Static,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ opts := overflowOptions{
+ startAtEntryIndex: 0,
+ }
+ if err := c.overflowCache(opts); err != nil {
+ t.Errorf("c.overflowCache(%+v): %s", opts, err)
+ }
+}
+
+// TestNeighborCacheOverwriteWithStaticEntryThenOverflow verifies that the LRU
+// cache eviction strategy keeps count of the dynamic entry count when an entry
+// is overwritten by a static entry. Static entries should not count towards
+// the size of the LRU cache.
+func TestNeighborCacheOverwriteWithStaticEntryThenOverflow(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ // Stay in Reachable so the cache can overflow
+ config.BaseReachableTime = infiniteDuration
+ config.MinRandomFactor = 1
+ config.MaxRandomFactor = 1
+
+ c := newTestContext(config)
+
+ // Add a dynamic entry
+ entry, ok := c.store.entry(0)
+ if !ok {
+ t.Fatalf("c.store.entry(0) not found")
+ }
+ _, _, err := c.neigh.entry(entry.Addr, entry.LocalAddr, c.linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Errorf("got c.neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ c.clock.advance(typicalLatency)
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ // Override the entry with a static one using the same address
+ staticLinkAddr := entry.LinkAddr + "static"
+ c.neigh.addStaticEntry(entry.Addr, staticLinkAddr)
+ {
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: staticLinkAddr,
+ State: Static,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ opts := overflowOptions{
+ startAtEntryIndex: 1,
+ wantStaticEntries: []NeighborEntry{
+ {
+ Addr: entry.Addr,
+ LocalAddr: "", // static entries don't need a local address
+ LinkAddr: staticLinkAddr,
+ State: Static,
+ },
+ },
+ }
+ if err := c.overflowCache(opts); err != nil {
+ t.Errorf("c.overflowCache(%+v): %s", opts, err)
+ }
+}
+
+func TestNeighborCacheNotifiesWaker(t *testing.T) {
+ config := DefaultNUDConfigurations()
+
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ w := sleep.Waker{}
+ s := sleep.Sleeper{}
+ const wakerID = 1
+ s.AddWaker(&w, wakerID)
+
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ _, doneCh, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, &w)
+ if err != tcpip.ErrWouldBlock {
+ t.Fatalf("got neigh.entry(%s, %s, _, _ = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ if doneCh == nil {
+ t.Fatalf("expected done channel from neigh.entry(%s, %s, _, _)", entry.Addr, entry.LocalAddr)
+ }
+ clock.advance(typicalLatency)
+
+ select {
+ case <-doneCh:
+ default:
+ t.Fatal("expected notification from done channel")
+ }
+
+ id, ok := s.Fetch(false /* block */)
+ if !ok {
+ t.Errorf("expected waker to be notified after neigh.entry(%s, %s, _, _)", entry.Addr, entry.LocalAddr)
+ }
+ if id != wakerID {
+ t.Errorf("got s.Fetch(false) = %d, want = %d", id, wakerID)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ defer nudDisp.mu.Unlock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+func TestNeighborCacheRemoveWaker(t *testing.T) {
+ config := DefaultNUDConfigurations()
+
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ w := sleep.Waker{}
+ s := sleep.Sleeper{}
+ const wakerID = 1
+ s.AddWaker(&w, wakerID)
+
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ _, doneCh, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, &w)
+ if err != tcpip.ErrWouldBlock {
+ t.Fatalf("got neigh.entry(%s, %s, _, _) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ if doneCh == nil {
+ t.Fatalf("expected done channel from neigh.entry(%s, %s, _, _)", entry.Addr, entry.LocalAddr)
+ }
+
+ // Remove the waker before the neighbor cache has the opportunity to send a
+ // notification.
+ neigh.removeWaker(entry.Addr, &w)
+ clock.advance(typicalLatency)
+
+ select {
+ case <-doneCh:
+ default:
+ t.Fatal("expected notification from done channel")
+ }
+
+ if id, ok := s.Fetch(false /* block */); ok {
+ t.Errorf("unexpected notification from waker with id %d", id)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ defer nudDisp.mu.Unlock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+func TestNeighborCacheAddStaticEntryThenOverflow(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ // Stay in Reachable so the cache can overflow
+ config.BaseReachableTime = infiniteDuration
+ config.MinRandomFactor = 1
+ config.MaxRandomFactor = 1
+
+ c := newTestContext(config)
+
+ entry, ok := c.store.entry(0)
+ if !ok {
+ t.Fatalf("c.store.entry(0) not found")
+ }
+ c.neigh.addStaticEntry(entry.Addr, entry.LinkAddr)
+ e, _, err := c.neigh.entry(entry.Addr, "", nil, nil)
+ if err != nil {
+ t.Errorf("unexpected error from c.neigh.entry(%s, \"\", nil nil): %s", entry.Addr, err)
+ }
+ want := NeighborEntry{
+ Addr: entry.Addr,
+ LocalAddr: "", // static entries don't need a local address
+ LinkAddr: entry.LinkAddr,
+ State: Static,
+ }
+ if diff := cmp.Diff(e, want, entryDiffOpts()...); diff != "" {
+ t.Errorf("c.neigh.entry(%s, \"\", nil, nil) mismatch (-got, +want):\n%s", entry.Addr, diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Static,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ opts := overflowOptions{
+ startAtEntryIndex: 1,
+ wantStaticEntries: []NeighborEntry{
+ {
+ Addr: entry.Addr,
+ LocalAddr: "", // static entries don't need a local address
+ LinkAddr: entry.LinkAddr,
+ State: Static,
+ },
+ },
+ }
+ if err := c.overflowCache(opts); err != nil {
+ t.Errorf("c.overflowCache(%+v): %s", opts, err)
+ }
+}
+
+func TestNeighborCacheClear(t *testing.T) {
+ config := DefaultNUDConfigurations()
+
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ // Add a dynamic entry.
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ clock.advance(typicalLatency)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...)
+ nudDisp.events = nil
+ nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ // Add a static entry.
+ neigh.addStaticEntry(entryTestAddr1, entryTestLinkAddr1)
+
+ {
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Static,
+ },
+ }
+ nudDisp.mu.Lock()
+ diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...)
+ nudDisp.events = nil
+ nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ // Clear shoud remove both dynamic and static entries.
+ neigh.clear()
+
+ // Remove events dispatched from clear() have no deterministic order so they
+ // need to be sorted beforehand.
+ wantUnsortedEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Static,
+ },
+ }
+ nudDisp.mu.Lock()
+ defer nudDisp.mu.Unlock()
+ if diff := cmp.Diff(nudDisp.events, wantUnsortedEvents, eventDiffOptsWithSort()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+// TestNeighborCacheClearThenOverflow verifies that the LRU cache eviction
+// strategy keeps count of the dynamic entry count when all entries are
+// cleared.
+func TestNeighborCacheClearThenOverflow(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ // Stay in Reachable so the cache can overflow
+ config.BaseReachableTime = infiniteDuration
+ config.MinRandomFactor = 1
+ config.MaxRandomFactor = 1
+
+ c := newTestContext(config)
+
+ // Add a dynamic entry
+ entry, ok := c.store.entry(0)
+ if !ok {
+ t.Fatalf("c.store.entry(0) not found")
+ }
+ _, _, err := c.neigh.entry(entry.Addr, entry.LocalAddr, c.linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Errorf("got c.neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ c.clock.advance(typicalLatency)
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+
+ // Clear the cache.
+ c.neigh.clear()
+ {
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ c.nudDisp.mu.Lock()
+ diff := cmp.Diff(c.nudDisp.events, wantEvents, eventDiffOpts()...)
+ c.nudDisp.events = nil
+ c.nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ opts := overflowOptions{
+ startAtEntryIndex: 0,
+ }
+ if err := c.overflowCache(opts); err != nil {
+ t.Errorf("c.overflowCache(%+v): %s", opts, err)
+ }
+}
+
+func TestNeighborCacheKeepFrequentlyUsed(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ // Stay in Reachable so the cache can overflow
+ config.BaseReachableTime = infiniteDuration
+ config.MinRandomFactor = 1
+ config.MaxRandomFactor = 1
+
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ frequentlyUsedEntry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+
+ // The following logic is very similar to overflowCache, but
+ // periodically refreshes the frequently used entry.
+
+ // Fill the neighbor cache to capacity
+ for i := 0; i < neighborCacheSize; i++ {
+ entry, ok := store.entry(i)
+ if !ok {
+ t.Fatalf("store.entry(%d) not found", i)
+ }
+ _, doneCh, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ clock.advance(typicalLatency)
+ select {
+ case <-doneCh:
+ default:
+ t.Fatalf("expected notification from done channel returned by neigh.entry(%s, %s, _, nil)", entry.Addr, entry.LocalAddr)
+ }
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...)
+ nudDisp.events = nil
+ nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ // Keep adding more entries
+ for i := neighborCacheSize; i < store.size(); i++ {
+ // Periodically refresh the frequently used entry
+ if i%(neighborCacheSize/2) == 0 {
+ _, _, err := neigh.entry(frequentlyUsedEntry.Addr, frequentlyUsedEntry.LocalAddr, linkRes, nil)
+ if err != nil {
+ t.Errorf("unexpected error from neigh.entry(%s, %s, _, nil): %s", frequentlyUsedEntry.Addr, frequentlyUsedEntry.LocalAddr, err)
+ }
+ }
+
+ entry, ok := store.entry(i)
+ if !ok {
+ t.Fatalf("store.entry(%d) not found", i)
+ }
+ _, doneCh, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ clock.advance(typicalLatency)
+ select {
+ case <-doneCh:
+ default:
+ t.Fatalf("expected notification from done channel returned by neigh.entry(%s, %s, _, nil)", entry.Addr, entry.LocalAddr)
+ }
+
+ // An entry should have been removed, as per the LRU eviction strategy
+ removedEntry, ok := store.entry(i - neighborCacheSize + 1)
+ if !ok {
+ t.Fatalf("store.entry(%d) not found", i-neighborCacheSize+1)
+ }
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestRemoved,
+ NICID: 1,
+ Addr: removedEntry.Addr,
+ LinkAddr: removedEntry.LinkAddr,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestAdded,
+ NICID: 1,
+ Addr: entry.Addr,
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: 1,
+ Addr: entry.Addr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...)
+ nudDisp.events = nil
+ nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ // Expect to find only the frequently used entry and the most recent entries.
+ // The order of entries reported by entries() is undeterministic, so entries
+ // have to be sorted before comparison.
+ wantUnsortedEntries := []NeighborEntry{
+ {
+ Addr: frequentlyUsedEntry.Addr,
+ LocalAddr: frequentlyUsedEntry.LocalAddr,
+ LinkAddr: frequentlyUsedEntry.LinkAddr,
+ State: Reachable,
+ },
+ }
+
+ for i := store.size() - neighborCacheSize + 1; i < store.size(); i++ {
+ entry, ok := store.entry(i)
+ if !ok {
+ t.Fatalf("store.entry(%d) not found", i)
+ }
+ wantEntry := NeighborEntry{
+ Addr: entry.Addr,
+ LocalAddr: entry.LocalAddr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ }
+ wantUnsortedEntries = append(wantUnsortedEntries, wantEntry)
+ }
+
+ if diff := cmp.Diff(neigh.entries(), wantUnsortedEntries, entryDiffOptsWithSort()...); diff != "" {
+ t.Errorf("neighbor entries mismatch (-got, +want):\n%s", diff)
+ }
+
+ // No more events should have been dispatched.
+ nudDisp.mu.Lock()
+ defer nudDisp.mu.Unlock()
+ if diff := cmp.Diff(nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+func TestNeighborCacheConcurrent(t *testing.T) {
+ const concurrentProcesses = 16
+
+ config := DefaultNUDConfigurations()
+
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ storeEntries := store.entries()
+ for _, entry := range storeEntries {
+ var wg sync.WaitGroup
+ for r := 0; r < concurrentProcesses; r++ {
+ wg.Add(1)
+ go func(entry NeighborEntry) {
+ defer wg.Done()
+ e, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != nil && err != tcpip.ErrWouldBlock {
+ t.Errorf("got neigh.entry(%s, %s, _, nil) = (%+v, _, %s), want (_, _, nil) or (_, _, %s)", entry.Addr, entry.LocalAddr, e, err, tcpip.ErrWouldBlock)
+ }
+ }(entry)
+ }
+
+ // Wait for all gorountines to send a request
+ wg.Wait()
+
+ // Process all the requests for a single entry concurrently
+ clock.advance(typicalLatency)
+ }
+
+ // All goroutines add in the same order and add more values than can fit in
+ // the cache. Our eviction strategy requires that the last entries are
+ // present, up to the size of the neighbor cache, and the rest are missing.
+ // The order of entries reported by entries() is undeterministic, so entries
+ // have to be sorted before comparison.
+ var wantUnsortedEntries []NeighborEntry
+ for i := store.size() - neighborCacheSize; i < store.size(); i++ {
+ entry, ok := store.entry(i)
+ if !ok {
+ t.Errorf("store.entry(%d) not found", i)
+ }
+ wantEntry := NeighborEntry{
+ Addr: entry.Addr,
+ LocalAddr: entry.LocalAddr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ }
+ wantUnsortedEntries = append(wantUnsortedEntries, wantEntry)
+ }
+
+ if diff := cmp.Diff(neigh.entries(), wantUnsortedEntries, entryDiffOptsWithSort()...); diff != "" {
+ t.Errorf("neighbor entries mismatch (-got, +want):\n%s", diff)
+ }
+}
+
+func TestNeighborCacheReplace(t *testing.T) {
+ config := DefaultNUDConfigurations()
+
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ // Add an entry
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ _, doneCh, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ clock.advance(typicalLatency)
+ select {
+ case <-doneCh:
+ default:
+ t.Fatalf("expected notification from done channel returned by neigh.entry(%s, %s, _, nil)", entry.Addr, entry.LocalAddr)
+ }
+
+ // Verify the entry exists
+ e, doneCh, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != nil {
+ t.Errorf("unexpected error from neigh.entry(%s, %s, _, nil): %s", entry.Addr, entry.LocalAddr, err)
+ }
+ if doneCh != nil {
+ t.Errorf("unexpected done channel from neigh.entry(%s, %s, _, nil): %v", entry.Addr, entry.LocalAddr, doneCh)
+ }
+ if t.Failed() {
+ t.FailNow()
+ }
+ want := NeighborEntry{
+ Addr: entry.Addr,
+ LocalAddr: entry.LocalAddr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ }
+ if diff := cmp.Diff(e, want, entryDiffOpts()...); diff != "" {
+ t.Errorf("neigh.entry(%s, %s, _, nil) mismatch (-got, +want):\n%s", entry.Addr, entry.LinkAddr, diff)
+ }
+
+ // Notify of a link address change
+ var updatedLinkAddr tcpip.LinkAddress
+ {
+ entry, ok := store.entry(1)
+ if !ok {
+ t.Fatalf("store.entry(1) not found")
+ }
+ updatedLinkAddr = entry.LinkAddr
+ }
+ store.set(0, updatedLinkAddr)
+ neigh.HandleConfirmation(entry.Addr, updatedLinkAddr, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: true,
+ IsRouter: false,
+ })
+
+ // Requesting the entry again should start address resolution
+ {
+ _, doneCh, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ clock.advance(config.DelayFirstProbeTime + typicalLatency)
+ select {
+ case <-doneCh:
+ default:
+ t.Fatalf("expected notification from done channel returned by neigh.entry(%s, %s, _, nil)", entry.Addr, entry.LocalAddr)
+ }
+ }
+
+ // Verify the entry's new link address
+ {
+ e, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ clock.advance(typicalLatency)
+ if err != nil {
+ t.Errorf("unexpected error from neigh.entry(%s, %s, _, nil): %s", entry.Addr, entry.LocalAddr, err)
+ }
+ want = NeighborEntry{
+ Addr: entry.Addr,
+ LocalAddr: entry.LocalAddr,
+ LinkAddr: updatedLinkAddr,
+ State: Reachable,
+ }
+ if diff := cmp.Diff(e, want, entryDiffOpts()...); diff != "" {
+ t.Errorf("neigh.entry(%s, %s, _, nil) mismatch (-got, +want):\n%s", entry.Addr, entry.LocalAddr, diff)
+ }
+ }
+}
+
+func TestNeighborCacheResolutionFailed(t *testing.T) {
+ config := DefaultNUDConfigurations()
+
+ nudDisp := testNUDDispatcher{}
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(&nudDisp, config, clock)
+ store := newTestEntryStore()
+
+ var requestCount uint32
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ onLinkAddressRequest: func() {
+ atomic.AddUint32(&requestCount, 1)
+ },
+ }
+
+ // First, sanity check that resolution is working
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrWouldBlock {
+ t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ clock.advance(typicalLatency)
+ got, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != nil {
+ t.Fatalf("unexpected error from neigh.entry(%s, %s, _, nil): %s", entry.Addr, entry.LocalAddr, err)
+ }
+ want := NeighborEntry{
+ Addr: entry.Addr,
+ LocalAddr: entry.LocalAddr,
+ LinkAddr: entry.LinkAddr,
+ State: Reachable,
+ }
+ if diff := cmp.Diff(got, want, entryDiffOpts()...); diff != "" {
+ t.Errorf("neigh.entry(%s, %s, _, nil) mismatch (-got, +want):\n%s", entry.Addr, entry.LocalAddr, diff)
+ }
+
+ // Verify that address resolution for an unknown address returns ErrNoLinkAddress
+ before := atomic.LoadUint32(&requestCount)
+
+ entry.Addr += "2"
+ if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrWouldBlock {
+ t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ waitFor := config.DelayFirstProbeTime + typicalLatency*time.Duration(config.MaxMulticastProbes)
+ clock.advance(waitFor)
+ if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrNoLinkAddress {
+ t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrNoLinkAddress)
+ }
+
+ maxAttempts := neigh.config().MaxUnicastProbes
+ if got, want := atomic.LoadUint32(&requestCount)-before, maxAttempts; got != want {
+ t.Errorf("got link address request count = %d, want = %d", got, want)
+ }
+}
+
+// TestNeighborCacheResolutionTimeout simulates sending MaxMulticastProbes
+// probes and not retrieving a confirmation before the duration defined by
+// MaxMulticastProbes * RetransmitTimer.
+func TestNeighborCacheResolutionTimeout(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ config.RetransmitTimer = time.Millisecond // small enough to cause timeout
+
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(nil, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: time.Minute, // large enough to cause timeout
+ }
+
+ entry, ok := store.entry(0)
+ if !ok {
+ t.Fatalf("store.entry(0) not found")
+ }
+ if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrWouldBlock {
+ t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ waitFor := config.RetransmitTimer * time.Duration(config.MaxMulticastProbes)
+ clock.advance(waitFor)
+ if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrNoLinkAddress {
+ t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrNoLinkAddress)
+ }
+}
+
+// TestNeighborCacheStaticResolution checks that static link addresses are
+// resolved immediately and don't send resolution requests.
+func TestNeighborCacheStaticResolution(t *testing.T) {
+ config := DefaultNUDConfigurations()
+ clock := newFakeClock()
+ neigh := newTestNeighborCache(nil, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: typicalLatency,
+ }
+
+ got, _, err := neigh.entry(testEntryBroadcastAddr, testEntryLocalAddr, linkRes, nil)
+ if err != nil {
+ t.Fatalf("unexpected error from neigh.entry(%s, %s, _, nil): %s", testEntryBroadcastAddr, testEntryLocalAddr, err)
+ }
+ want := NeighborEntry{
+ Addr: testEntryBroadcastAddr,
+ LocalAddr: testEntryLocalAddr,
+ LinkAddr: testEntryBroadcastLinkAddr,
+ State: Static,
+ }
+ if diff := cmp.Diff(got, want, entryDiffOpts()...); diff != "" {
+ t.Errorf("neigh.entry(%s, %s, _, nil) mismatch (-got, +want):\n%s", testEntryBroadcastAddr, testEntryLocalAddr, diff)
+ }
+}
+
+func BenchmarkCacheClear(b *testing.B) {
+ b.StopTimer()
+ config := DefaultNUDConfigurations()
+ clock := &tcpip.StdClock{}
+ neigh := newTestNeighborCache(nil, config, clock)
+ store := newTestEntryStore()
+ linkRes := &testNeighborResolver{
+ clock: clock,
+ neigh: neigh,
+ entries: store,
+ delay: 0,
+ }
+
+ // Clear for every possible size of the cache
+ for cacheSize := 0; cacheSize < neighborCacheSize; cacheSize++ {
+ // Fill the neighbor cache to capacity.
+ for i := 0; i < cacheSize; i++ {
+ entry, ok := store.entry(i)
+ if !ok {
+ b.Fatalf("store.entry(%d) not found", i)
+ }
+ _, doneCh, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil)
+ if err != tcpip.ErrWouldBlock {
+ b.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock)
+ }
+ if doneCh != nil {
+ <-doneCh
+ }
+ }
+
+ b.StartTimer()
+ neigh.clear()
+ b.StopTimer()
+ }
+}
diff --git a/pkg/tcpip/stack/neighbor_entry.go b/pkg/tcpip/stack/neighbor_entry.go
new file mode 100644
index 000000000..0068cacb8
--- /dev/null
+++ b/pkg/tcpip/stack/neighbor_entry.go
@@ -0,0 +1,482 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package stack
+
+import (
+ "fmt"
+ "sync"
+ "time"
+
+ "gvisor.dev/gvisor/pkg/sleep"
+ "gvisor.dev/gvisor/pkg/tcpip"
+)
+
+// NeighborEntry describes a neighboring device in the local network.
+type NeighborEntry struct {
+ Addr tcpip.Address
+ LocalAddr tcpip.Address
+ LinkAddr tcpip.LinkAddress
+ State NeighborState
+ UpdatedAt time.Time
+}
+
+// NeighborState defines the state of a NeighborEntry within the Neighbor
+// Unreachability Detection state machine, as per RFC 4861 section 7.3.2.
+type NeighborState uint8
+
+const (
+ // Unknown means reachability has not been verified yet. This is the initial
+ // state of entries that have been created automatically by the Neighbor
+ // Unreachability Detection state machine.
+ Unknown NeighborState = iota
+ // Incomplete means that there is an outstanding request to resolve the
+ // address.
+ Incomplete
+ // Reachable means the path to the neighbor is functioning properly for both
+ // receive and transmit paths.
+ Reachable
+ // Stale means reachability to the neighbor is unknown, but packets are still
+ // able to be transmitted to the possibly stale link address.
+ Stale
+ // Delay means reachability to the neighbor is unknown and pending
+ // confirmation from an upper-level protocol like TCP, but packets are still
+ // able to be transmitted to the possibly stale link address.
+ Delay
+ // Probe means a reachability confirmation is actively being sought by
+ // periodically retransmitting reachability probes until a reachability
+ // confirmation is received, or until the max amount of probes has been sent.
+ Probe
+ // Static describes entries that have been explicitly added by the user. They
+ // do not expire and are not deleted until explicitly removed.
+ Static
+ // Failed means traffic should not be sent to this neighbor since attempts of
+ // reachability have returned inconclusive.
+ Failed
+)
+
+// neighborEntry implements a neighbor entry's individual node behavior, as per
+// RFC 4861 section 7.3.3. Neighbor Unreachability Detection operates in
+// parallel with the sending of packets to a neighbor, necessitating the
+// entry's lock to be acquired for all operations.
+type neighborEntry struct {
+ neighborEntryEntry
+
+ nic *NIC
+ protocol tcpip.NetworkProtocolNumber
+
+ // linkRes provides the functionality to send reachability probes, used in
+ // Neighbor Unreachability Detection.
+ linkRes LinkAddressResolver
+
+ // nudState points to the Neighbor Unreachability Detection configuration.
+ nudState *NUDState
+
+ // mu protects the fields below.
+ mu sync.RWMutex
+
+ neigh NeighborEntry
+
+ // wakers is a set of waiters for address resolution result. Anytime state
+ // transitions out of incomplete these waiters are notified. It is nil iff
+ // address resolution is ongoing and no clients are waiting for the result.
+ wakers map[*sleep.Waker]struct{}
+
+ // done is used to allow callers to wait on address resolution. It is nil
+ // iff nudState is not Reachable and address resolution is not yet in
+ // progress.
+ done chan struct{}
+
+ isRouter bool
+ job *tcpip.Job
+}
+
+// newNeighborEntry creates a neighbor cache entry starting at the default
+// state, Unknown. Transition out of Unknown by calling either
+// `handlePacketQueuedLocked` or `handleProbeLocked` on the newly created
+// neighborEntry.
+func newNeighborEntry(nic *NIC, remoteAddr tcpip.Address, localAddr tcpip.Address, nudState *NUDState, linkRes LinkAddressResolver) *neighborEntry {
+ return &neighborEntry{
+ nic: nic,
+ linkRes: linkRes,
+ nudState: nudState,
+ neigh: NeighborEntry{
+ Addr: remoteAddr,
+ LocalAddr: localAddr,
+ State: Unknown,
+ },
+ }
+}
+
+// newStaticNeighborEntry creates a neighbor cache entry starting at the Static
+// state. The entry can only transition out of Static by directly calling
+// `setStateLocked`.
+func newStaticNeighborEntry(nic *NIC, addr tcpip.Address, linkAddr tcpip.LinkAddress, state *NUDState) *neighborEntry {
+ if nic.stack.nudDisp != nil {
+ nic.stack.nudDisp.OnNeighborAdded(nic.id, addr, linkAddr, Static, time.Now())
+ }
+ return &neighborEntry{
+ nic: nic,
+ nudState: state,
+ neigh: NeighborEntry{
+ Addr: addr,
+ LinkAddr: linkAddr,
+ State: Static,
+ UpdatedAt: time.Now(),
+ },
+ }
+}
+
+// addWaker adds w to the list of wakers waiting for address resolution.
+// Assumes the entry has already been appropriately locked.
+func (e *neighborEntry) addWakerLocked(w *sleep.Waker) {
+ if w == nil {
+ return
+ }
+ if e.wakers == nil {
+ e.wakers = make(map[*sleep.Waker]struct{})
+ }
+ e.wakers[w] = struct{}{}
+}
+
+// notifyWakersLocked notifies those waiting for address resolution, whether it
+// succeeded or failed. Assumes the entry has already been appropriately locked.
+func (e *neighborEntry) notifyWakersLocked() {
+ for w := range e.wakers {
+ w.Assert()
+ }
+ e.wakers = nil
+ if ch := e.done; ch != nil {
+ close(ch)
+ e.done = nil
+ }
+}
+
+// dispatchAddEventLocked signals to stack's NUD Dispatcher that the entry has
+// been added.
+func (e *neighborEntry) dispatchAddEventLocked(nextState NeighborState) {
+ if nudDisp := e.nic.stack.nudDisp; nudDisp != nil {
+ nudDisp.OnNeighborAdded(e.nic.id, e.neigh.Addr, e.neigh.LinkAddr, nextState, time.Now())
+ }
+}
+
+// dispatchChangeEventLocked signals to stack's NUD Dispatcher that the entry
+// has changed state or link-layer address.
+func (e *neighborEntry) dispatchChangeEventLocked(nextState NeighborState) {
+ if nudDisp := e.nic.stack.nudDisp; nudDisp != nil {
+ nudDisp.OnNeighborChanged(e.nic.id, e.neigh.Addr, e.neigh.LinkAddr, nextState, time.Now())
+ }
+}
+
+// dispatchRemoveEventLocked signals to stack's NUD Dispatcher that the entry
+// has been removed.
+func (e *neighborEntry) dispatchRemoveEventLocked() {
+ if nudDisp := e.nic.stack.nudDisp; nudDisp != nil {
+ nudDisp.OnNeighborRemoved(e.nic.id, e.neigh.Addr, e.neigh.LinkAddr, e.neigh.State, time.Now())
+ }
+}
+
+// setStateLocked transitions the entry to the specified state immediately.
+//
+// Follows the logic defined in RFC 4861 section 7.3.3.
+//
+// e.mu MUST be locked.
+func (e *neighborEntry) setStateLocked(next NeighborState) {
+ // Cancel the previously scheduled action, if there is one. Entries in
+ // Unknown, Stale, or Static state do not have scheduled actions.
+ if timer := e.job; timer != nil {
+ timer.Cancel()
+ }
+
+ prev := e.neigh.State
+ e.neigh.State = next
+ e.neigh.UpdatedAt = time.Now()
+ config := e.nudState.Config()
+
+ switch next {
+ case Incomplete:
+ var retryCounter uint32
+ var sendMulticastProbe func()
+
+ sendMulticastProbe = func() {
+ if retryCounter == config.MaxMulticastProbes {
+ // "If no Neighbor Advertisement is received after
+ // MAX_MULTICAST_SOLICIT solicitations, address resolution has failed.
+ // The sender MUST return ICMP destination unreachable indications with
+ // code 3 (Address Unreachable) for each packet queued awaiting address
+ // resolution." - RFC 4861 section 7.2.2
+ //
+ // There is no need to send an ICMP destination unreachable indication
+ // since the failure to resolve the address is expected to only occur
+ // on this node. Thus, redirecting traffic is currently not supported.
+ //
+ // "If the error occurs on a node other than the node originating the
+ // packet, an ICMP error message is generated. If the error occurs on
+ // the originating node, an implementation is not required to actually
+ // create and send an ICMP error packet to the source, as long as the
+ // upper-layer sender is notified through an appropriate mechanism
+ // (e.g. return value from a procedure call). Note, however, that an
+ // implementation may find it convenient in some cases to return errors
+ // to the sender by taking the offending packet, generating an ICMP
+ // error message, and then delivering it (locally) through the generic
+ // error-handling routines.' - RFC 4861 section 2.1
+ e.dispatchRemoveEventLocked()
+ e.setStateLocked(Failed)
+ return
+ }
+
+ if err := e.linkRes.LinkAddressRequest(e.neigh.Addr, e.neigh.LocalAddr, "", e.nic.linkEP); err != nil {
+ // There is no need to log the error here; the NUD implementation may
+ // assume a working link. A valid link should be the responsibility of
+ // the NIC/stack.LinkEndpoint.
+ e.dispatchRemoveEventLocked()
+ e.setStateLocked(Failed)
+ return
+ }
+
+ retryCounter++
+ e.job = e.nic.stack.newJob(&e.mu, sendMulticastProbe)
+ e.job.Schedule(config.RetransmitTimer)
+ }
+
+ sendMulticastProbe()
+
+ case Reachable:
+ e.job = e.nic.stack.newJob(&e.mu, func() {
+ e.dispatchChangeEventLocked(Stale)
+ e.setStateLocked(Stale)
+ })
+ e.job.Schedule(e.nudState.ReachableTime())
+
+ case Delay:
+ e.job = e.nic.stack.newJob(&e.mu, func() {
+ e.dispatchChangeEventLocked(Probe)
+ e.setStateLocked(Probe)
+ })
+ e.job.Schedule(config.DelayFirstProbeTime)
+
+ case Probe:
+ var retryCounter uint32
+ var sendUnicastProbe func()
+
+ sendUnicastProbe = func() {
+ if retryCounter == config.MaxUnicastProbes {
+ e.dispatchRemoveEventLocked()
+ e.setStateLocked(Failed)
+ return
+ }
+
+ if err := e.linkRes.LinkAddressRequest(e.neigh.Addr, e.neigh.LocalAddr, e.neigh.LinkAddr, e.nic.linkEP); err != nil {
+ e.dispatchRemoveEventLocked()
+ e.setStateLocked(Failed)
+ return
+ }
+
+ retryCounter++
+ if retryCounter == config.MaxUnicastProbes {
+ e.dispatchRemoveEventLocked()
+ e.setStateLocked(Failed)
+ return
+ }
+
+ e.job = e.nic.stack.newJob(&e.mu, sendUnicastProbe)
+ e.job.Schedule(config.RetransmitTimer)
+ }
+
+ sendUnicastProbe()
+
+ case Failed:
+ e.notifyWakersLocked()
+ e.job = e.nic.stack.newJob(&e.mu, func() {
+ e.nic.neigh.removeEntryLocked(e)
+ })
+ e.job.Schedule(config.UnreachableTime)
+
+ case Unknown, Stale, Static:
+ // Do nothing
+
+ default:
+ panic(fmt.Sprintf("Invalid state transition from %q to %q", prev, next))
+ }
+}
+
+// handlePacketQueuedLocked advances the state machine according to a packet
+// being queued for outgoing transmission.
+//
+// Follows the logic defined in RFC 4861 section 7.3.3.
+func (e *neighborEntry) handlePacketQueuedLocked() {
+ switch e.neigh.State {
+ case Unknown:
+ e.dispatchAddEventLocked(Incomplete)
+ e.setStateLocked(Incomplete)
+
+ case Stale:
+ e.dispatchChangeEventLocked(Delay)
+ e.setStateLocked(Delay)
+
+ case Incomplete, Reachable, Delay, Probe, Static, Failed:
+ // Do nothing
+
+ default:
+ panic(fmt.Sprintf("Invalid cache entry state: %s", e.neigh.State))
+ }
+}
+
+// handleProbeLocked processes an incoming neighbor probe (e.g. ARP request or
+// Neighbor Solicitation for ARP or NDP, respectively).
+//
+// Follows the logic defined in RFC 4861 section 7.2.3.
+func (e *neighborEntry) handleProbeLocked(remoteLinkAddr tcpip.LinkAddress) {
+ // Probes MUST be silently discarded if the target address is tentative, does
+ // not exist, or not bound to the NIC as per RFC 4861 section 7.2.3. These
+ // checks MUST be done by the NetworkEndpoint.
+
+ switch e.neigh.State {
+ case Unknown, Incomplete, Failed:
+ e.neigh.LinkAddr = remoteLinkAddr
+ e.dispatchAddEventLocked(Stale)
+ e.setStateLocked(Stale)
+ e.notifyWakersLocked()
+
+ case Reachable, Delay, Probe:
+ if e.neigh.LinkAddr != remoteLinkAddr {
+ e.neigh.LinkAddr = remoteLinkAddr
+ e.dispatchChangeEventLocked(Stale)
+ e.setStateLocked(Stale)
+ }
+
+ case Stale:
+ if e.neigh.LinkAddr != remoteLinkAddr {
+ e.neigh.LinkAddr = remoteLinkAddr
+ e.dispatchChangeEventLocked(Stale)
+ }
+
+ case Static:
+ // Do nothing
+
+ default:
+ panic(fmt.Sprintf("Invalid cache entry state: %s", e.neigh.State))
+ }
+}
+
+// handleConfirmationLocked processes an incoming neighbor confirmation
+// (e.g. ARP reply or Neighbor Advertisement for ARP or NDP, respectively).
+//
+// Follows the state machine defined by RFC 4861 section 7.2.5.
+//
+// TODO(gvisor.dev/issue/2277): To protect against ARP poisoning and other
+// attacks against NDP functions, Secure Neighbor Discovery (SEND) Protocol
+// should be deployed where preventing access to the broadcast segment might
+// not be possible. SEND uses RSA key pairs to produce Cryptographically
+// Generated Addresses (CGA), as defined in RFC 3972. This ensures that the
+// claimed source of an NDP message is the owner of the claimed address.
+func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, flags ReachabilityConfirmationFlags) {
+ switch e.neigh.State {
+ case Incomplete:
+ if len(linkAddr) == 0 {
+ // "If the link layer has addresses and no Target Link-Layer Address
+ // option is included, the receiving node SHOULD silently discard the
+ // received advertisement." - RFC 4861 section 7.2.5
+ break
+ }
+
+ e.neigh.LinkAddr = linkAddr
+ if flags.Solicited {
+ e.dispatchChangeEventLocked(Reachable)
+ e.setStateLocked(Reachable)
+ } else {
+ e.dispatchChangeEventLocked(Stale)
+ e.setStateLocked(Stale)
+ }
+ e.isRouter = flags.IsRouter
+ e.notifyWakersLocked()
+
+ // "Note that the Override flag is ignored if the entry is in the
+ // INCOMPLETE state." - RFC 4861 section 7.2.5
+
+ case Reachable, Stale, Delay, Probe:
+ sameLinkAddr := e.neigh.LinkAddr == linkAddr
+
+ if !sameLinkAddr {
+ if !flags.Override {
+ if e.neigh.State == Reachable {
+ e.dispatchChangeEventLocked(Stale)
+ e.setStateLocked(Stale)
+ }
+ break
+ }
+
+ e.neigh.LinkAddr = linkAddr
+
+ if !flags.Solicited {
+ if e.neigh.State != Stale {
+ e.dispatchChangeEventLocked(Stale)
+ e.setStateLocked(Stale)
+ } else {
+ // Notify the LinkAddr change, even though NUD state hasn't changed.
+ e.dispatchChangeEventLocked(e.neigh.State)
+ }
+ break
+ }
+ }
+
+ if flags.Solicited && (flags.Override || sameLinkAddr) {
+ if e.neigh.State != Reachable {
+ e.dispatchChangeEventLocked(Reachable)
+ }
+ // Set state to Reachable again to refresh timers.
+ e.setStateLocked(Reachable)
+ e.notifyWakersLocked()
+ }
+
+ if e.isRouter && !flags.IsRouter {
+ // "In those cases where the IsRouter flag changes from TRUE to FALSE as
+ // a result of this update, the node MUST remove that router from the
+ // Default Router List and update the Destination Cache entries for all
+ // destinations using that neighbor as a router as specified in Section
+ // 7.3.3. This is needed to detect when a node that is used as a router
+ // stops forwarding packets due to being configured as a host."
+ // - RFC 4861 section 7.2.5
+ e.nic.mu.Lock()
+ e.nic.mu.ndp.invalidateDefaultRouter(e.neigh.Addr)
+ e.nic.mu.Unlock()
+ }
+ e.isRouter = flags.IsRouter
+
+ case Unknown, Failed, Static:
+ // Do nothing
+
+ default:
+ panic(fmt.Sprintf("Invalid cache entry state: %s", e.neigh.State))
+ }
+}
+
+// handleUpperLevelConfirmationLocked processes an incoming upper-level protocol
+// (e.g. TCP acknowledgements) reachability confirmation.
+func (e *neighborEntry) handleUpperLevelConfirmationLocked() {
+ switch e.neigh.State {
+ case Reachable, Stale, Delay, Probe:
+ if e.neigh.State != Reachable {
+ e.dispatchChangeEventLocked(Reachable)
+ // Set state to Reachable again to refresh timers.
+ }
+ e.setStateLocked(Reachable)
+
+ case Unknown, Incomplete, Failed, Static:
+ // Do nothing
+
+ default:
+ panic(fmt.Sprintf("Invalid cache entry state: %s", e.neigh.State))
+ }
+}
diff --git a/pkg/tcpip/stack/neighbor_entry_test.go b/pkg/tcpip/stack/neighbor_entry_test.go
new file mode 100644
index 000000000..08c9ccd25
--- /dev/null
+++ b/pkg/tcpip/stack/neighbor_entry_test.go
@@ -0,0 +1,2770 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package stack
+
+import (
+ "fmt"
+ "math"
+ "math/rand"
+ "strings"
+ "sync"
+ "testing"
+ "time"
+
+ "github.com/google/go-cmp/cmp"
+ "github.com/google/go-cmp/cmp/cmpopts"
+ "gvisor.dev/gvisor/pkg/sleep"
+ "gvisor.dev/gvisor/pkg/tcpip"
+)
+
+const (
+ entryTestNetNumber tcpip.NetworkProtocolNumber = math.MaxUint32
+
+ entryTestNICID tcpip.NICID = 1
+ entryTestAddr1 = tcpip.Address("\x00\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01")
+ entryTestAddr2 = tcpip.Address("\x00\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02")
+
+ entryTestLinkAddr1 = tcpip.LinkAddress("\x0a\x00\x00\x00\x00\x01")
+ entryTestLinkAddr2 = tcpip.LinkAddress("\x0a\x00\x00\x00\x00\x02")
+
+ // entryTestNetDefaultMTU is the MTU, in bytes, used throughout the tests,
+ // except where another value is explicitly used. It is chosen to match the
+ // MTU of loopback interfaces on Linux systems.
+ entryTestNetDefaultMTU = 65536
+)
+
+// eventDiffOpts are the options passed to cmp.Diff to compare entry events.
+// The UpdatedAt field is ignored due to a lack of a deterministic method to
+// predict the time that an event will be dispatched.
+func eventDiffOpts() []cmp.Option {
+ return []cmp.Option{
+ cmpopts.IgnoreFields(testEntryEventInfo{}, "UpdatedAt"),
+ }
+}
+
+// eventDiffOptsWithSort is like eventDiffOpts but also includes an option to
+// sort slices of events for cases where ordering must be ignored.
+func eventDiffOptsWithSort() []cmp.Option {
+ return []cmp.Option{
+ cmpopts.IgnoreFields(testEntryEventInfo{}, "UpdatedAt"),
+ cmpopts.SortSlices(func(a, b testEntryEventInfo) bool {
+ return strings.Compare(string(a.Addr), string(b.Addr)) < 0
+ }),
+ }
+}
+
+// The following unit tests exercise every state transition and verify its
+// behavior with RFC 4681.
+//
+// | From | To | Cause | Action | Event |
+// | ========== | ========== | ========================================== | =============== | ======= |
+// | Unknown | Unknown | Confirmation w/ unknown address | | Added |
+// | Unknown | Incomplete | Packet queued to unknown address | Send probe | Added |
+// | Unknown | Stale | Probe w/ unknown address | | Added |
+// | Incomplete | Incomplete | Retransmit timer expired | Send probe | Changed |
+// | Incomplete | Reachable | Solicited confirmation | Notify wakers | Changed |
+// | Incomplete | Stale | Unsolicited confirmation | Notify wakers | Changed |
+// | Incomplete | Failed | Max probes sent without reply | Notify wakers | Removed |
+// | Reachable | Reachable | Confirmation w/ different isRouter flag | Update IsRouter | |
+// | Reachable | Stale | Reachable timer expired | | Changed |
+// | Reachable | Stale | Probe or confirmation w/ different address | | Changed |
+// | Stale | Reachable | Solicited override confirmation | Update LinkAddr | Changed |
+// | Stale | Stale | Override confirmation | Update LinkAddr | Changed |
+// | Stale | Stale | Probe w/ different address | Update LinkAddr | Changed |
+// | Stale | Delay | Packet sent | | Changed |
+// | Delay | Reachable | Upper-layer confirmation | | Changed |
+// | Delay | Reachable | Solicited override confirmation | Update LinkAddr | Changed |
+// | Delay | Stale | Probe or confirmation w/ different address | | Changed |
+// | Delay | Probe | Delay timer expired | Send probe | Changed |
+// | Probe | Reachable | Solicited override confirmation | Update LinkAddr | Changed |
+// | Probe | Reachable | Solicited confirmation w/ same address | Notify wakers | Changed |
+// | Probe | Stale | Probe or confirmation w/ different address | | Changed |
+// | Probe | Probe | Retransmit timer expired | Send probe | Changed |
+// | Probe | Failed | Max probes sent without reply | Notify wakers | Removed |
+// | Failed | | Unreachability timer expired | Delete entry | |
+
+type testEntryEventType uint8
+
+const (
+ entryTestAdded testEntryEventType = iota
+ entryTestChanged
+ entryTestRemoved
+)
+
+func (t testEntryEventType) String() string {
+ switch t {
+ case entryTestAdded:
+ return "add"
+ case entryTestChanged:
+ return "change"
+ case entryTestRemoved:
+ return "remove"
+ default:
+ return fmt.Sprintf("unknown (%d)", t)
+ }
+}
+
+// Fields are exported for use with cmp.Diff.
+type testEntryEventInfo struct {
+ EventType testEntryEventType
+ NICID tcpip.NICID
+ Addr tcpip.Address
+ LinkAddr tcpip.LinkAddress
+ State NeighborState
+ UpdatedAt time.Time
+}
+
+func (e testEntryEventInfo) String() string {
+ return fmt.Sprintf("%s event for NIC #%d, addr=%q, linkAddr=%q, state=%q", e.EventType, e.NICID, e.Addr, e.LinkAddr, e.State)
+}
+
+// testNUDDispatcher implements NUDDispatcher to validate the dispatching of
+// events upon certain NUD state machine events.
+type testNUDDispatcher struct {
+ mu sync.Mutex
+ events []testEntryEventInfo
+}
+
+var _ NUDDispatcher = (*testNUDDispatcher)(nil)
+
+func (d *testNUDDispatcher) queueEvent(e testEntryEventInfo) {
+ d.mu.Lock()
+ defer d.mu.Unlock()
+ d.events = append(d.events, e)
+}
+
+func (d *testNUDDispatcher) OnNeighborAdded(nicID tcpip.NICID, addr tcpip.Address, linkAddr tcpip.LinkAddress, state NeighborState, updatedAt time.Time) {
+ d.queueEvent(testEntryEventInfo{
+ EventType: entryTestAdded,
+ NICID: nicID,
+ Addr: addr,
+ LinkAddr: linkAddr,
+ State: state,
+ UpdatedAt: updatedAt,
+ })
+}
+
+func (d *testNUDDispatcher) OnNeighborChanged(nicID tcpip.NICID, addr tcpip.Address, linkAddr tcpip.LinkAddress, state NeighborState, updatedAt time.Time) {
+ d.queueEvent(testEntryEventInfo{
+ EventType: entryTestChanged,
+ NICID: nicID,
+ Addr: addr,
+ LinkAddr: linkAddr,
+ State: state,
+ UpdatedAt: updatedAt,
+ })
+}
+
+func (d *testNUDDispatcher) OnNeighborRemoved(nicID tcpip.NICID, addr tcpip.Address, linkAddr tcpip.LinkAddress, state NeighborState, updatedAt time.Time) {
+ d.queueEvent(testEntryEventInfo{
+ EventType: entryTestRemoved,
+ NICID: nicID,
+ Addr: addr,
+ LinkAddr: linkAddr,
+ State: state,
+ UpdatedAt: updatedAt,
+ })
+}
+
+type entryTestLinkResolver struct {
+ mu sync.Mutex
+ probes []entryTestProbeInfo
+}
+
+var _ LinkAddressResolver = (*entryTestLinkResolver)(nil)
+
+type entryTestProbeInfo struct {
+ RemoteAddress tcpip.Address
+ RemoteLinkAddress tcpip.LinkAddress
+ LocalAddress tcpip.Address
+}
+
+func (p entryTestProbeInfo) String() string {
+ return fmt.Sprintf("probe with RemoteAddress=%q, RemoteLinkAddress=%q, LocalAddress=%q", p.RemoteAddress, p.RemoteLinkAddress, p.LocalAddress)
+}
+
+// LinkAddressRequest sends a request for the LinkAddress of addr. Broadcasts
+// to the local network if linkAddr is the zero value.
+func (r *entryTestLinkResolver) LinkAddressRequest(addr, localAddr tcpip.Address, linkAddr tcpip.LinkAddress, linkEP LinkEndpoint) *tcpip.Error {
+ p := entryTestProbeInfo{
+ RemoteAddress: addr,
+ RemoteLinkAddress: linkAddr,
+ LocalAddress: localAddr,
+ }
+ r.mu.Lock()
+ defer r.mu.Unlock()
+ r.probes = append(r.probes, p)
+ return nil
+}
+
+// ResolveStaticAddress attempts to resolve address without sending requests.
+// It either resolves the name immediately or returns the empty LinkAddress.
+func (r *entryTestLinkResolver) ResolveStaticAddress(addr tcpip.Address) (tcpip.LinkAddress, bool) {
+ return "", false
+}
+
+// LinkAddressProtocol returns the network protocol of the addresses this
+// resolver can resolve.
+func (r *entryTestLinkResolver) LinkAddressProtocol() tcpip.NetworkProtocolNumber {
+ return entryTestNetNumber
+}
+
+func entryTestSetup(c NUDConfigurations) (*neighborEntry, *testNUDDispatcher, *entryTestLinkResolver, *fakeClock) {
+ clock := newFakeClock()
+ disp := testNUDDispatcher{}
+ nic := NIC{
+ id: entryTestNICID,
+ linkEP: nil, // entryTestLinkResolver doesn't use a LinkEndpoint
+ stack: &Stack{
+ clock: clock,
+ nudDisp: &disp,
+ },
+ }
+
+ rng := rand.New(rand.NewSource(time.Now().UnixNano()))
+ nudState := NewNUDState(c, rng)
+ linkRes := entryTestLinkResolver{}
+ entry := newNeighborEntry(&nic, entryTestAddr1, entryTestAddr2, nudState, &linkRes)
+
+ // Stub out ndpState to verify modification of default routers.
+ nic.mu.ndp = ndpState{
+ nic: &nic,
+ defaultRouters: make(map[tcpip.Address]defaultRouterState),
+ }
+
+ // Stub out the neighbor cache to verify deletion from the cache.
+ nic.neigh = &neighborCache{
+ nic: &nic,
+ state: nudState,
+ cache: make(map[tcpip.Address]*neighborEntry, neighborCacheSize),
+ }
+ nic.neigh.cache[entryTestAddr1] = entry
+
+ return entry, &disp, &linkRes, clock
+}
+
+// TestEntryInitiallyUnknown verifies that the state of a newly created
+// neighborEntry is Unknown.
+func TestEntryInitiallyUnknown(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Unknown; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ clock.advance(c.RetransmitTimer)
+
+ // No probes should have been sent.
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, []entryTestProbeInfo(nil))
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ // No events should have been dispatched.
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryUnknownToUnknownWhenConfirmationWithUnknownAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Unknown; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ clock.advance(time.Hour)
+
+ // No probes should have been sent.
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, []entryTestProbeInfo(nil))
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ // No events should have been dispatched.
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, []testEntryEventInfo(nil)); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryUnknownToIncomplete(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Incomplete; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ }
+ {
+ nudDisp.mu.Lock()
+ diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...)
+ nudDisp.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ }
+}
+
+func TestEntryUnknownToStale(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handleProbeLocked(entryTestLinkAddr1)
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ // No probes should have been sent.
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, []entryTestProbeInfo(nil))
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryIncompleteToIncompleteDoesNotChangeUpdatedAt(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ c.MaxMulticastProbes = 3
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Incomplete; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ updatedAt := e.neigh.UpdatedAt
+ e.mu.Unlock()
+
+ clock.advance(c.RetransmitTimer)
+
+ // UpdatedAt should remain the same during address resolution.
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.probes = nil
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ e.mu.Lock()
+ if got, want := e.neigh.UpdatedAt, updatedAt; got != want {
+ t.Errorf("got e.neigh.UpdatedAt = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ clock.advance(c.RetransmitTimer)
+
+ // UpdatedAt should change after failing address resolution. Timing out after
+ // sending the last probe transitions the entry to Failed.
+ {
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+ }
+
+ clock.advance(c.RetransmitTimer)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestRemoved,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, notWant := e.neigh.UpdatedAt, updatedAt; got == notWant {
+ t.Errorf("expected e.neigh.UpdatedAt to change, got = %q", got)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryIncompleteToReachable(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Incomplete; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+// TestEntryAddsAndClearsWakers verifies that wakers are added when
+// addWakerLocked is called and cleared when address resolution finishes. In
+// this case, address resolution will finish when transitioning from Incomplete
+// to Reachable.
+func TestEntryAddsAndClearsWakers(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ w := sleep.Waker{}
+ s := sleep.Sleeper{}
+ s.AddWaker(&w, 123)
+ defer s.Done()
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ if got := e.wakers; got != nil {
+ t.Errorf("got e.wakers = %v, want = nil", got)
+ }
+ e.addWakerLocked(&w)
+ if got, want := w.IsAsserted(), false; got != want {
+ t.Errorf("waker.IsAsserted() = %t, want = %t", got, want)
+ }
+ if e.wakers == nil {
+ t.Error("expected e.wakers to be non-nil")
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ if e.wakers != nil {
+ t.Errorf("got e.wakers = %v, want = nil", e.wakers)
+ }
+ if got, want := w.IsAsserted(), true; got != want {
+ t.Errorf("waker.IsAsserted() = %t, want = %t", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryIncompleteToReachableWithRouterFlag(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Incomplete; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: true,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.isRouter, true; got != want {
+ t.Errorf("got e.isRouter = %t, want = %t", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ if diff := cmp.Diff(linkRes.probes, wantProbes); diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+ linkRes.mu.Unlock()
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryIncompleteToStale(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Incomplete; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryIncompleteToFailed(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ c.MaxMulticastProbes = 3
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Incomplete; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ waitFor := c.RetransmitTimer * time.Duration(c.MaxMulticastProbes)
+ clock.advance(waitFor)
+
+ wantProbes := []entryTestProbeInfo{
+ // The Incomplete-to-Incomplete state transition is tested here by
+ // verifying that 3 reachability probes were sent.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestRemoved,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Failed; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+type testLocker struct{}
+
+var _ sync.Locker = (*testLocker)(nil)
+
+func (*testLocker) Lock() {}
+func (*testLocker) Unlock() {}
+
+func TestEntryStaysReachableWhenConfirmationWithRouterFlag(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: true,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.isRouter, true; got != want {
+ t.Errorf("got e.isRouter = %t, want = %t", got, want)
+ }
+ e.nic.mu.ndp.defaultRouters[entryTestAddr1] = defaultRouterState{
+ invalidationJob: e.nic.stack.newJob(&testLocker{}, func() {}),
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.isRouter, false; got != want {
+ t.Errorf("got e.isRouter = %t, want = %t", got, want)
+ }
+ if _, ok := e.nic.mu.ndp.defaultRouters[entryTestAddr1]; ok {
+ t.Errorf("unexpected defaultRouter for %s", entryTestAddr1)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryStaysReachableWhenProbeWithSameAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleProbeLocked(entryTestLinkAddr1)
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr1; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryReachableToStaleWhenTimeout(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ // Eliminate random factors from ReachableTime computation so the transition
+ // from Stale to Reachable will only take BaseReachableTime duration.
+ c.MinRandomFactor = 1
+ c.MaxRandomFactor = 1
+
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ clock.advance(c.BaseReachableTime)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryReachableToStaleWhenProbeWithDifferentAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleProbeLocked(entryTestLinkAddr2)
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryReachableToStaleWhenConfirmationWithDifferentAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryReachableToStaleWhenConfirmationWithDifferentAddressAndOverride(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryStaysStaleWhenProbeWithSameAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleProbeLocked(entryTestLinkAddr1)
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr1; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryStaleToReachableWhenSolicitedOverrideConfirmation(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr2; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Reachable,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryStaleToStaleWhenOverrideConfirmation(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr2; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryStaleToStaleWhenProbeUpdateAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleProbeLocked(entryTestLinkAddr2)
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr2; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryStaleToDelay(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Delay; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryDelayToReachableWhenUpperLevelConfirmation(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ // Eliminate random factors from ReachableTime computation so the transition
+ // from Stale to Reachable will only take BaseReachableTime duration.
+ c.MinRandomFactor = 1
+ c.MaxRandomFactor = 1
+
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Delay; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleUpperLevelConfirmationLocked()
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ clock.advance(c.BaseReachableTime)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryDelayToReachableWhenSolicitedOverrideConfirmation(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ c.MaxMulticastProbes = 1
+ // Eliminate random factors from ReachableTime computation so the transition
+ // from Stale to Reachable will only take BaseReachableTime duration.
+ c.MinRandomFactor = 1
+ c.MaxRandomFactor = 1
+
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Delay; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr2; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ clock.advance(c.BaseReachableTime)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryStaysDelayWhenOverrideConfirmationWithSameAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Delay; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Delay; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr1; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryDelayToStaleWhenProbeWithDifferentAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Delay; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleProbeLocked(entryTestLinkAddr2)
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryDelayToStaleWhenConfirmationWithDifferentAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, _ := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Delay; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantProbes := []entryTestProbeInfo{
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryDelayToProbe(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ if got, want := e.neigh.State, Delay; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ clock.advance(c.DelayFirstProbeTime)
+
+ wantProbes := []entryTestProbeInfo{
+ // The first probe is caused by the Unknown-to-Incomplete transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ // The second probe is caused by the Delay-to-Probe transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Probe; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryProbeToStaleWhenProbeWithDifferentAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ e.mu.Unlock()
+
+ clock.advance(c.DelayFirstProbeTime)
+
+ wantProbes := []entryTestProbeInfo{
+ // The first probe is caused by the Unknown-to-Incomplete transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ // The second probe is caused by the Delay-to-Probe transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Probe; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleProbeLocked(entryTestLinkAddr2)
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryProbeToStaleWhenConfirmationWithDifferentAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ e.mu.Unlock()
+
+ clock.advance(c.DelayFirstProbeTime)
+
+ wantProbes := []entryTestProbeInfo{
+ // The first probe is caused by the Unknown-to-Incomplete transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ // The second probe is caused by the Delay-to-Probe transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Probe; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Stale; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryStaysProbeWhenOverrideConfirmationWithSameAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ e.mu.Unlock()
+
+ clock.advance(c.DelayFirstProbeTime)
+
+ wantProbes := []entryTestProbeInfo{
+ // The first probe is caused by the Unknown-to-Incomplete transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ // The second probe is caused by the Delay-to-Probe transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Probe; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Probe; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr1; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryProbeToReachableWhenSolicitedOverrideConfirmation(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ // Eliminate random factors from ReachableTime computation so the transition
+ // from Stale to Reachable will only take BaseReachableTime duration.
+ c.MinRandomFactor = 1
+ c.MaxRandomFactor = 1
+
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ e.mu.Unlock()
+
+ clock.advance(c.DelayFirstProbeTime)
+
+ wantProbes := []entryTestProbeInfo{
+ // The first probe is caused by the Unknown-to-Incomplete transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ // The second probe is caused by the Delay-to-Probe transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Probe; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr2, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: true,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ if got, want := e.neigh.LinkAddr, entryTestLinkAddr2; got != want {
+ t.Errorf("got e.neigh.LinkAddr = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ clock.advance(c.BaseReachableTime)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr2,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryProbeToReachableWhenSolicitedConfirmationWithSameAddress(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ // Eliminate random factors from ReachableTime computation so the transition
+ // from Stale to Reachable will only take BaseReachableTime duration.
+ c.MinRandomFactor = 1
+ c.MaxRandomFactor = 1
+
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ e.mu.Unlock()
+
+ clock.advance(c.DelayFirstProbeTime)
+
+ wantProbes := []entryTestProbeInfo{
+ // The first probe is caused by the Unknown-to-Incomplete transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ // The second probe is caused by the Delay-to-Probe transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Probe; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: true,
+ Override: false,
+ IsRouter: false,
+ })
+ if got, want := e.neigh.State, Reachable; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+
+ clock.advance(c.BaseReachableTime)
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Reachable,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+}
+
+func TestEntryProbeToFailed(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ c.MaxMulticastProbes = 3
+ c.MaxUnicastProbes = 3
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ e.mu.Unlock()
+
+ waitFor := c.DelayFirstProbeTime + c.RetransmitTimer*time.Duration(c.MaxUnicastProbes)
+ clock.advance(waitFor)
+
+ wantProbes := []entryTestProbeInfo{
+ // The first probe is caused by the Unknown-to-Incomplete transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ // The next three probe are caused by the Delay-to-Probe transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ {
+ EventType: entryTestRemoved,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ e.mu.Lock()
+ if got, want := e.neigh.State, Failed; got != want {
+ t.Errorf("got e.neigh.State = %q, want = %q", got, want)
+ }
+ e.mu.Unlock()
+}
+
+func TestEntryFailedGetsDeleted(t *testing.T) {
+ c := DefaultNUDConfigurations()
+ c.MaxMulticastProbes = 3
+ c.MaxUnicastProbes = 3
+ e, nudDisp, linkRes, clock := entryTestSetup(c)
+
+ // Verify the cache contains the entry.
+ if _, ok := e.nic.neigh.cache[entryTestAddr1]; !ok {
+ t.Errorf("expected entry %q to exist in the neighbor cache", entryTestAddr1)
+ }
+
+ e.mu.Lock()
+ e.handlePacketQueuedLocked()
+ e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{
+ Solicited: false,
+ Override: false,
+ IsRouter: false,
+ })
+ e.handlePacketQueuedLocked()
+ e.mu.Unlock()
+
+ waitFor := c.DelayFirstProbeTime + c.RetransmitTimer*time.Duration(c.MaxUnicastProbes) + c.UnreachableTime
+ clock.advance(waitFor)
+
+ wantProbes := []entryTestProbeInfo{
+ // The first probe is caused by the Unknown-to-Incomplete transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: tcpip.LinkAddress(""),
+ LocalAddress: entryTestAddr2,
+ },
+ // The next three probe are caused by the Delay-to-Probe transition.
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ {
+ RemoteAddress: entryTestAddr1,
+ RemoteLinkAddress: entryTestLinkAddr1,
+ LocalAddress: entryTestAddr2,
+ },
+ }
+ linkRes.mu.Lock()
+ diff := cmp.Diff(linkRes.probes, wantProbes)
+ linkRes.mu.Unlock()
+ if diff != "" {
+ t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff)
+ }
+
+ wantEvents := []testEntryEventInfo{
+ {
+ EventType: entryTestAdded,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: tcpip.LinkAddress(""),
+ State: Incomplete,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Stale,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Delay,
+ },
+ {
+ EventType: entryTestChanged,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ {
+ EventType: entryTestRemoved,
+ NICID: entryTestNICID,
+ Addr: entryTestAddr1,
+ LinkAddr: entryTestLinkAddr1,
+ State: Probe,
+ },
+ }
+ nudDisp.mu.Lock()
+ if diff := cmp.Diff(nudDisp.events, wantEvents, eventDiffOpts()...); diff != "" {
+ t.Errorf("nud dispatcher events mismatch (-got, +want):\n%s", diff)
+ }
+ nudDisp.mu.Unlock()
+
+ // Verify the cache no longer contains the entry.
+ if _, ok := e.nic.neigh.cache[entryTestAddr1]; ok {
+ t.Errorf("entry %q should have been deleted from the neighbor cache", entryTestAddr1)
+ }
+}
diff --git a/pkg/tcpip/stack/neighborstate_string.go b/pkg/tcpip/stack/neighborstate_string.go
new file mode 100644
index 000000000..aa7311ec6
--- /dev/null
+++ b/pkg/tcpip/stack/neighborstate_string.go
@@ -0,0 +1,44 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Code generated by "stringer -type NeighborState"; DO NOT EDIT.
+
+package stack
+
+import "strconv"
+
+func _() {
+ // An "invalid array index" compiler error signifies that the constant values have changed.
+ // Re-run the stringer command to generate them again.
+ var x [1]struct{}
+ _ = x[Unknown-0]
+ _ = x[Incomplete-1]
+ _ = x[Reachable-2]
+ _ = x[Stale-3]
+ _ = x[Delay-4]
+ _ = x[Probe-5]
+ _ = x[Static-6]
+ _ = x[Failed-7]
+}
+
+const _NeighborState_name = "UnknownIncompleteReachableStaleDelayProbeStaticFailed"
+
+var _NeighborState_index = [...]uint8{0, 7, 17, 26, 31, 36, 41, 47, 53}
+
+func (i NeighborState) String() string {
+ if i >= NeighborState(len(_NeighborState_index)-1) {
+ return "NeighborState(" + strconv.FormatInt(int64(i), 10) + ")"
+ }
+ return _NeighborState_name[_NeighborState_index[i]:_NeighborState_index[i+1]]
+}
diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go
index 9256d4d43..f21066fce 100644
--- a/pkg/tcpip/stack/nic.go
+++ b/pkg/tcpip/stack/nic.go
@@ -16,6 +16,7 @@ package stack
import (
"fmt"
+ "math/rand"
"reflect"
"sort"
"strings"
@@ -45,6 +46,7 @@ type NIC struct {
context NICContext
stats NICStats
+ neigh *neighborCache
mu struct {
sync.RWMutex
@@ -141,6 +143,16 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC
nic.mu.packetEPs[netProto.Number()] = []PacketEndpoint{}
}
+ // Check for Neighbor Unreachability Detection support.
+ if ep.Capabilities()&CapabilityResolutionRequired != 0 && len(stack.linkAddrResolvers) != 0 {
+ rng := rand.New(rand.NewSource(stack.clock.NowNanoseconds()))
+ nic.neigh = &neighborCache{
+ nic: nic,
+ state: NewNUDState(stack.nudConfigs, rng),
+ cache: make(map[tcpip.Address]*neighborEntry, neighborCacheSize),
+ }
+ }
+
nic.linkEP.Attach(nic)
return nic
@@ -1540,6 +1552,27 @@ func (n *NIC) setNDPConfigs(c NDPConfigurations) {
n.mu.Unlock()
}
+// NUDConfigs gets the NUD configurations for n.
+func (n *NIC) NUDConfigs() (NUDConfigurations, *tcpip.Error) {
+ if n.neigh == nil {
+ return NUDConfigurations{}, tcpip.ErrNotSupported
+ }
+ return n.neigh.config(), nil
+}
+
+// setNUDConfigs sets the NUD configurations for n.
+//
+// Note, if c contains invalid NUD configuration values, it will be fixed to
+// use default values for the erroneous values.
+func (n *NIC) setNUDConfigs(c NUDConfigurations) *tcpip.Error {
+ if n.neigh == nil {
+ return tcpip.ErrNotSupported
+ }
+ c.resetInvalidFields()
+ n.neigh.setConfig(c)
+ return nil
+}
+
// handleNDPRA handles an NDP Router Advertisement message that arrived on n.
func (n *NIC) handleNDPRA(ip tcpip.Address, ra header.NDPRouterAdvert) {
n.mu.Lock()
diff --git a/pkg/tcpip/stack/nud.go b/pkg/tcpip/stack/nud.go
new file mode 100644
index 000000000..f848d50ad
--- /dev/null
+++ b/pkg/tcpip/stack/nud.go
@@ -0,0 +1,466 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package stack
+
+import (
+ "math"
+ "sync"
+ "time"
+
+ "gvisor.dev/gvisor/pkg/tcpip"
+)
+
+const (
+ // defaultBaseReachableTime is the default base duration for computing the
+ // random reachable time.
+ //
+ // Reachable time is the duration for which a neighbor is considered
+ // reachable after a positive reachability confirmation is received. It is a
+ // function of a uniformly distributed random value between the minimum and
+ // maximum random factors, multiplied by the base reachable time. Using a
+ // random component eliminates the possibility that Neighbor Unreachability
+ // Detection messages will synchronize with each other.
+ //
+ // Default taken from REACHABLE_TIME of RFC 4861 section 10.
+ defaultBaseReachableTime = 30 * time.Second
+
+ // minimumBaseReachableTime is the minimum base duration for computing the
+ // random reachable time.
+ //
+ // Minimum = 1ms
+ minimumBaseReachableTime = time.Millisecond
+
+ // defaultMinRandomFactor is the default minimum value of the random factor
+ // used for computing reachable time.
+ //
+ // Default taken from MIN_RANDOM_FACTOR of RFC 4861 section 10.
+ defaultMinRandomFactor = 0.5
+
+ // defaultMaxRandomFactor is the default maximum value of the random factor
+ // used for computing reachable time.
+ //
+ // The default value depends on the value of MinRandomFactor.
+ // If MinRandomFactor is less than MAX_RANDOM_FACTOR of RFC 4861 section 10,
+ // the value from the RFC will be used; otherwise, the default is
+ // MinRandomFactor multiplied by three.
+ defaultMaxRandomFactor = 1.5
+
+ // defaultRetransmitTimer is the default amount of time to wait between
+ // sending reachability probes.
+ //
+ // Default taken from RETRANS_TIMER of RFC 4861 section 10.
+ defaultRetransmitTimer = time.Second
+
+ // minimumRetransmitTimer is the minimum amount of time to wait between
+ // sending reachability probes.
+ //
+ // Note, RFC 4861 does not impose a minimum Retransmit Timer, but we do here
+ // to make sure the messages are not sent all at once. We also come to this
+ // value because in the RetransmitTimer field of a Router Advertisement, a
+ // value of 0 means unspecified, so the smallest valid value is 1. Note, the
+ // unit of the RetransmitTimer field in the Router Advertisement is
+ // milliseconds.
+ minimumRetransmitTimer = time.Millisecond
+
+ // defaultDelayFirstProbeTime is the default duration to wait for a
+ // non-Neighbor-Discovery related protocol to reconfirm reachability after
+ // entering the DELAY state. After this time, a reachability probe will be
+ // sent and the entry will transition to the PROBE state.
+ //
+ // Default taken from DELAY_FIRST_PROBE_TIME of RFC 4861 section 10.
+ defaultDelayFirstProbeTime = 5 * time.Second
+
+ // defaultMaxMulticastProbes is the default number of reachabililty probes
+ // to send before concluding negative reachability and deleting the neighbor
+ // entry from the INCOMPLETE state.
+ //
+ // Default taken from MAX_MULTICAST_SOLICIT of RFC 4861 section 10.
+ defaultMaxMulticastProbes = 3
+
+ // defaultMaxUnicastProbes is the default number of reachability probes to
+ // send before concluding retransmission from within the PROBE state should
+ // cease and the entry SHOULD be deleted.
+ //
+ // Default taken from MAX_UNICASE_SOLICIT of RFC 4861 section 10.
+ defaultMaxUnicastProbes = 3
+
+ // defaultMaxAnycastDelayTime is the default time in which the stack SHOULD
+ // delay sending a response for a random time between 0 and this time, if the
+ // target address is an anycast address.
+ //
+ // Default taken from MAX_ANYCAST_DELAY_TIME of RFC 4861 section 10.
+ defaultMaxAnycastDelayTime = time.Second
+
+ // defaultMaxReachbilityConfirmations is the default amount of unsolicited
+ // reachability confirmation messages a node MAY send to all-node multicast
+ // address when it determines its link-layer address has changed.
+ //
+ // Default taken from MAX_NEIGHBOR_ADVERTISEMENT of RFC 4861 section 10.
+ defaultMaxReachbilityConfirmations = 3
+
+ // defaultUnreachableTime is the default duration for how long an entry will
+ // remain in the FAILED state before being removed from the neighbor cache.
+ //
+ // Note, there is no equivalent protocol constant defined in RFC 4861. It
+ // leaves the specifics of any garbage collection mechanism up to the
+ // implementation.
+ defaultUnreachableTime = 5 * time.Second
+)
+
+// NUDDispatcher is the interface integrators of netstack must implement to
+// receive and handle NUD related events.
+type NUDDispatcher interface {
+ // OnNeighborAdded will be called when a new entry is added to a NIC's (with
+ // ID nicID) neighbor table.
+ //
+ // This function is permitted to block indefinitely without interfering with
+ // the stack's operation.
+ //
+ // May be called concurrently.
+ OnNeighborAdded(nicID tcpip.NICID, ipAddr tcpip.Address, linkAddr tcpip.LinkAddress, state NeighborState, updatedAt time.Time)
+
+ // OnNeighborChanged will be called when an entry in a NIC's (with ID nicID)
+ // neighbor table changes state and/or link address.
+ //
+ // This function is permitted to block indefinitely without interfering with
+ // the stack's operation.
+ //
+ // May be called concurrently.
+ OnNeighborChanged(nicID tcpip.NICID, ipAddr tcpip.Address, linkAddr tcpip.LinkAddress, state NeighborState, updatedAt time.Time)
+
+ // OnNeighborRemoved will be called when an entry is removed from a NIC's
+ // (with ID nicID) neighbor table.
+ //
+ // This function is permitted to block indefinitely without interfering with
+ // the stack's operation.
+ //
+ // May be called concurrently.
+ OnNeighborRemoved(nicID tcpip.NICID, ipAddr tcpip.Address, linkAddr tcpip.LinkAddress, state NeighborState, updatedAt time.Time)
+}
+
+// ReachabilityConfirmationFlags describes the flags used within a reachability
+// confirmation (e.g. ARP reply or Neighbor Advertisement for ARP or NDP,
+// respectively).
+type ReachabilityConfirmationFlags struct {
+ // Solicited indicates that the advertisement was sent in response to a
+ // reachability probe.
+ Solicited bool
+
+ // Override indicates that the reachability confirmation should override an
+ // existing neighbor cache entry and update the cached link-layer address.
+ // When Override is not set the confirmation will not update a cached
+ // link-layer address, but will update an existing neighbor cache entry for
+ // which no link-layer address is known.
+ Override bool
+
+ // IsRouter indicates that the sender is a router.
+ IsRouter bool
+}
+
+// NUDHandler communicates external events to the Neighbor Unreachability
+// Detection state machine, which is implemented per-interface. This is used by
+// network endpoints to inform the Neighbor Cache of probes and confirmations.
+type NUDHandler interface {
+ // HandleProbe processes an incoming neighbor probe (e.g. ARP request or
+ // Neighbor Solicitation for ARP or NDP, respectively). Validation of the
+ // probe needs to be performed before calling this function since the
+ // Neighbor Cache doesn't have access to view the NIC's assigned addresses.
+ HandleProbe(remoteAddr, localAddr tcpip.Address, protocol tcpip.NetworkProtocolNumber, remoteLinkAddr tcpip.LinkAddress)
+
+ // HandleConfirmation processes an incoming neighbor confirmation (e.g. ARP
+ // reply or Neighbor Advertisement for ARP or NDP, respectively).
+ HandleConfirmation(addr tcpip.Address, linkAddr tcpip.LinkAddress, flags ReachabilityConfirmationFlags)
+
+ // HandleUpperLevelConfirmation processes an incoming upper-level protocol
+ // (e.g. TCP acknowledgements) reachability confirmation.
+ HandleUpperLevelConfirmation(addr tcpip.Address)
+}
+
+// NUDConfigurations is the NUD configurations for the netstack. This is used
+// by the neighbor cache to operate the NUD state machine on each device in the
+// local network.
+type NUDConfigurations struct {
+ // BaseReachableTime is the base duration for computing the random reachable
+ // time.
+ //
+ // Reachable time is the duration for which a neighbor is considered
+ // reachable after a positive reachability confirmation is received. It is a
+ // function of uniformly distributed random value between minRandomFactor and
+ // maxRandomFactor multiplied by baseReachableTime. Using a random component
+ // eliminates the possibility that Neighbor Unreachability Detection messages
+ // will synchronize with each other.
+ //
+ // After this time, a neighbor entry will transition from REACHABLE to STALE
+ // state.
+ //
+ // Must be greater than 0.
+ BaseReachableTime time.Duration
+
+ // LearnBaseReachableTime enables learning BaseReachableTime during runtime
+ // from the neighbor discovery protocol, if supported.
+ //
+ // TODO(gvisor.dev/issue/2240): Implement this NUD configuration option.
+ LearnBaseReachableTime bool
+
+ // MinRandomFactor is the minimum value of the random factor used for
+ // computing reachable time.
+ //
+ // See BaseReachbleTime for more information on computing the reachable time.
+ //
+ // Must be greater than 0.
+ MinRandomFactor float32
+
+ // MaxRandomFactor is the maximum value of the random factor used for
+ // computing reachabile time.
+ //
+ // See BaseReachbleTime for more information on computing the reachable time.
+ //
+ // Must be great than or equal to MinRandomFactor.
+ MaxRandomFactor float32
+
+ // RetransmitTimer is the duration between retransmission of reachability
+ // probes in the PROBE state.
+ RetransmitTimer time.Duration
+
+ // LearnRetransmitTimer enables learning RetransmitTimer during runtime from
+ // the neighbor discovery protocol, if supported.
+ //
+ // TODO(gvisor.dev/issue/2241): Implement this NUD configuration option.
+ LearnRetransmitTimer bool
+
+ // DelayFirstProbeTime is the duration to wait for a non-Neighbor-Discovery
+ // related protocol to reconfirm reachability after entering the DELAY state.
+ // After this time, a reachability probe will be sent and the entry will
+ // transition to the PROBE state.
+ //
+ // Must be greater than 0.
+ DelayFirstProbeTime time.Duration
+
+ // MaxMulticastProbes is the number of reachability probes to send before
+ // concluding negative reachability and deleting the neighbor entry from the
+ // INCOMPLETE state.
+ //
+ // Must be greater than 0.
+ MaxMulticastProbes uint32
+
+ // MaxUnicastProbes is the number of reachability probes to send before
+ // concluding retransmission from within the PROBE state should cease and
+ // entry SHOULD be deleted.
+ //
+ // Must be greater than 0.
+ MaxUnicastProbes uint32
+
+ // MaxAnycastDelayTime is the time in which the stack SHOULD delay sending a
+ // response for a random time between 0 and this time, if the target address
+ // is an anycast address.
+ //
+ // TODO(gvisor.dev/issue/2242): Use this option when sending solicited
+ // neighbor confirmations to anycast addresses and proxying neighbor
+ // confirmations.
+ MaxAnycastDelayTime time.Duration
+
+ // MaxReachabilityConfirmations is the number of unsolicited reachability
+ // confirmation messages a node MAY send to all-node multicast address when
+ // it determines its link-layer address has changed.
+ //
+ // TODO(gvisor.dev/issue/2246): Discuss if implementation of this NUD
+ // configuration option is necessary.
+ MaxReachabilityConfirmations uint32
+
+ // UnreachableTime describes how long an entry will remain in the FAILED
+ // state before being removed from the neighbor cache.
+ UnreachableTime time.Duration
+}
+
+// DefaultNUDConfigurations returns a NUDConfigurations populated with default
+// values defined by RFC 4861 section 10.
+func DefaultNUDConfigurations() NUDConfigurations {
+ return NUDConfigurations{
+ BaseReachableTime: defaultBaseReachableTime,
+ LearnBaseReachableTime: true,
+ MinRandomFactor: defaultMinRandomFactor,
+ MaxRandomFactor: defaultMaxRandomFactor,
+ RetransmitTimer: defaultRetransmitTimer,
+ LearnRetransmitTimer: true,
+ DelayFirstProbeTime: defaultDelayFirstProbeTime,
+ MaxMulticastProbes: defaultMaxMulticastProbes,
+ MaxUnicastProbes: defaultMaxUnicastProbes,
+ MaxAnycastDelayTime: defaultMaxAnycastDelayTime,
+ MaxReachabilityConfirmations: defaultMaxReachbilityConfirmations,
+ UnreachableTime: defaultUnreachableTime,
+ }
+}
+
+// resetInvalidFields modifies an invalid NDPConfigurations with valid values.
+// If invalid values are present in c, the corresponding default values will be
+// used instead. This is needed to check, and conditionally fix, user-specified
+// NUDConfigurations.
+func (c *NUDConfigurations) resetInvalidFields() {
+ if c.BaseReachableTime < minimumBaseReachableTime {
+ c.BaseReachableTime = defaultBaseReachableTime
+ }
+ if c.MinRandomFactor <= 0 {
+ c.MinRandomFactor = defaultMinRandomFactor
+ }
+ if c.MaxRandomFactor < c.MinRandomFactor {
+ c.MaxRandomFactor = calcMaxRandomFactor(c.MinRandomFactor)
+ }
+ if c.RetransmitTimer < minimumRetransmitTimer {
+ c.RetransmitTimer = defaultRetransmitTimer
+ }
+ if c.DelayFirstProbeTime == 0 {
+ c.DelayFirstProbeTime = defaultDelayFirstProbeTime
+ }
+ if c.MaxMulticastProbes == 0 {
+ c.MaxMulticastProbes = defaultMaxMulticastProbes
+ }
+ if c.MaxUnicastProbes == 0 {
+ c.MaxUnicastProbes = defaultMaxUnicastProbes
+ }
+ if c.UnreachableTime == 0 {
+ c.UnreachableTime = defaultUnreachableTime
+ }
+}
+
+// calcMaxRandomFactor calculates the maximum value of the random factor used
+// for computing reachable time. This function is necessary for when the
+// default specified in RFC 4861 section 10 is less than the current
+// MinRandomFactor.
+//
+// Assumes minRandomFactor is positive since validation of the minimum value
+// should come before the validation of the maximum.
+func calcMaxRandomFactor(minRandomFactor float32) float32 {
+ if minRandomFactor > defaultMaxRandomFactor {
+ return minRandomFactor * 3
+ }
+ return defaultMaxRandomFactor
+}
+
+// A Rand is a source of random numbers.
+type Rand interface {
+ // Float32 returns, as a float32, a pseudo-random number in [0.0,1.0).
+ Float32() float32
+}
+
+// NUDState stores states needed for calculating reachable time.
+type NUDState struct {
+ rng Rand
+
+ // mu protects the fields below.
+ //
+ // It is necessary for NUDState to handle its own locking since neighbor
+ // entries may access the NUD state from within the goroutine spawned by
+ // time.AfterFunc(). This goroutine may run concurrently with the main
+ // process for controlling the neighbor cache and would otherwise introduce
+ // race conditions if NUDState was not locked properly.
+ mu sync.RWMutex
+
+ config NUDConfigurations
+
+ // reachableTime is the duration to wait for a REACHABLE entry to
+ // transition into STALE after inactivity. This value is calculated with
+ // the algorithm defined in RFC 4861 section 6.3.2.
+ reachableTime time.Duration
+
+ expiration time.Time
+ prevBaseReachableTime time.Duration
+ prevMinRandomFactor float32
+ prevMaxRandomFactor float32
+}
+
+// NewNUDState returns new NUDState using c as configuration and the specified
+// random number generator for use in recomputing ReachableTime.
+func NewNUDState(c NUDConfigurations, rng Rand) *NUDState {
+ s := &NUDState{
+ rng: rng,
+ }
+ s.config = c
+ return s
+}
+
+// Config returns the NUD configuration.
+func (s *NUDState) Config() NUDConfigurations {
+ s.mu.RLock()
+ defer s.mu.RUnlock()
+ return s.config
+}
+
+// SetConfig replaces the existing NUD configurations with c.
+func (s *NUDState) SetConfig(c NUDConfigurations) {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+ s.config = c
+}
+
+// ReachableTime returns the duration to wait for a REACHABLE entry to
+// transition into STALE after inactivity. This value is recalculated for new
+// values of BaseReachableTime, MinRandomFactor, and MaxRandomFactor using the
+// algorithm defined in RFC 4861 section 6.3.2.
+func (s *NUDState) ReachableTime() time.Duration {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+
+ if time.Now().After(s.expiration) ||
+ s.config.BaseReachableTime != s.prevBaseReachableTime ||
+ s.config.MinRandomFactor != s.prevMinRandomFactor ||
+ s.config.MaxRandomFactor != s.prevMaxRandomFactor {
+ return s.recomputeReachableTimeLocked()
+ }
+ return s.reachableTime
+}
+
+// recomputeReachableTimeLocked forces a recalculation of ReachableTime using
+// the algorithm defined in RFC 4861 section 6.3.2.
+//
+// This SHOULD automatically be invoked during certain situations, as per
+// RFC 4861 section 6.3.4:
+//
+// If the received Reachable Time value is non-zero, the host SHOULD set its
+// BaseReachableTime variable to the received value. If the new value
+// differs from the previous value, the host SHOULD re-compute a new random
+// ReachableTime value. ReachableTime is computed as a uniformly
+// distributed random value between MIN_RANDOM_FACTOR and MAX_RANDOM_FACTOR
+// times the BaseReachableTime. Using a random component eliminates the
+// possibility that Neighbor Unreachability Detection messages will
+// synchronize with each other.
+//
+// In most cases, the advertised Reachable Time value will be the same in
+// consecutive Router Advertisements, and a host's BaseReachableTime rarely
+// changes. In such cases, an implementation SHOULD ensure that a new
+// random value gets re-computed at least once every few hours.
+//
+// s.mu MUST be locked for writing.
+func (s *NUDState) recomputeReachableTimeLocked() time.Duration {
+ s.prevBaseReachableTime = s.config.BaseReachableTime
+ s.prevMinRandomFactor = s.config.MinRandomFactor
+ s.prevMaxRandomFactor = s.config.MaxRandomFactor
+
+ randomFactor := s.config.MinRandomFactor + s.rng.Float32()*(s.config.MaxRandomFactor-s.config.MinRandomFactor)
+
+ // Check for overflow, given that minRandomFactor and maxRandomFactor are
+ // guaranteed to be positive numbers.
+ if float32(math.MaxInt64)/randomFactor < float32(s.config.BaseReachableTime) {
+ s.reachableTime = time.Duration(math.MaxInt64)
+ } else if randomFactor == 1 {
+ // Avoid loss of precision when a large base reachable time is used.
+ s.reachableTime = s.config.BaseReachableTime
+ } else {
+ reachableTime := int64(float32(s.config.BaseReachableTime) * randomFactor)
+ s.reachableTime = time.Duration(reachableTime)
+ }
+
+ s.expiration = time.Now().Add(2 * time.Hour)
+ return s.reachableTime
+}
diff --git a/pkg/tcpip/stack/nud_test.go b/pkg/tcpip/stack/nud_test.go
new file mode 100644
index 000000000..2494ee610
--- /dev/null
+++ b/pkg/tcpip/stack/nud_test.go
@@ -0,0 +1,795 @@
+// Copyright 2020 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package stack_test
+
+import (
+ "math"
+ "testing"
+ "time"
+
+ "gvisor.dev/gvisor/pkg/tcpip"
+ "gvisor.dev/gvisor/pkg/tcpip/link/channel"
+ "gvisor.dev/gvisor/pkg/tcpip/network/ipv6"
+ "gvisor.dev/gvisor/pkg/tcpip/stack"
+)
+
+const (
+ defaultBaseReachableTime = 30 * time.Second
+ minimumBaseReachableTime = time.Millisecond
+ defaultMinRandomFactor = 0.5
+ defaultMaxRandomFactor = 1.5
+ defaultRetransmitTimer = time.Second
+ minimumRetransmitTimer = time.Millisecond
+ defaultDelayFirstProbeTime = 5 * time.Second
+ defaultMaxMulticastProbes = 3
+ defaultMaxUnicastProbes = 3
+ defaultMaxAnycastDelayTime = time.Second
+ defaultMaxReachbilityConfirmations = 3
+ defaultUnreachableTime = 5 * time.Second
+
+ defaultFakeRandomNum = 0.5
+)
+
+// fakeRand is a deterministic random number generator.
+type fakeRand struct {
+ num float32
+}
+
+var _ stack.Rand = (*fakeRand)(nil)
+
+func (f *fakeRand) Float32() float32 {
+ return f.num
+}
+
+// TestSetNUDConfigurationFailsForBadNICID tests to make sure we get an error if
+// we attempt to update NUD configurations using an invalid NICID.
+func TestSetNUDConfigurationFailsForBadNICID(t *testing.T) {
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The networking
+ // stack will only allocate neighbor caches if a protocol providing link
+ // address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ })
+
+ // No NIC with ID 1 yet.
+ config := stack.NUDConfigurations{}
+ if err := s.SetNUDConfigurations(1, config); err != tcpip.ErrUnknownNICID {
+ t.Fatalf("got s.SetNDPConfigurations(1, %+v) = %v, want = %s", config, err, tcpip.ErrUnknownNICID)
+ }
+}
+
+// TestNUDConfigurationFailsForNotSupported tests to make sure we get a
+// NotSupported error if we attempt to retrieve NUD configurations when the
+// stack doesn't support NUD.
+//
+// The stack will report to not support NUD if a neighbor cache for a given NIC
+// is not allocated. The networking stack will only allocate neighbor caches if
+// a protocol providing link address resolution is specified (e.g. ARP, IPv6).
+func TestNUDConfigurationFailsForNotSupported(t *testing.T) {
+ const nicID = 1
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ NUDConfigs: stack.DefaultNUDConfigurations(),
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ if _, err := s.NUDConfigurations(nicID); err != tcpip.ErrNotSupported {
+ t.Fatalf("got s.NDPConfigurations(%d) = %v, want = %s", nicID, err, tcpip.ErrNotSupported)
+ }
+}
+
+// TestNUDConfigurationFailsForNotSupported tests to make sure we get a
+// NotSupported error if we attempt to set NUD configurations when the stack
+// doesn't support NUD.
+//
+// The stack will report to not support NUD if a neighbor cache for a given NIC
+// is not allocated. The networking stack will only allocate neighbor caches if
+// a protocol providing link address resolution is specified (e.g. ARP, IPv6).
+func TestSetNUDConfigurationFailsForNotSupported(t *testing.T) {
+ const nicID = 1
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ NUDConfigs: stack.DefaultNUDConfigurations(),
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+
+ config := stack.NUDConfigurations{}
+ if err := s.SetNUDConfigurations(nicID, config); err != tcpip.ErrNotSupported {
+ t.Fatalf("got s.SetNDPConfigurations(%d, %+v) = %v, want = %s", nicID, config, err, tcpip.ErrNotSupported)
+ }
+}
+
+// TestDefaultNUDConfigurationIsValid verifies that calling
+// resetInvalidFields() on the result of DefaultNUDConfigurations() does not
+// change anything. DefaultNUDConfigurations() should return a valid
+// NUDConfigurations.
+func TestDefaultNUDConfigurations(t *testing.T) {
+ const nicID = 1
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The networking
+ // stack will only allocate neighbor caches if a protocol providing link
+ // address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: stack.DefaultNUDConfigurations(),
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ c, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got, want := c, stack.DefaultNUDConfigurations(); got != want {
+ t.Errorf("got stack.NUDConfigurations(%d) = %+v, want = %+v", nicID, got, want)
+ }
+}
+
+func TestNUDConfigurationsBaseReachableTime(t *testing.T) {
+ tests := []struct {
+ name string
+ baseReachableTime time.Duration
+ want time.Duration
+ }{
+ // Invalid cases
+ {
+ name: "EqualToZero",
+ baseReachableTime: 0,
+ want: defaultBaseReachableTime,
+ },
+ // Valid cases
+ {
+ name: "MoreThanZero",
+ baseReachableTime: time.Millisecond,
+ want: time.Millisecond,
+ },
+ {
+ name: "MoreThanDefaultBaseReachableTime",
+ baseReachableTime: 2 * defaultBaseReachableTime,
+ want: 2 * defaultBaseReachableTime,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ const nicID = 1
+
+ c := stack.DefaultNUDConfigurations()
+ c.BaseReachableTime = test.baseReachableTime
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The
+ // networking stack will only allocate neighbor caches if a protocol
+ // providing link address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: c,
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ sc, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got := sc.BaseReachableTime; got != test.want {
+ t.Errorf("got BaseReachableTime = %q, want = %q", got, test.want)
+ }
+ })
+ }
+}
+
+func TestNUDConfigurationsMinRandomFactor(t *testing.T) {
+ tests := []struct {
+ name string
+ minRandomFactor float32
+ want float32
+ }{
+ // Invalid cases
+ {
+ name: "LessThanZero",
+ minRandomFactor: -1,
+ want: defaultMinRandomFactor,
+ },
+ {
+ name: "EqualToZero",
+ minRandomFactor: 0,
+ want: defaultMinRandomFactor,
+ },
+ // Valid cases
+ {
+ name: "MoreThanZero",
+ minRandomFactor: 1,
+ want: 1,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ const nicID = 1
+
+ c := stack.DefaultNUDConfigurations()
+ c.MinRandomFactor = test.minRandomFactor
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The
+ // networking stack will only allocate neighbor caches if a protocol
+ // providing link address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: c,
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ sc, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got := sc.MinRandomFactor; got != test.want {
+ t.Errorf("got MinRandomFactor = %f, want = %f", got, test.want)
+ }
+ })
+ }
+}
+
+func TestNUDConfigurationsMaxRandomFactor(t *testing.T) {
+ tests := []struct {
+ name string
+ minRandomFactor float32
+ maxRandomFactor float32
+ want float32
+ }{
+ // Invalid cases
+ {
+ name: "LessThanZero",
+ minRandomFactor: defaultMinRandomFactor,
+ maxRandomFactor: -1,
+ want: defaultMaxRandomFactor,
+ },
+ {
+ name: "EqualToZero",
+ minRandomFactor: defaultMinRandomFactor,
+ maxRandomFactor: 0,
+ want: defaultMaxRandomFactor,
+ },
+ {
+ name: "LessThanMinRandomFactor",
+ minRandomFactor: defaultMinRandomFactor,
+ maxRandomFactor: defaultMinRandomFactor * 0.99,
+ want: defaultMaxRandomFactor,
+ },
+ {
+ name: "MoreThanMinRandomFactorWhenMinRandomFactorIsLargerThanMaxRandomFactorDefault",
+ minRandomFactor: defaultMaxRandomFactor * 2,
+ maxRandomFactor: defaultMaxRandomFactor,
+ want: defaultMaxRandomFactor * 6,
+ },
+ // Valid cases
+ {
+ name: "EqualToMinRandomFactor",
+ minRandomFactor: defaultMinRandomFactor,
+ maxRandomFactor: defaultMinRandomFactor,
+ want: defaultMinRandomFactor,
+ },
+ {
+ name: "MoreThanMinRandomFactor",
+ minRandomFactor: defaultMinRandomFactor,
+ maxRandomFactor: defaultMinRandomFactor * 1.1,
+ want: defaultMinRandomFactor * 1.1,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ const nicID = 1
+
+ c := stack.DefaultNUDConfigurations()
+ c.MinRandomFactor = test.minRandomFactor
+ c.MaxRandomFactor = test.maxRandomFactor
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The
+ // networking stack will only allocate neighbor caches if a protocol
+ // providing link address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: c,
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ sc, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got := sc.MaxRandomFactor; got != test.want {
+ t.Errorf("got MaxRandomFactor = %f, want = %f", got, test.want)
+ }
+ })
+ }
+}
+
+func TestNUDConfigurationsRetransmitTimer(t *testing.T) {
+ tests := []struct {
+ name string
+ retransmitTimer time.Duration
+ want time.Duration
+ }{
+ // Invalid cases
+ {
+ name: "EqualToZero",
+ retransmitTimer: 0,
+ want: defaultRetransmitTimer,
+ },
+ {
+ name: "LessThanMinimumRetransmitTimer",
+ retransmitTimer: minimumRetransmitTimer - time.Nanosecond,
+ want: defaultRetransmitTimer,
+ },
+ // Valid cases
+ {
+ name: "EqualToMinimumRetransmitTimer",
+ retransmitTimer: minimumRetransmitTimer,
+ want: minimumBaseReachableTime,
+ },
+ {
+ name: "LargetThanMinimumRetransmitTimer",
+ retransmitTimer: 2 * minimumBaseReachableTime,
+ want: 2 * minimumBaseReachableTime,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ const nicID = 1
+
+ c := stack.DefaultNUDConfigurations()
+ c.RetransmitTimer = test.retransmitTimer
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The
+ // networking stack will only allocate neighbor caches if a protocol
+ // providing link address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: c,
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ sc, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got := sc.RetransmitTimer; got != test.want {
+ t.Errorf("got RetransmitTimer = %q, want = %q", got, test.want)
+ }
+ })
+ }
+}
+
+func TestNUDConfigurationsDelayFirstProbeTime(t *testing.T) {
+ tests := []struct {
+ name string
+ delayFirstProbeTime time.Duration
+ want time.Duration
+ }{
+ // Invalid cases
+ {
+ name: "EqualToZero",
+ delayFirstProbeTime: 0,
+ want: defaultDelayFirstProbeTime,
+ },
+ // Valid cases
+ {
+ name: "MoreThanZero",
+ delayFirstProbeTime: time.Millisecond,
+ want: time.Millisecond,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ const nicID = 1
+
+ c := stack.DefaultNUDConfigurations()
+ c.DelayFirstProbeTime = test.delayFirstProbeTime
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The
+ // networking stack will only allocate neighbor caches if a protocol
+ // providing link address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: c,
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ sc, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got := sc.DelayFirstProbeTime; got != test.want {
+ t.Errorf("got DelayFirstProbeTime = %q, want = %q", got, test.want)
+ }
+ })
+ }
+}
+
+func TestNUDConfigurationsMaxMulticastProbes(t *testing.T) {
+ tests := []struct {
+ name string
+ maxMulticastProbes uint32
+ want uint32
+ }{
+ // Invalid cases
+ {
+ name: "EqualToZero",
+ maxMulticastProbes: 0,
+ want: defaultMaxMulticastProbes,
+ },
+ // Valid cases
+ {
+ name: "MoreThanZero",
+ maxMulticastProbes: 1,
+ want: 1,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ const nicID = 1
+
+ c := stack.DefaultNUDConfigurations()
+ c.MaxMulticastProbes = test.maxMulticastProbes
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The
+ // networking stack will only allocate neighbor caches if a protocol
+ // providing link address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: c,
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ sc, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got := sc.MaxMulticastProbes; got != test.want {
+ t.Errorf("got MaxMulticastProbes = %q, want = %q", got, test.want)
+ }
+ })
+ }
+}
+
+func TestNUDConfigurationsMaxUnicastProbes(t *testing.T) {
+ tests := []struct {
+ name string
+ maxUnicastProbes uint32
+ want uint32
+ }{
+ // Invalid cases
+ {
+ name: "EqualToZero",
+ maxUnicastProbes: 0,
+ want: defaultMaxUnicastProbes,
+ },
+ // Valid cases
+ {
+ name: "MoreThanZero",
+ maxUnicastProbes: 1,
+ want: 1,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ const nicID = 1
+
+ c := stack.DefaultNUDConfigurations()
+ c.MaxUnicastProbes = test.maxUnicastProbes
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The
+ // networking stack will only allocate neighbor caches if a protocol
+ // providing link address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: c,
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ sc, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got := sc.MaxUnicastProbes; got != test.want {
+ t.Errorf("got MaxUnicastProbes = %q, want = %q", got, test.want)
+ }
+ })
+ }
+}
+
+func TestNUDConfigurationsUnreachableTime(t *testing.T) {
+ tests := []struct {
+ name string
+ unreachableTime time.Duration
+ want time.Duration
+ }{
+ // Invalid cases
+ {
+ name: "EqualToZero",
+ unreachableTime: 0,
+ want: defaultUnreachableTime,
+ },
+ // Valid cases
+ {
+ name: "MoreThanZero",
+ unreachableTime: time.Millisecond,
+ want: time.Millisecond,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ const nicID = 1
+
+ c := stack.DefaultNUDConfigurations()
+ c.UnreachableTime = test.unreachableTime
+
+ e := channel.New(0, 1280, linkAddr1)
+ e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+
+ s := stack.New(stack.Options{
+ // A neighbor cache is required to store NUDConfigurations. The
+ // networking stack will only allocate neighbor caches if a protocol
+ // providing link address resolution is specified (e.g. ARP or IPv6).
+ NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()},
+ NUDConfigs: c,
+ })
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
+ sc, err := s.NUDConfigurations(nicID)
+ if err != nil {
+ t.Fatalf("got stack.NUDConfigurations(%d) = %s", nicID, err)
+ }
+ if got := sc.UnreachableTime; got != test.want {
+ t.Errorf("got UnreachableTime = %q, want = %q", got, test.want)
+ }
+ })
+ }
+}
+
+// TestNUDStateReachableTime verifies the correctness of the ReachableTime
+// computation.
+func TestNUDStateReachableTime(t *testing.T) {
+ tests := []struct {
+ name string
+ baseReachableTime time.Duration
+ minRandomFactor float32
+ maxRandomFactor float32
+ want time.Duration
+ }{
+ {
+ name: "AllZeros",
+ baseReachableTime: 0,
+ minRandomFactor: 0,
+ maxRandomFactor: 0,
+ want: 0,
+ },
+ {
+ name: "ZeroMaxRandomFactor",
+ baseReachableTime: time.Second,
+ minRandomFactor: 0,
+ maxRandomFactor: 0,
+ want: 0,
+ },
+ {
+ name: "ZeroMinRandomFactor",
+ baseReachableTime: time.Second,
+ minRandomFactor: 0,
+ maxRandomFactor: 1,
+ want: time.Duration(defaultFakeRandomNum * float32(time.Second)),
+ },
+ {
+ name: "FractionalRandomFactor",
+ baseReachableTime: time.Duration(math.MaxInt64),
+ minRandomFactor: 0.001,
+ maxRandomFactor: 0.002,
+ want: time.Duration((0.001 + (0.001 * defaultFakeRandomNum)) * float32(math.MaxInt64)),
+ },
+ {
+ name: "MinAndMaxRandomFactorsEqual",
+ baseReachableTime: time.Second,
+ minRandomFactor: 1,
+ maxRandomFactor: 1,
+ want: time.Second,
+ },
+ {
+ name: "MinAndMaxRandomFactorsDifferent",
+ baseReachableTime: time.Second,
+ minRandomFactor: 1,
+ maxRandomFactor: 2,
+ want: time.Duration((1.0 + defaultFakeRandomNum) * float32(time.Second)),
+ },
+ {
+ name: "MaxInt64",
+ baseReachableTime: time.Duration(math.MaxInt64),
+ minRandomFactor: 1,
+ maxRandomFactor: 1,
+ want: time.Duration(math.MaxInt64),
+ },
+ {
+ name: "Overflow",
+ baseReachableTime: time.Duration(math.MaxInt64),
+ minRandomFactor: 1.5,
+ maxRandomFactor: 1.5,
+ want: time.Duration(math.MaxInt64),
+ },
+ {
+ name: "DoubleOverflow",
+ baseReachableTime: time.Duration(math.MaxInt64),
+ minRandomFactor: 2.5,
+ maxRandomFactor: 2.5,
+ want: time.Duration(math.MaxInt64),
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ c := stack.NUDConfigurations{
+ BaseReachableTime: test.baseReachableTime,
+ MinRandomFactor: test.minRandomFactor,
+ MaxRandomFactor: test.maxRandomFactor,
+ }
+ // A fake random number generator is used to ensure deterministic
+ // results.
+ rng := fakeRand{
+ num: defaultFakeRandomNum,
+ }
+ s := stack.NewNUDState(c, &rng)
+ if got, want := s.ReachableTime(), test.want; got != want {
+ t.Errorf("got ReachableTime = %q, want = %q", got, want)
+ }
+ })
+ }
+}
+
+// TestNUDStateRecomputeReachableTime exercises the ReachableTime function
+// twice to verify recomputation of reachable time when the min random factor,
+// max random factor, or base reachable time changes.
+func TestNUDStateRecomputeReachableTime(t *testing.T) {
+ const defaultBase = time.Second
+ const defaultMin = 2.0 * defaultMaxRandomFactor
+ const defaultMax = 3.0 * defaultMaxRandomFactor
+
+ tests := []struct {
+ name string
+ baseReachableTime time.Duration
+ minRandomFactor float32
+ maxRandomFactor float32
+ want time.Duration
+ }{
+ {
+ name: "BaseReachableTime",
+ baseReachableTime: 2 * defaultBase,
+ minRandomFactor: defaultMin,
+ maxRandomFactor: defaultMax,
+ want: time.Duration((defaultMin + (defaultMax-defaultMin)*defaultFakeRandomNum) * float32(2*defaultBase)),
+ },
+ {
+ name: "MinRandomFactor",
+ baseReachableTime: defaultBase,
+ minRandomFactor: defaultMax,
+ maxRandomFactor: defaultMax,
+ want: time.Duration(defaultMax * float32(defaultBase)),
+ },
+ {
+ name: "MaxRandomFactor",
+ baseReachableTime: defaultBase,
+ minRandomFactor: defaultMin,
+ maxRandomFactor: defaultMin,
+ want: time.Duration(defaultMin * float32(defaultBase)),
+ },
+ {
+ name: "BothRandomFactor",
+ baseReachableTime: defaultBase,
+ minRandomFactor: 2 * defaultMin,
+ maxRandomFactor: 2 * defaultMax,
+ want: time.Duration((2*defaultMin + (2*defaultMax-2*defaultMin)*defaultFakeRandomNum) * float32(defaultBase)),
+ },
+ {
+ name: "BaseReachableTimeAndBothRandomFactors",
+ baseReachableTime: 2 * defaultBase,
+ minRandomFactor: 2 * defaultMin,
+ maxRandomFactor: 2 * defaultMax,
+ want: time.Duration((2*defaultMin + (2*defaultMax-2*defaultMin)*defaultFakeRandomNum) * float32(2*defaultBase)),
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ c := stack.DefaultNUDConfigurations()
+ c.BaseReachableTime = defaultBase
+ c.MinRandomFactor = defaultMin
+ c.MaxRandomFactor = defaultMax
+
+ // A fake random number generator is used to ensure deterministic
+ // results.
+ rng := fakeRand{
+ num: defaultFakeRandomNum,
+ }
+ s := stack.NewNUDState(c, &rng)
+ old := s.ReachableTime()
+
+ if got, want := s.ReachableTime(), old; got != want {
+ t.Errorf("got ReachableTime = %q, want = %q", got, want)
+ }
+
+ // Check for recomputation when changing the min random factor, the max
+ // random factor, the base reachability time, or any permutation of those
+ // three options.
+ c.BaseReachableTime = test.baseReachableTime
+ c.MinRandomFactor = test.minRandomFactor
+ c.MaxRandomFactor = test.maxRandomFactor
+ s.SetConfig(c)
+
+ if got, want := s.ReachableTime(), test.want; got != want {
+ t.Errorf("got ReachableTime = %q, want = %q", got, want)
+ }
+
+ // Verify that ReachableTime isn't recomputed when none of the
+ // configuration options change. The random factor is changed so that if
+ // a recompution were to occur, ReachableTime would change.
+ rng.num = defaultFakeRandomNum / 2.0
+ if got, want := s.ReachableTime(), test.want; got != want {
+ t.Errorf("got ReachableTime = %q, want = %q", got, want)
+ }
+ })
+ }
+}
diff --git a/pkg/tcpip/stack/route.go b/pkg/tcpip/stack/route.go
index d65f8049e..91e0110f1 100644
--- a/pkg/tcpip/stack/route.go
+++ b/pkg/tcpip/stack/route.go
@@ -48,6 +48,10 @@ type Route struct {
// Loop controls where WritePacket should send packets.
Loop PacketLooping
+
+ // directedBroadcast indicates whether this route is sending a directed
+ // broadcast packet.
+ directedBroadcast bool
}
// makeRoute initializes a new route. It takes ownership of the provided
@@ -275,6 +279,12 @@ func (r *Route) Stack() *Stack {
return r.ref.stack()
}
+// IsBroadcast returns true if the route is to send a broadcast packet.
+func (r *Route) IsBroadcast() bool {
+ // Only IPv4 has a notion of broadcast.
+ return r.directedBroadcast || r.RemoteAddress == header.IPv4Broadcast
+}
+
// ReverseRoute returns new route with given source and destination address.
func (r *Route) ReverseRoute(src tcpip.Address, dst tcpip.Address) Route {
return Route{
diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go
index a6faa22c2..3f07e4159 100644
--- a/pkg/tcpip/stack/stack.go
+++ b/pkg/tcpip/stack/stack.go
@@ -445,6 +445,9 @@ type Stack struct {
// ndpConfigs is the default NDP configurations used by interfaces.
ndpConfigs NDPConfigurations
+ // nudConfigs is the default NUD configurations used by interfaces.
+ nudConfigs NUDConfigurations
+
// autoGenIPv6LinkLocal determines whether or not the stack will attempt
// to auto-generate an IPv6 link-local address for newly enabled non-loopback
// NICs. See the AutoGenIPv6LinkLocal field of Options for more details.
@@ -454,6 +457,10 @@ type Stack struct {
// integrator NDP related events.
ndpDisp NDPDispatcher
+ // nudDisp is the NUD event dispatcher that is used to send the netstack
+ // integrator NUD related events.
+ nudDisp NUDDispatcher
+
// uniqueIDGenerator is a generator of unique identifiers.
uniqueIDGenerator UniqueID
@@ -518,6 +525,9 @@ type Options struct {
// before assigning an address to a NIC.
NDPConfigs NDPConfigurations
+ // NUDConfigs is the default NUD configurations used by interfaces.
+ NUDConfigs NUDConfigurations
+
// AutoGenIPv6LinkLocal determines whether or not the stack will attempt to
// auto-generate an IPv6 link-local address for newly enabled non-loopback
// NICs.
@@ -536,6 +546,10 @@ type Options struct {
// receive NDP related events.
NDPDisp NDPDispatcher
+ // NUDDisp is the NUD event dispatcher that an integrator can provide to
+ // receive NUD related events.
+ NUDDisp NUDDispatcher
+
// RawFactory produces raw endpoints. Raw endpoints are enabled only if
// this is non-nil.
RawFactory RawFactory
@@ -670,6 +684,8 @@ func New(opts Options) *Stack {
// Make sure opts.NDPConfigs contains valid values only.
opts.NDPConfigs.validate()
+ opts.NUDConfigs.resetInvalidFields()
+
s := &Stack{
transportProtocols: make(map[tcpip.TransportProtocolNumber]*transportProtocolState),
networkProtocols: make(map[tcpip.NetworkProtocolNumber]NetworkProtocol),
@@ -685,9 +701,11 @@ func New(opts Options) *Stack {
icmpRateLimiter: NewICMPRateLimiter(),
seed: generateRandUint32(),
ndpConfigs: opts.NDPConfigs,
+ nudConfigs: opts.NUDConfigs,
autoGenIPv6LinkLocal: opts.AutoGenIPv6LinkLocal,
uniqueIDGenerator: opts.UniqueID,
ndpDisp: opts.NDPDisp,
+ nudDisp: opts.NUDDisp,
opaqueIIDOpts: opts.OpaqueIIDOpts,
tempIIDSeed: opts.TempIIDSeed,
forwarder: newForwardQueue(),
@@ -1284,9 +1302,9 @@ func (s *Stack) FindRoute(id tcpip.NICID, localAddr, remoteAddr tcpip.Address, n
s.mu.RLock()
defer s.mu.RUnlock()
- isBroadcast := remoteAddr == header.IPv4Broadcast
+ isLocalBroadcast := remoteAddr == header.IPv4Broadcast
isMulticast := header.IsV4MulticastAddress(remoteAddr) || header.IsV6MulticastAddress(remoteAddr)
- needRoute := !(isBroadcast || isMulticast || header.IsV6LinkLocalAddress(remoteAddr))
+ needRoute := !(isLocalBroadcast || isMulticast || header.IsV6LinkLocalAddress(remoteAddr))
if id != 0 && !needRoute {
if nic, ok := s.nics[id]; ok && nic.enabled() {
if ref := s.getRefEP(nic, localAddr, remoteAddr, netProto); ref != nil {
@@ -1307,9 +1325,16 @@ func (s *Stack) FindRoute(id tcpip.NICID, localAddr, remoteAddr tcpip.Address, n
}
r := makeRoute(netProto, ref.ep.ID().LocalAddress, remoteAddr, nic.linkEP.LinkAddress(), ref, s.handleLocal && !nic.isLoopback(), multicastLoop && !nic.isLoopback())
- if needRoute {
- r.NextHop = route.Gateway
+ r.directedBroadcast = route.Destination.IsBroadcast(remoteAddr)
+
+ if len(route.Gateway) > 0 {
+ if needRoute {
+ r.NextHop = route.Gateway
+ }
+ } else if r.directedBroadcast {
+ r.RemoteLinkAddress = header.EthernetBroadcastAddress
}
+
return r, nil
}
}
@@ -1862,10 +1887,38 @@ func (s *Stack) SetNDPConfigurations(id tcpip.NICID, c NDPConfigurations) *tcpip
}
nic.setNDPConfigs(c)
-
return nil
}
+// NUDConfigurations gets the per-interface NUD configurations.
+func (s *Stack) NUDConfigurations(id tcpip.NICID) (NUDConfigurations, *tcpip.Error) {
+ s.mu.RLock()
+ nic, ok := s.nics[id]
+ s.mu.RUnlock()
+
+ if !ok {
+ return NUDConfigurations{}, tcpip.ErrUnknownNICID
+ }
+
+ return nic.NUDConfigs()
+}
+
+// SetNUDConfigurations sets the per-interface NUD configurations.
+//
+// Note, if c contains invalid NUD configuration values, it will be fixed to
+// use default values for the erroneous values.
+func (s *Stack) SetNUDConfigurations(id tcpip.NICID, c NUDConfigurations) *tcpip.Error {
+ s.mu.RLock()
+ nic, ok := s.nics[id]
+ s.mu.RUnlock()
+
+ if !ok {
+ return tcpip.ErrUnknownNICID
+ }
+
+ return nic.setNUDConfigs(c)
+}
+
// HandleNDPRA provides a NIC with ID id a validated NDP Router Advertisement
// message that it needs to handle.
func (s *Stack) HandleNDPRA(id tcpip.NICID, ip tcpip.Address, ra header.NDPRouterAdvert) *tcpip.Error {
diff --git a/pkg/tcpip/stack/stack_test.go b/pkg/tcpip/stack/stack_test.go
index 101ca2206..f22062889 100644
--- a/pkg/tcpip/stack/stack_test.go
+++ b/pkg/tcpip/stack/stack_test.go
@@ -27,6 +27,7 @@ import (
"time"
"github.com/google/go-cmp/cmp"
+ "github.com/google/go-cmp/cmp/cmpopts"
"gvisor.dev/gvisor/pkg/rand"
"gvisor.dev/gvisor/pkg/tcpip"
"gvisor.dev/gvisor/pkg/tcpip/buffer"
@@ -3418,3 +3419,225 @@ func TestStackSendBufferSizeOption(t *testing.T) {
})
}
}
+
+func TestOutgoingSubnetBroadcast(t *testing.T) {
+ const (
+ unspecifiedNICID = 0
+ nicID1 = 1
+ )
+
+ defaultAddr := tcpip.AddressWithPrefix{
+ Address: header.IPv4Any,
+ PrefixLen: 0,
+ }
+ defaultSubnet := defaultAddr.Subnet()
+ ipv4Addr := tcpip.AddressWithPrefix{
+ Address: "\xc0\xa8\x01\x3a",
+ PrefixLen: 24,
+ }
+ ipv4Subnet := ipv4Addr.Subnet()
+ ipv4SubnetBcast := ipv4Subnet.Broadcast()
+ ipv4Gateway := tcpip.Address("\xc0\xa8\x01\x01")
+ ipv4AddrPrefix31 := tcpip.AddressWithPrefix{
+ Address: "\xc0\xa8\x01\x3a",
+ PrefixLen: 31,
+ }
+ ipv4Subnet31 := ipv4AddrPrefix31.Subnet()
+ ipv4Subnet31Bcast := ipv4Subnet31.Broadcast()
+ ipv4AddrPrefix32 := tcpip.AddressWithPrefix{
+ Address: "\xc0\xa8\x01\x3a",
+ PrefixLen: 32,
+ }
+ ipv4Subnet32 := ipv4AddrPrefix32.Subnet()
+ ipv4Subnet32Bcast := ipv4Subnet32.Broadcast()
+ ipv6Addr := tcpip.AddressWithPrefix{
+ Address: "\x20\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01",
+ PrefixLen: 64,
+ }
+ ipv6Subnet := ipv6Addr.Subnet()
+ ipv6SubnetBcast := ipv6Subnet.Broadcast()
+ remNetAddr := tcpip.AddressWithPrefix{
+ Address: "\x64\x0a\x7b\x18",
+ PrefixLen: 24,
+ }
+ remNetSubnet := remNetAddr.Subnet()
+ remNetSubnetBcast := remNetSubnet.Broadcast()
+
+ tests := []struct {
+ name string
+ nicAddr tcpip.ProtocolAddress
+ routes []tcpip.Route
+ remoteAddr tcpip.Address
+ expectedRoute stack.Route
+ }{
+ // Broadcast to a locally attached subnet populates the broadcast MAC.
+ {
+ name: "IPv4 Broadcast to local subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4Addr,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: ipv4Subnet,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: ipv4SubnetBcast,
+ expectedRoute: stack.Route{
+ LocalAddress: ipv4Addr.Address,
+ RemoteAddress: ipv4SubnetBcast,
+ RemoteLinkAddress: header.EthernetBroadcastAddress,
+ NetProto: header.IPv4ProtocolNumber,
+ Loop: stack.PacketOut,
+ },
+ },
+ // Broadcast to a locally attached /31 subnet does not populate the
+ // broadcast MAC.
+ {
+ name: "IPv4 Broadcast to local /31 subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4AddrPrefix31,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: ipv4Subnet31,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: ipv4Subnet31Bcast,
+ expectedRoute: stack.Route{
+ LocalAddress: ipv4AddrPrefix31.Address,
+ RemoteAddress: ipv4Subnet31Bcast,
+ NetProto: header.IPv4ProtocolNumber,
+ Loop: stack.PacketOut,
+ },
+ },
+ // Broadcast to a locally attached /32 subnet does not populate the
+ // broadcast MAC.
+ {
+ name: "IPv4 Broadcast to local /32 subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4AddrPrefix32,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: ipv4Subnet32,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: ipv4Subnet32Bcast,
+ expectedRoute: stack.Route{
+ LocalAddress: ipv4AddrPrefix32.Address,
+ RemoteAddress: ipv4Subnet32Bcast,
+ NetProto: header.IPv4ProtocolNumber,
+ Loop: stack.PacketOut,
+ },
+ },
+ // IPv6 has no notion of a broadcast.
+ {
+ name: "IPv6 'Broadcast' to local subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv6ProtocolNumber,
+ AddressWithPrefix: ipv6Addr,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: ipv6Subnet,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: ipv6SubnetBcast,
+ expectedRoute: stack.Route{
+ LocalAddress: ipv6Addr.Address,
+ RemoteAddress: ipv6SubnetBcast,
+ NetProto: header.IPv6ProtocolNumber,
+ Loop: stack.PacketOut,
+ },
+ },
+ // Broadcast to a remote subnet in the route table is send to the next-hop
+ // gateway.
+ {
+ name: "IPv4 Broadcast to remote subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4Addr,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: remNetSubnet,
+ Gateway: ipv4Gateway,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: remNetSubnetBcast,
+ expectedRoute: stack.Route{
+ LocalAddress: ipv4Addr.Address,
+ RemoteAddress: remNetSubnetBcast,
+ NextHop: ipv4Gateway,
+ NetProto: header.IPv4ProtocolNumber,
+ Loop: stack.PacketOut,
+ },
+ },
+ // Broadcast to an unknown subnet follows the default route. Note that this
+ // is essentially just routing an unknown destination IP, because w/o any
+ // subnet prefix information a subnet broadcast address is just a normal IP.
+ {
+ name: "IPv4 Broadcast to unknown subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4Addr,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: defaultSubnet,
+ Gateway: ipv4Gateway,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: remNetSubnetBcast,
+ expectedRoute: stack.Route{
+ LocalAddress: ipv4Addr.Address,
+ RemoteAddress: remNetSubnetBcast,
+ NextHop: ipv4Gateway,
+ NetProto: header.IPv4ProtocolNumber,
+ Loop: stack.PacketOut,
+ },
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ s := stack.New(stack.Options{
+ NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()},
+ })
+ ep := channel.New(0, defaultMTU, "")
+ if err := s.CreateNIC(nicID1, ep); err != nil {
+ t.Fatalf("CreateNIC(%d, _): %s", nicID1, err)
+ }
+ if err := s.AddProtocolAddress(nicID1, test.nicAddr); err != nil {
+ t.Fatalf("AddProtocolAddress(%d, %+v): %s", nicID1, test.nicAddr, err)
+ }
+
+ s.SetRouteTable(test.routes)
+
+ var netProto tcpip.NetworkProtocolNumber
+ switch l := len(test.remoteAddr); l {
+ case header.IPv4AddressSize:
+ netProto = header.IPv4ProtocolNumber
+ case header.IPv6AddressSize:
+ netProto = header.IPv6ProtocolNumber
+ default:
+ t.Fatalf("got unexpected address length = %d bytes", l)
+ }
+
+ if r, err := s.FindRoute(unspecifiedNICID, "" /* localAddr */, test.remoteAddr, netProto, false /* multicastLoop */); err != nil {
+ t.Fatalf("FindRoute(%d, '', %s, %d): %s", unspecifiedNICID, test.remoteAddr, netProto, err)
+ } else if diff := cmp.Diff(r, test.expectedRoute, cmpopts.IgnoreUnexported(r)); diff != "" {
+ t.Errorf("route mismatch (-want +got):\n%s", diff)
+ }
+ })
+ }
+}
diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go
index 21aafb0a2..a634b9b60 100644
--- a/pkg/tcpip/tcpip.go
+++ b/pkg/tcpip/tcpip.go
@@ -43,6 +43,9 @@ import (
"gvisor.dev/gvisor/pkg/waiter"
)
+// Using header.IPv4AddressSize would cause an import cycle.
+const ipv4AddressSize = 4
+
// Error represents an error in the netstack error space. Using a special type
// ensures that errors outside of this space are not accidentally introduced.
//
@@ -320,6 +323,29 @@ func (s *Subnet) Broadcast() Address {
return Address(addr)
}
+// IsBroadcast returns true if the address is considered a broadcast address.
+func (s *Subnet) IsBroadcast(address Address) bool {
+ // Only IPv4 supports the notion of a broadcast address.
+ if len(address) != ipv4AddressSize {
+ return false
+ }
+
+ // Normally, we would just compare address with the subnet's broadcast
+ // address but there is an exception where a simple comparison is not
+ // correct. This exception is for /31 and /32 IPv4 subnets where all
+ // addresses are considered valid host addresses.
+ //
+ // For /31 subnets, the case is easy. RFC 3021 Section 2.1 states that
+ // both addresses in a /31 subnet "MUST be interpreted as host addresses."
+ //
+ // For /32, the case is a bit more vague. RFC 3021 makes no mention of /32
+ // subnets. However, the same reasoning applies - if an exception is not
+ // made, then there do not exist any host addresses in a /32 subnet. RFC
+ // 4632 Section 3.1 also vaguely implies this interpretation by referring
+ // to addresses in /32 subnets as "host routes."
+ return s.Prefix() <= 30 && s.Broadcast() == address
+}
+
// Equal returns true if s equals o.
//
// Needed to use cmp.Equal on Subnet as its fields are unexported.
diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go
index 5d6174a59..b34e47bbd 100644
--- a/pkg/tcpip/transport/tcp/protocol.go
+++ b/pkg/tcpip/transport/tcp/protocol.go
@@ -49,7 +49,7 @@ const (
// DefaultReceiveBufferSize is the default size of the receive buffer
// for an endpoint.
- DefaultReceiveBufferSize = 32 << 10 // 32KB
+ DefaultReceiveBufferSize = 1 << 20 // 1MB
// MaxBufferSize is the largest size a receive/send buffer can grow to.
MaxBufferSize = 4 << 20 // 4MB
diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go
index 6e692da07..b7d735889 100644
--- a/pkg/tcpip/transport/udp/endpoint.go
+++ b/pkg/tcpip/transport/udp/endpoint.go
@@ -483,10 +483,6 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
nicID = e.BindNICID
}
- if to.Addr == header.IPv4Broadcast && !e.broadcast {
- return 0, nil, tcpip.ErrBroadcastDisabled
- }
-
dst, netProto, err := e.checkV4MappedLocked(*to)
if err != nil {
return 0, nil, err
@@ -503,6 +499,10 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
resolve = route.Resolve
}
+ if !e.broadcast && route.IsBroadcast() {
+ return 0, nil, tcpip.ErrBroadcastDisabled
+ }
+
if route.IsResolutionRequired() {
if ch, err := resolve(nil); err != nil {
if err == tcpip.ErrWouldBlock {
diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go
index 90781cf49..66e8911c8 100644
--- a/pkg/tcpip/transport/udp/udp_test.go
+++ b/pkg/tcpip/transport/udp/udp_test.go
@@ -2142,3 +2142,192 @@ func (c *testContext) checkEndpointReadStats(incr uint64, want tcpip.TransportEn
c.t.Errorf("Endpoint stats not matching for error %s got %+v want %+v", err, got, want)
}
}
+
+func TestOutgoingSubnetBroadcast(t *testing.T) {
+ const nicID1 = 1
+
+ ipv4Addr := tcpip.AddressWithPrefix{
+ Address: "\xc0\xa8\x01\x3a",
+ PrefixLen: 24,
+ }
+ ipv4Subnet := ipv4Addr.Subnet()
+ ipv4SubnetBcast := ipv4Subnet.Broadcast()
+ ipv4Gateway := tcpip.Address("\xc0\xa8\x01\x01")
+ ipv4AddrPrefix31 := tcpip.AddressWithPrefix{
+ Address: "\xc0\xa8\x01\x3a",
+ PrefixLen: 31,
+ }
+ ipv4Subnet31 := ipv4AddrPrefix31.Subnet()
+ ipv4Subnet31Bcast := ipv4Subnet31.Broadcast()
+ ipv4AddrPrefix32 := tcpip.AddressWithPrefix{
+ Address: "\xc0\xa8\x01\x3a",
+ PrefixLen: 32,
+ }
+ ipv4Subnet32 := ipv4AddrPrefix32.Subnet()
+ ipv4Subnet32Bcast := ipv4Subnet32.Broadcast()
+ ipv6Addr := tcpip.AddressWithPrefix{
+ Address: "\x20\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01",
+ PrefixLen: 64,
+ }
+ ipv6Subnet := ipv6Addr.Subnet()
+ ipv6SubnetBcast := ipv6Subnet.Broadcast()
+ remNetAddr := tcpip.AddressWithPrefix{
+ Address: "\x64\x0a\x7b\x18",
+ PrefixLen: 24,
+ }
+ remNetSubnet := remNetAddr.Subnet()
+ remNetSubnetBcast := remNetSubnet.Broadcast()
+
+ tests := []struct {
+ name string
+ nicAddr tcpip.ProtocolAddress
+ routes []tcpip.Route
+ remoteAddr tcpip.Address
+ requiresBroadcastOpt bool
+ }{
+ {
+ name: "IPv4 Broadcast to local subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4Addr,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: ipv4Subnet,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: ipv4SubnetBcast,
+ requiresBroadcastOpt: true,
+ },
+ {
+ name: "IPv4 Broadcast to local /31 subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4AddrPrefix31,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: ipv4Subnet31,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: ipv4Subnet31Bcast,
+ requiresBroadcastOpt: false,
+ },
+ {
+ name: "IPv4 Broadcast to local /32 subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4AddrPrefix32,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: ipv4Subnet32,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: ipv4Subnet32Bcast,
+ requiresBroadcastOpt: false,
+ },
+ // IPv6 has no notion of a broadcast.
+ {
+ name: "IPv6 'Broadcast' to local subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv6ProtocolNumber,
+ AddressWithPrefix: ipv6Addr,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: ipv6Subnet,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: ipv6SubnetBcast,
+ requiresBroadcastOpt: false,
+ },
+ {
+ name: "IPv4 Broadcast to remote subnet",
+ nicAddr: tcpip.ProtocolAddress{
+ Protocol: header.IPv4ProtocolNumber,
+ AddressWithPrefix: ipv4Addr,
+ },
+ routes: []tcpip.Route{
+ {
+ Destination: remNetSubnet,
+ Gateway: ipv4Gateway,
+ NIC: nicID1,
+ },
+ },
+ remoteAddr: remNetSubnetBcast,
+ requiresBroadcastOpt: true,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ s := stack.New(stack.Options{
+ NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()},
+
+ TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()},
+ })
+ e := channel.New(0, defaultMTU, "")
+ if err := s.CreateNIC(nicID1, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _): %s", nicID1, err)
+ }
+ if err := s.AddProtocolAddress(nicID1, test.nicAddr); err != nil {
+ t.Fatalf("AddProtocolAddress(%d, %+v): %s", nicID1, test.nicAddr, err)
+ }
+
+ s.SetRouteTable(test.routes)
+
+ var netProto tcpip.NetworkProtocolNumber
+ switch l := len(test.remoteAddr); l {
+ case header.IPv4AddressSize:
+ netProto = header.IPv4ProtocolNumber
+ case header.IPv6AddressSize:
+ netProto = header.IPv6ProtocolNumber
+ default:
+ t.Fatalf("got unexpected address length = %d bytes", l)
+ }
+
+ wq := waiter.Queue{}
+ ep, err := s.NewEndpoint(udp.ProtocolNumber, netProto, &wq)
+ if err != nil {
+ t.Fatalf("NewEndpoint(%d, %d, _): %s", udp.ProtocolNumber, netProto, err)
+ }
+ defer ep.Close()
+
+ data := tcpip.SlicePayload([]byte{1, 2, 3, 4})
+ to := tcpip.FullAddress{
+ Addr: test.remoteAddr,
+ Port: 80,
+ }
+ opts := tcpip.WriteOptions{To: &to}
+ expectedErrWithoutBcastOpt := tcpip.ErrBroadcastDisabled
+ if !test.requiresBroadcastOpt {
+ expectedErrWithoutBcastOpt = nil
+ }
+
+ if n, _, err := ep.Write(data, opts); err != expectedErrWithoutBcastOpt {
+ t.Fatalf("got ep.Write(_, _) = (%d, _, %v), want = (_, _, %v)", n, err, expectedErrWithoutBcastOpt)
+ }
+
+ if err := ep.SetSockOptBool(tcpip.BroadcastOption, true); err != nil {
+ t.Fatalf("got SetSockOptBool(BroadcastOption, true): %s", err)
+ }
+
+ if n, _, err := ep.Write(data, opts); err != nil {
+ t.Fatalf("got ep.Write(_, _) = (%d, _, %s), want = (_, _, nil)", n, err)
+ }
+
+ if err := ep.SetSockOptBool(tcpip.BroadcastOption, false); err != nil {
+ t.Fatalf("got SetSockOptBool(BroadcastOption, false): %s", err)
+ }
+
+ if n, _, err := ep.Write(data, opts); err != expectedErrWithoutBcastOpt {
+ t.Fatalf("got ep.Write(_, _) = (%d, _, %v), want = (_, _, %v)", n, err, expectedErrWithoutBcastOpt)
+ }
+ })
+ }
+}
diff --git a/pkg/test/dockerutil/container.go b/pkg/test/dockerutil/container.go
index 441173ec2..5a2157951 100644
--- a/pkg/test/dockerutil/container.go
+++ b/pkg/test/dockerutil/container.go
@@ -454,15 +454,19 @@ func (c *Container) Wait(ctx context.Context) error {
// WaitTimeout waits for the container to exit with a timeout.
func (c *Container) WaitTimeout(ctx context.Context, timeout time.Duration) error {
- timeoutChan := time.After(timeout)
+ ctx, cancel := context.WithTimeout(ctx, timeout)
+ defer cancel()
statusChan, errChan := c.client.ContainerWait(ctx, c.id, container.WaitConditionNotRunning)
select {
+ case <-ctx.Done():
+ if ctx.Err() == context.DeadlineExceeded {
+ return fmt.Errorf("container %s timed out after %v seconds", c.Name, timeout.Seconds())
+ }
+ return nil
case err := <-errChan:
return err
case <-statusChan:
return nil
- case <-timeoutChan:
- return fmt.Errorf("container %s timed out after %v seconds", c.Name, timeout.Seconds())
}
}
@@ -487,6 +491,12 @@ func (c *Container) WaitForOutputSubmatch(ctx context.Context, pattern string, t
}
for exp := time.Now().Add(timeout); time.Now().Before(exp); {
+ select {
+ case <-ctx.Done():
+ return nil, ctx.Err()
+ default:
+ }
+
c.streams.Conn.SetDeadline(time.Now().Add(50 * time.Millisecond))
_, err := stdcopy.StdCopy(&c.streamBuf, &c.streamBuf, c.streams.Reader)