summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
Diffstat (limited to 'pkg')
-rw-r--r--pkg/control/server/server.go6
-rw-r--r--pkg/errors/linuxerr/BUILD1
-rw-r--r--pkg/errors/linuxerr/linuxerr.go23
-rw-r--r--pkg/errors/linuxerr/linuxerr_test.go61
-rw-r--r--pkg/flipcall/BUILD4
-rw-r--r--pkg/flipcall/flipcall.go7
-rw-r--r--pkg/flipcall/packet_window.go (renamed from pkg/flipcall/packet_window_allocator.go)0
-rw-r--r--pkg/iovec/BUILD18
-rw-r--r--pkg/iovec/iovec.go71
-rw-r--r--pkg/iovec/iovec_test.go120
-rw-r--r--pkg/memutil/BUILD6
-rw-r--r--pkg/memutil/memfd_linux_unsafe.go (renamed from pkg/memutil/memutil_unsafe.go)1
-rw-r--r--pkg/memutil/memutil.go (renamed from pkg/flipcall/packet_window_mmap_arm64.go)15
-rw-r--r--pkg/memutil/mmap.go (renamed from pkg/flipcall/packet_window_mmap_amd64.go)18
-rw-r--r--pkg/p9/BUILD3
-rw-r--r--pkg/p9/handlers.go42
-rw-r--r--pkg/p9/server.go19
-rw-r--r--pkg/sentry/fs/BUILD2
-rw-r--r--pkg/sentry/fs/copy_up.go3
-rw-r--r--pkg/sentry/fs/dirent.go3
-rw-r--r--pkg/sentry/fs/fdpipe/BUILD1
-rw-r--r--pkg/sentry/fs/fdpipe/pipe.go3
-rw-r--r--pkg/sentry/fs/gofer/BUILD1
-rw-r--r--pkg/sentry/fs/gofer/path.go5
-rw-r--r--pkg/sentry/fs/host/BUILD3
-rw-r--r--pkg/sentry/fs/host/socket.go3
-rw-r--r--pkg/sentry/fs/host/socket_iovec.go4
-rw-r--r--pkg/sentry/fs/host/tty.go3
-rw-r--r--pkg/sentry/fs/host/util.go4
-rw-r--r--pkg/sentry/fs/inode_overlay.go7
-rw-r--r--pkg/sentry/fs/inode_overlay_test.go5
-rw-r--r--pkg/sentry/fs/proc/BUILD1
-rw-r--r--pkg/sentry/fs/proc/net.go4
-rw-r--r--pkg/sentry/fs/splice.go5
-rw-r--r--pkg/sentry/fs/user/BUILD1
-rw-r--r--pkg/sentry/fs/user/path.go5
-rw-r--r--pkg/sentry/fsimpl/ext/BUILD2
-rw-r--r--pkg/sentry/fsimpl/ext/ext_test.go8
-rw-r--r--pkg/sentry/fsimpl/fuse/BUILD2
-rw-r--r--pkg/sentry/fsimpl/fuse/connection_test.go4
-rw-r--r--pkg/sentry/fsimpl/fuse/fusefs.go3
-rw-r--r--pkg/sentry/fsimpl/gofer/BUILD1
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go17
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go3
-rw-r--r--pkg/sentry/fsimpl/gofer/host_named_pipe.go3
-rw-r--r--pkg/sentry/fsimpl/gofer/save_restore.go4
-rw-r--r--pkg/sentry/fsimpl/gofer/special_file.go5
-rw-r--r--pkg/sentry/fsimpl/host/BUILD2
-rw-r--r--pkg/sentry/fsimpl/host/host.go5
-rw-r--r--pkg/sentry/fsimpl/host/socket.go3
-rw-r--r--pkg/sentry/fsimpl/host/socket_iovec.go4
-rw-r--r--pkg/sentry/fsimpl/host/tty.go3
-rw-r--r--pkg/sentry/fsimpl/host/util.go4
-rw-r--r--pkg/sentry/fsimpl/kernfs/BUILD2
-rw-r--r--pkg/sentry/fsimpl/kernfs/filesystem.go13
-rw-r--r--pkg/sentry/fsimpl/kernfs/kernfs_test.go5
-rw-r--r--pkg/sentry/fsimpl/overlay/BUILD1
-rw-r--r--pkg/sentry/fsimpl/overlay/copy_up.go3
-rw-r--r--pkg/sentry/fsimpl/overlay/filesystem.go13
-rw-r--r--pkg/sentry/fsimpl/proc/BUILD3
-rw-r--r--pkg/sentry/fsimpl/proc/task_net.go4
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_test.go6
-rw-r--r--pkg/sentry/fsimpl/tmpfs/BUILD1
-rw-r--r--pkg/sentry/fsimpl/tmpfs/pipe_test.go3
-rw-r--r--pkg/sentry/fsimpl/verity/BUILD3
-rw-r--r--pkg/sentry/fsimpl/verity/filesystem.go33
-rw-r--r--pkg/sentry/fsimpl/verity/verity.go13
-rw-r--r--pkg/sentry/fsimpl/verity/verity_test.go16
-rw-r--r--pkg/sentry/hostfd/hostfd_linux.go9
-rw-r--r--pkg/sentry/hostfd/hostfd_unsafe.go17
-rw-r--r--pkg/sentry/kernel/BUILD1
-rw-r--r--pkg/sentry/kernel/fd_table.go4
-rw-r--r--pkg/sentry/kernel/pipe/BUILD1
-rw-r--r--pkg/sentry/kernel/pipe/node_test.go3
-rw-r--r--pkg/sentry/kernel/ptrace.go2
-rw-r--r--pkg/sentry/kernel/sessions.go5
-rw-r--r--pkg/sentry/kernel/task_block.go3
-rw-r--r--pkg/sentry/kernel/task_syscall.go3
-rw-r--r--pkg/sentry/loader/BUILD1
-rw-r--r--pkg/sentry/loader/elf.go3
-rw-r--r--pkg/sentry/mm/BUILD3
-rw-r--r--pkg/sentry/mm/mm_test.go8
-rw-r--r--pkg/sentry/mm/syscalls.go5
-rw-r--r--pkg/sentry/platform/kvm/bluepill_arm64.go23
-rw-r--r--pkg/sentry/platform/kvm/bluepill_arm64_unsafe.go29
-rw-r--r--pkg/sentry/platform/kvm/machine_arm64_unsafe.go13
-rw-r--r--pkg/sentry/socket/hostinet/BUILD1
-rw-r--r--pkg/sentry/socket/hostinet/socket.go194
-rw-r--r--pkg/sentry/socket/netlink/BUILD1
-rw-r--r--pkg/sentry/socket/netlink/socket.go3
-rw-r--r--pkg/sentry/socket/netstack/BUILD1
-rw-r--r--pkg/sentry/socket/netstack/netstack.go5
-rw-r--r--pkg/sentry/socket/socket.go2
-rw-r--r--pkg/sentry/socket/unix/BUILD1
-rw-r--r--pkg/sentry/socket/unix/unix.go5
-rw-r--r--pkg/sentry/socket/unix/unix_vfs2.go3
-rw-r--r--pkg/sentry/state/BUILD2
-rw-r--r--pkg/sentry/state/state.go4
-rw-r--r--pkg/sentry/syscalls/BUILD1
-rw-r--r--pkg/sentry/syscalls/epoll.go3
-rw-r--r--pkg/sentry/syscalls/linux/BUILD1
-rw-r--r--pkg/sentry/syscalls/linux/error.go25
-rw-r--r--pkg/sentry/syscalls/linux/sys_aio.go3
-rw-r--r--pkg/sentry/syscalls/linux/sys_file.go15
-rw-r--r--pkg/sentry/syscalls/linux/sys_poll.go11
-rw-r--r--pkg/sentry/syscalls/linux/sys_prctl.go3
-rw-r--r--pkg/sentry/syscalls/linux/sys_read.go3
-rw-r--r--pkg/sentry/syscalls/linux/sys_sem.go3
-rw-r--r--pkg/sentry/syscalls/linux/sys_signal.go9
-rw-r--r--pkg/sentry/syscalls/linux/sys_socket.go3
-rw-r--r--pkg/sentry/syscalls/linux/sys_time.go7
-rw-r--r--pkg/sentry/syscalls/linux/sys_write.go3
-rw-r--r--pkg/sentry/syscalls/linux/sys_xattr.go3
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/BUILD1
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/epoll.go3
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/poll.go11
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/read_write.go9
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/socket.go3
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/xattr.go3
-rw-r--r--pkg/sentry/usage/memory.go2
-rw-r--r--pkg/sentry/usage/memory_unsafe.go6
-rw-r--r--pkg/sentry/vfs/BUILD2
-rw-r--r--pkg/sentry/vfs/file_description.go5
-rw-r--r--pkg/sentry/vfs/file_description_impl_util_test.go9
-rw-r--r--pkg/sentry/vfs/vfs.go13
-rw-r--r--pkg/sentry/watchdog/watchdog.go7
-rw-r--r--pkg/shim/BUILD10
-rw-r--r--pkg/shim/debug.go48
-rw-r--r--pkg/shim/errors.go59
-rw-r--r--pkg/shim/errors_test.go47
-rw-r--r--pkg/shim/proc/deleted_state.go30
-rw-r--r--pkg/shim/proc/exec.go17
-rw-r--r--pkg/shim/proc/exec_state.go76
-rw-r--r--pkg/shim/proc/init.go84
-rw-r--r--pkg/shim/proc/init_state.go174
-rw-r--r--pkg/shim/proc/proc.go18
-rw-r--r--pkg/shim/runsc/runsc.go87
-rw-r--r--pkg/shim/service.go57
-rw-r--r--pkg/syserr/BUILD1
-rw-r--r--pkg/syserr/syserr.go3
-rw-r--r--pkg/tcpip/BUILD1
-rw-r--r--pkg/tcpip/header/checksum.go62
-rw-r--r--pkg/tcpip/header/checksum_test.go203
-rw-r--r--pkg/tcpip/header/interfaces.go38
-rw-r--r--pkg/tcpip/header/ipv4.go12
-rw-r--r--pkg/tcpip/header/ndp_options.go5
-rw-r--r--pkg/tcpip/header/tcp.go29
-rw-r--r--pkg/tcpip/header/udp.go29
-rw-r--r--pkg/tcpip/link/fdbased/BUILD1
-rw-r--r--pkg/tcpip/link/fdbased/endpoint.go207
-rw-r--r--pkg/tcpip/link/rawfile/rawfile_unsafe.go61
-rw-r--r--pkg/tcpip/network/ipv6/ipv6.go5
-rw-r--r--pkg/tcpip/network/ipv6/ndp.go209
-rw-r--r--pkg/tcpip/network/ipv6/ndp_test.go4
-rw-r--r--pkg/tcpip/stack/BUILD2
-rw-r--r--pkg/tcpip/stack/conntrack.go51
-rw-r--r--pkg/tcpip/stack/iptables_targets.go97
-rw-r--r--pkg/tcpip/stack/ndp_test.go1314
-rw-r--r--pkg/tcpip/stack/tcp.go3
-rw-r--r--pkg/tcpip/stack/transport_demuxer.go22
-rw-r--r--pkg/tcpip/stack/transport_demuxer_test.go50
-rw-r--r--pkg/tcpip/tests/integration/loopback_test.go2
-rw-r--r--pkg/tcpip/transport/packet/endpoint.go4
-rw-r--r--pkg/tcpip/transport/tcp/connect.go45
-rw-r--r--pkg/tcpip/transport/tcp/endpoint.go35
-rw-r--r--pkg/tcpip/transport/tcp/rcv.go1
-rw-r--r--pkg/tcpip/transport/tcp/tcp_test.go93
-rw-r--r--pkg/urpc/urpc.go24
-rw-r--r--pkg/usermem/BUILD1
-rw-r--r--pkg/usermem/usermem_test.go3
170 files changed, 2639 insertions, 1877 deletions
diff --git a/pkg/control/server/server.go b/pkg/control/server/server.go
index eca06791c..889568177 100644
--- a/pkg/control/server/server.go
+++ b/pkg/control/server/server.go
@@ -21,8 +21,8 @@ implementations of the control interface.
package server
import (
- "context"
"os"
+ "time"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sync"
@@ -66,13 +66,13 @@ func (s *Server) Wait() {
// Stop stops the server. Note that this function should only be called once
// and the server should not be used afterwards.
-func (s *Server) Stop(ctx context.Context) {
+func (s *Server) Stop(timeout time.Duration) {
s.socket.Close()
s.Wait()
// This will cause existing clients to be terminated safely. If the
// registered handlers have a Stop callback, it will be called.
- s.server.Stop(ctx)
+ s.server.Stop(timeout)
}
// StartServing starts listening for connect and spawns the main service
diff --git a/pkg/errors/linuxerr/BUILD b/pkg/errors/linuxerr/BUILD
index 8afc9688c..201727780 100644
--- a/pkg/errors/linuxerr/BUILD
+++ b/pkg/errors/linuxerr/BUILD
@@ -9,6 +9,7 @@ go_library(
deps = [
"//pkg/abi/linux/errno",
"//pkg/errors",
+ "@org_golang_x_sys//unix:go_default_library",
],
)
diff --git a/pkg/errors/linuxerr/linuxerr.go b/pkg/errors/linuxerr/linuxerr.go
index 23d9f9f75..9246f2e89 100644
--- a/pkg/errors/linuxerr/linuxerr.go
+++ b/pkg/errors/linuxerr/linuxerr.go
@@ -20,6 +20,7 @@ package linuxerr
import (
"fmt"
+ "golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux/errno"
"gvisor.dev/gvisor/pkg/errors"
)
@@ -27,6 +28,7 @@ import (
const maxErrno uint32 = errno.EHWPOISON + 1
var (
+ NOERROR = errors.New(errno.NOERRNO, "not an error")
EPERM = errors.New(errno.EPERM, "operation not permitted")
ENOENT = errors.New(errno.ENOENT, "no such file or directory")
ESRCH = errors.New(errno.ESRCH, "no such process")
@@ -176,7 +178,7 @@ var errNotValidError = errors.New(errno.Errno(maxErrno), "not a valid error")
// errnos (especially uint32(sycall.Errno)) and *Error.
var errorSlice = []*errors.Error{
// Errno values from include/uapi/asm-generic/errno-base.h.
- errno.NOERRNO: nil,
+ errno.NOERRNO: NOERROR,
errno.EPERM: EPERM,
errno.ENOENT: ENOENT,
errno.ESRCH: ESRCH,
@@ -324,3 +326,22 @@ func ErrorFromErrno(e errno.Errno) *errors.Error {
}
panic(fmt.Sprintf("invalid error requested with errno: %d", e))
}
+
+// Equals compars a linuxerr to a given error
+// TODO(b/34162363): Remove when syserror is removed.
+func Equals(e *errors.Error, err error) bool {
+ if err == nil {
+ return e == NOERROR || e == nil
+ }
+ if e == nil {
+ return err == NOERROR || err == unix.Errno(0)
+ }
+
+ switch err.(type) {
+ case *errors.Error:
+ return e == err
+ case unix.Errno, error:
+ return unix.Errno(e.Errno()) == err
+ }
+ return false
+}
diff --git a/pkg/errors/linuxerr/linuxerr_test.go b/pkg/errors/linuxerr/linuxerr_test.go
index a81dd9560..62743c338 100644
--- a/pkg/errors/linuxerr/linuxerr_test.go
+++ b/pkg/errors/linuxerr/linuxerr_test.go
@@ -16,6 +16,8 @@ package syserror_test
import (
"errors"
+ "io"
+ "io/fs"
"syscall"
"testing"
@@ -243,3 +245,62 @@ func TestSyscallErrnoToErrors(t *testing.T) {
})
}
}
+
+// TestEqualsMethod tests that the Equals method correctly compares syerror,
+// unix.Errno and linuxerr.
+// TODO (b/34162363): Remove this.
+func TestEqualsMethod(t *testing.T) {
+ for _, tc := range []struct {
+ name string
+ linuxErr []*gErrors.Error
+ err []error
+ equal bool
+ }{
+ {
+ name: "compare nil",
+ linuxErr: []*gErrors.Error{nil, linuxerr.NOERROR},
+ err: []error{nil, linuxerr.NOERROR, unix.Errno(0)},
+ equal: true,
+ },
+ {
+ name: "linuxerr nil error not",
+ linuxErr: []*gErrors.Error{nil, linuxerr.NOERROR},
+ err: []error{unix.Errno(1), linuxerr.EPERM, syserror.EACCES},
+ equal: false,
+ },
+ {
+ name: "linuxerr not nil error nil",
+ linuxErr: []*gErrors.Error{linuxerr.ENOENT},
+ err: []error{nil, unix.Errno(0), linuxerr.NOERROR},
+ equal: false,
+ },
+ {
+ name: "equal errors",
+ linuxErr: []*gErrors.Error{linuxerr.ESRCH},
+ err: []error{linuxerr.ESRCH, syserror.ESRCH, unix.Errno(linuxerr.ESRCH.Errno())},
+ equal: true,
+ },
+ {
+ name: "unequal errors",
+ linuxErr: []*gErrors.Error{linuxerr.ENOENT},
+ err: []error{linuxerr.ESRCH, syserror.ESRCH, unix.Errno(linuxerr.ESRCH.Errno())},
+ equal: false,
+ },
+ {
+ name: "other error",
+ linuxErr: []*gErrors.Error{nil, linuxerr.NOERROR, linuxerr.E2BIG, linuxerr.EINVAL},
+ err: []error{fs.ErrInvalid, io.EOF},
+ equal: false,
+ },
+ } {
+ t.Run(tc.name, func(t *testing.T) {
+ for _, le := range tc.linuxErr {
+ for _, e := range tc.err {
+ if linuxerr.Equals(le, e) != tc.equal {
+ t.Fatalf("Expected %t from Equals method for linuxerr: %s %T and error: %s %T", tc.equal, le, le, e, e)
+ }
+ }
+ }
+ })
+ }
+}
diff --git a/pkg/flipcall/BUILD b/pkg/flipcall/BUILD
index 9730b88c1..c810c7946 100644
--- a/pkg/flipcall/BUILD
+++ b/pkg/flipcall/BUILD
@@ -10,9 +10,7 @@ go_library(
"flipcall_unsafe.go",
"futex_linux.go",
"io.go",
- "packet_window_allocator.go",
- "packet_window_mmap_amd64.go",
- "packet_window_mmap_arm64.go",
+ "packet_window.go",
],
visibility = ["//visibility:public"],
deps = [
diff --git a/pkg/flipcall/flipcall.go b/pkg/flipcall/flipcall.go
index 8d8309a73..f0e4ff487 100644
--- a/pkg/flipcall/flipcall.go
+++ b/pkg/flipcall/flipcall.go
@@ -22,6 +22,7 @@ import (
"sync/atomic"
"golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/memutil"
)
// An Endpoint provides the ability to synchronously transfer data and control
@@ -96,9 +97,9 @@ func (ep *Endpoint) Init(side EndpointSide, pwd PacketWindowDescriptor, opts ...
if pwd.Length > math.MaxUint32 {
return fmt.Errorf("packet window size (%d) exceeds maximum (%d)", pwd.Length, math.MaxUint32)
}
- m, e := packetWindowMmap(pwd)
- if e != 0 {
- return fmt.Errorf("failed to mmap packet window: %v", e)
+ m, err := memutil.MapFile(0, uintptr(pwd.Length), unix.PROT_READ|unix.PROT_WRITE, unix.MAP_SHARED, uintptr(pwd.FD), uintptr(pwd.Offset))
+ if err != nil {
+ return fmt.Errorf("failed to mmap packet window: %v", err)
}
ep.packet = m
ep.dataCap = uint32(pwd.Length) - uint32(PacketHeaderBytes)
diff --git a/pkg/flipcall/packet_window_allocator.go b/pkg/flipcall/packet_window.go
index 9122c97b7..9122c97b7 100644
--- a/pkg/flipcall/packet_window_allocator.go
+++ b/pkg/flipcall/packet_window.go
diff --git a/pkg/iovec/BUILD b/pkg/iovec/BUILD
deleted file mode 100644
index f4e9a6af9..000000000
--- a/pkg/iovec/BUILD
+++ /dev/null
@@ -1,18 +0,0 @@
-load("//tools:defs.bzl", "go_library", "go_test")
-
-package(licenses = ["notice"])
-
-go_library(
- name = "iovec",
- srcs = ["iovec.go"],
- visibility = ["//:sandbox"],
- deps = ["@org_golang_x_sys//unix:go_default_library"],
-)
-
-go_test(
- name = "iovec_test",
- size = "small",
- srcs = ["iovec_test.go"],
- library = ":iovec",
- deps = ["@org_golang_x_sys//unix:go_default_library"],
-)
diff --git a/pkg/iovec/iovec.go b/pkg/iovec/iovec.go
deleted file mode 100644
index a281c05b6..000000000
--- a/pkg/iovec/iovec.go
+++ /dev/null
@@ -1,71 +0,0 @@
-// 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.
-
-// +build linux
-
-// Package iovec provides helpers to interact with vectorized I/O on host
-// system.
-package iovec
-
-import (
- "golang.org/x/sys/unix"
-)
-
-// MaxIovs is the maximum number of iovecs host platform can accept.
-var MaxIovs = 1024
-
-// Builder is a builder for slice of unix.Iovec.
-type Builder struct {
- iovec []unix.Iovec
- storage [8]unix.Iovec
-
- // overflow tracks the last buffer when iovec length is at MaxIovs.
- overflow []byte
-}
-
-// Add adds buf to b preparing to be written. Zero-length buf won't be added.
-func (b *Builder) Add(buf []byte) {
- if len(buf) == 0 {
- return
- }
- if b.iovec == nil {
- b.iovec = b.storage[:0]
- }
- if len(b.iovec) >= MaxIovs {
- b.addByAppend(buf)
- return
- }
-
- b.iovec = append(b.iovec, unix.Iovec{Base: &buf[0]})
- b.iovec[len(b.iovec)-1].SetLen(len(buf))
-
- // Keep the last buf if iovec is at max capacity. We will need to append to it
- // for later bufs.
- if len(b.iovec) == MaxIovs {
- n := len(buf)
- b.overflow = buf[:n:n]
- }
-}
-
-func (b *Builder) addByAppend(buf []byte) {
- b.overflow = append(b.overflow, buf...)
- b.iovec[len(b.iovec)-1] = unix.Iovec{Base: &b.overflow[0]}
- b.iovec[len(b.iovec)-1].SetLen(len(b.overflow))
-}
-
-// Build returns the final Iovec slice. The length of returned iovec will not
-// excceed MaxIovs.
-func (b *Builder) Build() []unix.Iovec {
- return b.iovec
-}
diff --git a/pkg/iovec/iovec_test.go b/pkg/iovec/iovec_test.go
deleted file mode 100644
index f6deb4208..000000000
--- a/pkg/iovec/iovec_test.go
+++ /dev/null
@@ -1,120 +0,0 @@
-// 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.
-
-// +build linux
-
-package iovec
-
-import (
- "bytes"
- "fmt"
- "testing"
- "unsafe"
-
- "golang.org/x/sys/unix"
-)
-
-func TestBuilderEmpty(t *testing.T) {
- var builder Builder
- iovecs := builder.Build()
- if got, want := len(iovecs), 0; got != want {
- t.Errorf("len(iovecs) = %d, want %d", got, want)
- }
-}
-
-func TestBuilderBuild(t *testing.T) {
- a := []byte{1, 2}
- b := []byte{3, 4, 5}
-
- var builder Builder
- builder.Add(a)
- builder.Add(b)
- builder.Add(nil) // Nil slice won't be added.
- builder.Add([]byte{}) // Empty slice won't be added.
- iovecs := builder.Build()
-
- if got, want := len(iovecs), 2; got != want {
- t.Fatalf("len(iovecs) = %d, want %d", got, want)
- }
- for i, data := range [][]byte{a, b} {
- if got, want := *iovecs[i].Base, data[0]; got != want {
- t.Fatalf("*iovecs[%d].Base = %d, want %d", i, got, want)
- }
- if got, want := iovecs[i].Len, uint64(len(data)); got != want {
- t.Fatalf("iovecs[%d].Len = %d, want %d", i, got, want)
- }
- }
-}
-
-func TestBuilderBuildMaxIov(t *testing.T) {
- for _, test := range []struct {
- numIov int
- }{
- {
- numIov: MaxIovs - 1,
- },
- {
- numIov: MaxIovs,
- },
- {
- numIov: MaxIovs + 1,
- },
- {
- numIov: MaxIovs + 10,
- },
- } {
- name := fmt.Sprintf("numIov=%v", test.numIov)
- t.Run(name, func(t *testing.T) {
- var data []byte
- var builder Builder
- for i := 0; i < test.numIov; i++ {
- buf := []byte{byte(i)}
- builder.Add(buf)
- data = append(data, buf...)
- }
- iovec := builder.Build()
-
- // Check the expected length of iovec.
- wantNum := test.numIov
- if wantNum > MaxIovs {
- wantNum = MaxIovs
- }
- if got, want := len(iovec), wantNum; got != want {
- t.Errorf("len(iovec) = %d, want %d", got, want)
- }
-
- // Test a real read-write.
- var fds [2]int
- if err := unix.Pipe(fds[:]); err != nil {
- t.Fatalf("Pipe: %v", err)
- }
- defer unix.Close(fds[0])
- defer unix.Close(fds[1])
-
- wrote, _, e := unix.RawSyscall(unix.SYS_WRITEV, uintptr(fds[1]), uintptr(unsafe.Pointer(&iovec[0])), uintptr(len(iovec)))
- if int(wrote) != len(data) || e != 0 {
- t.Fatalf("writev: %v, %v; want %v, 0", wrote, e, len(data))
- }
-
- got := make([]byte, len(data))
- if n, err := unix.Read(fds[0], got); n != len(got) || err != nil {
- t.Fatalf("read: %v, %v; want %v, nil", n, err, len(got))
- }
-
- if !bytes.Equal(got, data) {
- t.Errorf("read: got data %v, want %v", got, data)
- }
- })
- }
-}
diff --git a/pkg/memutil/BUILD b/pkg/memutil/BUILD
index 9d07d98b4..bea595286 100644
--- a/pkg/memutil/BUILD
+++ b/pkg/memutil/BUILD
@@ -4,7 +4,11 @@ package(licenses = ["notice"])
go_library(
name = "memutil",
- srcs = ["memutil_unsafe.go"],
+ srcs = [
+ "memfd_linux_unsafe.go",
+ "memutil.go",
+ "mmap.go",
+ ],
visibility = ["//visibility:public"],
deps = ["@org_golang_x_sys//unix:go_default_library"],
)
diff --git a/pkg/memutil/memutil_unsafe.go b/pkg/memutil/memfd_linux_unsafe.go
index 6676d1ce3..504382213 100644
--- a/pkg/memutil/memutil_unsafe.go
+++ b/pkg/memutil/memfd_linux_unsafe.go
@@ -14,7 +14,6 @@
// +build linux
-// Package memutil provides a wrapper for the memfd_create() system call.
package memutil
import (
diff --git a/pkg/flipcall/packet_window_mmap_arm64.go b/pkg/memutil/memutil.go
index 87ad1a4a1..3185882fd 100644
--- a/pkg/flipcall/packet_window_mmap_arm64.go
+++ b/pkg/memutil/memutil.go
@@ -1,4 +1,4 @@
-// Copyright 2020 The gVisor Authors.
+// Copyright 2018 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.
@@ -12,14 +12,5 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-// +build arm64
-
-package flipcall
-
-import "golang.org/x/sys/unix"
-
-// Return a memory mapping of the pwd in memory that can be shared outside the sandbox.
-func packetWindowMmap(pwd PacketWindowDescriptor) (uintptr, unix.Errno) {
- m, _, err := unix.RawSyscall6(unix.SYS_MMAP, 0, uintptr(pwd.Length), unix.PROT_READ|unix.PROT_WRITE, unix.MAP_SHARED, uintptr(pwd.FD), uintptr(pwd.Offset))
- return m, err
-}
+// Package memutil provides utilities for working with shared memory files.
+package memutil
diff --git a/pkg/flipcall/packet_window_mmap_amd64.go b/pkg/memutil/mmap.go
index ced587a2a..7c939293f 100644
--- a/pkg/flipcall/packet_window_mmap_amd64.go
+++ b/pkg/memutil/mmap.go
@@ -12,12 +12,18 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-package flipcall
+package memutil
-import "golang.org/x/sys/unix"
+import (
+ "golang.org/x/sys/unix"
+)
-// Return a memory mapping of the pwd in memory that can be shared outside the sandbox.
-func packetWindowMmap(pwd PacketWindowDescriptor) (uintptr, unix.Errno) {
- m, _, err := unix.RawSyscall6(unix.SYS_MMAP, 0, uintptr(pwd.Length), unix.PROT_READ|unix.PROT_WRITE, unix.MAP_SHARED, uintptr(pwd.FD), uintptr(pwd.Offset))
- return m, err
+// MapFile returns a memory mapping configured by the given options as per
+// mmap(2).
+func MapFile(addr, len, prot, flags, fd, offset uintptr) (uintptr, error) {
+ m, _, e := unix.RawSyscall6(unix.SYS_MMAP, addr, len, prot, flags, fd, offset)
+ if e != 0 {
+ return 0, e
+ }
+ return m, nil
}
diff --git a/pkg/p9/BUILD b/pkg/p9/BUILD
index b2291ef97..2b22b2203 100644
--- a/pkg/p9/BUILD
+++ b/pkg/p9/BUILD
@@ -22,6 +22,9 @@ go_library(
"version.go",
],
deps = [
+ "//pkg/abi/linux/errno",
+ "//pkg/errors",
+ "//pkg/errors/linuxerr",
"//pkg/fd",
"//pkg/fdchannel",
"//pkg/flipcall",
diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go
index 758e11b13..161b451cc 100644
--- a/pkg/p9/handlers.go
+++ b/pkg/p9/handlers.go
@@ -23,6 +23,9 @@ import (
"sync/atomic"
"golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/abi/linux/errno"
+ "gvisor.dev/gvisor/pkg/errors"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fd"
"gvisor.dev/gvisor/pkg/log"
)
@@ -62,6 +65,45 @@ func newErr(err error) *Rlerror {
return &Rlerror{Error: uint32(ExtractErrno(err))}
}
+// ExtractLinuxerrErrno extracts a *errors.Error from a error, best effort.
+// TODO(b/34162363): Merge this with ExtractErrno.
+func ExtractLinuxerrErrno(err error) *errors.Error {
+ switch err {
+ case os.ErrNotExist:
+ return linuxerr.ENOENT
+ case os.ErrExist:
+ return linuxerr.EEXIST
+ case os.ErrPermission:
+ return linuxerr.EACCES
+ case os.ErrInvalid:
+ return linuxerr.EINVAL
+ }
+
+ // Attempt to unwrap.
+ switch e := err.(type) {
+ case *errors.Error:
+ return e
+ case unix.Errno:
+ return linuxerr.ErrorFromErrno(errno.Errno(e))
+ case *os.PathError:
+ return ExtractLinuxerrErrno(e.Err)
+ case *os.SyscallError:
+ return ExtractLinuxerrErrno(e.Err)
+ case *os.LinkError:
+ return ExtractLinuxerrErrno(e.Err)
+ }
+
+ // Default case.
+ log.Warningf("unknown error: %v", err)
+ return linuxerr.EIO
+}
+
+// newErrFromLinuxerr returns an Rlerror from the linuxerr list.
+// TODO(b/34162363): Merge this with newErr.
+func newErrFromLinuxerr(err error) *Rlerror {
+ return &Rlerror{Error: uint32(ExtractLinuxerrErrno(err).Errno())}
+}
+
// handler is implemented for server-handled messages.
//
// See server.go for call information.
diff --git a/pkg/p9/server.go b/pkg/p9/server.go
index ff1172ed6..241ab44ef 100644
--- a/pkg/p9/server.go
+++ b/pkg/p9/server.go
@@ -19,7 +19,8 @@ import (
"runtime/debug"
"sync/atomic"
- "golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/abi/linux/errno"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fd"
"gvisor.dev/gvisor/pkg/fdchannel"
"gvisor.dev/gvisor/pkg/flipcall"
@@ -483,7 +484,7 @@ func (cs *connState) lookupChannel(id uint32) *channel {
func (cs *connState) handle(m message) (r message) {
if !cs.reqGate.Enter() {
// connState.stop() has been called; the connection is shutting down.
- r = newErr(unix.ECONNRESET)
+ r = newErrFromLinuxerr(linuxerr.ECONNRESET)
return
}
defer func() {
@@ -498,15 +499,23 @@ func (cs *connState) handle(m message) (r message) {
// Wrap in an EFAULT error; we don't really have a
// better way to describe this kind of error. It will
// usually manifest as a result of the test framework.
- r = newErr(unix.EFAULT)
+ r = newErrFromLinuxerr(linuxerr.EFAULT)
}
}()
if handler, ok := m.(handler); ok {
// Call the message handler.
r = handler.handle(cs)
+ // TODO(b/34162363):This is only here to make sure the server works with
+ // only linuxerr Errors, as the handlers work with both client and server.
+ // It will be removed a followup, when all the unix.Errno errors are
+ // replaced with linuxerr.
+ if rlError, ok := r.(*Rlerror); ok {
+ e := linuxerr.ErrorFromErrno(errno.Errno(rlError.Error))
+ r = newErrFromLinuxerr(e)
+ }
} else {
// Produce an ENOSYS error.
- r = newErr(unix.ENOSYS)
+ r = newErrFromLinuxerr(linuxerr.ENOSYS)
}
return
}
@@ -553,7 +562,7 @@ func (cs *connState) handleRequest() bool {
// If it's not a connection error, but some other protocol error,
// we can send a response immediately.
cs.sendMu.Lock()
- err := send(cs.conn, tag, newErr(err))
+ err := send(cs.conn, tag, newErrFromLinuxerr(err))
cs.sendMu.Unlock()
if err != nil {
log.Debugf("p9.send: %v", err)
diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD
index 0dc100f9b..74adbfa55 100644
--- a/pkg/sentry/fs/BUILD
+++ b/pkg/sentry/fs/BUILD
@@ -48,6 +48,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/amutex",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/log",
"//pkg/p9",
@@ -110,6 +111,7 @@ go_test(
deps = [
":fs",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/sentry/fs/fsutil",
"//pkg/sentry/fs/ramfs",
"//pkg/sentry/fs/tmpfs",
diff --git a/pkg/sentry/fs/copy_up.go b/pkg/sentry/fs/copy_up.go
index 5aa668873..ae282d14e 100644
--- a/pkg/sentry/fs/copy_up.go
+++ b/pkg/sentry/fs/copy_up.go
@@ -20,6 +20,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/memmap"
@@ -410,7 +411,7 @@ func copyAttributesLocked(ctx context.Context, upper *Inode, lower *Inode) error
return err
}
lowerXattr, err := lower.ListXattr(ctx, linux.XATTR_SIZE_MAX)
- if err != nil && err != syserror.EOPNOTSUPP {
+ if err != nil && !linuxerr.Equals(linuxerr.EOPNOTSUPP, err) {
return err
}
diff --git a/pkg/sentry/fs/dirent.go b/pkg/sentry/fs/dirent.go
index 9d5d40954..e45749be6 100644
--- a/pkg/sentry/fs/dirent.go
+++ b/pkg/sentry/fs/dirent.go
@@ -22,6 +22,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/refs"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
@@ -1439,7 +1440,7 @@ func Rename(ctx context.Context, root *Dirent, oldParent *Dirent, oldName string
// replaced is the dirent that is being overwritten by rename.
replaced, err := newParent.walk(ctx, root, newName, false /* may unlock */)
if err != nil {
- if err != syserror.ENOENT {
+ if !linuxerr.Equals(linuxerr.ENOENT, err) {
return err
}
diff --git a/pkg/sentry/fs/fdpipe/BUILD b/pkg/sentry/fs/fdpipe/BUILD
index 2120f2bad..7fc53ed22 100644
--- a/pkg/sentry/fs/fdpipe/BUILD
+++ b/pkg/sentry/fs/fdpipe/BUILD
@@ -13,6 +13,7 @@ go_library(
visibility = ["//pkg/sentry:internal"],
deps = [
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fd",
"//pkg/fdnotifier",
"//pkg/log",
diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go
index 757b7d511..f8a29816b 100644
--- a/pkg/sentry/fs/fdpipe/pipe.go
+++ b/pkg/sentry/fs/fdpipe/pipe.go
@@ -20,6 +20,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fd"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/log"
@@ -158,7 +159,7 @@ func (p *pipeOperations) Write(ctx context.Context, file *fs.File, src usermem.I
// isBlockError unwraps os errors and checks if they are caused by EAGAIN or
// EWOULDBLOCK. This is so they can be transformed into syserror.ErrWouldBlock.
func isBlockError(err error) bool {
- if err == syserror.EAGAIN || err == syserror.EWOULDBLOCK {
+ if linuxerr.Equals(linuxerr.EAGAIN, err) || linuxerr.Equals(linuxerr.EWOULDBLOCK, err) {
return true
}
if pe, ok := err.(*os.PathError); ok {
diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD
index 94cb05246..c08301d19 100644
--- a/pkg/sentry/fs/gofer/BUILD
+++ b/pkg/sentry/fs/gofer/BUILD
@@ -26,6 +26,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fd",
"//pkg/hostarch",
"//pkg/log",
diff --git a/pkg/sentry/fs/gofer/path.go b/pkg/sentry/fs/gofer/path.go
index 940838a44..1a6f353d0 100644
--- a/pkg/sentry/fs/gofer/path.go
+++ b/pkg/sentry/fs/gofer/path.go
@@ -18,6 +18,7 @@ import (
"fmt"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/device"
@@ -66,7 +67,7 @@ func (i *inodeOperations) Lookup(ctx context.Context, dir *fs.Inode, name string
// Get a p9.File for name.
qids, newFile, mask, p9attr, err := i.fileState.file.walkGetAttr(ctx, []string{name})
if err != nil {
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
if cp.cacheNegativeDirents() {
// Return a negative Dirent. It will stay cached until something
// is created over it.
@@ -298,7 +299,7 @@ func (i *inodeOperations) CreateFifo(ctx context.Context, dir *fs.Inode, name st
// N.B. FIFOs use major/minor numbers 0.
if _, err := i.fileState.file.mknod(ctx, name, mode, 0, 0, p9.UID(owner.UID), p9.GID(owner.GID)); err != nil {
- if i.session().overrides == nil || err != syserror.EPERM {
+ if i.session().overrides == nil || !linuxerr.Equals(linuxerr.EPERM, err) {
return err
}
// If gofer doesn't support mknod, check if we can create an internal fifo.
diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD
index 3c45f6cc5..24fc6305c 100644
--- a/pkg/sentry/fs/host/BUILD
+++ b/pkg/sentry/fs/host/BUILD
@@ -28,9 +28,9 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fd",
"//pkg/fdnotifier",
- "//pkg/iovec",
"//pkg/log",
"//pkg/marshal/primitive",
"//pkg/refs",
@@ -40,6 +40,7 @@ go_library(
"//pkg/sentry/device",
"//pkg/sentry/fs",
"//pkg/sentry/fs/fsutil",
+ "//pkg/sentry/hostfd",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/kernel/time",
diff --git a/pkg/sentry/fs/host/socket.go b/pkg/sentry/fs/host/socket.go
index 46a2dc47d..225244868 100644
--- a/pkg/sentry/fs/host/socket.go
+++ b/pkg/sentry/fs/host/socket.go
@@ -21,6 +21,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fd"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/refs"
@@ -213,7 +214,7 @@ func (c *ConnectedEndpoint) Send(ctx context.Context, data [][]byte, controlMess
// block (and only for stream sockets).
err = syserror.EAGAIN
}
- if n > 0 && err != syserror.EAGAIN {
+ if n > 0 && !linuxerr.Equals(linuxerr.EAGAIN, err) {
// The caller may need to block to send more data, but
// otherwise there isn't anything that can be done about an
// error with a partial write.
diff --git a/pkg/sentry/fs/host/socket_iovec.go b/pkg/sentry/fs/host/socket_iovec.go
index 7380d75e7..fd48aff11 100644
--- a/pkg/sentry/fs/host/socket_iovec.go
+++ b/pkg/sentry/fs/host/socket_iovec.go
@@ -16,7 +16,7 @@ package host
import (
"golang.org/x/sys/unix"
- "gvisor.dev/gvisor/pkg/iovec"
+ "gvisor.dev/gvisor/pkg/sentry/hostfd"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -72,7 +72,7 @@ func buildIovec(bufs [][]byte, maxlen int64, truncate bool) (length int64, iovec
}
}
- if iovsRequired > iovec.MaxIovs {
+ if iovsRequired > hostfd.MaxSendRecvMsgIov {
// The kernel will reject our call if we pass this many iovs.
// Use a single intermediate buffer instead.
b := make([]byte, stopLen)
diff --git a/pkg/sentry/fs/host/tty.go b/pkg/sentry/fs/host/tty.go
index 1183727ab..77613bfd5 100644
--- a/pkg/sentry/fs/host/tty.go
+++ b/pkg/sentry/fs/host/tty.go
@@ -17,6 +17,7 @@ package host
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -191,7 +192,7 @@ func (t *TTYFileOperations) Ioctl(ctx context.Context, _ *fs.File, io usermem.IO
if err := t.checkChange(ctx, linux.SIGTTOU); err != nil {
// drivers/tty/tty_io.c:tiocspgrp() converts -EIO from
// tty_check_change() to -ENOTTY.
- if err == syserror.EIO {
+ if linuxerr.Equals(linuxerr.EIO, err) {
return 0, syserror.ENOTTY
}
return 0, err
diff --git a/pkg/sentry/fs/host/util.go b/pkg/sentry/fs/host/util.go
index ab74724a3..e7db79189 100644
--- a/pkg/sentry/fs/host/util.go
+++ b/pkg/sentry/fs/host/util.go
@@ -19,12 +19,12 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/device"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
- "gvisor.dev/gvisor/pkg/syserror"
)
func nodeType(s *unix.Stat_t) fs.InodeType {
@@ -98,7 +98,7 @@ type dirInfo struct {
// isBlockError unwraps os errors and checks if they are caused by EAGAIN or
// EWOULDBLOCK. This is so they can be transformed into syserror.ErrWouldBlock.
func isBlockError(err error) bool {
- if err == syserror.EAGAIN || err == syserror.EWOULDBLOCK {
+ if linuxerr.Equals(linuxerr.EAGAIN, err) || linuxerr.Equals(linuxerr.EWOULDBLOCK, err) {
return true
}
if pe, ok := err.(*os.PathError); ok {
diff --git a/pkg/sentry/fs/inode_overlay.go b/pkg/sentry/fs/inode_overlay.go
index e97afc626..bd1125dcc 100644
--- a/pkg/sentry/fs/inode_overlay.go
+++ b/pkg/sentry/fs/inode_overlay.go
@@ -19,6 +19,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/syserror"
@@ -71,7 +72,7 @@ func overlayLookup(ctx context.Context, parent *overlayEntry, inode *Inode, name
// A file could have been created over a whiteout, so we need to
// check if something exists in the upper file system first.
child, err := parent.upper.Lookup(ctx, name)
- if err != nil && err != syserror.ENOENT {
+ if err != nil && !linuxerr.Equals(linuxerr.ENOENT, err) {
// We encountered an error that an overlay cannot handle,
// we must propagate it to the caller.
parent.copyMu.RUnlock()
@@ -125,7 +126,7 @@ func overlayLookup(ctx context.Context, parent *overlayEntry, inode *Inode, name
// Check the lower file system.
child, err := parent.lower.Lookup(ctx, name)
// Same song and dance as above.
- if err != nil && err != syserror.ENOENT {
+ if err != nil && !linuxerr.Equals(linuxerr.ENOENT, err) {
// Don't leak resources.
if upperInode != nil {
upperInode.DecRef(ctx)
@@ -396,7 +397,7 @@ func overlayRename(ctx context.Context, o *overlayEntry, oldParent *Dirent, rena
// newName has been removed out from under us. That's fine;
// filesystems where that can happen must handle stale
// 'replaced'.
- if err != nil && err != syserror.ENOENT {
+ if err != nil && !linuxerr.Equals(linuxerr.ENOENT, err) {
return err
}
if err == nil {
diff --git a/pkg/sentry/fs/inode_overlay_test.go b/pkg/sentry/fs/inode_overlay_test.go
index aa9851b26..cc5ffa6f1 100644
--- a/pkg/sentry/fs/inode_overlay_test.go
+++ b/pkg/sentry/fs/inode_overlay_test.go
@@ -18,6 +18,7 @@ import (
"testing"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
"gvisor.dev/gvisor/pkg/sentry/fs/ramfs"
@@ -191,11 +192,11 @@ func TestLookup(t *testing.T) {
} {
t.Run(test.desc, func(t *testing.T) {
dirent, err := test.dir.Lookup(ctx, test.name)
- if test.found && (err == syserror.ENOENT || dirent.IsNegative()) {
+ if test.found && (linuxerr.Equals(linuxerr.ENOENT, err) || dirent.IsNegative()) {
t.Fatalf("lookup %q expected to find positive dirent, got dirent %v err %v", test.name, dirent, err)
}
if !test.found {
- if err != syserror.ENOENT && !dirent.IsNegative() {
+ if !linuxerr.Equals(linuxerr.ENOENT, err) && !dirent.IsNegative() {
t.Errorf("lookup %q expected to return ENOENT or negative dirent, got dirent %v err %v", test.name, dirent, err)
}
// Nothing more to check.
diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD
index 7af7e0b45..e6d74b949 100644
--- a/pkg/sentry/fs/proc/BUILD
+++ b/pkg/sentry/fs/proc/BUILD
@@ -30,6 +30,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/log",
"//pkg/sentry/fs",
diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go
index 91c35eea9..187e9a921 100644
--- a/pkg/sentry/fs/proc/net.go
+++ b/pkg/sentry/fs/proc/net.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -34,7 +35,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/socket"
"gvisor.dev/gvisor/pkg/sentry/socket/unix"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
- "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/tcpip/header"
)
@@ -291,7 +291,7 @@ func (n *netSnmp) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]s
continue
}
if err := n.s.Statistics(stat, line.prefix); err != nil {
- if err == syserror.EOPNOTSUPP {
+ if linuxerr.Equals(linuxerr.EOPNOTSUPP, err) {
log.Infof("Failed to retrieve %s of /proc/net/snmp: %v", line.prefix, err)
} else {
log.Warningf("Failed to retrieve %s of /proc/net/snmp: %v", line.prefix, err)
diff --git a/pkg/sentry/fs/splice.go b/pkg/sentry/fs/splice.go
index 33da82868..ca9f645f6 100644
--- a/pkg/sentry/fs/splice.go
+++ b/pkg/sentry/fs/splice.go
@@ -19,6 +19,7 @@ import (
"sync/atomic"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -139,7 +140,7 @@ func Splice(ctx context.Context, dst *File, src *File, opts SpliceOpts) (int64,
// Attempt to do a WriteTo; this is likely the most efficient.
n, err := src.FileOperations.WriteTo(ctx, src, w, opts.Length, opts.Dup)
- if n == 0 && err == syserror.ENOSYS && !opts.Dup {
+ if n == 0 && linuxerr.Equals(linuxerr.ENOSYS, err) && !opts.Dup {
// Attempt as a ReadFrom. If a WriteTo, a ReadFrom may also be
// more efficient than a copy if buffers are cached or readily
// available. (It's unlikely that they can actually be donated).
@@ -151,7 +152,7 @@ func Splice(ctx context.Context, dst *File, src *File, opts SpliceOpts) (int64,
// if we block at some point, we could lose data. If the source is
// not a pipe then reading is not destructive; if the destination
// is a regular file, then it is guaranteed not to block writing.
- if n == 0 && err == syserror.ENOSYS && !opts.Dup && (!dstPipe || !srcPipe) {
+ if n == 0 && linuxerr.Equals(linuxerr.ENOSYS, err) && !opts.Dup && (!dstPipe || !srcPipe) {
// Fallback to an in-kernel copy.
n, err = io.Copy(w, &io.LimitedReader{
R: r,
diff --git a/pkg/sentry/fs/user/BUILD b/pkg/sentry/fs/user/BUILD
index 66e949c95..4acc73ee0 100644
--- a/pkg/sentry/fs/user/BUILD
+++ b/pkg/sentry/fs/user/BUILD
@@ -12,6 +12,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/log",
"//pkg/sentry/fs",
diff --git a/pkg/sentry/fs/user/path.go b/pkg/sentry/fs/user/path.go
index 124bc95ed..f6eaab2bd 100644
--- a/pkg/sentry/fs/user/path.go
+++ b/pkg/sentry/fs/user/path.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -93,7 +94,7 @@ func resolve(ctx context.Context, mns *fs.MountNamespace, paths []string, name s
binPath := path.Join(p, name)
traversals := uint(linux.MaxSymlinkTraversals)
d, err := mns.FindInode(ctx, root, nil, binPath, &traversals)
- if err == syserror.ENOENT || err == syserror.EACCES {
+ if linuxerr.Equals(linuxerr.ENOENT, err) || linuxerr.Equals(linuxerr.EACCES, err) {
// Didn't find it here.
continue
}
@@ -142,7 +143,7 @@ func resolveVFS2(ctx context.Context, creds *auth.Credentials, mns *vfs.MountNam
Flags: linux.O_RDONLY,
}
dentry, err := root.Mount().Filesystem().VirtualFilesystem().OpenAt(ctx, creds, pop, opts)
- if err == syserror.ENOENT || err == syserror.EACCES {
+ if linuxerr.Equals(linuxerr.ENOENT, err) || linuxerr.Equals(linuxerr.EACCES, err) {
// Didn't find it here.
continue
}
diff --git a/pkg/sentry/fsimpl/ext/BUILD b/pkg/sentry/fsimpl/ext/BUILD
index 2dbc6bfd5..1060b5301 100644
--- a/pkg/sentry/fsimpl/ext/BUILD
+++ b/pkg/sentry/fsimpl/ext/BUILD
@@ -88,13 +88,13 @@ go_test(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/marshal/primitive",
"//pkg/sentry/contexttest",
"//pkg/sentry/fsimpl/ext/disklayout",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/vfs",
- "//pkg/syserror",
"//pkg/test/testutil",
"//pkg/usermem",
"@com_github_google_go_cmp//cmp:go_default_library",
diff --git a/pkg/sentry/fsimpl/ext/ext_test.go b/pkg/sentry/fsimpl/ext/ext_test.go
index d9fd4590c..db712e71f 100644
--- a/pkg/sentry/fsimpl/ext/ext_test.go
+++ b/pkg/sentry/fsimpl/ext/ext_test.go
@@ -26,12 +26,12 @@ import (
"github.com/google/go-cmp/cmp/cmpopts"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/contexttest"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/ext/disklayout"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/vfs"
- "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/test/testutil"
"gvisor.dev/gvisor/pkg/usermem"
)
@@ -173,7 +173,7 @@ func TestSeek(t *testing.T) {
}
// EINVAL should be returned if the resulting offset is negative.
- if _, err := fd.Seek(ctx, -1, linux.SEEK_SET); err != syserror.EINVAL {
+ if _, err := fd.Seek(ctx, -1, linux.SEEK_SET); !linuxerr.Equals(linuxerr.EINVAL, err) {
t.Errorf("expected error EINVAL but got %v", err)
}
@@ -187,7 +187,7 @@ func TestSeek(t *testing.T) {
}
// EINVAL should be returned if the resulting offset is negative.
- if _, err := fd.Seek(ctx, -(size + 2), linux.SEEK_CUR); err != syserror.EINVAL {
+ if _, err := fd.Seek(ctx, -(size + 2), linux.SEEK_CUR); !linuxerr.Equals(linuxerr.EINVAL, err) {
t.Errorf("expected error EINVAL but got %v", err)
}
@@ -204,7 +204,7 @@ func TestSeek(t *testing.T) {
}
// EINVAL should be returned if the resulting offset is negative.
- if _, err := fd.Seek(ctx, -(size + 1), linux.SEEK_END); err != syserror.EINVAL {
+ if _, err := fd.Seek(ctx, -(size + 1), linux.SEEK_END); !linuxerr.Equals(linuxerr.EINVAL, err) {
t.Errorf("expected error EINVAL but got %v", err)
}
}
diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD
index 3a4777fbe..871df5984 100644
--- a/pkg/sentry/fsimpl/fuse/BUILD
+++ b/pkg/sentry/fsimpl/fuse/BUILD
@@ -46,6 +46,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/log",
"//pkg/marshal",
@@ -76,6 +77,7 @@ go_test(
library = ":fuse",
deps = [
"//pkg/abi/linux",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/marshal",
"//pkg/sentry/fsimpl/testutil",
diff --git a/pkg/sentry/fsimpl/fuse/connection_test.go b/pkg/sentry/fsimpl/fuse/connection_test.go
index 78ea6a31e..1fddd858e 100644
--- a/pkg/sentry/fsimpl/fuse/connection_test.go
+++ b/pkg/sentry/fsimpl/fuse/connection_test.go
@@ -19,9 +19,9 @@ import (
"testing"
"golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
- "gvisor.dev/gvisor/pkg/syserror"
)
// TestConnectionInitBlock tests if initialization
@@ -104,7 +104,7 @@ func TestConnectionAbort(t *testing.T) {
// After abort, Call() should return directly with ENOTCONN.
req := conn.NewRequest(creds, 0, 0, 0, testObj)
_, err = conn.Call(task, req)
- if err != syserror.ENOTCONN {
+ if !linuxerr.Equals(linuxerr.ENOTCONN, err) {
t.Fatalf("Incorrect error code received for Call() after connection aborted")
}
diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go
index 167c899e2..47794810c 100644
--- a/pkg/sentry/fsimpl/fuse/fusefs.go
+++ b/pkg/sentry/fsimpl/fuse/fusefs.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/marshal"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
@@ -440,7 +441,7 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, d *kernfs.Dentr
if err != nil {
return nil, err
}
- if err := res.Error(); err == syserror.ENOSYS && !isDir {
+ if err := res.Error(); linuxerr.Equals(linuxerr.ENOSYS, err) && !isDir {
i.fs.conn.noOpen = true
} else if err != nil {
return nil, err
diff --git a/pkg/sentry/fsimpl/gofer/BUILD b/pkg/sentry/fsimpl/gofer/BUILD
index 368272f12..752060044 100644
--- a/pkg/sentry/fsimpl/gofer/BUILD
+++ b/pkg/sentry/fsimpl/gofer/BUILD
@@ -49,6 +49,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fd",
"//pkg/fdnotifier",
"//pkg/fspath",
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index eb09d54c3..af2b773c3 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/host"
@@ -255,7 +256,7 @@ func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name s
qid, file, attrMask, attr, err := parent.file.walkGetAttrOne(ctx, name)
if err != nil {
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
parent.cacheNegativeLookupLocked(name)
}
return nil, err
@@ -382,7 +383,7 @@ func (fs *filesystem) doCreateAt(ctx context.Context, rp *vfs.ResolvingPath, dir
return syserror.EEXIST
}
checkExistence := func() error {
- if child, err := fs.getChildLocked(ctx, parent, name, &ds); err != nil && err != syserror.ENOENT {
+ if child, err := fs.getChildLocked(ctx, parent, name, &ds); err != nil && !linuxerr.Equals(linuxerr.ENOENT, err) {
return err
} else if child != nil {
return syserror.EEXIST
@@ -715,7 +716,7 @@ func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
mode |= linux.S_ISGID
}
if _, err := parent.file.mkdir(ctx, name, p9.FileMode(mode), (p9.UID)(creds.EffectiveKUID), p9.GID(kgid)); err != nil {
- if !opts.ForSyntheticMountpoint || err == syserror.EEXIST {
+ if !opts.ForSyntheticMountpoint || linuxerr.Equals(linuxerr.EEXIST, err) {
return err
}
ctx.Infof("Failed to create remote directory %q: %v; falling back to synthetic directory", name, err)
@@ -752,7 +753,7 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, name string, ds **[]*dentry) error {
creds := rp.Credentials()
_, err := parent.file.mknod(ctx, name, (p9.FileMode)(opts.Mode), opts.DevMajor, opts.DevMinor, (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
- if err != syserror.EPERM {
+ if !linuxerr.Equals(linuxerr.EPERM, err) {
return err
}
@@ -765,7 +766,7 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
case err == nil:
// Step succeeded, another file exists.
return syserror.EEXIST
- case err != syserror.ENOENT:
+ case !linuxerr.Equals(linuxerr.ENOENT, err):
// Unexpected error.
return err
}
@@ -862,7 +863,7 @@ afterTrailingSymlink:
// Determine whether or not we need to create a file.
parent.dirMu.Lock()
child, _, err := fs.stepLocked(ctx, rp, parent, false /* mayFollowSymlinks */, &ds)
- if err == syserror.ENOENT && mayCreate {
+ if linuxerr.Equals(linuxerr.ENOENT, err) && mayCreate {
if parent.isSynthetic() {
parent.dirMu.Unlock()
return nil, syserror.EPERM
@@ -1033,7 +1034,7 @@ func (d *dentry) openSpecialFile(ctx context.Context, mnt *vfs.Mount, opts *vfs.
retry:
h, err := openHandle(ctx, d.file, ats.MayRead(), ats.MayWrite(), opts.Flags&linux.O_TRUNC != 0)
if err != nil {
- if isBlockingOpenOfNamedPipe && ats == vfs.MayWrite && err == syserror.ENXIO {
+ if isBlockingOpenOfNamedPipe && ats == vfs.MayWrite && linuxerr.Equals(linuxerr.ENXIO, err) {
// An attempt to open a named pipe with O_WRONLY|O_NONBLOCK fails
// with ENXIO if opening the same named pipe with O_WRONLY would
// block because there are no readers of the pipe.
@@ -1284,7 +1285,7 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
return syserror.ENOENT
}
replaced, err := fs.getChildLocked(ctx, newParent, newName, &ds)
- if err != nil && err != syserror.ENOENT {
+ if err != nil && !linuxerr.Equals(linuxerr.ENOENT, err) {
return err
}
var replacedVFSD *vfs.Dentry
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go
index cf69e1b7a..496e31e34 100644
--- a/pkg/sentry/fsimpl/gofer/gofer.go
+++ b/pkg/sentry/fsimpl/gofer/gofer.go
@@ -46,6 +46,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/p9"
@@ -1763,7 +1764,7 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
openReadable := !d.readFile.isNil() || read
openWritable := !d.writeFile.isNil() || write
h, err := openHandle(ctx, d.file, openReadable, openWritable, trunc)
- if err == syserror.EACCES && (openReadable != read || openWritable != write) {
+ if linuxerr.Equals(linuxerr.EACCES, err) && (openReadable != read || openWritable != write) {
// It may not be possible to use a single handle for both
// reading and writing, since permissions on the file may have
// changed to e.g. disallow reading after previously being
diff --git a/pkg/sentry/fsimpl/gofer/host_named_pipe.go b/pkg/sentry/fsimpl/gofer/host_named_pipe.go
index c7bf10007..398288ee3 100644
--- a/pkg/sentry/fsimpl/gofer/host_named_pipe.go
+++ b/pkg/sentry/fsimpl/gofer/host_named_pipe.go
@@ -21,6 +21,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -78,7 +79,7 @@ func nonblockingPipeHasWriter(fd int32) (bool, error) {
defer tempPipeMu.Unlock()
// Copy 1 byte from fd into the temporary pipe.
n, err := unix.Tee(int(fd), tempPipeWriteFD, 1, unix.SPLICE_F_NONBLOCK)
- if err == syserror.EAGAIN {
+ if linuxerr.Equals(linuxerr.EAGAIN, err) {
// The pipe represented by fd is empty, but has a writer.
return true, nil
}
diff --git a/pkg/sentry/fsimpl/gofer/save_restore.go b/pkg/sentry/fsimpl/gofer/save_restore.go
index 83e841a51..e67422a2f 100644
--- a/pkg/sentry/fsimpl/gofer/save_restore.go
+++ b/pkg/sentry/fsimpl/gofer/save_restore.go
@@ -21,13 +21,13 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/refsvfs2"
"gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/sentry/vfs"
- "gvisor.dev/gvisor/pkg/syserror"
)
type saveRestoreContextID int
@@ -92,7 +92,7 @@ func (fd *specialFileFD) savePipeData(ctx context.Context) error {
fd.buf = append(fd.buf, buf[:n]...)
}
if err != nil {
- if err == io.EOF || err == syserror.EAGAIN {
+ if err == io.EOF || linuxerr.Equals(linuxerr.EAGAIN, err) {
break
}
return err
diff --git a/pkg/sentry/fsimpl/gofer/special_file.go b/pkg/sentry/fsimpl/gofer/special_file.go
index c12444b7e..3d7b5506e 100644
--- a/pkg/sentry/fsimpl/gofer/special_file.go
+++ b/pkg/sentry/fsimpl/gofer/special_file.go
@@ -20,6 +20,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/metric"
"gvisor.dev/gvisor/pkg/p9"
@@ -228,7 +229,7 @@ func (fd *specialFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offs
// Just buffer the read instead.
buf := make([]byte, dst.NumBytes())
n, err := fd.handle.readToBlocksAt(ctx, safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf)), uint64(offset))
- if err == syserror.EAGAIN {
+ if linuxerr.Equals(linuxerr.EAGAIN, err) {
err = syserror.ErrWouldBlock
}
if n == 0 {
@@ -316,7 +317,7 @@ func (fd *specialFileFD) pwrite(ctx context.Context, src usermem.IOSequence, off
return 0, offset, copyErr
}
n, err := fd.handle.writeFromBlocksAt(ctx, safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf[:copied])), uint64(offset))
- if err == syserror.EAGAIN {
+ if linuxerr.Equals(linuxerr.EAGAIN, err) {
err = syserror.ErrWouldBlock
}
// Update offset if the offset is valid.
diff --git a/pkg/sentry/fsimpl/host/BUILD b/pkg/sentry/fsimpl/host/BUILD
index b94dfeb7f..476545d00 100644
--- a/pkg/sentry/fsimpl/host/BUILD
+++ b/pkg/sentry/fsimpl/host/BUILD
@@ -45,10 +45,10 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fdnotifier",
"//pkg/fspath",
"//pkg/hostarch",
- "//pkg/iovec",
"//pkg/log",
"//pkg/marshal/primitive",
"//pkg/refs",
diff --git a/pkg/sentry/fsimpl/host/host.go b/pkg/sentry/fsimpl/host/host.go
index a81f550b1..2dbfbdecf 100644
--- a/pkg/sentry/fsimpl/host/host.go
+++ b/pkg/sentry/fsimpl/host/host.go
@@ -24,6 +24,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/hostarch"
@@ -109,7 +110,7 @@ type inode struct {
func newInode(ctx context.Context, fs *filesystem, hostFD int, savable bool, fileType linux.FileMode, isTTY bool) (*inode, error) {
// Determine if hostFD is seekable.
_, err := unix.Seek(hostFD, 0, linux.SEEK_CUR)
- seekable := err != syserror.ESPIPE
+ seekable := !linuxerr.Equals(linuxerr.ESPIPE, err)
// We expect regular files to be seekable, as this is required for them to
// be memory-mappable.
if !seekable && fileType == unix.S_IFREG {
@@ -301,7 +302,7 @@ func (i *inode) Stat(ctx context.Context, vfsfs *vfs.Filesystem, opts vfs.StatOp
mask := opts.Mask & linux.STATX_ALL
var s unix.Statx_t
err := unix.Statx(i.hostFD, "", int(unix.AT_EMPTY_PATH|opts.Sync), int(mask), &s)
- if err == syserror.ENOSYS {
+ if linuxerr.Equals(linuxerr.ENOSYS, err) {
// Fallback to fstat(2), if statx(2) is not supported on the host.
//
// TODO(b/151263641): Remove fallback.
diff --git a/pkg/sentry/fsimpl/host/socket.go b/pkg/sentry/fsimpl/host/socket.go
index ca85f5601..8cce36212 100644
--- a/pkg/sentry/fsimpl/host/socket.go
+++ b/pkg/sentry/fsimpl/host/socket.go
@@ -21,6 +21,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/socket/control"
@@ -160,7 +161,7 @@ func (c *ConnectedEndpoint) Send(ctx context.Context, data [][]byte, controlMess
// block (and only for stream sockets).
err = syserror.EAGAIN
}
- if n > 0 && err != syserror.EAGAIN {
+ if n > 0 && !linuxerr.Equals(linuxerr.EAGAIN, err) {
// The caller may need to block to send more data, but
// otherwise there isn't anything that can be done about an
// error with a partial write.
diff --git a/pkg/sentry/fsimpl/host/socket_iovec.go b/pkg/sentry/fsimpl/host/socket_iovec.go
index b123a63ee..e090bb725 100644
--- a/pkg/sentry/fsimpl/host/socket_iovec.go
+++ b/pkg/sentry/fsimpl/host/socket_iovec.go
@@ -16,7 +16,7 @@ package host
import (
"golang.org/x/sys/unix"
- "gvisor.dev/gvisor/pkg/iovec"
+ "gvisor.dev/gvisor/pkg/sentry/hostfd"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -70,7 +70,7 @@ func buildIovec(bufs [][]byte, maxlen int64, truncate bool) (length int64, iovec
}
}
- if iovsRequired > iovec.MaxIovs {
+ if iovsRequired > hostfd.MaxSendRecvMsgIov {
// The kernel will reject our call if we pass this many iovs.
// Use a single intermediate buffer instead.
b := make([]byte, stopLen)
diff --git a/pkg/sentry/fsimpl/host/tty.go b/pkg/sentry/fsimpl/host/tty.go
index 0f9e20a84..2cf360065 100644
--- a/pkg/sentry/fsimpl/host/tty.go
+++ b/pkg/sentry/fsimpl/host/tty.go
@@ -17,6 +17,7 @@ package host
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/kernel"
@@ -211,7 +212,7 @@ func (t *TTYFileDescription) Ioctl(ctx context.Context, io usermem.IO, args arch
if err := t.checkChange(ctx, linux.SIGTTOU); err != nil {
// drivers/tty/tty_io.c:tiocspgrp() converts -EIO from tty_check_change()
// to -ENOTTY.
- if err == syserror.EIO {
+ if linuxerr.Equals(linuxerr.EIO, err) {
return 0, syserror.ENOTTY
}
return 0, err
diff --git a/pkg/sentry/fsimpl/host/util.go b/pkg/sentry/fsimpl/host/util.go
index 63b465859..95d7ebe2e 100644
--- a/pkg/sentry/fsimpl/host/util.go
+++ b/pkg/sentry/fsimpl/host/util.go
@@ -17,7 +17,7 @@ package host
import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
- "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
)
func toTimespec(ts linux.StatxTimestamp, omit bool) unix.Timespec {
@@ -44,5 +44,5 @@ func timespecToStatxTimestamp(ts unix.Timespec) linux.StatxTimestamp {
// isBlockError checks if an error is EAGAIN or EWOULDBLOCK.
// If so, they can be transformed into syserror.ErrWouldBlock.
func isBlockError(err error) bool {
- return err == syserror.EAGAIN || err == syserror.EWOULDBLOCK
+ return linuxerr.Equals(linuxerr.EAGAIN, err) || linuxerr.Equals(linuxerr.EWOULDBLOCK, err)
}
diff --git a/pkg/sentry/fsimpl/kernfs/BUILD b/pkg/sentry/fsimpl/kernfs/BUILD
index b7d13cced..d53937db6 100644
--- a/pkg/sentry/fsimpl/kernfs/BUILD
+++ b/pkg/sentry/fsimpl/kernfs/BUILD
@@ -104,6 +104,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/hostarch",
"//pkg/log",
@@ -135,6 +136,7 @@ go_test(
":kernfs",
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go
index 8fac53c60..20319ab76 100644
--- a/pkg/sentry/fsimpl/kernfs/filesystem.go
+++ b/pkg/sentry/fsimpl/kernfs/filesystem.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
@@ -411,7 +412,7 @@ func (fs *Filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
defer rp.Mount().EndWrite()
childI, err := parent.inode.NewDir(ctx, pc, opts)
if err != nil {
- if !opts.ForSyntheticMountpoint || err == syserror.EEXIST {
+ if !opts.ForSyntheticMountpoint || linuxerr.Equals(linuxerr.EEXIST, err) {
return err
}
childI = newSyntheticDirectory(ctx, rp.Credentials(), opts.Mode)
@@ -546,7 +547,7 @@ afterTrailingSymlink:
}
// Determine whether or not we need to create a file.
child, err := fs.stepExistingLocked(ctx, rp, parent, false /* mayFollowSymlinks */)
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
// Already checked for searchability above; now check for writability.
if err := parent.inode.CheckPermissions(ctx, rp.Credentials(), vfs.MayWrite); err != nil {
return nil, err
@@ -684,10 +685,12 @@ func (fs *Filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
}
return syserror.EBUSY
}
- switch err := checkCreateLocked(ctx, rp.Credentials(), newName, dstDir); err {
- case nil:
+
+ err = checkCreateLocked(ctx, rp.Credentials(), newName, dstDir)
+ switch {
+ case err == nil:
// Ok, continue with rename as replacement.
- case syserror.EEXIST:
+ case linuxerr.Equals(linuxerr.EEXIST, err):
if noReplace {
// Won't overwrite existing node since RENAME_NOREPLACE was requested.
return syserror.EEXIST
diff --git a/pkg/sentry/fsimpl/kernfs/kernfs_test.go b/pkg/sentry/fsimpl/kernfs/kernfs_test.go
index 1cd3137e6..de046ce1f 100644
--- a/pkg/sentry/fsimpl/kernfs/kernfs_test.go
+++ b/pkg/sentry/fsimpl/kernfs/kernfs_test.go
@@ -22,6 +22,7 @@ import (
"github.com/google/go-cmp/cmp"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/contexttest"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/testutil"
@@ -318,10 +319,10 @@ func TestDirFDReadWrite(t *testing.T) {
defer fd.DecRef(sys.Ctx)
// Read/Write should fail for directory FDs.
- if _, err := fd.Read(sys.Ctx, usermem.BytesIOSequence([]byte{}), vfs.ReadOptions{}); err != syserror.EISDIR {
+ if _, err := fd.Read(sys.Ctx, usermem.BytesIOSequence([]byte{}), vfs.ReadOptions{}); !linuxerr.Equals(linuxerr.EISDIR, err) {
t.Fatalf("Read for directory FD failed with unexpected error: %v", err)
}
- if _, err := fd.Write(sys.Ctx, usermem.BytesIOSequence([]byte{}), vfs.WriteOptions{}); err != syserror.EBADF {
+ if _, err := fd.Write(sys.Ctx, usermem.BytesIOSequence([]byte{}), vfs.WriteOptions{}); !linuxerr.Equals(linuxerr.EBADF, err) {
t.Fatalf("Write for directory FD failed with unexpected error: %v", err)
}
}
diff --git a/pkg/sentry/fsimpl/overlay/BUILD b/pkg/sentry/fsimpl/overlay/BUILD
index 5504476c8..ed730e215 100644
--- a/pkg/sentry/fsimpl/overlay/BUILD
+++ b/pkg/sentry/fsimpl/overlay/BUILD
@@ -29,6 +29,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/hostarch",
"//pkg/log",
diff --git a/pkg/sentry/fsimpl/overlay/copy_up.go b/pkg/sentry/fsimpl/overlay/copy_up.go
index 45aa5a494..8fd51e9d0 100644
--- a/pkg/sentry/fsimpl/overlay/copy_up.go
+++ b/pkg/sentry/fsimpl/overlay/copy_up.go
@@ -20,6 +20,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -349,7 +350,7 @@ func (d *dentry) copyXattrsLocked(ctx context.Context) error {
lowerXattrs, err := vfsObj.ListXattrAt(ctx, d.fs.creds, lowerPop, 0)
if err != nil {
- if err == syserror.EOPNOTSUPP {
+ if linuxerr.Equals(linuxerr.EOPNOTSUPP, err) {
// There are no guarantees as to the contents of lowerXattrs.
return nil
}
diff --git a/pkg/sentry/fsimpl/overlay/filesystem.go b/pkg/sentry/fsimpl/overlay/filesystem.go
index 6b6fa0bd5..81745bccd 100644
--- a/pkg/sentry/fsimpl/overlay/filesystem.go
+++ b/pkg/sentry/fsimpl/overlay/filesystem.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -218,7 +219,7 @@ func (fs *filesystem) lookupLocked(ctx context.Context, parent *dentry, name str
Start: parentVD,
Path: childPath,
}, &vfs.GetDentryOptions{})
- if err == syserror.ENOENT || err == syserror.ENAMETOOLONG {
+ if linuxerr.Equals(linuxerr.ENOENT, err) || linuxerr.Equals(linuxerr.ENAMETOOLONG, err) {
// The file doesn't exist on this layer. Proceed to the next one.
return true
}
@@ -352,7 +353,7 @@ func (fs *filesystem) lookupLayerLocked(ctx context.Context, parent *dentry, nam
}, &vfs.StatOptions{
Mask: linux.STATX_TYPE,
})
- if err == syserror.ENOENT || err == syserror.ENAMETOOLONG {
+ if linuxerr.Equals(linuxerr.ENOENT, err) || linuxerr.Equals(linuxerr.ENAMETOOLONG, err) {
// The file doesn't exist on this layer. Proceed to the next
// one.
return true
@@ -811,7 +812,7 @@ afterTrailingSymlink:
// Determine whether or not we need to create a file.
parent.dirMu.Lock()
child, topLookupLayer, err := fs.stepLocked(ctx, rp, parent, false /* mayFollowSymlinks */, &ds)
- if err == syserror.ENOENT && mayCreate {
+ if linuxerr.Equals(linuxerr.ENOENT, err) && mayCreate {
fd, err := fs.createAndOpenLocked(ctx, rp, parent, &opts, &ds, topLookupLayer == lookupLayerUpperWhiteout)
parent.dirMu.Unlock()
return fd, err
@@ -1094,7 +1095,7 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
whiteouts map[string]bool
)
replaced, replacedLayer, err = fs.getChildLocked(ctx, newParent, newName, &ds)
- if err != nil && err != syserror.ENOENT {
+ if err != nil && !linuxerr.Equals(linuxerr.ENOENT, err) {
return err
}
if replaced != nil {
@@ -1177,7 +1178,7 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
Root: replaced.upperVD,
Start: replaced.upperVD,
Path: fspath.Parse(whiteoutName),
- }); err != nil && err != syserror.EEXIST {
+ }); err != nil && !linuxerr.Equals(linuxerr.EEXIST, err) {
panic(fmt.Sprintf("unrecoverable overlayfs inconsistency: failed to recreate deleted whiteout after RenameAt failure: %v", err))
}
}
@@ -1344,7 +1345,7 @@ func (fs *filesystem) RmdirAt(ctx context.Context, rp *vfs.ResolvingPath) error
Root: child.upperVD,
Start: child.upperVD,
Path: fspath.Parse(whiteoutName),
- }); err != nil && err != syserror.EEXIST {
+ }); err != nil && !linuxerr.Equals(linuxerr.EEXIST, err) {
panic(fmt.Sprintf("unrecoverable overlayfs inconsistency: failed to recreate deleted whiteout after RmdirAt failure: %v", err))
}
}
diff --git a/pkg/sentry/fsimpl/proc/BUILD b/pkg/sentry/fsimpl/proc/BUILD
index 2b628bd55..1d3d2d95f 100644
--- a/pkg/sentry/fsimpl/proc/BUILD
+++ b/pkg/sentry/fsimpl/proc/BUILD
@@ -81,6 +81,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/log",
"//pkg/refs",
@@ -119,6 +120,7 @@ go_test(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/sentry/contexttest",
"//pkg/sentry/fsimpl/testutil",
@@ -127,7 +129,6 @@ go_test(
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/vfs",
- "//pkg/syserror",
"//pkg/usermem",
],
)
diff --git a/pkg/sentry/fsimpl/proc/task_net.go b/pkg/sentry/fsimpl/proc/task_net.go
index 177cb828f..ab47ea5a7 100644
--- a/pkg/sentry/fsimpl/proc/task_net.go
+++ b/pkg/sentry/fsimpl/proc/task_net.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
@@ -33,7 +34,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/socket/unix"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/sentry/vfs"
- "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/tcpip/header"
)
@@ -679,7 +679,7 @@ func (d *netSnmpData) Generate(ctx context.Context, buf *bytes.Buffer) error {
continue
}
if err := d.stack.Statistics(stat, line.prefix); err != nil {
- if err == syserror.EOPNOTSUPP {
+ if linuxerr.Equals(linuxerr.EOPNOTSUPP, err) {
log.Infof("Failed to retrieve %s of /proc/net/snmp: %v", line.prefix, err)
} else {
log.Warningf("Failed to retrieve %s of /proc/net/snmp: %v", line.prefix, err)
diff --git a/pkg/sentry/fsimpl/proc/tasks_test.go b/pkg/sentry/fsimpl/proc/tasks_test.go
index e534fbca8..14f806c3c 100644
--- a/pkg/sentry/fsimpl/proc/tasks_test.go
+++ b/pkg/sentry/fsimpl/proc/tasks_test.go
@@ -23,13 +23,13 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"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"
- "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
)
@@ -227,7 +227,7 @@ func TestTasks(t *testing.T) {
defer fd.DecRef(s.Ctx)
buf := make([]byte, 1)
bufIOSeq := usermem.BytesIOSequence(buf)
- if _, err := fd.Read(s.Ctx, bufIOSeq, vfs.ReadOptions{}); err != syserror.EISDIR {
+ if _, err := fd.Read(s.Ctx, bufIOSeq, vfs.ReadOptions{}); !linuxerr.Equals(linuxerr.EISDIR, err) {
t.Errorf("wrong error reading directory: %v", err)
}
}
@@ -237,7 +237,7 @@ func TestTasks(t *testing.T) {
s.Creds,
s.PathOpAtRoot("/proc/9999"),
&vfs.OpenOptions{},
- ); err != syserror.ENOENT {
+ ); !linuxerr.Equals(linuxerr.ENOENT, err) {
t.Fatalf("wrong error from vfsfs.OpenAt(/proc/9999): %v", err)
}
}
diff --git a/pkg/sentry/fsimpl/tmpfs/BUILD b/pkg/sentry/fsimpl/tmpfs/BUILD
index e21fddd7f..341b4f904 100644
--- a/pkg/sentry/fsimpl/tmpfs/BUILD
+++ b/pkg/sentry/fsimpl/tmpfs/BUILD
@@ -118,6 +118,7 @@ go_test(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/sentry/contexttest",
"//pkg/sentry/fs/lock",
diff --git a/pkg/sentry/fsimpl/tmpfs/pipe_test.go b/pkg/sentry/fsimpl/tmpfs/pipe_test.go
index 2f856ce36..418c7994e 100644
--- a/pkg/sentry/fsimpl/tmpfs/pipe_test.go
+++ b/pkg/sentry/fsimpl/tmpfs/pipe_test.go
@@ -20,6 +20,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/contexttest"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -114,7 +115,7 @@ func TestNonblockingWriteError(t *testing.T) {
}
openOpts := vfs.OpenOptions{Flags: linux.O_WRONLY | linux.O_NONBLOCK}
_, err := vfsObj.OpenAt(ctx, creds, &pop, &openOpts)
- if err != syserror.ENXIO {
+ if !linuxerr.Equals(linuxerr.ENXIO, err) {
t.Fatalf("expected ENXIO, but got error: %v", err)
}
}
diff --git a/pkg/sentry/fsimpl/verity/BUILD b/pkg/sentry/fsimpl/verity/BUILD
index d473a922d..1d855234c 100644
--- a/pkg/sentry/fsimpl/verity/BUILD
+++ b/pkg/sentry/fsimpl/verity/BUILD
@@ -13,6 +13,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/hostarch",
"//pkg/marshal/primitive",
@@ -41,6 +42,7 @@ go_test(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/sentry/arch",
"//pkg/sentry/fsimpl/testutil",
@@ -48,7 +50,6 @@ go_test(
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/vfs",
- "//pkg/syserror",
"//pkg/usermem",
],
)
diff --git a/pkg/sentry/fsimpl/verity/filesystem.go b/pkg/sentry/fsimpl/verity/filesystem.go
index 3582d14c9..e84452421 100644
--- a/pkg/sentry/fsimpl/verity/filesystem.go
+++ b/pkg/sentry/fsimpl/verity/filesystem.go
@@ -25,6 +25,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/merkletree"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -195,7 +196,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
// The Merkle tree file for the child should have been created and
// contains the expected xattrs. If the file or the xattr does not
// exist, it indicates unexpected modifications to the file system.
- if err == syserror.ENOENT || err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENOENT, err) || linuxerr.Equals(linuxerr.ENODATA, err) {
return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for %s: %v", merkleOffsetInParentXattr, childPath, err))
}
if err != nil {
@@ -218,7 +219,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
// The parent Merkle tree file should have been created. If it's
// missing, it indicates an unexpected modification to the file system.
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to open parent Merkle file for %s: %v", childPath, err))
}
if err != nil {
@@ -238,7 +239,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
// The Merkle tree file for the child should have been created and
// contains the expected xattrs. If the file or the xattr does not
// exist, it indicates unexpected modifications to the file system.
- if err == syserror.ENOENT || err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENOENT, err) || linuxerr.Equals(linuxerr.ENODATA, err) {
return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for %s: %v", merkleSizeXattr, childPath, err))
}
if err != nil {
@@ -261,7 +262,7 @@ func (fs *filesystem) verifyChildLocked(ctx context.Context, parent *dentry, chi
Root: parent.lowerVD,
Start: parent.lowerVD,
}, &vfs.StatOptions{})
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
return nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get parent stat for %s: %v", childPath, err))
}
if err != nil {
@@ -327,7 +328,7 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
}, &vfs.OpenOptions{
Flags: linux.O_RDONLY,
})
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
return fs.alertIntegrityViolation(fmt.Sprintf("Failed to open merkle file for %s: %v", childPath, err))
}
if err != nil {
@@ -341,7 +342,7 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
Size: sizeOfStringInt32,
})
- if err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENODATA, err) {
return fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", merkleSizeXattr, childPath, err))
}
if err != nil {
@@ -359,7 +360,7 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
Size: sizeOfStringInt32,
})
- if err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENODATA, err) {
return fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", childrenOffsetXattr, childPath, err))
}
if err != nil {
@@ -375,7 +376,7 @@ func (fs *filesystem) verifyStatAndChildrenLocked(ctx context.Context, d *dentry
Size: sizeOfStringInt32,
})
- if err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENODATA, err) {
return fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s for merkle file of %s: %v", childrenSizeXattr, childPath, err))
}
if err != nil {
@@ -465,7 +466,7 @@ func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name s
}
childVD, err := parent.getLowerAt(ctx, vfsObj, name)
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
// The file was previously accessed. If the
// file does not exist now, it indicates an
// unexpected modification to the file system.
@@ -480,7 +481,7 @@ func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name s
// The Merkle tree file was previous accessed. If it
// does not exist now, it indicates an unexpected
// modification to the file system.
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
return nil, fs.alertIntegrityViolation(fmt.Sprintf("Expected Merkle file for target %s but none found", path))
}
if err != nil {
@@ -551,7 +552,7 @@ func (fs *filesystem) lookupAndVerifyLocked(ctx context.Context, parent *dentry,
}
childVD, err := parent.getLowerAt(ctx, vfsObj, name)
- if parent.verityEnabled() && err == syserror.ENOENT {
+ if parent.verityEnabled() && linuxerr.Equals(linuxerr.ENOENT, err) {
return nil, fs.alertIntegrityViolation(fmt.Sprintf("file %s expected but not found", parentPath+"/"+name))
}
if err != nil {
@@ -564,7 +565,7 @@ func (fs *filesystem) lookupAndVerifyLocked(ctx context.Context, parent *dentry,
childMerkleVD, err := parent.getLowerAt(ctx, vfsObj, merklePrefix+name)
if err != nil {
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
if parent.verityEnabled() {
return nil, fs.alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", parentPath+"/"+name))
}
@@ -854,7 +855,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
// The file should exist, as we succeeded in finding its dentry. If it's
// missing, it indicates an unexpected modification to the file system.
if err != nil {
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
return nil, d.fs.alertIntegrityViolation(fmt.Sprintf("File %s expected but not found", path))
}
return nil, err
@@ -877,7 +878,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
// dentry. If it's missing, it indicates an unexpected modification to
// the file system.
if err != nil {
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
return nil, d.fs.alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", path))
}
return nil, err
@@ -902,7 +903,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
Flags: linux.O_WRONLY | linux.O_APPEND,
})
if err != nil {
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
return nil, d.fs.alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", path))
}
return nil, err
@@ -919,7 +920,7 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
Flags: linux.O_WRONLY | linux.O_APPEND,
})
if err != nil {
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
parentPath, _ := d.fs.vfsfs.VirtualFilesystem().PathnameWithDeleted(ctx, d.fs.rootDentry.lowerVD, d.parent.lowerVD)
return nil, d.fs.alertIntegrityViolation(fmt.Sprintf("Merkle file for %s expected but not found", parentPath))
}
diff --git a/pkg/sentry/fsimpl/verity/verity.go b/pkg/sentry/fsimpl/verity/verity.go
index 969003613..c5f59d851 100644
--- a/pkg/sentry/fsimpl/verity/verity.go
+++ b/pkg/sentry/fsimpl/verity/verity.go
@@ -45,6 +45,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal/primitive"
@@ -358,7 +359,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
// If runtime enable is allowed, the root merkle tree may be absent. We
// should create the tree file.
- if err == syserror.ENOENT && fs.allowRuntimeEnable {
+ if linuxerr.Equals(linuxerr.ENOENT, err) && fs.allowRuntimeEnable {
lowerMerkleFD, err := vfsObj.OpenAt(ctx, fs.creds, &vfs.PathOperation{
Root: lowerVD,
Start: lowerVD,
@@ -451,7 +452,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
Name: childrenOffsetXattr,
Size: sizeOfStringInt32,
})
- if err == syserror.ENOENT || err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENOENT, err) || linuxerr.Equals(linuxerr.ENODATA, err) {
return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", childrenOffsetXattr, err))
}
if err != nil {
@@ -470,7 +471,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
Name: childrenSizeXattr,
Size: sizeOfStringInt32,
})
- if err == syserror.ENOENT || err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENOENT, err) || linuxerr.Equals(linuxerr.ENODATA, err) {
return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", childrenSizeXattr, err))
}
if err != nil {
@@ -487,7 +488,7 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
}, &vfs.OpenOptions{
Flags: linux.O_RDONLY,
})
- if err == syserror.ENOENT {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
return nil, nil, fs.alertIntegrityViolation(fmt.Sprintf("Failed to open root Merkle file: %v", err))
}
if err != nil {
@@ -1227,7 +1228,7 @@ func (fd *fileDescription) PRead(ctx context.Context, dst usermem.IOSequence, of
// The Merkle tree file for the child should have been created and
// contains the expected xattrs. If the xattr does not exist, it
// indicates unexpected modifications to the file system.
- if err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENODATA, err) {
return 0, fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", merkleSizeXattr, err))
}
if err != nil {
@@ -1349,7 +1350,7 @@ func (fd *fileDescription) Translate(ctx context.Context, required, optional mem
// The Merkle tree file for the child should have been created and
// contains the expected xattrs. If the xattr does not exist, it
// indicates unexpected modifications to the file system.
- if err == syserror.ENODATA {
+ if linuxerr.Equals(linuxerr.ENODATA, err) {
return nil, fd.d.fs.alertIntegrityViolation(fmt.Sprintf("Failed to get xattr %s: %v", merkleSizeXattr, err))
}
if err != nil {
diff --git a/pkg/sentry/fsimpl/verity/verity_test.go b/pkg/sentry/fsimpl/verity/verity_test.go
index 5c78a0019..65465b814 100644
--- a/pkg/sentry/fsimpl/verity/verity_test.go
+++ b/pkg/sentry/fsimpl/verity/verity_test.go
@@ -24,6 +24,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/testutil"
@@ -31,7 +32,6 @@ 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/usermem"
)
@@ -476,7 +476,7 @@ func TestOpenNonexistentFile(t *testing.T) {
// Ensure open an unexpected file in the parent directory fails with
// ENOENT rather than verification failure.
- if _, err = openVerityAt(ctx, vfsObj, root, filename+"abc", linux.O_RDONLY, linux.ModeRegular); err != syserror.ENOENT {
+ if _, err = openVerityAt(ctx, vfsObj, root, filename+"abc", linux.O_RDONLY, linux.ModeRegular); !linuxerr.Equals(linuxerr.ENOENT, err) {
t.Errorf("OpenAt unexpected error: %v", err)
}
}
@@ -767,7 +767,7 @@ func TestOpenDeletedFileFails(t *testing.T) {
}
// Ensure reopening the verity enabled file fails.
- if _, err = openVerityAt(ctx, vfsObj, root, filename, linux.O_RDONLY, linux.ModeRegular); err != syserror.EIO {
+ if _, err = openVerityAt(ctx, vfsObj, root, filename, linux.O_RDONLY, linux.ModeRegular); !linuxerr.Equals(linuxerr.EIO, err) {
t.Errorf("got OpenAt error: %v, expected EIO", err)
}
})
@@ -829,7 +829,7 @@ func TestOpenRenamedFileFails(t *testing.T) {
}
// Ensure reopening the verity enabled file fails.
- if _, err = openVerityAt(ctx, vfsObj, root, filename, linux.O_RDONLY, linux.ModeRegular); err != syserror.EIO {
+ if _, err = openVerityAt(ctx, vfsObj, root, filename, linux.O_RDONLY, linux.ModeRegular); !linuxerr.Equals(linuxerr.EIO, err) {
t.Errorf("got OpenAt error: %v, expected EIO", err)
}
})
@@ -1063,14 +1063,14 @@ func TestDeletedSymlinkFileReadFails(t *testing.T) {
Root: root,
Start: root,
Path: fspath.Parse(symlink),
- }); err != syserror.EIO {
+ }); !linuxerr.Equals(linuxerr.EIO, err) {
t.Fatalf("ReadlinkAt succeeded with modified symlink: %v", err)
}
if tc.testWalk {
fileInSymlinkDirectory := symlink + "/verity-test-file"
// Ensure opening the verity enabled file in the symlink directory fails.
- if _, err := openVerityAt(ctx, vfsObj, root, fileInSymlinkDirectory, linux.O_RDONLY, linux.ModeRegular); err != syserror.EIO {
+ if _, err := openVerityAt(ctx, vfsObj, root, fileInSymlinkDirectory, linux.O_RDONLY, linux.ModeRegular); !linuxerr.Equals(linuxerr.EIO, err) {
t.Errorf("Open succeeded with modified symlink: %v", err)
}
}
@@ -1195,14 +1195,14 @@ func TestModifiedSymlinkFileReadFails(t *testing.T) {
Root: root,
Start: root,
Path: fspath.Parse(symlink),
- }); err != syserror.EIO {
+ }); !linuxerr.Equals(linuxerr.EIO, err) {
t.Fatalf("ReadlinkAt succeeded with modified symlink: %v", err)
}
if tc.testWalk {
fileInSymlinkDirectory := symlink + "/verity-test-file"
// Ensure opening the verity enabled file in the symlink directory fails.
- if _, err := openVerityAt(ctx, vfsObj, root, fileInSymlinkDirectory, linux.O_RDONLY, linux.ModeRegular); err != syserror.EIO {
+ if _, err := openVerityAt(ctx, vfsObj, root, fileInSymlinkDirectory, linux.O_RDONLY, linux.ModeRegular); !linuxerr.Equals(linuxerr.EIO, err) {
t.Errorf("Open succeeded with modified symlink: %v", err)
}
}
diff --git a/pkg/sentry/hostfd/hostfd_linux.go b/pkg/sentry/hostfd/hostfd_linux.go
index 1cabc848f..e103e7296 100644
--- a/pkg/sentry/hostfd/hostfd_linux.go
+++ b/pkg/sentry/hostfd/hostfd_linux.go
@@ -14,5 +14,10 @@
package hostfd
-// maxIov is the maximum permitted size of a struct iovec array.
-const maxIov = 1024 // UIO_MAXIOV
+// MaxReadWriteIov is the maximum permitted size of a struct iovec array in a
+// readv, writev, preadv, or pwritev host syscall.
+const MaxReadWriteIov = 1024 // UIO_MAXIOV
+
+// MaxSendRecvMsgIov is the maximum permitted size of a struct iovec array in a
+// sendmsg or recvmsg host syscall.
+const MaxSendRecvMsgIov = 1024 // UIO_MAXIOV
diff --git a/pkg/sentry/hostfd/hostfd_unsafe.go b/pkg/sentry/hostfd/hostfd_unsafe.go
index 03c6d2a16..a43311eb4 100644
--- a/pkg/sentry/hostfd/hostfd_unsafe.go
+++ b/pkg/sentry/hostfd/hostfd_unsafe.go
@@ -23,6 +23,11 @@ import (
"gvisor.dev/gvisor/pkg/safemem"
)
+const (
+ sizeofIovec = unsafe.Sizeof(unix.Iovec{})
+ sizeofMsghdr = unsafe.Sizeof(unix.Msghdr{})
+)
+
// Preadv2 reads up to dsts.NumBytes() bytes from host file descriptor fd into
// dsts. offset and flags are interpreted as for preadv2(2).
//
@@ -44,9 +49,9 @@ func Preadv2(fd int32, dsts safemem.BlockSeq, offset int64, flags uint32) (uint6
}
} else {
iovs := safemem.IovecsFromBlockSeq(dsts)
- if len(iovs) > maxIov {
- log.Debugf("hostfd.Preadv2: truncating from %d iovecs to %d", len(iovs), maxIov)
- iovs = iovs[:maxIov]
+ if len(iovs) > MaxReadWriteIov {
+ log.Debugf("hostfd.Preadv2: truncating from %d iovecs to %d", len(iovs), MaxReadWriteIov)
+ iovs = iovs[:MaxReadWriteIov]
}
n, _, e = unix.Syscall6(unix.SYS_PREADV2, uintptr(fd), uintptr((unsafe.Pointer)(&iovs[0])), uintptr(len(iovs)), uintptr(offset), 0 /* pos_h */, uintptr(flags))
}
@@ -80,9 +85,9 @@ func Pwritev2(fd int32, srcs safemem.BlockSeq, offset int64, flags uint32) (uint
}
} else {
iovs := safemem.IovecsFromBlockSeq(srcs)
- if len(iovs) > maxIov {
- log.Debugf("hostfd.Preadv2: truncating from %d iovecs to %d", len(iovs), maxIov)
- iovs = iovs[:maxIov]
+ if len(iovs) > MaxReadWriteIov {
+ log.Debugf("hostfd.Preadv2: truncating from %d iovecs to %d", len(iovs), MaxReadWriteIov)
+ iovs = iovs[:MaxReadWriteIov]
}
n, _, e = unix.Syscall6(unix.SYS_PWRITEV2, uintptr(fd), uintptr((unsafe.Pointer)(&iovs[0])), uintptr(len(iovs)), uintptr(offset), 0 /* pos_h */, uintptr(flags))
}
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index a82d641da..9a4b08469 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -226,6 +226,7 @@ go_library(
"//pkg/context",
"//pkg/coverage",
"//pkg/cpuid",
+ "//pkg/errors/linuxerr",
"//pkg/eventchannel",
"//pkg/fspath",
"//pkg/goid",
diff --git a/pkg/sentry/kernel/fd_table.go b/pkg/sentry/kernel/fd_table.go
index 62777faa8..8786a70b5 100644
--- a/pkg/sentry/kernel/fd_table.go
+++ b/pkg/sentry/kernel/fd_table.go
@@ -23,12 +23,12 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/fs/lock"
"gvisor.dev/gvisor/pkg/sentry/limits"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
- "gvisor.dev/gvisor/pkg/syserror"
)
// FDFlags define flags for an individual descriptor.
@@ -156,7 +156,7 @@ func (f *FDTable) dropVFS2(ctx context.Context, file *vfs.FileDescription) {
// Release any POSIX lock possibly held by the FDTable.
if file.SupportsLocks() {
err := file.UnlockPOSIX(ctx, f, lock.LockRange{0, lock.LockEOF})
- if err != nil && err != syserror.ENOLCK {
+ if err != nil && !linuxerr.Equals(linuxerr.ENOLCK, err) {
panic(fmt.Sprintf("UnlockPOSIX failed: %v", err))
}
}
diff --git a/pkg/sentry/kernel/pipe/BUILD b/pkg/sentry/kernel/pipe/BUILD
index 34c617b08..af46b3e08 100644
--- a/pkg/sentry/kernel/pipe/BUILD
+++ b/pkg/sentry/kernel/pipe/BUILD
@@ -47,6 +47,7 @@ go_test(
library = ":pipe",
deps = [
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/sentry/contexttest",
"//pkg/sentry/fs",
"//pkg/syserror",
diff --git a/pkg/sentry/kernel/pipe/node_test.go b/pkg/sentry/kernel/pipe/node_test.go
index d6fb0fdb8..d25cf658e 100644
--- a/pkg/sentry/kernel/pipe/node_test.go
+++ b/pkg/sentry/kernel/pipe/node_test.go
@@ -19,6 +19,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/contexttest"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/syserror"
@@ -258,7 +259,7 @@ func TestNonblockingWriteOpenFileNoReaders(t *testing.T) {
ctx := newSleeperContext(t)
f := NewInodeOperations(ctx, perms, newNamedPipe(t))
- if _, err := testOpen(ctx, t, f, fs.FileFlags{Write: true, NonBlocking: true}, nil); err != syserror.ENXIO {
+ if _, err := testOpen(ctx, t, f, fs.FileFlags{Write: true, NonBlocking: true}, nil); !linuxerr.Equals(linuxerr.ENXIO, err) {
t.Fatalf("Nonblocking open for write failed unexpected error %v.", err)
}
}
diff --git a/pkg/sentry/kernel/ptrace.go b/pkg/sentry/kernel/ptrace.go
index a6287fd6a..20563f02a 100644
--- a/pkg/sentry/kernel/ptrace.go
+++ b/pkg/sentry/kernel/ptrace.go
@@ -294,7 +294,7 @@ func (t *Task) isYAMADescendantOfLocked(ancestor *Task) bool {
// Precondition: the TaskSet mutex must be locked (for reading or writing).
func (t *Task) hasYAMAExceptionForLocked(tracer *Task) bool {
- allowed, ok := t.k.ptraceExceptions[t]
+ allowed, ok := t.k.ptraceExceptions[t.tg.leader]
if !ok {
return false
}
diff --git a/pkg/sentry/kernel/sessions.go b/pkg/sentry/kernel/sessions.go
index ca9076406..973d708a3 100644
--- a/pkg/sentry/kernel/sessions.go
+++ b/pkg/sentry/kernel/sessions.go
@@ -369,6 +369,11 @@ func (tg *ThreadGroup) CreateProcessGroup() error {
// Get the ID for this thread in the current namespace.
id := tg.pidns.tgids[tg]
+ // Check whether a process still exists or not.
+ if id == 0 {
+ return syserror.ESRCH
+ }
+
// Per above, check for a Session leader or existing group.
for s := tg.pidns.owner.sessions.Front(); s != nil; s = s.Next() {
if s.leader.pidns != tg.pidns {
diff --git a/pkg/sentry/kernel/task_block.go b/pkg/sentry/kernel/task_block.go
index ecbe8f920..07533d982 100644
--- a/pkg/sentry/kernel/task_block.go
+++ b/pkg/sentry/kernel/task_block.go
@@ -19,6 +19,7 @@ import (
"runtime/trace"
"time"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
@@ -45,7 +46,7 @@ func (t *Task) BlockWithTimeout(C chan struct{}, haveTimeout bool, timeout time.
err := t.BlockWithDeadline(C, true, deadline)
// Timeout, explicitly return a remaining duration of 0.
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
return 0, err
}
diff --git a/pkg/sentry/kernel/task_syscall.go b/pkg/sentry/kernel/task_syscall.go
index 601fc0d3a..1874f74e5 100644
--- a/pkg/sentry/kernel/task_syscall.go
+++ b/pkg/sentry/kernel/task_syscall.go
@@ -22,6 +22,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/bits"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal"
"gvisor.dev/gvisor/pkg/metric"
@@ -357,7 +358,7 @@ func (t *Task) doVsyscallInvoke(sysno uintptr, args arch.SyscallArguments, calle
t.Arch().SetReturn(uintptr(rval))
} else {
t.Debugf("vsyscall %d, caller %x: emulated syscall returned error: %v", sysno, t.Arch().Value(caller), err)
- if err == syserror.EFAULT {
+ if linuxerr.Equals(linuxerr.EFAULT, err) {
t.forceSignal(linux.SIGSEGV, false /* unconditional */)
t.SendSignal(SignalInfoPriv(linux.SIGSEGV))
// A return is not emulated in this case.
diff --git a/pkg/sentry/loader/BUILD b/pkg/sentry/loader/BUILD
index 80f862628..54bfed644 100644
--- a/pkg/sentry/loader/BUILD
+++ b/pkg/sentry/loader/BUILD
@@ -20,6 +20,7 @@ go_library(
"//pkg/abi/linux/errno",
"//pkg/context",
"//pkg/cpuid",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/log",
"//pkg/rand",
diff --git a/pkg/sentry/loader/elf.go b/pkg/sentry/loader/elf.go
index 8fc3e2a79..4c7666e33 100644
--- a/pkg/sentry/loader/elf.go
+++ b/pkg/sentry/loader/elf.go
@@ -24,6 +24,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/cpuid"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/arch"
@@ -621,7 +622,7 @@ func loadInitialELF(ctx context.Context, m *mm.MemoryManager, fs *cpuid.FeatureS
func loadInterpreterELF(ctx context.Context, m *mm.MemoryManager, f fsbridge.File, initial loadedELF) (loadedELF, error) {
info, err := parseHeader(ctx, f)
if err != nil {
- if err == syserror.ENOEXEC {
+ if linuxerr.Equals(linuxerr.ENOEXEC, err) {
// Bad interpreter.
err = syserror.ELIBBAD
}
diff --git a/pkg/sentry/mm/BUILD b/pkg/sentry/mm/BUILD
index b417c2da7..69aff21b6 100644
--- a/pkg/sentry/mm/BUILD
+++ b/pkg/sentry/mm/BUILD
@@ -125,6 +125,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/atomicbitops",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/log",
"//pkg/refs",
@@ -156,6 +157,7 @@ go_test(
library = ":mm",
deps = [
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/sentry/arch",
"//pkg/sentry/contexttest",
@@ -163,7 +165,6 @@ go_test(
"//pkg/sentry/memmap",
"//pkg/sentry/pgalloc",
"//pkg/sentry/platform",
- "//pkg/syserror",
"//pkg/usermem",
],
)
diff --git a/pkg/sentry/mm/mm_test.go b/pkg/sentry/mm/mm_test.go
index 1304b0a2f..84cb8158d 100644
--- a/pkg/sentry/mm/mm_test.go
+++ b/pkg/sentry/mm/mm_test.go
@@ -18,6 +18,7 @@ import (
"testing"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/contexttest"
@@ -25,7 +26,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/memmap"
"gvisor.dev/gvisor/pkg/sentry/pgalloc"
"gvisor.dev/gvisor/pkg/sentry/platform"
- "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
)
@@ -171,7 +171,7 @@ func TestIOAfterUnmap(t *testing.T) {
}
n, err = mm.CopyIn(ctx, addr, b, usermem.IOOpts{})
- if err != syserror.EFAULT {
+ if !linuxerr.Equals(linuxerr.EFAULT, err) {
t.Errorf("CopyIn got err %v want EFAULT", err)
}
if n != 0 {
@@ -212,7 +212,7 @@ func TestIOAfterMProtect(t *testing.T) {
// Without IgnorePermissions, CopyOut should no longer succeed.
n, err = mm.CopyOut(ctx, addr, b, usermem.IOOpts{})
- if err != syserror.EFAULT {
+ if !linuxerr.Equals(linuxerr.EFAULT, err) {
t.Errorf("CopyOut got err %v want EFAULT", err)
}
if n != 0 {
@@ -249,7 +249,7 @@ func TestAIOPrepareAfterDestroy(t *testing.T) {
mm.DestroyAIOContext(ctx, id)
// Prepare should fail because aioCtx should be destroyed.
- if err := aioCtx.Prepare(); err != syserror.EINVAL {
+ if err := aioCtx.Prepare(); !linuxerr.Equals(linuxerr.EINVAL, err) {
t.Errorf("aioCtx.Prepare got err %v want nil", err)
} else if err == nil {
aioCtx.CancelPendingRequest()
diff --git a/pkg/sentry/mm/syscalls.go b/pkg/sentry/mm/syscalls.go
index 7ad6b7c21..f46f85eb1 100644
--- a/pkg/sentry/mm/syscalls.go
+++ b/pkg/sentry/mm/syscalls.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/futex"
@@ -855,10 +856,10 @@ func (mm *MemoryManager) MLock(ctx context.Context, addr hostarch.Addr, length u
mm.activeMu.Unlock()
mm.mappingMu.RUnlock()
// Linux: mm/mlock.c:__mlock_posix_error_return()
- if err == syserror.EFAULT {
+ if linuxerr.Equals(linuxerr.EFAULT, err) {
return syserror.ENOMEM
}
- if err == syserror.ENOMEM {
+ if linuxerr.Equals(linuxerr.ENOMEM, err) {
return syserror.EAGAIN
}
return err
diff --git a/pkg/sentry/platform/kvm/bluepill_arm64.go b/pkg/sentry/platform/kvm/bluepill_arm64.go
index 578852c3f..9e5c52923 100644
--- a/pkg/sentry/platform/kvm/bluepill_arm64.go
+++ b/pkg/sentry/platform/kvm/bluepill_arm64.go
@@ -25,29 +25,6 @@ import (
var (
// The action for bluepillSignal is changed by sigaction().
bluepillSignal = unix.SIGILL
-
- // vcpuSErrBounce is the event of system error for bouncing KVM.
- vcpuSErrBounce = kvmVcpuEvents{
- exception: exception{
- sErrPending: 1,
- },
- }
-
- // vcpuSErrNMI is the event of system error to trigger sigbus.
- vcpuSErrNMI = kvmVcpuEvents{
- exception: exception{
- sErrPending: 1,
- sErrHasEsr: 1,
- sErrEsr: _ESR_ELx_SERR_NMI,
- },
- }
-
- // vcpuExtDabt is the event of ext_dabt.
- vcpuExtDabt = kvmVcpuEvents{
- exception: exception{
- extDabtPending: 1,
- },
- }
)
// getTLS returns the value of TPIDR_EL0 register.
diff --git a/pkg/sentry/platform/kvm/bluepill_arm64_unsafe.go b/pkg/sentry/platform/kvm/bluepill_arm64_unsafe.go
index 07fc4f216..f105fdbd0 100644
--- a/pkg/sentry/platform/kvm/bluepill_arm64_unsafe.go
+++ b/pkg/sentry/platform/kvm/bluepill_arm64_unsafe.go
@@ -80,11 +80,18 @@ func getHypercallID(addr uintptr) int {
//
//go:nosplit
func bluepillStopGuest(c *vCPU) {
+ // vcpuSErrBounce is the event of system error for bouncing KVM.
+ vcpuSErrBounce := &kvmVcpuEvents{
+ exception: exception{
+ sErrPending: 1,
+ },
+ }
+
if _, _, errno := unix.RawSyscall( // escapes: no.
unix.SYS_IOCTL,
uintptr(c.fd),
_KVM_SET_VCPU_EVENTS,
- uintptr(unsafe.Pointer(&vcpuSErrBounce))); errno != 0 {
+ uintptr(unsafe.Pointer(vcpuSErrBounce))); errno != 0 {
throw("bounce sErr injection failed")
}
}
@@ -93,12 +100,21 @@ func bluepillStopGuest(c *vCPU) {
//
//go:nosplit
func bluepillSigBus(c *vCPU) {
+ // vcpuSErrNMI is the event of system error to trigger sigbus.
+ vcpuSErrNMI := &kvmVcpuEvents{
+ exception: exception{
+ sErrPending: 1,
+ sErrHasEsr: 1,
+ sErrEsr: _ESR_ELx_SERR_NMI,
+ },
+ }
+
// Host must support ARM64_HAS_RAS_EXTN.
if _, _, errno := unix.RawSyscall( // escapes: no.
unix.SYS_IOCTL,
uintptr(c.fd),
_KVM_SET_VCPU_EVENTS,
- uintptr(unsafe.Pointer(&vcpuSErrNMI))); errno != 0 {
+ uintptr(unsafe.Pointer(vcpuSErrNMI))); errno != 0 {
if errno == unix.EINVAL {
throw("No ARM64_HAS_RAS_EXTN feature in host.")
}
@@ -110,11 +126,18 @@ func bluepillSigBus(c *vCPU) {
//
//go:nosplit
func bluepillExtDabt(c *vCPU) {
+ // vcpuExtDabt is the event of ext_dabt.
+ vcpuExtDabt := &kvmVcpuEvents{
+ exception: exception{
+ extDabtPending: 1,
+ },
+ }
+
if _, _, errno := unix.RawSyscall( // escapes: no.
unix.SYS_IOCTL,
uintptr(c.fd),
_KVM_SET_VCPU_EVENTS,
- uintptr(unsafe.Pointer(&vcpuExtDabt))); errno != 0 {
+ uintptr(unsafe.Pointer(vcpuExtDabt))); errno != 0 {
throw("ext_dabt injection failed")
}
}
diff --git a/pkg/sentry/platform/kvm/machine_arm64_unsafe.go b/pkg/sentry/platform/kvm/machine_arm64_unsafe.go
index 1b0a6e0a7..f6aa519b1 100644
--- a/pkg/sentry/platform/kvm/machine_arm64_unsafe.go
+++ b/pkg/sentry/platform/kvm/machine_arm64_unsafe.go
@@ -140,22 +140,15 @@ func (c *vCPU) initArchState() error {
// vbar_el1
reg.id = _KVM_ARM64_REGS_VBAR_EL1
-
- fromLocation := reflect.ValueOf(ring0.Vectors).Pointer()
- offset := fromLocation & (1<<11 - 1)
- if offset != 0 {
- offset = 1<<11 - offset
- }
-
- toLocation := fromLocation + offset
- data = uint64(ring0.KernelStartAddress | toLocation)
+ vectorLocation := reflect.ValueOf(ring0.Vectors).Pointer()
+ data = uint64(ring0.KernelStartAddress | vectorLocation)
if err := c.setOneRegister(&reg); err != nil {
return err
}
// Use the address of the exception vector table as
// the MMIO address base.
- arm64HypercallMMIOBase = toLocation
+ arm64HypercallMMIOBase = vectorLocation
// Initialize the PCID database.
if hasGuestPCID {
diff --git a/pkg/sentry/socket/hostinet/BUILD b/pkg/sentry/socket/hostinet/BUILD
index 3c6511ead..3950caa0f 100644
--- a/pkg/sentry/socket/hostinet/BUILD
+++ b/pkg/sentry/socket/hostinet/BUILD
@@ -18,6 +18,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fdnotifier",
"//pkg/hostarch",
"//pkg/log",
diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go
index 52ae4bc9c..38cb2c99c 100644
--- a/pkg/sentry/socket/hostinet/socket.go
+++ b/pkg/sentry/socket/hostinet/socket.go
@@ -20,6 +20,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
@@ -67,23 +68,6 @@ type socketOperations struct {
socketOpsCommon
}
-// socketOpsCommon contains the socket operations common to VFS1 and VFS2.
-//
-// +stateify savable
-type socketOpsCommon struct {
- socket.SendReceiveTimeout
-
- family int // Read-only.
- stype linux.SockType // Read-only.
- protocol int // Read-only.
- queue waiter.Queue
-
- // fd is the host socket fd. It must have O_NONBLOCK, so that operations
- // will return EWOULDBLOCK instead of blocking on the host. This allows us to
- // handle blocking behavior independently in the sentry.
- fd int
-}
-
var _ = socket.Socket(&socketOperations{})
func newSocketFile(ctx context.Context, family int, stype linux.SockType, protocol int, fd int, nonblock bool) (*fs.File, *syserr.Error) {
@@ -103,29 +87,6 @@ func newSocketFile(ctx context.Context, family int, stype linux.SockType, protoc
return fs.NewFile(ctx, dirent, fs.FileFlags{NonBlocking: nonblock, Read: true, Write: true, NonSeekable: true}, s), nil
}
-// Release implements fs.FileOperations.Release.
-func (s *socketOpsCommon) Release(context.Context) {
- fdnotifier.RemoveFD(int32(s.fd))
- unix.Close(s.fd)
-}
-
-// Readiness implements waiter.Waitable.Readiness.
-func (s *socketOpsCommon) Readiness(mask waiter.EventMask) waiter.EventMask {
- return fdnotifier.NonBlockingPoll(int32(s.fd), mask)
-}
-
-// EventRegister implements waiter.Waitable.EventRegister.
-func (s *socketOpsCommon) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
- s.queue.EventRegister(e, mask)
- fdnotifier.UpdateFD(int32(s.fd))
-}
-
-// EventUnregister implements waiter.Waitable.EventUnregister.
-func (s *socketOpsCommon) EventUnregister(e *waiter.Entry) {
- s.queue.EventUnregister(e)
- fdnotifier.UpdateFD(int32(s.fd))
-}
-
// Ioctl implements fs.FileOperations.Ioctl.
func (s *socketOperations) Ioctl(ctx context.Context, _ *fs.File, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
return ioctl(ctx, s.fd, io, args)
@@ -177,6 +138,96 @@ func (s *socketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IO
return int64(n), err
}
+// Socket implements socket.Provider.Socket.
+func (p *socketProvider) Socket(t *kernel.Task, stypeflags linux.SockType, protocol int) (*fs.File, *syserr.Error) {
+ // Check that we are using the host network stack.
+ stack := t.NetworkContext()
+ if stack == nil {
+ return nil, nil
+ }
+ if _, ok := stack.(*Stack); !ok {
+ return nil, nil
+ }
+
+ // Only accept TCP and UDP.
+ stype := stypeflags & linux.SOCK_TYPE_MASK
+ switch stype {
+ case unix.SOCK_STREAM:
+ switch protocol {
+ case 0, unix.IPPROTO_TCP:
+ // ok
+ default:
+ return nil, nil
+ }
+ case unix.SOCK_DGRAM:
+ switch protocol {
+ case 0, unix.IPPROTO_UDP:
+ // ok
+ default:
+ return nil, nil
+ }
+ default:
+ return nil, nil
+ }
+
+ // Conservatively ignore all flags specified by the application and add
+ // SOCK_NONBLOCK since socketOperations requires it. Pass a protocol of 0
+ // to simplify the syscall filters, since 0 and IPPROTO_* are equivalent.
+ fd, err := unix.Socket(p.family, int(stype)|unix.SOCK_NONBLOCK|unix.SOCK_CLOEXEC, 0)
+ if err != nil {
+ return nil, syserr.FromError(err)
+ }
+ return newSocketFile(t, p.family, stype, protocol, fd, stypeflags&unix.SOCK_NONBLOCK != 0)
+}
+
+// Pair implements socket.Provider.Pair.
+func (p *socketProvider) Pair(t *kernel.Task, stype linux.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) {
+ // Not supported by AF_INET/AF_INET6.
+ return nil, nil, nil
+}
+
+// LINT.ThenChange(./socket_vfs2.go)
+
+// socketOpsCommon contains the socket operations common to VFS1 and VFS2.
+//
+// +stateify savable
+type socketOpsCommon struct {
+ socket.SendReceiveTimeout
+
+ family int // Read-only.
+ stype linux.SockType // Read-only.
+ protocol int // Read-only.
+ queue waiter.Queue
+
+ // fd is the host socket fd. It must have O_NONBLOCK, so that operations
+ // will return EWOULDBLOCK instead of blocking on the host. This allows us to
+ // handle blocking behavior independently in the sentry.
+ fd int
+}
+
+// Release implements fs.FileOperations.Release.
+func (s *socketOpsCommon) Release(context.Context) {
+ fdnotifier.RemoveFD(int32(s.fd))
+ unix.Close(s.fd)
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (s *socketOpsCommon) Readiness(mask waiter.EventMask) waiter.EventMask {
+ return fdnotifier.NonBlockingPoll(int32(s.fd), mask)
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (s *socketOpsCommon) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ s.queue.EventRegister(e, mask)
+ fdnotifier.UpdateFD(int32(s.fd))
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (s *socketOpsCommon) EventUnregister(e *waiter.Entry) {
+ s.queue.EventUnregister(e)
+ fdnotifier.UpdateFD(int32(s.fd))
+}
+
// Connect implements socket.Socket.Connect.
func (s *socketOpsCommon) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error {
if len(sockaddr) > sizeofSockaddr {
@@ -596,6 +647,17 @@ func (s *socketOpsCommon) SendMsg(t *kernel.Task, src usermem.IOSequence, to []b
return 0, syserr.ErrInvalidArgument
}
+ // If the src is zero-length, call SENDTO directly with a null buffer in
+ // order to generate poll/epoll notifications.
+ if src.NumBytes() == 0 {
+ sysflags := flags | unix.MSG_DONTWAIT
+ n, _, errno := unix.Syscall6(unix.SYS_SENDTO, uintptr(s.fd), 0, 0, uintptr(sysflags), uintptr(firstBytePtr(to)), uintptr(len(to)))
+ if errno != 0 {
+ return 0, syserr.FromError(errno)
+ }
+ return int(n), nil
+ }
+
space := uint64(control.CmsgsSpace(t, controlMessages))
if space > maxControlLen {
space = maxControlLen
@@ -653,7 +715,7 @@ func (s *socketOpsCommon) SendMsg(t *kernel.Task, src usermem.IOSequence, to []b
}
if ch != nil {
if err = t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = syserror.ErrWouldBlock
}
break
@@ -709,56 +771,6 @@ type socketProvider struct {
family int
}
-// Socket implements socket.Provider.Socket.
-func (p *socketProvider) Socket(t *kernel.Task, stypeflags linux.SockType, protocol int) (*fs.File, *syserr.Error) {
- // Check that we are using the host network stack.
- stack := t.NetworkContext()
- if stack == nil {
- return nil, nil
- }
- if _, ok := stack.(*Stack); !ok {
- return nil, nil
- }
-
- // Only accept TCP and UDP.
- stype := stypeflags & linux.SOCK_TYPE_MASK
- switch stype {
- case unix.SOCK_STREAM:
- switch protocol {
- case 0, unix.IPPROTO_TCP:
- // ok
- default:
- return nil, nil
- }
- case unix.SOCK_DGRAM:
- switch protocol {
- case 0, unix.IPPROTO_UDP:
- // ok
- default:
- return nil, nil
- }
- default:
- return nil, nil
- }
-
- // Conservatively ignore all flags specified by the application and add
- // SOCK_NONBLOCK since socketOperations requires it. Pass a protocol of 0
- // to simplify the syscall filters, since 0 and IPPROTO_* are equivalent.
- fd, err := unix.Socket(p.family, int(stype)|unix.SOCK_NONBLOCK|unix.SOCK_CLOEXEC, 0)
- if err != nil {
- return nil, syserr.FromError(err)
- }
- return newSocketFile(t, p.family, stype, protocol, fd, stypeflags&unix.SOCK_NONBLOCK != 0)
-}
-
-// Pair implements socket.Provider.Pair.
-func (p *socketProvider) Pair(t *kernel.Task, stype linux.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) {
- // Not supported by AF_INET/AF_INET6.
- return nil, nil, nil
-}
-
-// LINT.ThenChange(./socket_vfs2.go)
-
func init() {
for _, family := range []int{unix.AF_INET, unix.AF_INET6} {
socket.RegisterProvider(family, &socketProvider{family})
diff --git a/pkg/sentry/socket/netlink/BUILD b/pkg/sentry/socket/netlink/BUILD
index 6b83698ad..ed85404da 100644
--- a/pkg/sentry/socket/netlink/BUILD
+++ b/pkg/sentry/socket/netlink/BUILD
@@ -17,6 +17,7 @@ go_library(
"//pkg/abi/linux/errno",
"//pkg/bits",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/marshal",
"//pkg/marshal/primitive",
diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go
index c9f784cf4..d53f23a9a 100644
--- a/pkg/sentry/socket/netlink/socket.go
+++ b/pkg/sentry/socket/netlink/socket.go
@@ -22,6 +22,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/abi/linux/errno"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal"
"gvisor.dev/gvisor/pkg/marshal/primitive"
@@ -559,7 +560,7 @@ func (s *socketOpsCommon) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags
}
if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
return 0, 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain
}
return 0, 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err)
diff --git a/pkg/sentry/socket/netstack/BUILD b/pkg/sentry/socket/netstack/BUILD
index 96c425619..e828982eb 100644
--- a/pkg/sentry/socket/netstack/BUILD
+++ b/pkg/sentry/socket/netstack/BUILD
@@ -21,6 +21,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/abi/linux/errno",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/log",
"//pkg/marshal",
diff --git a/pkg/sentry/socket/netstack/netstack.go b/pkg/sentry/socket/netstack/netstack.go
index 66d0fcb47..11f75628c 100644
--- a/pkg/sentry/socket/netstack/netstack.go
+++ b/pkg/sentry/socket/netstack/netstack.go
@@ -38,6 +38,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/abi/linux/errno"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/marshal"
@@ -2809,7 +2810,7 @@ func (s *socketOpsCommon) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags
if n > 0 {
return n, msgFlags, senderAddr, senderAddrLen, controlMessages, nil
}
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
return 0, 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain
}
return 0, 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err)
@@ -2877,7 +2878,7 @@ func (s *socketOpsCommon) SendMsg(t *kernel.Task, src usermem.IOSequence, to []b
// became available between when we last checked and when we setup
// the notification.
if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
return int(total), syserr.ErrTryAgain
}
// handleIOError will consume errors from t.Block if needed.
diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go
index 353f4ade0..f5da3c509 100644
--- a/pkg/sentry/socket/socket.go
+++ b/pkg/sentry/socket/socket.go
@@ -659,7 +659,6 @@ func ConvertAddress(family int, addr tcpip.FullAddress) (linux.SockAddr, uint32)
return &out, uint32(sockAddrInet6Size)
case linux.AF_PACKET:
- // TODO(gvisor.dev/issue/173): Return protocol too.
var out linux.SockAddrLink
out.Family = linux.AF_PACKET
out.InterfaceIndex = int32(addr.NIC)
@@ -749,7 +748,6 @@ func AddressAndFamily(addr []byte) (tcpip.FullAddress, uint16, *syserr.Error) {
return tcpip.FullAddress{}, family, syserr.ErrInvalidArgument
}
- // TODO(gvisor.dev/issue/173): Return protocol too.
return tcpip.FullAddress{
NIC: tcpip.NICID(a.InterfaceIndex),
Addr: tcpip.Address(a.HardwareAddr[:header.EthernetAddressSize]),
diff --git a/pkg/sentry/socket/unix/BUILD b/pkg/sentry/socket/unix/BUILD
index c9cbefb3a..5c3cdef6a 100644
--- a/pkg/sentry/socket/unix/BUILD
+++ b/pkg/sentry/socket/unix/BUILD
@@ -39,6 +39,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/hostarch",
"//pkg/log",
diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go
index db7b1affe..8ccdadae9 100644
--- a/pkg/sentry/socket/unix/unix.go
+++ b/pkg/sentry/socket/unix/unix.go
@@ -23,6 +23,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal"
@@ -518,7 +519,7 @@ func (s *socketOpsCommon) SendMsg(t *kernel.Task, src usermem.IOSequence, to []b
}
if err = t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = syserror.ErrWouldBlock
}
break
@@ -719,7 +720,7 @@ func (s *socketOpsCommon) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags
if total > 0 {
err = nil
}
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
return int(total), msgFlags, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain
}
return int(total), msgFlags, nil, 0, socket.ControlMessages{}, syserr.FromError(err)
diff --git a/pkg/sentry/socket/unix/unix_vfs2.go b/pkg/sentry/socket/unix/unix_vfs2.go
index c39e317ff..08a00a12f 100644
--- a/pkg/sentry/socket/unix/unix_vfs2.go
+++ b/pkg/sentry/socket/unix/unix_vfs2.go
@@ -17,6 +17,7 @@ package unix
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal"
@@ -236,7 +237,7 @@ func (s *SocketVFS2) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
Mode: linux.FileMode(linux.S_IFSOCK | uint(stat.Mode)&^t.FSContext().Umask()),
Endpoint: bep,
})
- if err == syserror.EEXIST {
+ if linuxerr.Equals(linuxerr.EEXIST, err) {
return syserr.ErrAddressInUse
}
return syserr.FromError(err)
diff --git a/pkg/sentry/state/BUILD b/pkg/sentry/state/BUILD
index 3e801182c..7f02807c5 100644
--- a/pkg/sentry/state/BUILD
+++ b/pkg/sentry/state/BUILD
@@ -13,6 +13,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/log",
"//pkg/sentry/inet",
"//pkg/sentry/kernel",
@@ -20,7 +21,6 @@ go_library(
"//pkg/sentry/vfs",
"//pkg/sentry/watchdog",
"//pkg/state/statefile",
- "//pkg/syserror",
"@org_golang_x_sys//unix:go_default_library",
],
)
diff --git a/pkg/sentry/state/state.go b/pkg/sentry/state/state.go
index 2f0aba4e2..e9d544f3d 100644
--- a/pkg/sentry/state/state.go
+++ b/pkg/sentry/state/state.go
@@ -20,6 +20,7 @@ import (
"io"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/inet"
"gvisor.dev/gvisor/pkg/sentry/kernel"
@@ -27,7 +28,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sentry/watchdog"
"gvisor.dev/gvisor/pkg/state/statefile"
- "gvisor.dev/gvisor/pkg/syserror"
)
var previousMetadata map[string]string
@@ -88,7 +88,7 @@ func (opts SaveOpts) Save(ctx context.Context, k *kernel.Kernel, w *watchdog.Wat
// ENOSPC is a state file error. This error can only come from
// writing the state file, and not from fs.FileOperations.Fsync
// because we wrap those in kernel.TaskSet.flushWritesToFiles.
- if err == syserror.ENOSPC {
+ if linuxerr.Equals(linuxerr.ENOSPC, err) {
err = ErrStateFile{err}
}
diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD
index b8d1bd415..f2c55588f 100644
--- a/pkg/sentry/syscalls/BUILD
+++ b/pkg/sentry/syscalls/BUILD
@@ -11,6 +11,7 @@ go_library(
visibility = ["//:sandbox"],
deps = [
"//pkg/abi/linux",
+ "//pkg/errors/linuxerr",
"//pkg/sentry/arch",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/epoll",
diff --git a/pkg/sentry/syscalls/epoll.go b/pkg/sentry/syscalls/epoll.go
index 3b4d79889..02debfc7e 100644
--- a/pkg/sentry/syscalls/epoll.go
+++ b/pkg/sentry/syscalls/epoll.go
@@ -18,6 +18,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/epoll"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
@@ -163,7 +164,7 @@ func WaitEpoll(t *kernel.Task, fd int32, max int, timeoutInNanos int64) ([]linux
}
if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
return nil, nil
}
diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD
index 408a6c422..a2f612f45 100644
--- a/pkg/sentry/syscalls/linux/BUILD
+++ b/pkg/sentry/syscalls/linux/BUILD
@@ -64,6 +64,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/bpf",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/log",
"//pkg/marshal",
diff --git a/pkg/sentry/syscalls/linux/error.go b/pkg/sentry/syscalls/linux/error.go
index 6eabfd219..165922332 100644
--- a/pkg/sentry/syscalls/linux/error.go
+++ b/pkg/sentry/syscalls/linux/error.go
@@ -19,6 +19,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/metric"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -94,13 +95,13 @@ func handleIOErrorImpl(ctx context.Context, partialResult bool, errOrig, intr er
if errno, ok := syserror.TranslateError(errOrig); ok {
translatedErr = errno
}
- switch translatedErr {
- case io.EOF:
+ switch {
+ case translatedErr == io.EOF:
// EOF is always consumed. If this is a partial read/write
// (result != 0), the application will see that, otherwise
// they will see 0.
return true, nil
- case syserror.EFBIG:
+ case linuxerr.Equals(linuxerr.EFBIG, translatedErr):
t := kernel.TaskFromContext(ctx)
if t == nil {
panic("I/O error should only occur from a context associated with a Task")
@@ -113,7 +114,7 @@ func handleIOErrorImpl(ctx context.Context, partialResult bool, errOrig, intr er
// Simultaneously send a SIGXFSZ per setrlimit(2).
t.SendSignal(kernel.SignalInfoNoInfo(linux.SIGXFSZ, t, t))
return true, syserror.EFBIG
- case syserror.EINTR:
+ case linuxerr.Equals(linuxerr.EINTR, translatedErr):
// The syscall was interrupted. Return nil if it completed
// partially, otherwise return the error code that the syscall
// needs (to indicate to the kernel what it should do).
@@ -128,21 +129,21 @@ func handleIOErrorImpl(ctx context.Context, partialResult bool, errOrig, intr er
return true, errOrig
}
- switch translatedErr {
- case syserror.EINTR:
+ switch {
+ case linuxerr.Equals(linuxerr.EINTR, translatedErr):
// Syscall interrupted, but completed a partial
// read/write. Like ErrWouldBlock, since we have a
// partial read/write, we consume the error and return
// the partial result.
return true, nil
- case syserror.EFAULT:
+ case linuxerr.Equals(linuxerr.EFAULT, translatedErr):
// EFAULT is only shown the user if nothing was
// read/written. If we read something (this case), they see
// a partial read/write. They will then presumably try again
// with an incremented buffer, which will EFAULT with
// result == 0.
return true, nil
- case syserror.EPIPE:
+ case linuxerr.Equals(linuxerr.EPIPE, translatedErr):
// Writes to a pipe or socket will return EPIPE if the other
// side is gone. The partial write is returned. EPIPE will be
// returned on the next call.
@@ -150,15 +151,17 @@ func handleIOErrorImpl(ctx context.Context, partialResult bool, errOrig, intr er
// TODO(gvisor.dev/issue/161): In some cases SIGPIPE should
// also be sent to the application.
return true, nil
- case syserror.ENOSPC:
+ case linuxerr.Equals(linuxerr.ENOSPC, translatedErr):
// Similar to EPIPE. Return what we wrote this time, and let
// ENOSPC be returned on the next call.
return true, nil
- case syserror.ECONNRESET, syserror.ETIMEDOUT:
+ case linuxerr.Equals(linuxerr.ECONNRESET, translatedErr):
+ fallthrough
+ case linuxerr.Equals(linuxerr.ETIMEDOUT, translatedErr):
// For TCP sendfile connections, we may have a reset or timeout. But we
// should just return n as the result.
return true, nil
- case syserror.EWOULDBLOCK:
+ case linuxerr.Equals(linuxerr.EWOULDBLOCK, translatedErr):
// Syscall would block, but completed a partial read/write.
// This case should only be returned by IssueIO for nonblocking
// files. Since we have a partial read/write, we consume
diff --git a/pkg/sentry/syscalls/linux/sys_aio.go b/pkg/sentry/syscalls/linux/sys_aio.go
index 70e8569a8..c338a4cc9 100644
--- a/pkg/sentry/syscalls/linux/sys_aio.go
+++ b/pkg/sentry/syscalls/linux/sys_aio.go
@@ -17,6 +17,7 @@ package linux
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/sentry/arch"
@@ -134,7 +135,7 @@ func IoGetevents(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.S
var err error
v, err = waitForRequest(ctx, t, haveDeadline, deadline)
if err != nil {
- if count > 0 || err == syserror.ETIMEDOUT {
+ if count > 0 || linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
return uintptr(count), nil, nil
}
return 0, nil, syserror.ConvertIntr(err, syserror.EINTR)
diff --git a/pkg/sentry/syscalls/linux/sys_file.go b/pkg/sentry/syscalls/linux/sys_file.go
index 90a719ba2..6109a2d8c 100644
--- a/pkg/sentry/syscalls/linux/sys_file.go
+++ b/pkg/sentry/syscalls/linux/sys_file.go
@@ -18,6 +18,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/sentry/arch"
@@ -394,8 +395,8 @@ func createAt(t *kernel.Task, dirFD int32, addr hostarch.Addr, flags uint, mode
}
var newFile *fs.File
- switch err {
- case nil:
+ switch {
+ case err == nil:
// Like sys_open, check for a few things about the
// filesystem before trying to get a reference to the
// fs.File. The same constraints on Check apply.
@@ -418,7 +419,7 @@ func createAt(t *kernel.Task, dirFD int32, addr hostarch.Addr, flags uint, mode
return syserror.ConvertIntr(err, syserror.ERESTARTSYS)
}
defer newFile.DecRef(t)
- case syserror.ENOENT:
+ case linuxerr.Equals(linuxerr.ENOENT, err):
// File does not exist. Proceed with creation.
// Do we have write permissions on the parent?
@@ -1178,12 +1179,12 @@ func mkdirAt(t *kernel.Task, dirFD int32, addr hostarch.Addr, mode linux.FileMod
// Does this directory exist already?
remainingTraversals := uint(linux.MaxSymlinkTraversals)
f, err := t.MountNamespace().FindInode(t, root, d, name, &remainingTraversals)
- switch err {
- case nil:
+ switch {
+ case err == nil:
// The directory existed.
defer f.DecRef(t)
return syserror.EEXIST
- case syserror.EACCES:
+ case linuxerr.Equals(linuxerr.EACCES, err):
// Permission denied while walking to the directory.
return err
default:
@@ -1464,7 +1465,7 @@ func readlinkAt(t *kernel.Task, dirFD int32, addr hostarch.Addr, bufAddr hostarc
}
s, err := d.Inode.Readlink(t)
- if err == syserror.ENOLINK {
+ if linuxerr.Equals(linuxerr.ENOLINK, err) {
return syserror.EINVAL
}
if err != nil {
diff --git a/pkg/sentry/syscalls/linux/sys_poll.go b/pkg/sentry/syscalls/linux/sys_poll.go
index da548a14a..024632475 100644
--- a/pkg/sentry/syscalls/linux/sys_poll.go
+++ b/pkg/sentry/syscalls/linux/sys_poll.go
@@ -18,6 +18,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -128,7 +129,7 @@ func pollBlock(t *kernel.Task, pfd []linux.PollFD, timeout time.Duration) (time.
// Wait for a notification.
timeout, err = t.BlockWithTimeout(ch, !forever, timeout)
if err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = nil
}
return timeout, 0, err
@@ -404,7 +405,7 @@ func (p *pollRestartBlock) Restart(t *kernel.Task) (uintptr, error) {
func poll(t *kernel.Task, pfdAddr hostarch.Addr, nfds uint, timeout time.Duration) (uintptr, error) {
remainingTimeout, n, err := doPoll(t, pfdAddr, nfds, timeout)
// On an interrupt poll(2) is restarted with the remaining timeout.
- if err == syserror.EINTR {
+ if linuxerr.Equals(linuxerr.EINTR, err) {
t.SetSyscallRestartBlock(&pollRestartBlock{
pfdAddr: pfdAddr,
nfds: nfds,
@@ -463,7 +464,7 @@ func Ppoll(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall
//
// Note that this means that if err is nil but copyErr is not, copyErr is
// ignored. This is consistent with Linux.
- if err == syserror.EINTR && copyErr == nil {
+ if linuxerr.Equals(linuxerr.EINTR, err) && copyErr == nil {
err = syserror.ERESTARTNOHAND
}
return n, nil, err
@@ -493,7 +494,7 @@ func Select(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscal
n, err := doSelect(t, nfds, readFDs, writeFDs, exceptFDs, timeout)
copyErr := copyOutTimevalRemaining(t, startNs, timeout, timevalAddr)
// See comment in Ppoll.
- if err == syserror.EINTR && copyErr == nil {
+ if linuxerr.Equals(linuxerr.EINTR, err) && copyErr == nil {
err = syserror.ERESTARTNOHAND
}
return n, nil, err
@@ -538,7 +539,7 @@ func Pselect(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sysca
n, err := doSelect(t, nfds, readFDs, writeFDs, exceptFDs, timeout)
copyErr := copyOutTimespecRemaining(t, startNs, timeout, timespecAddr)
// See comment in Ppoll.
- if err == syserror.EINTR && copyErr == nil {
+ if linuxerr.Equals(linuxerr.EINTR, err) && copyErr == nil {
err = syserror.ERESTARTNOHAND
}
return n, nil, err
diff --git a/pkg/sentry/syscalls/linux/sys_prctl.go b/pkg/sentry/syscalls/linux/sys_prctl.go
index 9890dd946..30c15af4a 100644
--- a/pkg/sentry/syscalls/linux/sys_prctl.go
+++ b/pkg/sentry/syscalls/linux/sys_prctl.go
@@ -18,6 +18,7 @@ import (
"fmt"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -98,7 +99,7 @@ func Prctl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall
case linux.PR_SET_NAME:
addr := args[1].Pointer()
name, err := t.CopyInString(addr, linux.TASK_COMM_LEN-1)
- if err != nil && err != syserror.ENAMETOOLONG {
+ if err != nil && !linuxerr.Equals(linuxerr.ENAMETOOLONG, err) {
return 0, nil, err
}
t.SetName(name)
diff --git a/pkg/sentry/syscalls/linux/sys_read.go b/pkg/sentry/syscalls/linux/sys_read.go
index 13e5e3a51..0f9329fe8 100644
--- a/pkg/sentry/syscalls/linux/sys_read.go
+++ b/pkg/sentry/syscalls/linux/sys_read.go
@@ -18,6 +18,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/kernel"
@@ -331,7 +332,7 @@ func readv(t *kernel.Task, f *fs.File, dst usermem.IOSequence) (int64, error) {
// Wait for a notification that we should retry.
if err = t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = syserror.ErrWouldBlock
}
break
diff --git a/pkg/sentry/syscalls/linux/sys_sem.go b/pkg/sentry/syscalls/linux/sys_sem.go
index c84260080..cb320c536 100644
--- a/pkg/sentry/syscalls/linux/sys_sem.go
+++ b/pkg/sentry/syscalls/linux/sys_sem.go
@@ -19,6 +19,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/sentry/arch"
@@ -81,7 +82,7 @@ func Semtimedop(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sy
}
if err := semTimedOp(t, id, ops, true, timeout.ToDuration()); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
return 0, nil, syserror.EAGAIN
}
return 0, nil, err
diff --git a/pkg/sentry/syscalls/linux/sys_signal.go b/pkg/sentry/syscalls/linux/sys_signal.go
index 27a7f7fe1..db763c68e 100644
--- a/pkg/sentry/syscalls/linux/sys_signal.go
+++ b/pkg/sentry/syscalls/linux/sys_signal.go
@@ -19,6 +19,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -90,7 +91,7 @@ func Kill(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallC
}
info.SetPID(int32(target.PIDNamespace().IDOfTask(t)))
info.SetUID(int32(t.Credentials().RealKUID.In(target.UserNamespace()).OrOverflow()))
- if err := target.SendGroupSignal(info); err != syserror.ESRCH {
+ if err := target.SendGroupSignal(info); !linuxerr.Equals(linuxerr.ESRCH, err) {
return 0, nil, err
}
}
@@ -130,7 +131,7 @@ func Kill(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallC
info.SetPID(int32(tg.PIDNamespace().IDOfTask(t)))
info.SetUID(int32(t.Credentials().RealKUID.In(tg.Leader().UserNamespace()).OrOverflow()))
err := tg.SendSignal(info)
- if err == syserror.ESRCH {
+ if linuxerr.Equals(linuxerr.ESRCH, err) {
// ESRCH is ignored because it means the task
// exited while we were iterating. This is a
// race which would not normally exist on
@@ -174,7 +175,7 @@ func Kill(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallC
info.SetPID(int32(tg.PIDNamespace().IDOfTask(t)))
info.SetUID(int32(t.Credentials().RealKUID.In(tg.Leader().UserNamespace()).OrOverflow()))
// See note above regarding ESRCH race above.
- if err := tg.SendSignal(info); err != syserror.ESRCH {
+ if err := tg.SendSignal(info); !linuxerr.Equals(linuxerr.ESRCH, err) {
lastErr = err
}
}
@@ -433,7 +434,7 @@ func RtSigqueueinfo(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kerne
return 0, nil, syserror.EPERM
}
- if err := target.SendGroupSignal(&info); err != syserror.ESRCH {
+ if err := target.SendGroupSignal(&info); !linuxerr.Equals(linuxerr.ESRCH, err) {
return 0, nil, err
}
}
diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go
index e07917613..3bd21a911 100644
--- a/pkg/sentry/syscalls/linux/sys_socket.go
+++ b/pkg/sentry/syscalls/linux/sys_socket.go
@@ -18,6 +18,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal"
"gvisor.dev/gvisor/pkg/marshal/primitive"
@@ -305,7 +306,7 @@ func accept(t *kernel.Task, fd int32, addr hostarch.Addr, addrLen hostarch.Addr,
if peerRequested {
// NOTE(magi): Linux does not give you an error if it can't
// write the data back out so neither do we.
- if err := writeAddress(t, peer, peerLen, addr, addrLen); err == syserror.EINVAL {
+ if err := writeAddress(t, peer, peerLen, addr, addrLen); linuxerr.Equals(linuxerr.EINVAL, err) {
return 0, err
}
}
diff --git a/pkg/sentry/syscalls/linux/sys_time.go b/pkg/sentry/syscalls/linux/sys_time.go
index 5c3b3dee2..2ec74b33a 100644
--- a/pkg/sentry/syscalls/linux/sys_time.go
+++ b/pkg/sentry/syscalls/linux/sys_time.go
@@ -19,6 +19,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/sentry/arch"
@@ -209,11 +210,11 @@ func clockNanosleepUntil(t *kernel.Task, c ktime.Clock, end ktime.Time, rem host
timer.Destroy()
- switch err {
- case syserror.ETIMEDOUT:
+ switch {
+ case linuxerr.Equals(linuxerr.ETIMEDOUT, err):
// Slept for entire timeout.
return nil
- case syserror.ErrInterrupted:
+ case err == syserror.ErrInterrupted:
// Interrupted.
remaining := end.Sub(c.Now())
if remaining <= 0 {
diff --git a/pkg/sentry/syscalls/linux/sys_write.go b/pkg/sentry/syscalls/linux/sys_write.go
index 95bfe6606..cff355550 100644
--- a/pkg/sentry/syscalls/linux/sys_write.go
+++ b/pkg/sentry/syscalls/linux/sys_write.go
@@ -18,6 +18,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/kernel"
@@ -301,7 +302,7 @@ func writev(t *kernel.Task, f *fs.File, src usermem.IOSequence) (int64, error) {
// Wait for a notification that we should retry.
if err = t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = syserror.ErrWouldBlock
}
break
diff --git a/pkg/sentry/syscalls/linux/sys_xattr.go b/pkg/sentry/syscalls/linux/sys_xattr.go
index 28ad6a60e..37fb67f80 100644
--- a/pkg/sentry/syscalls/linux/sys_xattr.go
+++ b/pkg/sentry/syscalls/linux/sys_xattr.go
@@ -18,6 +18,7 @@ import (
"strings"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
@@ -217,7 +218,7 @@ func setXattr(t *kernel.Task, d *fs.Dirent, nameAddr, valueAddr hostarch.Addr, s
func copyInXattrName(t *kernel.Task, nameAddr hostarch.Addr) (string, error) {
name, err := t.CopyInString(nameAddr, linux.XATTR_NAME_MAX+1)
if err != nil {
- if err == syserror.ENAMETOOLONG {
+ if linuxerr.Equals(linuxerr.ENAMETOOLONG, err) {
return "", syserror.ERANGE
}
return "", err
diff --git a/pkg/sentry/syscalls/linux/vfs2/BUILD b/pkg/sentry/syscalls/linux/vfs2/BUILD
index 5ce0bc714..a73f096ff 100644
--- a/pkg/sentry/syscalls/linux/vfs2/BUILD
+++ b/pkg/sentry/syscalls/linux/vfs2/BUILD
@@ -41,6 +41,7 @@ go_library(
"//pkg/abi/linux",
"//pkg/bits",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fspath",
"//pkg/gohacks",
"//pkg/hostarch",
diff --git a/pkg/sentry/syscalls/linux/vfs2/epoll.go b/pkg/sentry/syscalls/linux/vfs2/epoll.go
index 047d955b6..7aff01343 100644
--- a/pkg/sentry/syscalls/linux/vfs2/epoll.go
+++ b/pkg/sentry/syscalls/linux/vfs2/epoll.go
@@ -19,6 +19,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/kernel"
@@ -174,7 +175,7 @@ func waitEpoll(t *kernel.Task, epfd int32, eventsAddr hostarch.Addr, maxEvents i
haveDeadline = true
}
if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = nil
}
return 0, nil, err
diff --git a/pkg/sentry/syscalls/linux/vfs2/poll.go b/pkg/sentry/syscalls/linux/vfs2/poll.go
index a69c80edd..b16773d65 100644
--- a/pkg/sentry/syscalls/linux/vfs2/poll.go
+++ b/pkg/sentry/syscalls/linux/vfs2/poll.go
@@ -19,6 +19,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/kernel"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
@@ -132,7 +133,7 @@ func pollBlock(t *kernel.Task, pfd []linux.PollFD, timeout time.Duration) (time.
// Wait for a notification.
timeout, err = t.BlockWithTimeout(ch, haveTimeout, timeout)
if err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = nil
}
return timeout, 0, err
@@ -410,7 +411,7 @@ func (p *pollRestartBlock) Restart(t *kernel.Task) (uintptr, error) {
func poll(t *kernel.Task, pfdAddr hostarch.Addr, nfds uint, timeout time.Duration) (uintptr, error) {
remainingTimeout, n, err := doPoll(t, pfdAddr, nfds, timeout)
// On an interrupt poll(2) is restarted with the remaining timeout.
- if err == syserror.EINTR {
+ if linuxerr.Equals(linuxerr.EINTR, err) {
t.SetSyscallRestartBlock(&pollRestartBlock{
pfdAddr: pfdAddr,
nfds: nfds,
@@ -462,7 +463,7 @@ func Ppoll(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall
//
// Note that this means that if err is nil but copyErr is not, copyErr is
// ignored. This is consistent with Linux.
- if err == syserror.EINTR && copyErr == nil {
+ if linuxerr.Equals(linuxerr.EINTR, err) && copyErr == nil {
err = syserror.ERESTARTNOHAND
}
return n, nil, err
@@ -492,7 +493,7 @@ func Select(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscal
n, err := doSelect(t, nfds, readFDs, writeFDs, exceptFDs, timeout)
copyErr := copyOutTimevalRemaining(t, startNs, timeout, timevalAddr)
// See comment in Ppoll.
- if err == syserror.EINTR && copyErr == nil {
+ if linuxerr.Equals(linuxerr.EINTR, err) && copyErr == nil {
err = syserror.ERESTARTNOHAND
}
return n, nil, err
@@ -539,7 +540,7 @@ func Pselect(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sysca
n, err := doSelect(t, nfds, readFDs, writeFDs, exceptFDs, timeout)
copyErr := copyOutTimespecRemaining(t, startNs, timeout, timespecAddr)
// See comment in Ppoll.
- if err == syserror.EINTR && copyErr == nil {
+ if linuxerr.Equals(linuxerr.EINTR, err) && copyErr == nil {
err = syserror.ERESTARTNOHAND
}
return n, nil, err
diff --git a/pkg/sentry/syscalls/linux/vfs2/read_write.go b/pkg/sentry/syscalls/linux/vfs2/read_write.go
index b863d7b84..bbfa4c6d7 100644
--- a/pkg/sentry/syscalls/linux/vfs2/read_write.go
+++ b/pkg/sentry/syscalls/linux/vfs2/read_write.go
@@ -18,6 +18,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/kernel"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
@@ -120,7 +121,7 @@ func read(t *kernel.Task, file *vfs.FileDescription, dst usermem.IOSequence, opt
// Wait for a notification that we should retry.
if err = t.BlockWithDeadline(ch, hasDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = syserror.ErrWouldBlock
}
break
@@ -275,7 +276,7 @@ func pread(t *kernel.Task, file *vfs.FileDescription, dst usermem.IOSequence, of
// Wait for a notification that we should retry.
if err = t.BlockWithDeadline(ch, hasDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = syserror.ErrWouldBlock
}
break
@@ -371,7 +372,7 @@ func write(t *kernel.Task, file *vfs.FileDescription, src usermem.IOSequence, op
// Wait for a notification that we should retry.
if err = t.BlockWithDeadline(ch, hasDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = syserror.ErrWouldBlock
}
break
@@ -525,7 +526,7 @@ func pwrite(t *kernel.Task, file *vfs.FileDescription, src usermem.IOSequence, o
// Wait for a notification that we should retry.
if err = t.BlockWithDeadline(ch, hasDeadline, deadline); err != nil {
- if err == syserror.ETIMEDOUT {
+ if linuxerr.Equals(linuxerr.ETIMEDOUT, err) {
err = syserror.ErrWouldBlock
}
break
diff --git a/pkg/sentry/syscalls/linux/vfs2/socket.go b/pkg/sentry/syscalls/linux/vfs2/socket.go
index 69f69e3af..9a4b5e5fc 100644
--- a/pkg/sentry/syscalls/linux/vfs2/socket.go
+++ b/pkg/sentry/syscalls/linux/vfs2/socket.go
@@ -18,6 +18,7 @@ import (
"time"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/marshal"
"gvisor.dev/gvisor/pkg/marshal/primitive"
"gvisor.dev/gvisor/pkg/sentry/arch"
@@ -309,7 +310,7 @@ func accept(t *kernel.Task, fd int32, addr hostarch.Addr, addrLen hostarch.Addr,
if peerRequested {
// NOTE(magi): Linux does not give you an error if it can't
// write the data back out so neither do we.
- if err := writeAddress(t, peer, peerLen, addr, addrLen); err == syserror.EINVAL {
+ if err := writeAddress(t, peer, peerLen, addr, addrLen); linuxerr.Equals(linuxerr.EINVAL, err) {
return 0, err
}
}
diff --git a/pkg/sentry/syscalls/linux/vfs2/xattr.go b/pkg/sentry/syscalls/linux/vfs2/xattr.go
index c261050c6..c779c6465 100644
--- a/pkg/sentry/syscalls/linux/vfs2/xattr.go
+++ b/pkg/sentry/syscalls/linux/vfs2/xattr.go
@@ -18,6 +18,7 @@ import (
"bytes"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/gohacks"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/kernel"
@@ -295,7 +296,7 @@ func Fremovexattr(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.
func copyInXattrName(t *kernel.Task, nameAddr hostarch.Addr) (string, error) {
name, err := t.CopyInString(nameAddr, linux.XATTR_NAME_MAX+1)
if err != nil {
- if err == syserror.ENAMETOOLONG {
+ if linuxerr.Equals(linuxerr.ENAMETOOLONG, err) {
return "", syserror.ERANGE
}
return "", err
diff --git a/pkg/sentry/usage/memory.go b/pkg/sentry/usage/memory.go
index 581862ee2..e7073ec87 100644
--- a/pkg/sentry/usage/memory.go
+++ b/pkg/sentry/usage/memory.go
@@ -132,7 +132,7 @@ func Init() error {
// always be the case for a newly mapped page from /dev/shm. If we obtain
// the shared memory through some other means in the future, we may have to
// explicitly zero the page.
- mmap, err := unix.Mmap(int(file.Fd()), 0, int(RTMemoryStatsSize), unix.PROT_READ|unix.PROT_WRITE, unix.MAP_SHARED)
+ mmap, err := memutil.MapFile(0, RTMemoryStatsSize, unix.PROT_READ|unix.PROT_WRITE, unix.MAP_SHARED, file.Fd(), 0)
if err != nil {
return fmt.Errorf("error mapping usage file: %v", err)
}
diff --git a/pkg/sentry/usage/memory_unsafe.go b/pkg/sentry/usage/memory_unsafe.go
index 9e0014ca0..bc1531b91 100644
--- a/pkg/sentry/usage/memory_unsafe.go
+++ b/pkg/sentry/usage/memory_unsafe.go
@@ -21,7 +21,7 @@ import (
// RTMemoryStatsSize is the size of the RTMemoryStats struct.
var RTMemoryStatsSize = unsafe.Sizeof(RTMemoryStats{})
-// RTMemoryStatsPointer casts the address of the byte slice into a RTMemoryStats pointer.
-func RTMemoryStatsPointer(b []byte) *RTMemoryStats {
- return (*RTMemoryStats)(unsafe.Pointer(&b[0]))
+// RTMemoryStatsPointer casts addr to a RTMemoryStats pointer.
+func RTMemoryStatsPointer(addr uintptr) *RTMemoryStats {
+ return (*RTMemoryStats)(unsafe.Pointer(addr))
}
diff --git a/pkg/sentry/vfs/BUILD b/pkg/sentry/vfs/BUILD
index ac60fe8bf..a2032162d 100644
--- a/pkg/sentry/vfs/BUILD
+++ b/pkg/sentry/vfs/BUILD
@@ -95,6 +95,7 @@ go_library(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/fd",
"//pkg/fdnotifier",
"//pkg/fspath",
@@ -133,6 +134,7 @@ go_test(
deps = [
"//pkg/abi/linux",
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/sentry/contexttest",
"//pkg/sync",
"//pkg/syserror",
diff --git a/pkg/sentry/vfs/file_description.go b/pkg/sentry/vfs/file_description.go
index ef8d8a813..2bc33d424 100644
--- a/pkg/sentry/vfs/file_description.go
+++ b/pkg/sentry/vfs/file_description.go
@@ -20,6 +20,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs/lock"
"gvisor.dev/gvisor/pkg/sentry/fsmetric"
@@ -708,8 +709,8 @@ func (fd *FileDescription) ListXattr(ctx context.Context, size uint64) ([]string
return names, err
}
names, err := fd.impl.ListXattr(ctx, size)
- if err == syserror.ENOTSUP {
- // Linux doesn't actually return ENOTSUP in this case; instead,
+ if linuxerr.Equals(linuxerr.EOPNOTSUPP, err) {
+ // Linux doesn't actually return EOPNOTSUPP in this case; instead,
// fs/xattr.c:vfs_listxattr() falls back to allowing the security
// subsystem to return security extended attributes, which by default
// don't exist.
diff --git a/pkg/sentry/vfs/file_description_impl_util_test.go b/pkg/sentry/vfs/file_description_impl_util_test.go
index 1cd607c0a..566ad856a 100644
--- a/pkg/sentry/vfs/file_description_impl_util_test.go
+++ b/pkg/sentry/vfs/file_description_impl_util_test.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/sentry/contexttest"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
@@ -155,10 +156,10 @@ func TestGenCountFD(t *testing.T) {
}
// Write and PWrite fails.
- if _, err := fd.Write(ctx, ioseq, WriteOptions{}); err != syserror.EIO {
+ if _, err := fd.Write(ctx, ioseq, WriteOptions{}); !linuxerr.Equals(linuxerr.EIO, err) {
t.Errorf("Write: got err %v, wanted %v", err, syserror.EIO)
}
- if _, err := fd.PWrite(ctx, ioseq, 0, WriteOptions{}); err != syserror.EIO {
+ if _, err := fd.PWrite(ctx, ioseq, 0, WriteOptions{}); !linuxerr.Equals(linuxerr.EIO, err) {
t.Errorf("Write: got err %v, wanted %v", err, syserror.EIO)
}
}
@@ -215,10 +216,10 @@ func TestWritable(t *testing.T) {
if n, err := fd.Seek(ctx, 1, linux.SEEK_SET); n != 0 && err != nil {
t.Errorf("Seek: got err (%v, %v), wanted (0, nil)", n, err)
}
- if n, err := fd.Write(ctx, writeIOSeq, WriteOptions{}); n != 0 && err != syserror.EINVAL {
+ if n, err := fd.Write(ctx, writeIOSeq, WriteOptions{}); n != 0 && !linuxerr.Equals(linuxerr.EINVAL, err) {
t.Errorf("Write: got err (%v, %v), wanted (0, EINVAL)", n, err)
}
- if n, err := fd.PWrite(ctx, writeIOSeq, 2, WriteOptions{}); n != 0 && err != syserror.EINVAL {
+ if n, err := fd.PWrite(ctx, writeIOSeq, 2, WriteOptions{}); n != 0 && !linuxerr.Equals(linuxerr.EINVAL, err) {
t.Errorf("PWrite: got err (%v, %v), wanted (0, EINVAL)", n, err)
}
}
diff --git a/pkg/sentry/vfs/vfs.go b/pkg/sentry/vfs/vfs.go
index 87fdcf403..b96de247f 100644
--- a/pkg/sentry/vfs/vfs.go
+++ b/pkg/sentry/vfs/vfs.go
@@ -42,6 +42,7 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/sentry/fsmetric"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
@@ -731,8 +732,8 @@ func (vfs *VirtualFilesystem) ListXattrAt(ctx context.Context, creds *auth.Crede
rp.Release(ctx)
return names, nil
}
- if err == syserror.ENOTSUP {
- // Linux doesn't actually return ENOTSUP in this case; instead,
+ if linuxerr.Equals(linuxerr.EOPNOTSUPP, err) {
+ // Linux doesn't actually return EOPNOTSUPP in this case; instead,
// fs/xattr.c:vfs_listxattr() falls back to allowing the security
// subsystem to return security extended attributes, which by
// default don't exist.
@@ -830,14 +831,14 @@ func (vfs *VirtualFilesystem) MkdirAllAt(ctx context.Context, currentPath string
Path: fspath.Parse(currentPath),
}
stat, err := vfs.StatAt(ctx, creds, pop, &StatOptions{Mask: linux.STATX_TYPE})
- switch err {
- case nil:
+ switch {
+ case err == nil:
if stat.Mask&linux.STATX_TYPE == 0 || stat.Mode&linux.FileTypeMask != linux.ModeDirectory {
return syserror.ENOTDIR
}
// Directory already exists.
return nil
- case syserror.ENOENT:
+ case linuxerr.Equals(linuxerr.ENOENT, err):
// Expected, we will create the dir.
default:
return fmt.Errorf("stat failed for %q during directory creation: %w", currentPath, err)
@@ -871,7 +872,7 @@ func (vfs *VirtualFilesystem) MakeSyntheticMountpoint(ctx context.Context, targe
Root: root,
Start: root,
Path: fspath.Parse(target),
- }, mkdirOpts); err != nil && err != syserror.EEXIST {
+ }, mkdirOpts); err != nil && !linuxerr.Equals(linuxerr.EEXIST, err) {
return fmt.Errorf("failed to create mountpoint %q: %w", target, err)
}
return nil
diff --git a/pkg/sentry/watchdog/watchdog.go b/pkg/sentry/watchdog/watchdog.go
index 8d563d53a..e8f7d1f01 100644
--- a/pkg/sentry/watchdog/watchdog.go
+++ b/pkg/sentry/watchdog/watchdog.go
@@ -77,11 +77,6 @@ var DefaultOpts = Opts{
// trigger it.
const descheduleThreshold = 1 * time.Second
-var (
- stuckStartup = metric.MustCreateNewUint64Metric("/watchdog/stuck_startup_detected", true /* sync */, "Incremented once on startup watchdog timeout")
- stuckTasks = metric.MustCreateNewUint64Metric("/watchdog/stuck_tasks_detected", true /* sync */, "Cumulative count of stuck tasks detected")
-)
-
// Amount of time to wait before dumping the stack to the log again when the same task(s) remains stuck.
var stackDumpSameTaskPeriod = time.Minute
@@ -242,7 +237,6 @@ func (w *Watchdog) waitForStart() {
return
}
- stuckStartup.Increment()
metric.WeirdnessMetric.Increment("watchdog_stuck_startup")
var buf bytes.Buffer
@@ -316,7 +310,6 @@ func (w *Watchdog) runTurn() {
// unless they are surrounded by
// Task.UninterruptibleSleepStart/Finish.
tc = &offender{lastUpdateTime: lastUpdateTime}
- stuckTasks.Increment()
metric.WeirdnessMetric.Increment("watchdog_stuck_tasks")
newTaskFound = true
}
diff --git a/pkg/shim/BUILD b/pkg/shim/BUILD
index fd6127b97..b115556f5 100644
--- a/pkg/shim/BUILD
+++ b/pkg/shim/BUILD
@@ -6,7 +6,9 @@ go_library(
name = "shim",
srcs = [
"api.go",
+ "debug.go",
"epoll.go",
+ "errors.go",
"options.go",
"service.go",
"service_linux.go",
@@ -43,6 +45,8 @@ go_library(
"@com_github_gogo_protobuf//types:go_default_library",
"@com_github_opencontainers_runtime_spec//specs-go:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
+ "@org_golang_google_grpc//codes:go_default_library",
+ "@org_golang_google_grpc//status:go_default_library",
"@org_golang_x_sys//unix:go_default_library",
],
)
@@ -50,10 +54,14 @@ go_library(
go_test(
name = "shim_test",
size = "small",
- srcs = ["service_test.go"],
+ srcs = [
+ "errors_test.go",
+ "service_test.go",
+ ],
library = ":shim",
deps = [
"//pkg/shim/utils",
+ "@com_github_containerd_containerd//errdefs:go_default_library",
"@com_github_opencontainers_runtime_spec//specs-go:go_default_library",
],
)
diff --git a/pkg/shim/debug.go b/pkg/shim/debug.go
new file mode 100644
index 000000000..49f01990e
--- /dev/null
+++ b/pkg/shim/debug.go
@@ -0,0 +1,48 @@
+// Copyright 2021 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
+//
+// https://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 shim
+
+import (
+ "os"
+ "os/signal"
+ "runtime"
+ "sync"
+ "syscall"
+
+ "github.com/containerd/containerd/log"
+)
+
+var once sync.Once
+
+func setDebugSigHandler() {
+ once.Do(func() {
+ dumpCh := make(chan os.Signal, 1)
+ signal.Notify(dumpCh, syscall.SIGUSR2)
+ go func() {
+ buf := make([]byte, 10240)
+ for range dumpCh {
+ for {
+ n := runtime.Stack(buf, true)
+ if n >= len(buf) {
+ buf = make([]byte, 2*len(buf))
+ continue
+ }
+ log.L.Debugf("User requested stack trace:\n%s", buf[:n])
+ }
+ }
+ }()
+ log.L.Debugf("For full process dump run: kill -%d %d", syscall.SIGUSR2, os.Getpid())
+ })
+}
diff --git a/pkg/shim/errors.go b/pkg/shim/errors.go
new file mode 100644
index 000000000..75d036411
--- /dev/null
+++ b/pkg/shim/errors.go
@@ -0,0 +1,59 @@
+// Copyright 2021 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
+//
+// https://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 shim
+
+import (
+ "context"
+ "errors"
+
+ "github.com/containerd/containerd/errdefs"
+ "google.golang.org/grpc/codes"
+ "google.golang.org/grpc/status"
+)
+
+// errToGRPC wraps containerd's ToGRPC error mapper which depends on
+// github.com/pkg/errors to work correctly. Once we upgrade to containerd v1.4,
+// this function can go away and we can use errdefs.ToGRPC directly instead.
+//
+// TODO(gvisor.dev/issue/6232): Remove after upgrading to containerd v1.4
+func errToGRPC(err error) error {
+ if err == nil {
+ return nil
+ }
+ if _, ok := status.FromError(err); ok {
+ return err
+ }
+
+ switch {
+ case errors.Is(err, errdefs.ErrInvalidArgument):
+ return status.Errorf(codes.InvalidArgument, err.Error())
+ case errors.Is(err, errdefs.ErrNotFound):
+ return status.Errorf(codes.NotFound, err.Error())
+ case errors.Is(err, errdefs.ErrAlreadyExists):
+ return status.Errorf(codes.AlreadyExists, err.Error())
+ case errors.Is(err, errdefs.ErrFailedPrecondition):
+ return status.Errorf(codes.FailedPrecondition, err.Error())
+ case errors.Is(err, errdefs.ErrUnavailable):
+ return status.Errorf(codes.Unavailable, err.Error())
+ case errors.Is(err, errdefs.ErrNotImplemented):
+ return status.Errorf(codes.Unimplemented, err.Error())
+ case errors.Is(err, context.Canceled):
+ return status.Errorf(codes.Canceled, err.Error())
+ case errors.Is(err, context.DeadlineExceeded):
+ return status.Errorf(codes.DeadlineExceeded, err.Error())
+ }
+
+ return errdefs.ToGRPC(err)
+}
diff --git a/pkg/shim/errors_test.go b/pkg/shim/errors_test.go
new file mode 100644
index 000000000..3c10866cc
--- /dev/null
+++ b/pkg/shim/errors_test.go
@@ -0,0 +1,47 @@
+// Copyright 2021 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
+//
+// https://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 shim
+
+import (
+ "fmt"
+ "testing"
+
+ "github.com/containerd/containerd/errdefs"
+)
+
+func TestGRPCRoundTripsErrors(t *testing.T) {
+ for _, tc := range []struct {
+ name string
+ err error
+ test func(err error) bool
+ }{
+ {
+ name: "passthrough",
+ err: errdefs.ErrNotFound,
+ test: errdefs.IsNotFound,
+ },
+ {
+ name: "wrapped",
+ err: fmt.Errorf("oh no: %w", errdefs.ErrNotFound),
+ test: errdefs.IsNotFound,
+ },
+ } {
+ t.Run(tc.name, func(t *testing.T) {
+ if err := errdefs.FromGRPC(errToGRPC(tc.err)); !tc.test(err) {
+ t.Errorf("got %+v", err)
+ }
+ })
+ }
+}
diff --git a/pkg/shim/proc/deleted_state.go b/pkg/shim/proc/deleted_state.go
index d9b970c4d..b0bbe4d7e 100644
--- a/pkg/shim/proc/deleted_state.go
+++ b/pkg/shim/proc/deleted_state.go
@@ -22,28 +22,38 @@ import (
"github.com/containerd/console"
"github.com/containerd/containerd/errdefs"
"github.com/containerd/containerd/pkg/process"
+ runc "github.com/containerd/go-runc"
)
type deletedState struct{}
-func (*deletedState) Resize(ws console.WinSize) error {
- return fmt.Errorf("cannot resize a deleted process.ss")
+func (*deletedState) Resize(console.WinSize) error {
+ return fmt.Errorf("cannot resize a deleted container/process")
}
-func (*deletedState) Start(ctx context.Context) error {
- return fmt.Errorf("cannot start a deleted process.ss")
+func (*deletedState) Start(context.Context) error {
+ return fmt.Errorf("cannot start a deleted container/process")
}
-func (*deletedState) Delete(ctx context.Context) error {
- return fmt.Errorf("cannot delete a deleted process.ss: %w", errdefs.ErrNotFound)
+func (*deletedState) Delete(context.Context) error {
+ return fmt.Errorf("cannot delete a deleted container/process: %w", errdefs.ErrNotFound)
}
-func (*deletedState) Kill(ctx context.Context, sig uint32, all bool) error {
- return fmt.Errorf("cannot kill a deleted process.ss: %w", errdefs.ErrNotFound)
+func (*deletedState) Kill(_ context.Context, signal uint32, _ bool) error {
+ return handleStoppedKill(signal)
}
-func (*deletedState) SetExited(status int) {}
+func (*deletedState) SetExited(int) {}
-func (*deletedState) Exec(ctx context.Context, path string, r *ExecConfig) (process.Process, error) {
+func (*deletedState) Exec(context.Context, string, *ExecConfig) (process.Process, error) {
return nil, fmt.Errorf("cannot exec in a deleted state")
}
+
+func (s *deletedState) State(context.Context) (string, error) {
+ // There is no "deleted" state, closest one is stopped.
+ return "stopped", nil
+}
+
+func (s *deletedState) Stats(context.Context, string) (*runc.Stats, error) {
+ return nil, fmt.Errorf("cannot stat a stopped container/process")
+}
diff --git a/pkg/shim/proc/exec.go b/pkg/shim/proc/exec.go
index e7968d9d5..14df3a778 100644
--- a/pkg/shim/proc/exec.go
+++ b/pkg/shim/proc/exec.go
@@ -145,16 +145,13 @@ func (e *execProcess) Kill(ctx context.Context, sig uint32, _ bool) error {
func (e *execProcess) kill(ctx context.Context, sig uint32, _ bool) error {
internalPid := e.internalPid
- if internalPid != 0 {
- if err := e.parent.runtime.Kill(ctx, e.parent.id, int(sig), &runsc.KillOpts{
- Pid: internalPid,
- }); err != nil {
- // If this returns error, consider the process has
- // already stopped.
- //
- // TODO: Fix after signal handling is fixed.
- return fmt.Errorf("%s: %w", err.Error(), errdefs.ErrNotFound)
- }
+ if internalPid == 0 {
+ return nil
+ }
+
+ opts := runsc.KillOpts{Pid: internalPid}
+ if err := e.parent.runtime.Kill(ctx, e.parent.id, int(sig), &opts); err != nil {
+ return fmt.Errorf("%s: %w", err.Error(), errdefs.ErrNotFound)
}
return nil
}
diff --git a/pkg/shim/proc/exec_state.go b/pkg/shim/proc/exec_state.go
index 4dcda8b44..9c6edd3f5 100644
--- a/pkg/shim/proc/exec_state.go
+++ b/pkg/shim/proc/exec_state.go
@@ -34,18 +34,21 @@ type execCreatedState struct {
p *execProcess
}
-func (s *execCreatedState) transition(name string) error {
- switch name {
- case "running":
+func (s *execCreatedState) name() string {
+ return "created"
+}
+
+func (s *execCreatedState) transition(transition stateTransition) {
+ switch transition {
+ case running:
s.p.execState = &execRunningState{p: s.p}
- case "stopped":
+ case stopped:
s.p.execState = &execStoppedState{p: s.p}
- case "deleted":
+ case deleted:
s.p.execState = &deletedState{}
default:
- return fmt.Errorf("invalid state transition %q to %q", stateName(s), name)
+ panic(fmt.Sprintf("invalid state transition %q to %q", s.name(), transition))
}
- return nil
}
func (s *execCreatedState) Resize(ws console.WinSize) error {
@@ -56,14 +59,16 @@ func (s *execCreatedState) Start(ctx context.Context) error {
if err := s.p.start(ctx); err != nil {
return err
}
- return s.transition("running")
+ s.transition(running)
+ return nil
}
func (s *execCreatedState) Delete(ctx context.Context) error {
if err := s.p.delete(ctx); err != nil {
return err
}
- return s.transition("deleted")
+ s.transition(deleted)
+ return nil
}
func (s *execCreatedState) Kill(ctx context.Context, sig uint32, all bool) error {
@@ -72,35 +77,35 @@ func (s *execCreatedState) Kill(ctx context.Context, sig uint32, all bool) error
func (s *execCreatedState) SetExited(status int) {
s.p.setExited(status)
-
- if err := s.transition("stopped"); err != nil {
- panic(err)
- }
+ s.transition(stopped)
}
type execRunningState struct {
p *execProcess
}
-func (s *execRunningState) transition(name string) error {
- switch name {
- case "stopped":
+func (s *execRunningState) name() string {
+ return "running"
+}
+
+func (s *execRunningState) transition(transition stateTransition) {
+ switch transition {
+ case stopped:
s.p.execState = &execStoppedState{p: s.p}
default:
- return fmt.Errorf("invalid state transition %q to %q", stateName(s), name)
+ panic(fmt.Sprintf("invalid state transition %q to %q", s.name(), transition))
}
- return nil
}
func (s *execRunningState) Resize(ws console.WinSize) error {
return s.p.resize(ws)
}
-func (s *execRunningState) Start(ctx context.Context) error {
+func (s *execRunningState) Start(context.Context) error {
return fmt.Errorf("cannot start a running process")
}
-func (s *execRunningState) Delete(ctx context.Context) error {
+func (s *execRunningState) Delete(context.Context) error {
return fmt.Errorf("cannot delete a running process")
}
@@ -110,31 +115,31 @@ func (s *execRunningState) Kill(ctx context.Context, sig uint32, all bool) error
func (s *execRunningState) SetExited(status int) {
s.p.setExited(status)
-
- if err := s.transition("stopped"); err != nil {
- panic(err)
- }
+ s.transition(stopped)
}
type execStoppedState struct {
p *execProcess
}
-func (s *execStoppedState) transition(name string) error {
- switch name {
- case "deleted":
+func (s *execStoppedState) name() string {
+ return "stopped"
+}
+
+func (s *execStoppedState) transition(transition stateTransition) {
+ switch transition {
+ case deleted:
s.p.execState = &deletedState{}
default:
- return fmt.Errorf("invalid state transition %q to %q", stateName(s), name)
+ panic(fmt.Sprintf("invalid state transition %q to %q", s.name(), transition))
}
- return nil
}
-func (s *execStoppedState) Resize(ws console.WinSize) error {
+func (s *execStoppedState) Resize(console.WinSize) error {
return fmt.Errorf("cannot resize a stopped container")
}
-func (s *execStoppedState) Start(ctx context.Context) error {
+func (s *execStoppedState) Start(context.Context) error {
return fmt.Errorf("cannot start a stopped process")
}
@@ -142,13 +147,14 @@ func (s *execStoppedState) Delete(ctx context.Context) error {
if err := s.p.delete(ctx); err != nil {
return err
}
- return s.transition("deleted")
+ s.transition(deleted)
+ return nil
}
-func (s *execStoppedState) Kill(ctx context.Context, sig uint32, all bool) error {
- return s.p.kill(ctx, sig, all)
+func (s *execStoppedState) Kill(_ context.Context, sig uint32, _ bool) error {
+ return handleStoppedKill(sig)
}
-func (s *execStoppedState) SetExited(status int) {
+func (s *execStoppedState) SetExited(int) {
// no op
}
diff --git a/pkg/shim/proc/init.go b/pkg/shim/proc/init.go
index 664465e0d..6bf090813 100644
--- a/pkg/shim/proc/init.go
+++ b/pkg/shim/proc/init.go
@@ -39,6 +39,8 @@ import (
"gvisor.dev/gvisor/pkg/shim/runsc"
)
+const statusStopped = "stopped"
+
// Init represents an initial process for a container.
type Init struct {
wg sync.WaitGroup
@@ -201,10 +203,15 @@ func (p *Init) ExitedAt() time.Time {
func (p *Init) Status(ctx context.Context) (string, error) {
p.mu.Lock()
defer p.mu.Unlock()
+
+ return p.initState.State(ctx)
+}
+
+func (p *Init) state(ctx context.Context) (string, error) {
c, err := p.runtime.State(ctx, p.id)
if err != nil {
if strings.Contains(err.Error(), "does not exist") {
- return "stopped", nil
+ return statusStopped, nil
}
return "", p.runtimeError(err, "OCI runtime state failed")
}
@@ -231,10 +238,7 @@ func (p *Init) start(ctx context.Context) error {
status, err := p.runtime.Wait(context.Background(), p.id)
if err != nil {
log.G(ctx).WithError(err).Errorf("Failed to wait for container %q", p.id)
- // TODO(random-liu): Handle runsc kill error.
- if err := p.killAll(ctx); err != nil {
- log.G(ctx).WithError(err).Errorf("Failed to kill container %q", p.id)
- }
+ p.killAllLocked(ctx)
status = internalErrorCode
}
ExitCh <- Exit{
@@ -255,6 +259,12 @@ func (p *Init) SetExited(status int) {
}
func (p *Init) setExited(status int) {
+ if !p.exited.IsZero() {
+ log.L.Debugf("Status already set to %d, ignoring status: %d", p.status, status)
+ return
+ }
+
+ log.L.Debugf("Setting status: %d", status)
p.exited = time.Now()
p.status = status
p.Platform.ShutdownConsole(context.Background(), p.console)
@@ -270,15 +280,16 @@ func (p *Init) Delete(ctx context.Context) error {
}
func (p *Init) delete(ctx context.Context) error {
- p.killAll(ctx)
+ p.killAllLocked(ctx)
p.wg.Wait()
+
err := p.runtime.Delete(ctx, p.id, nil)
- // ignore errors if a runtime has already deleted the process
- // but we still hold metadata and pipes
- //
- // this is common during a checkpoint, runc will delete the container state
- // after a checkpoint and the container will no longer exist within runc
if err != nil {
+ // ignore errors if a runtime has already deleted the process
+ // but we still hold metadata and pipes
+ //
+ // this is common during a checkpoint, runc will delete the container state
+ // after a checkpoint and the container will no longer exist within runc
if strings.Contains(err.Error(), "does not exist") {
err = nil
} else {
@@ -326,29 +337,24 @@ func (p *Init) Kill(ctx context.Context, signal uint32, all bool) error {
return p.initState.Kill(ctx, signal, all)
}
-func (p *Init) kill(context context.Context, signal uint32, all bool) error {
+func (p *Init) kill(ctx context.Context, signal uint32, all bool) error {
var (
killErr error
backoff = 100 * time.Millisecond
)
- timeout := 1 * time.Second
- for start := time.Now(); time.Now().Sub(start) < timeout; {
- c, err := p.runtime.State(context, p.id)
+ const timeout = time.Second
+ for start := time.Now(); time.Since(start) < timeout; {
+ state, err := p.initState.State(ctx)
if err != nil {
- if strings.Contains(err.Error(), "does not exist") {
- return fmt.Errorf("no such process: %w", errdefs.ErrNotFound)
- }
return p.runtimeError(err, "OCI runtime state failed")
}
// For runsc, signal only works when container is running state.
// If the container is not in running state, directly return
// "no such process"
- if p.convertStatus(c.Status) == "stopped" {
+ if state == statusStopped {
return fmt.Errorf("no such process: %w", errdefs.ErrNotFound)
}
- killErr = p.runtime.Kill(context, p.id, int(signal), &runsc.KillOpts{
- All: all,
- })
+ killErr = p.runtime.Kill(ctx, p.id, int(signal), &runsc.KillOpts{All: all})
if killErr == nil {
return nil
}
@@ -358,22 +364,18 @@ func (p *Init) kill(context context.Context, signal uint32, all bool) error {
return p.runtimeError(killErr, "kill timeout")
}
-// KillAll kills all processes belonging to the init process.
-func (p *Init) KillAll(context context.Context) error {
+// KillAll kills all processes belonging to the init process. If
+// `runsc kill --all` returns error, assume the container has already stopped.
+func (p *Init) KillAll(context context.Context) {
p.mu.Lock()
defer p.mu.Unlock()
- return p.killAll(context)
+ p.killAllLocked(context)
}
-func (p *Init) killAll(context context.Context) error {
- p.runtime.Kill(context, p.id, int(unix.SIGKILL), &runsc.KillOpts{
- All: true,
- })
- // Ignore error handling for `runsc kill --all` for now.
- // * If it doesn't return error, it is good;
- // * If it returns error, consider the container has already stopped.
- // TODO: Fix `runsc kill --all` error handling.
- return nil
+func (p *Init) killAllLocked(context context.Context) {
+ if err := p.runtime.Kill(context, p.id, int(unix.SIGKILL), &runsc.KillOpts{All: true}); err != nil {
+ log.L.Warningf("Ignoring error killing container %q: %v", p.id, err)
+ }
}
// Stdin returns the stdin of the process.
@@ -396,7 +398,6 @@ func (p *Init) Exec(ctx context.Context, path string, r *ExecConfig) (process.Pr
// exec returns a new exec'd process.
func (p *Init) exec(path string, r *ExecConfig) (process.Process, error) {
- // process exec request
var spec specs.Process
if err := json.Unmarshal(r.Spec.Value, &spec); err != nil {
return nil, err
@@ -420,6 +421,17 @@ func (p *Init) exec(path string, r *ExecConfig) (process.Process, error) {
return e, nil
}
+func (p *Init) Stats(ctx context.Context, id string) (*runc.Stats, error) {
+ p.mu.Lock()
+ defer p.mu.Unlock()
+
+ return p.initState.Stats(ctx, id)
+}
+
+func (p *Init) stats(ctx context.Context, id string) (*runc.Stats, error) {
+ return p.Runtime().Stats(ctx, id)
+}
+
// Stdio returns the stdio of the process.
func (p *Init) Stdio() stdio.Stdio {
return p.stdio
@@ -444,7 +456,7 @@ func (p *Init) runtimeError(rErr error, msg string) error {
func (p *Init) convertStatus(status string) string {
if status == "created" && !p.Sandbox && p.status == internalErrorCode {
// Treat start failure state for non-root container as stopped.
- return "stopped"
+ return statusStopped
}
return status
}
diff --git a/pkg/shim/proc/init_state.go b/pkg/shim/proc/init_state.go
index 0065fc385..d65020e76 100644
--- a/pkg/shim/proc/init_state.go
+++ b/pkg/shim/proc/init_state.go
@@ -19,16 +19,39 @@ import (
"context"
"fmt"
- "github.com/containerd/console"
"github.com/containerd/containerd/errdefs"
"github.com/containerd/containerd/pkg/process"
+ runc "github.com/containerd/go-runc"
+ "golang.org/x/sys/unix"
)
+type stateTransition int
+
+const (
+ running stateTransition = iota
+ stopped
+ deleted
+)
+
+func (s stateTransition) String() string {
+ switch s {
+ case running:
+ return "running"
+ case stopped:
+ return "stopped"
+ case deleted:
+ return "deleted"
+ default:
+ panic(fmt.Sprintf("unknown state: %d", s))
+ }
+}
+
type initState interface {
- Resize(console.WinSize) error
Start(context.Context) error
Delete(context.Context) error
Exec(context.Context, string, *ExecConfig) (process.Process, error)
+ State(ctx context.Context) (string, error)
+ Stats(context.Context, string) (*runc.Stats, error)
Kill(context.Context, uint32, bool) error
SetExited(int)
}
@@ -37,22 +60,21 @@ type createdState struct {
p *Init
}
-func (s *createdState) transition(name string) error {
- switch name {
- case "running":
+func (s *createdState) name() string {
+ return "created"
+}
+
+func (s *createdState) transition(transition stateTransition) {
+ switch transition {
+ case running:
s.p.initState = &runningState{p: s.p}
- case "stopped":
- s.p.initState = &stoppedState{p: s.p}
- case "deleted":
+ case stopped:
+ s.p.initState = &stoppedState{process: s.p}
+ case deleted:
s.p.initState = &deletedState{}
default:
- return fmt.Errorf("invalid state transition %q to %q", stateName(s), name)
+ panic(fmt.Sprintf("invalid state transition %q to %q", s.name(), transition))
}
- return nil
-}
-
-func (s *createdState) Resize(ws console.WinSize) error {
- return s.p.resize(ws)
}
func (s *createdState) Start(ctx context.Context) error {
@@ -66,20 +88,20 @@ func (s *createdState) Start(ctx context.Context) error {
if !s.p.Sandbox {
s.p.io.Close()
s.p.setExited(internalErrorCode)
- if err := s.transition("stopped"); err != nil {
- panic(err)
- }
+ s.transition(stopped)
}
return err
}
- return s.transition("running")
+ s.transition(running)
+ return nil
}
func (s *createdState) Delete(ctx context.Context) error {
if err := s.p.delete(ctx); err != nil {
return err
}
- return s.transition("deleted")
+ s.transition(deleted)
+ return nil
}
func (s *createdState) Kill(ctx context.Context, sig uint32, all bool) error {
@@ -88,40 +110,48 @@ func (s *createdState) Kill(ctx context.Context, sig uint32, all bool) error {
func (s *createdState) SetExited(status int) {
s.p.setExited(status)
-
- if err := s.transition("stopped"); err != nil {
- panic(err)
- }
+ s.transition(stopped)
}
func (s *createdState) Exec(ctx context.Context, path string, r *ExecConfig) (process.Process, error) {
return s.p.exec(path, r)
}
+func (s *createdState) State(ctx context.Context) (string, error) {
+ state, err := s.p.state(ctx)
+ if err == nil && state == statusStopped {
+ s.transition(stopped)
+ }
+ return state, err
+}
+
+func (s *createdState) Stats(ctx context.Context, id string) (*runc.Stats, error) {
+ return s.p.stats(ctx, id)
+}
+
type runningState struct {
p *Init
}
-func (s *runningState) transition(name string) error {
- switch name {
- case "stopped":
- s.p.initState = &stoppedState{p: s.p}
- default:
- return fmt.Errorf("invalid state transition %q to %q", stateName(s), name)
- }
- return nil
+func (s *runningState) name() string {
+ return "running"
}
-func (s *runningState) Resize(ws console.WinSize) error {
- return s.p.resize(ws)
+func (s *runningState) transition(transition stateTransition) {
+ switch transition {
+ case stopped:
+ s.p.initState = &stoppedState{process: s.p}
+ default:
+ panic(fmt.Sprintf("invalid state transition %q to %q", s.name(), transition))
+ }
}
func (s *runningState) Start(ctx context.Context) error {
- return fmt.Errorf("cannot start a running process.ss")
+ return fmt.Errorf("cannot start a running container")
}
func (s *runningState) Delete(ctx context.Context) error {
- return fmt.Errorf("cannot delete a running process.ss")
+ return fmt.Errorf("cannot delete a running container")
}
func (s *runningState) Kill(ctx context.Context, sig uint32, all bool) error {
@@ -130,53 +160,81 @@ func (s *runningState) Kill(ctx context.Context, sig uint32, all bool) error {
func (s *runningState) SetExited(status int) {
s.p.setExited(status)
+ s.transition(stopped)
+}
+
+func (s *runningState) Exec(_ context.Context, path string, r *ExecConfig) (process.Process, error) {
+ return s.p.exec(path, r)
+}
- if err := s.transition("stopped"); err != nil {
- panic(err)
+func (s *runningState) State(ctx context.Context) (string, error) {
+ state, err := s.p.state(ctx)
+ if err == nil && state == "stopped" {
+ s.transition(stopped)
}
+ return state, err
}
-func (s *runningState) Exec(ctx context.Context, path string, r *ExecConfig) (process.Process, error) {
- return s.p.exec(path, r)
+func (s *runningState) Stats(ctx context.Context, id string) (*runc.Stats, error) {
+ return s.p.stats(ctx, id)
}
type stoppedState struct {
- p *Init
+ process *Init
}
-func (s *stoppedState) transition(name string) error {
- switch name {
- case "deleted":
- s.p.initState = &deletedState{}
- default:
- return fmt.Errorf("invalid state transition %q to %q", stateName(s), name)
- }
- return nil
+func (s *stoppedState) name() string {
+ return "stopped"
}
-func (s *stoppedState) Resize(ws console.WinSize) error {
- return fmt.Errorf("cannot resize a stopped container")
+func (s *stoppedState) transition(transition stateTransition) {
+ switch transition {
+ case deleted:
+ s.process.initState = &deletedState{}
+ default:
+ panic(fmt.Sprintf("invalid state transition %q to %q", s.name(), transition))
+ }
}
-func (s *stoppedState) Start(ctx context.Context) error {
- return fmt.Errorf("cannot start a stopped process.ss")
+func (s *stoppedState) Start(context.Context) error {
+ return fmt.Errorf("cannot start a stopped container")
}
func (s *stoppedState) Delete(ctx context.Context) error {
- if err := s.p.delete(ctx); err != nil {
+ if err := s.process.delete(ctx); err != nil {
return err
}
- return s.transition("deleted")
+ s.transition(deleted)
+ return nil
}
-func (s *stoppedState) Kill(ctx context.Context, sig uint32, all bool) error {
- return errdefs.ToGRPCf(errdefs.ErrNotFound, "process.ss %s not found", s.p.id)
+func (s *stoppedState) Kill(_ context.Context, signal uint32, _ bool) error {
+ return handleStoppedKill(signal)
}
func (s *stoppedState) SetExited(status int) {
- // no op
+ s.process.setExited(status)
}
-func (s *stoppedState) Exec(ctx context.Context, path string, r *ExecConfig) (process.Process, error) {
+func (s *stoppedState) Exec(context.Context, string, *ExecConfig) (process.Process, error) {
return nil, fmt.Errorf("cannot exec in a stopped state")
}
+
+func (s *stoppedState) State(context.Context) (string, error) {
+ return "stopped", nil
+}
+
+func (s *stoppedState) Stats(context.Context, string) (*runc.Stats, error) {
+ return nil, fmt.Errorf("cannot stat a stopped container")
+}
+
+func handleStoppedKill(signal uint32) error {
+ switch unix.Signal(signal) {
+ case unix.SIGTERM, unix.SIGKILL:
+ // Container is already stopped, so everything inside the container has
+ // already been killed.
+ return nil
+ default:
+ return errdefs.ToGRPCf(errdefs.ErrNotFound, "process not found")
+ }
+}
diff --git a/pkg/shim/proc/proc.go b/pkg/shim/proc/proc.go
index edba3fca5..89ad3f505 100644
--- a/pkg/shim/proc/proc.go
+++ b/pkg/shim/proc/proc.go
@@ -17,23 +17,5 @@
// the sandbox process running the container.
package proc
-import (
- "fmt"
-)
-
// RunscRoot is the path to the root runsc state directory.
const RunscRoot = "/run/containerd/runsc"
-
-func stateName(v interface{}) string {
- switch v.(type) {
- case *runningState, *execRunningState:
- return "running"
- case *createdState, *execCreatedState:
- return "created"
- case *deletedState:
- return "deleted"
- case *stoppedState:
- return "stopped"
- }
- panic(fmt.Errorf("invalid state %v", v))
-}
diff --git a/pkg/shim/runsc/runsc.go b/pkg/shim/runsc/runsc.go
index ff0521d73..888cb0bcb 100644
--- a/pkg/shim/runsc/runsc.go
+++ b/pkg/shim/runsc/runsc.go
@@ -17,6 +17,7 @@
package runsc
import (
+ "bytes"
"context"
"encoding/json"
"fmt"
@@ -73,9 +74,9 @@ type Runsc struct {
// List returns all containers created inside the provided runsc root directory.
func (r *Runsc) List(context context.Context) ([]*runc.Container, error) {
- data, err := cmdOutput(r.command(context, "list", "--format=json"), false)
+ data, stderr, err := cmdOutput(r.command(context, "list", "--format=json"), false)
if err != nil {
- return nil, err
+ return nil, fmt.Errorf("%w: %s", err, stderr)
}
var out []*runc.Container
if err := json.Unmarshal(data, &out); err != nil {
@@ -86,9 +87,9 @@ func (r *Runsc) List(context context.Context) ([]*runc.Container, error) {
// State returns the state for the container provided by id.
func (r *Runsc) State(context context.Context, id string) (*runc.Container, error) {
- data, err := cmdOutput(r.command(context, "state", id), true)
+ data, stderr, err := cmdOutput(r.command(context, "state", id), false)
if err != nil {
- return nil, fmt.Errorf("%s: %s", err, data)
+ return nil, fmt.Errorf("%w: %s", err, stderr)
}
var c runc.Container
if err := json.Unmarshal(data, &c); err != nil {
@@ -142,9 +143,9 @@ func (r *Runsc) Create(context context.Context, id, bundle string, opts *CreateO
}
if cmd.Stdout == nil && cmd.Stderr == nil {
- data, err := cmdOutput(cmd, true)
+ out, _, err := cmdOutput(cmd, true)
if err != nil {
- return fmt.Errorf("%s: %s", err, data)
+ return fmt.Errorf("%w: %s", err, out)
}
return nil
}
@@ -168,15 +169,15 @@ func (r *Runsc) Create(context context.Context, id, bundle string, opts *CreateO
}
func (r *Runsc) Pause(context context.Context, id string) error {
- if _, err := cmdOutput(r.command(context, "pause", id), true); err != nil {
- return fmt.Errorf("unable to pause: %w", err)
+ if out, _, err := cmdOutput(r.command(context, "pause", id), true); err != nil {
+ return fmt.Errorf("unable to pause: %w: %s", err, out)
}
return nil
}
func (r *Runsc) Resume(context context.Context, id string) error {
- if _, err := cmdOutput(r.command(context, "resume", id), true); err != nil {
- return fmt.Errorf("unable to resume: %w", err)
+ if out, _, err := cmdOutput(r.command(context, "resume", id), true); err != nil {
+ return fmt.Errorf("unable to resume: %w: %s", err, out)
}
return nil
}
@@ -189,9 +190,9 @@ func (r *Runsc) Start(context context.Context, id string, cio runc.IO) error {
}
if cmd.Stdout == nil && cmd.Stderr == nil {
- data, err := cmdOutput(cmd, true)
+ out, _, err := cmdOutput(cmd, true)
if err != nil {
- return fmt.Errorf("%s: %s", err, data)
+ return fmt.Errorf("%w: %s", err, out)
}
return nil
}
@@ -221,12 +222,10 @@ type waitResult struct {
}
// Wait will wait for a running container, and return its exit status.
-//
-// TODO(random-liu): Add exec process support.
func (r *Runsc) Wait(context context.Context, id string) (int, error) {
- data, err := cmdOutput(r.command(context, "wait", id), true)
+ data, stderr, err := cmdOutput(r.command(context, "wait", id), false)
if err != nil {
- return 0, fmt.Errorf("%s: %s", err, data)
+ return 0, fmt.Errorf("%w: %s", err, stderr)
}
var res waitResult
if err := json.Unmarshal(data, &res); err != nil {
@@ -294,9 +293,9 @@ func (r *Runsc) Exec(context context.Context, id string, spec specs.Process, opt
opts.Set(cmd)
}
if cmd.Stdout == nil && cmd.Stderr == nil {
- data, err := cmdOutput(cmd, true)
+ out, _, err := cmdOutput(cmd, true)
if err != nil {
- return fmt.Errorf("%s: %s", err, data)
+ return fmt.Errorf("%w: %s", err, out)
}
return nil
}
@@ -391,20 +390,12 @@ func (r *Runsc) Kill(context context.Context, id string, sig int, opts *KillOpts
// Stats return the stats for a container like cpu, memory, and I/O.
func (r *Runsc) Stats(context context.Context, id string) (*runc.Stats, error) {
cmd := r.command(context, "events", "--stats", id)
- rd, err := cmd.StdoutPipe()
- if err != nil {
- return nil, err
- }
- ec, err := Monitor.Start(cmd)
+ data, stderr, err := cmdOutput(cmd, false)
if err != nil {
- return nil, err
+ return nil, fmt.Errorf("%w: %s", err, stderr)
}
- defer func() {
- rd.Close()
- Monitor.Wait(cmd, ec)
- }()
var e runc.Event
- if err := json.NewDecoder(rd).Decode(&e); err != nil {
+ if err := json.Unmarshal(data, &e); err != nil {
log.L.Debugf("Parsing events error: %v", err)
return nil, err
}
@@ -459,9 +450,9 @@ func (r *Runsc) Events(context context.Context, id string, interval time.Duratio
// Ps lists all the processes inside the container returning their pids.
func (r *Runsc) Ps(context context.Context, id string) ([]int, error) {
- data, err := cmdOutput(r.command(context, "ps", "--format", "json", id), true)
+ data, stderr, err := cmdOutput(r.command(context, "ps", "--format", "json", id), false)
if err != nil {
- return nil, fmt.Errorf("%s: %s", err, data)
+ return nil, fmt.Errorf("%w: %s", err, stderr)
}
var pids []int
if err := json.Unmarshal(data, &pids); err != nil {
@@ -472,9 +463,9 @@ func (r *Runsc) Ps(context context.Context, id string) ([]int, error) {
// Top lists all the processes inside the container returning the full ps data.
func (r *Runsc) Top(context context.Context, id string) (*runc.TopResults, error) {
- data, err := cmdOutput(r.command(context, "ps", "--format", "table", id), true)
+ data, stderr, err := cmdOutput(r.command(context, "ps", "--format", "table", id), false)
if err != nil {
- return nil, fmt.Errorf("%s: %s", err, data)
+ return nil, fmt.Errorf("%w: %s", err, stderr)
}
topResults, err := runc.ParsePSOutput(data)
@@ -517,9 +508,9 @@ func (r *Runsc) runOrError(cmd *exec.Cmd) error {
}
return err
}
- data, err := cmdOutput(cmd, true)
+ out, _, err := cmdOutput(cmd, true)
if err != nil {
- return fmt.Errorf("%s: %s", err, data)
+ return fmt.Errorf("%w: %s", err, out)
}
return nil
}
@@ -540,23 +531,29 @@ func (r *Runsc) command(context context.Context, args ...string) *exec.Cmd {
return cmd
}
-func cmdOutput(cmd *exec.Cmd, combined bool) ([]byte, error) {
- b := getBuf()
- defer putBuf(b)
+func cmdOutput(cmd *exec.Cmd, combined bool) ([]byte, []byte, error) {
+ stdout := getBuf()
+ defer putBuf(stdout)
+ cmd.Stdout = stdout
+ cmd.Stderr = stdout
- cmd.Stdout = b
- if combined {
- cmd.Stderr = b
+ var stderr *bytes.Buffer
+ if !combined {
+ stderr = getBuf()
+ defer putBuf(stderr)
+ cmd.Stderr = stderr
}
ec, err := Monitor.Start(cmd)
if err != nil {
- return nil, err
+ return nil, nil, err
}
status, err := Monitor.Wait(cmd, ec)
if err == nil && status != 0 {
- err = fmt.Errorf("%s did not terminate sucessfully", cmd.Args[0])
+ err = fmt.Errorf("%q did not terminate sucessfully", cmd.Args[0])
}
-
- return b.Bytes(), err
+ if stderr == nil {
+ return stdout.Bytes(), nil, err
+ }
+ return stdout.Bytes(), stderr.Bytes(), err
}
diff --git a/pkg/shim/service.go b/pkg/shim/service.go
index 1f9adcb65..0b41f0e72 100644
--- a/pkg/shim/service.go
+++ b/pkg/shim/service.go
@@ -81,8 +81,6 @@ const (
// New returns a new shim service that can be used via GRPC.
func New(ctx context.Context, id string, publisher shim.Publisher, cancel func()) (shim.Shim, error) {
- log.L.Debugf("service.New, id: %s", id)
-
var opts shim.Opts
if ctxOpts := ctx.Value(shim.OptsKey{}); ctxOpts != nil {
opts = ctxOpts.(shim.Opts)
@@ -304,8 +302,6 @@ func (s *service) Cleanup(ctx context.Context) (*taskAPI.DeleteResponse, error)
// Create creates a new initial process and container with the underlying OCI
// runtime.
func (s *service) Create(ctx context.Context, r *taskAPI.CreateTaskRequest) (*taskAPI.CreateTaskResponse, error) {
- log.L.Debugf("Create, id: %s, bundle: %q", r.ID, r.Bundle)
-
s.mu.Lock()
defer s.mu.Unlock()
@@ -396,6 +392,9 @@ func (s *service) Create(ctx context.Context, r *taskAPI.CreateTaskRequest) (*ta
log.L.Debugf("stdout: %s", r.Stdout)
log.L.Debugf("stderr: %s", r.Stderr)
log.L.Debugf("***************************")
+ if log.L.Logger.IsLevelEnabled(logrus.DebugLevel) {
+ setDebugSigHandler()
+ }
}
// Save state before any action is taken to ensure Cleanup() will have all
@@ -453,10 +452,10 @@ func (s *service) Create(ctx context.Context, r *taskAPI.CreateTaskRequest) (*ta
}
process, err := newInit(r.Bundle, filepath.Join(r.Bundle, "work"), ns, s.platform, config, &s.opts, st.Rootfs)
if err != nil {
- return nil, errdefs.ToGRPC(err)
+ return nil, errToGRPC(err)
}
if err := process.Create(ctx, config); err != nil {
- return nil, errdefs.ToGRPC(err)
+ return nil, errToGRPC(err)
}
// Set up OOM notification on the sandbox's cgroup. This is done on
@@ -506,9 +505,6 @@ func (s *service) Delete(ctx context.Context, r *taskAPI.DeleteRequest) (*taskAP
if err != nil {
return nil, err
}
- if p == nil {
- return nil, errdefs.ToGRPCf(errdefs.ErrFailedPrecondition, "container must be created")
- }
if err := p.Delete(ctx); err != nil {
return nil, err
}
@@ -548,7 +544,7 @@ func (s *service) Exec(ctx context.Context, r *taskAPI.ExecProcessRequest) (*typ
Spec: r.Spec,
})
if err != nil {
- return nil, errdefs.ToGRPC(err)
+ return nil, errToGRPC(err)
}
s.mu.Lock()
s.processes[r.ExecID] = process
@@ -569,7 +565,7 @@ func (s *service) ResizePty(ctx context.Context, r *taskAPI.ResizePtyRequest) (*
Height: uint16(r.Height),
}
if err := p.Resize(ws); err != nil {
- return nil, errdefs.ToGRPC(err)
+ return nil, errToGRPC(err)
}
return empty, nil
}
@@ -580,10 +576,12 @@ func (s *service) State(ctx context.Context, r *taskAPI.StateRequest) (*taskAPI.
p, err := s.getProcess(r.ExecID)
if err != nil {
+ log.L.Debugf("State failed to find process: %v", err)
return nil, err
}
st, err := p.Status(ctx)
if err != nil {
+ log.L.Debugf("State failed: %v", err)
return nil, err
}
status := task.StatusUnknown
@@ -596,7 +594,7 @@ func (s *service) State(ctx context.Context, r *taskAPI.StateRequest) (*taskAPI.
status = task.StatusStopped
}
sio := p.Stdio()
- return &taskAPI.StateResponse{
+ res := &taskAPI.StateResponse{
ID: p.ID(),
Bundle: s.bundle,
Pid: uint32(p.Pid()),
@@ -607,7 +605,9 @@ func (s *service) State(ctx context.Context, r *taskAPI.StateRequest) (*taskAPI.
Terminal: sio.Terminal,
ExitStatus: uint32(p.ExitStatus()),
ExitedAt: p.ExitedAt(),
- }, nil
+ }
+ log.L.Debugf("State succeeded, response: %+v", res)
+ return res, nil
}
// Pause the container.
@@ -646,12 +646,11 @@ func (s *service) Kill(ctx context.Context, r *taskAPI.KillRequest) (*types.Empt
if err != nil {
return nil, err
}
- if p == nil {
- return nil, errdefs.ToGRPCf(errdefs.ErrFailedPrecondition, "container must be created")
- }
if err := p.Kill(ctx, r.Signal, r.All); err != nil {
- return nil, errdefs.ToGRPC(err)
+ log.L.Debugf("Kill failed: %v", err)
+ return nil, errToGRPC(err)
}
+ log.L.Debugf("Kill succeeded")
return empty, nil
}
@@ -661,7 +660,7 @@ func (s *service) Pids(ctx context.Context, r *taskAPI.PidsRequest) (*taskAPI.Pi
pids, err := s.getContainerPids(ctx, r.ID)
if err != nil {
- return nil, errdefs.ToGRPC(err)
+ return nil, errToGRPC(err)
}
var processes []*task.ProcessInfo
for _, pid := range pids {
@@ -740,7 +739,7 @@ func (s *service) Stats(ctx context.Context, r *taskAPI.StatsRequest) (*taskAPI.
log.L.Debugf("Stats error, id: %s: container not created", r.ID)
return nil, errdefs.ToGRPCf(errdefs.ErrFailedPrecondition, "container must be created")
}
- stats, err := s.task.Runtime().Stats(ctx, s.id)
+ stats, err := s.task.Stats(ctx, s.id)
if err != nil {
log.L.Debugf("Stats error, id: %s: %v", r.ID, err)
return nil, err
@@ -821,17 +820,17 @@ func (s *service) Wait(ctx context.Context, r *taskAPI.WaitRequest) (*taskAPI.Wa
p, err := s.getProcess(r.ExecID)
if err != nil {
+ log.L.Debugf("Wait failed to find process: %v", err)
return nil, err
}
- if p == nil {
- return nil, errdefs.ToGRPCf(errdefs.ErrFailedPrecondition, "container must be created")
- }
p.Wait()
- return &taskAPI.WaitResponse{
+ res := &taskAPI.WaitResponse{
ExitStatus: uint32(p.ExitStatus()),
ExitedAt: p.ExitedAt(),
- }, nil
+ }
+ log.L.Debugf("Wait succeeded, response: %+v", res)
+ return res, nil
}
func (s *service) processExits(ctx context.Context) {
@@ -848,10 +847,7 @@ func (s *service) checkProcesses(ctx context.Context, e proc.Exit) {
if ip, ok := p.(*proc.Init); ok {
// Ensure all children are killed.
log.L.Debugf("Container init process exited, killing all container processes")
- if err := ip.KillAll(ctx); err != nil {
- log.G(ctx).WithError(err).WithField("id", ip.ID()).
- Error("failed to kill init's children")
- }
+ ip.KillAll(ctx)
}
p.SetExited(e.Status)
s.events <- &events.TaskExit{
@@ -909,9 +905,14 @@ func (s *service) forward(ctx context.Context, publisher shim.Publisher) {
func (s *service) getProcess(execID string) (process.Process, error) {
s.mu.Lock()
defer s.mu.Unlock()
+
if execID == "" {
+ if s.task == nil {
+ return nil, errdefs.ToGRPCf(errdefs.ErrFailedPrecondition, "container must be created")
+ }
return s.task, nil
}
+
p := s.processes[execID]
if p == nil {
return nil, errdefs.ToGRPCf(errdefs.ErrNotFound, "process does not exist %s", execID)
diff --git a/pkg/syserr/BUILD b/pkg/syserr/BUILD
index 7b3160309..5205fa7e4 100644
--- a/pkg/syserr/BUILD
+++ b/pkg/syserr/BUILD
@@ -12,6 +12,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/abi/linux/errno",
+ "//pkg/errors/linuxerr",
"//pkg/syserror",
"//pkg/tcpip",
"@org_golang_x_sys//unix:go_default_library",
diff --git a/pkg/syserr/syserr.go b/pkg/syserr/syserr.go
index fb77ac8bd..7d0a5125b 100644
--- a/pkg/syserr/syserr.go
+++ b/pkg/syserr/syserr.go
@@ -22,6 +22,7 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux/errno"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -54,7 +55,7 @@ func New(message string, linuxTranslation errno.Errno) *Error {
// enables proper blocking semantics. This should temporary address the
// class of blocking bugs that keep popping up with the current state of
// the error space.
- if e == syserror.EWOULDBLOCK {
+ if err.errno == linuxerr.EWOULDBLOCK.Errno() {
e = syserror.ErrWouldBlock
}
linuxBackwardsTranslations[err.errno] = linuxBackwardsTranslation{err: e, ok: true}
diff --git a/pkg/tcpip/BUILD b/pkg/tcpip/BUILD
index ed4d7e958..f00cfd0f5 100644
--- a/pkg/tcpip/BUILD
+++ b/pkg/tcpip/BUILD
@@ -46,7 +46,6 @@ deps_test(
"//pkg/gohacks",
"//pkg/goid",
"//pkg/ilist",
- "//pkg/iovec",
"//pkg/linewriter",
"//pkg/log",
"//pkg/rand",
diff --git a/pkg/tcpip/header/checksum.go b/pkg/tcpip/header/checksum.go
index 6aa9acfa8..e2c85e220 100644
--- a/pkg/tcpip/header/checksum.go
+++ b/pkg/tcpip/header/checksum.go
@@ -18,6 +18,7 @@ package header
import (
"encoding/binary"
+ "fmt"
"gvisor.dev/gvisor/pkg/tcpip"
"gvisor.dev/gvisor/pkg/tcpip/buffer"
@@ -234,3 +235,64 @@ func PseudoHeaderChecksum(protocol tcpip.TransportProtocolNumber, srcAddr tcpip.
return Checksum([]byte{0, uint8(protocol)}, xsum)
}
+
+// checksumUpdate2ByteAlignedUint16 updates a uint16 value in a calculated
+// checksum.
+//
+// The value MUST begin at a 2-byte boundary in the original buffer.
+func checksumUpdate2ByteAlignedUint16(xsum, old, new uint16) uint16 {
+ // As per RFC 1071 page 4,
+ // (4) Incremental Update
+ //
+ // ...
+ //
+ // To update the checksum, simply add the differences of the
+ // sixteen bit integers that have been changed. To see why this
+ // works, observe that every 16-bit integer has an additive inverse
+ // and that addition is associative. From this it follows that
+ // given the original value m, the new value m', and the old
+ // checksum C, the new checksum C' is:
+ //
+ // C' = C + (-m) + m' = C + (m' - m)
+ return ChecksumCombine(xsum, ChecksumCombine(new, ^old))
+}
+
+// checksumUpdate2ByteAlignedAddress updates an address in a calculated
+// checksum.
+//
+// The addresses must have the same length and must contain an even number
+// of bytes. The address MUST begin at a 2-byte boundary in the original buffer.
+func checksumUpdate2ByteAlignedAddress(xsum uint16, old, new tcpip.Address) uint16 {
+ const uint16Bytes = 2
+
+ if len(old) != len(new) {
+ panic(fmt.Sprintf("buffer lengths are different; old = %d, new = %d", len(old), len(new)))
+ }
+
+ if len(old)%uint16Bytes != 0 {
+ panic(fmt.Sprintf("buffer has an odd number of bytes; got = %d", len(old)))
+ }
+
+ // As per RFC 1071 page 4,
+ // (4) Incremental Update
+ //
+ // ...
+ //
+ // To update the checksum, simply add the differences of the
+ // sixteen bit integers that have been changed. To see why this
+ // works, observe that every 16-bit integer has an additive inverse
+ // and that addition is associative. From this it follows that
+ // given the original value m, the new value m', and the old
+ // checksum C, the new checksum C' is:
+ //
+ // C' = C + (-m) + m' = C + (m' - m)
+ for len(old) != 0 {
+ // Convert the 2 byte sequences to uint16 values then apply the increment
+ // update.
+ xsum = checksumUpdate2ByteAlignedUint16(xsum, (uint16(old[0])<<8)+uint16(old[1]), (uint16(new[0])<<8)+uint16(new[1]))
+ old = old[uint16Bytes:]
+ new = new[uint16Bytes:]
+ }
+
+ return xsum
+}
diff --git a/pkg/tcpip/header/checksum_test.go b/pkg/tcpip/header/checksum_test.go
index d267dabd0..3445511f4 100644
--- a/pkg/tcpip/header/checksum_test.go
+++ b/pkg/tcpip/header/checksum_test.go
@@ -23,6 +23,7 @@ import (
"sync"
"testing"
+ "gvisor.dev/gvisor/pkg/tcpip"
"gvisor.dev/gvisor/pkg/tcpip/buffer"
"gvisor.dev/gvisor/pkg/tcpip/header"
)
@@ -256,3 +257,205 @@ func TestICMPv6Checksum(t *testing.T) {
})
}, want, fmt.Sprintf("header: {% x} data {% x}", h, vv.ToView()))
}
+
+func randomAddress(size int) tcpip.Address {
+ s := make([]byte, size)
+ for i := 0; i < size; i++ {
+ s[i] = byte(rand.Uint32())
+ }
+ return tcpip.Address(s)
+}
+
+func TestChecksummableNetworkUpdateAddress(t *testing.T) {
+ tests := []struct {
+ name string
+ update func(header.IPv4, tcpip.Address)
+ }{
+ {
+ name: "SetSourceAddressWithChecksumUpdate",
+ update: header.IPv4.SetSourceAddressWithChecksumUpdate,
+ },
+ {
+ name: "SetDestinationAddressWithChecksumUpdate",
+ update: header.IPv4.SetDestinationAddressWithChecksumUpdate,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ for i := 0; i < 1000; i++ {
+ var origBytes [header.IPv4MinimumSize]byte
+ header.IPv4(origBytes[:]).Encode(&header.IPv4Fields{
+ TOS: 1,
+ TotalLength: header.IPv4MinimumSize,
+ ID: 2,
+ Flags: 3,
+ FragmentOffset: 4,
+ TTL: 5,
+ Protocol: 6,
+ Checksum: 0,
+ SrcAddr: randomAddress(header.IPv4AddressSize),
+ DstAddr: randomAddress(header.IPv4AddressSize),
+ })
+
+ addr := randomAddress(header.IPv4AddressSize)
+
+ bytesCopy := origBytes
+ h := header.IPv4(bytesCopy[:])
+ origXSum := h.CalculateChecksum()
+ h.SetChecksum(^origXSum)
+
+ test.update(h, addr)
+ got := ^h.Checksum()
+ h.SetChecksum(0)
+ want := h.CalculateChecksum()
+ if got != want {
+ t.Errorf("got h.Checksum() = 0x%x, want = 0x%x; originalBytes = 0x%x, new addr = %s", got, want, origBytes, addr)
+ }
+ }
+ })
+ }
+}
+
+func TestChecksummableTransportUpdatePort(t *testing.T) {
+ // The fields in the pseudo header is not tested here so we just use 0.
+ const pseudoHeaderXSum = 0
+
+ tests := []struct {
+ name string
+ transportHdr func(_, _ uint16) (header.ChecksummableTransport, func(uint16) uint16)
+ proto tcpip.TransportProtocolNumber
+ }{
+ {
+ name: "TCP",
+ transportHdr: func(src, dst uint16) (header.ChecksummableTransport, func(uint16) uint16) {
+ h := header.TCP(make([]byte, header.TCPMinimumSize))
+ h.Encode(&header.TCPFields{
+ SrcPort: src,
+ DstPort: dst,
+ SeqNum: 1,
+ AckNum: 2,
+ DataOffset: header.TCPMinimumSize,
+ Flags: 3,
+ WindowSize: 4,
+ Checksum: 0,
+ UrgentPointer: 5,
+ })
+ h.SetChecksum(^h.CalculateChecksum(pseudoHeaderXSum))
+ return h, h.CalculateChecksum
+ },
+ proto: header.TCPProtocolNumber,
+ },
+ {
+ name: "UDP",
+ transportHdr: func(src, dst uint16) (header.ChecksummableTransport, func(uint16) uint16) {
+ h := header.UDP(make([]byte, header.UDPMinimumSize))
+ h.Encode(&header.UDPFields{
+ SrcPort: src,
+ DstPort: dst,
+ Length: 0,
+ Checksum: 0,
+ })
+ h.SetChecksum(^h.CalculateChecksum(pseudoHeaderXSum))
+ return h, h.CalculateChecksum
+ },
+ proto: header.UDPProtocolNumber,
+ },
+ }
+
+ for i := 0; i < 1000; i++ {
+ origSrcPort := uint16(rand.Uint32())
+ origDstPort := uint16(rand.Uint32())
+ newPort := uint16(rand.Uint32())
+
+ t.Run(fmt.Sprintf("OrigSrcPort=%d,OrigDstPort=%d,NewPort=%d", origSrcPort, origDstPort, newPort), func(*testing.T) {
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ for _, subTest := range []struct {
+ name string
+ update func(header.ChecksummableTransport)
+ }{
+ {
+ name: "Source port",
+ update: func(h header.ChecksummableTransport) { h.SetSourcePortWithChecksumUpdate(newPort) },
+ },
+ {
+ name: "Destination port",
+ update: func(h header.ChecksummableTransport) { h.SetDestinationPortWithChecksumUpdate(newPort) },
+ },
+ } {
+ t.Run(subTest.name, func(t *testing.T) {
+ h, calcXSum := test.transportHdr(origSrcPort, origDstPort)
+ subTest.update(h)
+ // TCP and UDP hold the 1s complement of the fully calculated
+ // checksum.
+ got := ^h.Checksum()
+ h.SetChecksum(0)
+
+ if want := calcXSum(pseudoHeaderXSum); got != want {
+ h, _ := test.transportHdr(origSrcPort, origDstPort)
+ t.Errorf("got Checksum() = 0x%x, want = 0x%x; originalBytes = %#v, new port = %d", got, want, h, newPort)
+ }
+ })
+ }
+ })
+ }
+ })
+ }
+}
+
+func TestChecksummableTransportUpdatePseudoHeaderAddress(t *testing.T) {
+ const addressSize = 6
+
+ tests := []struct {
+ name string
+ transportHdr func() header.ChecksummableTransport
+ proto tcpip.TransportProtocolNumber
+ }{
+ {
+ name: "TCP",
+ transportHdr: func() header.ChecksummableTransport { return header.TCP(make([]byte, header.TCPMinimumSize)) },
+ proto: header.TCPProtocolNumber,
+ },
+ {
+ name: "UDP",
+ transportHdr: func() header.ChecksummableTransport { return header.UDP(make([]byte, header.UDPMinimumSize)) },
+ proto: header.UDPProtocolNumber,
+ },
+ }
+
+ for i := 0; i < 1000; i++ {
+ permanent := randomAddress(addressSize)
+ old := randomAddress(addressSize)
+ new := randomAddress(addressSize)
+
+ t.Run(fmt.Sprintf("Permanent=%q,Old=%q,New=%q", permanent, old, new), func(t *testing.T) {
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ for _, fullChecksum := range []bool{true, false} {
+ t.Run(fmt.Sprintf("FullChecksum=%t", fullChecksum), func(t *testing.T) {
+ initialXSum := header.PseudoHeaderChecksum(test.proto, permanent, old, 0)
+ if fullChecksum {
+ // TCP and UDP hold the 1s complement of the fully calculated
+ // checksum.
+ initialXSum = ^initialXSum
+ }
+
+ h := test.transportHdr()
+ h.SetChecksum(initialXSum)
+ h.UpdateChecksumPseudoHeaderAddress(old, new, fullChecksum)
+
+ got := h.Checksum()
+ if fullChecksum {
+ got = ^got
+ }
+ if want := header.PseudoHeaderChecksum(test.proto, permanent, new, 0); got != want {
+ t.Errorf("got Checksum() = 0x%x, want = 0x%x; h = %#v", got, want, h)
+ }
+ })
+ }
+ })
+ }
+ })
+ }
+}
diff --git a/pkg/tcpip/header/interfaces.go b/pkg/tcpip/header/interfaces.go
index 861cbbb70..3a41adfc4 100644
--- a/pkg/tcpip/header/interfaces.go
+++ b/pkg/tcpip/header/interfaces.go
@@ -53,6 +53,31 @@ type Transport interface {
Payload() []byte
}
+// ChecksummableTransport is a Transport that supports checksumming.
+type ChecksummableTransport interface {
+ Transport
+
+ // SetSourcePortWithChecksumUpdate sets the source port and updates
+ // the checksum.
+ //
+ // The receiver's checksum must be a fully calculated checksum.
+ SetSourcePortWithChecksumUpdate(port uint16)
+
+ // SetDestinationPortWithChecksumUpdate sets the destination port and updates
+ // the checksum.
+ //
+ // The receiver's checksum must be a fully calculated checksum.
+ SetDestinationPortWithChecksumUpdate(port uint16)
+
+ // UpdateChecksumPseudoHeaderAddress updates the checksum to reflect an
+ // updated address in the pseudo header.
+ //
+ // If fullChecksum is true, the receiver's checksum field is assumed to hold a
+ // fully calculated checksum. Otherwise, it is assumed to hold a partially
+ // calculated checksum which only reflects the pseudo header.
+ UpdateChecksumPseudoHeaderAddress(old, new tcpip.Address, fullChecksum bool)
+}
+
// Network offers generic methods to query and/or update the fields of the
// header of a network protocol buffer.
type Network interface {
@@ -90,3 +115,16 @@ type Network interface {
// SetTOS sets the values of the "type of service" and "flow label" fields.
SetTOS(t uint8, l uint32)
}
+
+// ChecksummableNetwork is a Network that supports checksumming.
+type ChecksummableNetwork interface {
+ Network
+
+ // SetSourceAddressAndChecksum sets the source address and updates the
+ // checksum to reflect the new address.
+ SetSourceAddressWithChecksumUpdate(tcpip.Address)
+
+ // SetDestinationAddressAndChecksum sets the destination address and
+ // updates the checksum to reflect the new address.
+ SetDestinationAddressWithChecksumUpdate(tcpip.Address)
+}
diff --git a/pkg/tcpip/header/ipv4.go b/pkg/tcpip/header/ipv4.go
index e9abbb709..dcc549c7b 100644
--- a/pkg/tcpip/header/ipv4.go
+++ b/pkg/tcpip/header/ipv4.go
@@ -305,6 +305,18 @@ func (b IPv4) DestinationAddress() tcpip.Address {
return tcpip.Address(b[dstAddr : dstAddr+IPv4AddressSize])
}
+// SetSourceAddressWithChecksumUpdate implements ChecksummableNetwork.
+func (b IPv4) SetSourceAddressWithChecksumUpdate(new tcpip.Address) {
+ b.SetChecksum(^checksumUpdate2ByteAlignedAddress(^b.Checksum(), b.SourceAddress(), new))
+ b.SetSourceAddress(new)
+}
+
+// SetDestinationAddressWithChecksumUpdate implements ChecksummableNetwork.
+func (b IPv4) SetDestinationAddressWithChecksumUpdate(new tcpip.Address) {
+ b.SetChecksum(^checksumUpdate2ByteAlignedAddress(^b.Checksum(), b.DestinationAddress(), new))
+ b.SetDestinationAddress(new)
+}
+
// padIPv4OptionsLength returns the total length for IPv4 options of length l
// after applying padding according to RFC 791:
// The internet header padding is used to ensure that the internet
diff --git a/pkg/tcpip/header/ndp_options.go b/pkg/tcpip/header/ndp_options.go
index d6cad3a94..b1f39e6e6 100644
--- a/pkg/tcpip/header/ndp_options.go
+++ b/pkg/tcpip/header/ndp_options.go
@@ -148,15 +148,10 @@ const (
// NDP option. That is, the length field for NDP options is in units of
// 8 octets, as per RFC 4861 section 4.6.
lengthByteUnits = 8
-)
-var (
// NDPInfiniteLifetime is a value that represents infinity for the
// 4-byte lifetime fields found in various NDP options. Its value is
// (2^32 - 1)s = 4294967295s.
- //
- // This is a variable instead of a constant so that tests can change
- // this value to a smaller value. It should only be modified by tests.
NDPInfiniteLifetime = time.Second * math.MaxUint32
)
diff --git a/pkg/tcpip/header/tcp.go b/pkg/tcpip/header/tcp.go
index 8dabe3354..a75e51a28 100644
--- a/pkg/tcpip/header/tcp.go
+++ b/pkg/tcpip/header/tcp.go
@@ -390,6 +390,35 @@ func (b TCP) EncodePartial(partialChecksum, length uint16, seqnum, acknum uint32
b.SetChecksum(^checksum)
}
+// SetSourcePortWithChecksumUpdate implements ChecksummableTransport.
+func (b TCP) SetSourcePortWithChecksumUpdate(new uint16) {
+ old := b.SourcePort()
+ b.SetSourcePort(new)
+ b.SetChecksum(^checksumUpdate2ByteAlignedUint16(^b.Checksum(), old, new))
+}
+
+// SetDestinationPortWithChecksumUpdate implements ChecksummableTransport.
+func (b TCP) SetDestinationPortWithChecksumUpdate(new uint16) {
+ old := b.DestinationPort()
+ b.SetDestinationPort(new)
+ b.SetChecksum(^checksumUpdate2ByteAlignedUint16(^b.Checksum(), old, new))
+}
+
+// UpdateChecksumPseudoHeaderAddress implements ChecksummableTransport.
+func (b TCP) UpdateChecksumPseudoHeaderAddress(old, new tcpip.Address, fullChecksum bool) {
+ xsum := b.Checksum()
+ if fullChecksum {
+ xsum = ^xsum
+ }
+
+ xsum = checksumUpdate2ByteAlignedAddress(xsum, old, new)
+ if fullChecksum {
+ xsum = ^xsum
+ }
+
+ b.SetChecksum(xsum)
+}
+
// ParseSynOptions parses the options received in a SYN segment and returns the
// relevant ones. opts should point to the option part of the TCP header.
func ParseSynOptions(opts []byte, isAck bool) TCPSynOptions {
diff --git a/pkg/tcpip/header/udp.go b/pkg/tcpip/header/udp.go
index ae9d167ff..f69d53314 100644
--- a/pkg/tcpip/header/udp.go
+++ b/pkg/tcpip/header/udp.go
@@ -130,3 +130,32 @@ func (b UDP) Encode(u *UDPFields) {
binary.BigEndian.PutUint16(b[udpLength:], u.Length)
binary.BigEndian.PutUint16(b[udpChecksum:], u.Checksum)
}
+
+// SetSourcePortWithChecksumUpdate implements ChecksummableTransport.
+func (b UDP) SetSourcePortWithChecksumUpdate(new uint16) {
+ old := b.SourcePort()
+ b.SetSourcePort(new)
+ b.SetChecksum(^checksumUpdate2ByteAlignedUint16(^b.Checksum(), old, new))
+}
+
+// SetDestinationPortWithChecksumUpdate implements ChecksummableTransport.
+func (b UDP) SetDestinationPortWithChecksumUpdate(new uint16) {
+ old := b.DestinationPort()
+ b.SetDestinationPort(new)
+ b.SetChecksum(^checksumUpdate2ByteAlignedUint16(^b.Checksum(), old, new))
+}
+
+// UpdateChecksumPseudoHeaderAddress implements ChecksummableTransport.
+func (b UDP) UpdateChecksumPseudoHeaderAddress(old, new tcpip.Address, fullChecksum bool) {
+ xsum := b.Checksum()
+ if fullChecksum {
+ xsum = ^xsum
+ }
+
+ xsum = checksumUpdate2ByteAlignedAddress(xsum, old, new)
+ if fullChecksum {
+ xsum = ^xsum
+ }
+
+ b.SetChecksum(xsum)
+}
diff --git a/pkg/tcpip/link/fdbased/BUILD b/pkg/tcpip/link/fdbased/BUILD
index d971194e6..1d0163823 100644
--- a/pkg/tcpip/link/fdbased/BUILD
+++ b/pkg/tcpip/link/fdbased/BUILD
@@ -14,7 +14,6 @@ go_library(
],
visibility = ["//visibility:public"],
deps = [
- "//pkg/iovec",
"//pkg/sync",
"//pkg/tcpip",
"//pkg/tcpip/buffer",
diff --git a/pkg/tcpip/link/fdbased/endpoint.go b/pkg/tcpip/link/fdbased/endpoint.go
index 735c28da1..1b56d2b72 100644
--- a/pkg/tcpip/link/fdbased/endpoint.go
+++ b/pkg/tcpip/link/fdbased/endpoint.go
@@ -44,7 +44,6 @@ import (
"sync/atomic"
"golang.org/x/sys/unix"
- "gvisor.dev/gvisor/pkg/iovec"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/tcpip"
"gvisor.dev/gvisor/pkg/tcpip/buffer"
@@ -138,6 +137,20 @@ type endpoint struct {
// gsoKind is the supported kind of GSO.
gsoKind stack.SupportedGSO
+
+ // maxSyscallHeaderBytes has the same meaning as
+ // Options.MaxSyscallHeaderBytes.
+ maxSyscallHeaderBytes uintptr
+
+ // writevMaxIovs is the maximum number of iovecs that may be passed to
+ // rawfile.NonBlockingWriteIovec, as possibly limited by
+ // maxSyscallHeaderBytes. (No analogous limit is defined for
+ // rawfile.NonBlockingSendMMsg, since in that case the maximum number of
+ // iovecs also depends on the number of mmsghdrs. Instead, if sendBatch
+ // encounters a packet whose iovec count is limited by
+ // maxSyscallHeaderBytes, it falls back to writing the packet using writev
+ // via WritePacket.)
+ writevMaxIovs int
}
// Options specify the details about the fd-based endpoint to be created.
@@ -186,6 +199,11 @@ type Options struct {
// RXChecksumOffload if true, indicates that this endpoints capability
// set should include CapabilityRXChecksumOffload.
RXChecksumOffload bool
+
+ // If MaxSyscallHeaderBytes is non-zero, it is the maximum number of bytes
+ // of struct iovec, msghdr, and mmsghdr that may be passed by each host
+ // system call.
+ MaxSyscallHeaderBytes int
}
// fanoutID is used for AF_PACKET based endpoints to enable PACKET_FANOUT
@@ -235,14 +253,25 @@ func New(opts *Options) (stack.LinkEndpoint, error) {
return nil, fmt.Errorf("opts.FD is empty, at least one FD must be specified")
}
+ if opts.MaxSyscallHeaderBytes < 0 {
+ return nil, fmt.Errorf("opts.MaxSyscallHeaderBytes is negative")
+ }
+
e := &endpoint{
- fds: opts.FDs,
- mtu: opts.MTU,
- caps: caps,
- closed: opts.ClosedFunc,
- addr: opts.Address,
- hdrSize: hdrSize,
- packetDispatchMode: opts.PacketDispatchMode,
+ fds: opts.FDs,
+ mtu: opts.MTU,
+ caps: caps,
+ closed: opts.ClosedFunc,
+ addr: opts.Address,
+ hdrSize: hdrSize,
+ packetDispatchMode: opts.PacketDispatchMode,
+ maxSyscallHeaderBytes: uintptr(opts.MaxSyscallHeaderBytes),
+ writevMaxIovs: rawfile.MaxIovs,
+ }
+ if e.maxSyscallHeaderBytes != 0 {
+ if max := int(e.maxSyscallHeaderBytes / rawfile.SizeofIovec); max < e.writevMaxIovs {
+ e.writevMaxIovs = max
+ }
}
// Increment fanoutID to ensure that we don't re-use the same fanoutID for
@@ -470,9 +499,8 @@ func (e *endpoint) WritePacket(r stack.RouteInfo, protocol tcpip.NetworkProtocol
e.AddHeader(r.LocalLinkAddress, r.RemoteLinkAddress, protocol, pkt)
}
- var builder iovec.Builder
-
fd := e.fds[pkt.Hash%uint32(len(e.fds))]
+ var vnetHdrBuf []byte
if e.gsoKind == stack.HWGSOSupported {
vnetHdr := virtioNetHdr{}
if pkt.GSOOptions.Type != stack.GSONone {
@@ -494,71 +522,123 @@ func (e *endpoint) WritePacket(r stack.RouteInfo, protocol tcpip.NetworkProtocol
vnetHdr.gsoSize = pkt.GSOOptions.MSS
}
}
+ vnetHdrBuf = vnetHdr.marshal()
+ }
- vnetHdrBuf := vnetHdr.marshal()
- builder.Add(vnetHdrBuf)
+ views := pkt.Views()
+ numIovecs := len(views)
+ if len(vnetHdrBuf) != 0 {
+ numIovecs++
+ }
+ if numIovecs > e.writevMaxIovs {
+ numIovecs = e.writevMaxIovs
}
- for _, v := range pkt.Views() {
- builder.Add(v)
+ // Allocate small iovec arrays on the stack.
+ var iovecsArr [8]unix.Iovec
+ iovecs := iovecsArr[:0]
+ if numIovecs > len(iovecsArr) {
+ iovecs = make([]unix.Iovec, 0, numIovecs)
+ }
+ iovecs = rawfile.AppendIovecFromBytes(iovecs, vnetHdrBuf, numIovecs)
+ for _, v := range views {
+ iovecs = rawfile.AppendIovecFromBytes(iovecs, v, numIovecs)
}
- return rawfile.NonBlockingWriteIovec(fd, builder.Build())
+ return rawfile.NonBlockingWriteIovec(fd, iovecs)
}
-func (e *endpoint) sendBatch(batchFD int, batch []*stack.PacketBuffer) (int, tcpip.Error) {
+func (e *endpoint) sendBatch(batchFD int, pkts []*stack.PacketBuffer) (int, tcpip.Error) {
// Send a batch of packets through batchFD.
- mmsgHdrs := make([]rawfile.MMsgHdr, 0, len(batch))
- for _, pkt := range batch {
- if e.hdrSize > 0 {
- e.AddHeader(pkt.EgressRoute.LocalLinkAddress, pkt.EgressRoute.RemoteLinkAddress, pkt.NetworkProtocolNumber, pkt)
- }
+ mmsgHdrsStorage := make([]rawfile.MMsgHdr, 0, len(pkts))
+ packets := 0
+ for packets < len(pkts) {
+ mmsgHdrs := mmsgHdrsStorage
+ batch := pkts[packets:]
+ syscallHeaderBytes := uintptr(0)
+ for _, pkt := range batch {
+ if e.hdrSize > 0 {
+ e.AddHeader(pkt.EgressRoute.LocalLinkAddress, pkt.EgressRoute.RemoteLinkAddress, pkt.NetworkProtocolNumber, pkt)
+ }
- var vnetHdrBuf []byte
- if e.gsoKind == stack.HWGSOSupported {
- vnetHdr := virtioNetHdr{}
- if pkt.GSOOptions.Type != stack.GSONone {
- vnetHdr.hdrLen = uint16(pkt.HeaderSize())
- if pkt.GSOOptions.NeedsCsum {
- vnetHdr.flags = _VIRTIO_NET_HDR_F_NEEDS_CSUM
- vnetHdr.csumStart = header.EthernetMinimumSize + pkt.GSOOptions.L3HdrLen
- vnetHdr.csumOffset = pkt.GSOOptions.CsumOffset
- }
- if pkt.GSOOptions.Type != stack.GSONone && uint16(pkt.Data().Size()) > pkt.GSOOptions.MSS {
- switch pkt.GSOOptions.Type {
- case stack.GSOTCPv4:
- vnetHdr.gsoType = _VIRTIO_NET_HDR_GSO_TCPV4
- case stack.GSOTCPv6:
- vnetHdr.gsoType = _VIRTIO_NET_HDR_GSO_TCPV6
- default:
- panic(fmt.Sprintf("Unknown gso type: %v", pkt.GSOOptions.Type))
+ var vnetHdrBuf []byte
+ if e.gsoKind == stack.HWGSOSupported {
+ vnetHdr := virtioNetHdr{}
+ if pkt.GSOOptions.Type != stack.GSONone {
+ vnetHdr.hdrLen = uint16(pkt.HeaderSize())
+ if pkt.GSOOptions.NeedsCsum {
+ vnetHdr.flags = _VIRTIO_NET_HDR_F_NEEDS_CSUM
+ vnetHdr.csumStart = header.EthernetMinimumSize + pkt.GSOOptions.L3HdrLen
+ vnetHdr.csumOffset = pkt.GSOOptions.CsumOffset
+ }
+ if pkt.GSOOptions.Type != stack.GSONone && uint16(pkt.Data().Size()) > pkt.GSOOptions.MSS {
+ switch pkt.GSOOptions.Type {
+ case stack.GSOTCPv4:
+ vnetHdr.gsoType = _VIRTIO_NET_HDR_GSO_TCPV4
+ case stack.GSOTCPv6:
+ vnetHdr.gsoType = _VIRTIO_NET_HDR_GSO_TCPV6
+ default:
+ panic(fmt.Sprintf("Unknown gso type: %v", pkt.GSOOptions.Type))
+ }
+ vnetHdr.gsoSize = pkt.GSOOptions.MSS
}
- vnetHdr.gsoSize = pkt.GSOOptions.MSS
}
+ vnetHdrBuf = vnetHdr.marshal()
}
- vnetHdrBuf = vnetHdr.marshal()
- }
- var builder iovec.Builder
- builder.Add(vnetHdrBuf)
- for _, v := range pkt.Views() {
- builder.Add(v)
- }
- iovecs := builder.Build()
+ views := pkt.Views()
+ numIovecs := len(views)
+ if len(vnetHdrBuf) != 0 {
+ numIovecs++
+ }
+ if numIovecs > rawfile.MaxIovs {
+ numIovecs = rawfile.MaxIovs
+ }
+ if e.maxSyscallHeaderBytes != 0 {
+ syscallHeaderBytes += rawfile.SizeofMMsgHdr + uintptr(numIovecs)*rawfile.SizeofIovec
+ if syscallHeaderBytes > e.maxSyscallHeaderBytes {
+ // We can't fit this packet into this call to sendmmsg().
+ // We could potentially do so if we reduced numIovecs
+ // further, but this might incur considerable extra
+ // copying. Leave it to the next batch instead.
+ break
+ }
+ }
- var mmsgHdr rawfile.MMsgHdr
- mmsgHdr.Msg.Iov = &iovecs[0]
- mmsgHdr.Msg.SetIovlen((len(iovecs)))
- mmsgHdrs = append(mmsgHdrs, mmsgHdr)
- }
+ // We can't easily allocate iovec arrays on the stack here since
+ // they will escape this loop iteration via mmsgHdrs.
+ iovecs := make([]unix.Iovec, 0, numIovecs)
+ iovecs = rawfile.AppendIovecFromBytes(iovecs, vnetHdrBuf, numIovecs)
+ for _, v := range views {
+ iovecs = rawfile.AppendIovecFromBytes(iovecs, v, numIovecs)
+ }
- packets := 0
- for len(mmsgHdrs) > 0 {
- sent, err := rawfile.NonBlockingSendMMsg(batchFD, mmsgHdrs)
- if err != nil {
- return packets, err
+ var mmsgHdr rawfile.MMsgHdr
+ mmsgHdr.Msg.Iov = &iovecs[0]
+ mmsgHdr.Msg.SetIovlen(len(iovecs))
+ mmsgHdrs = append(mmsgHdrs, mmsgHdr)
+ }
+
+ if len(mmsgHdrs) == 0 {
+ // We can't fit batch[0] into a mmsghdr while staying under
+ // e.maxSyscallHeaderBytes. Use WritePacket, which will avoid the
+ // mmsghdr (by using writev) and re-buffer iovecs more aggressively
+ // if necessary (by using e.writevMaxIovs instead of
+ // rawfile.MaxIovs).
+ pkt := batch[0]
+ if err := e.WritePacket(pkt.EgressRoute, pkt.NetworkProtocolNumber, pkt); err != nil {
+ return packets, err
+ }
+ packets++
+ } else {
+ for len(mmsgHdrs) > 0 {
+ sent, err := rawfile.NonBlockingSendMMsg(batchFD, mmsgHdrs)
+ if err != nil {
+ return packets, err
+ }
+ packets += sent
+ mmsgHdrs = mmsgHdrs[sent:]
+ }
}
- packets += sent
- mmsgHdrs = mmsgHdrs[sent:]
}
return packets, nil
@@ -676,8 +756,9 @@ func NewInjectable(fd int, mtu uint32, capabilities stack.LinkEndpointCapabiliti
unix.SetNonblock(fd, true)
return &InjectableEndpoint{endpoint: endpoint{
- fds: []int{fd},
- mtu: mtu,
- caps: capabilities,
+ fds: []int{fd},
+ mtu: mtu,
+ caps: capabilities,
+ writevMaxIovs: rawfile.MaxIovs,
}}
}
diff --git a/pkg/tcpip/link/rawfile/rawfile_unsafe.go b/pkg/tcpip/link/rawfile/rawfile_unsafe.go
index ba92aedbc..43fe57830 100644
--- a/pkg/tcpip/link/rawfile/rawfile_unsafe.go
+++ b/pkg/tcpip/link/rawfile/rawfile_unsafe.go
@@ -19,12 +19,66 @@
package rawfile
import (
+ "reflect"
"unsafe"
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/tcpip"
)
+// SizeofIovec is the size of a unix.Iovec in bytes.
+const SizeofIovec = unsafe.Sizeof(unix.Iovec{})
+
+// MaxIovs is UIO_MAXIOV, the maximum number of iovecs that may be passed to a
+// host system call in a single array.
+const MaxIovs = 1024
+
+// IovecFromBytes returns a unix.Iovec representing bs.
+//
+// Preconditions: len(bs) > 0.
+func IovecFromBytes(bs []byte) unix.Iovec {
+ iov := unix.Iovec{
+ Base: &bs[0],
+ }
+ iov.SetLen(len(bs))
+ return iov
+}
+
+func bytesFromIovec(iov unix.Iovec) (bs []byte) {
+ sh := (*reflect.SliceHeader)(unsafe.Pointer(&bs))
+ sh.Data = uintptr(unsafe.Pointer(iov.Base))
+ sh.Len = int(iov.Len)
+ sh.Cap = int(iov.Len)
+ return
+}
+
+// AppendIovecFromBytes returns append(iovs, IovecFromBytes(bs)). If len(bs) ==
+// 0, AppendIovecFromBytes returns iovs without modification. If len(iovs) >=
+// max, AppendIovecFromBytes replaces the final iovec in iovs with one that
+// also includes the contents of bs. Note that this implies that
+// AppendIovecFromBytes is only usable when the returned iovec slice is used as
+// the source of a write.
+func AppendIovecFromBytes(iovs []unix.Iovec, bs []byte, max int) []unix.Iovec {
+ if len(bs) == 0 {
+ return iovs
+ }
+ if len(iovs) < max {
+ return append(iovs, IovecFromBytes(bs))
+ }
+ iovs[len(iovs)-1] = IovecFromBytes(append(bytesFromIovec(iovs[len(iovs)-1]), bs...))
+ return iovs
+}
+
+// MMsgHdr represents the mmsg_hdr structure required by recvmmsg() on linux.
+type MMsgHdr struct {
+ Msg unix.Msghdr
+ Len uint32
+ _ [4]byte
+}
+
+// SizeofMMsgHdr is the size of a MMsgHdr in bytes.
+const SizeofMMsgHdr = unsafe.Sizeof(MMsgHdr{})
+
// GetMTU determines the MTU of a network interface device.
func GetMTU(name string) (uint32, error) {
fd, err := unix.Socket(unix.AF_UNIX, unix.SOCK_DGRAM, 0)
@@ -137,13 +191,6 @@ func BlockingReadv(fd int, iovecs []unix.Iovec) (int, tcpip.Error) {
}
}
-// MMsgHdr represents the mmsg_hdr structure required by recvmmsg() on linux.
-type MMsgHdr struct {
- Msg unix.Msghdr
- Len uint32
- _ [4]byte
-}
-
// BlockingRecvMMsg reads from a file descriptor that is set up as non-blocking
// and stores the received messages in a slice of MMsgHdr structures. If no data
// is available, it will block in a poll() syscall until the file descriptor
diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go
index f5693defe..b1aec5312 100644
--- a/pkg/tcpip/network/ipv6/ipv6.go
+++ b/pkg/tcpip/network/ipv6/ipv6.go
@@ -344,7 +344,10 @@ func (e *endpoint) onAddressAssignedLocked(addr tcpip.Address) {
func (e *endpoint) InvalidateDefaultRouter(rtr tcpip.Address) {
e.mu.Lock()
defer e.mu.Unlock()
- e.mu.ndp.invalidateDefaultRouter(rtr)
+
+ // We represent default routers with a default (off-link) route through the
+ // router.
+ e.mu.ndp.invalidateOffLinkRoute(offLinkRoute{dest: header.IPv6EmptySubnet, router: rtr})
}
// SetNDPConfigurations implements NDPEndpoint.
diff --git a/pkg/tcpip/network/ipv6/ndp.go b/pkg/tcpip/network/ipv6/ndp.go
index ee36ed254..9cd283eba 100644
--- a/pkg/tcpip/network/ipv6/ndp.go
+++ b/pkg/tcpip/network/ipv6/ndp.go
@@ -78,13 +78,13 @@ const (
// we cannot have a negative delay.
minimumMaxRtrSolicitationDelay = 0
- // MaxDiscoveredDefaultRouters is the maximum number of discovered
- // default routers. The stack should stop discovering new routers after
- // discovering MaxDiscoveredDefaultRouters routers.
+ // MaxDiscoveredOffLinkRoutes is the maximum number of discovered off-link
+ // routes. The stack should stop discovering new off-link routes after
+ // this limit is reached.
//
// This value MUST be at minimum 2 as per RFC 4861 section 6.3.4, and
// SHOULD be more.
- MaxDiscoveredDefaultRouters = 10
+ MaxDiscoveredOffLinkRoutes = 10
// MaxDiscoveredOnLinkPrefixes is the maximum number of discovered
// on-link prefixes. The stack should stop discovering new on-link
@@ -127,25 +127,17 @@ const (
// maxSLAACAddrLocalRegenAttempts is the maximum number of times to attempt
// SLAAC address regenerations in response to an IPv6 endpoint-local conflict.
maxSLAACAddrLocalRegenAttempts = 10
-)
-var (
// MinPrefixInformationValidLifetimeForUpdate is the minimum Valid
// Lifetime to update the valid lifetime of a generated address by
// SLAAC.
//
- // This is exported as a variable (instead of a constant) so tests
- // can update it to a smaller value.
- //
// Min = 2hrs.
MinPrefixInformationValidLifetimeForUpdate = 2 * time.Hour
// MaxDesyncFactor is the upper bound for the preferred lifetime's desync
// factor for temporary SLAAC addresses.
//
- // This is exported as a variable (instead of a constant) so tests
- // can update it to a smaller value.
- //
// Must be greater than 0.
//
// Max = 10m (from RFC 4941 section 5).
@@ -154,9 +146,6 @@ var (
// MinMaxTempAddrPreferredLifetime is the minimum value allowed for the
// maximum preferred lifetime for temporary SLAAC addresses.
//
- // This is exported as a variable (instead of a constant) so tests
- // can update it to a smaller value.
- //
// This value guarantees that a temporary address is preferred for at
// least 1hr if the SLAAC prefix is valid for at least that time.
MinMaxTempAddrPreferredLifetime = defaultRegenAdvanceDuration + MaxDesyncFactor + time.Hour
@@ -164,9 +153,6 @@ var (
// MinMaxTempAddrValidLifetime is the minimum value allowed for the
// maximum valid lifetime for temporary SLAAC addresses.
//
- // This is exported as a variable (instead of a constant) so tests
- // can update it to a smaller value.
- //
// This value guarantees that a temporary address is valid for at least
// 2hrs if the SLAAC prefix is valid for at least that time.
MinMaxTempAddrValidLifetime = 2 * time.Hour
@@ -218,7 +204,7 @@ type NDPDispatcher interface {
//
// This function is not permitted to block indefinitely. This function
// is also not permitted to call into the stack.
- OnOffLinkRouteUpdated(tcpip.NICID, tcpip.Subnet, tcpip.Address)
+ OnOffLinkRouteUpdated(tcpip.NICID, tcpip.Subnet, tcpip.Address, header.NDPRoutePreference)
// OnOffLinkRouteInvalidated is called when an off-link route is invalidated.
//
@@ -462,6 +448,11 @@ type timer struct {
timer tcpip.Timer
}
+type offLinkRoute struct {
+ dest tcpip.Subnet
+ router tcpip.Address
+}
+
// ndpState is the per-Interface NDP state.
type ndpState struct {
// Do not allow overwriting this state.
@@ -476,8 +467,8 @@ type ndpState struct {
// The DAD timers to send the next NS message, or resolve the address.
dad ip.DAD
- // The default routers discovered through Router Advertisements.
- defaultRouters map[tcpip.Address]defaultRouterState
+ // The off-link routes discovered through Router Advertisements.
+ offLinkRoutes map[offLinkRoute]offLinkRouteState
// rtrSolicitTimer is the timer used to send the next router solicitation
// message.
@@ -505,10 +496,12 @@ type ndpState struct {
temporaryAddressDesyncFactor time.Duration
}
-// defaultRouterState holds data associated with a default router discovered by
+// offLinkRouteState holds data associated with an off-link route discovered by
// a Router Advertisement (RA).
-type defaultRouterState struct {
- // Job to invalidate the default router.
+type offLinkRouteState struct {
+ prf header.NDPRoutePreference
+
+ // Job to invalidate the route.
//
// Must not be nil.
invalidationJob *tcpip.Job
@@ -564,11 +557,11 @@ type slaacPrefixState struct {
// Must not be nil.
invalidationJob *tcpip.Job
- // Nonzero only when the address is not valid forever.
- validUntil tcpip.MonotonicTime
+ // nil iff the address is valid forever.
+ validUntil *tcpip.MonotonicTime
- // Nonzero only when the address is not preferred forever.
- preferredUntil tcpip.MonotonicTime
+ // nil iff the address is preferred forever.
+ preferredUntil *tcpip.MonotonicTime
// State associated with the stable address generated for the prefix.
stableAddr struct {
@@ -726,30 +719,22 @@ func (ndp *ndpState) handleRA(ip tcpip.Address, ra header.NDPRouterAdvert) {
// Is the IPv6 endpoint configured to discover default routers?
if ndp.configs.DiscoverDefaultRouters {
- rtr, ok := ndp.defaultRouters[ip]
- rl := ra.RouterLifetime()
- switch {
- case !ok && rl != 0:
- // This is a new default router we are discovering.
+ prf := ra.DefaultRouterPreference()
+ if prf == header.ReservedRoutePreference {
+ // As per RFC 4191 section 2.2,
//
- // Only remember it if we currently know about less than
- // MaxDiscoveredDefaultRouters routers.
- if len(ndp.defaultRouters) < MaxDiscoveredDefaultRouters {
- ndp.rememberDefaultRouter(ip, rl)
- }
-
- case ok && rl != 0:
- // This is an already discovered default router. Update
- // the invalidation job.
- rtr.invalidationJob.Cancel()
- rtr.invalidationJob.Schedule(rl)
- ndp.defaultRouters[ip] = rtr
-
- case ok && rl == 0:
- // We know about the router but it is no longer to be
- // used as a default router so invalidate it.
- ndp.invalidateDefaultRouter(ip)
+ // Prf (Default Router Preference)
+ //
+ // If the Reserved (10) value is received, the receiver MUST treat the
+ // value as if it were (00).
+ //
+ // Note that the value 00 is the medium (default) router preference value.
+ prf = header.MediumRoutePreference
}
+
+ // We represent default routers with a default (off-link) route through the
+ // router.
+ ndp.handleOffLinkRouteDiscovery(offLinkRoute{dest: header.IPv6EmptySubnet, router: ip}, ra.RouterLifetime(), prf)
}
// TODO(b/141556115): Do (RetransTimer, ReachableTime)) Parameter
@@ -807,51 +792,75 @@ func (ndp *ndpState) handleRA(ip tcpip.Address, ra header.NDPRouterAdvert) {
}
}
-// invalidateDefaultRouter invalidates a discovered default router.
+// invalidateOffLinkRoute invalidates a discovered off-link route.
//
// The IPv6 endpoint that ndp belongs to MUST be locked.
-func (ndp *ndpState) invalidateDefaultRouter(ip tcpip.Address) {
- rtr, ok := ndp.defaultRouters[ip]
-
- // Is the router still discovered?
+func (ndp *ndpState) invalidateOffLinkRoute(route offLinkRoute) {
+ state, ok := ndp.offLinkRoutes[route]
if !ok {
- // ...Nope, do nothing further.
return
}
- rtr.invalidationJob.Cancel()
- delete(ndp.defaultRouters, ip)
+ state.invalidationJob.Cancel()
+ delete(ndp.offLinkRoutes, route)
- // Let the integrator know a discovered default router is invalidated.
+ // Let the integrator know a discovered off-link route is invalidated.
if ndpDisp := ndp.ep.protocol.options.NDPDisp; ndpDisp != nil {
- ndpDisp.OnOffLinkRouteInvalidated(ndp.ep.nic.ID(), header.IPv6EmptySubnet, ip)
+ ndpDisp.OnOffLinkRouteInvalidated(ndp.ep.nic.ID(), route.dest, route.router)
}
}
-// rememberDefaultRouter remembers a newly discovered default router with IPv6
-// link-local address ip with lifetime rl.
-//
-// The router identified by ip MUST NOT already be known by the IPv6 endpoint.
+// handleOffLinkRouteDiscovery handles the discovery of an off-link route.
//
-// The IPv6 endpoint that ndp belongs to MUST be locked.
-func (ndp *ndpState) rememberDefaultRouter(ip tcpip.Address, rl time.Duration) {
+// Precondition: ndp.ep.mu must be locked.
+func (ndp *ndpState) handleOffLinkRouteDiscovery(route offLinkRoute, lifetime time.Duration, prf header.NDPRoutePreference) {
ndpDisp := ndp.ep.protocol.options.NDPDisp
if ndpDisp == nil {
return
}
- // Inform the integrator when we discovered a default router.
- ndpDisp.OnOffLinkRouteUpdated(ndp.ep.nic.ID(), header.IPv6EmptySubnet, ip)
+ state, ok := ndp.offLinkRoutes[route]
+ switch {
+ case !ok && lifetime != 0:
+ // This is a new route we are discovering.
+ //
+ // Only remember it if we currently know about less than
+ // MaxDiscoveredOffLinkRoutes routers.
+ if len(ndp.offLinkRoutes) < MaxDiscoveredOffLinkRoutes {
+ // Inform the integrator when we discovered an off-link route.
+ ndpDisp.OnOffLinkRouteUpdated(ndp.ep.nic.ID(), route.dest, route.router, prf)
+
+ state := offLinkRouteState{
+ prf: prf,
+ invalidationJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() {
+ ndp.invalidateOffLinkRoute(route)
+ }),
+ }
- state := defaultRouterState{
- invalidationJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() {
- ndp.invalidateDefaultRouter(ip)
- }),
- }
+ state.invalidationJob.Schedule(lifetime)
- state.invalidationJob.Schedule(rl)
+ ndp.offLinkRoutes[route] = state
+ }
- ndp.defaultRouters[ip] = state
+ case ok && lifetime != 0:
+ // This is an already discovered off-link route. Update the lifetime.
+ state.invalidationJob.Cancel()
+ state.invalidationJob.Schedule(lifetime)
+
+ if prf != state.prf {
+ state.prf = prf
+
+ // Inform the integrator about route preference updates.
+ ndpDisp.OnOffLinkRouteUpdated(ndp.ep.nic.ID(), route.dest, route.router, prf)
+ }
+
+ ndp.offLinkRoutes[route] = state
+
+ case ok && lifetime == 0:
+ // The already discovered off-link route is no longer considered valid so we
+ // invalidate it immediately.
+ ndp.invalidateOffLinkRoute(route)
+ }
}
// rememberOnLinkPrefix remembers a newly discovered on-link prefix with IPv6
@@ -1040,7 +1049,8 @@ func (ndp *ndpState) doSLAAC(prefix tcpip.Subnet, pl, vl time.Duration) {
// The time an address is preferred until is needed to properly generate the
// address.
if pl < header.NDPInfiniteLifetime {
- state.preferredUntil = now.Add(pl)
+ t := now.Add(pl)
+ state.preferredUntil = &t
}
if !ndp.generateSLAACAddr(prefix, &state) {
@@ -1058,7 +1068,8 @@ func (ndp *ndpState) doSLAAC(prefix tcpip.Subnet, pl, vl time.Duration) {
if vl < header.NDPInfiniteLifetime {
state.invalidationJob.Schedule(vl)
- state.validUntil = now.Add(vl)
+ t := now.Add(vl)
+ state.validUntil = &t
}
// If the address is assigned (DAD resolved), generate a temporary address.
@@ -1163,7 +1174,8 @@ func (ndp *ndpState) generateSLAACAddr(prefix tcpip.Subnet, state *slaacPrefixSt
state.stableAddr.localGenerationFailures++
}
- if addressEndpoint := ndp.addAndAcquireSLAACAddr(generatedAddr, stack.AddressConfigSlaac, ndp.ep.protocol.stack.Clock().NowMonotonic().Sub(state.preferredUntil) >= 0 /* deprecated */); addressEndpoint != nil {
+ deprecated := state.preferredUntil != nil && !state.preferredUntil.After(ndp.ep.protocol.stack.Clock().NowMonotonic())
+ if addressEndpoint := ndp.addAndAcquireSLAACAddr(generatedAddr, stack.AddressConfigSlaac, deprecated); addressEndpoint != nil {
state.stableAddr.addressEndpoint = addressEndpoint
state.generationAttempts++
return true
@@ -1224,7 +1236,7 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla
// address is the lower of the valid lifetime of the stable address or the
// maximum temporary address valid lifetime.
vl := ndp.configs.MaxTempAddrValidLifetime
- if prefixState.validUntil != (tcpip.MonotonicTime{}) {
+ if prefixState.validUntil != nil {
if prefixVL := prefixState.validUntil.Sub(now); vl > prefixVL {
vl = prefixVL
}
@@ -1240,7 +1252,7 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla
// maximum temporary address preferred lifetime - the temporary address desync
// factor.
pl := ndp.configs.MaxTempAddrPreferredLifetime - ndp.temporaryAddressDesyncFactor
- if prefixState.preferredUntil != (tcpip.MonotonicTime{}) {
+ if prefixState.preferredUntil != nil {
if prefixPL := prefixState.preferredUntil.Sub(now); pl > prefixPL {
// Respect the preferred lifetime of the prefix, as per RFC 4941 section
// 3.3 step 4.
@@ -1382,9 +1394,10 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat
if !deprecated {
prefixState.deprecationJob.Schedule(pl)
}
- prefixState.preferredUntil = now.Add(pl)
+ t := now.Add(pl)
+ prefixState.preferredUntil = &t
} else {
- prefixState.preferredUntil = tcpip.MonotonicTime{}
+ prefixState.preferredUntil = nil
}
// As per RFC 4862 section 5.5.3.e, update the valid lifetime for prefix:
@@ -1402,14 +1415,14 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat
// Handle the infinite valid lifetime separately as we do not schedule a
// job in this case.
prefixState.invalidationJob.Cancel()
- prefixState.validUntil = tcpip.MonotonicTime{}
+ prefixState.validUntil = nil
} else {
var effectiveVl time.Duration
var rl time.Duration
// If the prefix was originally set to be valid forever, assume the
// remaining time to be the maximum possible value.
- if prefixState.validUntil == (tcpip.MonotonicTime{}) {
+ if prefixState.validUntil == nil {
rl = header.NDPInfiniteLifetime
} else {
rl = prefixState.validUntil.Sub(now)
@@ -1424,7 +1437,8 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat
if effectiveVl != 0 {
prefixState.invalidationJob.Cancel()
prefixState.invalidationJob.Schedule(effectiveVl)
- prefixState.validUntil = now.Add(effectiveVl)
+ t := now.Add(effectiveVl)
+ prefixState.validUntil = &t
}
}
@@ -1444,8 +1458,8 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat
// maximum temporary address valid lifetime. Note, the valid lifetime of a
// temporary address is relative to the address's creation time.
validUntil := tempAddrState.createdAt.Add(ndp.configs.MaxTempAddrValidLifetime)
- if prefixState.validUntil != (tcpip.MonotonicTime{}) && validUntil.Sub(prefixState.validUntil) > 0 {
- validUntil = prefixState.validUntil
+ if prefixState.validUntil != nil && prefixState.validUntil.Before(validUntil) {
+ validUntil = *prefixState.validUntil
}
// If the address is no longer valid, invalidate it immediately. Otherwise,
@@ -1464,14 +1478,15 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat
// desync factor. Note, the preferred lifetime of a temporary address is
// relative to the address's creation time.
preferredUntil := tempAddrState.createdAt.Add(ndp.configs.MaxTempAddrPreferredLifetime - ndp.temporaryAddressDesyncFactor)
- if prefixState.preferredUntil != (tcpip.MonotonicTime{}) && preferredUntil.Sub(prefixState.preferredUntil) > 0 {
- preferredUntil = prefixState.preferredUntil
+ if prefixState.preferredUntil != nil && prefixState.preferredUntil.Before(preferredUntil) {
+ preferredUntil = *prefixState.preferredUntil
}
// If the address is no longer preferred, deprecate it immediately.
// Otherwise, schedule the deprecation job again.
newPreferredLifetime := preferredUntil.Sub(now)
tempAddrState.deprecationJob.Cancel()
+
if newPreferredLifetime <= 0 {
ndp.deprecateSLAACAddress(tempAddrState.addressEndpoint)
} else {
@@ -1661,12 +1676,12 @@ func (ndp *ndpState) cleanupState() {
panic(fmt.Sprintf("ndp: still have discovered on-link prefixes after cleaning up; found = %d", got))
}
- for router := range ndp.defaultRouters {
- ndp.invalidateDefaultRouter(router)
+ for route := range ndp.offLinkRoutes {
+ ndp.invalidateOffLinkRoute(route)
}
- if got := len(ndp.defaultRouters); got != 0 {
- panic(fmt.Sprintf("ndp: still have discovered default routers after cleaning up; found = %d", got))
+ if got := len(ndp.offLinkRoutes); got != 0 {
+ panic(fmt.Sprintf("ndp: still have discovered off-link routes after cleaning up; found = %d", got))
}
ndp.dhcpv6Configuration = 0
@@ -1829,21 +1844,19 @@ func (ndp *ndpState) stopSolicitingRouters() {
}
func (ndp *ndpState) init(ep *endpoint, dadOptions ip.DADOptions) {
- if ndp.defaultRouters != nil {
+ if ndp.offLinkRoutes != nil {
panic("attempted to initialize NDP state twice")
}
ndp.ep = ep
ndp.configs = ep.protocol.options.NDPConfigs
ndp.dad.Init(&ndp.ep.mu, ep.protocol.options.DADConfigs, dadOptions)
- ndp.defaultRouters = make(map[tcpip.Address]defaultRouterState)
+ ndp.offLinkRoutes = make(map[offLinkRoute]offLinkRouteState)
ndp.onLinkPrefixes = make(map[tcpip.Subnet]onLinkPrefixState)
ndp.slaacPrefixes = make(map[tcpip.Subnet]slaacPrefixState)
header.InitialTempIID(ndp.temporaryIIDHistory[:], ndp.ep.protocol.options.TempIIDSeed, ndp.ep.nic.ID())
- if MaxDesyncFactor != 0 {
- ndp.temporaryAddressDesyncFactor = time.Duration(ep.protocol.stack.Rand().Int63n(int64(MaxDesyncFactor)))
- }
+ ndp.temporaryAddressDesyncFactor = time.Duration(ep.protocol.stack.Rand().Int63n(int64(MaxDesyncFactor)))
}
func (ndp *ndpState) SendDADMessage(addr tcpip.Address, nonce []byte) tcpip.Error {
diff --git a/pkg/tcpip/network/ipv6/ndp_test.go b/pkg/tcpip/network/ipv6/ndp_test.go
index 95d23f200..f0186c64e 100644
--- a/pkg/tcpip/network/ipv6/ndp_test.go
+++ b/pkg/tcpip/network/ipv6/ndp_test.go
@@ -42,7 +42,7 @@ type testNDPDispatcher struct {
func (*testNDPDispatcher) OnDuplicateAddressDetectionResult(tcpip.NICID, tcpip.Address, stack.DADResult) {
}
-func (t *testNDPDispatcher) OnOffLinkRouteUpdated(_ tcpip.NICID, _ tcpip.Subnet, addr tcpip.Address) {
+func (t *testNDPDispatcher) OnOffLinkRouteUpdated(_ tcpip.NICID, _ tcpip.Subnet, addr tcpip.Address, _ header.NDPRoutePreference) {
t.addr = addr
}
@@ -93,7 +93,7 @@ func TestStackNDPEndpointInvalidateDefaultRouter(t *testing.T) {
ipv6EP := ep.(*endpoint)
ipv6EP.mu.Lock()
- ipv6EP.mu.ndp.rememberDefaultRouter(lladdr1, time.Hour)
+ ipv6EP.mu.ndp.handleOffLinkRouteDiscovery(offLinkRoute{dest: header.IPv6EmptySubnet, router: lladdr1}, time.Hour, header.MediumRoutePreference)
ipv6EP.mu.Unlock()
if ndpDisp.addr != lladdr1 {
diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD
index 395ff9a07..e0847e58a 100644
--- a/pkg/tcpip/stack/BUILD
+++ b/pkg/tcpip/stack/BUILD
@@ -95,7 +95,7 @@ go_library(
go_test(
name = "stack_x_test",
- size = "medium",
+ size = "small",
srcs = [
"addressable_endpoint_state_test.go",
"ndp_test.go",
diff --git a/pkg/tcpip/stack/conntrack.go b/pkg/tcpip/stack/conntrack.go
index 18e0d4374..782e74b24 100644
--- a/pkg/tcpip/stack/conntrack.go
+++ b/pkg/tcpip/stack/conntrack.go
@@ -405,16 +405,23 @@ func (ct *ConnTrack) handlePacket(pkt *PacketBuffer, hook Hook, r *Route) bool {
// validated if checksum offloading is off. It may require IP defrag if the
// packets are fragmented.
+ var newAddr tcpip.Address
+ var newPort uint16
+
+ updateSRCFields := false
+
switch hook {
case Prerouting, Output:
if conn.manip == manipDestination {
switch dir {
case dirOriginal:
- tcpHeader.SetDestinationPort(conn.reply.srcPort)
- netHeader.SetDestinationAddress(conn.reply.srcAddr)
+ newPort = conn.reply.srcPort
+ newAddr = conn.reply.srcAddr
case dirReply:
- tcpHeader.SetSourcePort(conn.original.dstPort)
- netHeader.SetSourceAddress(conn.original.dstAddr)
+ newPort = conn.original.dstPort
+ newAddr = conn.original.dstAddr
+
+ updateSRCFields = true
}
pkt.NatDone = true
}
@@ -422,11 +429,13 @@ func (ct *ConnTrack) handlePacket(pkt *PacketBuffer, hook Hook, r *Route) bool {
if conn.manip == manipSource {
switch dir {
case dirOriginal:
- tcpHeader.SetSourcePort(conn.reply.dstPort)
- netHeader.SetSourceAddress(conn.reply.dstAddr)
+ newPort = conn.reply.dstPort
+ newAddr = conn.reply.dstAddr
+
+ updateSRCFields = true
case dirReply:
- tcpHeader.SetDestinationPort(conn.original.srcPort)
- netHeader.SetDestinationAddress(conn.original.srcAddr)
+ newPort = conn.original.srcPort
+ newAddr = conn.original.srcAddr
}
pkt.NatDone = true
}
@@ -437,29 +446,31 @@ func (ct *ConnTrack) handlePacket(pkt *PacketBuffer, hook Hook, r *Route) bool {
return false
}
+ fullChecksum := false
+ updatePseudoHeader := false
switch hook {
case Prerouting, Input:
case Output, Postrouting:
// Calculate the TCP checksum and set it.
- tcpHeader.SetChecksum(0)
- length := uint16(len(tcpHeader) + pkt.Data().Size())
- xsum := header.PseudoHeaderChecksum(header.TCPProtocolNumber, netHeader.SourceAddress(), netHeader.DestinationAddress(), length)
if pkt.GSOOptions.Type != GSONone && pkt.GSOOptions.NeedsCsum {
- tcpHeader.SetChecksum(xsum)
+ updatePseudoHeader = true
} else if r.RequiresTXTransportChecksum() {
- xsum = header.ChecksumCombine(xsum, pkt.Data().AsRange().Checksum())
- tcpHeader.SetChecksum(^tcpHeader.CalculateChecksum(xsum))
+ fullChecksum = true
+ updatePseudoHeader = true
}
default:
panic(fmt.Sprintf("unrecognized hook = %s", hook))
}
- // After modification, IPv4 packets need a valid checksum.
- if pkt.NetworkProtocolNumber == header.IPv4ProtocolNumber {
- netHeader := header.IPv4(pkt.NetworkHeader().View())
- netHeader.SetChecksum(0)
- netHeader.SetChecksum(^netHeader.CalculateChecksum())
- }
+ rewritePacket(
+ netHeader,
+ tcpHeader,
+ updateSRCFields,
+ fullChecksum,
+ updatePseudoHeader,
+ newPort,
+ newAddr,
+ )
// Update the state of tcb.
conn.mu.Lock()
diff --git a/pkg/tcpip/stack/iptables_targets.go b/pkg/tcpip/stack/iptables_targets.go
index 91e266de8..96cc899bb 100644
--- a/pkg/tcpip/stack/iptables_targets.go
+++ b/pkg/tcpip/stack/iptables_targets.go
@@ -133,29 +133,23 @@ func (rt *RedirectTarget) Action(pkt *PacketBuffer, ct *ConnTrack, hook Hook, r
switch protocol := pkt.TransportProtocolNumber; protocol {
case header.UDPProtocolNumber:
udpHeader := header.UDP(pkt.TransportHeader().View())
- udpHeader.SetDestinationPort(rt.Port)
- // Calculate UDP checksum and set it.
if hook == Output {
- udpHeader.SetChecksum(0)
- netHeader := pkt.Network()
- netHeader.SetDestinationAddress(address)
-
// Only calculate the checksum if offloading isn't supported.
- if r.RequiresTXTransportChecksum() {
- length := uint16(pkt.Size()) - uint16(len(pkt.NetworkHeader().View()))
- xsum := header.PseudoHeaderChecksum(protocol, netHeader.SourceAddress(), netHeader.DestinationAddress(), length)
- xsum = header.ChecksumCombine(xsum, pkt.Data().AsRange().Checksum())
- udpHeader.SetChecksum(^udpHeader.CalculateChecksum(xsum))
- }
+ requiresChecksum := r.RequiresTXTransportChecksum()
+ rewritePacket(
+ pkt.Network(),
+ udpHeader,
+ false, /* updateSRCFields */
+ requiresChecksum,
+ requiresChecksum,
+ rt.Port,
+ address,
+ )
+ } else {
+ udpHeader.SetDestinationPort(rt.Port)
}
- // After modification, IPv4 packets need a valid checksum.
- if pkt.NetworkProtocolNumber == header.IPv4ProtocolNumber {
- netHeader := header.IPv4(pkt.NetworkHeader().View())
- netHeader.SetChecksum(0)
- netHeader.SetChecksum(^netHeader.CalculateChecksum())
- }
pkt.NatDone = true
case header.TCPProtocolNumber:
if ct == nil {
@@ -214,26 +208,18 @@ func (st *SNATTarget) Action(pkt *PacketBuffer, ct *ConnTrack, hook Hook, r *Rou
switch protocol := pkt.TransportProtocolNumber; protocol {
case header.UDPProtocolNumber:
- udpHeader := header.UDP(pkt.TransportHeader().View())
- udpHeader.SetChecksum(0)
- udpHeader.SetSourcePort(st.Port)
- netHeader := pkt.Network()
- netHeader.SetSourceAddress(st.Addr)
-
// Only calculate the checksum if offloading isn't supported.
- if r.RequiresTXTransportChecksum() {
- length := uint16(pkt.Size()) - uint16(len(pkt.NetworkHeader().View()))
- xsum := header.PseudoHeaderChecksum(protocol, netHeader.SourceAddress(), netHeader.DestinationAddress(), length)
- xsum = header.ChecksumCombine(xsum, pkt.Data().AsRange().Checksum())
- udpHeader.SetChecksum(^udpHeader.CalculateChecksum(xsum))
- }
+ requiresChecksum := r.RequiresTXTransportChecksum()
+ rewritePacket(
+ pkt.Network(),
+ header.UDP(pkt.TransportHeader().View()),
+ true, /* updateSRCFields */
+ requiresChecksum,
+ requiresChecksum,
+ st.Port,
+ st.Addr,
+ )
- // After modification, IPv4 packets need a valid checksum.
- if pkt.NetworkProtocolNumber == header.IPv4ProtocolNumber {
- netHeader := header.IPv4(pkt.NetworkHeader().View())
- netHeader.SetChecksum(0)
- netHeader.SetChecksum(^netHeader.CalculateChecksum())
- }
pkt.NatDone = true
case header.TCPProtocolNumber:
if ct == nil {
@@ -252,3 +238,42 @@ func (st *SNATTarget) Action(pkt *PacketBuffer, ct *ConnTrack, hook Hook, r *Rou
return RuleAccept, 0
}
+
+func rewritePacket(n header.Network, t header.ChecksummableTransport, updateSRCFields, fullChecksum, updatePseudoHeader bool, newPort uint16, newAddr tcpip.Address) {
+ if updateSRCFields {
+ if fullChecksum {
+ t.SetSourcePortWithChecksumUpdate(newPort)
+ } else {
+ t.SetSourcePort(newPort)
+ }
+ } else {
+ if fullChecksum {
+ t.SetDestinationPortWithChecksumUpdate(newPort)
+ } else {
+ t.SetDestinationPort(newPort)
+ }
+ }
+
+ if updatePseudoHeader {
+ var oldAddr tcpip.Address
+ if updateSRCFields {
+ oldAddr = n.SourceAddress()
+ } else {
+ oldAddr = n.DestinationAddress()
+ }
+
+ t.UpdateChecksumPseudoHeaderAddress(oldAddr, newAddr, fullChecksum)
+ }
+
+ if checksummableNetHeader, ok := n.(header.ChecksummableNetwork); ok {
+ if updateSRCFields {
+ checksummableNetHeader.SetSourceAddressWithChecksumUpdate(newAddr)
+ } else {
+ checksummableNetHeader.SetDestinationAddressWithChecksumUpdate(newAddr)
+ }
+ } else if updateSRCFields {
+ n.SetSourceAddress(newAddr)
+ } else {
+ n.SetDestinationAddress(newAddr)
+ }
+}
diff --git a/pkg/tcpip/stack/ndp_test.go b/pkg/tcpip/stack/ndp_test.go
index b5c6626d6..9623d9c28 100644
--- a/pkg/tcpip/stack/ndp_test.go
+++ b/pkg/tcpip/stack/ndp_test.go
@@ -52,17 +52,6 @@ const (
linkAddr4 = tcpip.LinkAddress("\x02\x02\x03\x04\x05\x09")
defaultPrefixLen = 128
-
- // Extra time to use when waiting for an async event to occur.
- defaultAsyncPositiveEventTimeout = 10 * time.Second
-
- // Extra time to use when waiting for an async event to not occur.
- //
- // Since a negative check is used to make sure an event did not happen, it is
- // okay to use a smaller timeout compared to the positive case since execution
- // stall in regards to the monotonic clock will not affect the expected
- // outcome.
- defaultAsyncNegativeEventTimeout = time.Second
)
var (
@@ -116,6 +105,7 @@ type ndpOffLinkRouteEvent struct {
nicID tcpip.NICID
subnet tcpip.Subnet
router tcpip.Address
+ prf header.NDPRoutePreference
// true if route was updated, false if invalidated.
updated bool
}
@@ -141,6 +131,10 @@ type ndpAutoGenAddrEvent struct {
eventType ndpAutoGenAddrEventType
}
+func (e ndpAutoGenAddrEvent) String() string {
+ return fmt.Sprintf("%T{nicID=%d addr=%s eventType=%d}", e, e.nicID, e.addr, e.eventType)
+}
+
type ndpRDNSS struct {
addrs []tcpip.Address
lifetime time.Duration
@@ -189,12 +183,13 @@ func (n *ndpDispatcher) OnDuplicateAddressDetectionResult(nicID tcpip.NICID, add
}
// Implements ipv6.NDPDispatcher.OnOffLinkRouteUpdated.
-func (n *ndpDispatcher) OnOffLinkRouteUpdated(nicID tcpip.NICID, subnet tcpip.Subnet, router tcpip.Address) {
+func (n *ndpDispatcher) OnOffLinkRouteUpdated(nicID tcpip.NICID, subnet tcpip.Subnet, router tcpip.Address, prf header.NDPRoutePreference) {
if c := n.offLinkRouteC; c != nil {
c <- ndpOffLinkRouteEvent{
nicID,
subnet,
router,
+ prf,
true,
}
}
@@ -203,10 +198,12 @@ func (n *ndpDispatcher) OnOffLinkRouteUpdated(nicID tcpip.NICID, subnet tcpip.Su
// Implements ipv6.NDPDispatcher.OnOffLinkRouteInvalidated.
func (n *ndpDispatcher) OnOffLinkRouteInvalidated(nicID tcpip.NICID, subnet tcpip.Subnet, router tcpip.Address) {
if c := n.offLinkRouteC; c != nil {
+ var prf header.NDPRoutePreference
c <- ndpOffLinkRouteEvent{
nicID,
subnet,
router,
+ prf,
false,
}
}
@@ -1035,9 +1032,12 @@ func TestSetNDPConfigurations(t *testing.T) {
}
}
-// raBufWithOptsAndDHCPv6 returns a valid NDP Router Advertisement with options
-// and DHCPv6 configurations specified.
-func raBufWithOptsAndDHCPv6(ip tcpip.Address, rl uint16, managedAddress, otherConfigurations bool, optSer header.NDPOptionsSerializer) *stack.PacketBuffer {
+// raBuf returns a valid NDP Router Advertisement with options, router
+// preference and DHCPv6 configurations specified.
+func raBuf(ip tcpip.Address, rl uint16, managedAddress, otherConfigurations bool, prf header.NDPRoutePreference, optSer header.NDPOptionsSerializer) *stack.PacketBuffer {
+ const flagsByte = 1
+ const routerLifetimeOffset = 2
+
icmpSize := header.ICMPv6HeaderSize + header.NDPRAMinimumSize + optSer.Length()
hdr := buffer.NewPrependable(header.IPv6MinimumSize + icmpSize)
pkt := header.ICMPv6(hdr.Prepend(icmpSize))
@@ -1046,19 +1046,19 @@ func raBufWithOptsAndDHCPv6(ip tcpip.Address, rl uint16, managedAddress, otherCo
raPayload := pkt.MessageBody()
ra := header.NDPRouterAdvert(raPayload)
// Populate the Router Lifetime.
- binary.BigEndian.PutUint16(raPayload[2:], rl)
+ binary.BigEndian.PutUint16(raPayload[routerLifetimeOffset:], rl)
// Populate the Managed Address flag field.
if managedAddress {
- // The Managed Addresses flag field is the 7th bit of byte #1 (0-indexing)
- // of the RA payload.
- raPayload[1] |= 1 << 7
+ // The Managed Addresses flag field is the 7th bit of the flags byte.
+ raPayload[flagsByte] |= 1 << 7
}
// Populate the Other Configurations flag field.
if otherConfigurations {
- // The Other Configurations flag field is the 6th bit of byte #1
- // (0-indexing) of the RA payload.
- raPayload[1] |= 1 << 6
+ // The Other Configurations flag field is the 6th bit of the flags byte.
+ raPayload[flagsByte] |= 1 << 6
}
+ // The Prf field is held in the flags byte.
+ raPayload[flagsByte] |= byte(prf) << 3
opts := ra.Options()
opts.Serialize(optSer)
pkt.SetChecksum(header.ICMPv6Checksum(header.ICMPv6ChecksumParams{
@@ -1086,7 +1086,7 @@ func raBufWithOptsAndDHCPv6(ip tcpip.Address, rl uint16, managedAddress, otherCo
// Note, raBufWithOpts does not populate any of the RA fields other than the
// Router Lifetime.
func raBufWithOpts(ip tcpip.Address, rl uint16, optSer header.NDPOptionsSerializer) *stack.PacketBuffer {
- return raBufWithOptsAndDHCPv6(ip, rl, false, false, optSer)
+ return raBuf(ip, rl, false /* managedAddress */, false /* otherConfigurations */, 0 /* prf */, optSer)
}
// raBufWithDHCPv6 returns a valid NDP Router Advertisement with DHCPv6 related
@@ -1094,18 +1094,26 @@ func raBufWithOpts(ip tcpip.Address, rl uint16, optSer header.NDPOptionsSerializ
//
// Note, raBufWithDHCPv6 does not populate any of the RA fields other than the
// DHCPv6 related ones.
-func raBufWithDHCPv6(ip tcpip.Address, managedAddresses, otherConfiguratiosns bool) *stack.PacketBuffer {
- return raBufWithOptsAndDHCPv6(ip, 0, managedAddresses, otherConfiguratiosns, header.NDPOptionsSerializer{})
+func raBufWithDHCPv6(ip tcpip.Address, managedAddresses, otherConfigurations bool) *stack.PacketBuffer {
+ return raBuf(ip, 0, managedAddresses, otherConfigurations, 0 /* prf */, header.NDPOptionsSerializer{})
}
// raBuf returns a valid NDP Router Advertisement.
//
// Note, raBuf does not populate any of the RA fields other than the
// Router Lifetime.
-func raBuf(ip tcpip.Address, rl uint16) *stack.PacketBuffer {
+func raBufSimple(ip tcpip.Address, rl uint16) *stack.PacketBuffer {
return raBufWithOpts(ip, rl, header.NDPOptionsSerializer{})
}
+// raBufWithPrf returns a valid NDP Router Advertisement with a preference.
+//
+// Note, raBufWithPrf does not populate any of the RA fields other than the
+// Router Lifetime and Default Router Preference fields.
+func raBufWithPrf(ip tcpip.Address, rl uint16, prf header.NDPRoutePreference) *stack.PacketBuffer {
+ return raBuf(ip, rl, false /* managedAddress */, false /* otherConfigurations */, prf, header.NDPOptionsSerializer{})
+}
+
// raBufWithPI returns a valid NDP Router Advertisement with a single Prefix
// Information option.
//
@@ -1165,7 +1173,7 @@ func TestDynamicConfigurationsDisabled(t *testing.T) {
config: func(enable bool) ipv6.NDPConfigurations {
return ipv6.NDPConfigurations{DiscoverDefaultRouters: enable}
},
- ra: raBuf(llAddr2, 1000),
+ ra: raBufSimple(llAddr2, 1000),
},
{
name: "No Prefix Discovery",
@@ -1300,10 +1308,8 @@ func boolToUint64(v bool) uint64 {
return 0
}
-// Check e to make sure that the event is for addr on nic with ID 1, and the
-// update flag set to updated.
-func checkOffLinkRouteEvent(e ndpOffLinkRouteEvent, router tcpip.Address, updated bool) string {
- return cmp.Diff(ndpOffLinkRouteEvent{nicID: 1, subnet: header.IPv6EmptySubnet, router: router, updated: updated}, e, cmp.AllowUnexported(e))
+func checkOffLinkRouteEvent(e ndpOffLinkRouteEvent, nicID tcpip.NICID, router tcpip.Address, prf header.NDPRoutePreference, updated bool) string {
+ return cmp.Diff(ndpOffLinkRouteEvent{nicID: nicID, subnet: header.IPv6EmptySubnet, router: router, prf: prf, updated: updated}, e, cmp.AllowUnexported(e))
}
func testWithRAs(t *testing.T, f func(*testing.T, ipv6.HandleRAsConfiguration, bool)) {
@@ -1337,6 +1343,8 @@ func testWithRAs(t *testing.T, f func(*testing.T, ipv6.HandleRAsConfiguration, b
}
func TestRouterDiscovery(t *testing.T) {
+ const nicID = 1
+
testWithRAs(t, func(t *testing.T, handleRAs ipv6.HandleRAsConfiguration, forwarding bool) {
ndpDisp := ndpDispatcher{
offLinkRouteC: make(chan ndpOffLinkRouteEvent, 1),
@@ -1354,12 +1362,12 @@ func TestRouterDiscovery(t *testing.T) {
Clock: clock,
})
- expectOffLinkRouteEvent := func(addr tcpip.Address, updated bool) {
+ expectOffLinkRouteEvent := func(addr tcpip.Address, prf header.NDPRoutePreference, updated bool) {
t.Helper()
select {
case e := <-ndpDisp.offLinkRouteC:
- if diff := checkOffLinkRouteEvent(e, addr, updated); diff != "" {
+ if diff := checkOffLinkRouteEvent(e, nicID, addr, prf, updated); diff != "" {
t.Errorf("off-link route event mismatch (-want +got):\n%s", diff)
}
default:
@@ -1373,7 +1381,8 @@ func TestRouterDiscovery(t *testing.T) {
clock.Advance(timeout)
select {
case e := <-ndpDisp.offLinkRouteC:
- if diff := checkOffLinkRouteEvent(e, addr, false); diff != "" {
+ var prf header.NDPRoutePreference
+ if diff := checkOffLinkRouteEvent(e, nicID, addr, prf, false); diff != "" {
t.Errorf("off-link route event mismatch (-want +got):\n%s", diff)
}
default:
@@ -1385,37 +1394,44 @@ func TestRouterDiscovery(t *testing.T) {
t.Fatalf("SetForwardingDefaultAndAllNICs(%d, %t): %s", ipv6.ProtocolNumber, forwarding, err)
}
- if err := s.CreateNIC(1, e); err != nil {
- t.Fatalf("CreateNIC(1) = %s", err)
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _): %s", nicID, err)
}
// Rx an RA from lladdr2 with zero lifetime. It should not be
// remembered.
- e.InjectInbound(header.IPv6ProtocolNumber, raBuf(llAddr2, 0))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufSimple(llAddr2, 0))
select {
case <-ndpDisp.offLinkRouteC:
t.Fatal("unexpectedly updated an off-link route with 0 lifetime")
default:
}
- // Rx an RA from lladdr2 with a huge lifetime.
- e.InjectInbound(header.IPv6ProtocolNumber, raBuf(llAddr2, 1000))
- expectOffLinkRouteEvent(llAddr2, true)
+ // Rx an RA from lladdr2 with a huge lifetime and reserved preference value
+ // (which should be interpreted as the default (medium) preference value).
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPrf(llAddr2, 1000, header.ReservedRoutePreference))
+ expectOffLinkRouteEvent(llAddr2, header.MediumRoutePreference, true)
- // Rx an RA from another router (lladdr3) with non-zero lifetime.
+ // Rx an RA from another router (lladdr3) with non-zero lifetime and
+ // non-default preference value.
const l3LifetimeSeconds = 6
- e.InjectInbound(header.IPv6ProtocolNumber, raBuf(llAddr3, l3LifetimeSeconds))
- expectOffLinkRouteEvent(llAddr3, true)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPrf(llAddr3, l3LifetimeSeconds, header.HighRoutePreference))
+ expectOffLinkRouteEvent(llAddr3, header.HighRoutePreference, true)
- // Rx an RA from lladdr2 with lesser lifetime.
+ // Rx an RA from lladdr2 with lesser lifetime and default (medium)
+ // preference value.
const l2LifetimeSeconds = 2
- e.InjectInbound(header.IPv6ProtocolNumber, raBuf(llAddr2, l2LifetimeSeconds))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufSimple(llAddr2, l2LifetimeSeconds))
select {
case <-ndpDisp.offLinkRouteC:
t.Fatal("should not receive a off-link route event when updating lifetimes for known routers")
default:
}
+ // Rx an RA from lladdr2 with a different preference.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPrf(llAddr2, l2LifetimeSeconds, header.LowRoutePreference))
+ expectOffLinkRouteEvent(llAddr2, header.LowRoutePreference, true)
+
// Wait for lladdr2's router invalidation job to execute. The lifetime
// of the router should have been updated to the most recent (smaller)
// lifetime.
@@ -1426,12 +1442,12 @@ func TestRouterDiscovery(t *testing.T) {
expectAsyncOffLinkRouteInvalidationEvent(llAddr2, l2LifetimeSeconds*time.Second)
// Rx an RA from lladdr2 with huge lifetime.
- e.InjectInbound(header.IPv6ProtocolNumber, raBuf(llAddr2, 1000))
- expectOffLinkRouteEvent(llAddr2, true)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufSimple(llAddr2, 1000))
+ expectOffLinkRouteEvent(llAddr2, header.MediumRoutePreference, true)
// Rx an RA from lladdr2 with zero lifetime. It should be invalidated.
- e.InjectInbound(header.IPv6ProtocolNumber, raBuf(llAddr2, 0))
- expectOffLinkRouteEvent(llAddr2, false)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufSimple(llAddr2, 0))
+ expectOffLinkRouteEvent(llAddr2, header.MediumRoutePreference, false)
// Wait for lladdr3's router invalidation job to execute. The lifetime
// of the router should have been updated to the most recent (smaller)
@@ -1445,8 +1461,10 @@ func TestRouterDiscovery(t *testing.T) {
}
// TestRouterDiscoveryMaxRouters tests that only
-// ipv6.MaxDiscoveredDefaultRouters discovered routers are remembered.
+// ipv6.MaxDiscoveredOffLinkRoutes discovered routers are remembered.
func TestRouterDiscoveryMaxRouters(t *testing.T) {
+ const nicID = 1
+
ndpDisp := ndpDispatcher{
offLinkRouteC: make(chan ndpOffLinkRouteEvent, 1),
}
@@ -1461,22 +1479,22 @@ func TestRouterDiscoveryMaxRouters(t *testing.T) {
})},
})
- if err := s.CreateNIC(1, e); err != nil {
- t.Fatalf("CreateNIC(1) = %s", err)
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _): %s", nicID, err)
}
// Receive an RA from 2 more than the max number of discovered routers.
- for i := 1; i <= ipv6.MaxDiscoveredDefaultRouters+2; i++ {
+ for i := 1; i <= ipv6.MaxDiscoveredOffLinkRoutes+2; i++ {
linkAddr := []byte{2, 2, 3, 4, 5, 0}
linkAddr[5] = byte(i)
llAddr := header.LinkLocalAddr(tcpip.LinkAddress(linkAddr))
- e.InjectInbound(header.IPv6ProtocolNumber, raBuf(llAddr, 5))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufSimple(llAddr, 5))
- if i <= ipv6.MaxDiscoveredDefaultRouters {
+ if i <= ipv6.MaxDiscoveredOffLinkRoutes {
select {
case e := <-ndpDisp.offLinkRouteC:
- if diff := checkOffLinkRouteEvent(e, llAddr, true); diff != "" {
+ if diff := checkOffLinkRouteEvent(e, nicID, llAddr, header.MediumRoutePreference, true); diff != "" {
t.Errorf("off-link route event mismatch (-want +got):\n%s", diff)
}
default:
@@ -1596,17 +1614,6 @@ func TestPrefixDiscovery(t *testing.T) {
}
func TestPrefixDiscoveryWithInfiniteLifetime(t *testing.T) {
- // Update the infinite lifetime value to a smaller value so we can test
- // that when we receive a PI with such a lifetime value, we do not
- // invalidate the prefix.
- const testInfiniteLifetimeSeconds = 2
- const testInfiniteLifetime = testInfiniteLifetimeSeconds * time.Second
- saved := header.NDPInfiniteLifetime
- header.NDPInfiniteLifetime = testInfiniteLifetime
- defer func() {
- header.NDPInfiniteLifetime = saved
- }()
-
prefix := tcpip.AddressWithPrefix{
Address: testutil.MustParse6("102:304:506:708::"),
PrefixLen: 64,
@@ -1648,9 +1655,9 @@ func TestPrefixDiscoveryWithInfiniteLifetime(t *testing.T) {
// Receive an RA with prefix in an NDP Prefix Information option (PI)
// with infinite valid lifetime which should not get invalidated.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, testInfiniteLifetimeSeconds, 0))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, infiniteLifetimeSeconds, 0))
expectPrefixEvent(subnet, true)
- clock.Advance(testInfiniteLifetime)
+ clock.Advance(header.NDPInfiniteLifetime)
select {
case <-ndpDisp.prefixC:
t.Fatal("unexpectedly invalidated a prefix with infinite lifetime")
@@ -1658,9 +1665,8 @@ func TestPrefixDiscoveryWithInfiniteLifetime(t *testing.T) {
}
// Receive an RA with finite lifetime.
- // The prefix should get invalidated after 1s.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, testInfiniteLifetimeSeconds-1, 0))
- clock.Advance(testInfiniteLifetime)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, infiniteLifetimeSeconds-1, 0))
+ clock.Advance(header.NDPInfiniteLifetime - time.Second)
select {
case e := <-ndpDisp.prefixC:
if diff := checkPrefixEvent(e, subnet, false); diff != "" {
@@ -1671,23 +1677,13 @@ func TestPrefixDiscoveryWithInfiniteLifetime(t *testing.T) {
}
// Receive an RA with finite lifetime.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, testInfiniteLifetimeSeconds-1, 0))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, infiniteLifetimeSeconds-1, 0))
expectPrefixEvent(subnet, true)
// Receive an RA with prefix with an infinite lifetime.
// The prefix should not be invalidated.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, testInfiniteLifetimeSeconds, 0))
- clock.Advance(testInfiniteLifetime)
- select {
- case <-ndpDisp.prefixC:
- t.Fatal("unexpectedly invalidated a prefix with infinite lifetime")
- default:
- }
-
- // Receive an RA with a prefix with a lifetime value greater than the
- // set infinite lifetime value.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, testInfiniteLifetimeSeconds+1, 0))
- clock.Advance((testInfiniteLifetimeSeconds + 1) * time.Second)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, false, infiniteLifetimeSeconds, 0))
+ clock.Advance(header.NDPInfiniteLifetime)
select {
case <-ndpDisp.prefixC:
t.Fatal("unexpectedly invalidated a prefix with infinite lifetime")
@@ -1781,17 +1777,12 @@ func checkAutoGenAddrEvent(e ndpAutoGenAddrEvent, addr tcpip.AddressWithPrefix,
return cmp.Diff(ndpAutoGenAddrEvent{nicID: 1, addr: addr, eventType: eventType}, e, cmp.AllowUnexported(e))
}
+const minVLSeconds = uint32(ipv6.MinPrefixInformationValidLifetimeForUpdate / time.Second)
+const infiniteLifetimeSeconds = uint32(header.NDPInfiniteLifetime / time.Second)
+
// TestAutoGenAddr tests that an address is properly generated and invalidated
// when configured to do so.
func TestAutoGenAddr(t *testing.T) {
- const newMinVL = 2
- newMinVLDuration := newMinVL * time.Second
- saved := ipv6.MinPrefixInformationValidLifetimeForUpdate
- defer func() {
- ipv6.MinPrefixInformationValidLifetimeForUpdate = saved
- }()
- ipv6.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration
-
prefix1, _, addr1 := prefixSubnetAddr(0, linkAddr1)
prefix2, _, addr2 := prefixSubnetAddr(1, linkAddr1)
@@ -1800,6 +1791,7 @@ func TestAutoGenAddr(t *testing.T) {
autoGenAddrC: make(chan ndpAutoGenAddrEvent, 1),
}
e := channel.New(0, 1280, linkAddr1)
+ clock := faketime.NewManualClock()
s := stack.New(stack.Options{
NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
NDPConfigs: ipv6.NDPConfigurations{
@@ -1808,6 +1800,7 @@ func TestAutoGenAddr(t *testing.T) {
},
NDPDisp: &ndpDisp,
})},
+ Clock: clock,
})
if err := s.SetForwardingDefaultAndAllNICs(ipv6.ProtocolNumber, forwarding); err != nil {
@@ -1857,8 +1850,9 @@ func TestAutoGenAddr(t *testing.T) {
default:
}
- // Receive an RA with prefix2 in a PI.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, 100, 0))
+ // Receive an RA with prefix2 in a PI with a valid lifetime that exceeds
+ // the minimum.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, minVLSeconds+1, 0))
expectAutoGenAddrEvent(addr2, newAddr)
if !containsV6Addr(s.NICInfo()[1].ProtocolAddresses, addr1) {
t.Fatalf("Should have %s in the list of addresses", addr1)
@@ -1868,7 +1862,7 @@ func TestAutoGenAddr(t *testing.T) {
}
// Refresh valid lifetime for addr of prefix1.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, newMinVL, 0))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 0))
select {
case <-ndpDisp.autoGenAddrC:
t.Fatal("unexpectedly auto-generated an address when we already have an address for a prefix")
@@ -1876,12 +1870,13 @@ func TestAutoGenAddr(t *testing.T) {
}
// Wait for addr of prefix1 to be invalidated.
+ clock.Advance(ipv6.MinPrefixInformationValidLifetimeForUpdate)
select {
case e := <-ndpDisp.autoGenAddrC:
if diff := checkAutoGenAddrEvent(e, addr1, invalidatedAddr); diff != "" {
t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(newMinVLDuration + defaultAsyncPositiveEventTimeout):
+ default:
t.Fatal("timed out waiting for addr auto gen event")
}
if containsV6Addr(s.NICInfo()[1].ProtocolAddresses, addr1) {
@@ -1911,20 +1906,7 @@ func addressCheck(addrs []tcpip.ProtocolAddress, containList, notContainList []t
// TestAutoGenTempAddr tests that temporary SLAAC addresses are generated when
// configured to do so as part of IPv6 Privacy Extensions.
func TestAutoGenTempAddr(t *testing.T) {
- const (
- nicID = 1
- newMinVL = 5
- newMinVLDuration = newMinVL * time.Second
- )
-
- savedMinPrefixInformationValidLifetimeForUpdate := ipv6.MinPrefixInformationValidLifetimeForUpdate
- savedMaxDesync := ipv6.MaxDesyncFactor
- defer func() {
- ipv6.MinPrefixInformationValidLifetimeForUpdate = savedMinPrefixInformationValidLifetimeForUpdate
- ipv6.MaxDesyncFactor = savedMaxDesync
- }()
- ipv6.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration
- ipv6.MaxDesyncFactor = time.Nanosecond
+ const nicID = 1
prefix1, _, addr1 := prefixSubnetAddr(0, linkAddr1)
prefix2, _, addr2 := prefixSubnetAddr(1, linkAddr1)
@@ -1944,218 +1926,211 @@ func TestAutoGenTempAddr(t *testing.T) {
},
}
- // This Run will not return until the parallel tests finish.
- //
- // We need this because we need to do some teardown work after the
- // parallel tests complete.
- //
- // See https://godoc.org/testing#hdr-Subtests_and_Sub_benchmarks for
- // more details.
- t.Run("group", func(t *testing.T) {
- for i, test := range tests {
- i := i
- test := test
-
- t.Run(test.name, func(t *testing.T) {
- t.Parallel()
-
- seed := []byte{uint8(i)}
- var tempIIDHistory [header.IIDSize]byte
- header.InitialTempIID(tempIIDHistory[:], seed, nicID)
- newTempAddr := func(stableAddr tcpip.Address) tcpip.AddressWithPrefix {
- return header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], stableAddr)
- }
-
- ndpDisp := ndpDispatcher{
- dadC: make(chan ndpDADEvent, 2),
- autoGenAddrC: make(chan ndpAutoGenAddrEvent, 2),
- }
- e := channel.New(0, 1280, linkAddr1)
- s := stack.New(stack.Options{
- NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
- DADConfigs: stack.DADConfigurations{
- DupAddrDetectTransmits: test.dupAddrTransmits,
- RetransmitTimer: test.retransmitTimer,
- },
- NDPConfigs: ipv6.NDPConfigurations{
- HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
- AutoGenGlobalAddresses: true,
- AutoGenTempGlobalAddresses: true,
- },
- NDPDisp: &ndpDisp,
- TempIIDSeed: seed,
- })},
- })
-
- if err := s.CreateNIC(nicID, e); err != nil {
- t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
- }
-
- expectAutoGenAddrEvent := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
- t.Helper()
-
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, addr, eventType); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- default:
- t.Fatal("expected addr auto gen event")
- }
- }
-
- expectAutoGenAddrEventAsync := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
- t.Helper()
+ for i, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ seed := []byte{uint8(i)}
+ var tempIIDHistory [header.IIDSize]byte
+ header.InitialTempIID(tempIIDHistory[:], seed, nicID)
+ newTempAddr := func(stableAddr tcpip.Address) tcpip.AddressWithPrefix {
+ return header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], stableAddr)
+ }
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, addr, eventType); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- case <-time.After(defaultAsyncPositiveEventTimeout):
- t.Fatal("timed out waiting for addr auto gen event")
- }
- }
+ ndpDisp := ndpDispatcher{
+ dadC: make(chan ndpDADEvent, 2),
+ autoGenAddrC: make(chan ndpAutoGenAddrEvent, 2),
+ }
+ e := channel.New(0, 1280, linkAddr1)
+ clock := faketime.NewManualClock()
+ s := stack.New(stack.Options{
+ NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
+ DADConfigs: stack.DADConfigurations{
+ DupAddrDetectTransmits: test.dupAddrTransmits,
+ RetransmitTimer: test.retransmitTimer,
+ },
+ NDPConfigs: ipv6.NDPConfigurations{
+ HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
+ AutoGenGlobalAddresses: true,
+ AutoGenTempGlobalAddresses: true,
+ MaxTempAddrValidLifetime: 2 * ipv6.MinPrefixInformationValidLifetimeForUpdate,
+ MaxTempAddrPreferredLifetime: 2 * ipv6.MinPrefixInformationValidLifetimeForUpdate,
+ },
+ NDPDisp: &ndpDisp,
+ TempIIDSeed: seed,
+ })},
+ Clock: clock,
+ })
- expectDADEventAsync := func(addr tcpip.Address) {
- t.Helper()
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
- select {
- case e := <-ndpDisp.dadC:
- if diff := checkDADEvent(e, nicID, addr, &stack.DADSucceeded{}); diff != "" {
- t.Errorf("DAD event mismatch (-want +got):\n%s", diff)
- }
- case <-time.After(time.Duration(test.dupAddrTransmits)*test.retransmitTimer + defaultAsyncPositiveEventTimeout):
- t.Fatal("timed out waiting for DAD event")
- }
- }
+ expectAutoGenAddrEvent := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
+ t.Helper()
- // Receive an RA with prefix1 in an NDP Prefix Information option (PI)
- // with zero valid lifetime.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 0, 0))
select {
case e := <-ndpDisp.autoGenAddrC:
- t.Fatalf("unexpectedly auto-generated an address with 0 lifetime; event = %+v", e)
+ if diff := checkAutoGenAddrEvent(e, addr, eventType); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
+ }
default:
+ t.Fatal("expected addr auto gen event")
}
+ }
+
+ expectAutoGenAddrEventAsync := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
+ t.Helper()
- // Receive an RA with prefix1 in an NDP Prefix Information option (PI)
- // with non-zero valid lifetime.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 0))
- expectAutoGenAddrEvent(addr1, newAddr)
- expectDADEventAsync(addr1.Address)
+ clock.RunImmediatelyScheduledJobs()
select {
case e := <-ndpDisp.autoGenAddrC:
- t.Fatalf("unexpectedly got an auto gen addr event = %+v", e)
+ if diff := checkAutoGenAddrEvent(e, addr, eventType); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
+ }
default:
+ t.Fatal("timed out waiting for addr auto gen event")
}
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1}, nil); mismatch != "" {
- t.Fatal(mismatch)
- }
+ }
- // Receive an RA with prefix1 in an NDP Prefix Information option (PI)
- // with non-zero valid & preferred lifetimes.
- tempAddr1 := newTempAddr(addr1.Address)
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 100))
- expectAutoGenAddrEvent(tempAddr1, newAddr)
- expectDADEventAsync(tempAddr1.Address)
- if mismatch := addressCheck(s.NICInfo()[1].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1}, nil); mismatch != "" {
- t.Fatal(mismatch)
- }
+ expectDADEventAsync := func(addr tcpip.Address) {
+ t.Helper()
- // Receive an RA with prefix2 in an NDP Prefix Information option (PI)
- // with preferred lifetime > valid lifetime
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 5, 6))
+ clock.Advance(time.Duration(test.dupAddrTransmits) * test.retransmitTimer)
select {
- case e := <-ndpDisp.autoGenAddrC:
- t.Fatalf("unexpectedly auto-generated an address with preferred lifetime > valid lifetime; event = %+v", e)
+ case e := <-ndpDisp.dadC:
+ if diff := checkDADEvent(e, nicID, addr, &stack.DADSucceeded{}); diff != "" {
+ t.Errorf("DAD event mismatch (-want +got):\n%s", diff)
+ }
default:
+ t.Fatal("timed out waiting for DAD event")
}
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1}, nil); mismatch != "" {
- t.Fatal(mismatch)
- }
+ }
- // Receive an RA with prefix2 in a PI w/ non-zero valid and preferred
- // lifetimes.
- tempAddr2 := newTempAddr(addr2.Address)
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, 100, 100))
- expectAutoGenAddrEvent(addr2, newAddr)
- expectDADEventAsync(addr2.Address)
- expectAutoGenAddrEventAsync(tempAddr2, newAddr)
- expectDADEventAsync(tempAddr2.Address)
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1, addr2, tempAddr2}, nil); mismatch != "" {
- t.Fatal(mismatch)
- }
+ // Receive an RA with prefix1 in an NDP Prefix Information option (PI)
+ // with zero valid lifetime.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 0, 0))
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ t.Fatalf("unexpectedly auto-generated an address with 0 lifetime; event = %+v", e)
+ default:
+ }
- // Deprecate prefix1.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 0))
- expectAutoGenAddrEvent(addr1, deprecatedAddr)
- expectAutoGenAddrEvent(tempAddr1, deprecatedAddr)
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1, addr2, tempAddr2}, nil); mismatch != "" {
- t.Fatal(mismatch)
- }
+ // Receive an RA with prefix1 in an NDP Prefix Information option (PI)
+ // with non-zero valid lifetime.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 0))
+ expectAutoGenAddrEvent(addr1, newAddr)
+ expectDADEventAsync(addr1.Address)
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ t.Fatalf("unexpectedly got an auto gen addr event = %+v", e)
+ default:
+ }
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1}, nil); mismatch != "" {
+ t.Fatal(mismatch)
+ }
- // Refresh lifetimes for prefix1.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 100))
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1, addr2, tempAddr2}, nil); mismatch != "" {
- t.Fatal(mismatch)
- }
+ // Receive an RA with prefix1 in an NDP Prefix Information option (PI)
+ // with non-zero valid & preferred lifetimes.
+ tempAddr1 := newTempAddr(addr1.Address)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 100))
+ expectAutoGenAddrEvent(tempAddr1, newAddr)
+ expectDADEventAsync(tempAddr1.Address)
+ if mismatch := addressCheck(s.NICInfo()[1].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1}, nil); mismatch != "" {
+ t.Fatal(mismatch)
+ }
- // Reduce valid lifetime and deprecate addresses of prefix1.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, newMinVL, 0))
- expectAutoGenAddrEvent(addr1, deprecatedAddr)
- expectAutoGenAddrEvent(tempAddr1, deprecatedAddr)
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1, addr2, tempAddr2}, nil); mismatch != "" {
- t.Fatal(mismatch)
- }
+ // Receive an RA with prefix2 in an NDP Prefix Information option (PI)
+ // with preferred lifetime > valid lifetime
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 5, 6))
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ t.Fatalf("unexpectedly auto-generated an address with preferred lifetime > valid lifetime; event = %+v", e)
+ default:
+ }
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1}, nil); mismatch != "" {
+ t.Fatal(mismatch)
+ }
- // Wait for addrs of prefix1 to be invalidated. They should be
- // invalidated at the same time.
- select {
- case e := <-ndpDisp.autoGenAddrC:
- var nextAddr tcpip.AddressWithPrefix
- if e.addr == addr1 {
- if diff := checkAutoGenAddrEvent(e, addr1, invalidatedAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- nextAddr = tempAddr1
- } else {
- if diff := checkAutoGenAddrEvent(e, tempAddr1, invalidatedAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- nextAddr = addr1
- }
+ // Receive an RA with prefix2 in a PI with a valid lifetime that exceeds
+ // the minimum and won't be reached in this test.
+ tempAddr2 := newTempAddr(addr2.Address)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, 2*minVLSeconds, 2*minVLSeconds))
+ expectAutoGenAddrEvent(addr2, newAddr)
+ expectDADEventAsync(addr2.Address)
+ expectAutoGenAddrEventAsync(tempAddr2, newAddr)
+ expectDADEventAsync(tempAddr2.Address)
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1, addr2, tempAddr2}, nil); mismatch != "" {
+ t.Fatal(mismatch)
+ }
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, nextAddr, invalidatedAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- case <-time.After(defaultAsyncPositiveEventTimeout):
- t.Fatal("timed out waiting for addr auto gen event")
+ // Deprecate prefix1.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 0))
+ expectAutoGenAddrEvent(addr1, deprecatedAddr)
+ expectAutoGenAddrEvent(tempAddr1, deprecatedAddr)
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1, addr2, tempAddr2}, nil); mismatch != "" {
+ t.Fatal(mismatch)
+ }
+
+ // Refresh lifetimes for prefix1.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, 100, 100))
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1, addr2, tempAddr2}, nil); mismatch != "" {
+ t.Fatal(mismatch)
+ }
+
+ // Reduce valid lifetime and deprecate addresses of prefix1.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, minVLSeconds, 0))
+ expectAutoGenAddrEvent(addr1, deprecatedAddr)
+ expectAutoGenAddrEvent(tempAddr1, deprecatedAddr)
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr1, tempAddr1, addr2, tempAddr2}, nil); mismatch != "" {
+ t.Fatal(mismatch)
+ }
+
+ // Wait for addrs of prefix1 to be invalidated. They should be
+ // invalidated at the same time.
+ clock.Advance(ipv6.MinPrefixInformationValidLifetimeForUpdate)
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ var nextAddr tcpip.AddressWithPrefix
+ if e.addr == addr1 {
+ if diff := checkAutoGenAddrEvent(e, addr1, invalidatedAddr); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(newMinVLDuration + defaultAsyncPositiveEventTimeout):
- t.Fatal("timed out waiting for addr auto gen event")
- }
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr2, tempAddr2}, []tcpip.AddressWithPrefix{addr1, tempAddr1}); mismatch != "" {
- t.Fatal(mismatch)
+ nextAddr = tempAddr1
+ } else {
+ if diff := checkAutoGenAddrEvent(e, tempAddr1, invalidatedAddr); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
+ }
+ nextAddr = addr1
}
- // Receive an RA with prefix2 in a PI w/ 0 lifetimes.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, 0, 0))
- expectAutoGenAddrEvent(addr2, deprecatedAddr)
- expectAutoGenAddrEvent(tempAddr2, deprecatedAddr)
select {
case e := <-ndpDisp.autoGenAddrC:
- t.Errorf("got unexpected auto gen addr event = %+v", e)
+ if diff := checkAutoGenAddrEvent(e, nextAddr, invalidatedAddr); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
+ }
default:
+ t.Fatal("timed out waiting for addr auto gen event")
}
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr2, tempAddr2}, []tcpip.AddressWithPrefix{addr1, tempAddr1}); mismatch != "" {
- t.Fatal(mismatch)
- }
- })
- }
- })
+ default:
+ t.Fatal("timed out waiting for addr auto gen event")
+ }
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr2, tempAddr2}, []tcpip.AddressWithPrefix{addr1, tempAddr1}); mismatch != "" {
+ t.Fatal(mismatch)
+ }
+
+ // Receive an RA with prefix2 in a PI w/ 0 lifetimes.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, 0, 0))
+ expectAutoGenAddrEvent(addr2, deprecatedAddr)
+ expectAutoGenAddrEvent(tempAddr2, deprecatedAddr)
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ t.Errorf("got unexpected auto gen addr event = %+v", e)
+ default:
+ }
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr2, tempAddr2}, []tcpip.AddressWithPrefix{addr1, tempAddr1}); mismatch != "" {
+ t.Fatal(mismatch)
+ }
+ })
+ }
}
// TestNoAutoGenTempAddrForLinkLocal test that temporary SLAAC addresses are not
@@ -2163,12 +2138,6 @@ func TestAutoGenTempAddr(t *testing.T) {
func TestNoAutoGenTempAddrForLinkLocal(t *testing.T) {
const nicID = 1
- savedMaxDesyncFactor := ipv6.MaxDesyncFactor
- defer func() {
- ipv6.MaxDesyncFactor = savedMaxDesyncFactor
- }()
- ipv6.MaxDesyncFactor = time.Nanosecond
-
tests := []struct {
name string
dupAddrTransmits uint8
@@ -2184,66 +2153,56 @@ func TestNoAutoGenTempAddrForLinkLocal(t *testing.T) {
},
}
- // This Run will not return until the parallel tests finish.
- //
- // We need this because we need to do some teardown work after the
- // parallel tests complete.
- //
- // See https://godoc.org/testing#hdr-Subtests_and_Sub_benchmarks for
- // more details.
- t.Run("group", func(t *testing.T) {
- for _, test := range tests {
- test := test
-
- t.Run(test.name, func(t *testing.T) {
- t.Parallel()
-
- ndpDisp := ndpDispatcher{
- dadC: make(chan ndpDADEvent, 1),
- autoGenAddrC: make(chan ndpAutoGenAddrEvent, 1),
- }
- e := channel.New(0, 1280, linkAddr1)
- s := stack.New(stack.Options{
- NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
- NDPConfigs: ipv6.NDPConfigurations{
- AutoGenTempGlobalAddresses: true,
- },
- NDPDisp: &ndpDisp,
- AutoGenLinkLocal: true,
- })},
- })
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ ndpDisp := ndpDispatcher{
+ dadC: make(chan ndpDADEvent, 1),
+ autoGenAddrC: make(chan ndpAutoGenAddrEvent, 1),
+ }
+ e := channel.New(0, 1280, linkAddr1)
+ clock := faketime.NewManualClock()
+ s := stack.New(stack.Options{
+ NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
+ NDPConfigs: ipv6.NDPConfigurations{
+ AutoGenTempGlobalAddresses: true,
+ },
+ NDPDisp: &ndpDisp,
+ AutoGenLinkLocal: true,
+ })},
+ Clock: clock,
+ })
- if err := s.CreateNIC(nicID, e); err != nil {
- t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
- }
+ if err := s.CreateNIC(nicID, e); err != nil {
+ t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
+ }
- // The stable link-local address should auto-generate and resolve DAD.
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, tcpip.AddressWithPrefix{Address: llAddr1, PrefixLen: header.IIDOffsetInIPv6Address * 8}, newAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- default:
- t.Fatal("expected addr auto gen event")
+ // The stable link-local address should auto-generate and resolve DAD.
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ if diff := checkAutoGenAddrEvent(e, tcpip.AddressWithPrefix{Address: llAddr1, PrefixLen: header.IIDOffsetInIPv6Address * 8}, newAddr); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- select {
- case e := <-ndpDisp.dadC:
- if diff := checkDADEvent(e, nicID, llAddr1, &stack.DADSucceeded{}); diff != "" {
- t.Errorf("DAD event mismatch (-want +got):\n%s", diff)
- }
- case <-time.After(time.Duration(test.dupAddrTransmits)*test.retransmitTimer + defaultAsyncPositiveEventTimeout):
- t.Fatal("timed out waiting for DAD event")
+ default:
+ t.Fatal("expected addr auto gen event")
+ }
+ clock.Advance(time.Duration(test.dupAddrTransmits) * test.retransmitTimer)
+ select {
+ case e := <-ndpDisp.dadC:
+ if diff := checkDADEvent(e, nicID, llAddr1, &stack.DADSucceeded{}); diff != "" {
+ t.Errorf("DAD event mismatch (-want +got):\n%s", diff)
}
+ default:
+ t.Fatal("timed out waiting for DAD event")
+ }
- // No new addresses should be generated.
- select {
- case e := <-ndpDisp.autoGenAddrC:
- t.Errorf("got unxpected auto gen addr event = %+v", e)
- case <-time.After(defaultAsyncNegativeEventTimeout):
- }
- })
- }
- })
+ // No new addresses should be generated.
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ t.Errorf("got unxpected auto gen addr event = %+v", e)
+ default:
+ }
+ })
+ }
}
// TestNoAutoGenTempAddrWithoutStableAddr tests that a temporary SLAAC address
@@ -2256,12 +2215,6 @@ func TestNoAutoGenTempAddrWithoutStableAddr(t *testing.T) {
retransmitTimer = 2 * time.Second
)
- savedMaxDesyncFactor := ipv6.MaxDesyncFactor
- defer func() {
- ipv6.MaxDesyncFactor = savedMaxDesyncFactor
- }()
- ipv6.MaxDesyncFactor = 0
-
prefix, _, addr := prefixSubnetAddr(0, linkAddr1)
var tempIIDHistory [header.IIDSize]byte
header.InitialTempIID(tempIIDHistory[:], nil, nicID)
@@ -2272,6 +2225,7 @@ func TestNoAutoGenTempAddrWithoutStableAddr(t *testing.T) {
autoGenAddrC: make(chan ndpAutoGenAddrEvent, 1),
}
e := channel.New(0, 1280, linkAddr1)
+ clock := faketime.NewManualClock()
s := stack.New(stack.Options{
NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
DADConfigs: stack.DADConfigurations{
@@ -2285,6 +2239,7 @@ func TestNoAutoGenTempAddrWithoutStableAddr(t *testing.T) {
},
NDPDisp: &ndpDisp,
})},
+ Clock: clock,
})
if err := s.CreateNIC(nicID, e); err != nil {
@@ -2314,12 +2269,13 @@ func TestNoAutoGenTempAddrWithoutStableAddr(t *testing.T) {
// Wait for DAD to complete for the stable address then expect the temporary
// address to be generated.
+ clock.Advance(dadTransmits * retransmitTimer)
select {
case e := <-ndpDisp.dadC:
if diff := checkDADEvent(e, nicID, addr.Address, &stack.DADSucceeded{}); diff != "" {
t.Errorf("DAD event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(dadTransmits*retransmitTimer + defaultAsyncPositiveEventTimeout):
+ default:
t.Fatal("timed out waiting for DAD event")
}
select {
@@ -2327,7 +2283,7 @@ func TestNoAutoGenTempAddrWithoutStableAddr(t *testing.T) {
if diff := checkAutoGenAddrEvent(e, tempAddr, newAddr); diff != "" {
t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(defaultAsyncPositiveEventTimeout):
+ default:
t.Fatal("timed out waiting for addr auto gen event")
}
}
@@ -2336,46 +2292,44 @@ func TestNoAutoGenTempAddrWithoutStableAddr(t *testing.T) {
// regenerated.
func TestAutoGenTempAddrRegen(t *testing.T) {
const (
- nicID = 1
- regenAfter = 2 * time.Second
- newMinVL = 10
- newMinVLDuration = newMinVL * time.Second
- )
+ nicID = 1
+ regenAdv = 2 * time.Second
- savedMaxDesyncFactor := ipv6.MaxDesyncFactor
- savedMinMaxTempAddrPreferredLifetime := ipv6.MinMaxTempAddrPreferredLifetime
- savedMinMaxTempAddrValidLifetime := ipv6.MinMaxTempAddrValidLifetime
- defer func() {
- ipv6.MaxDesyncFactor = savedMaxDesyncFactor
- ipv6.MinMaxTempAddrPreferredLifetime = savedMinMaxTempAddrPreferredLifetime
- ipv6.MinMaxTempAddrValidLifetime = savedMinMaxTempAddrValidLifetime
- }()
- ipv6.MaxDesyncFactor = 0
- ipv6.MinMaxTempAddrPreferredLifetime = newMinVLDuration
- ipv6.MinMaxTempAddrValidLifetime = newMinVLDuration
+ numTempAddrs = 3
+ maxTempAddrValidLifetime = numTempAddrs * ipv6.MinPrefixInformationValidLifetimeForUpdate
+ )
prefix, _, addr := prefixSubnetAddr(0, linkAddr1)
var tempIIDHistory [header.IIDSize]byte
header.InitialTempIID(tempIIDHistory[:], nil, nicID)
- tempAddr1 := header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], addr.Address)
- tempAddr2 := header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], addr.Address)
- tempAddr3 := header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], addr.Address)
+ var tempAddrs [numTempAddrs]tcpip.AddressWithPrefix
+ for i := 0; i < len(tempAddrs); i++ {
+ tempAddrs[i] = header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], addr.Address)
+ }
ndpDisp := ndpDispatcher{
autoGenAddrC: make(chan ndpAutoGenAddrEvent, 2),
}
e := channel.New(0, 1280, linkAddr1)
ndpConfigs := ipv6.NDPConfigurations{
- HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
- AutoGenGlobalAddresses: true,
- AutoGenTempGlobalAddresses: true,
- RegenAdvanceDuration: newMinVLDuration - regenAfter,
+ HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
+ AutoGenGlobalAddresses: true,
+ AutoGenTempGlobalAddresses: true,
+ RegenAdvanceDuration: regenAdv,
+ MaxTempAddrValidLifetime: maxTempAddrValidLifetime,
+ MaxTempAddrPreferredLifetime: ipv6.MinPrefixInformationValidLifetimeForUpdate,
+ }
+ clock := faketime.NewManualClock()
+ randSource := savingRandSource{
+ s: rand.NewSource(time.Now().UnixNano()),
}
s := stack.New(stack.Options{
NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
NDPConfigs: ndpConfigs,
NDPDisp: &ndpDisp,
})},
+ Clock: clock,
+ RandSource: &randSource,
})
if err := s.CreateNIC(nicID, e); err != nil {
@@ -2398,36 +2352,43 @@ func TestAutoGenTempAddrRegen(t *testing.T) {
expectAutoGenAddrEventAsync := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType, timeout time.Duration) {
t.Helper()
+ clock.Advance(timeout)
select {
case e := <-ndpDisp.autoGenAddrC:
if diff := checkAutoGenAddrEvent(e, addr, eventType); diff != "" {
t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(timeout):
+ default:
t.Fatal("timed out waiting for addr auto gen event")
}
}
+ tempDesyncFactor := time.Duration(randSource.lastInt63) % ipv6.MaxDesyncFactor
+ effectiveMaxTempAddrPL := ipv6.MinPrefixInformationValidLifetimeForUpdate - tempDesyncFactor
+ // The time since the last regeneration before a new temporary address is
+ // generated.
+ tempAddrRegenenerationTime := effectiveMaxTempAddrPL - regenAdv
+
// Receive an RA with prefix1 in an NDP Prefix Information option (PI)
// with non-zero valid & preferred lifetimes.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 100))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, minVLSeconds, minVLSeconds))
expectAutoGenAddrEvent(addr, newAddr)
- expectAutoGenAddrEvent(tempAddr1, newAddr)
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr, tempAddr1}, nil); mismatch != "" {
+ expectAutoGenAddrEvent(tempAddrs[0], newAddr)
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr, tempAddrs[0]}, nil); mismatch != "" {
t.Fatal(mismatch)
}
// Wait for regeneration
- expectAutoGenAddrEventAsync(tempAddr2, newAddr, regenAfter+defaultAsyncPositiveEventTimeout)
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr, tempAddr1, tempAddr2}, nil); mismatch != "" {
+ expectAutoGenAddrEventAsync(tempAddrs[1], newAddr, tempAddrRegenenerationTime)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, minVLSeconds, minVLSeconds))
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr, tempAddrs[0], tempAddrs[1]}, nil); mismatch != "" {
t.Fatal(mismatch)
}
+ expectAutoGenAddrEventAsync(tempAddrs[0], deprecatedAddr, regenAdv)
// Wait for regeneration
- expectAutoGenAddrEventAsync(tempAddr3, newAddr, regenAfter+defaultAsyncPositiveEventTimeout)
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr, tempAddr1, tempAddr2, tempAddr3}, nil); mismatch != "" {
- t.Fatal(mismatch)
- }
+ expectAutoGenAddrEventAsync(tempAddrs[2], newAddr, tempAddrRegenenerationTime-regenAdv)
+ expectAutoGenAddrEventAsync(tempAddrs[1], deprecatedAddr, regenAdv)
// Stop generating temporary addresses
ndpConfigs.AutoGenTempGlobalAddresses = false
@@ -2438,45 +2399,24 @@ func TestAutoGenTempAddrRegen(t *testing.T) {
ndpEP.SetNDPConfigurations(ndpConfigs)
}
+ // Refresh lifetimes and wait for the last temporary address to be deprecated.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, minVLSeconds, minVLSeconds))
+ expectAutoGenAddrEventAsync(tempAddrs[2], deprecatedAddr, effectiveMaxTempAddrPL-regenAdv)
+
+ // Refresh lifetimes such that the prefix is valid and preferred forever.
+ //
+ // This should not affect the lifetimes of temporary addresses because they
+ // are capped by the maximum valid and preferred lifetimes for temporary
+ // addresses.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, infiniteLifetimeSeconds, infiniteLifetimeSeconds))
+
// Wait for all the temporary addresses to get invalidated.
- tempAddrs := []tcpip.AddressWithPrefix{tempAddr1, tempAddr2, tempAddr3}
- invalidateAfter := newMinVLDuration - 2*regenAfter
+ invalidateAfter := maxTempAddrValidLifetime - clock.NowMonotonic().Sub(tcpip.MonotonicTime{})
for _, addr := range tempAddrs {
- // Wait for a deprecation then invalidation event, or just an invalidation
- // event. We need to cover both cases but cannot deterministically hit both
- // cases because the deprecation and invalidation jobs could execute in any
- // order.
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, addr, deprecatedAddr); diff == "" {
- // If we get a deprecation event first, we should get an invalidation
- // event almost immediately after.
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, addr, invalidatedAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- case <-time.After(defaultAsyncPositiveEventTimeout):
- t.Fatal("timed out waiting for addr auto gen event")
- }
- } else if diff := checkAutoGenAddrEvent(e, addr, invalidatedAddr); diff == "" {
- // If we get an invalidation event first, we shouldn't get a deprecation
- // event after.
- select {
- case e := <-ndpDisp.autoGenAddrC:
- t.Fatalf("unexpectedly got an auto-generated event = %+v", e)
- case <-time.After(defaultAsyncNegativeEventTimeout):
- }
- } else {
- t.Fatalf("got unexpected auto-generated event = %+v", e)
- }
- case <-time.After(invalidateAfter + defaultAsyncPositiveEventTimeout):
- t.Fatal("timed out waiting for addr auto gen event")
- }
-
- invalidateAfter = regenAfter
+ expectAutoGenAddrEventAsync(addr, invalidatedAddr, invalidateAfter)
+ invalidateAfter = tempAddrRegenenerationTime
}
- if mismatch := addressCheck(s.NICInfo()[1].ProtocolAddresses, []tcpip.AddressWithPrefix{addr}, tempAddrs); mismatch != "" {
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr}, tempAddrs[:]); mismatch != "" {
t.Fatal(mismatch)
}
}
@@ -2485,52 +2425,54 @@ func TestAutoGenTempAddrRegen(t *testing.T) {
// regeneration job gets updated when refreshing the address's lifetimes.
func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) {
const (
- nicID = 1
- regenAfter = 2 * time.Second
- newMinVL = 10
- newMinVLDuration = newMinVL * time.Second
- )
+ nicID = 1
+ regenAdv = 2 * time.Second
- savedMaxDesyncFactor := ipv6.MaxDesyncFactor
- savedMinMaxTempAddrPreferredLifetime := ipv6.MinMaxTempAddrPreferredLifetime
- savedMinMaxTempAddrValidLifetime := ipv6.MinMaxTempAddrValidLifetime
- defer func() {
- ipv6.MaxDesyncFactor = savedMaxDesyncFactor
- ipv6.MinMaxTempAddrPreferredLifetime = savedMinMaxTempAddrPreferredLifetime
- ipv6.MinMaxTempAddrValidLifetime = savedMinMaxTempAddrValidLifetime
- }()
- ipv6.MaxDesyncFactor = 0
- ipv6.MinMaxTempAddrPreferredLifetime = newMinVLDuration
- ipv6.MinMaxTempAddrValidLifetime = newMinVLDuration
+ numTempAddrs = 3
+ maxTempAddrPreferredLifetime = ipv6.MinPrefixInformationValidLifetimeForUpdate
+ maxTempAddrPreferredLifetimeSeconds = uint32(maxTempAddrPreferredLifetime / time.Second)
+ )
prefix, _, addr := prefixSubnetAddr(0, linkAddr1)
var tempIIDHistory [header.IIDSize]byte
header.InitialTempIID(tempIIDHistory[:], nil, nicID)
- tempAddr1 := header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], addr.Address)
- tempAddr2 := header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], addr.Address)
- tempAddr3 := header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], addr.Address)
+ var tempAddrs [numTempAddrs]tcpip.AddressWithPrefix
+ for i := 0; i < len(tempAddrs); i++ {
+ tempAddrs[i] = header.GenerateTempIPv6SLAACAddr(tempIIDHistory[:], addr.Address)
+ }
ndpDisp := ndpDispatcher{
autoGenAddrC: make(chan ndpAutoGenAddrEvent, 2),
}
e := channel.New(0, 1280, linkAddr1)
ndpConfigs := ipv6.NDPConfigurations{
- HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
- AutoGenGlobalAddresses: true,
- AutoGenTempGlobalAddresses: true,
- RegenAdvanceDuration: newMinVLDuration - regenAfter,
+ HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
+ AutoGenGlobalAddresses: true,
+ AutoGenTempGlobalAddresses: true,
+ RegenAdvanceDuration: regenAdv,
+ MaxTempAddrPreferredLifetime: maxTempAddrPreferredLifetime,
+ MaxTempAddrValidLifetime: maxTempAddrPreferredLifetime * 2,
+ }
+ clock := faketime.NewManualClock()
+ initialTime := clock.NowMonotonic()
+ randSource := savingRandSource{
+ s: rand.NewSource(time.Now().UnixNano()),
}
s := stack.New(stack.Options{
NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
NDPConfigs: ndpConfigs,
NDPDisp: &ndpDisp,
})},
+ Clock: clock,
+ RandSource: &randSource,
})
if err := s.CreateNIC(nicID, e); err != nil {
t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
}
+ tempDesyncFactor := time.Duration(randSource.lastInt63) % ipv6.MaxDesyncFactor
+
expectAutoGenAddrEvent := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
t.Helper()
@@ -2547,22 +2489,23 @@ func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) {
expectAutoGenAddrEventAsync := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType, timeout time.Duration) {
t.Helper()
+ clock.Advance(timeout)
select {
case e := <-ndpDisp.autoGenAddrC:
if diff := checkAutoGenAddrEvent(e, addr, eventType); diff != "" {
t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(timeout):
+ default:
t.Fatal("timed out waiting for addr auto gen event")
}
}
// Receive an RA with prefix1 in an NDP Prefix Information option (PI)
// with non-zero valid & preferred lifetimes.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 100))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, maxTempAddrPreferredLifetimeSeconds, maxTempAddrPreferredLifetimeSeconds))
expectAutoGenAddrEvent(addr, newAddr)
- expectAutoGenAddrEvent(tempAddr1, newAddr)
- if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr, tempAddr1}, nil); mismatch != "" {
+ expectAutoGenAddrEvent(tempAddrs[0], newAddr)
+ if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, []tcpip.AddressWithPrefix{addr, tempAddrs[0]}, nil); mismatch != "" {
t.Fatal(mismatch)
}
@@ -2570,13 +2513,27 @@ func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) {
//
// A new temporary address should be generated after the regeneration
// time has passed since the prefix is deprecated.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 0))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, maxTempAddrPreferredLifetimeSeconds, 0))
expectAutoGenAddrEvent(addr, deprecatedAddr)
- expectAutoGenAddrEvent(tempAddr1, deprecatedAddr)
+ expectAutoGenAddrEvent(tempAddrs[0], deprecatedAddr)
select {
case e := <-ndpDisp.autoGenAddrC:
- t.Fatalf("unexpected auto gen addr event = %+v", e)
- case <-time.After(regenAfter + defaultAsyncNegativeEventTimeout):
+ t.Fatalf("unexpected auto gen addr event = %#v", e)
+ default:
+ }
+
+ effectiveMaxTempAddrPL := maxTempAddrPreferredLifetime - tempDesyncFactor
+ // The time since the last regeneration before a new temporary address is
+ // generated.
+ tempAddrRegenenerationTime := effectiveMaxTempAddrPL - regenAdv
+
+ // Advance the clock by the regeneration time but don't expect a new temporary
+ // address as the prefix is deprecated.
+ clock.Advance(tempAddrRegenenerationTime)
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ t.Fatalf("unexpected auto gen addr event = %#v", e)
+ default:
}
// Prefer the prefix again.
@@ -2584,8 +2541,15 @@ func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) {
// A new temporary address should immediately be generated since the
// regeneration time has already passed since the last address was generated
// - this regeneration does not depend on a job.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 100))
- expectAutoGenAddrEvent(tempAddr2, newAddr)
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, maxTempAddrPreferredLifetimeSeconds, maxTempAddrPreferredLifetimeSeconds))
+ expectAutoGenAddrEvent(tempAddrs[1], newAddr)
+ // Wait for the first temporary address to be deprecated.
+ expectAutoGenAddrEventAsync(tempAddrs[0], deprecatedAddr, regenAdv)
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ t.Fatalf("unexpected auto gen addr event = %s", e)
+ default:
+ }
// Increase the maximum lifetimes for temporary addresses to large values
// then refresh the lifetimes of the prefix.
@@ -2596,34 +2560,30 @@ func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) {
// regenerate a new temporary address. Note, new addresses are only
// regenerated after the preferred lifetime - the regenerate advance duration
// as paased.
- ndpConfigs.MaxTempAddrValidLifetime = 100 * time.Second
- ndpConfigs.MaxTempAddrPreferredLifetime = 100 * time.Second
+ const largeLifetimeSeconds = minVLSeconds * 2
+ const largeLifetime = time.Duration(largeLifetimeSeconds) * time.Second
+ ndpConfigs.MaxTempAddrValidLifetime = 2 * largeLifetime
+ ndpConfigs.MaxTempAddrPreferredLifetime = largeLifetime
ipv6Ep, err := s.GetNetworkEndpoint(nicID, header.IPv6ProtocolNumber)
if err != nil {
t.Fatalf("s.GetNetworkEndpoint(%d, %d): %s", nicID, header.IPv6ProtocolNumber, err)
}
ndpEP := ipv6Ep.(ipv6.NDPEndpoint)
ndpEP.SetNDPConfigurations(ndpConfigs)
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 100))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, largeLifetimeSeconds, largeLifetimeSeconds))
+ timeSinceInitialTime := clock.NowMonotonic().Sub(initialTime)
+ clock.Advance(largeLifetime - timeSinceInitialTime)
+ expectAutoGenAddrEvent(tempAddrs[0], deprecatedAddr)
+ // to offset the advement of time to test the first temporary address's
+ // deprecation after the second was generated
+ advLess := regenAdv
+ expectAutoGenAddrEventAsync(tempAddrs[2], newAddr, timeSinceInitialTime-advLess-(tempDesyncFactor+regenAdv))
+ expectAutoGenAddrEventAsync(tempAddrs[1], deprecatedAddr, regenAdv)
select {
case e := <-ndpDisp.autoGenAddrC:
t.Fatalf("unexpected auto gen addr event = %+v", e)
- case <-time.After(regenAfter + defaultAsyncNegativeEventTimeout):
+ default:
}
-
- // Set the maximum lifetimes for temporary addresses such that on the next
- // RA, the regeneration job gets scheduled again.
- //
- // The maximum lifetime is the sum of the minimum lifetimes for temporary
- // addresses + the time that has already passed since the last address was
- // generated so that the regeneration job is needed to generate the next
- // address.
- newLifetimes := newMinVLDuration + regenAfter + defaultAsyncNegativeEventTimeout
- ndpConfigs.MaxTempAddrValidLifetime = newLifetimes
- ndpConfigs.MaxTempAddrPreferredLifetime = newLifetimes
- ndpEP.SetNDPConfigurations(ndpConfigs)
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 100))
- expectAutoGenAddrEventAsync(tempAddr3, newAddr, regenAfter+defaultAsyncPositiveEventTimeout)
}
// TestMixedSLAACAddrConflictRegen tests SLAAC address regeneration in response
@@ -2851,13 +2811,14 @@ func TestMixedSLAACAddrConflictRegen(t *testing.T) {
// stack.Stack will have a default route through the router (llAddr3) installed
// and a static link-address (linkAddr3) added to the link address cache for the
// router.
-func stackAndNdpDispatcherWithDefaultRoute(t *testing.T, nicID tcpip.NICID) (*ndpDispatcher, *channel.Endpoint, *stack.Stack) {
+func stackAndNdpDispatcherWithDefaultRoute(t *testing.T, nicID tcpip.NICID) (*ndpDispatcher, *channel.Endpoint, *stack.Stack, *faketime.ManualClock) {
t.Helper()
ndpDisp := &ndpDispatcher{
autoGenAddrC: make(chan ndpAutoGenAddrEvent, 1),
}
e := channel.New(0, 1280, linkAddr1)
e.LinkEPCapabilities |= stack.CapabilityResolutionRequired
+ clock := faketime.NewManualClock()
s := stack.New(stack.Options{
NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
NDPConfigs: ipv6.NDPConfigurations{
@@ -2867,6 +2828,7 @@ func stackAndNdpDispatcherWithDefaultRoute(t *testing.T, nicID tcpip.NICID) (*nd
NDPDisp: ndpDisp,
})},
TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol},
+ Clock: clock,
})
if err := s.CreateNIC(nicID, e); err != nil {
t.Fatalf("CreateNIC(%d, _) = %s", nicID, err)
@@ -2880,7 +2842,7 @@ func stackAndNdpDispatcherWithDefaultRoute(t *testing.T, nicID tcpip.NICID) (*nd
if err := s.AddStaticNeighbor(nicID, ipv6.ProtocolNumber, llAddr3, linkAddr3); err != nil {
t.Fatalf("s.AddStaticNeighbor(%d, %d, %s, %s): %s", nicID, ipv6.ProtocolNumber, llAddr3, linkAddr3, err)
}
- return ndpDisp, e, s
+ return ndpDisp, e, s, clock
}
// addrForNewConnectionTo returns the local address used when creating a new
@@ -2954,7 +2916,7 @@ func TestAutoGenAddrDeprecateFromPI(t *testing.T) {
prefix1, _, addr1 := prefixSubnetAddr(0, linkAddr1)
prefix2, _, addr2 := prefixSubnetAddr(1, linkAddr1)
- ndpDisp, e, s := stackAndNdpDispatcherWithDefaultRoute(t, nicID)
+ ndpDisp, e, s, _ := stackAndNdpDispatcherWithDefaultRoute(t, nicID)
expectAutoGenAddrEvent := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
t.Helper()
@@ -3057,19 +3019,11 @@ func TestAutoGenAddrDeprecateFromPI(t *testing.T) {
// when its preferred lifetime expires.
func TestAutoGenAddrJobDeprecation(t *testing.T) {
const nicID = 1
- const newMinVL = 2
- newMinVLDuration := newMinVL * time.Second
-
- saved := ipv6.MinPrefixInformationValidLifetimeForUpdate
- defer func() {
- ipv6.MinPrefixInformationValidLifetimeForUpdate = saved
- }()
- ipv6.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration
prefix1, _, addr1 := prefixSubnetAddr(0, linkAddr1)
prefix2, _, addr2 := prefixSubnetAddr(1, linkAddr1)
- ndpDisp, e, s := stackAndNdpDispatcherWithDefaultRoute(t, nicID)
+ ndpDisp, e, s, clock := stackAndNdpDispatcherWithDefaultRoute(t, nicID)
expectAutoGenAddrEvent := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
t.Helper()
@@ -3087,12 +3041,13 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
expectAutoGenAddrEventAfter := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType, timeout time.Duration) {
t.Helper()
+ clock.Advance(timeout)
select {
case e := <-ndpDisp.autoGenAddrC:
if diff := checkAutoGenAddrEvent(e, addr, eventType); diff != "" {
t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(timeout):
+ default:
t.Fatal("timed out waiting for addr auto gen event")
}
}
@@ -3110,7 +3065,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
}
// Receive PI for prefix2.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, 100, 100))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, infiniteLifetimeSeconds, infiniteLifetimeSeconds))
expectAutoGenAddrEvent(addr2, newAddr)
if !containsV6Addr(s.NICInfo()[nicID].ProtocolAddresses, addr2) {
t.Fatalf("should have %s in the list of addresses", addr2)
@@ -3129,7 +3084,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
expectPrimaryAddr(addr1)
// Refresh lifetime for addr of prefix1.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, newMinVL, newMinVL-1))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, minVLSeconds, minVLSeconds-1))
select {
case <-ndpDisp.autoGenAddrC:
t.Fatal("unexpectedly got an auto-generated event")
@@ -3138,7 +3093,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
expectPrimaryAddr(addr1)
// Wait for addr of prefix1 to be deprecated.
- expectAutoGenAddrEventAfter(addr1, deprecatedAddr, newMinVLDuration-time.Second+defaultAsyncPositiveEventTimeout)
+ expectAutoGenAddrEventAfter(addr1, deprecatedAddr, ipv6.MinPrefixInformationValidLifetimeForUpdate-time.Second)
if !containsV6Addr(s.NICInfo()[nicID].ProtocolAddresses, addr1) {
t.Fatalf("should not have %s in the list of addresses", addr1)
}
@@ -3148,6 +3103,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
// addr2 should be the primary endpoint now since addr1 is deprecated but
// addr2 is not.
expectPrimaryAddr(addr2)
+
// addr1 is deprecated but if explicitly requested, it should be used.
fullAddr1 := tcpip.FullAddress{Addr: addr1.Address, NIC: nicID}
if got := addrForNewConnectionWithAddr(t, s, fullAddr1); got != addr1.Address {
@@ -3156,7 +3112,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
// Refresh valid lifetime for addr of prefix1, w/ 0 preferred lifetime to make
// sure we do not get a deprecation event again.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, newMinVL, 0))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, minVLSeconds, 0))
select {
case <-ndpDisp.autoGenAddrC:
t.Fatal("unexpectedly got an auto-generated event")
@@ -3168,7 +3124,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
}
// Refresh lifetimes for addr of prefix1.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, newMinVL, newMinVL-1))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix1, true, true, minVLSeconds, minVLSeconds-1))
select {
case <-ndpDisp.autoGenAddrC:
t.Fatal("unexpectedly got an auto-generated event")
@@ -3178,7 +3134,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
expectPrimaryAddr(addr1)
// Wait for addr of prefix1 to be deprecated.
- expectAutoGenAddrEventAfter(addr1, deprecatedAddr, newMinVLDuration-time.Second+defaultAsyncPositiveEventTimeout)
+ expectAutoGenAddrEventAfter(addr1, deprecatedAddr, ipv6.MinPrefixInformationValidLifetimeForUpdate-time.Second)
if !containsV6Addr(s.NICInfo()[nicID].ProtocolAddresses, addr1) {
t.Fatalf("should not have %s in the list of addresses", addr1)
}
@@ -3192,7 +3148,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
}
// Wait for addr of prefix1 to be invalidated.
- expectAutoGenAddrEventAfter(addr1, invalidatedAddr, time.Second+defaultAsyncPositiveEventTimeout)
+ expectAutoGenAddrEventAfter(addr1, invalidatedAddr, time.Second)
if containsV6Addr(s.NICInfo()[nicID].ProtocolAddresses, addr1) {
t.Fatalf("should not have %s in the list of addresses", addr1)
}
@@ -3202,7 +3158,7 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
expectPrimaryAddr(addr2)
// Refresh both lifetimes for addr of prefix2 to the same value.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, newMinVL, newMinVL))
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix2, true, true, minVLSeconds, minVLSeconds))
select {
case <-ndpDisp.autoGenAddrC:
t.Fatal("unexpectedly got an auto-generated event")
@@ -3214,6 +3170,17 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
// cases because the deprecation and invalidation handlers could be handled in
// either deprecation then invalidation, or invalidation then deprecation
// (which should be cancelled by the invalidation handler).
+ //
+ // Since we're about to cause both events to fire, we need the dispatcher
+ // channel to be able to hold both.
+ if got, want := len(ndpDisp.autoGenAddrC), 0; got != want {
+ t.Fatalf("got len(ndpDisp.autoGenAddrC) = %d, want %d", got, want)
+ }
+ if got, want := cap(ndpDisp.autoGenAddrC), 1; got != want {
+ t.Fatalf("got cap(ndpDisp.autoGenAddrC) = %d, want %d", got, want)
+ }
+ ndpDisp.autoGenAddrC = make(chan ndpAutoGenAddrEvent, 2)
+ clock.Advance(ipv6.MinPrefixInformationValidLifetimeForUpdate)
select {
case e := <-ndpDisp.autoGenAddrC:
if diff := checkAutoGenAddrEvent(e, addr2, deprecatedAddr); diff == "" {
@@ -3224,21 +3191,21 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
if diff := checkAutoGenAddrEvent(e, addr2, invalidatedAddr); diff != "" {
t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(defaultAsyncPositiveEventTimeout):
+ default:
t.Fatal("timed out waiting for addr auto gen event")
}
} else if diff := checkAutoGenAddrEvent(e, addr2, invalidatedAddr); diff == "" {
- // If we get an invalidation event first, we should not get a deprecation
+ // If we get an invalidation event first, we should not get a deprecation
// event after.
select {
case <-ndpDisp.autoGenAddrC:
t.Fatal("unexpectedly got an auto-generated event")
- case <-time.After(defaultAsyncNegativeEventTimeout):
+ default:
}
} else {
t.Fatalf("got unexpected auto-generated event")
}
- case <-time.After(newMinVLDuration + defaultAsyncPositiveEventTimeout):
+ default:
t.Fatal("timed out waiting for addr auto gen event")
}
if containsV6Addr(s.NICInfo()[nicID].ProtocolAddresses, addr1) {
@@ -3275,15 +3242,6 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) {
// infinite values.
func TestAutoGenAddrFiniteToInfiniteToFiniteVL(t *testing.T) {
const infiniteVLSeconds = 2
- const minVLSeconds = 1
- savedIL := header.NDPInfiniteLifetime
- savedMinVL := ipv6.MinPrefixInformationValidLifetimeForUpdate
- defer func() {
- ipv6.MinPrefixInformationValidLifetimeForUpdate = savedMinVL
- header.NDPInfiniteLifetime = savedIL
- }()
- ipv6.MinPrefixInformationValidLifetimeForUpdate = minVLSeconds * time.Second
- header.NDPInfiniteLifetime = infiniteVLSeconds * time.Second
prefix, _, addr := prefixSubnetAddr(0, linkAddr1)
@@ -3307,68 +3265,58 @@ func TestAutoGenAddrFiniteToInfiniteToFiniteVL(t *testing.T) {
},
}
- // This Run will not return until the parallel tests finish.
- //
- // We need this because we need to do some teardown work after the
- // parallel tests complete.
- //
- // See https://godoc.org/testing#hdr-Subtests_and_Sub_benchmarks for
- // more details.
- t.Run("group", func(t *testing.T) {
- for _, test := range tests {
- test := test
-
- t.Run(test.name, func(t *testing.T) {
- t.Parallel()
-
- ndpDisp := ndpDispatcher{
- autoGenAddrC: make(chan ndpAutoGenAddrEvent, 1),
- }
- e := channel.New(0, 1280, linkAddr1)
- s := stack.New(stack.Options{
- NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
- NDPConfigs: ipv6.NDPConfigurations{
- HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
- AutoGenGlobalAddresses: true,
- },
- NDPDisp: &ndpDisp,
- })},
- })
-
- if err := s.CreateNIC(1, e); err != nil {
- t.Fatalf("CreateNIC(1) = %s", err)
- }
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ ndpDisp := ndpDispatcher{
+ autoGenAddrC: make(chan ndpAutoGenAddrEvent, 1),
+ }
+ e := channel.New(0, 1280, linkAddr1)
+ clock := faketime.NewManualClock()
+ s := stack.New(stack.Options{
+ NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
+ NDPConfigs: ipv6.NDPConfigurations{
+ HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
+ AutoGenGlobalAddresses: true,
+ },
+ NDPDisp: &ndpDisp,
+ })},
+ Clock: clock,
+ })
- // Receive an RA with finite prefix.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, minVLSeconds, 0))
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, addr, newAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
+ if err := s.CreateNIC(1, e); err != nil {
+ t.Fatalf("CreateNIC(1) = %s", err)
+ }
- default:
- t.Fatal("expected addr auto gen event")
+ // Receive an RA with finite prefix.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, minVLSeconds, 0))
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ if diff := checkAutoGenAddrEvent(e, addr, newAddr); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- // Receive an new RA with prefix with infinite VL.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, test.infiniteVL, 0))
+ default:
+ t.Fatal("expected addr auto gen event")
+ }
- // Receive a new RA with prefix with finite VL.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, minVLSeconds, 0))
+ // Receive an new RA with prefix with infinite VL.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, test.infiniteVL, 0))
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, addr, invalidatedAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
+ // Receive a new RA with prefix with finite VL.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, minVLSeconds, 0))
- case <-time.After(minVLSeconds*time.Second + defaultAsyncPositiveEventTimeout):
- t.Fatal("timeout waiting for addr auto gen event")
+ clock.Advance(ipv6.MinPrefixInformationValidLifetimeForUpdate)
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ if diff := checkAutoGenAddrEvent(e, addr, invalidatedAddr); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- })
- }
- })
+
+ default:
+ t.Fatal("timeout waiting for addr auto gen event")
+ }
+ })
+ }
}
// TestAutoGenAddrValidLifetimeUpdates tests that the valid lifetime of an
@@ -3376,12 +3324,6 @@ func TestAutoGenAddrFiniteToInfiniteToFiniteVL(t *testing.T) {
// RFC 4862 section 5.5.3.e.
func TestAutoGenAddrValidLifetimeUpdates(t *testing.T) {
const infiniteVL = 4294967295
- const newMinVL = 4
- saved := ipv6.MinPrefixInformationValidLifetimeForUpdate
- defer func() {
- ipv6.MinPrefixInformationValidLifetimeForUpdate = saved
- }()
- ipv6.MinPrefixInformationValidLifetimeForUpdate = newMinVL * time.Second
prefix, _, addr := prefixSubnetAddr(0, linkAddr1)
@@ -3392,137 +3334,129 @@ func TestAutoGenAddrValidLifetimeUpdates(t *testing.T) {
evl uint32
}{
// Should update the VL to the minimum VL for updating if the
- // new VL is less than newMinVL but was originally greater than
+ // new VL is less than minVLSeconds but was originally greater than
// it.
{
"LargeVLToVLLessThanMinVLForUpdate",
9999,
1,
- newMinVL,
+ minVLSeconds,
},
{
"LargeVLTo0",
9999,
0,
- newMinVL,
+ minVLSeconds,
},
{
"InfiniteVLToVLLessThanMinVLForUpdate",
infiniteVL,
1,
- newMinVL,
+ minVLSeconds,
},
{
"InfiniteVLTo0",
infiniteVL,
0,
- newMinVL,
+ minVLSeconds,
},
- // Should not update VL if original VL was less than newMinVL
- // and the new VL is also less than newMinVL.
+ // Should not update VL if original VL was less than minVLSeconds
+ // and the new VL is also less than minVLSeconds.
{
"ShouldNotUpdateWhenBothOldAndNewAreLessThanMinVLForUpdate",
- newMinVL - 1,
- newMinVL - 3,
- newMinVL - 1,
+ minVLSeconds - 1,
+ minVLSeconds - 3,
+ minVLSeconds - 1,
},
// Should take the new VL if the new VL is greater than the
- // remaining time or is greater than newMinVL.
+ // remaining time or is greater than minVLSeconds.
{
"MorethanMinVLToLesserButStillMoreThanMinVLForUpdate",
- newMinVL + 5,
- newMinVL + 3,
- newMinVL + 3,
+ minVLSeconds + 5,
+ minVLSeconds + 3,
+ minVLSeconds + 3,
},
{
"SmallVLToGreaterVLButStillLessThanMinVLForUpdate",
- newMinVL - 3,
- newMinVL - 1,
- newMinVL - 1,
+ minVLSeconds - 3,
+ minVLSeconds - 1,
+ minVLSeconds - 1,
},
{
"SmallVLToGreaterVLThatIsMoreThaMinVLForUpdate",
- newMinVL - 3,
- newMinVL + 1,
- newMinVL + 1,
+ minVLSeconds - 3,
+ minVLSeconds + 1,
+ minVLSeconds + 1,
},
}
- // This Run will not return until the parallel tests finish.
- //
- // We need this because we need to do some teardown work after the
- // parallel tests complete.
- //
- // See https://godoc.org/testing#hdr-Subtests_and_Sub_benchmarks for
- // more details.
- t.Run("group", func(t *testing.T) {
- for _, test := range tests {
- test := test
-
- t.Run(test.name, func(t *testing.T) {
- t.Parallel()
-
- ndpDisp := ndpDispatcher{
- autoGenAddrC: make(chan ndpAutoGenAddrEvent, 10),
- }
- e := channel.New(10, 1280, linkAddr1)
- s := stack.New(stack.Options{
- NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
- NDPConfigs: ipv6.NDPConfigurations{
- HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
- AutoGenGlobalAddresses: true,
- },
- NDPDisp: &ndpDisp,
- })},
- })
+ for _, test := range tests {
+ t.Run(test.name, func(t *testing.T) {
+ ndpDisp := ndpDispatcher{
+ autoGenAddrC: make(chan ndpAutoGenAddrEvent, 10),
+ }
+ e := channel.New(10, 1280, linkAddr1)
+ clock := faketime.NewManualClock()
+ s := stack.New(stack.Options{
+ NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
+ NDPConfigs: ipv6.NDPConfigurations{
+ HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
+ AutoGenGlobalAddresses: true,
+ },
+ NDPDisp: &ndpDisp,
+ })},
+ Clock: clock,
+ })
- if err := s.CreateNIC(1, e); err != nil {
- t.Fatalf("CreateNIC(1) = %s", err)
- }
+ if err := s.CreateNIC(1, e); err != nil {
+ t.Fatalf("CreateNIC(1) = %s", err)
+ }
- // Receive an RA with prefix with initial VL,
- // test.ovl.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, test.ovl, 0))
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, addr, newAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- default:
- t.Fatal("expected addr auto gen event")
+ // Receive an RA with prefix with initial VL,
+ // test.ovl.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, test.ovl, 0))
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ if diff := checkAutoGenAddrEvent(e, addr, newAddr); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
+ default:
+ t.Fatal("expected addr auto gen event")
+ }
- // Receive an new RA with prefix with new VL,
- // test.nvl.
- e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, test.nvl, 0))
+ // Receive an new RA with prefix with new VL,
+ // test.nvl.
+ e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, test.nvl, 0))
- //
- // Validate that the VL for the address got set
- // to test.evl.
- //
+ //
+ // Validate that the VL for the address got set
+ // to test.evl.
+ //
- // The address should not be invalidated until the effective valid
- // lifetime has passed.
- select {
- case <-ndpDisp.autoGenAddrC:
- t.Fatal("unexpectedly received an auto gen addr event")
- case <-time.After(time.Duration(test.evl)*time.Second - defaultAsyncNegativeEventTimeout):
- }
+ // The address should not be invalidated until the effective valid
+ // lifetime has passed.
+ const delta = 1
+ clock.Advance(time.Duration(test.evl)*time.Second - delta)
+ select {
+ case <-ndpDisp.autoGenAddrC:
+ t.Fatal("unexpectedly received an auto gen addr event")
+ default:
+ }
- // Wait for the invalidation event.
- select {
- case e := <-ndpDisp.autoGenAddrC:
- if diff := checkAutoGenAddrEvent(e, addr, invalidatedAddr); diff != "" {
- t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
- }
- case <-time.After(defaultAsyncPositiveEventTimeout):
- t.Fatal("timeout waiting for addr auto gen event")
+ // Wait for the invalidation event.
+ clock.Advance(delta)
+ select {
+ case e := <-ndpDisp.autoGenAddrC:
+ if diff := checkAutoGenAddrEvent(e, addr, invalidatedAddr); diff != "" {
+ t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- })
- }
- })
+ default:
+ t.Fatal("timeout waiting for addr auto gen event")
+ }
+ })
+ }
}
// TestAutoGenAddrRemoval tests that when auto-generated addresses are removed
@@ -3593,7 +3527,7 @@ func TestAutoGenAddrAfterRemoval(t *testing.T) {
prefix1, _, addr1 := prefixSubnetAddr(0, linkAddr1)
prefix2, _, addr2 := prefixSubnetAddr(1, linkAddr1)
- ndpDisp, e, s := stackAndNdpDispatcherWithDefaultRoute(t, nicID)
+ ndpDisp, e, s, _ := stackAndNdpDispatcherWithDefaultRoute(t, nicID)
expectAutoGenAddrEvent := func(addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
t.Helper()
@@ -3873,13 +3807,6 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
const maxMaxRetries = 3
const lifetimeSeconds = 10
- // Needed for the temporary address sub test.
- savedMaxDesync := ipv6.MaxDesyncFactor
- defer func() {
- ipv6.MaxDesyncFactor = savedMaxDesync
- }()
- ipv6.MaxDesyncFactor = time.Nanosecond
-
secretKey := makeSecretKey(t)
prefix, subnet, _ := prefixSubnetAddr(0, linkAddr1)
@@ -3905,22 +3832,24 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
}
}
- expectAutoGenAddrEventAsync := func(t *testing.T, ndpDisp *ndpDispatcher, addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
+ expectAutoGenAddrEventAsync := func(t *testing.T, clock *faketime.ManualClock, ndpDisp *ndpDispatcher, addr tcpip.AddressWithPrefix, eventType ndpAutoGenAddrEventType) {
t.Helper()
+ clock.RunImmediatelyScheduledJobs()
select {
case e := <-ndpDisp.autoGenAddrC:
if diff := checkAutoGenAddrEvent(e, addr, eventType); diff != "" {
t.Errorf("auto-gen addr event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(defaultAsyncPositiveEventTimeout):
+ default:
t.Fatal("timed out waiting for addr auto gen event")
}
}
- expectDADEvent := func(t *testing.T, ndpDisp *ndpDispatcher, addr tcpip.Address, res stack.DADResult) {
+ expectDADEvent := func(t *testing.T, clock *faketime.ManualClock, ndpDisp *ndpDispatcher, addr tcpip.Address, res stack.DADResult) {
t.Helper()
+ clock.RunImmediatelyScheduledJobs()
select {
case e := <-ndpDisp.dadC:
if diff := checkDADEvent(e, nicID, addr, res); diff != "" {
@@ -3931,15 +3860,16 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
}
}
- expectDADEventAsync := func(t *testing.T, ndpDisp *ndpDispatcher, addr tcpip.Address, res stack.DADResult) {
+ expectDADEventAsync := func(t *testing.T, clock *faketime.ManualClock, ndpDisp *ndpDispatcher, addr tcpip.Address, res stack.DADResult) {
t.Helper()
+ clock.Advance(dadTransmits * retransmitTimer)
select {
case e := <-ndpDisp.dadC:
if diff := checkDADEvent(e, nicID, addr, res); diff != "" {
t.Errorf("DAD event mismatch (-want +got):\n%s", diff)
}
- case <-time.After(dadTransmits*retransmitTimer + defaultAsyncPositiveEventTimeout):
+ default:
t.Fatal("timed out waiting for DAD event")
}
}
@@ -3950,7 +3880,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
name string
ndpConfigs ipv6.NDPConfigurations
autoGenLinkLocal bool
- prepareFn func(t *testing.T, ndpDisp *ndpDispatcher, e *channel.Endpoint, tempIIDHistory []byte) []tcpip.AddressWithPrefix
+ prepareFn func(t *testing.T, clock *faketime.ManualClock, ndpDisp *ndpDispatcher, e *channel.Endpoint, tempIIDHistory []byte) []tcpip.AddressWithPrefix
addrGenFn func(dadCounter uint8, tempIIDHistory []byte) tcpip.AddressWithPrefix
}{
{
@@ -3959,7 +3889,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
HandleRAs: ipv6.HandlingRAsEnabledWhenForwardingDisabled,
AutoGenGlobalAddresses: true,
},
- prepareFn: func(_ *testing.T, _ *ndpDispatcher, e *channel.Endpoint, _ []byte) []tcpip.AddressWithPrefix {
+ prepareFn: func(_ *testing.T, _ *faketime.ManualClock, _ *ndpDispatcher, e *channel.Endpoint, _ []byte) []tcpip.AddressWithPrefix {
// Receive an RA with prefix1 in a PI.
e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, lifetimeSeconds, lifetimeSeconds))
return nil
@@ -3973,7 +3903,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
name: "LinkLocal address",
ndpConfigs: ipv6.NDPConfigurations{},
autoGenLinkLocal: true,
- prepareFn: func(*testing.T, *ndpDispatcher, *channel.Endpoint, []byte) []tcpip.AddressWithPrefix {
+ prepareFn: func(*testing.T, *faketime.ManualClock, *ndpDispatcher, *channel.Endpoint, []byte) []tcpip.AddressWithPrefix {
return nil
},
addrGenFn: func(dadCounter uint8, _ []byte) tcpip.AddressWithPrefix {
@@ -3987,14 +3917,14 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
AutoGenGlobalAddresses: true,
AutoGenTempGlobalAddresses: true,
},
- prepareFn: func(t *testing.T, ndpDisp *ndpDispatcher, e *channel.Endpoint, tempIIDHistory []byte) []tcpip.AddressWithPrefix {
+ prepareFn: func(t *testing.T, clock *faketime.ManualClock, ndpDisp *ndpDispatcher, e *channel.Endpoint, tempIIDHistory []byte) []tcpip.AddressWithPrefix {
header.InitialTempIID(tempIIDHistory, nil, nicID)
// Generate a stable SLAAC address so temporary addresses will be
// generated.
e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 100))
expectAutoGenAddrEvent(t, ndpDisp, stableAddrForTempAddrTest, newAddr)
- expectDADEventAsync(t, ndpDisp, stableAddrForTempAddrTest.Address, &stack.DADSucceeded{})
+ expectDADEventAsync(t, clock, ndpDisp, stableAddrForTempAddrTest.Address, &stack.DADSucceeded{})
// The stable address will be assigned throughout the test.
return []tcpip.AddressWithPrefix{stableAddrForTempAddrTest}
@@ -4006,14 +3936,6 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
}
for _, addrType := range addrTypes {
- // This Run will not return until the parallel tests finish.
- //
- // We need this because we need to do some teardown work after the parallel
- // tests complete and limit the number of parallel tests running at the same
- // time to reduce flakes.
- //
- // See https://godoc.org/testing#hdr-Subtests_and_Sub_benchmarks for
- // more details.
t.Run(addrType.name, func(t *testing.T) {
for maxRetries := uint8(0); maxRetries <= maxMaxRetries; maxRetries++ {
for numFailures := uint8(0); numFailures <= maxRetries+1; numFailures++ {
@@ -4022,8 +3944,6 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
addrType := addrType
t.Run(fmt.Sprintf("%d max retries and %d failures", maxRetries, numFailures), func(t *testing.T) {
- t.Parallel()
-
ndpDisp := ndpDispatcher{
dadC: make(chan ndpDADEvent, 1),
autoGenAddrC: make(chan ndpAutoGenAddrEvent, 2),
@@ -4031,6 +3951,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
e := channel.New(0, 1280, linkAddr1)
ndpConfigs := addrType.ndpConfigs
ndpConfigs.AutoGenAddressConflictRetries = maxRetries
+ clock := faketime.NewManualClock()
s := stack.New(stack.Options{
NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{
AutoGenLinkLocal: addrType.autoGenLinkLocal,
@@ -4047,6 +3968,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
SecretKey: secretKey,
},
})},
+ Clock: clock,
})
opts := stack.NICOptions{Name: nicName}
if err := s.CreateNICWithOptions(nicID, e, opts); err != nil {
@@ -4054,12 +3976,12 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
}
var tempIIDHistory [header.IIDSize]byte
- stableAddrs := addrType.prepareFn(t, &ndpDisp, e, tempIIDHistory[:])
+ stableAddrs := addrType.prepareFn(t, clock, &ndpDisp, e, tempIIDHistory[:])
// Simulate DAD conflicts so the address is regenerated.
for i := uint8(0); i < numFailures; i++ {
addr := addrType.addrGenFn(i, tempIIDHistory[:])
- expectAutoGenAddrEventAsync(t, &ndpDisp, addr, newAddr)
+ expectAutoGenAddrEventAsync(t, clock, &ndpDisp, addr, newAddr)
// Should not have any new addresses assigned to the NIC.
if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, stableAddrs, nil); mismatch != "" {
@@ -4069,7 +3991,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
// Simulate a DAD conflict.
rxNDPSolicit(e, addr.Address)
expectAutoGenAddrEvent(t, &ndpDisp, addr, invalidatedAddr)
- expectDADEvent(t, &ndpDisp, addr.Address, &stack.DADDupAddrDetected{})
+ expectDADEvent(t, clock, &ndpDisp, addr.Address, &stack.DADDupAddrDetected{})
// Attempting to add the address manually should not fail if the
// address's state was cleaned up when DAD failed.
@@ -4079,7 +4001,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
if err := s.RemoveAddress(nicID, addr.Address); err != nil {
t.Fatalf("RemoveAddress(%d, %s) = %s", nicID, addr.Address, err)
}
- expectDADEvent(t, &ndpDisp, addr.Address, &stack.DADAborted{})
+ expectDADEvent(t, clock, &ndpDisp, addr.Address, &stack.DADAborted{})
}
// Should not have any new addresses assigned to the NIC.
@@ -4091,8 +4013,8 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
// an address after DAD resolves.
if maxRetries+1 > numFailures {
addr := addrType.addrGenFn(numFailures, tempIIDHistory[:])
- expectAutoGenAddrEventAsync(t, &ndpDisp, addr, newAddr)
- expectDADEventAsync(t, &ndpDisp, addr.Address, &stack.DADSucceeded{})
+ expectAutoGenAddrEventAsync(t, clock, &ndpDisp, addr, newAddr)
+ expectDADEventAsync(t, clock, &ndpDisp, addr.Address, &stack.DADSucceeded{})
if mismatch := addressCheck(s.NICInfo()[nicID].ProtocolAddresses, append(stableAddrs, addr), nil); mismatch != "" {
t.Fatal(mismatch)
}
@@ -4102,7 +4024,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) {
select {
case e := <-ndpDisp.autoGenAddrC:
t.Fatalf("unexpectedly got an auto-generated address event = %+v", e)
- case <-time.After(defaultAsyncNegativeEventTimeout):
+ default:
}
})
}
@@ -4661,7 +4583,7 @@ func TestNoCleanupNDPStateWhenForwardingEnabled(t *testing.T) {
)
select {
case e := <-ndpDisp.offLinkRouteC:
- if diff := checkOffLinkRouteEvent(e, llAddr3, true /* discovered */); diff != "" {
+ if diff := checkOffLinkRouteEvent(e, nicID, llAddr3, header.MediumRoutePreference, true /* discovered */); diff != "" {
t.Errorf("off-link route event mismatch (-want +got):\n%s", diff)
}
default:
diff --git a/pkg/tcpip/stack/tcp.go b/pkg/tcpip/stack/tcp.go
index e90c1a770..90a8ba6cf 100644
--- a/pkg/tcpip/stack/tcp.go
+++ b/pkg/tcpip/stack/tcp.go
@@ -380,9 +380,6 @@ type TCPSndBufState struct {
// SndClosed indicates that the endpoint has been closed for sends.
SndClosed bool
- // SndBufInQueue is the number of bytes in the send queue.
- SndBufInQueue seqnum.Size
-
// PacketTooBigCount is used to notify the main protocol routine how
// many times a "packet too big" control packet is received.
PacketTooBigCount int
diff --git a/pkg/tcpip/stack/transport_demuxer.go b/pkg/tcpip/stack/transport_demuxer.go
index 21af0cf37..dda57e225 100644
--- a/pkg/tcpip/stack/transport_demuxer.go
+++ b/pkg/tcpip/stack/transport_demuxer.go
@@ -216,10 +216,17 @@ func (epsByNIC *endpointsByNIC) registerEndpoint(d *transportDemuxer, netProto t
netProto: netProto,
transProto: transProto,
}
- epsByNIC.endpoints[bindToDevice] = multiPortEp
}
- return multiPortEp.singleRegisterEndpoint(t, flags)
+ if err := multiPortEp.singleRegisterEndpoint(t, flags); err != nil {
+ return err
+ }
+ // Only add this newly created multiportEndpoint if the singleRegisterEndpoint
+ // succeeded.
+ if !ok {
+ epsByNIC.endpoints[bindToDevice] = multiPortEp
+ }
+ return nil
}
func (epsByNIC *endpointsByNIC) checkEndpoint(flags ports.Flags, bindToDevice tcpip.NICID) tcpip.Error {
@@ -406,7 +413,6 @@ func (ep *multiPortEndpoint) handlePacketAll(id TransportEndpointID, pkt *Packet
func (ep *multiPortEndpoint) singleRegisterEndpoint(t TransportEndpoint, flags ports.Flags) tcpip.Error {
ep.mu.Lock()
defer ep.mu.Unlock()
-
bits := flags.Bits() & ports.MultiBindFlagMask
if len(ep.endpoints) != 0 {
@@ -469,17 +475,21 @@ func (d *transportDemuxer) singleRegisterEndpoint(netProto tcpip.NetworkProtocol
eps.mu.Lock()
defer eps.mu.Unlock()
-
epsByNIC, ok := eps.endpoints[id]
if !ok {
epsByNIC = &endpointsByNIC{
endpoints: make(map[tcpip.NICID]*multiPortEndpoint),
seed: d.stack.Seed(),
}
+ }
+ if err := epsByNIC.registerEndpoint(d, netProto, protocol, ep, flags, bindToDevice); err != nil {
+ return err
+ }
+ // Only add this newly created epsByNIC if registerEndpoint succeeded.
+ if !ok {
eps.endpoints[id] = epsByNIC
}
-
- return epsByNIC.registerEndpoint(d, netProto, protocol, ep, flags, bindToDevice)
+ return nil
}
func (d *transportDemuxer) singleCheckEndpoint(netProto tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, flags ports.Flags, bindToDevice tcpip.NICID) tcpip.Error {
diff --git a/pkg/tcpip/stack/transport_demuxer_test.go b/pkg/tcpip/stack/transport_demuxer_test.go
index 0972c94de..45b09110d 100644
--- a/pkg/tcpip/stack/transport_demuxer_test.go
+++ b/pkg/tcpip/stack/transport_demuxer_test.go
@@ -203,6 +203,56 @@ func TestTransportDemuxerRegister(t *testing.T) {
}
}
+func TestTransportDemuxerRegisterMultiple(t *testing.T) {
+ type test struct {
+ flags ports.Flags
+ want tcpip.Error
+ }
+ for _, subtest := range []struct {
+ name string
+ tests []test
+ }{
+ {"zeroFlags", []test{
+ {ports.Flags{}, nil},
+ {ports.Flags{}, &tcpip.ErrPortInUse{}},
+ }},
+ {"multibindFlags", []test{
+ // Allow multiple registrations same TransportEndpointID with multibind flags.
+ {ports.Flags{LoadBalanced: true, MostRecent: true}, nil},
+ {ports.Flags{LoadBalanced: true, MostRecent: true}, nil},
+ // Disallow registration w/same ID for a non-multibindflag.
+ {ports.Flags{TupleOnly: true}, &tcpip.ErrPortInUse{}},
+ }},
+ } {
+ t.Run(subtest.name, func(t *testing.T) {
+ s := stack.New(stack.Options{
+ NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol},
+ TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol},
+ })
+ var eps []tcpip.Endpoint
+ for idx, test := range subtest.tests {
+ var wq waiter.Queue
+ ep, err := s.NewEndpoint(udp.ProtocolNumber, ipv4.ProtocolNumber, &wq)
+ if err != nil {
+ t.Fatal(err)
+ }
+ eps = append(eps, ep)
+ tEP, ok := ep.(stack.TransportEndpoint)
+ if !ok {
+ t.Fatalf("%T does not implement stack.TransportEndpoint", ep)
+ }
+ id := stack.TransportEndpointID{LocalPort: 1}
+ if got, want := s.RegisterTransportEndpoint([]tcpip.NetworkProtocolNumber{ipv4.ProtocolNumber}, udp.ProtocolNumber, id, tEP, test.flags, 0), test.want; got != want {
+ t.Fatalf("test index: %d, s.RegisterTransportEndpoint(ipv4.ProtocolNumber, udp.ProtocolNumber, _, _, %+v, 0) = %s, want %s", idx, test.flags, got, want)
+ }
+ }
+ for _, ep := range eps {
+ ep.Close()
+ }
+ })
+ }
+}
+
// TestBindToDeviceDistribution injects varied packets on input devices and checks that
// the distribution of packets received matches expectations.
func TestBindToDeviceDistribution(t *testing.T) {
diff --git a/pkg/tcpip/tests/integration/loopback_test.go b/pkg/tcpip/tests/integration/loopback_test.go
index b4b2ec723..b2008f0b2 100644
--- a/pkg/tcpip/tests/integration/loopback_test.go
+++ b/pkg/tcpip/tests/integration/loopback_test.go
@@ -44,7 +44,7 @@ type ndpDispatcher struct{}
func (*ndpDispatcher) OnDuplicateAddressDetectionResult(tcpip.NICID, tcpip.Address, stack.DADResult) {
}
-func (*ndpDispatcher) OnOffLinkRouteUpdated(tcpip.NICID, tcpip.Subnet, tcpip.Address) {
+func (*ndpDispatcher) OnOffLinkRouteUpdated(tcpip.NICID, tcpip.Subnet, tcpip.Address, header.NDPRoutePreference) {
}
func (*ndpDispatcher) OnOffLinkRouteInvalidated(tcpip.NICID, tcpip.Subnet, tcpip.Address) {}
diff --git a/pkg/tcpip/transport/packet/endpoint.go b/pkg/tcpip/transport/packet/endpoint.go
index ed21c92ad..8e7bb6c6e 100644
--- a/pkg/tcpip/transport/packet/endpoint.go
+++ b/pkg/tcpip/transport/packet/endpoint.go
@@ -208,7 +208,6 @@ func (ep *endpoint) Read(dst io.Writer, opts tcpip.ReadOptions) (tcpip.ReadResul
}
func (*endpoint) Write(tcpip.Payloader, tcpip.WriteOptions) (int64, tcpip.Error) {
- // TODO(gvisor.dev/issue/173): Implement.
return 0, &tcpip.ErrInvalidOptionValue{}
}
@@ -244,8 +243,6 @@ func (*endpoint) Accept(*tcpip.FullAddress) (tcpip.Endpoint, *waiter.Queue, tcpi
// Bind implements tcpip.Endpoint.Bind.
func (ep *endpoint) Bind(addr tcpip.FullAddress) tcpip.Error {
- // TODO(gvisor.dev/issue/173): Add Bind support.
-
// "By default, all packets of the specified protocol type are passed
// to a packet socket. To get packets only from a specific interface
// use bind(2) specifying an address in a struct sockaddr_ll to bind
@@ -385,7 +382,6 @@ func (ep *endpoint) HandlePacket(nicID tcpip.NICID, localAddr tcpip.LinkAddress,
// Push new packet into receive list and increment the buffer size.
var packet packet
- // TODO(gvisor.dev/issue/173): Return network protocol.
if !pkt.LinkHeader().View().IsEmpty() {
// Get info directly from the ethernet header.
hdr := header.Ethernet(pkt.LinkHeader().View())
diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go
index 2137ebc25..e39d1623d 100644
--- a/pkg/tcpip/transport/tcp/connect.go
+++ b/pkg/tcpip/transport/tcp/connect.go
@@ -909,30 +909,13 @@ func (e *endpoint) sendRaw(data buffer.VectorisedView, flags header.TCPFlags, se
return err
}
-func (e *endpoint) handleWrite() {
- e.sndQueueInfo.sndQueueMu.Lock()
- next := e.drainSendQueueLocked()
- e.sndQueueInfo.sndQueueMu.Unlock()
-
- e.sendData(next)
-}
-
-// Move packets from send queue to send list.
-//
-// Precondition: e.sndBufMu must be locked.
-func (e *endpoint) drainSendQueueLocked() *segment {
- first := e.sndQueueInfo.sndQueue.Front()
- if first != nil {
- e.snd.writeList.PushBackList(&e.sndQueueInfo.sndQueue)
- e.sndQueueInfo.SndBufInQueue = 0
- }
- return first
-}
-
// Precondition: e.mu must be locked.
func (e *endpoint) sendData(next *segment) {
// Initialize the next segment to write if it's currently nil.
if e.snd.writeNext == nil {
+ if next == nil {
+ return
+ }
e.snd.writeNext = next
}
@@ -940,17 +923,6 @@ func (e *endpoint) sendData(next *segment) {
e.snd.sendData()
}
-func (e *endpoint) handleClose() {
- if !e.EndpointState().connected() {
- return
- }
- // Drain the send queue.
- e.handleWrite()
-
- // Mark send side as closed.
- e.snd.Closed = true
-}
-
// resetConnectionLocked puts the endpoint in an error state with the given
// error code and sends a RST if and only if the error is not ErrConnectionReset
// indicating that the connection is being reset due to receiving a RST. This
@@ -1130,7 +1102,7 @@ func (e *endpoint) handleReset(s *segment) (ok bool, err tcpip.Error) {
func (e *endpoint) handleSegmentsLocked(fastPath bool) tcpip.Error {
checkRequeue := true
for i := 0; i < maxSegmentsPerWake; i++ {
- if e.EndpointState().closed() {
+ if state := e.EndpointState(); state.closed() || state == StateTimeWait {
return nil
}
s := e.segmentQueue.dequeue()
@@ -1402,14 +1374,7 @@ func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{
{
w: &e.sndQueueInfo.sndWaker,
f: func() tcpip.Error {
- e.handleWrite()
- return nil
- },
- },
- {
- w: &e.sndQueueInfo.sndCloseWaker,
- f: func() tcpip.Error {
- e.handleClose()
+ e.sendData(nil /* next */)
return nil
},
},
diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go
index a27e2110b..4acddc959 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -293,16 +293,9 @@ type sndQueueInfo struct {
sndQueueMu sync.Mutex `state:"nosave"`
stack.TCPSndBufState
- // sndQueue holds segments that are ready to be sent.
- sndQueue segmentList `state:"wait"`
-
- // sndWaker is used to signal the protocol goroutine when segments are
- // added to the `sndQueue`.
+ // sndWaker is used to signal the protocol goroutine when there may be
+ // segments that need to be sent.
sndWaker sleep.Waker `state:"manual"`
-
- // sndCloseWaker is used to notify the protocol goroutine when the send
- // side is closed.
- sndCloseWaker sleep.Waker `state:"manual"`
}
// rcvQueueInfo contains the endpoint's rcvQueue and associated metadata.
@@ -1558,10 +1551,9 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, tcp
// Add data to the send queue.
s := newOutgoingSegment(e.TransportEndpointInfo.ID, e.stack.Clock(), v)
e.sndQueueInfo.SndBufUsed += len(v)
- e.sndQueueInfo.SndBufInQueue += seqnum.Size(len(v))
- e.sndQueueInfo.sndQueue.PushBack(s)
+ e.snd.writeList.PushBack(s)
- return e.drainSendQueueLocked(), len(v), nil
+ return s, len(v), nil
}()
// Return if either we didn't queue anything or if an error occurred while
// attempting to queue data.
@@ -2314,7 +2306,7 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) tcp
// connection setting here.
if !handshake {
e.segmentQueue.mu.Lock()
- for _, l := range []segmentList{e.segmentQueue.list, e.sndQueueInfo.sndQueue, e.snd.writeList} {
+ for _, l := range []segmentList{e.segmentQueue.list, e.snd.writeList} {
for s := l.Front(); s != nil; s = s.Next() {
s.id = e.TransportEndpointInfo.ID
e.sndQueueInfo.sndWaker.Assert()
@@ -2372,6 +2364,9 @@ func (e *endpoint) shutdownLocked(flags tcpip.ShutdownFlags) tcpip.Error {
e.notifyProtocolGoroutine(notifyTickleWorker)
return nil
}
+ // Wake up any readers that maybe waiting for the stream to become
+ // readable.
+ e.waiterQueue.Notify(waiter.ReadableEvents)
}
// Close for write.
@@ -2388,12 +2383,20 @@ func (e *endpoint) shutdownLocked(flags tcpip.ShutdownFlags) tcpip.Error {
// Queue fin segment.
s := newOutgoingSegment(e.TransportEndpointInfo.ID, e.stack.Clock(), nil)
- e.sndQueueInfo.sndQueue.PushBack(s)
- e.sndQueueInfo.SndBufInQueue++
+ e.snd.writeList.PushBack(s)
// Mark endpoint as closed.
e.sndQueueInfo.SndClosed = true
e.sndQueueInfo.sndQueueMu.Unlock()
- e.handleClose()
+
+ // Drain the send queue.
+ e.sendData(s)
+
+ // Mark send side as closed.
+ e.snd.Closed = true
+
+ // Wake up any writers that maybe waiting for the stream to become
+ // writable.
+ e.waiterQueue.Notify(waiter.WritableEvents)
}
return nil
diff --git a/pkg/tcpip/transport/tcp/rcv.go b/pkg/tcpip/transport/tcp/rcv.go
index 661ca604a..9ce8fcae9 100644
--- a/pkg/tcpip/transport/tcp/rcv.go
+++ b/pkg/tcpip/transport/tcp/rcv.go
@@ -559,7 +559,6 @@ func (r *receiver) handleTimeWaitSegment(s *segment) (resetTimeWait bool, newSyn
// (2) returns to TIME-WAIT state if the SYN turns out
// to be an old duplicate".
if s.flags.Contains(header.TCPFlagSyn) && r.RcvNxt.LessThan(segSeq) {
-
return false, true
}
diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go
index 9bbe9bc3e..d1314fcdf 100644
--- a/pkg/tcpip/transport/tcp/tcp_test.go
+++ b/pkg/tcpip/transport/tcp/tcp_test.go
@@ -3451,17 +3451,13 @@ loop:
for {
switch _, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{}); err.(type) {
case *tcpip.ErrWouldBlock:
- select {
- case <-ch:
- // Expect the state to be StateError and subsequent Reads to fail with HardError.
- _, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{})
- if d := cmp.Diff(&tcpip.ErrConnectionReset{}, err); d != "" {
- t.Fatalf("c.EP.Read() mismatch (-want +got):\n%s", d)
- }
- break loop
- case <-time.After(1 * time.Second):
- t.Fatalf("Timed out waiting for reset to arrive")
+ <-ch
+ // Expect the state to be StateError and subsequent Reads to fail with HardError.
+ _, err := c.EP.Read(ioutil.Discard, tcpip.ReadOptions{})
+ if d := cmp.Diff(&tcpip.ErrConnectionReset{}, err); d != "" {
+ t.Fatalf("c.EP.Read() mismatch (-want +got):\n%s", d)
}
+ break loop
case *tcpip.ErrConnectionReset:
break loop
default:
@@ -3472,14 +3468,27 @@ loop:
if tcp.EndpointState(c.EP.State()) != tcp.StateError {
t.Fatalf("got EP state is not StateError")
}
- if got := c.Stack().Stats().TCP.EstablishedResets.Value(); got != 1 {
- t.Errorf("got stats.TCP.EstablishedResets.Value() = %d, want = 1", got)
+
+ checkValid := func() []error {
+ var errors []error
+ if got := c.Stack().Stats().TCP.EstablishedResets.Value(); got != 1 {
+ errors = append(errors, fmt.Errorf("got stats.TCP.EstablishedResets.Value() = %d, want = 1", got))
+ }
+ if got := c.Stack().Stats().TCP.CurrentEstablished.Value(); got != 0 {
+ errors = append(errors, fmt.Errorf("got stats.TCP.CurrentEstablished.Value() = %d, want = 0", got))
+ }
+ if got := c.Stack().Stats().TCP.CurrentConnected.Value(); got != 0 {
+ errors = append(errors, fmt.Errorf("got stats.TCP.CurrentConnected.Value() = %d, want = 0", got))
+ }
+ return errors
}
- if got := c.Stack().Stats().TCP.CurrentEstablished.Value(); got != 0 {
- t.Errorf("got stats.TCP.CurrentEstablished.Value() = %d, want = 0", got)
+
+ start := time.Now()
+ for time.Since(start) < time.Minute && len(checkValid()) > 0 {
+ time.Sleep(50 * time.Millisecond)
}
- if got := c.Stack().Stats().TCP.CurrentConnected.Value(); got != 0 {
- t.Errorf("got stats.TCP.CurrentConnected.Value() = %d, want = 0", got)
+ for _, err := range checkValid() {
+ t.Error(err)
}
}
@@ -6092,15 +6101,10 @@ func TestSynRcvdBadSeqNumber(t *testing.T) {
defer c.WQ.EventUnregister(&we)
// Wait for connection to be established.
- select {
- case <-ch:
- newEP, _, err = c.EP.Accept(nil)
- if err != nil {
- t.Fatalf("Accept failed: %s", err)
- }
-
- case <-time.After(1 * time.Second):
- t.Fatalf("Timed out waiting for accept")
+ <-ch
+ newEP, _, err = c.EP.Accept(nil)
+ if err != nil {
+ t.Fatalf("Accept failed: %s", err)
}
}
@@ -6209,12 +6213,26 @@ func TestPassiveFailedConnectionAttemptIncrement(t *testing.T) {
RcvWnd: 30000,
})
- time.Sleep(50 * time.Millisecond)
- if got := stats.TCP.ListenOverflowSynDrop.Value(); got != want {
- t.Errorf("got stats.TCP.ListenOverflowSynDrop.Value() = %d, want = %d", got, want)
+ checkValid := func() []error {
+ var errors []error
+ if got := stats.TCP.ListenOverflowSynDrop.Value(); got != want {
+ errors = append(errors, fmt.Errorf("got stats.TCP.ListenOverflowSynDrop.Value() = %d, want = %d", got, want))
+ }
+ if got := c.EP.Stats().(*tcp.Stats).ReceiveErrors.ListenOverflowSynDrop.Value(); got != want {
+ errors = append(errors, fmt.Errorf("got EP stats Stats.ReceiveErrors.ListenOverflowSynDrop = %d, want = %d", got, want))
+ }
+ return errors
}
- if got := c.EP.Stats().(*tcp.Stats).ReceiveErrors.ListenOverflowSynDrop.Value(); got != want {
- t.Errorf("got EP stats Stats.ReceiveErrors.ListenOverflowSynDrop = %d, want = %d", got, want)
+
+ start := time.Now()
+ for time.Since(start) < time.Minute && len(checkValid()) > 0 {
+ time.Sleep(50 * time.Millisecond)
+ }
+ for _, err := range checkValid() {
+ t.Error(err)
+ }
+ if t.Failed() {
+ t.FailNow()
}
we, ch := waiter.NewChannelEntry(nil)
@@ -6225,15 +6243,10 @@ func TestPassiveFailedConnectionAttemptIncrement(t *testing.T) {
_, _, err = c.EP.Accept(nil)
if cmp.Equal(&tcpip.ErrWouldBlock{}, err) {
// Wait for connection to be established.
- select {
- case <-ch:
- _, _, err = c.EP.Accept(nil)
- if err != nil {
- t.Fatalf("Accept failed: %s", err)
- }
-
- case <-time.After(1 * time.Second):
- t.Fatalf("Timed out waiting for accept")
+ <-ch
+ _, _, err = c.EP.Accept(nil)
+ if err != nil {
+ t.Fatalf("Accept failed: %s", err)
}
}
}
@@ -7483,7 +7496,7 @@ func TestTCPUserTimeout(t *testing.T) {
select {
case <-notifyCh:
case <-time.After(2 * initRTO):
- t.Fatalf("connection still alive after %s, should have been closed after :%s", 2*initRTO, userTimeout)
+ t.Fatalf("connection still alive after %s, should have been closed after %s", 2*initRTO, userTimeout)
}
// No packet should be received as the connection should be silently
diff --git a/pkg/urpc/urpc.go b/pkg/urpc/urpc.go
index 7872d6fa1..0ef635a2f 100644
--- a/pkg/urpc/urpc.go
+++ b/pkg/urpc/urpc.go
@@ -20,7 +20,6 @@ package urpc
import (
"bytes"
- "context"
"encoding/json"
"errors"
"fmt"
@@ -28,6 +27,7 @@ import (
"os"
"reflect"
"runtime"
+ "time"
"gvisor.dev/gvisor/pkg/fd"
"gvisor.dev/gvisor/pkg/log"
@@ -459,18 +459,24 @@ func (s *Server) StartHandling(client *unet.Socket) {
// No new requests should be initiated after calling Stop. Existing clients
// will be closed after completing any pending RPCs. This method will block
// until all clients have disconnected.
-func (s *Server) Stop(ctx context.Context) {
- done := make(chan bool)
-
+//
+// timeout is the time for clients to complete ongoing RPCs.
+func (s *Server) Stop(timeout time.Duration) {
// Call any Stop callbacks.
for _, stopper := range s.stoppers {
stopper.Stop()
}
+
+ done := make(chan bool, 1)
go func() {
- select {
- case <-done:
- return
- case <-ctx.Done():
+ if timeout != 0 {
+ timer := time.NewTicker(timeout)
+ defer timer.Stop()
+ select {
+ case <-done:
+ return
+ case <-timer.C:
+ }
}
// Close all known clients.
@@ -488,10 +494,10 @@ func (s *Server) Stop(ctx context.Context) {
}
}
}()
+
// Wait for all outstanding requests.
s.wg.Wait()
done <- true
-
}
// Client is a urpc client.
diff --git a/pkg/usermem/BUILD b/pkg/usermem/BUILD
index d7decd78a..229a8341b 100644
--- a/pkg/usermem/BUILD
+++ b/pkg/usermem/BUILD
@@ -30,6 +30,7 @@ go_test(
library = ":usermem",
deps = [
"//pkg/context",
+ "//pkg/errors/linuxerr",
"//pkg/hostarch",
"//pkg/safemem",
"//pkg/syserror",
diff --git a/pkg/usermem/usermem_test.go b/pkg/usermem/usermem_test.go
index 9b697b593..6ef2b571f 100644
--- a/pkg/usermem/usermem_test.go
+++ b/pkg/usermem/usermem_test.go
@@ -22,6 +22,7 @@ import (
"testing"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/syserror"
@@ -272,7 +273,7 @@ func TestCopyInt32StringsInVecRequiresOneValidValue(t *testing.T) {
src := BytesIOSequence([]byte(s))
initial := []int32{1, 2}
dsts := append([]int32(nil), initial...)
- if n, err := CopyInt32StringsInVec(newContext(), src.IO, src.Addrs, dsts, src.Opts); err != syserror.EINVAL {
+ if n, err := CopyInt32StringsInVec(newContext(), src.IO, src.Addrs, dsts, src.Opts); !linuxerr.Equals(linuxerr.EINVAL, err) {
t.Errorf("CopyInt32StringsInVec: got (%d, %v), wanted (_, %v)", n, err, syserror.EINVAL)
}
if !reflect.DeepEqual(dsts, initial) {