From d02b74a5dcfed4bfc8f2f8e545bca4d2afabb296 Mon Sep 17 00:00:00 2001 From: Googler Date: Fri, 27 Apr 2018 10:37:02 -0700 Subject: Check in gVisor. PiperOrigin-RevId: 194583126 Change-Id: Ica1d8821a90f74e7e745962d71801c598c652463 --- pkg/sentry/socket/rpcinet/BUILD | 59 +++ pkg/sentry/socket/rpcinet/conn/BUILD | 17 + pkg/sentry/socket/rpcinet/conn/conn.go | 167 ++++++++ pkg/sentry/socket/rpcinet/device.go | 19 + pkg/sentry/socket/rpcinet/notifier/BUILD | 15 + pkg/sentry/socket/rpcinet/notifier/notifier.go | 230 ++++++++++ pkg/sentry/socket/rpcinet/rpcinet.go | 16 + pkg/sentry/socket/rpcinet/socket.go | 567 +++++++++++++++++++++++++ pkg/sentry/socket/rpcinet/stack.go | 175 ++++++++ pkg/sentry/socket/rpcinet/stack_unsafe.go | 193 +++++++++ pkg/sentry/socket/rpcinet/syscall_rpc.proto | 351 +++++++++++++++ 11 files changed, 1809 insertions(+) create mode 100644 pkg/sentry/socket/rpcinet/BUILD create mode 100644 pkg/sentry/socket/rpcinet/conn/BUILD create mode 100644 pkg/sentry/socket/rpcinet/conn/conn.go create mode 100644 pkg/sentry/socket/rpcinet/device.go create mode 100644 pkg/sentry/socket/rpcinet/notifier/BUILD create mode 100644 pkg/sentry/socket/rpcinet/notifier/notifier.go create mode 100644 pkg/sentry/socket/rpcinet/rpcinet.go create mode 100644 pkg/sentry/socket/rpcinet/socket.go create mode 100644 pkg/sentry/socket/rpcinet/stack.go create mode 100644 pkg/sentry/socket/rpcinet/stack_unsafe.go create mode 100644 pkg/sentry/socket/rpcinet/syscall_rpc.proto (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD new file mode 100644 index 000000000..b0351b363 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -0,0 +1,59 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "rpcinet", + srcs = [ + "device.go", + "rpcinet.go", + "socket.go", + "stack.go", + "stack_unsafe.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet", + visibility = ["//pkg/sentry:internal"], + deps = [ + ":syscall_rpc_go_proto", + "//pkg/abi/linux", + "//pkg/binary", + "//pkg/sentry/arch", + "//pkg/sentry/context", + "//pkg/sentry/device", + "//pkg/sentry/fs", + "//pkg/sentry/fs/fsutil", + "//pkg/sentry/inet", + "//pkg/sentry/kernel", + "//pkg/sentry/kernel/kdefs", + "//pkg/sentry/kernel/time", + "//pkg/sentry/socket", + "//pkg/sentry/socket/hostinet", + "//pkg/sentry/socket/rpcinet/conn", + "//pkg/sentry/socket/rpcinet/notifier", + "//pkg/sentry/usermem", + "//pkg/syserr", + "//pkg/syserror", + "//pkg/tcpip/buffer", + "//pkg/tcpip/transport/unix", + "//pkg/unet", + "//pkg/waiter", + ], +) + +proto_library( + name = "syscall_rpc_proto", + srcs = ["syscall_rpc.proto"], + visibility = [ + "//visibility:public", + ], +) + +go_proto_library( + name = "syscall_rpc_go_proto", + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto", + proto = ":syscall_rpc_proto", + visibility = [ + "//visibility:public", + ], +) diff --git a/pkg/sentry/socket/rpcinet/conn/BUILD b/pkg/sentry/socket/rpcinet/conn/BUILD new file mode 100644 index 000000000..4923dee4b --- /dev/null +++ b/pkg/sentry/socket/rpcinet/conn/BUILD @@ -0,0 +1,17 @@ +package(licenses = ["notice"]) # BSD + +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "conn", + srcs = ["conn.go"], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn", + visibility = ["//pkg/sentry:internal"], + deps = [ + "//pkg/binary", + "//pkg/sentry/socket/rpcinet:syscall_rpc_go_proto", + "//pkg/syserr", + "//pkg/unet", + "@com_github_golang_protobuf//proto:go_default_library", + ], +) diff --git a/pkg/sentry/socket/rpcinet/conn/conn.go b/pkg/sentry/socket/rpcinet/conn/conn.go new file mode 100644 index 000000000..ea6ec87ed --- /dev/null +++ b/pkg/sentry/socket/rpcinet/conn/conn.go @@ -0,0 +1,167 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package conn is an RPC connection to a syscall RPC server. +package conn + +import ( + "fmt" + "sync" + "sync/atomic" + "syscall" + + "github.com/golang/protobuf/proto" + "gvisor.googlesource.com/gvisor/pkg/binary" + "gvisor.googlesource.com/gvisor/pkg/syserr" + "gvisor.googlesource.com/gvisor/pkg/unet" + + pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" +) + +type request struct { + response []byte + ready chan struct{} + ignoreResult bool +} + +// RPCConnection represents a single RPC connection to a syscall gofer. +type RPCConnection struct { + // reqID is the ID of the last request and must be accessed atomically. + reqID uint64 + + sendMu sync.Mutex + socket *unet.Socket + + reqMu sync.Mutex + requests map[uint64]request +} + +// NewRPCConnection initializes a RPC connection to a socket gofer. +func NewRPCConnection(s *unet.Socket) *RPCConnection { + conn := &RPCConnection{socket: s, requests: map[uint64]request{}} + go func() { // S/R-FIXME + var nums [16]byte + for { + for n := 0; n < len(nums); { + nn, err := conn.socket.Read(nums[n:]) + if err != nil { + panic(fmt.Sprint("error reading length from socket rpc gofer: ", err)) + } + n += nn + } + + b := make([]byte, binary.LittleEndian.Uint64(nums[:8])) + id := binary.LittleEndian.Uint64(nums[8:]) + + for n := 0; n < len(b); { + nn, err := conn.socket.Read(b[n:]) + if err != nil { + panic(fmt.Sprint("error reading request from socket rpc gofer: ", err)) + } + n += nn + } + + conn.reqMu.Lock() + r := conn.requests[id] + if r.ignoreResult { + delete(conn.requests, id) + } else { + r.response = b + conn.requests[id] = r + } + conn.reqMu.Unlock() + close(r.ready) + } + }() + return conn +} + +// NewRequest makes a request to the RPC gofer and returns the request ID and a +// channel which will be closed once the request completes. +func (c *RPCConnection) NewRequest(req pb.SyscallRequest, ignoreResult bool) (uint64, chan struct{}) { + b, err := proto.Marshal(&req) + if err != nil { + panic(fmt.Sprint("invalid proto: ", err)) + } + + id := atomic.AddUint64(&c.reqID, 1) + ch := make(chan struct{}) + + c.reqMu.Lock() + c.requests[id] = request{ready: ch, ignoreResult: ignoreResult} + c.reqMu.Unlock() + + c.sendMu.Lock() + defer c.sendMu.Unlock() + + var nums [16]byte + binary.LittleEndian.PutUint64(nums[:8], uint64(len(b))) + binary.LittleEndian.PutUint64(nums[8:], id) + for n := 0; n < len(nums); { + nn, err := c.socket.Write(nums[n:]) + if err != nil { + panic(fmt.Sprint("error writing length and ID to socket gofer: ", err)) + } + n += nn + } + + for n := 0; n < len(b); { + nn, err := c.socket.Write(b[n:]) + if err != nil { + panic(fmt.Sprint("error writing request to socket gofer: ", err)) + } + n += nn + } + + return id, ch +} + +// RPCReadFile will execute the ReadFile helper RPC method which avoids the +// common pattern of open(2), read(2), close(2) by doing all three operations +// as a single RPC. It will read the entire file or return EFBIG if the file +// was too large. +func (c *RPCConnection) RPCReadFile(path string) ([]byte, *syserr.Error) { + req := &pb.SyscallRequest_ReadFile{&pb.ReadFileRequest{ + Path: path, + }} + + id, ch := c.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-ch + + res := c.Request(id).Result.(*pb.SyscallResponse_ReadFile).ReadFile.Result + if e, ok := res.(*pb.ReadFileResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.ReadFileResponse_Data).Data, nil +} + +// Request retrieves the request corresponding to the given request ID. +// +// The channel returned by NewRequest must have been closed before Request can +// be called. This will happen automatically, do not manually close the +// channel. +func (c *RPCConnection) Request(id uint64) pb.SyscallResponse { + c.reqMu.Lock() + r := c.requests[id] + delete(c.requests, id) + c.reqMu.Unlock() + + var resp pb.SyscallResponse + if err := proto.Unmarshal(r.response, &resp); err != nil { + panic(fmt.Sprint("invalid proto: ", err)) + } + + return resp +} diff --git a/pkg/sentry/socket/rpcinet/device.go b/pkg/sentry/socket/rpcinet/device.go new file mode 100644 index 000000000..f7b63436e --- /dev/null +++ b/pkg/sentry/socket/rpcinet/device.go @@ -0,0 +1,19 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rpcinet + +import "gvisor.googlesource.com/gvisor/pkg/sentry/device" + +var socketDevice = device.NewAnonDevice() diff --git a/pkg/sentry/socket/rpcinet/notifier/BUILD b/pkg/sentry/socket/rpcinet/notifier/BUILD new file mode 100644 index 000000000..6f3b06a05 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/notifier/BUILD @@ -0,0 +1,15 @@ +package(licenses = ["notice"]) # BSD + +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "notifier", + srcs = ["notifier.go"], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/notifier", + visibility = ["//pkg/sentry:internal"], + deps = [ + "//pkg/sentry/socket/rpcinet:syscall_rpc_go_proto", + "//pkg/sentry/socket/rpcinet/conn", + "//pkg/waiter", + ], +) diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier.go b/pkg/sentry/socket/rpcinet/notifier/notifier.go new file mode 100644 index 000000000..f88a908ed --- /dev/null +++ b/pkg/sentry/socket/rpcinet/notifier/notifier.go @@ -0,0 +1,230 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package notifier implements an FD notifier implementation over RPC. +package notifier + +import ( + "fmt" + "sync" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn" + pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +type fdInfo struct { + queue *waiter.Queue + waiting bool +} + +// Notifier holds all the state necessary to issue notifications when IO events +// occur in the observed FDs. +type Notifier struct { + // rpcConn is the connection that is used for sending RPCs. + rpcConn *conn.RPCConnection + + // epFD is the epoll file descriptor used to register for io + // notifications. + epFD uint32 + + // mu protects fdMap. + mu sync.Mutex + + // fdMap maps file descriptors to their notification queues and waiting + // status. + fdMap map[uint32]*fdInfo +} + +// NewRPCNotifier creates a new notifier object. +func NewRPCNotifier(cn *conn.RPCConnection) (*Notifier, error) { + id, c := cn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollCreate1{&pb.EpollCreate1Request{}}}, false /* ignoreResult */) + <-c + + res := cn.Request(id).Result.(*pb.SyscallResponse_EpollCreate1).EpollCreate1.Result + if e, ok := res.(*pb.EpollCreate1Response_ErrorNumber); ok { + return nil, syscall.Errno(e.ErrorNumber) + } + + w := &Notifier{ + rpcConn: cn, + epFD: res.(*pb.EpollCreate1Response_Fd).Fd, + fdMap: make(map[uint32]*fdInfo), + } + + go w.waitAndNotify() // S/R-FIXME + + return w, nil +} + +// waitFD waits on mask for fd. The fdMap mutex must be hold. +func (n *Notifier) waitFD(fd uint32, fi *fdInfo, mask waiter.EventMask) error { + if !fi.waiting && mask == 0 { + return nil + } + + e := pb.EpollEvent{ + Events: uint32(mask) | -syscall.EPOLLET, + Fd: fd, + } + + switch { + case !fi.waiting && mask != 0: + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollCtl{&pb.EpollCtlRequest{Epfd: n.epFD, Op: syscall.EPOLL_CTL_ADD, Fd: fd, Event: &e}}}, false /* ignoreResult */) + <-c + + e := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_EpollCtl).EpollCtl.ErrorNumber + if e != 0 { + return syscall.Errno(e) + } + + fi.waiting = true + case fi.waiting && mask == 0: + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollCtl{&pb.EpollCtlRequest{Epfd: n.epFD, Op: syscall.EPOLL_CTL_DEL, Fd: fd}}}, false /* ignoreResult */) + <-c + n.rpcConn.Request(id) + + fi.waiting = false + case fi.waiting && mask != 0: + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollCtl{&pb.EpollCtlRequest{Epfd: n.epFD, Op: syscall.EPOLL_CTL_MOD, Fd: fd, Event: &e}}}, false /* ignoreResult */) + <-c + + e := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_EpollCtl).EpollCtl.ErrorNumber + if e != 0 { + return syscall.Errno(e) + } + } + + return nil +} + +// addFD adds an FD to the list of FDs observed by n. +func (n *Notifier) addFD(fd uint32, queue *waiter.Queue) { + n.mu.Lock() + defer n.mu.Unlock() + + // Panic if we're already notifying on this FD. + if _, ok := n.fdMap[fd]; ok { + panic(fmt.Sprintf("File descriptor %d added twice", fd)) + } + + // We have nothing to wait for at the moment. Just add it to the map. + n.fdMap[fd] = &fdInfo{queue: queue} +} + +// updateFD updates the set of events the FD needs to be notified on. +func (n *Notifier) updateFD(fd uint32) error { + n.mu.Lock() + defer n.mu.Unlock() + + if fi, ok := n.fdMap[fd]; ok { + return n.waitFD(fd, fi, fi.queue.Events()) + } + + return nil +} + +// RemoveFD removes an FD from the list of FDs observed by n. +func (n *Notifier) removeFD(fd uint32) { + n.mu.Lock() + defer n.mu.Unlock() + + // Remove from map, then from epoll object. + n.waitFD(fd, n.fdMap[fd], 0) + delete(n.fdMap, fd) +} + +// hasFD returns true if the FD is in the list of observed FDs. +func (n *Notifier) hasFD(fd uint32) bool { + n.mu.Lock() + defer n.mu.Unlock() + + _, ok := n.fdMap[fd] + return ok +} + +// waitAndNotify loops waiting for io event notifications from the epoll +// object. Once notifications arrive, they are dispatched to the +// registered queue. +func (n *Notifier) waitAndNotify() error { + for { + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_EpollWait{&pb.EpollWaitRequest{Fd: n.epFD, NumEvents: 100, Msec: -1}}}, false /* ignoreResult */) + <-c + + res := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_EpollWait).EpollWait.Result + if e, ok := res.(*pb.EpollWaitResponse_ErrorNumber); ok { + err := syscall.Errno(e.ErrorNumber) + // NOTE: I don't think epoll_wait can return EAGAIN but I'm being + // conseratively careful here since exiting the notification thread + // would be really bad. + if err == syscall.EINTR || err == syscall.EAGAIN { + continue + } + return err + } + + n.mu.Lock() + for _, e := range res.(*pb.EpollWaitResponse_Events).Events.Events { + if fi, ok := n.fdMap[e.Fd]; ok { + fi.queue.Notify(waiter.EventMask(e.Events)) + } + } + n.mu.Unlock() + } +} + +// AddFD adds an FD to the list of observed FDs. +func (n *Notifier) AddFD(fd uint32, queue *waiter.Queue) error { + n.addFD(fd, queue) + return nil +} + +// UpdateFD updates the set of events the FD needs to be notified on. +func (n *Notifier) UpdateFD(fd uint32) error { + return n.updateFD(fd) +} + +// RemoveFD removes an FD from the list of observed FDs. +func (n *Notifier) RemoveFD(fd uint32) { + n.removeFD(fd) +} + +// HasFD returns true if the FD is in the list of observed FDs. +// +// This should only be used by tests to assert that FDs are correctly +// registered. +func (n *Notifier) HasFD(fd uint32) bool { + return n.hasFD(fd) +} + +// NonBlockingPoll polls the given fd in non-blocking fashion. It is used just +// to query the FD's current state; this method will block on the RPC response +// although the syscall is non-blocking. +func (n *Notifier) NonBlockingPoll(fd uint32, mask waiter.EventMask) waiter.EventMask { + for { + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Poll{&pb.PollRequest{Fd: fd, Events: uint32(mask)}}}, false /* ignoreResult */) + <-c + + res := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_Poll).Poll.Result + if e, ok := res.(*pb.PollResponse_ErrorNumber); ok { + if syscall.Errno(e.ErrorNumber) == syscall.EINTR { + continue + } + return mask + } + + return waiter.EventMask(res.(*pb.PollResponse_Events).Events) + } +} diff --git a/pkg/sentry/socket/rpcinet/rpcinet.go b/pkg/sentry/socket/rpcinet/rpcinet.go new file mode 100644 index 000000000..10b0dedc2 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/rpcinet.go @@ -0,0 +1,16 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package rpcinet implements sockets using an RPC for each syscall. +package rpcinet diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go new file mode 100644 index 000000000..574d99ba5 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -0,0 +1,567 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rpcinet + +import ( + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" + ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/notifier" + pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserr" + "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" + "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// socketOperations implements fs.FileOperations and socket.Socket for a socket +// implemented using a host socket. +type socketOperations struct { + socket.ReceiveTimeout + fsutil.PipeSeek `state:"nosave"` + fsutil.NotDirReaddir `state:"nosave"` + fsutil.NoFsync `state:"nosave"` + fsutil.NoopFlush `state:"nosave"` + fsutil.NoMMap `state:"nosave"` + + fd uint32 // must be O_NONBLOCK + wq *waiter.Queue + rpcConn *conn.RPCConnection + notifier *notifier.Notifier +} + +// Verify that we actually implement socket.Socket. +var _ = socket.Socket(&socketOperations{}) + +// New creates a new RPC socket. +func newSocketFile(ctx context.Context, stack *Stack, family int, skType int, protocol int) (*fs.File, *syserr.Error) { + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Socket{&pb.SocketRequest{Family: int64(family), Type: int64(skType | syscall.SOCK_NONBLOCK), Protocol: int64(protocol)}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Socket).Socket.Result + if e, ok := res.(*pb.SocketResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + fd := res.(*pb.SocketResponse_Fd).Fd + + var wq waiter.Queue + stack.notifier.AddFD(fd, &wq) + + dirent := socket.NewDirent(ctx, socketDevice) + return fs.NewFile(ctx, dirent, fs.FileFlags{Read: true, Write: true}, &socketOperations{ + wq: &wq, + fd: fd, + rpcConn: stack.rpcConn, + notifier: stack.notifier, + }), nil +} + +func isBlockingErrno(err error) bool { + return err == syscall.EAGAIN || err == syscall.EWOULDBLOCK +} + +func translateIOSyscallError(err error) error { + if isBlockingErrno(err) { + return syserror.ErrWouldBlock + } + return err +} + +// Release implements fs.FileOperations.Release. +func (s *socketOperations) Release() { + s.notifier.RemoveFD(s.fd) + + // We always need to close the FD. + _, _ = s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Close{&pb.CloseRequest{Fd: s.fd}}}, true /* ignoreResult */) +} + +// Readiness implements waiter.Waitable.Readiness. +func (s *socketOperations) Readiness(mask waiter.EventMask) waiter.EventMask { + return s.notifier.NonBlockingPoll(s.fd, mask) +} + +// EventRegister implements waiter.Waitable.EventRegister. +func (s *socketOperations) EventRegister(e *waiter.Entry, mask waiter.EventMask) { + s.wq.EventRegister(e, mask) + s.notifier.UpdateFD(s.fd) +} + +// EventUnregister implements waiter.Waitable.EventUnregister. +func (s *socketOperations) EventUnregister(e *waiter.Entry) { + s.wq.EventUnregister(e) + s.notifier.UpdateFD(s.fd) +} + +func rpcRead(t *kernel.Task, req *pb.SyscallRequest_Read) (*pb.ReadResponse_Data, *syserr.Error) { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Read).Read.Result + if e, ok := res.(*pb.ReadResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.ReadResponse_Data), nil +} + +// Read implements fs.FileOperations.Read. +func (s *socketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, _ int64) (int64, error) { + req := &pb.SyscallRequest_Read{&pb.ReadRequest{ + Fd: s.fd, + Length: uint32(dst.NumBytes()), + }} + + res, se := rpcRead(ctx.(*kernel.Task), req) + if se == nil { + n, e := dst.CopyOut(ctx, res.Data) + return int64(n), e + } + if se != syserr.ErrWouldBlock { + return 0, se.ToError() + } + + // We'll have to block. Register for notifications and read again when ready. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventIn) + defer s.EventUnregister(&e) + + for { + res, se := rpcRead(ctx.(*kernel.Task), req) + if se == nil { + n, e := dst.CopyOut(ctx, res.Data) + return int64(n), e + } + if se != syserr.ErrWouldBlock { + return 0, se.ToError() + } + + if err := ctx.(*kernel.Task).Block(ch); err != nil { + return 0, err + } + } +} + +func rpcWrite(t *kernel.Task, req *pb.SyscallRequest_Write) (uint32, *syserr.Error) { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Write).Write.Result + if e, ok := res.(*pb.WriteResponse_ErrorNumber); ok { + return 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.WriteResponse_Length).Length, nil +} + +// Write implements fs.FileOperations.Write. +func (s *socketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IOSequence, _ int64) (int64, error) { + t := ctx.(*kernel.Task) + v := buffer.NewView(int(src.NumBytes())) + + // Copy all the data into the buffer. + if _, err := src.CopyIn(t, v); err != nil { + return 0, err + } + + n, err := rpcWrite(t, &pb.SyscallRequest_Write{&pb.WriteRequest{Fd: s.fd, Data: v}}) + return int64(n), err.ToError() +} + +func rpcConnect(t *kernel.Task, fd uint32, sockaddr []byte) *syserr.Error { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Connect{&pb.ConnectRequest{Fd: uint32(fd), Address: sockaddr}}}, false /* ignoreResult */) + <-c + + if e := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Connect).Connect.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// Connect implements socket.Socket.Connect. +func (s *socketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error { + if !blocking { + return rpcConnect(t, s.fd, sockaddr) + } + + // Register for notification when the endpoint becomes writable, then + // initiate the connection. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventOut) + defer s.EventUnregister(&e) + + if err := rpcConnect(t, s.fd, sockaddr); err != syserr.ErrConnectStarted && err != syserr.ErrAlreadyConnecting { + return err + } + + // It's pending, so we have to wait for a notification, and fetch the + // result once the wait completes. + if err := t.Block(ch); err != nil { + return syserr.FromError(err) + } + + // Call Connect() again after blocking to find connect's result. + return rpcConnect(t, s.fd, sockaddr) +} + +func rpcAccept(t *kernel.Task, fd uint32, peer bool) (*pb.AcceptResponse_ResultPayload, *syserr.Error) { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Accept{&pb.AcceptRequest{Fd: fd, Peer: peer, Flags: syscall.SOCK_NONBLOCK}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Accept).Accept.Result + if e, ok := res.(*pb.AcceptResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + return res.(*pb.AcceptResponse_Payload).Payload, nil +} + +// Accept implements socket.Socket.Accept. +func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (kdefs.FD, interface{}, uint32, *syserr.Error) { + payload, se := rpcAccept(t, s.fd, peerRequested) + + // Check if we need to block. + if blocking && se == syserr.ErrWouldBlock { + // Register for notifications. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventIn) + defer s.EventUnregister(&e) + + // Try to accept the connection again; if it fails, then wait until we + // get a notification. + for { + if payload, se = rpcAccept(t, s.fd, peerRequested); se != syserr.ErrWouldBlock { + break + } + + if err := t.Block(ch); err != nil { + return 0, nil, 0, syserr.FromError(err) + } + } + } + + // Handle any error from accept. + if se != nil { + return 0, nil, 0, se + } + + var wq waiter.Queue + s.notifier.AddFD(payload.Fd, &wq) + + dirent := socket.NewDirent(t, socketDevice) + file := fs.NewFile(t, dirent, fs.FileFlags{Read: true, Write: true, NonBlocking: flags&linux.SOCK_NONBLOCK != 0}, &socketOperations{ + wq: &wq, + fd: payload.Fd, + notifier: s.notifier, + }) + + fdFlags := kernel.FDFlags{ + CloseOnExec: flags&linux.SOCK_CLOEXEC != 0, + } + fd, err := t.FDMap().NewFDFrom(0, file, fdFlags, t.ThreadGroup().Limits()) + if err != nil { + return 0, nil, 0, syserr.FromError(err) + } + + return fd, payload.Address.Address, payload.Address.Length, nil +} + +// Bind implements socket.Socket.Bind. +func (s *socketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Bind{&pb.BindRequest{Fd: s.fd, Address: sockaddr}}}, false /* ignoreResult */) + <-c + + if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Bind).Bind.ErrorNumber; e != 0 { + syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// Listen implements socket.Socket.Listen. +func (s *socketOperations) Listen(t *kernel.Task, backlog int) *syserr.Error { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Listen{&pb.ListenRequest{Fd: s.fd, Backlog: int64(backlog)}}}, false /* ignoreResult */) + <-c + + if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Listen).Listen.ErrorNumber; e != 0 { + syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// Shutdown implements socket.Socket.Shutdown. +func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Shutdown{&pb.ShutdownRequest{Fd: s.fd, How: int64(how)}}}, false /* ignoreResult */) + <-c + + if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Shutdown).Shutdown.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// GetSockOpt implements socket.Socket.GetSockOpt. +func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outLen int) (interface{}, *syserr.Error) { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_GetSockOpt{&pb.GetSockOptRequest{Fd: s.fd, Level: int64(level), Name: int64(name), Length: uint32(outLen)}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_GetSockOpt).GetSockOpt.Result + if e, ok := res.(*pb.GetSockOptResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.GetSockOptResponse_Opt).Opt, nil +} + +// SetSockOpt implements socket.Socket.SetSockOpt. +func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_SetSockOpt{&pb.SetSockOptRequest{Fd: s.fd, Level: int64(level), Name: int64(name), Opt: opt}}}, false /* ignoreResult */) + <-c + + if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_SetSockOpt).SetSockOpt.ErrorNumber; e != 0 { + syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +// GetPeerName implements socket.Socket.GetPeerName. +func (s *socketOperations) GetPeerName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_GetPeerName{&pb.GetPeerNameRequest{Fd: s.fd}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_GetPeerName).GetPeerName.Result + if e, ok := res.(*pb.GetPeerNameResponse_ErrorNumber); ok { + return nil, 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + addr := res.(*pb.GetPeerNameResponse_Address).Address + return addr.Address, addr.Length, nil +} + +// GetSockName implements socket.Socket.GetSockName. +func (s *socketOperations) GetSockName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { + stack := t.NetworkContext().(*Stack) + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_GetSockName{&pb.GetSockNameRequest{Fd: s.fd}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_GetSockName).GetSockName.Result + if e, ok := res.(*pb.GetSockNameResponse_ErrorNumber); ok { + return nil, 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + addr := res.(*pb.GetSockNameResponse_Address).Address + return addr.Address, addr.Length, nil +} + +// Ioctl implements fs.FileOperations.Ioctl. +func (s *socketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { + return 0, syserror.ENOTTY +} + +func rpcRecvMsg(t *kernel.Task, req *pb.SyscallRequest_Recvmsg) (*pb.RecvmsgResponse_ResultPayload, *syserr.Error) { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Recvmsg).Recvmsg.Result + if e, ok := res.(*pb.RecvmsgResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.RecvmsgResponse_Payload).Payload, nil +} + +// RecvMsg implements socket.Socket.RecvMsg. +func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, unix.ControlMessages, *syserr.Error) { + req := &pb.SyscallRequest_Recvmsg{&pb.RecvmsgRequest{ + Fd: s.fd, + Length: uint32(dst.NumBytes()), + Sender: senderRequested, + Trunc: flags&linux.MSG_TRUNC != 0, + Peek: flags&linux.MSG_PEEK != 0, + }} + + res, err := rpcRecvMsg(t, req) + if err == nil { + n, e := dst.CopyOut(t, res.Data) + return int(n), res.Address.GetAddress(), res.Address.GetLength(), unix.ControlMessages{}, syserr.FromError(e) + } + if err != syserr.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { + return 0, nil, 0, unix.ControlMessages{}, err + } + + // We'll have to block. Register for notifications and keep trying to + // send all the data. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventIn) + defer s.EventUnregister(&e) + + for { + res, err := rpcRecvMsg(t, req) + if err == nil { + n, e := dst.CopyOut(t, res.Data) + return int(n), res.Address.GetAddress(), res.Address.GetLength(), unix.ControlMessages{}, syserr.FromError(e) + } + if err != syserr.ErrWouldBlock { + return 0, nil, 0, unix.ControlMessages{}, err + } + + if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { + if err == syserror.ETIMEDOUT { + return 0, nil, 0, unix.ControlMessages{}, syserr.ErrTryAgain + } + return 0, nil, 0, unix.ControlMessages{}, syserr.FromError(err) + } + } +} + +func rpcSendMsg(t *kernel.Task, req *pb.SyscallRequest_Sendmsg) (uint32, *syserr.Error) { + s := t.NetworkContext().(*Stack) + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Sendmsg).Sendmsg.Result + if e, ok := res.(*pb.SendmsgResponse_ErrorNumber); ok { + return 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.SendmsgResponse_Length).Length, nil +} + +// SendMsg implements socket.Socket.SendMsg. +func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { + // Whitelist flags. + if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 { + return 0, syserr.ErrInvalidArgument + } + + // Reject control messages. + if !controlMessages.Empty() { + return 0, syserr.ErrInvalidArgument + } + + v := buffer.NewView(int(src.NumBytes())) + + // Copy all the data into the buffer. + if _, err := src.CopyIn(t, v); err != nil { + return 0, syserr.FromError(err) + } + + // TODO: this needs to change to map directly to a SendMsg syscall + // in the RPC. + req := &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ + Fd: uint32(s.fd), + Data: v, + Address: to, + More: flags&linux.MSG_MORE != 0, + EndOfRecord: flags&linux.MSG_EOR != 0, + }} + + n, err := rpcSendMsg(t, req) + if err != syserr.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { + return int(n), err + } + + // We'll have to block. Register for notification and keep trying to + // send all the data. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventOut) + defer s.EventUnregister(&e) + + for { + n, err := rpcSendMsg(t, req) + if err != syserr.ErrWouldBlock { + return int(n), err + } + + if err := t.Block(ch); err != nil { + return 0, syserr.FromError(err) + } + } +} + +type socketProvider struct { + family int +} + +// Socket implements socket.Provider.Socket. +func (p *socketProvider) Socket(t *kernel.Task, stypeflags unix.SockType, protocol int) (*fs.File, *syserr.Error) { + // Check that we are using the RPC network stack. + stack := t.NetworkContext() + if stack == nil { + return nil, nil + } + + s, ok := stack.(*Stack) + if !ok { + return nil, nil + } + + // Only accept TCP and UDP. + // + // Try to restrict the flags we will accept to minimize backwards + // incompatability with netstack. + stype := int(stypeflags) & linux.SOCK_TYPE_MASK + switch stype { + case syscall.SOCK_STREAM: + switch protocol { + case 0, syscall.IPPROTO_TCP: + // ok + default: + return nil, nil + } + case syscall.SOCK_DGRAM: + switch protocol { + case 0, syscall.IPPROTO_UDP: + // ok + default: + return nil, nil + } + default: + return nil, nil + } + + return newSocketFile(t, s, p.family, stype, 0) +} + +// Pair implements socket.Provider.Pair. +func (p *socketProvider) Pair(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { + // Not supported by AF_INET/AF_INET6. + return nil, nil, nil +} + +func init() { + for _, family := range []int{syscall.AF_INET, syscall.AF_INET6} { + socket.RegisterProvider(family, &socketProvider{family}) + } +} diff --git a/pkg/sentry/socket/rpcinet/stack.go b/pkg/sentry/socket/rpcinet/stack.go new file mode 100644 index 000000000..503e0e932 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/stack.go @@ -0,0 +1,175 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rpcinet + +import ( + "fmt" + "strings" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/inet" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/hostinet" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/notifier" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/unet" +) + +// Stack implements inet.Stack for RPC backed sockets. +type Stack struct { + // We intentionally do not allow these values to be changed to remain + // consistent with the other networking stacks. + interfaces map[int32]inet.Interface + interfaceAddrs map[int32][]inet.InterfaceAddr + supportsIPv6 bool + tcpRecvBufSize inet.TCPBufferSize + tcpSendBufSize inet.TCPBufferSize + tcpSACKEnabled bool + rpcConn *conn.RPCConnection + notifier *notifier.Notifier +} + +func readTCPBufferSizeFile(conn *conn.RPCConnection, filename string) (inet.TCPBufferSize, error) { + contents, se := conn.RPCReadFile(filename) + if se != nil { + return inet.TCPBufferSize{}, fmt.Errorf("failed to read %s: %v", filename, se) + } + ioseq := usermem.BytesIOSequence(contents) + fields := make([]int32, 3) + if n, err := usermem.CopyInt32StringsInVec(context.Background(), ioseq.IO, ioseq.Addrs, fields, ioseq.Opts); n != ioseq.NumBytes() || err != nil { + return inet.TCPBufferSize{}, fmt.Errorf("failed to parse %s (%q): got %v after %d/%d bytes", filename, contents, err, n, ioseq.NumBytes()) + } + return inet.TCPBufferSize{ + Min: int(fields[0]), + Default: int(fields[1]), + Max: int(fields[2]), + }, nil +} + +// NewStack returns a Stack containing the current state of the host network +// stack. +func NewStack(fd int32) (*Stack, error) { + sock, err := unet.NewSocket(int(fd)) + if err != nil { + return nil, err + } + + stack := &Stack{ + interfaces: make(map[int32]inet.Interface), + interfaceAddrs: make(map[int32][]inet.InterfaceAddr), + rpcConn: conn.NewRPCConnection(sock), + } + + var e error + stack.notifier, e = notifier.NewRPCNotifier(stack.rpcConn) + if e != nil { + return nil, e + } + + // Load the configuration values from procfs. + tcpRMem, e := readTCPBufferSizeFile(stack.rpcConn, "/proc/sys/net/ipv4/tcp_rmem") + if e != nil { + return nil, e + } + stack.tcpRecvBufSize = tcpRMem + + tcpWMem, e := readTCPBufferSizeFile(stack.rpcConn, "/proc/sys/net/ipv4/tcp_wmem") + if e != nil { + return nil, e + } + stack.tcpSendBufSize = tcpWMem + + ipv6, se := stack.rpcConn.RPCReadFile("/proc/net/if_inet6") + if len(string(ipv6)) > 0 { + stack.supportsIPv6 = true + } + + sackFile := "/proc/sys/net/ipv4/tcp_sack" + sack, se := stack.rpcConn.RPCReadFile(sackFile) + if se != nil { + return nil, fmt.Errorf("failed to read %s: %v", sackFile, se) + } + stack.tcpSACKEnabled = strings.TrimSpace(string(sack)) != "0" + + links, err := stack.DoNetlinkRouteRequest(syscall.RTM_GETLINK) + if err != nil { + return nil, fmt.Errorf("RTM_GETLINK failed: %v", err) + } + + addrs, err := stack.DoNetlinkRouteRequest(syscall.RTM_GETADDR) + if err != nil { + return nil, fmt.Errorf("RTM_GETADDR failed: %v", err) + } + + e = hostinet.ExtractHostInterfaces(links, addrs, stack.interfaces, stack.interfaceAddrs) + if e != nil { + return nil, e + } + + return stack, nil +} + +// Interfaces implements inet.Stack.Interfaces. +func (s *Stack) Interfaces() map[int32]inet.Interface { + return s.interfaces +} + +// InterfaceAddrs implements inet.Stack.InterfaceAddrs. +func (s *Stack) InterfaceAddrs() map[int32][]inet.InterfaceAddr { + return s.interfaceAddrs +} + +// SupportsIPv6 implements inet.Stack.SupportsIPv6. +func (s *Stack) SupportsIPv6() bool { + return s.supportsIPv6 +} + +// TCPReceiveBufferSize implements inet.Stack.TCPReceiveBufferSize. +func (s *Stack) TCPReceiveBufferSize() (inet.TCPBufferSize, error) { + return s.tcpRecvBufSize, nil +} + +// SetTCPReceiveBufferSize implements inet.Stack.SetTCPReceiveBufferSize. +func (s *Stack) SetTCPReceiveBufferSize(size inet.TCPBufferSize) error { + // To keep all the supported stacks consistent we don't allow changing this + // value even though it would be possible via an RPC. + return syserror.EACCES +} + +// TCPSendBufferSize implements inet.Stack.TCPSendBufferSize. +func (s *Stack) TCPSendBufferSize() (inet.TCPBufferSize, error) { + return s.tcpSendBufSize, nil +} + +// SetTCPSendBufferSize implements inet.Stack.SetTCPSendBufferSize. +func (s *Stack) SetTCPSendBufferSize(size inet.TCPBufferSize) error { + // To keep all the supported stacks consistent we don't allow changing this + // value even though it would be possible via an RPC. + return syserror.EACCES +} + +// TCPSACKEnabled implements inet.Stack.TCPSACKEnabled. +func (s *Stack) TCPSACKEnabled() (bool, error) { + return s.tcpSACKEnabled, nil +} + +// SetTCPSACKEnabled implements inet.Stack.SetTCPSACKEnabled. +func (s *Stack) SetTCPSACKEnabled(enabled bool) error { + // To keep all the supported stacks consistent we don't allow changing this + // value even though it would be possible via an RPC. + return syserror.EACCES +} diff --git a/pkg/sentry/socket/rpcinet/stack_unsafe.go b/pkg/sentry/socket/rpcinet/stack_unsafe.go new file mode 100644 index 000000000..9a896c623 --- /dev/null +++ b/pkg/sentry/socket/rpcinet/stack_unsafe.go @@ -0,0 +1,193 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package rpcinet + +import ( + "syscall" + "unsafe" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/binary" + pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserr" +) + +// NewNetlinkRouteRequest builds a netlink message for getting the RIB, +// the routing information base. +func newNetlinkRouteRequest(proto, seq, family int) []byte { + rr := &syscall.NetlinkRouteRequest{} + rr.Header.Len = uint32(syscall.NLMSG_HDRLEN + syscall.SizeofRtGenmsg) + rr.Header.Type = uint16(proto) + rr.Header.Flags = syscall.NLM_F_DUMP | syscall.NLM_F_REQUEST + rr.Header.Seq = uint32(seq) + rr.Data.Family = uint8(family) + return netlinkRRtoWireFormat(rr) +} + +func netlinkRRtoWireFormat(rr *syscall.NetlinkRouteRequest) []byte { + b := make([]byte, rr.Header.Len) + *(*uint32)(unsafe.Pointer(&b[0:4][0])) = rr.Header.Len + *(*uint16)(unsafe.Pointer(&b[4:6][0])) = rr.Header.Type + *(*uint16)(unsafe.Pointer(&b[6:8][0])) = rr.Header.Flags + *(*uint32)(unsafe.Pointer(&b[8:12][0])) = rr.Header.Seq + *(*uint32)(unsafe.Pointer(&b[12:16][0])) = rr.Header.Pid + b[16] = byte(rr.Data.Family) + return b +} + +func (s *Stack) getNetlinkFd() (uint32, *syserr.Error) { + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Socket{&pb.SocketRequest{Family: int64(syscall.AF_NETLINK), Type: int64(syscall.SOCK_RAW | syscall.SOCK_NONBLOCK), Protocol: int64(syscall.NETLINK_ROUTE)}}}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Socket).Socket.Result + if e, ok := res.(*pb.SocketResponse_ErrorNumber); ok { + return 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + return res.(*pb.SocketResponse_Fd).Fd, nil +} + +func (s *Stack) bindNetlinkFd(fd uint32, sockaddr []byte) *syserr.Error { + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Bind{&pb.BindRequest{Fd: fd, Address: sockaddr}}}, false /* ignoreResult */) + <-c + + if e := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Bind).Bind.ErrorNumber; e != 0 { + return syserr.FromHost(syscall.Errno(e)) + } + return nil +} + +func (s *Stack) closeNetlinkFd(fd uint32) { + _, _ = s.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Close{&pb.CloseRequest{Fd: fd}}}, true /* ignoreResult */) +} + +func (s *Stack) rpcSendMsg(req *pb.SyscallRequest_Sendmsg) (uint32, *syserr.Error) { + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Sendmsg).Sendmsg.Result + if e, ok := res.(*pb.SendmsgResponse_ErrorNumber); ok { + return 0, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.SendmsgResponse_Length).Length, nil +} + +func (s *Stack) sendMsg(fd uint32, buf []byte, to []byte, flags int) (int, *syserr.Error) { + // Whitelist flags. + if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 { + return 0, syserr.ErrInvalidArgument + } + + req := &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ + Fd: fd, + Data: buf, + Address: to, + More: flags&linux.MSG_MORE != 0, + EndOfRecord: flags&linux.MSG_EOR != 0, + }} + + n, err := s.rpcSendMsg(req) + return int(n), err +} + +func (s *Stack) rpcRecvMsg(req *pb.SyscallRequest_Recvmsg) (*pb.RecvmsgResponse_ResultPayload, *syserr.Error) { + id, c := s.rpcConn.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-c + + res := s.rpcConn.Request(id).Result.(*pb.SyscallResponse_Recvmsg).Recvmsg.Result + if e, ok := res.(*pb.RecvmsgResponse_ErrorNumber); ok { + return nil, syserr.FromHost(syscall.Errno(e.ErrorNumber)) + } + + return res.(*pb.RecvmsgResponse_Payload).Payload, nil +} + +func (s *Stack) recvMsg(fd, l, flags uint32) ([]byte, *syserr.Error) { + req := &pb.SyscallRequest_Recvmsg{&pb.RecvmsgRequest{ + Fd: fd, + Length: l, + Sender: false, + Trunc: flags&linux.MSG_TRUNC != 0, + Peek: flags&linux.MSG_PEEK != 0, + }} + + res, err := s.rpcRecvMsg(req) + if err != nil { + return nil, err + } + return res.Data, nil +} + +func (s *Stack) netlinkRequest(proto, family int) ([]byte, error) { + fd, err := s.getNetlinkFd() + if err != nil { + return nil, err.ToError() + } + defer s.closeNetlinkFd(fd) + + lsa := syscall.SockaddrNetlink{Family: syscall.AF_NETLINK} + b := binary.Marshal(nil, usermem.ByteOrder, &lsa) + if err := s.bindNetlinkFd(fd, b); err != nil { + return nil, err.ToError() + } + + wb := newNetlinkRouteRequest(proto, 1, family) + _, err = s.sendMsg(fd, wb, b, 0) + if err != nil { + return nil, err.ToError() + } + + var tab []byte +done: + for { + rb, err := s.recvMsg(fd, uint32(syscall.Getpagesize()), 0) + nr := len(rb) + if err != nil { + return nil, err.ToError() + } + + if nr < syscall.NLMSG_HDRLEN { + return nil, syserr.ErrInvalidArgument.ToError() + } + + tab = append(tab, rb...) + msgs, e := syscall.ParseNetlinkMessage(rb) + if e != nil { + return nil, e + } + + for _, m := range msgs { + if m.Header.Type == syscall.NLMSG_DONE { + break done + } + if m.Header.Type == syscall.NLMSG_ERROR { + return nil, syserr.ErrInvalidArgument.ToError() + } + } + } + + return tab, nil +} + +// DoNetlinkRouteRequest returns routing information base, also known as RIB, +// which consists of network facility information, states and parameters. +func (s *Stack) DoNetlinkRouteRequest(req int) ([]syscall.NetlinkMessage, error) { + data, err := s.netlinkRequest(req, syscall.AF_UNSPEC) + if err != nil { + return nil, err + } + return syscall.ParseNetlinkMessage(data) +} diff --git a/pkg/sentry/socket/rpcinet/syscall_rpc.proto b/pkg/sentry/socket/rpcinet/syscall_rpc.proto new file mode 100644 index 000000000..b845b1bce --- /dev/null +++ b/pkg/sentry/socket/rpcinet/syscall_rpc.proto @@ -0,0 +1,351 @@ +syntax = "proto3"; + +// package syscall_rpc is a set of networking related system calls that can be +// forwarded to a socket gofer. +// +// TODO: Document individual RPCs. +package syscall_rpc; + +message SendmsgRequest { + uint32 fd = 1; + bytes data = 2; + bytes address = 3; + bool more = 4; + bool end_of_record = 5; +} + +message SendmsgResponse { + oneof result { + uint32 error_number = 1; + uint32 length = 2; + } +} + +message IOCtlRequest { + uint32 fd = 1; + uint32 cmd = 2; + uint64 arg = 3; +} + +message IOCtlResponse { + oneof result { + uint32 error_number = 1; + uint64 value = 2; + } +} + +message RecvmsgRequest { + uint32 fd = 1; + uint32 length = 2; + bool sender = 3; + bool peek = 4; + bool trunc = 5; +} + +message OpenRequest { + bytes path = 1; + uint32 flags = 2; + uint32 mode = 3; +} + +message OpenResponse { + oneof result { + uint32 error_number = 1; + uint32 fd = 2; + } +} + +message ReadRequest { + uint32 fd = 1; + uint32 length = 2; +} + +message ReadResponse { + oneof result { + uint32 error_number = 1; + bytes data = 2; + } +} + +message ReadFileRequest { + string path = 1; +} + +message ReadFileResponse { + oneof result { + uint32 error_number = 1; + bytes data = 2; + } +} + +message WriteRequest { + uint32 fd = 1; + bytes data = 2; +} + +message WriteResponse { + oneof result { + uint32 error_number = 1; + uint32 length = 2; + } +} + +message WriteFileRequest { + string path = 1; + bytes content = 2; +} + +message WriteFileResponse { + uint32 error_number = 1; + uint32 written = 2; +} + +message AddressResponse { + bytes address = 1; + uint32 length = 2; +} + +message RecvmsgResponse { + message ResultPayload { + bytes data = 1; + AddressResponse address = 2; + uint32 length = 3; + } + oneof result { + uint32 error_number = 1; + ResultPayload payload = 2; + } +} + +message BindRequest { + uint32 fd = 1; + bytes address = 2; +} + +message BindResponse { + uint32 error_number = 1; +} + +message AcceptRequest { + uint32 fd = 1; + bool peer = 2; + int64 flags = 3; +} + +message AcceptResponse { + message ResultPayload { + uint32 fd = 1; + AddressResponse address = 2; + } + oneof result { + uint32 error_number = 1; + ResultPayload payload = 2; + } +} + +message ConnectRequest { + uint32 fd = 1; + bytes address = 2; +} + +message ConnectResponse { + uint32 error_number = 1; +} + +message ListenRequest { + uint32 fd = 1; + int64 backlog = 2; +} + +message ListenResponse { + uint32 error_number = 1; +} + +message ShutdownRequest { + uint32 fd = 1; + int64 how = 2; +} + +message ShutdownResponse { + uint32 error_number = 1; +} + +message CloseRequest { + uint32 fd = 1; +} + +message CloseResponse { + uint32 error_number = 1; +} + +message GetSockOptRequest { + uint32 fd = 1; + int64 level = 2; + int64 name = 3; + uint32 length = 4; +} + +message GetSockOptResponse { + oneof result { + uint32 error_number = 1; + bytes opt = 2; + } +} + +message SetSockOptRequest { + uint32 fd = 1; + int64 level = 2; + int64 name = 3; + bytes opt = 4; +} + +message SetSockOptResponse { + uint32 error_number = 1; +} + +message GetSockNameRequest { + uint32 fd = 1; +} + +message GetSockNameResponse { + oneof result { + uint32 error_number = 1; + AddressResponse address = 2; + } +} + +message GetPeerNameRequest { + uint32 fd = 1; +} + +message GetPeerNameResponse { + oneof result { + uint32 error_number = 1; + AddressResponse address = 2; + } +} + +message SocketRequest { + int64 family = 1; + int64 type = 2; + int64 protocol = 3; +} + +message SocketResponse { + oneof result { + uint32 error_number = 1; + uint32 fd = 2; + } +} + +message EpollWaitRequest { + uint32 fd = 1; + uint32 num_events = 2; + sint64 msec = 3; +} + +message EpollEvent { + uint32 fd = 1; + uint32 events = 2; +} + +message EpollEvents { + repeated EpollEvent events = 1; +} + +message EpollWaitResponse { + oneof result { + uint32 error_number = 1; + EpollEvents events = 2; + } +} + +message EpollCtlRequest { + uint32 epfd = 1; + int64 op = 2; + uint32 fd = 3; + EpollEvent event = 4; +} + +message EpollCtlResponse { + uint32 error_number = 1; +} + +message EpollCreate1Request { + int64 flag = 1; +} + +message EpollCreate1Response { + oneof result { + uint32 error_number = 1; + uint32 fd = 2; + } +} + +message PollRequest { + uint32 fd = 1; + uint32 events = 2; +} + +message PollResponse { + oneof result { + uint32 error_number = 1; + uint32 events = 2; + } +} + +message SyscallRequest { + oneof args { + SocketRequest socket = 1; + SendmsgRequest sendmsg = 2; + RecvmsgRequest recvmsg = 3; + BindRequest bind = 4; + AcceptRequest accept = 5; + ConnectRequest connect = 6; + ListenRequest listen = 7; + ShutdownRequest shutdown = 8; + CloseRequest close = 9; + GetSockOptRequest get_sock_opt = 10; + SetSockOptRequest set_sock_opt = 11; + GetSockNameRequest get_sock_name = 12; + GetPeerNameRequest get_peer_name = 13; + EpollWaitRequest epoll_wait = 14; + EpollCtlRequest epoll_ctl = 15; + EpollCreate1Request epoll_create1 = 16; + PollRequest poll = 17; + ReadRequest read = 18; + WriteRequest write = 19; + OpenRequest open = 20; + IOCtlRequest ioctl = 21; + WriteFileRequest write_file = 22; + ReadFileRequest read_file = 23; + } +} + +message SyscallResponse { + oneof result { + SocketResponse socket = 1; + SendmsgResponse sendmsg = 2; + RecvmsgResponse recvmsg = 3; + BindResponse bind = 4; + AcceptResponse accept = 5; + ConnectResponse connect = 6; + ListenResponse listen = 7; + ShutdownResponse shutdown = 8; + CloseResponse close = 9; + GetSockOptResponse get_sock_opt = 10; + SetSockOptResponse set_sock_opt = 11; + GetSockNameResponse get_sock_name = 12; + GetPeerNameResponse get_peer_name = 13; + EpollWaitResponse epoll_wait = 14; + EpollCtlResponse epoll_ctl = 15; + EpollCreate1Response epoll_create1 = 16; + PollResponse poll = 17; + ReadResponse read = 18; + WriteResponse write = 19; + OpenResponse open = 20; + IOCtlResponse ioctl = 21; + WriteFileResponse write_file = 22; + ReadFileResponse read_file = 23; + } +} -- cgit v1.2.3 From 3d3deef573a54e031cb98038b9f617f5fac31044 Mon Sep 17 00:00:00 2001 From: Ian Gudger Date: Tue, 1 May 2018 22:11:07 -0700 Subject: Implement SO_TIMESTAMP PiperOrigin-RevId: 195047018 Change-Id: I6d99528a00a2125f414e1e51e067205289ec9d3d --- pkg/dhcp/client.go | 4 +- pkg/dhcp/dhcp_test.go | 2 +- pkg/dhcp/server.go | 2 +- pkg/sentry/fs/host/socket_test.go | 2 +- pkg/sentry/kernel/kernel.go | 9 +++ pkg/sentry/socket/BUILD | 1 + pkg/sentry/socket/control/control.go | 35 +++++++++++ pkg/sentry/socket/epsocket/epsocket.go | 69 +++++++++++++++------- pkg/sentry/socket/hostinet/socket.go | 10 ++-- pkg/sentry/socket/netlink/socket.go | 16 ++--- pkg/sentry/socket/rpcinet/socket.go | 20 +++---- pkg/sentry/socket/socket.go | 12 +++- pkg/sentry/socket/unix/unix.go | 14 ++--- pkg/sentry/strace/socket.go | 29 ++++++++- pkg/sentry/syscalls/linux/sys_socket.go | 21 ++++--- pkg/tcpip/adapters/gonet/gonet.go | 4 +- pkg/tcpip/adapters/gonet/gonet_test.go | 2 +- pkg/tcpip/network/arp/arp_test.go | 2 +- pkg/tcpip/network/ipv4/icmp_test.go | 2 +- pkg/tcpip/sample/tun_tcp_connect/main.go | 4 +- pkg/tcpip/sample/tun_tcp_echo/main.go | 4 +- pkg/tcpip/stack/stack.go | 17 +++++- pkg/tcpip/stack/stack_test.go | 22 +++---- pkg/tcpip/stack/transport_test.go | 16 ++--- pkg/tcpip/tcpip.go | 48 +++++++++++++-- pkg/tcpip/transport/tcp/endpoint.go | 22 +++---- pkg/tcpip/transport/tcp/tcp_test.go | 46 +++++++-------- pkg/tcpip/transport/tcp/tcp_timestamp_test.go | 4 +- pkg/tcpip/transport/tcp/testing/context/context.go | 2 +- pkg/tcpip/transport/udp/endpoint.go | 37 ++++++++++-- pkg/tcpip/transport/udp/udp_test.go | 10 ++-- runsc/boot/loader.go | 7 ++- 32 files changed, 345 insertions(+), 150 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/dhcp/client.go b/pkg/dhcp/client.go index 9a4fd7ae4..37deb69ff 100644 --- a/pkg/dhcp/client.go +++ b/pkg/dhcp/client.go @@ -162,7 +162,7 @@ func (c *Client) Request(ctx context.Context, requestedAddr tcpip.Address) error // DHCPOFFER for { var addr tcpip.FullAddress - v, err := epin.Read(&addr) + v, _, err := epin.Read(&addr) if err == tcpip.ErrWouldBlock { select { case <-ch: @@ -216,7 +216,7 @@ func (c *Client) Request(ctx context.Context, requestedAddr tcpip.Address) error // DHCPACK for { var addr tcpip.FullAddress - v, err := epin.Read(&addr) + v, _, err := epin.Read(&addr) if err == tcpip.ErrWouldBlock { select { case <-ch: diff --git a/pkg/dhcp/dhcp_test.go b/pkg/dhcp/dhcp_test.go index d56b93997..ed884fcb6 100644 --- a/pkg/dhcp/dhcp_test.go +++ b/pkg/dhcp/dhcp_test.go @@ -36,7 +36,7 @@ func TestDHCP(t *testing.T) { } }() - s := stack.New([]string{ipv4.ProtocolName}, []string{udp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName}, []string{udp.ProtocolName}) const nicid tcpip.NICID = 1 if err := s.CreateNIC(nicid, id); err != nil { diff --git a/pkg/dhcp/server.go b/pkg/dhcp/server.go index d132d90b4..8816203a8 100644 --- a/pkg/dhcp/server.go +++ b/pkg/dhcp/server.go @@ -104,7 +104,7 @@ func (s *Server) reader(ctx context.Context) { for { var addr tcpip.FullAddress - v, err := s.ep.Read(&addr) + v, _, err := s.ep.Read(&addr) if err == tcpip.ErrWouldBlock { select { case <-ch: diff --git a/pkg/sentry/fs/host/socket_test.go b/pkg/sentry/fs/host/socket_test.go index 80c46dcfa..9b73c5173 100644 --- a/pkg/sentry/fs/host/socket_test.go +++ b/pkg/sentry/fs/host/socket_test.go @@ -142,7 +142,7 @@ func TestSocketSendMsgLen0(t *testing.T) { defer sfile.DecRef() s := sfile.FileOperations.(socket.Socket) - n, terr := s.SendMsg(nil, usermem.BytesIOSequence(nil), []byte{}, 0, unix.ControlMessages{}) + n, terr := s.SendMsg(nil, usermem.BytesIOSequence(nil), []byte{}, 0, socket.ControlMessages{}) if n != 0 { t.Fatalf("socket sendmsg() failed: %v wrote: %d", terr, n) } diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index 0932965e0..25c8dd885 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -887,6 +887,15 @@ func (k *Kernel) SetExitError(err error) { } } +// NowNanoseconds implements tcpip.Clock.NowNanoseconds. +func (k *Kernel) NowNanoseconds() int64 { + now, err := k.timekeeper.GetTime(sentrytime.Realtime) + if err != nil { + panic("Kernel.NowNanoseconds: " + err.Error()) + } + return now +} + // SupervisorContext returns a Context with maximum privileges in k. It should // only be used by goroutines outside the control of the emulated kernel // defined by e. diff --git a/pkg/sentry/socket/BUILD b/pkg/sentry/socket/BUILD index 87e32df37..5500a676e 100644 --- a/pkg/sentry/socket/BUILD +++ b/pkg/sentry/socket/BUILD @@ -32,6 +32,7 @@ go_library( "//pkg/sentry/usermem", "//pkg/state", "//pkg/syserr", + "//pkg/tcpip", "//pkg/tcpip/transport/unix", ], ) diff --git a/pkg/sentry/socket/control/control.go b/pkg/sentry/socket/control/control.go index cb34cbc85..17ecdd11c 100644 --- a/pkg/sentry/socket/control/control.go +++ b/pkg/sentry/socket/control/control.go @@ -208,6 +208,31 @@ func putCmsg(buf []byte, msgType uint32, align uint, data []int32) []byte { return alignSlice(buf, align) } +func putCmsgStruct(buf []byte, msgType uint32, align uint, data interface{}) []byte { + if cap(buf)-len(buf) < linux.SizeOfControlMessageHeader { + return buf + } + ob := buf + + buf = putUint64(buf, uint64(linux.SizeOfControlMessageHeader)) + buf = putUint32(buf, linux.SOL_SOCKET) + buf = putUint32(buf, msgType) + + hdrBuf := buf + + buf = binary.Marshal(buf, usermem.ByteOrder, data) + + // Check if we went over. + if cap(buf) != cap(ob) { + return hdrBuf + } + + // Fix up length. + putUint64(ob, uint64(len(buf)-len(ob))) + + return alignSlice(buf, align) +} + // Credentials implements SCMCredentials.Credentials. func (c *scmCredentials) Credentials(t *kernel.Task) (kernel.ThreadID, auth.UID, auth.GID) { // "When a process's user and group IDs are passed over a UNIX domain @@ -261,6 +286,16 @@ func alignSlice(buf []byte, align uint) []byte { return buf[:aligned] } +// PackTimestamp packs a SO_TIMESTAMP socket control message. +func PackTimestamp(t *kernel.Task, timestamp int64, buf []byte) []byte { + return putCmsgStruct( + buf, + linux.SO_TIMESTAMP, + t.Arch().Width(), + linux.NsecToTimeval(timestamp), + ) +} + // Parse parses a raw socket control message into portable objects. func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (unix.ControlMessages, error) { var ( diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 3fc3ea58f..5701ecfac 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -109,6 +109,7 @@ type SocketOperations struct { // readMu protects access to readView, control, and sender. readMu sync.Mutex `state:"nosave"` readView buffer.View + readCM tcpip.ControlMessages sender tcpip.FullAddress } @@ -210,12 +211,13 @@ func (s *SocketOperations) fetchReadView() *syserr.Error { s.readView = nil s.sender = tcpip.FullAddress{} - v, err := s.Endpoint.Read(&s.sender) + v, cms, err := s.Endpoint.Read(&s.sender) if err != nil { return syserr.TranslateNetstackError(err) } s.readView = v + s.readCM = cms return nil } @@ -230,7 +232,7 @@ func (s *SocketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOS if dst.NumBytes() == 0 { return 0, nil } - n, _, _, err := s.nonBlockingRead(ctx, dst, false, false, false) + n, _, _, _, err := s.nonBlockingRead(ctx, dst, false, false, false) if err == syserr.ErrWouldBlock { return int64(n), syserror.ErrWouldBlock } @@ -552,6 +554,18 @@ func GetSockOpt(t *kernel.Task, s socket.Socket, ep commonEndpoint, family int, } return linux.NsecToTimeval(s.RecvTimeout()), nil + + case linux.SO_TIMESTAMP: + if outLen < sizeOfInt32 { + return nil, syserr.ErrInvalidArgument + } + + var v tcpip.TimestampOption + if err := ep.GetSockOpt(&v); err != nil { + return nil, syserr.TranslateNetstackError(err) + } + + return int32(v), nil } case syscall.SOL_TCP: @@ -659,6 +673,14 @@ func SetSockOpt(t *kernel.Task, s socket.Socket, ep commonEndpoint, level int, n binary.Unmarshal(optVal[:linux.SizeOfTimeval], usermem.ByteOrder, &v) s.SetRecvTimeout(v.ToNsecCapped()) return nil + + case linux.SO_TIMESTAMP: + if len(optVal) < sizeOfInt32 { + return syserr.ErrInvalidArgument + } + + v := usermem.ByteOrder.Uint32(optVal) + return syserr.TranslateNetstackError(ep.SetSockOpt(tcpip.TimestampOption(v))) } case syscall.SOL_TCP: @@ -823,7 +845,9 @@ func (s *SocketOperations) coalescingRead(ctx context.Context, dst usermem.IOSeq } // nonBlockingRead issues a non-blocking read. -func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSequence, peek, trunc, senderRequested bool) (int, interface{}, uint32, *syserr.Error) { +// +// TODO: Support timestamps for stream sockets. +func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSequence, peek, trunc, senderRequested bool) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { isPacket := s.isPacketBased() // Fast path for regular reads from stream (e.g., TCP) endpoints. Note @@ -839,14 +863,14 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe s.readMu.Lock() n, err := s.coalescingRead(ctx, dst, trunc) s.readMu.Unlock() - return n, nil, 0, err + return n, nil, 0, socket.ControlMessages{}, err } s.readMu.Lock() defer s.readMu.Unlock() if err := s.fetchReadView(); err != nil { - return 0, nil, 0, err + return 0, nil, 0, socket.ControlMessages{}, err } if !isPacket && peek && trunc { @@ -854,14 +878,14 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe // amount that could be read. var rql tcpip.ReceiveQueueSizeOption if err := s.Endpoint.GetSockOpt(&rql); err != nil { - return 0, nil, 0, syserr.TranslateNetstackError(err) + return 0, nil, 0, socket.ControlMessages{}, syserr.TranslateNetstackError(err) } available := len(s.readView) + int(rql) bufLen := int(dst.NumBytes()) if available < bufLen { - return available, nil, 0, nil + return available, nil, 0, socket.ControlMessages{}, nil } - return bufLen, nil, 0, nil + return bufLen, nil, 0, socket.ControlMessages{}, nil } n, err := dst.CopyOut(ctx, s.readView) @@ -874,17 +898,18 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe if peek { if l := len(s.readView); trunc && l > n { // isPacket must be true. - return l, addr, addrLen, syserr.FromError(err) + return l, addr, addrLen, socket.ControlMessages{IP: s.readCM}, syserr.FromError(err) } if isPacket || err != nil { - return int(n), addr, addrLen, syserr.FromError(err) + return int(n), addr, addrLen, socket.ControlMessages{IP: s.readCM}, syserr.FromError(err) } // We need to peek beyond the first message. dst = dst.DropFirst(n) num, err := dst.CopyOutFrom(ctx, safemem.FromVecReaderFunc{func(dsts [][]byte) (int64, error) { - n, err := s.Endpoint.Peek(dsts) + n, _, err := s.Endpoint.Peek(dsts) + // TODO: Handle peek timestamp. if err != nil { return int64(n), syserr.TranslateNetstackError(err).ToError() } @@ -895,7 +920,7 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe // We got some data, so no need to return an error. err = nil } - return int(n), nil, 0, syserr.FromError(err) + return int(n), nil, 0, socket.ControlMessages{IP: s.readCM}, syserr.FromError(err) } var msgLen int @@ -908,15 +933,15 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe } if trunc { - return msgLen, addr, addrLen, syserr.FromError(err) + return msgLen, addr, addrLen, socket.ControlMessages{IP: s.readCM}, syserr.FromError(err) } - return int(n), addr, addrLen, syserr.FromError(err) + return int(n), addr, addrLen, socket.ControlMessages{IP: s.readCM}, syserr.FromError(err) } // RecvMsg implements the linux syscall recvmsg(2) for sockets backed by // tcpip.Endpoint. -func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages unix.ControlMessages, err *syserr.Error) { +func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) { trunc := flags&linux.MSG_TRUNC != 0 peek := flags&linux.MSG_PEEK != 0 @@ -924,7 +949,7 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags // Stream sockets ignore the sender address. senderRequested = false } - n, senderAddr, senderAddrLen, err = s.nonBlockingRead(t, dst, peek, trunc, senderRequested) + n, senderAddr, senderAddrLen, controlMessages, err = s.nonBlockingRead(t, dst, peek, trunc, senderRequested) if err != syserr.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { return } @@ -936,25 +961,25 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags defer s.EventUnregister(&e) for { - n, senderAddr, senderAddrLen, err = s.nonBlockingRead(t, dst, peek, trunc, senderRequested) + n, senderAddr, senderAddrLen, controlMessages, err = s.nonBlockingRead(t, dst, peek, trunc, senderRequested) if err != syserr.ErrWouldBlock { return } if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { if err == syserror.ETIMEDOUT { - return 0, nil, 0, unix.ControlMessages{}, syserr.ErrTryAgain + return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } - return 0, nil, 0, unix.ControlMessages{}, syserr.FromError(err) + return 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) } } } // SendMsg implements the linux syscall sendmsg(2) for sockets backed by // tcpip.Endpoint. -func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { - // Reject control messages. - if !controlMessages.Empty() { +func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { + // Reject Unix control messages. + if !controlMessages.Unix.Empty() { return 0, syserr.ErrInvalidArgument } diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index defa3db2c..02fad1c60 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -57,6 +57,8 @@ type socketOperations struct { queue waiter.Queue } +var _ = socket.Socket(&socketOperations{}) + func newSocketFile(ctx context.Context, fd int, nonblock bool) (*fs.File, *syserr.Error) { s := &socketOperations{fd: fd} if err := fdnotifier.AddFD(int32(fd), &s.queue); err != nil { @@ -339,14 +341,14 @@ func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt [ } // RecvMsg implements socket.Socket.RecvMsg. -func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, unix.ControlMessages, *syserr.Error) { +func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { // Whitelist flags. // // FIXME: We can't support MSG_ERRQUEUE because it uses ancillary // messages that netstack/tcpip/transport/unix doesn't understand. Kill the // Socket interface's dependence on netstack. if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_PEEK|syscall.MSG_TRUNC) != 0 { - return 0, nil, 0, unix.ControlMessages{}, syserr.ErrInvalidArgument + return 0, nil, 0, socket.ControlMessages{}, syserr.ErrInvalidArgument } var senderAddr []byte @@ -411,11 +413,11 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags } } - return int(n), senderAddr, uint32(len(senderAddr)), unix.ControlMessages{}, syserr.FromError(err) + return int(n), senderAddr, uint32(len(senderAddr)), socket.ControlMessages{}, syserr.FromError(err) } // SendMsg implements socket.Socket.SendMsg. -func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { +func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { // Whitelist flags. if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 { return 0, syserr.ErrInvalidArgument diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index 2d0e59ceb..0b8f528d0 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -305,7 +305,7 @@ func (s *Socket) GetPeerName(t *kernel.Task) (interface{}, uint32, *syserr.Error } // RecvMsg implements socket.Socket.RecvMsg. -func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, unix.ControlMessages, *syserr.Error) { +func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { from := linux.SockAddrNetlink{ Family: linux.AF_NETLINK, PortID: 0, @@ -323,7 +323,7 @@ func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, have if trunc { n = int64(r.MsgSize) } - return int(n), from, fromLen, unix.ControlMessages{}, syserr.FromError(err) + return int(n), from, fromLen, socket.ControlMessages{}, syserr.FromError(err) } // We'll have to block. Register for notification and keep trying to @@ -337,14 +337,14 @@ func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, have if trunc { n = int64(r.MsgSize) } - return int(n), from, fromLen, unix.ControlMessages{}, syserr.FromError(err) + return int(n), from, fromLen, socket.ControlMessages{}, syserr.FromError(err) } if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { if err == syserror.ETIMEDOUT { - return 0, nil, 0, unix.ControlMessages{}, syserr.ErrTryAgain + return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } - return 0, nil, 0, unix.ControlMessages{}, syserr.FromError(err) + return 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) } } } @@ -459,7 +459,7 @@ func (s *Socket) processMessages(ctx context.Context, buf []byte) *syserr.Error } // sendMsg is the core of message send, used for SendMsg and Write. -func (s *Socket) sendMsg(ctx context.Context, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { +func (s *Socket) sendMsg(ctx context.Context, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { dstPort := int32(0) if len(to) != 0 { @@ -506,12 +506,12 @@ func (s *Socket) sendMsg(ctx context.Context, src usermem.IOSequence, to []byte, } // SendMsg implements socket.Socket.SendMsg. -func (s *Socket) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { +func (s *Socket) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { return s.sendMsg(t, src, to, flags, controlMessages) } // Write implements fs.FileOperations.Write. func (s *Socket) Write(ctx context.Context, _ *fs.File, src usermem.IOSequence, _ int64) (int64, error) { - n, err := s.sendMsg(ctx, src, nil, 0, unix.ControlMessages{}) + n, err := s.sendMsg(ctx, src, nil, 0, socket.ControlMessages{}) return int64(n), err.ToError() } diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 574d99ba5..15047df01 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -402,7 +402,7 @@ func rpcRecvMsg(t *kernel.Task, req *pb.SyscallRequest_Recvmsg) (*pb.RecvmsgResp } // RecvMsg implements socket.Socket.RecvMsg. -func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, unix.ControlMessages, *syserr.Error) { +func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { req := &pb.SyscallRequest_Recvmsg{&pb.RecvmsgRequest{ Fd: s.fd, Length: uint32(dst.NumBytes()), @@ -414,10 +414,10 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags res, err := rpcRecvMsg(t, req) if err == nil { n, e := dst.CopyOut(t, res.Data) - return int(n), res.Address.GetAddress(), res.Address.GetLength(), unix.ControlMessages{}, syserr.FromError(e) + return int(n), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } if err != syserr.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { - return 0, nil, 0, unix.ControlMessages{}, err + return 0, nil, 0, socket.ControlMessages{}, err } // We'll have to block. Register for notifications and keep trying to @@ -430,17 +430,17 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags res, err := rpcRecvMsg(t, req) if err == nil { n, e := dst.CopyOut(t, res.Data) - return int(n), res.Address.GetAddress(), res.Address.GetLength(), unix.ControlMessages{}, syserr.FromError(e) + return int(n), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } if err != syserr.ErrWouldBlock { - return 0, nil, 0, unix.ControlMessages{}, err + return 0, nil, 0, socket.ControlMessages{}, err } if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { if err == syserror.ETIMEDOUT { - return 0, nil, 0, unix.ControlMessages{}, syserr.ErrTryAgain + return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } - return 0, nil, 0, unix.ControlMessages{}, syserr.FromError(err) + return 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) } } } @@ -459,14 +459,14 @@ func rpcSendMsg(t *kernel.Task, req *pb.SyscallRequest_Sendmsg) (uint32, *syserr } // SendMsg implements socket.Socket.SendMsg. -func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { +func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { // Whitelist flags. if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 { return 0, syserr.ErrInvalidArgument } - // Reject control messages. - if !controlMessages.Empty() { + // Reject Unix control messages. + if !controlMessages.Unix.Empty() { return 0, syserr.ErrInvalidArgument } diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index be3026bfa..bd4858a34 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -31,9 +31,17 @@ import ( ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" + "gvisor.googlesource.com/gvisor/pkg/tcpip" "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) +// ControlMessages represents the union of unix control messages and tcpip +// control messages. +type ControlMessages struct { + Unix unix.ControlMessages + IP tcpip.ControlMessages +} + // Socket is the interface containing socket syscalls used by the syscall layer // to redirect them to the appropriate implementation. type Socket interface { @@ -78,11 +86,11 @@ type Socket interface { // // senderAddrLen is the address length to be returned to the application, // not necessarily the actual length of the address. - RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages unix.ControlMessages, err *syserr.Error) + RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages ControlMessages, err *syserr.Error) // SendMsg implements the sendmsg(2) linux syscall. SendMsg does not take // ownership of the ControlMessage on error. - SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (n int, err *syserr.Error) + SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages ControlMessages) (n int, err *syserr.Error) // SetRecvTimeout sets the timeout (in ns) for recv operations. Zero means // no timeout. diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index a4b414851..f83156c8e 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -358,10 +358,10 @@ func (s *SocketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IO // SendMsg implements the linux syscall sendmsg(2) for unix sockets backed by // a unix.Endpoint. -func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { +func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { w := EndpointWriter{ Endpoint: s.ep, - Control: controlMessages, + Control: controlMessages.Unix, To: nil, } if len(to) > 0 { @@ -452,7 +452,7 @@ func (s *SocketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOS // RecvMsg implements the linux syscall recvmsg(2) for sockets backed by // a unix.Endpoint. -func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages unix.ControlMessages, err *syserr.Error) { +func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) { trunc := flags&linux.MSG_TRUNC != 0 peek := flags&linux.MSG_PEEK != 0 @@ -490,7 +490,7 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags if trunc { n = int64(r.MsgSize) } - return int(n), from, fromLen, r.Control, syserr.FromError(err) + return int(n), from, fromLen, socket.ControlMessages{Unix: r.Control}, syserr.FromError(err) } // We'll have to block. Register for notification and keep trying to @@ -509,14 +509,14 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags if trunc { n = int64(r.MsgSize) } - return int(n), from, fromLen, r.Control, syserr.FromError(err) + return int(n), from, fromLen, socket.ControlMessages{Unix: r.Control}, syserr.FromError(err) } if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { if err == syserror.ETIMEDOUT { - return 0, nil, 0, unix.ControlMessages{}, syserr.ErrTryAgain + return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } - return 0, nil, 0, unix.ControlMessages{}, syserr.FromError(err) + return 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) } } } diff --git a/pkg/sentry/strace/socket.go b/pkg/sentry/strace/socket.go index 48c072e96..1a2e8573e 100644 --- a/pkg/sentry/strace/socket.go +++ b/pkg/sentry/strace/socket.go @@ -440,6 +440,7 @@ var SocketProtocol = map[int32]abi.ValueSet{ var controlMessageType = map[int32]string{ linux.SCM_RIGHTS: "SCM_RIGHTS", linux.SCM_CREDENTIALS: "SCM_CREDENTIALS", + linux.SO_TIMESTAMP: "SO_TIMESTAMP", } func cmsghdr(t *kernel.Task, addr usermem.Addr, length uint64, maxBytes uint64) string { @@ -477,7 +478,7 @@ func cmsghdr(t *kernel.Task, addr usermem.Addr, length uint64, maxBytes uint64) typ = fmt.Sprint(h.Type) } - if h.Length > uint64(len(buf)-i) { + if h.Length > uint64(len(buf)-i+linux.SizeOfControlMessageHeader) { strs = append(strs, fmt.Sprintf( "{level=%s, type=%s, length=%d, content extends beyond buffer}", level, @@ -546,6 +547,32 @@ func cmsghdr(t *kernel.Task, addr usermem.Addr, length uint64, maxBytes uint64) i += control.AlignUp(length, width) + case linux.SO_TIMESTAMP: + if length < linux.SizeOfTimeval { + strs = append(strs, fmt.Sprintf( + "{level=%s, type=%s, length=%d, content too short}", + level, + typ, + h.Length, + )) + i += control.AlignUp(length, width) + break + } + + var tv linux.Timeval + binary.Unmarshal(buf[i:i+linux.SizeOfTimeval], usermem.ByteOrder, &tv) + + strs = append(strs, fmt.Sprintf( + "{level=%s, type=%s, length=%d, Sec: %d, Usec: %d}", + level, + typ, + h.Length, + tv.Sec, + tv.Usec, + )) + + i += control.AlignUp(length, width) + default: panic("unreachable") } diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index 70c618398..6258a1539 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -731,10 +731,11 @@ func recvSingleMsg(t *kernel.Task, s socket.Socket, msgPtr usermem.Addr, flags i // Fast path when no control message nor name buffers are provided. if msg.ControlLen == 0 && msg.NameLen == 0 { - n, _, _, _, err := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, false, 0) + n, _, _, cms, err := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, false, 0) if err != nil { return 0, syserror.ConvertIntr(err.ToError(), kernel.ERESTARTSYS) } + cms.Unix.Release() return uintptr(n), nil } @@ -745,17 +746,21 @@ func recvSingleMsg(t *kernel.Task, s socket.Socket, msgPtr usermem.Addr, flags i if e != nil { return 0, syserror.ConvertIntr(e.ToError(), kernel.ERESTARTSYS) } - defer cms.Release() + defer cms.Unix.Release() controlData := make([]byte, 0, msg.ControlLen) if cr, ok := s.(unix.Credentialer); ok && cr.Passcred() { - creds, _ := cms.Credentials.(control.SCMCredentials) + creds, _ := cms.Unix.Credentials.(control.SCMCredentials) controlData = control.PackCredentials(t, creds, controlData) } - if cms.Rights != nil { - controlData = control.PackRights(t, cms.Rights.(control.SCMRights), flags&linux.MSG_CMSG_CLOEXEC != 0, controlData) + if cms.IP.HasTimestamp { + controlData = control.PackTimestamp(t, cms.IP.Timestamp, controlData) + } + + if cms.Unix.Rights != nil { + controlData = control.PackRights(t, cms.Unix.Rights.(control.SCMRights), flags&linux.MSG_CMSG_CLOEXEC != 0, controlData) } // Copy the address to the caller. @@ -823,7 +828,7 @@ func recvFrom(t *kernel.Task, fd kdefs.FD, bufPtr usermem.Addr, bufLen uint64, f } n, sender, senderLen, cm, e := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, nameLenPtr != 0, 0) - cm.Release() + cm.Unix.Release() if e != nil { return 0, syserror.ConvertIntr(e.ToError(), kernel.ERESTARTSYS) } @@ -997,7 +1002,7 @@ func sendSingleMsg(t *kernel.Task, s socket.Socket, file *fs.File, msgPtr userme } // Call the syscall implementation. - n, e := s.SendMsg(t, src, to, int(flags), controlMessages) + n, e := s.SendMsg(t, src, to, int(flags), socket.ControlMessages{Unix: controlMessages}) err = handleIOError(t, n != 0, e.ToError(), kernel.ERESTARTSYS, "sendmsg", file) if err != nil { controlMessages.Release() @@ -1048,7 +1053,7 @@ func sendTo(t *kernel.Task, fd kdefs.FD, bufPtr usermem.Addr, bufLen uint64, fla } // Call the syscall implementation. - n, e := s.SendMsg(t, src, to, int(flags), control.New(t, s, nil)) + n, e := s.SendMsg(t, src, to, int(flags), socket.ControlMessages{Unix: control.New(t, s, nil)}) return uintptr(n), handleIOError(t, n != 0, e.ToError(), kernel.ERESTARTSYS, "sendto", file) } diff --git a/pkg/tcpip/adapters/gonet/gonet.go b/pkg/tcpip/adapters/gonet/gonet.go index 96a2d670d..5aa6b1aa2 100644 --- a/pkg/tcpip/adapters/gonet/gonet.go +++ b/pkg/tcpip/adapters/gonet/gonet.go @@ -268,7 +268,7 @@ type opErrorer interface { // commonRead implements the common logic between net.Conn.Read and // net.PacketConn.ReadFrom. func commonRead(ep tcpip.Endpoint, wq *waiter.Queue, deadline <-chan struct{}, addr *tcpip.FullAddress, errorer opErrorer) ([]byte, error) { - read, err := ep.Read(addr) + read, _, err := ep.Read(addr) if err == tcpip.ErrWouldBlock { // Create wait queue entry that notifies a channel. @@ -276,7 +276,7 @@ func commonRead(ep tcpip.Endpoint, wq *waiter.Queue, deadline <-chan struct{}, a wq.EventRegister(&waitEntry, waiter.EventIn) defer wq.EventUnregister(&waitEntry) for { - read, err = ep.Read(addr) + read, _, err = ep.Read(addr) if err != tcpip.ErrWouldBlock { break } diff --git a/pkg/tcpip/adapters/gonet/gonet_test.go b/pkg/tcpip/adapters/gonet/gonet_test.go index 2f86469eb..e3d0c6c84 100644 --- a/pkg/tcpip/adapters/gonet/gonet_test.go +++ b/pkg/tcpip/adapters/gonet/gonet_test.go @@ -47,7 +47,7 @@ func TestTimeouts(t *testing.T) { func newLoopbackStack() (*stack.Stack, *tcpip.Error) { // Create the stack and add a NIC. - s := stack.New([]string{ipv4.ProtocolName, ipv6.ProtocolName}, []string{tcp.ProtocolName, udp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName, ipv6.ProtocolName}, []string{tcp.ProtocolName, udp.ProtocolName}) if err := s.CreateNIC(NICID, loopback.New()); err != nil { return nil, err diff --git a/pkg/tcpip/network/arp/arp_test.go b/pkg/tcpip/network/arp/arp_test.go index 91ffdce4b..47b10e64e 100644 --- a/pkg/tcpip/network/arp/arp_test.go +++ b/pkg/tcpip/network/arp/arp_test.go @@ -32,7 +32,7 @@ type testContext struct { } func newTestContext(t *testing.T) *testContext { - s := stack.New([]string{ipv4.ProtocolName, arp.ProtocolName}, []string{ipv4.PingProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName, arp.ProtocolName}, []string{ipv4.PingProtocolName}) const defaultMTU = 65536 id, linkEP := channel.New(256, defaultMTU, stackLinkAddr) diff --git a/pkg/tcpip/network/ipv4/icmp_test.go b/pkg/tcpip/network/ipv4/icmp_test.go index 378fba74b..c55aa1835 100644 --- a/pkg/tcpip/network/ipv4/icmp_test.go +++ b/pkg/tcpip/network/ipv4/icmp_test.go @@ -26,7 +26,7 @@ type testContext struct { } func newTestContext(t *testing.T) *testContext { - s := stack.New([]string{ipv4.ProtocolName}, []string{ipv4.PingProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName}, []string{ipv4.PingProtocolName}) const defaultMTU = 65536 id, linkEP := channel.New(256, defaultMTU, "") diff --git a/pkg/tcpip/sample/tun_tcp_connect/main.go b/pkg/tcpip/sample/tun_tcp_connect/main.go index 332929c85..ef5c7ec60 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/main.go +++ b/pkg/tcpip/sample/tun_tcp_connect/main.go @@ -113,7 +113,7 @@ func main() { // Create the stack with ipv4 and tcp protocols, then add a tun-based // NIC and ipv4 address. - s := stack.New([]string{ipv4.ProtocolName}, []string{tcp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName}, []string{tcp.ProtocolName}) mtu, err := rawfile.GetMTU(tunName) if err != nil { @@ -183,7 +183,7 @@ func main() { // connection from its side. wq.EventRegister(&waitEntry, waiter.EventIn) for { - v, err := ep.Read(nil) + v, _, err := ep.Read(nil) if err != nil { if err == tcpip.ErrClosedForReceive { break diff --git a/pkg/tcpip/sample/tun_tcp_echo/main.go b/pkg/tcpip/sample/tun_tcp_echo/main.go index 10cd701af..8c166f643 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/main.go +++ b/pkg/tcpip/sample/tun_tcp_echo/main.go @@ -42,7 +42,7 @@ func echo(wq *waiter.Queue, ep tcpip.Endpoint) { defer wq.EventUnregister(&waitEntry) for { - v, err := ep.Read(nil) + v, _, err := ep.Read(nil) if err != nil { if err == tcpip.ErrWouldBlock { <-notifyCh @@ -99,7 +99,7 @@ func main() { // Create the stack with ip and tcp protocols, then add a tun-based // NIC and address. - s := stack.New([]string{ipv4.ProtocolName, ipv6.ProtocolName, arp.ProtocolName}, []string{tcp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName, ipv6.ProtocolName, arp.ProtocolName}, []string{tcp.ProtocolName}) mtu, err := rawfile.GetMTU(tunName) if err != nil { diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index 558ecdb72..b480bf812 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -270,6 +270,9 @@ type Stack struct { // If not nil, then any new endpoints will have this probe function // invoked everytime they receive a TCP segment. tcpProbeFunc TCPProbeFunc + + // clock is used to generate user-visible times. + clock tcpip.Clock } // New allocates a new networking stack with only the requested networking and @@ -279,7 +282,7 @@ type Stack struct { // SetNetworkProtocolOption/SetTransportProtocolOption methods provided by the // stack. Please refer to individual protocol implementations as to what options // are supported. -func New(network []string, transport []string) *Stack { +func New(clock tcpip.Clock, network []string, transport []string) *Stack { s := &Stack{ transportProtocols: make(map[tcpip.TransportProtocolNumber]*transportProtocolState), networkProtocols: make(map[tcpip.NetworkProtocolNumber]NetworkProtocol), @@ -287,6 +290,7 @@ func New(network []string, transport []string) *Stack { nics: make(map[tcpip.NICID]*NIC), linkAddrCache: newLinkAddrCache(ageLimit, resolutionTimeout, resolutionAttempts), PortManager: ports.NewPortManager(), + clock: clock, } // Add specified network protocols. @@ -388,6 +392,11 @@ func (s *Stack) SetTransportProtocolHandler(p tcpip.TransportProtocolNumber, h f } } +// NowNanoseconds implements tcpip.Clock.NowNanoseconds. +func (s *Stack) NowNanoseconds() int64 { + return s.clock.NowNanoseconds() +} + // Stats returns a snapshot of the current stats. // // NOTE: The underlying stats are updated using atomic instructions as a result @@ -474,6 +483,12 @@ func (s *Stack) CreateDisabledNIC(id tcpip.NICID, linkEP tcpip.LinkEndpointID) * return s.createNIC(id, "", linkEP, false) } +// CreateDisabledNamedNIC is a combination of CreateNamedNIC and +// CreateDisabledNIC. +func (s *Stack) CreateDisabledNamedNIC(id tcpip.NICID, name string, linkEP tcpip.LinkEndpointID) *tcpip.Error { + return s.createNIC(id, name, linkEP, false) +} + // EnableNIC enables the given NIC so that the link-layer endpoint can start // delivering packets to it. func (s *Stack) EnableNIC(id tcpip.NICID) *tcpip.Error { diff --git a/pkg/tcpip/stack/stack_test.go b/pkg/tcpip/stack/stack_test.go index b416065d7..ea7dccdc2 100644 --- a/pkg/tcpip/stack/stack_test.go +++ b/pkg/tcpip/stack/stack_test.go @@ -176,7 +176,7 @@ func TestNetworkReceive(t *testing.T) { // Create a stack with the fake network protocol, one nic, and two // addresses attached to it: 1 & 2. id, linkEP := channel.New(10, defaultMTU, "") - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) if err := s.CreateNIC(1, id); err != nil { t.Fatalf("CreateNIC failed: %v", err) } @@ -270,7 +270,7 @@ func TestNetworkSend(t *testing.T) { // address: 1. The route table sends all packets through the only // existing nic. id, linkEP := channel.New(10, defaultMTU, "") - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) if err := s.CreateNIC(1, id); err != nil { t.Fatalf("NewNIC failed: %v", err) } @@ -292,7 +292,7 @@ func TestNetworkSendMultiRoute(t *testing.T) { // Create a stack with the fake network protocol, two nics, and two // addresses per nic, the first nic has odd address, the second one has // even addresses. - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) id1, linkEP1 := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, id1); err != nil { @@ -371,7 +371,7 @@ func TestRoutes(t *testing.T) { // Create a stack with the fake network protocol, two nics, and two // addresses per nic, the first nic has odd address, the second one has // even addresses. - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) id1, _ := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, id1); err != nil { @@ -435,7 +435,7 @@ func TestRoutes(t *testing.T) { } func TestAddressRemoval(t *testing.T) { - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) id, linkEP := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, id); err != nil { @@ -479,7 +479,7 @@ func TestAddressRemoval(t *testing.T) { } func TestDelayedRemovalDueToRoute(t *testing.T) { - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) id, linkEP := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, id); err != nil { @@ -547,7 +547,7 @@ func TestDelayedRemovalDueToRoute(t *testing.T) { } func TestPromiscuousMode(t *testing.T) { - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) id, linkEP := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, id); err != nil { @@ -607,7 +607,7 @@ func TestAddressSpoofing(t *testing.T) { srcAddr := tcpip.Address("\x01") dstAddr := tcpip.Address("\x02") - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) id, _ := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, id); err != nil { @@ -648,7 +648,7 @@ func TestAddressSpoofing(t *testing.T) { // Set the subnet, then check that packet is delivered. func TestSubnetAcceptsMatchingPacket(t *testing.T) { - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) id, linkEP := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, id); err != nil { @@ -682,7 +682,7 @@ func TestSubnetAcceptsMatchingPacket(t *testing.T) { // Set destination outside the subnet, then check it doesn't get delivered. func TestSubnetRejectsNonmatchingPacket(t *testing.T) { - s := stack.New([]string{"fakeNet"}, nil) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, nil) id, linkEP := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, id); err != nil { @@ -714,7 +714,7 @@ func TestSubnetRejectsNonmatchingPacket(t *testing.T) { } func TestNetworkOptions(t *testing.T) { - s := stack.New([]string{"fakeNet"}, []string{}) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, []string{}) // Try an unsupported network protocol. if err := s.SetNetworkProtocolOption(tcpip.NetworkProtocolNumber(99999), fakeNetGoodOption(false)); err != tcpip.ErrUnknownProtocol { diff --git a/pkg/tcpip/stack/transport_test.go b/pkg/tcpip/stack/transport_test.go index 7e072e96e..b870ab375 100644 --- a/pkg/tcpip/stack/transport_test.go +++ b/pkg/tcpip/stack/transport_test.go @@ -46,8 +46,8 @@ func (*fakeTransportEndpoint) Readiness(mask waiter.EventMask) waiter.EventMask return mask } -func (*fakeTransportEndpoint) Read(*tcpip.FullAddress) (buffer.View, *tcpip.Error) { - return buffer.View{}, nil +func (*fakeTransportEndpoint) Read(*tcpip.FullAddress) (buffer.View, tcpip.ControlMessages, *tcpip.Error) { + return buffer.View{}, tcpip.ControlMessages{}, nil } func (f *fakeTransportEndpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) (uintptr, *tcpip.Error) { @@ -67,8 +67,8 @@ func (f *fakeTransportEndpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) return uintptr(len(v)), nil } -func (f *fakeTransportEndpoint) Peek([][]byte) (uintptr, *tcpip.Error) { - return 0, nil +func (f *fakeTransportEndpoint) Peek([][]byte) (uintptr, tcpip.ControlMessages, *tcpip.Error) { + return 0, tcpip.ControlMessages{}, nil } // SetSockOpt sets a socket option. Currently not supported. @@ -210,7 +210,7 @@ func (f *fakeTransportProtocol) Option(option interface{}) *tcpip.Error { func TestTransportReceive(t *testing.T) { id, linkEP := channel.New(10, defaultMTU, "") - s := stack.New([]string{"fakeNet"}, []string{"fakeTrans"}) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, []string{"fakeTrans"}) if err := s.CreateNIC(1, id); err != nil { t.Fatalf("CreateNIC failed: %v", err) } @@ -270,7 +270,7 @@ func TestTransportReceive(t *testing.T) { func TestTransportControlReceive(t *testing.T) { id, linkEP := channel.New(10, defaultMTU, "") - s := stack.New([]string{"fakeNet"}, []string{"fakeTrans"}) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, []string{"fakeTrans"}) if err := s.CreateNIC(1, id); err != nil { t.Fatalf("CreateNIC failed: %v", err) } @@ -336,7 +336,7 @@ func TestTransportControlReceive(t *testing.T) { func TestTransportSend(t *testing.T) { id, _ := channel.New(10, defaultMTU, "") - s := stack.New([]string{"fakeNet"}, []string{"fakeTrans"}) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, []string{"fakeTrans"}) if err := s.CreateNIC(1, id); err != nil { t.Fatalf("CreateNIC failed: %v", err) } @@ -373,7 +373,7 @@ func TestTransportSend(t *testing.T) { } func TestTransportOptions(t *testing.T) { - s := stack.New([]string{"fakeNet"}, []string{"fakeTrans"}) + s := stack.New(&tcpip.StdClock{}, []string{"fakeNet"}, []string{"fakeTrans"}) // Try an unsupported transport protocol. if err := s.SetTransportProtocolOption(tcpip.TransportProtocolNumber(99999), fakeTransportGoodOption(false)); err != tcpip.ErrUnknownProtocol { diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go index f3a94f353..f9df1d989 100644 --- a/pkg/tcpip/tcpip.go +++ b/pkg/tcpip/tcpip.go @@ -23,6 +23,7 @@ import ( "fmt" "strconv" "strings" + "time" "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" "gvisor.googlesource.com/gvisor/pkg/waiter" @@ -80,6 +81,24 @@ var ( errSubnetAddressMasked = errors.New("subnet address has bits set outside the mask") ) +// A Clock provides the current time. +// +// Times returned by a Clock should always be used for application-visible +// time, but never for netstack internal timekeeping. +type Clock interface { + // NowNanoseconds returns the current real time as a number of + // nanoseconds since some epoch. + NowNanoseconds() int64 +} + +// StdClock implements Clock with the time package. +type StdClock struct{} + +// NowNanoseconds implements Clock.NowNanoseconds. +func (*StdClock) NowNanoseconds() int64 { + return time.Now().UnixNano() +} + // Address is a byte slice cast as a string that represents the address of a // network node. Or, in the case of unix endpoints, it may represent a path. type Address string @@ -210,6 +229,16 @@ func (s SlicePayload) Size() int { return len(s) } +// A ControlMessages contains socket control messages for IP sockets. +type ControlMessages struct { + // HasTimestamp indicates whether Timestamp is valid/set. + HasTimestamp bool + + // Timestamp is the time (in ns) that the last packed used to create + // the read data was received. + Timestamp int64 +} + // Endpoint is the interface implemented by transport protocols (e.g., tcp, udp) // that exposes functionality like read, write, connect, etc. to users of the // networking stack. @@ -219,9 +248,13 @@ type Endpoint interface { Close() // Read reads data from the endpoint and optionally returns the sender. - // This method does not block if there is no data pending. - // It will also either return an error or data, never both. - Read(*FullAddress) (buffer.View, *Error) + // + // This method does not block if there is no data pending. It will also + // either return an error or data, never both. + // + // A timestamp (in ns) is optionally returned. A zero value indicates + // that no timestamp was available. + Read(*FullAddress) (buffer.View, ControlMessages, *Error) // Write writes data to the endpoint's peer. This method does not block if // the data cannot be written. @@ -238,7 +271,10 @@ type Endpoint interface { // Peek reads data without consuming it from the endpoint. // // This method does not block if there is no data pending. - Peek([][]byte) (uintptr, *Error) + // + // A timestamp (in ns) is optionally returned. A zero value indicates + // that no timestamp was available. + Peek([][]byte) (uintptr, ControlMessages, *Error) // Connect connects the endpoint to its peer. Specifying a NIC is // optional. @@ -347,6 +383,10 @@ type ReuseAddressOption int // Only supported on Unix sockets. type PasscredOption int +// TimestampOption is used by SetSockOpt/GetSockOpt to specify whether +// SO_TIMESTAMP socket control messages are enabled. +type TimestampOption int + // TCPInfoOption is used by GetSockOpt to expose TCP statistics. // // TODO: Add and populate stat fields. diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go index 5d62589d8..d84171b0c 100644 --- a/pkg/tcpip/transport/tcp/endpoint.go +++ b/pkg/tcpip/transport/tcp/endpoint.go @@ -374,7 +374,7 @@ func (e *endpoint) cleanup() { } // Read reads data from the endpoint. -func (e *endpoint) Read(*tcpip.FullAddress) (buffer.View, *tcpip.Error) { +func (e *endpoint) Read(*tcpip.FullAddress) (buffer.View, tcpip.ControlMessages, *tcpip.Error) { e.mu.RLock() // The endpoint can be read if it's connected, or if it's already closed // but has some pending unread data. Also note that a RST being received @@ -383,9 +383,9 @@ func (e *endpoint) Read(*tcpip.FullAddress) (buffer.View, *tcpip.Error) { if s := e.state; s != stateConnected && s != stateClosed && e.rcvBufUsed == 0 { e.mu.RUnlock() if s == stateError { - return buffer.View{}, e.hardError + return buffer.View{}, tcpip.ControlMessages{}, e.hardError } - return buffer.View{}, tcpip.ErrInvalidEndpointState + return buffer.View{}, tcpip.ControlMessages{}, tcpip.ErrInvalidEndpointState } e.rcvListMu.Lock() @@ -394,7 +394,7 @@ func (e *endpoint) Read(*tcpip.FullAddress) (buffer.View, *tcpip.Error) { e.mu.RUnlock() - return v, err + return v, tcpip.ControlMessages{}, err } func (e *endpoint) readLocked() (buffer.View, *tcpip.Error) { @@ -498,7 +498,7 @@ func (e *endpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) (uintptr, *tc // Peek reads data without consuming it from the endpoint. // // This method does not block if there is no data pending. -func (e *endpoint) Peek(vec [][]byte) (uintptr, *tcpip.Error) { +func (e *endpoint) Peek(vec [][]byte) (uintptr, tcpip.ControlMessages, *tcpip.Error) { e.mu.RLock() defer e.mu.RUnlock() @@ -506,9 +506,9 @@ func (e *endpoint) Peek(vec [][]byte) (uintptr, *tcpip.Error) { // but has some pending unread data. if s := e.state; s != stateConnected && s != stateClosed { if s == stateError { - return 0, e.hardError + return 0, tcpip.ControlMessages{}, e.hardError } - return 0, tcpip.ErrInvalidEndpointState + return 0, tcpip.ControlMessages{}, tcpip.ErrInvalidEndpointState } e.rcvListMu.Lock() @@ -516,9 +516,9 @@ func (e *endpoint) Peek(vec [][]byte) (uintptr, *tcpip.Error) { if e.rcvBufUsed == 0 { if e.rcvClosed || e.state != stateConnected { - return 0, tcpip.ErrClosedForReceive + return 0, tcpip.ControlMessages{}, tcpip.ErrClosedForReceive } - return 0, tcpip.ErrWouldBlock + return 0, tcpip.ControlMessages{}, tcpip.ErrWouldBlock } // Make a copy of vec so we can modify the slide headers. @@ -534,7 +534,7 @@ func (e *endpoint) Peek(vec [][]byte) (uintptr, *tcpip.Error) { for len(v) > 0 { if len(vec) == 0 { - return num, nil + return num, tcpip.ControlMessages{}, nil } if len(vec[0]) == 0 { vec = vec[1:] @@ -549,7 +549,7 @@ func (e *endpoint) Peek(vec [][]byte) (uintptr, *tcpip.Error) { } } - return num, nil + return num, tcpip.ControlMessages{}, nil } // zeroReceiveWindow checks if the receive window to be announced now would be diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go index 118d861ba..3c21a1ec3 100644 --- a/pkg/tcpip/transport/tcp/tcp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_test.go @@ -147,7 +147,7 @@ func TestSimpleReceive(t *testing.T) { c.WQ.EventRegister(&we, waiter.EventIn) defer c.WQ.EventUnregister(&we) - if _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { + if _, _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { t.Fatalf("Unexpected error from Read: %v", err) } @@ -169,7 +169,7 @@ func TestSimpleReceive(t *testing.T) { } // Receive data. - v, err := c.EP.Read(nil) + v, _, err := c.EP.Read(nil) if err != nil { t.Fatalf("Unexpected error from Read: %v", err) } @@ -199,7 +199,7 @@ func TestOutOfOrderReceive(t *testing.T) { c.WQ.EventRegister(&we, waiter.EventIn) defer c.WQ.EventUnregister(&we) - if _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { + if _, _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { t.Fatalf("Unexpected error from Read: %v", err) } @@ -226,7 +226,7 @@ func TestOutOfOrderReceive(t *testing.T) { // Wait 200ms and check that no data has been received. time.Sleep(200 * time.Millisecond) - if _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { + if _, _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { t.Fatalf("Unexpected error from Read: %v", err) } @@ -243,7 +243,7 @@ func TestOutOfOrderReceive(t *testing.T) { // Receive data. read := make([]byte, 0, 6) for len(read) < len(data) { - v, err := c.EP.Read(nil) + v, _, err := c.EP.Read(nil) if err != nil { if err == tcpip.ErrWouldBlock { // Wait for receive to be notified. @@ -284,7 +284,7 @@ func TestOutOfOrderFlood(t *testing.T) { opt := tcpip.ReceiveBufferSizeOption(10) c.CreateConnected(789, 30000, &opt) - if _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { + if _, _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { t.Fatalf("Unexpected error from Read: %v", err) } @@ -361,7 +361,7 @@ func TestRstOnCloseWithUnreadData(t *testing.T) { c.WQ.EventRegister(&we, waiter.EventIn) defer c.WQ.EventUnregister(&we) - if _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { + if _, _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { t.Fatalf("Unexpected error from Read: %v", err) } @@ -414,7 +414,7 @@ func TestFullWindowReceive(t *testing.T) { c.WQ.EventRegister(&we, waiter.EventIn) defer c.WQ.EventUnregister(&we) - _, err := c.EP.Read(nil) + _, _, err := c.EP.Read(nil) if err != tcpip.ErrWouldBlock { t.Fatalf("Unexpected error from Read: %v", err) } @@ -449,7 +449,7 @@ func TestFullWindowReceive(t *testing.T) { ) // Receive data and check it. - v, err := c.EP.Read(nil) + v, _, err := c.EP.Read(nil) if err != nil { t.Fatalf("Unexpected error from Read: %v", err) } @@ -487,7 +487,7 @@ func TestNoWindowShrinking(t *testing.T) { c.WQ.EventRegister(&we, waiter.EventIn) defer c.WQ.EventUnregister(&we) - _, err := c.EP.Read(nil) + _, _, err := c.EP.Read(nil) if err != tcpip.ErrWouldBlock { t.Fatalf("Unexpected error from Read: %v", err) } @@ -551,7 +551,7 @@ func TestNoWindowShrinking(t *testing.T) { // Receive data and check it. read := make([]byte, 0, 10) for len(read) < len(data) { - v, err := c.EP.Read(nil) + v, _, err := c.EP.Read(nil) if err != nil { t.Fatalf("Unexpected error from Read: %v", err) } @@ -954,7 +954,7 @@ func TestZeroScaledWindowReceive(t *testing.T) { } // Read some data. An ack should be sent in response to that. - v, err := c.EP.Read(nil) + v, _, err := c.EP.Read(nil) if err != nil { t.Fatalf("Unexpected error from Read: %v", err) } @@ -1337,7 +1337,7 @@ func TestReceiveOnResetConnection(t *testing.T) { loop: for { - switch _, err := c.EP.Read(nil); err { + switch _, _, err := c.EP.Read(nil); err { case nil: t.Fatalf("Unexpected success.") case tcpip.ErrWouldBlock: @@ -2293,7 +2293,7 @@ func TestReadAfterClosedState(t *testing.T) { c.WQ.EventRegister(&we, waiter.EventIn) defer c.WQ.EventUnregister(&we) - if _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { + if _, _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { t.Fatalf("Unexpected error from Read: %v", err) } @@ -2345,7 +2345,7 @@ func TestReadAfterClosedState(t *testing.T) { // Check that peek works. peekBuf := make([]byte, 10) - n, err := c.EP.Peek([][]byte{peekBuf}) + n, _, err := c.EP.Peek([][]byte{peekBuf}) if err != nil { t.Fatalf("Unexpected error from Peek: %v", err) } @@ -2356,7 +2356,7 @@ func TestReadAfterClosedState(t *testing.T) { } // Receive data. - v, err := c.EP.Read(nil) + v, _, err := c.EP.Read(nil) if err != nil { t.Fatalf("Unexpected error from Read: %v", err) } @@ -2367,11 +2367,11 @@ func TestReadAfterClosedState(t *testing.T) { // Now that we drained the queue, check that functions fail with the // right error code. - if _, err := c.EP.Read(nil); err != tcpip.ErrClosedForReceive { + if _, _, err := c.EP.Read(nil); err != tcpip.ErrClosedForReceive { t.Fatalf("Unexpected return from Read: got %v, want %v", err, tcpip.ErrClosedForReceive) } - if _, err := c.EP.Peek([][]byte{peekBuf}); err != tcpip.ErrClosedForReceive { + if _, _, err := c.EP.Peek([][]byte{peekBuf}); err != tcpip.ErrClosedForReceive { t.Fatalf("Unexpected return from Peek: got %v, want %v", err, tcpip.ErrClosedForReceive) } } @@ -2479,7 +2479,7 @@ func checkSendBufferSize(t *testing.T, ep tcpip.Endpoint, v int) { } func TestDefaultBufferSizes(t *testing.T) { - s := stack.New([]string{ipv4.ProtocolName}, []string{tcp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName}, []string{tcp.ProtocolName}) // Check the default values. ep, err := s.NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &waiter.Queue{}) @@ -2525,7 +2525,7 @@ func TestDefaultBufferSizes(t *testing.T) { } func TestMinMaxBufferSizes(t *testing.T) { - s := stack.New([]string{ipv4.ProtocolName}, []string{tcp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName}, []string{tcp.ProtocolName}) // Check the default values. ep, err := s.NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &waiter.Queue{}) @@ -2575,7 +2575,7 @@ func TestSelfConnect(t *testing.T) { // it checks that if an endpoint binds to say 127.0.0.1:1000 then // connects to 127.0.0.1:1000, then it will be connected to itself, and // is able to send and receive data through the same endpoint. - s := stack.New([]string{ipv4.ProtocolName}, []string{tcp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName}, []string{tcp.ProtocolName}) id := loopback.New() if testing.Verbose() { @@ -2637,13 +2637,13 @@ func TestSelfConnect(t *testing.T) { // Read back what was written. wq.EventUnregister(&waitEntry) wq.EventRegister(&waitEntry, waiter.EventIn) - rd, err := ep.Read(nil) + rd, _, err := ep.Read(nil) if err != nil { if err != tcpip.ErrWouldBlock { t.Fatalf("Read failed: %v", err) } <-notifyCh - rd, err = ep.Read(nil) + rd, _, err = ep.Read(nil) if err != nil { t.Fatalf("Read failed: %v", err) } diff --git a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go index d12081bb7..335262e43 100644 --- a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go @@ -95,7 +95,7 @@ func TestTimeStampEnabledConnect(t *testing.T) { // There should be 5 views to read and each of them should // contain the same data. for i := 0; i < 5; i++ { - got, err := c.EP.Read(nil) + got, _, err := c.EP.Read(nil) if err != nil { t.Fatalf("Unexpected error from Read: %v", err) } @@ -296,7 +296,7 @@ func TestSegmentDropWhenTimestampMissing(t *testing.T) { } // Issue a read and we should data. - got, err := c.EP.Read(nil) + got, _, err := c.EP.Read(nil) if err != nil { t.Fatalf("Unexpected error from Read: %v", err) } diff --git a/pkg/tcpip/transport/tcp/testing/context/context.go b/pkg/tcpip/transport/tcp/testing/context/context.go index 6a402d150..eb928553f 100644 --- a/pkg/tcpip/transport/tcp/testing/context/context.go +++ b/pkg/tcpip/transport/tcp/testing/context/context.go @@ -129,7 +129,7 @@ type Context struct { // New allocates and initializes a test context containing a new // stack and a link-layer endpoint. func New(t *testing.T, mtu uint32) *Context { - s := stack.New([]string{ipv4.ProtocolName, ipv6.ProtocolName}, []string{tcp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName, ipv6.ProtocolName}, []string{tcp.ProtocolName}) // Allow minimum send/receive buffer sizes to be 1 during tests. if err := s.SetTransportProtocolOption(tcp.ProtocolNumber, tcp.SendBufferSizeOption{1, tcp.DefaultBufferSize, tcp.DefaultBufferSize * 10}); err != nil { diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go index 80fa88c4c..f86fc6d5a 100644 --- a/pkg/tcpip/transport/udp/endpoint.go +++ b/pkg/tcpip/transport/udp/endpoint.go @@ -19,6 +19,8 @@ type udpPacket struct { udpPacketEntry senderAddress tcpip.FullAddress data buffer.VectorisedView `state:".(buffer.VectorisedView)"` + timestamp int64 + hasTimestamp bool // views is used as buffer for data when its length is large // enough to store a VectorisedView. views [8]buffer.View `state:"nosave"` @@ -52,6 +54,7 @@ type endpoint struct { rcvBufSizeMax int `state:".(int)"` rcvBufSize int rcvClosed bool + rcvTimestamp bool // The following fields are protected by the mu mutex. mu sync.RWMutex `state:"nosave"` @@ -134,7 +137,7 @@ func (e *endpoint) Close() { // Read reads data from the endpoint. This method does not block if // there is no data pending. -func (e *endpoint) Read(addr *tcpip.FullAddress) (buffer.View, *tcpip.Error) { +func (e *endpoint) Read(addr *tcpip.FullAddress) (buffer.View, tcpip.ControlMessages, *tcpip.Error) { e.rcvMu.Lock() if e.rcvList.Empty() { @@ -143,12 +146,13 @@ func (e *endpoint) Read(addr *tcpip.FullAddress) (buffer.View, *tcpip.Error) { err = tcpip.ErrClosedForReceive } e.rcvMu.Unlock() - return buffer.View{}, err + return buffer.View{}, tcpip.ControlMessages{}, err } p := e.rcvList.Front() e.rcvList.Remove(p) e.rcvBufSize -= p.data.Size() + ts := e.rcvTimestamp e.rcvMu.Unlock() @@ -156,7 +160,12 @@ func (e *endpoint) Read(addr *tcpip.FullAddress) (buffer.View, *tcpip.Error) { *addr = p.senderAddress } - return p.data.ToView(), nil + if ts && !p.hasTimestamp { + // Linux uses the current time. + p.timestamp = e.stack.NowNanoseconds() + } + + return p.data.ToView(), tcpip.ControlMessages{HasTimestamp: ts, Timestamp: p.timestamp}, nil } // prepareForWrite prepares the endpoint for sending data. In particular, it @@ -299,8 +308,8 @@ func (e *endpoint) Write(p tcpip.Payload, opts tcpip.WriteOptions) (uintptr, *tc } // Peek only returns data from a single datagram, so do nothing here. -func (e *endpoint) Peek([][]byte) (uintptr, *tcpip.Error) { - return 0, nil +func (e *endpoint) Peek([][]byte) (uintptr, tcpip.ControlMessages, *tcpip.Error) { + return 0, tcpip.ControlMessages{}, nil } // SetSockOpt sets a socket option. Currently not supported. @@ -322,6 +331,11 @@ func (e *endpoint) SetSockOpt(opt interface{}) *tcpip.Error { } e.v6only = v != 0 + + case tcpip.TimestampOption: + e.rcvMu.Lock() + e.rcvTimestamp = v != 0 + e.rcvMu.Unlock() } return nil } @@ -370,6 +384,14 @@ func (e *endpoint) GetSockOpt(opt interface{}) *tcpip.Error { } e.rcvMu.Unlock() return nil + + case *tcpip.TimestampOption: + e.rcvMu.Lock() + *o = 0 + if e.rcvTimestamp { + *o = 1 + } + e.rcvMu.Unlock() } return tcpip.ErrUnknownProtocolOption @@ -733,6 +755,11 @@ func (e *endpoint) HandlePacket(r *stack.Route, id stack.TransportEndpointID, vv e.rcvList.PushBack(pkt) e.rcvBufSize += vv.Size() + if e.rcvTimestamp { + pkt.timestamp = e.stack.NowNanoseconds() + pkt.hasTimestamp = true + } + e.rcvMu.Unlock() // Notify any waiters that there's data to be read now. diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go index 65c567952..1eb9ecb80 100644 --- a/pkg/tcpip/transport/udp/udp_test.go +++ b/pkg/tcpip/transport/udp/udp_test.go @@ -56,7 +56,7 @@ type headers struct { } func newDualTestContext(t *testing.T, mtu uint32) *testContext { - s := stack.New([]string{ipv4.ProtocolName, ipv6.ProtocolName}, []string{udp.ProtocolName}) + s := stack.New(&tcpip.StdClock{}, []string{ipv4.ProtocolName, ipv6.ProtocolName}, []string{udp.ProtocolName}) id, linkEP := channel.New(256, mtu, "") if testing.Verbose() { @@ -260,12 +260,12 @@ func testV4Read(c *testContext) { defer c.wq.EventUnregister(&we) var addr tcpip.FullAddress - v, err := c.ep.Read(&addr) + v, _, err := c.ep.Read(&addr) if err == tcpip.ErrWouldBlock { // Wait for data to become available. select { case <-ch: - v, err = c.ep.Read(&addr) + v, _, err = c.ep.Read(&addr) if err != nil { c.t.Fatalf("Read failed: %v", err) } @@ -355,12 +355,12 @@ func TestV6ReadOnV6(t *testing.T) { defer c.wq.EventUnregister(&we) var addr tcpip.FullAddress - v, err := c.ep.Read(&addr) + v, _, err := c.ep.Read(&addr) if err == tcpip.ErrWouldBlock { // Wait for data to become available. select { case <-ch: - v, err = c.ep.Read(&addr) + v, _, err = c.ep.Read(&addr) if err != nil { c.t.Fatalf("Read failed: %v", err) } diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go index a470cb054..d63a9028e 100644 --- a/runsc/boot/loader.go +++ b/runsc/boot/loader.go @@ -37,6 +37,7 @@ import ( slinux "gvisor.googlesource.com/gvisor/pkg/sentry/syscalls/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/time" "gvisor.googlesource.com/gvisor/pkg/sentry/watchdog" + "gvisor.googlesource.com/gvisor/pkg/tcpip" "gvisor.googlesource.com/gvisor/pkg/tcpip/link/sniffer" "gvisor.googlesource.com/gvisor/pkg/tcpip/network/arp" "gvisor.googlesource.com/gvisor/pkg/tcpip/network/ipv4" @@ -177,7 +178,7 @@ func New(spec *specs.Spec, conf *Config, controllerFD int, ioFDs []int, console // this point. Netns is configured before Run() is called. Netstack is // configured using a control uRPC message. Host network is configured inside // Run(). - networkStack := newEmptyNetworkStack(conf) + networkStack := newEmptyNetworkStack(conf, k) // Initiate the Kernel object, which is required by the Context passed // to createVFS in order to mount (among other things) procfs. @@ -337,7 +338,7 @@ func (l *Loader) WaitExit() kernel.ExitStatus { return l.k.GlobalInit().ExitStatus() } -func newEmptyNetworkStack(conf *Config) inet.Stack { +func newEmptyNetworkStack(conf *Config, clock tcpip.Clock) inet.Stack { switch conf.Network { case NetworkHost: return hostinet.NewStack() @@ -346,7 +347,7 @@ func newEmptyNetworkStack(conf *Config) inet.Stack { // NetworkNone sets up loopback using netstack. netProtos := []string{ipv4.ProtocolName, ipv6.ProtocolName, arp.ProtocolName} protoNames := []string{tcp.ProtocolName, udp.ProtocolName} - return &epsocket.Stack{stack.New(netProtos, protoNames)} + return &epsocket.Stack{stack.New(clock, netProtos, protoNames)} default: panic(fmt.Sprintf("invalid network configuration: %v", conf.Network)) -- cgit v1.2.3 From 04b79137babed361fb227e3ad579adb2df4bb188 Mon Sep 17 00:00:00 2001 From: Cyrille Hemidy Date: Thu, 3 May 2018 14:05:25 -0700 Subject: Fix misspellings. PiperOrigin-RevId: 195307689 Change-Id: I499f19af49875a43214797d63376f20ae788d2f4 --- pkg/log/log.go | 2 +- pkg/sentry/fs/file.go | 2 +- pkg/sentry/fs/fsutil/file.go | 2 +- pkg/sentry/fs/ramfs/dir.go | 4 ++-- pkg/sentry/fs/tty/line_discipline.go | 4 ++-- pkg/sentry/kernel/semaphore/semaphore.go | 2 +- pkg/sentry/kernel/task_exit.go | 2 +- pkg/sentry/mm/vma.go | 2 +- pkg/sentry/socket/rpcinet/socket.go | 2 +- pkg/sentry/strace/syscalls.go | 2 +- pkg/tcpip/header/ipv6.go | 2 +- pkg/tcpip/stack/stack.go | 2 +- pkg/tcpip/transport/tcp/snd.go | 2 +- runsc/sandbox/sandbox.go | 4 ++-- 14 files changed, 17 insertions(+), 17 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/log/log.go b/pkg/log/log.go index 110e0e196..cdfc0601a 100644 --- a/pkg/log/log.go +++ b/pkg/log/log.go @@ -149,7 +149,7 @@ func (t TestEmitter) Emit(level Level, timestamp time.Time, format string, v ... // Logger is a high-level logging interface. It is in fact, not used within the // log package. Rather it is provided for others to provide contextual loggers // that may append some addition information to log statement. BasicLogger -// satifies this interface, and may be passed around as a Logger. +// satisfies this interface, and may be passed around as a Logger. type Logger interface { // Debugf logs a debug statement. Debugf(format string, v ...interface{}) diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go index de2e80bf0..f2683bbd2 100644 --- a/pkg/sentry/fs/file.go +++ b/pkg/sentry/fs/file.go @@ -376,7 +376,7 @@ func (r *FileReader) Read(buf []byte) (int, error) { return int(n), err } -// ReadAt implementes io.Reader.ReadAt. +// ReadAt implements io.Reader.ReadAt. func (r *FileReader) ReadAt(buf []byte, offset int64) (int, error) { n, err := r.File.Preadv(r.Ctx, usermem.BytesIOSequence(buf), offset) return int(n), err diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index a7329f1c9..b17f11a5a 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -34,7 +34,7 @@ func (NoopRelease) Release() {} // SeekWithDirCursor is used to implement fs.FileOperations.Seek. If dirCursor // is not nil and the seek was on a directory, the cursor will be updated. // -// Currenly only seeking to 0 on a directory is supported. +// Currently only seeking to 0 on a directory is supported. // // FIXME: Lift directory seeking limitations. func SeekWithDirCursor(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64, dirCursor *string) (int64, error) { diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index bf4cd8dfd..19d5612ed 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -103,7 +103,7 @@ func (d *Dir) addChildLocked(name string, inode *fs.Inode) { } // Given we're now adding this inode to the directory we must also - // increase its link count. Similiarly we decremented it in removeChildLocked. + // increase its link count. Similarly we decremented it in removeChildLocked. inode.AddLink() } @@ -144,7 +144,7 @@ func (d *Dir) removeChildLocked(ctx context.Context, name string) (*fs.Inode, er inode.NotifyStatusChange(ctx) // Given we're now removing this inode to the directory we must also - // decrease its link count. Similiarly it is increased in addChildLocked. + // decrease its link count. Similarly it is increased in addChildLocked. inode.DropLink() return inode, nil diff --git a/pkg/sentry/fs/tty/line_discipline.go b/pkg/sentry/fs/tty/line_discipline.go index fde4e7941..a3aa95ece 100644 --- a/pkg/sentry/fs/tty/line_discipline.go +++ b/pkg/sentry/fs/tty/line_discipline.go @@ -244,8 +244,8 @@ func (l *lineDiscipline) queueWrite(ctx context.Context, src usermem.IOSequence, return int64(n), err } -// transformOutput does ouput processing for one end of the pty. See -// drivers/tty/n_tty.c:do_output_char for an analagous kernel function. +// transformOutput does output processing for one end of the pty. See +// drivers/tty/n_tty.c:do_output_char for an analogous kernel function. // // Precondition: l.termiosMu must be held. func (l *lineDiscipline) transformOutput(buf []byte) *bytes.Buffer { diff --git a/pkg/sentry/kernel/semaphore/semaphore.go b/pkg/sentry/kernel/semaphore/semaphore.go index 19ad5d537..fb8c2f98c 100644 --- a/pkg/sentry/kernel/semaphore/semaphore.go +++ b/pkg/sentry/kernel/semaphore/semaphore.go @@ -298,7 +298,7 @@ func (s *Set) GetVal(num int32, creds *auth.Credentials) (int16, error) { } // ExecuteOps attempts to execute a list of operations to the set. It only -// suceeds when all operations can be applied. No changes are made if it fails. +// succeeds when all operations can be applied. No changes are made if it fails. // // On failure, it may return an error (retries are hopeless) or it may return // a channel that can be waited on before attempting again. diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index 3d49ae350..d6604f37b 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -125,7 +125,7 @@ func (t *Task) killLocked() { Signo: int32(linux.SIGKILL), // Linux just sets SIGKILL in the pending signal bitmask without // enqueueing an actual siginfo, such that - // kernel/signal.c:collect_signal() initalizes si_code to SI_USER. + // kernel/signal.c:collect_signal() initializes si_code to SI_USER. Code: arch.SignalInfoUser, }) t.interrupt() diff --git a/pkg/sentry/mm/vma.go b/pkg/sentry/mm/vma.go index b6af48cb7..61aaa3195 100644 --- a/pkg/sentry/mm/vma.go +++ b/pkg/sentry/mm/vma.go @@ -243,7 +243,7 @@ func (mm *MemoryManager) getVMAsLocked(ctx context.Context, ar usermem.AddrRange } // getVecVMAsLocked ensures that vmas exist for all addresses in ars, and -// support access to type of (at, ignorePermissions). It retuns the subset of +// support access to type of (at, ignorePermissions). It returns the subset of // ars for which vmas exist. If this is not equal to ars, it returns a non-nil // error explaining why. // diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 15047df01..2911d3fd6 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -530,7 +530,7 @@ func (p *socketProvider) Socket(t *kernel.Task, stypeflags unix.SockType, protoc // Only accept TCP and UDP. // // Try to restrict the flags we will accept to minimize backwards - // incompatability with netstack. + // incompatibility with netstack. stype := int(stypeflags) & linux.SOCK_TYPE_MASK switch stype { case syscall.SOCK_STREAM: diff --git a/pkg/sentry/strace/syscalls.go b/pkg/sentry/strace/syscalls.go index d0e661706..eccee733e 100644 --- a/pkg/sentry/strace/syscalls.go +++ b/pkg/sentry/strace/syscalls.go @@ -82,7 +82,7 @@ const ( // PipeFDs is an array of two FDs, formatted after syscall execution. PipeFDs - // Uname is a pointer to a struct uname, formatted after syscall exection. + // Uname is a pointer to a struct uname, formatted after syscall execution. Uname // Stat is a pointer to a struct stat, formatted after syscall execution. diff --git a/pkg/tcpip/header/ipv6.go b/pkg/tcpip/header/ipv6.go index d8dc138b3..da0210539 100644 --- a/pkg/tcpip/header/ipv6.go +++ b/pkg/tcpip/header/ipv6.go @@ -60,7 +60,7 @@ const ( // IPv6ProtocolNumber is IPv6's network protocol number. IPv6ProtocolNumber tcpip.NetworkProtocolNumber = 0x86dd - // IPv6Version is the version of the ipv6 procotol. + // IPv6Version is the version of the ipv6 protocol. IPv6Version = 6 // IPv6MinimumMTU is the minimum MTU required by IPv6, per RFC 2460, diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index b480bf812..f0fbd8aad 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -165,7 +165,7 @@ type TCPSenderState struct { // window size from a segment. SndWndScale uint8 - // MaxSentAck is the highest acknowledgemnt number sent till now. + // MaxSentAck is the highest acknowledgement number sent till now. MaxSentAck seqnum.Value // FastRecovery holds the fast recovery state for the endpoint. diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go index ad94aecd8..6c363a929 100644 --- a/pkg/tcpip/transport/tcp/snd.go +++ b/pkg/tcpip/transport/tcp/snd.go @@ -152,7 +152,7 @@ func newSender(ep *endpoint, iss, irs seqnum.Value, sndWnd seqnum.Size, mss uint // updateMaxPayloadSize updates the maximum payload size based on the given // MTU. If this is in response to "packet too big" control packets (indicated -// by the count argument), it also reduces the number of oustanding packets and +// by the count argument), it also reduces the number of outstanding packets and // attempts to retransmit the first packet above the MTU size. func (s *sender) updateMaxPayloadSize(mtu, count int) { m := mtu - header.TCPMinimumSize diff --git a/runsc/sandbox/sandbox.go b/runsc/sandbox/sandbox.go index 64810b4ea..954824ada 100644 --- a/runsc/sandbox/sandbox.go +++ b/runsc/sandbox/sandbox.go @@ -58,7 +58,7 @@ func validateID(id string) error { // // Within a root directory, we maintain subdirectories for each sandbox named // with the sandbox id. The sandbox metadata is is stored as json within the -// sandbox directoy in a file named "meta.json". This metadata format is +// sandbox directory in a file named "meta.json". This metadata format is // defined by us, and is not part of the OCI spec. // // Sandboxes must write this metadata file after any change to their internal @@ -199,7 +199,7 @@ func Load(rootDir, id string) (*Sandbox, error) { // If the status is "Running" or "Created", check that the process // still exists, and set it to Stopped if it does not. // - // This is inherintly racey. + // This is inherently racey. if s.Status == Running || s.Status == Created { // Send signal 0 to check if process exists. if err := s.Signal(0); err != nil { -- cgit v1.2.3 From 8e1deb2ab8fb67da9a1f6521e31c5635ac587e71 Mon Sep 17 00:00:00 2001 From: Christopher Koch Date: Wed, 16 May 2018 14:53:57 -0700 Subject: Fix another socket Dirent refcount. PiperOrigin-RevId: 196893452 Change-Id: I5ea0f851fcabc5eac5859e61f15213323d996337 --- pkg/sentry/socket/rpcinet/socket.go | 2 ++ 1 file changed, 2 insertions(+) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 2911d3fd6..11925f8d8 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -71,6 +71,7 @@ func newSocketFile(ctx context.Context, stack *Stack, family int, skType int, pr stack.notifier.AddFD(fd, &wq) dirent := socket.NewDirent(ctx, socketDevice) + defer dirent.DecRef() return fs.NewFile(ctx, dirent, fs.FileFlags{Read: true, Write: true}, &socketOperations{ wq: &wq, fd: fd, @@ -274,6 +275,7 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, s.notifier.AddFD(payload.Fd, &wq) dirent := socket.NewDirent(t, socketDevice) + defer dirent.DecRef() file := fs.NewFile(t, dirent, fs.FileFlags{Read: true, Write: true, NonBlocking: flags&linux.SOCK_NONBLOCK != 0}, &socketOperations{ wq: &wq, fd: payload.Fd, -- cgit v1.2.3 From 7f62e9c32ea6af19ccd92107252fd869e6ef1005 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Thu, 24 May 2018 15:17:42 -0700 Subject: rpcinet connect doesn't handle all errnos correctly. These were causing non-blocking related errnos to be returned to the sentry when they were created as blocking FDs internally. PiperOrigin-RevId: 197962932 Change-Id: I3f843535ff87ebf4cb5827e9f3d26abfb79461b0 --- pkg/sentry/socket/rpcinet/socket.go | 22 ++++++++++------------ 1 file changed, 10 insertions(+), 12 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 11925f8d8..bca91ab5f 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -213,21 +213,19 @@ func (s *socketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking boo // Register for notification when the endpoint becomes writable, then // initiate the connection. e, ch := waiter.NewChannelEntry(nil) - s.EventRegister(&e, waiter.EventOut) + s.EventRegister(&e, waiter.EventOut|waiter.EventIn|waiter.EventHUp) defer s.EventUnregister(&e) + for { + if err := rpcConnect(t, s.fd, sockaddr); err == nil || err != syserr.ErrInProgress && err != syserr.ErrAlreadyInProgress { + return err + } - if err := rpcConnect(t, s.fd, sockaddr); err != syserr.ErrConnectStarted && err != syserr.ErrAlreadyConnecting { - return err - } - - // It's pending, so we have to wait for a notification, and fetch the - // result once the wait completes. - if err := t.Block(ch); err != nil { - return syserr.FromError(err) + // It's pending, so we have to wait for a notification, and fetch the + // result once the wait completes. + if err := t.Block(ch); err != nil { + return syserr.FromError(err) + } } - - // Call Connect() again after blocking to find connect's result. - return rpcConnect(t, s.fd, sockaddr) } func rpcAccept(t *kernel.Task, fd uint32, peer bool) (*pb.AcceptResponse_ResultPayload, *syserr.Error) { -- cgit v1.2.3 From 0212f222c74b9f88c5c74d920127e47e942dc376 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Fri, 1 Jun 2018 14:58:46 -0700 Subject: Fix refcount bug in rpcinet socketOperations.Accept. PiperOrigin-RevId: 198931222 Change-Id: I69ee12318e87b9a6a4a94b18a9bf0ae4e39d7eaf --- pkg/sentry/socket/rpcinet/socket.go | 2 ++ 1 file changed, 2 insertions(+) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index bca91ab5f..74cb84927 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -277,8 +277,10 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, file := fs.NewFile(t, dirent, fs.FileFlags{Read: true, Write: true, NonBlocking: flags&linux.SOCK_NONBLOCK != 0}, &socketOperations{ wq: &wq, fd: payload.Fd, + rpcConn: s.rpcConn, notifier: s.notifier, }) + defer file.DecRef() fdFlags := kernel.FDFlags{ CloseOnExec: flags&linux.SOCK_CLOEXEC != 0, -- cgit v1.2.3 From ff7b4a156f95a587b5df4de89a22c200fceabb96 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Tue, 5 Jun 2018 15:43:55 -0700 Subject: Add support for rpcinet owned procfs files. This change will add support for /proc/sys/net and /proc/net which will be managed and owned by rpcinet. This will allow these inodes to be forward as rpcs. PiperOrigin-RevId: 199370799 Change-Id: I2c876005d98fe55dd126145163bee5a645458ce4 --- pkg/sentry/fs/proc/BUILD | 2 + pkg/sentry/fs/proc/net.go | 17 +++ pkg/sentry/fs/proc/proc.go | 45 +++++++- pkg/sentry/fs/proc/rpcinet_proc.go | 193 +++++++++++++++++++++++++++++++++ pkg/sentry/fs/proc/sys.go | 10 +- pkg/sentry/fs/proc/sys_net.go | 66 ++++++++++- pkg/sentry/socket/rpcinet/conn/conn.go | 20 ++++ pkg/sentry/socket/rpcinet/stack.go | 90 +++++---------- 8 files changed, 374 insertions(+), 69 deletions(-) create mode 100644 pkg/sentry/fs/proc/rpcinet_proc.go (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD index 18372cfbf..21b5fc0c3 100644 --- a/pkg/sentry/fs/proc/BUILD +++ b/pkg/sentry/fs/proc/BUILD @@ -44,6 +44,7 @@ go_library( "net.go", "proc.go", "proc_state.go", + "rpcinet_proc.go", "stat.go", "sys.go", "sys_net.go", @@ -70,6 +71,7 @@ go_library( "//pkg/sentry/kernel/kdefs", "//pkg/sentry/kernel/time", "//pkg/sentry/mm", + "//pkg/sentry/socket/rpcinet", "//pkg/sentry/usage", "//pkg/sentry/usermem", "//pkg/state", diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go index 6e464857a..e6bd35f27 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -32,6 +32,23 @@ func (p *proc) newNetDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { if s := p.k.NetworkStack(); s != nil && s.SupportsIPv6() { d.AddChild(ctx, "dev", seqfile.NewSeqFileInode(ctx, &netDev{s: s}, msrc)) d.AddChild(ctx, "if_inet6", seqfile.NewSeqFileInode(ctx, &ifinet6{s: s}, msrc)) + + // The following files are simple stubs until they are implemented in + // netstack, if the file contains a header the stub is just the header + // otherwise it is an empty file. + d.AddChild(ctx, "arp", p.newStubProcFSFile(ctx, msrc, []byte("IP address HW type Flags HW address Mask Device"))) + d.AddChild(ctx, "ipv6_route", p.newStubProcFSFile(ctx, msrc, []byte(""))) + d.AddChild(ctx, "netlink", p.newStubProcFSFile(ctx, msrc, []byte("sk Eth Pid Groups Rmem Wmem Dump Locks Drops Inode"))) + d.AddChild(ctx, "netstat", p.newStubProcFSFile(ctx, msrc, []byte("TcpExt: SyncookiesSent SyncookiesRecv SyncookiesFailed EmbryonicRsts PruneCalled RcvPruned OfoPruned OutOfWindowIcmps LockDroppedIcmps ArpFilter TW TWRecycled TWKilled PAWSPassive PAWSActive PAWSEstab DelayedACKs DelayedACKLocked DelayedACKLost ListenOverflows ListenDrops TCPPrequeued TCPDirectCopyFromBacklog TCPDirectCopyFromPrequeue TCPPrequeueDropped TCPHPHits TCPHPHitsToUser TCPPureAcks TCPHPAcks TCPRenoRecovery TCPSackRecovery TCPSACKReneging TCPFACKReorder TCPSACKReorder TCPRenoReorder TCPTSReorder TCPFullUndo TCPPartialUndo TCPDSACKUndo TCPLossUndo TCPLostRetransmit TCPRenoFailures TCPSackFailures TCPLossFailures TCPFastRetrans TCPForwardRetrans TCPSlowStartRetrans TCPTimeouts TCPLossProbes TCPLossProbeRecovery TCPRenoRecoveryFail TCPSackRecoveryFail TCPSchedulerFailed TCPRcvCollapsed TCPDSACKOldSent TCPDSACKOfoSent TCPDSACKRecv TCPDSACKOfoRecv TCPAbortOnData TCPAbortOnClose TCPAbortOnMemory TCPAbortOnTimeout TCPAbortOnLinger TCPAbortFailed TCPMemoryPressures TCPSACKDiscard TCPDSACKIgnoredOld TCPDSACKIgnoredNoUndo TCPSpuriousRTOs TCPMD5NotFound TCPMD5Unexpected TCPMD5Failure TCPSackShifted TCPSackMerged TCPSackShiftFallback TCPBacklogDrop TCPMinTTLDrop TCPDeferAcceptDrop IPReversePathFilter TCPTimeWaitOverflow TCPReqQFullDoCookies TCPReqQFullDrop TCPRetransFail TCPRcvCoalesce TCPOFOQueue TCPOFODrop TCPOFOMerge TCPChallengeACK TCPSYNChallenge TCPFastOpenActive TCPFastOpenActiveFail TCPFastOpenPassive TCPFastOpenPassiveFail TCPFastOpenListenOverflow TCPFastOpenCookieReqd TCPSpuriousRtxHostQueues BusyPollRxPackets TCPAutoCorking TCPFromZeroWindowAdv TCPToZeroWindowAdv TCPWantZeroWindowAdv TCPSynRetrans TCPOrigDataSent TCPHystartTrainDetect TCPHystartTrainCwnd TCPHystartDelayDetect TCPHystartDelayCwnd TCPACKSkippedSynRecv TCPACKSkippedPAWS TCPACKSkippedSeq TCPACKSkippedFinWait2 TCPACKSkippedTimeWait TCPACKSkippedChallenge TCPWinProbe TCPKeepAlive TCPMTUPFail TCPMTUPSuccess"))) + d.AddChild(ctx, "packet", p.newStubProcFSFile(ctx, msrc, []byte("sk RefCnt Type Proto Iface R Rmem User Inode"))) + d.AddChild(ctx, "protocols", p.newStubProcFSFile(ctx, msrc, []byte("protocol size sockets memory press maxhdr slab module cl co di ac io in de sh ss gs se re sp bi br ha uh gp em"))) + d.AddChild(ctx, "psched", p.newStubProcFSFile(ctx, msrc, []byte(""))) + d.AddChild(ctx, "ptype", p.newStubProcFSFile(ctx, msrc, []byte("Type Device Function"))) + d.AddChild(ctx, "route", p.newStubProcFSFile(ctx, msrc, []byte("Iface Destination Gateway Flags RefCnt Use Metric Mask MTU Window IRTT"))) + d.AddChild(ctx, "tcp", p.newStubProcFSFile(ctx, msrc, []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode"))) + d.AddChild(ctx, "tcp6", p.newStubProcFSFile(ctx, msrc, []byte(" sl local_address remote_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode"))) + d.AddChild(ctx, "udp", p.newStubProcFSFile(ctx, msrc, []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode ref pointer drops"))) + d.AddChild(ctx, "udp6", p.newStubProcFSFile(ctx, msrc, []byte(" sl local_address remote_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode"))) } return newFile(d, msrc, fs.SpecialDirectory, nil) } diff --git a/pkg/sentry/fs/proc/proc.go b/pkg/sentry/fs/proc/proc.go index 459eb7e62..d727e1bc9 100644 --- a/pkg/sentry/fs/proc/proc.go +++ b/pkg/sentry/fs/proc/proc.go @@ -17,6 +17,7 @@ package proc import ( "fmt" + "io" "sort" "strconv" @@ -26,6 +27,9 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/seqfile" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserror" ) // proc is a root proc node. @@ -40,6 +44,30 @@ type proc struct { pidns *kernel.PIDNamespace } +// stubProcFSFile is a file type that can be used to return file contents +// which are constant. This file is not writable and will always have mode +// 0444. +type stubProcFSFile struct { + ramfs.Entry + + // contents are the immutable file contents that will always be returned. + contents []byte +} + +// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. +func (s *stubProcFSFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { + if offset < 0 { + return 0, syserror.EINVAL + } + + if offset >= int64(len(s.contents)) { + return 0, io.EOF + } + + n, err := dst.CopyOut(ctx, s.contents[offset:]) + return int64(n), err +} + // New returns the root node of a partial simple procfs. func New(ctx context.Context, msrc *fs.MountSource) (*fs.Inode, error) { k := kernel.KernelFromContext(ctx) @@ -83,6 +111,15 @@ func (p *proc) newSelf(ctx context.Context, msrc *fs.MountSource) *fs.Inode { return newFile(s, msrc, fs.Symlink, nil) } +// newStubProcFsFile returns a procfs file with constant contents. +func (p *proc) newStubProcFSFile(ctx context.Context, msrc *fs.MountSource, c []byte) *fs.Inode { + u := &stubProcFSFile{ + contents: c, + } + u.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0444)) + return newFile(u, msrc, fs.SpecialFile, nil) +} + // Readlink implements fs.InodeOperations.Readlink. func (s *self) Readlink(ctx context.Context, inode *fs.Inode) (string, error) { if t := kernel.TaskFromContext(ctx); t != nil { @@ -107,7 +144,13 @@ func (p *proc) Lookup(ctx context.Context, dir *fs.Inode, name string) (*fs.Dire // Is it a dynamic element? nfs := map[string]func() *fs.Inode{ - "net": func() *fs.Inode { return p.newNetDir(ctx, dir.MountSource) }, + "net": func() *fs.Inode { + // If we're using rpcinet we will let it manage /proc/net. + if _, ok := p.k.NetworkStack().(*rpcinet.Stack); ok { + return newRPCInetProcNet(ctx, dir.MountSource) + } + return p.newNetDir(ctx, dir.MountSource) + }, "self": func() *fs.Inode { return p.newSelf(ctx, dir.MountSource) }, "sys": func() *fs.Inode { return p.newSysDir(ctx, dir.MountSource) }, } diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go new file mode 100644 index 000000000..50d0271f9 --- /dev/null +++ b/pkg/sentry/fs/proc/rpcinet_proc.go @@ -0,0 +1,193 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package proc + +import ( + "io" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +// rpcinetFile implments fs.InodeOperations as RPCs. +type rpcinetFile struct { + ramfs.Entry + + // filepath is the full path of this rpcinetFile. + filepath string + + k *kernel.Kernel +} + +// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. +// This method can panic if an rpcinetFile was created without an rpcinet +// stack. +func (r rpcinetFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { + s, ok := r.k.NetworkStack().(*rpcinet.Stack) + if !ok { + panic("Network stack is not a rpcinet.") + } + + contents, se := s.RPCReadFile(r.filepath) + if se != nil || offset >= int64(len(contents)) { + return 0, io.EOF + } + + n, err := dst.CopyOut(ctx, contents[offset:]) + return int64(n), err +} + +// Truncate implements fs.InodeOperations.Truncate. +func (r rpcinetFile) Truncate(context.Context, *fs.Inode, int64) error { + return nil +} + +// DeprecatedPwritev implements fs.InodeOperations.DeprecatedPwritev. +// This method can panic if an rpcinetFile was created without an rpcinet +// stack. +func (r rpcinetFile) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { + s, ok := r.k.NetworkStack().(*rpcinet.Stack) + if !ok { + panic("Network stack is not a rpcinet.") + } + + if src.NumBytes() == 0 { + return 0, nil + } + + b := make([]byte, src.NumBytes(), src.NumBytes()) + n, err := src.CopyIn(ctx, b) + if err != nil { + return int64(n), err + } + + written, se := s.RPCWriteFile(r.filepath, b) + return int64(written), se.ToError() +} + +func newRPCProcFSFile(ctx context.Context, msrc *fs.MountSource, filepath string, mode linux.FileMode) *fs.Inode { + f := &rpcinetFile{ + filepath: filepath, + k: kernel.KernelFromContext(ctx), + } + f.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(mode)) + + fi := newFile(f, msrc, fs.SpecialFile, nil) + return fi +} + +// newRPCInetProcNet will build an inode for /proc/net. +func newRPCInetProcNet(ctx context.Context, msrc *fs.MountSource) *fs.Inode { + d := &ramfs.Dir{} + d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) + + // Add all the files we want to forward for /proc/net. + d.AddChild(ctx, "arp", newRPCProcFSFile(ctx, msrc, "/proc/net/arp", 0444)) + d.AddChild(ctx, "dev", newRPCProcFSFile(ctx, msrc, "/proc/net/dev", 0444)) + d.AddChild(ctx, "if_inet6", newRPCProcFSFile(ctx, msrc, "/proc/net/if_inet6", 0444)) + d.AddChild(ctx, "ipv6_route", newRPCProcFSFile(ctx, msrc, "/proc/net/ipv6_route", 0444)) + d.AddChild(ctx, "netlink", newRPCProcFSFile(ctx, msrc, "/proc/net/netlink", 0444)) + d.AddChild(ctx, "netstat", newRPCProcFSFile(ctx, msrc, "/proc/net/netstat", 0444)) + d.AddChild(ctx, "packet", newRPCProcFSFile(ctx, msrc, "/proc/net/packet", 0444)) + d.AddChild(ctx, "protocols", newRPCProcFSFile(ctx, msrc, "/proc/net/protocols", 0444)) + d.AddChild(ctx, "psched", newRPCProcFSFile(ctx, msrc, "/proc/net/psched", 0444)) + d.AddChild(ctx, "ptype", newRPCProcFSFile(ctx, msrc, "/proc/net/ptype", 0444)) + d.AddChild(ctx, "route", newRPCProcFSFile(ctx, msrc, "/proc/net/route", 0444)) + d.AddChild(ctx, "tcp", newRPCProcFSFile(ctx, msrc, "/proc/net/tcp", 0444)) + d.AddChild(ctx, "tcp6", newRPCProcFSFile(ctx, msrc, "/proc/net/tcp6", 0444)) + d.AddChild(ctx, "udp", newRPCProcFSFile(ctx, msrc, "/proc/net/udp", 0444)) + d.AddChild(ctx, "udp6", newRPCProcFSFile(ctx, msrc, "/proc/net/udp6", 0444)) + + return newFile(d, msrc, fs.SpecialDirectory, nil) +} + +// newRPCInetProcSysNet will build an inode for /proc/sys/net. +func newRPCInetProcSysNet(ctx context.Context, msrc *fs.MountSource) *fs.Inode { + d := &ramfs.Dir{} + d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) + d.AddChild(ctx, "ipv4", newRPCInetSysNetIPv4Dir(ctx, msrc)) + d.AddChild(ctx, "core", newRPCInetSysNetCore(ctx, msrc)) + + return newFile(d, msrc, fs.SpecialDirectory, nil) +} + +// newRPCInetSysNetCore builds the /proc/sys/net/core directory. +func newRPCInetSysNetCore(ctx context.Context, msrc *fs.MountSource) *fs.Inode { + d := &ramfs.Dir{} + d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) + + // Add all the files we want to forward over RPC for /proc/sys/net/core + d.AddChild(ctx, "default_qdisc", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/default_qdisc", 0444)) + d.AddChild(ctx, "message_burst", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/message_burst", 0444)) + d.AddChild(ctx, "message_cost", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/message_cost", 0444)) + d.AddChild(ctx, "optmem_max", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/optmem_max", 0444)) + d.AddChild(ctx, "rmem_default", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/rmem_default", 0444)) + d.AddChild(ctx, "rmem_max", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/rmem_max", 0444)) + d.AddChild(ctx, "somaxconn", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/somaxconn", 0444)) + d.AddChild(ctx, "wmem_default", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/wmem_default", 0444)) + d.AddChild(ctx, "wmem_max", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/wmem_max", 0444)) + + return newFile(d, msrc, fs.SpecialDirectory, nil) +} + +// newRPCInetSysNetIPv4Dir builds the /proc/sys/net/ipv4 directory. +func newRPCInetSysNetIPv4Dir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { + d := &ramfs.Dir{} + d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) + + // Add all the files we want to forward over RPC for /proc/sys/net/ipv4. + d.AddChild(ctx, "ip_local_port_range", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ip_local_port_range", 0444)) + d.AddChild(ctx, "ip_local_reserved_ports", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ip_local_reserved_ports", 0444)) + d.AddChild(ctx, "ipfrag_time", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ipfrag_time", 0444)) + d.AddChild(ctx, "ip_nonlocal_bind", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ip_nonlocal_bind", 0444)) + d.AddChild(ctx, "ip_no_pmtu_disc", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ip_no_pmtu_disc", 0444)) + + d.AddChild(ctx, "tcp_allowed_congestion_control", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_allowed_congestion_control", 0444)) + d.AddChild(ctx, "tcp_available_congestion_control", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_available_congestion_control", 0444)) + d.AddChild(ctx, "tcp_base_mss", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_base_mss", 0444)) + d.AddChild(ctx, "tcp_congestion_control", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_congestion_control", 0644)) + d.AddChild(ctx, "tcp_dsack", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_dsack", 0644)) + d.AddChild(ctx, "tcp_early_retrans", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_early_retrans", 0644)) + d.AddChild(ctx, "tcp_fack", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_fack", 0644)) + d.AddChild(ctx, "tcp_fastopen", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_fastopen", 0644)) + d.AddChild(ctx, "tcp_fastopen_key", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_fastopen_key", 0444)) + d.AddChild(ctx, "tcp_fin_timeout", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_fin_timeout", 0644)) + d.AddChild(ctx, "tcp_invalid_ratelimit", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_invalid_ratelimit", 0444)) + d.AddChild(ctx, "tcp_keepalive_intvl", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_intvl", 0644)) + d.AddChild(ctx, "tcp_keepalive_probes", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_probes", 0644)) + d.AddChild(ctx, "tcp_keepalive_time", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_time", 0644)) + d.AddChild(ctx, "tcp_mem", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_mem", 0444)) + d.AddChild(ctx, "tcp_mtu_probing", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_mtu_probing", 0644)) + d.AddChild(ctx, "tcp_no_metrics_save", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_no_metrics_save", 0444)) + d.AddChild(ctx, "tcp_probe_interval", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_probe_interval", 0444)) + d.AddChild(ctx, "tcp_probe_threshold", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_probe_threshold", 0444)) + d.AddChild(ctx, "tcp_retries1", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_retries1", 0644)) + d.AddChild(ctx, "tcp_retries2", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_retries2", 0644)) + d.AddChild(ctx, "tcp_rfc1337", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_rfc1337", 0444)) + d.AddChild(ctx, "tcp_rmem", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_rmem", 0444)) + d.AddChild(ctx, "tcp_sack", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_sack", 0644)) + d.AddChild(ctx, "tcp_slow_start_after_idle", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_slow_start_after_idle", 0644)) + d.AddChild(ctx, "tcp_synack_retries", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_synack_retries", 0644)) + d.AddChild(ctx, "tcp_syn_retries", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_syn_retries", 0644)) + d.AddChild(ctx, "tcp_timestamps", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_timestamps", 0644)) + d.AddChild(ctx, "tcp_wmem", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_wmem", 0444)) + + return newFile(d, msrc, fs.SpecialDirectory, nil) +} diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go index 4323f3650..db9ec83b9 100644 --- a/pkg/sentry/fs/proc/sys.go +++ b/pkg/sentry/fs/proc/sys.go @@ -23,6 +23,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/seqfile" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" ) @@ -112,6 +113,13 @@ func (p *proc) newSysDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) d.AddChild(ctx, "kernel", p.newKernelDir(ctx, msrc)) d.AddChild(ctx, "vm", p.newVMDir(ctx, msrc)) - d.AddChild(ctx, "net", p.newSysNetDir(ctx, msrc)) + + // If we're using rpcinet we will let it manage /proc/sys/net. + if _, ok := p.k.NetworkStack().(*rpcinet.Stack); ok { + d.AddChild(ctx, "net", newRPCInetProcSysNet(ctx, msrc)) + } else { + d.AddChild(ctx, "net", p.newSysNetDir(ctx, msrc)) + } + return newFile(d, msrc, fs.SpecialDirectory, nil) } diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index db44c95cb..2a108708c 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -158,7 +158,28 @@ func (s *tcpSack) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, return n, s.s.SetTCPSACKEnabled(v != 0) } -func newSysNetIPv4Dir(ctx context.Context, msrc *fs.MountSource, s inet.Stack) *fs.Inode { +func (p *proc) newSysNetCore(ctx context.Context, msrc *fs.MountSource, s inet.Stack) *fs.Inode { + d := &ramfs.Dir{} + d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) + + // The following files are simple stubs until they are implemented in + // netstack, most of these files are configuration related. We use the + // value closest to the actual netstack behavior or any empty file, + // all of these files will have mode 0444 (read-only for all users). + d.AddChild(ctx, "default_qdisc", p.newStubProcFSFile(ctx, msrc, []byte("pfifo_fast"))) + d.AddChild(ctx, "message_burst", p.newStubProcFSFile(ctx, msrc, []byte("10"))) + d.AddChild(ctx, "message_cost", p.newStubProcFSFile(ctx, msrc, []byte("5"))) + d.AddChild(ctx, "optmem_max", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "rmem_default", p.newStubProcFSFile(ctx, msrc, []byte("212992"))) + d.AddChild(ctx, "rmem_max", p.newStubProcFSFile(ctx, msrc, []byte("212992"))) + d.AddChild(ctx, "somaxconn", p.newStubProcFSFile(ctx, msrc, []byte("128"))) + d.AddChild(ctx, "wmem_default", p.newStubProcFSFile(ctx, msrc, []byte("212992"))) + d.AddChild(ctx, "wmem_max", p.newStubProcFSFile(ctx, msrc, []byte("212992"))) + + return newFile(d, msrc, fs.SpecialDirectory, nil) +} + +func (p *proc) newSysNetIPv4Dir(ctx context.Context, msrc *fs.MountSource, s inet.Stack) *fs.Inode { d := &ramfs.Dir{} d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) @@ -175,6 +196,46 @@ func newSysNetIPv4Dir(ctx context.Context, msrc *fs.MountSource, s inet.Stack) * // Add tcp_sack. d.AddChild(ctx, "tcp_sack", newTCPSackInode(ctx, msrc, s)) + // The following files are simple stubs until they are implemented in + // netstack, most of these files are configuration related. We use the + // value closest to the actual netstack behavior or any empty file, + // all of these files will have mode 0444 (read-only for all users). + d.AddChild(ctx, "ip_local_port_range", p.newStubProcFSFile(ctx, msrc, []byte("16000 65535"))) + d.AddChild(ctx, "ip_local_reserved_ports", p.newStubProcFSFile(ctx, msrc, []byte(""))) + d.AddChild(ctx, "ipfrag_time", p.newStubProcFSFile(ctx, msrc, []byte("30"))) + d.AddChild(ctx, "ip_nonlocal_bind", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "ip_no_pmtu_disc", p.newStubProcFSFile(ctx, msrc, []byte("1"))) + + // tcp_allowed_congestion_control tell the user what they are able to do as an + // unprivledged process so we leave it empty. + d.AddChild(ctx, "tcp_allowed_congestion_control", p.newStubProcFSFile(ctx, msrc, []byte(""))) + d.AddChild(ctx, "tcp_available_congestion_control", p.newStubProcFSFile(ctx, msrc, []byte("reno"))) + d.AddChild(ctx, "tcp_congestion_control", p.newStubProcFSFile(ctx, msrc, []byte("reno"))) + + // Many of the following stub files are features netstack doesn't support + // and are therefore "0" for disabled. + d.AddChild(ctx, "tcp_base_mss", p.newStubProcFSFile(ctx, msrc, []byte("1280"))) + d.AddChild(ctx, "tcp_dsack", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_early_retrans", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_fack", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_fastopen", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_fastopen_key", p.newStubProcFSFile(ctx, msrc, []byte(""))) + d.AddChild(ctx, "tcp_invalid_ratelimit", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_keepalive_intvl", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_keepalive_probes", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_keepalive_time", p.newStubProcFSFile(ctx, msrc, []byte("7200"))) + d.AddChild(ctx, "tcp_mtu_probing", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_no_metrics_save", p.newStubProcFSFile(ctx, msrc, []byte("1"))) + d.AddChild(ctx, "tcp_probe_interval", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_probe_threshold", p.newStubProcFSFile(ctx, msrc, []byte("0"))) + d.AddChild(ctx, "tcp_retries1", p.newStubProcFSFile(ctx, msrc, []byte("3"))) + d.AddChild(ctx, "tcp_retries2", p.newStubProcFSFile(ctx, msrc, []byte("15"))) + d.AddChild(ctx, "tcp_rfc1337", p.newStubProcFSFile(ctx, msrc, []byte("1"))) + d.AddChild(ctx, "tcp_slow_start_after_idle", p.newStubProcFSFile(ctx, msrc, []byte("1"))) + d.AddChild(ctx, "tcp_synack_retries", p.newStubProcFSFile(ctx, msrc, []byte("5"))) + d.AddChild(ctx, "tcp_syn_retries", p.newStubProcFSFile(ctx, msrc, []byte("3"))) + d.AddChild(ctx, "tcp_timestamps", p.newStubProcFSFile(ctx, msrc, []byte("1"))) + return newFile(d, msrc, fs.SpecialDirectory, nil) } @@ -182,7 +243,8 @@ func (p *proc) newSysNetDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode d := &ramfs.Dir{} d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) if s := p.k.NetworkStack(); s != nil { - d.AddChild(ctx, "ipv4", newSysNetIPv4Dir(ctx, msrc, s)) + d.AddChild(ctx, "ipv4", p.newSysNetIPv4Dir(ctx, msrc, s)) + d.AddChild(ctx, "core", p.newSysNetCore(ctx, msrc, s)) } return newFile(d, msrc, fs.SpecialDirectory, nil) } diff --git a/pkg/sentry/socket/rpcinet/conn/conn.go b/pkg/sentry/socket/rpcinet/conn/conn.go index ea6ec87ed..f4c8489b1 100644 --- a/pkg/sentry/socket/rpcinet/conn/conn.go +++ b/pkg/sentry/socket/rpcinet/conn/conn.go @@ -147,6 +147,26 @@ func (c *RPCConnection) RPCReadFile(path string) ([]byte, *syserr.Error) { return res.(*pb.ReadFileResponse_Data).Data, nil } +// RPCWriteFile will execute the WriteFile helper RPC method which avoids the +// common pattern of open(2), write(2), write(2), close(2) by doing all +// operations as a single RPC. +func (c *RPCConnection) RPCWriteFile(path string, data []byte) (int64, *syserr.Error) { + req := &pb.SyscallRequest_WriteFile{&pb.WriteFileRequest{ + Path: path, + Content: data, + }} + + id, ch := c.NewRequest(pb.SyscallRequest{Args: req}, false /* ignoreResult */) + <-ch + + res := c.Request(id).Result.(*pb.SyscallResponse_WriteFile).WriteFile + if e := res.ErrorNumber; e != 0 { + return int64(res.Written), syserr.FromHost(syscall.Errno(e)) + } + + return int64(res.Written), nil +} + // Request retrieves the request corresponding to the given request ID. // // The channel returned by NewRequest must have been closed before Request can diff --git a/pkg/sentry/socket/rpcinet/stack.go b/pkg/sentry/socket/rpcinet/stack.go index 503e0e932..bcb89fb34 100644 --- a/pkg/sentry/socket/rpcinet/stack.go +++ b/pkg/sentry/socket/rpcinet/stack.go @@ -16,50 +16,24 @@ package rpcinet import ( "fmt" - "strings" "syscall" - "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/inet" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/hostinet" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/notifier" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/unet" ) // Stack implements inet.Stack for RPC backed sockets. type Stack struct { - // We intentionally do not allow these values to be changed to remain - // consistent with the other networking stacks. interfaces map[int32]inet.Interface interfaceAddrs map[int32][]inet.InterfaceAddr - supportsIPv6 bool - tcpRecvBufSize inet.TCPBufferSize - tcpSendBufSize inet.TCPBufferSize - tcpSACKEnabled bool rpcConn *conn.RPCConnection notifier *notifier.Notifier } -func readTCPBufferSizeFile(conn *conn.RPCConnection, filename string) (inet.TCPBufferSize, error) { - contents, se := conn.RPCReadFile(filename) - if se != nil { - return inet.TCPBufferSize{}, fmt.Errorf("failed to read %s: %v", filename, se) - } - ioseq := usermem.BytesIOSequence(contents) - fields := make([]int32, 3) - if n, err := usermem.CopyInt32StringsInVec(context.Background(), ioseq.IO, ioseq.Addrs, fields, ioseq.Opts); n != ioseq.NumBytes() || err != nil { - return inet.TCPBufferSize{}, fmt.Errorf("failed to parse %s (%q): got %v after %d/%d bytes", filename, contents, err, n, ioseq.NumBytes()) - } - return inet.TCPBufferSize{ - Min: int(fields[0]), - Default: int(fields[1]), - Max: int(fields[2]), - }, nil -} - // NewStack returns a Stack containing the current state of the host network // stack. func NewStack(fd int32) (*Stack, error) { @@ -80,31 +54,6 @@ func NewStack(fd int32) (*Stack, error) { return nil, e } - // Load the configuration values from procfs. - tcpRMem, e := readTCPBufferSizeFile(stack.rpcConn, "/proc/sys/net/ipv4/tcp_rmem") - if e != nil { - return nil, e - } - stack.tcpRecvBufSize = tcpRMem - - tcpWMem, e := readTCPBufferSizeFile(stack.rpcConn, "/proc/sys/net/ipv4/tcp_wmem") - if e != nil { - return nil, e - } - stack.tcpSendBufSize = tcpWMem - - ipv6, se := stack.rpcConn.RPCReadFile("/proc/net/if_inet6") - if len(string(ipv6)) > 0 { - stack.supportsIPv6 = true - } - - sackFile := "/proc/sys/net/ipv4/tcp_sack" - sack, se := stack.rpcConn.RPCReadFile(sackFile) - if se != nil { - return nil, fmt.Errorf("failed to read %s: %v", sackFile, se) - } - stack.tcpSACKEnabled = strings.TrimSpace(string(sack)) != "0" - links, err := stack.DoNetlinkRouteRequest(syscall.RTM_GETLINK) if err != nil { return nil, fmt.Errorf("RTM_GETLINK failed: %v", err) @@ -123,6 +72,21 @@ func NewStack(fd int32) (*Stack, error) { return stack, nil } +// RPCReadFile will execute the ReadFile helper RPC method which avoids the +// common pattern of open(2), read(2), close(2) by doing all three operations +// as a single RPC. It will read the entire file or return EFBIG if the file +// was too large. +func (s *Stack) RPCReadFile(path string) ([]byte, *syserr.Error) { + return s.rpcConn.RPCReadFile(path) +} + +// RPCWriteFile will execute the WriteFile helper RPC method which avoids the +// common pattern of open(2), write(2), write(2), close(2) by doing all +// operations as a single RPC. +func (s *Stack) RPCWriteFile(path string, data []byte) (int64, *syserr.Error) { + return s.rpcConn.RPCWriteFile(path, data) +} + // Interfaces implements inet.Stack.Interfaces. func (s *Stack) Interfaces() map[int32]inet.Interface { return s.interfaces @@ -135,41 +99,37 @@ func (s *Stack) InterfaceAddrs() map[int32][]inet.InterfaceAddr { // SupportsIPv6 implements inet.Stack.SupportsIPv6. func (s *Stack) SupportsIPv6() bool { - return s.supportsIPv6 + panic("rpcinet handles procfs directly this method should not be called") } // TCPReceiveBufferSize implements inet.Stack.TCPReceiveBufferSize. func (s *Stack) TCPReceiveBufferSize() (inet.TCPBufferSize, error) { - return s.tcpRecvBufSize, nil + panic("rpcinet handles procfs directly this method should not be called") } // SetTCPReceiveBufferSize implements inet.Stack.SetTCPReceiveBufferSize. func (s *Stack) SetTCPReceiveBufferSize(size inet.TCPBufferSize) error { - // To keep all the supported stacks consistent we don't allow changing this - // value even though it would be possible via an RPC. - return syserror.EACCES + panic("rpcinet handles procfs directly this method should not be called") + } // TCPSendBufferSize implements inet.Stack.TCPSendBufferSize. func (s *Stack) TCPSendBufferSize() (inet.TCPBufferSize, error) { - return s.tcpSendBufSize, nil + panic("rpcinet handles procfs directly this method should not be called") + } // SetTCPSendBufferSize implements inet.Stack.SetTCPSendBufferSize. func (s *Stack) SetTCPSendBufferSize(size inet.TCPBufferSize) error { - // To keep all the supported stacks consistent we don't allow changing this - // value even though it would be possible via an RPC. - return syserror.EACCES + panic("rpcinet handles procfs directly this method should not be called") } // TCPSACKEnabled implements inet.Stack.TCPSACKEnabled. func (s *Stack) TCPSACKEnabled() (bool, error) { - return s.tcpSACKEnabled, nil + panic("rpcinet handles procfs directly this method should not be called") } // SetTCPSACKEnabled implements inet.Stack.SetTCPSACKEnabled. func (s *Stack) SetTCPSACKEnabled(enabled bool) error { - // To keep all the supported stacks consistent we don't allow changing this - // value even though it would be possible via an RPC. - return syserror.EACCES + panic("rpcinet handles procfs directly this method should not be called") } -- cgit v1.2.3 From 79fef54eb1b9e941e2c910f90b65f3cfe94e18c4 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Wed, 6 Jun 2018 15:52:29 -0700 Subject: Add support for rpcinet ioctl(2). This change will add support for ioctls that have previously been supported by netstack. LINE_LENGTH_IGNORE PiperOrigin-RevId: 199544114 Change-Id: I3769202c19502c3b7d05e06ea9552acfd9255893 --- pkg/sentry/socket/rpcinet/socket.go | 63 ++++++++++++++++++++++++++++- pkg/sentry/socket/rpcinet/syscall_rpc.proto | 14 +++---- 2 files changed, 68 insertions(+), 9 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 74cb84927..3356f7804 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -56,6 +56,10 @@ type socketOperations struct { // Verify that we actually implement socket.Socket. var _ = socket.Socket(&socketOperations{}) +const ( + sizeOfIfReq = 40 +) + // New creates a new RPC socket. func newSocketFile(ctx context.Context, stack *Stack, family int, skType int, protocol int) (*fs.File, *syserr.Error) { id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Socket{&pb.SocketRequest{Family: int64(family), Type: int64(skType | syscall.SOCK_NONBLOCK), Protocol: int64(protocol)}}}, false /* ignoreResult */) @@ -290,7 +294,11 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, return 0, nil, 0, syserr.FromError(err) } - return fd, payload.Address.Address, payload.Address.Length, nil + if peerRequested { + return fd, payload.Address.Address, payload.Address.Length, nil + } + + return fd, nil, 0, nil } // Bind implements socket.Socket.Bind. @@ -385,9 +393,60 @@ func (s *socketOperations) GetSockName(t *kernel.Task) (interface{}, uint32, *sy return addr.Address, addr.Length, nil } +func rpcIoctl(t *kernel.Task, fd, cmd uint32, arg []byte) ([]byte, error) { + stack := t.NetworkContext().(*Stack) + + id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Ioctl{&pb.IOCtlRequest{Fd: fd, Cmd: cmd, Arg: arg}}}, false /* ignoreResult */) + <-c + + res := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Ioctl).Ioctl.Result + if e, ok := res.(*pb.IOCtlResponse_ErrorNumber); ok { + return nil, syscall.Errno(e.ErrorNumber) + } + + return res.(*pb.IOCtlResponse_Value).Value, nil +} + // Ioctl implements fs.FileOperations.Ioctl. func (s *socketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { - return 0, syserror.ENOTTY + t := ctx.(*kernel.Task) + + cmd := uint32(args[1].Int()) + arg := args[2].Pointer() + + var buf []byte + switch cmd { + // The following ioctls take 4 byte argument parameters. + case syscall.TIOCINQ, syscall.TIOCOUTQ: + buf = make([]byte, 4) + // The following ioctls have args which are sizeof(struct ifreq). + case syscall.SIOCGIFINDEX, syscall.SIOCGIFNETMASK, syscall.SIOCGIFHWADDR, syscall.SIOCGIFNAME, syscall.SIOCGIFFLAGS: + buf = make([]byte, sizeOfIfReq) + default: + return 0, syserror.ENOTTY + } + + _, err := io.CopyIn(ctx, arg, buf, usermem.IOOpts{ + AddressSpaceActive: true, + }) + + if err != nil { + return 0, err + } + + v, err := rpcIoctl(t, s.fd, cmd, buf) + if err != nil { + return 0, err + } + + if len(v) != len(buf) { + return 0, syserror.EINVAL + } + + _, err = io.CopyOut(ctx, arg, v, usermem.IOOpts{ + AddressSpaceActive: true, + }) + return 0, err } func rpcRecvMsg(t *kernel.Task, req *pb.SyscallRequest_Recvmsg) (*pb.RecvmsgResponse_ResultPayload, *syserr.Error) { diff --git a/pkg/sentry/socket/rpcinet/syscall_rpc.proto b/pkg/sentry/socket/rpcinet/syscall_rpc.proto index b845b1bce..996962aae 100644 --- a/pkg/sentry/socket/rpcinet/syscall_rpc.proto +++ b/pkg/sentry/socket/rpcinet/syscall_rpc.proto @@ -8,7 +8,7 @@ package syscall_rpc; message SendmsgRequest { uint32 fd = 1; - bytes data = 2; + bytes data = 2 [ctype = CORD]; bytes address = 3; bool more = 4; bool end_of_record = 5; @@ -24,13 +24,13 @@ message SendmsgResponse { message IOCtlRequest { uint32 fd = 1; uint32 cmd = 2; - uint64 arg = 3; + bytes arg = 3; } message IOCtlResponse { oneof result { uint32 error_number = 1; - uint64 value = 2; + bytes value = 2; } } @@ -63,7 +63,7 @@ message ReadRequest { message ReadResponse { oneof result { uint32 error_number = 1; - bytes data = 2; + bytes data = 2 [ctype = CORD]; } } @@ -74,13 +74,13 @@ message ReadFileRequest { message ReadFileResponse { oneof result { uint32 error_number = 1; - bytes data = 2; + bytes data = 2 [ctype = CORD]; } } message WriteRequest { uint32 fd = 1; - bytes data = 2; + bytes data = 2 [ctype = CORD]; } message WriteResponse { @@ -107,7 +107,7 @@ message AddressResponse { message RecvmsgResponse { message ResultPayload { - bytes data = 1; + bytes data = 1 [ctype = CORD]; AddressResponse address = 2; uint32 length = 3; } -- cgit v1.2.3 From 7e9893eeb500ab56dcab80471300df50c12288ae Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Thu, 7 Jun 2018 11:36:26 -0700 Subject: Add missing rpcinet ioctls. PiperOrigin-RevId: 199669120 Change-Id: I0be88cdbba29760f967e9a5bb4144ca62c1ed7aa --- pkg/sentry/socket/rpcinet/socket.go | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 3356f7804..29546b683 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -417,10 +417,22 @@ func (s *socketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.S var buf []byte switch cmd { // The following ioctls take 4 byte argument parameters. - case syscall.TIOCINQ, syscall.TIOCOUTQ: + case syscall.TIOCINQ, + syscall.TIOCOUTQ: buf = make([]byte, 4) // The following ioctls have args which are sizeof(struct ifreq). - case syscall.SIOCGIFINDEX, syscall.SIOCGIFNETMASK, syscall.SIOCGIFHWADDR, syscall.SIOCGIFNAME, syscall.SIOCGIFFLAGS: + case syscall.SIOCGIFADDR, + syscall.SIOCGIFBRDADDR, + syscall.SIOCGIFDSTADDR, + syscall.SIOCGIFFLAGS, + syscall.SIOCGIFHWADDR, + syscall.SIOCGIFINDEX, + syscall.SIOCGIFMAP, + syscall.SIOCGIFMETRIC, + syscall.SIOCGIFMTU, + syscall.SIOCGIFNAME, + syscall.SIOCGIFNETMASK, + syscall.SIOCGIFTXQLEN: buf = make([]byte, sizeOfIfReq) default: return 0, syserror.ENOTTY -- cgit v1.2.3 From 5c37097e34a513845d77bb8b7240f0074aa1c1e9 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Thu, 7 Jun 2018 15:09:27 -0700 Subject: rpcinet should not block in read(2) rpcs. PiperOrigin-RevId: 199703609 Change-Id: I8153b0396b22a230a68d4b69c46652a5545f7630 --- pkg/sentry/socket/rpcinet/socket.go | 23 +---------------------- 1 file changed, 1 insertion(+), 22 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 29546b683..69cf604b7 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -145,29 +145,8 @@ func (s *socketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOS n, e := dst.CopyOut(ctx, res.Data) return int64(n), e } - if se != syserr.ErrWouldBlock { - return 0, se.ToError() - } - - // We'll have to block. Register for notifications and read again when ready. - e, ch := waiter.NewChannelEntry(nil) - s.EventRegister(&e, waiter.EventIn) - defer s.EventUnregister(&e) - for { - res, se := rpcRead(ctx.(*kernel.Task), req) - if se == nil { - n, e := dst.CopyOut(ctx, res.Data) - return int64(n), e - } - if se != syserr.ErrWouldBlock { - return 0, se.ToError() - } - - if err := ctx.(*kernel.Task).Block(ch); err != nil { - return 0, err - } - } + return 0, se.ToError() } func rpcWrite(t *kernel.Task, req *pb.SyscallRequest_Write) (uint32, *syserr.Error) { -- cgit v1.2.3 From 2f3895d6f7ad37915edcdd80706f880ce50c519c Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Fri, 8 Jun 2018 10:32:30 -0700 Subject: rpcinet is not correctly handling MSG_TRUNC on recvmsg(2). MSG_TRUNC can cause recvmsg(2) to return a value larger than the buffer size. In this situation it's an indication that the buffer was completely filled and that the msg was truncated. Previously in rpcinet we were returning the buffer size but we should actually be returning the payload length as returned by the syscall. PiperOrigin-RevId: 199814221 Change-Id: If09aa364219c1bf193603896fcc0dc5c55e85d21 --- pkg/sentry/socket/rpcinet/socket.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 69cf604b7..c4ecb30f5 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -465,8 +465,8 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags res, err := rpcRecvMsg(t, req) if err == nil { - n, e := dst.CopyOut(t, res.Data) - return int(n), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) + _, e := dst.CopyOut(t, res.Data) + return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } if err != syserr.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { return 0, nil, 0, socket.ControlMessages{}, err @@ -481,8 +481,8 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags for { res, err := rpcRecvMsg(t, req) if err == nil { - n, e := dst.CopyOut(t, res.Data) - return int(n), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) + _, e := dst.CopyOut(t, res.Data) + return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } if err != syserr.ErrWouldBlock { return 0, nil, 0, socket.ControlMessages{}, err -- cgit v1.2.3 From 2fbd1cf57cb06c5f0165a2d0e9225eed242a41f5 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Fri, 8 Jun 2018 15:57:33 -0700 Subject: Add checks for short CopyOut in rpcinet PiperOrigin-RevId: 199864753 Change-Id: Ibace6a1fdf99ee6ce368ac12c390aa8a02dbdfb7 --- pkg/sentry/socket/rpcinet/socket.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index c4ecb30f5..a9dd1780a 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -465,7 +465,10 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags res, err := rpcRecvMsg(t, req) if err == nil { - _, e := dst.CopyOut(t, res.Data) + n, e := dst.CopyOut(t, res.Data) + if e == nil && n != len(res.Data) { + panic("CopyOut failed to copy full buffer") + } return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } if err != syserr.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { @@ -481,7 +484,10 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags for { res, err := rpcRecvMsg(t, req) if err == nil { - _, e := dst.CopyOut(t, res.Data) + n, e := dst.CopyOut(t, res.Data) + if e == nil && n != len(res.Data) { + panic("CopyOut failed to copy full buffer") + } return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } if err != syserr.ErrWouldBlock { -- cgit v1.2.3 From 0412f17e06670fb1f1d1d85ddd73bbadde40c087 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Mon, 11 Jun 2018 15:33:07 -0700 Subject: rpcinet is treating EAGAIN and EWOULDBLOCK as different errnos. PiperOrigin-RevId: 200124614 Change-Id: I38a7b083f1464a2a586fe24db648e624c455fec5 --- pkg/sentry/socket/rpcinet/socket.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index a9dd1780a..ffe947500 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -228,7 +228,7 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, payload, se := rpcAccept(t, s.fd, peerRequested) // Check if we need to block. - if blocking && se == syserr.ErrWouldBlock { + if blocking && se == syserr.ErrTryAgain { // Register for notifications. e, ch := waiter.NewChannelEntry(nil) s.EventRegister(&e, waiter.EventIn) @@ -237,7 +237,7 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, // Try to accept the connection again; if it fails, then wait until we // get a notification. for { - if payload, se = rpcAccept(t, s.fd, peerRequested); se != syserr.ErrWouldBlock { + if payload, se = rpcAccept(t, s.fd, peerRequested); se != syserr.ErrTryAgain { break } @@ -471,7 +471,7 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags } return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } - if err != syserr.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain || flags&linux.MSG_DONTWAIT != 0 { return 0, nil, 0, socket.ControlMessages{}, err } @@ -490,7 +490,7 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags } return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } - if err != syserr.ErrWouldBlock { + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain { return 0, nil, 0, socket.ControlMessages{}, err } @@ -546,7 +546,7 @@ func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] }} n, err := rpcSendMsg(t, req) - if err != syserr.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain || flags&linux.MSG_DONTWAIT != 0 { return int(n), err } @@ -558,7 +558,7 @@ func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] for { n, err := rpcSendMsg(t, req) - if err != syserr.ErrWouldBlock { + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain { return int(n), err } -- cgit v1.2.3 From ab2c2575d61266725ce13dff570663464a171342 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Mon, 11 Jun 2018 16:39:39 -0700 Subject: Rpcinet is incorrectly handling MSG_TRUNC with SOCK_STREAM SOCK_STREAM has special behavior with respect to MSG_TRUNC. Specifically, the data isn't actually copied back out to userspace when MSG_TRUNC is provided on a SOCK_STREAM. According to tcp(7): "Since version 2.4, Linux supports the use of MSG_TRUNC in the flags argument of recv(2) (and recvmsg(2)). This flag causes the received bytes of data to be discarded, rather than passed back in a caller-supplied buffer." PiperOrigin-RevId: 200134860 Change-Id: I70f17a5f60ffe7794c3f0cfafd131c069202e90d --- pkg/sentry/socket/rpcinet/socket.go | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index ffe947500..6f1a4fe01 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -465,9 +465,13 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags res, err := rpcRecvMsg(t, req) if err == nil { - n, e := dst.CopyOut(t, res.Data) - if e == nil && n != len(res.Data) { - panic("CopyOut failed to copy full buffer") + var e error + var n int + if len(res.Data) > 0 { + n, e = dst.CopyOut(t, res.Data) + if e == nil && n != len(res.Data) { + panic("CopyOut failed to copy full buffer") + } } return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } @@ -484,9 +488,13 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags for { res, err := rpcRecvMsg(t, req) if err == nil { - n, e := dst.CopyOut(t, res.Data) - if e == nil && n != len(res.Data) { - panic("CopyOut failed to copy full buffer") + var e error + var n int + if len(res.Data) > 0 { + n, e = dst.CopyOut(t, res.Data) + if e == nil && n != len(res.Data) { + panic("CopyOut failed to copy full buffer") + } } return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) } -- cgit v1.2.3 From c2b3f04d1c7b5d376a3fa305fc5e309e9ec81d99 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Tue, 12 Jun 2018 16:15:21 -0700 Subject: Rpcinet doensn't handle SO_RCVTIMEO properly. Rpcinet already inherits socket.ReceiveTimeout; however, it's never set on setsockopt(2). The value is currently forwarded as an RPC and ignored as all sockets will be non-blocking on the RPC side. PiperOrigin-RevId: 200299260 Change-Id: I6c610ea22c808ff6420c63759dccfaeab17959dd --- pkg/sentry/socket/rpcinet/socket.go | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 6f1a4fe01..4ef8b91c3 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -18,6 +18,7 @@ import ( "syscall" "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/binary" "gvisor.googlesource.com/gvisor/pkg/sentry/arch" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" @@ -318,6 +319,15 @@ func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { // GetSockOpt implements socket.Socket.GetSockOpt. func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outLen int) (interface{}, *syserr.Error) { + // SO_RCVTIMEO is special because blocking is performed within the sentry. + if level == linux.SOL_SOCKET && name == linux.SO_RCVTIMEO { + if outLen < linux.SizeOfTimeval { + return nil, syserr.ErrInvalidArgument + } + + return linux.NsecToTimeval(s.RecvTimeout()), nil + } + stack := t.NetworkContext().(*Stack) id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_GetSockOpt{&pb.GetSockOptRequest{Fd: s.fd, Level: int64(level), Name: int64(name), Length: uint32(outLen)}}}, false /* ignoreResult */) <-c @@ -332,6 +342,20 @@ func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outLe // SetSockOpt implements socket.Socket.SetSockOpt. func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error { + // Because blocking actually happens within the sentry we need to inspect + // this socket option to determine if it's a SO_RCVTIMEO, and if so, we will + // save it and use it as the deadline for recv(2) related syscalls. + if level == linux.SOL_SOCKET && name == linux.SO_RCVTIMEO { + if len(opt) < linux.SizeOfTimeval { + return syserr.ErrInvalidArgument + } + + var v linux.Timeval + binary.Unmarshal(opt[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + s.SetRecvTimeout(v.ToNsecCapped()) + return nil + } + stack := t.NetworkContext().(*Stack) id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_SetSockOpt{&pb.SetSockOptRequest{Fd: s.fd, Level: int64(level), Name: int64(name), Opt: opt}}}, false /* ignoreResult */) <-c -- cgit v1.2.3 From 1170039e788db368615451a0a1f5cfccb1d28d41 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Wed, 13 Jun 2018 16:20:30 -0700 Subject: Fix missing returns in rpcinet. PiperOrigin-RevId: 200472634 Change-Id: I3f0fb9e3b2f8616e6aa1569188258f330bf1ed31 --- pkg/sentry/socket/rpcinet/socket.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 4ef8b91c3..d8c1f2c1a 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -288,7 +288,7 @@ func (s *socketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error { <-c if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Bind).Bind.ErrorNumber; e != 0 { - syserr.FromHost(syscall.Errno(e)) + return syserr.FromHost(syscall.Errno(e)) } return nil } @@ -300,7 +300,7 @@ func (s *socketOperations) Listen(t *kernel.Task, backlog int) *syserr.Error { <-c if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Listen).Listen.ErrorNumber; e != 0 { - syserr.FromHost(syscall.Errno(e)) + return syserr.FromHost(syscall.Errno(e)) } return nil } @@ -361,7 +361,7 @@ func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt [ <-c if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_SetSockOpt).SetSockOpt.ErrorNumber; e != 0 { - syserr.FromHost(syscall.Errno(e)) + return syserr.FromHost(syscall.Errno(e)) } return nil } -- cgit v1.2.3 From 563a71ef243360bc20db0e481b3adbfb07cd8702 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Sun, 17 Jun 2018 17:05:36 -0700 Subject: Add rpcinet support for control messages. Add support for control messages, but at this time the only control message that the sentry will support here is SO_TIMESTAMP. PiperOrigin-RevId: 200922230 Change-Id: I63a852d9305255625d9df1d989bd46a66e93c446 --- pkg/sentry/socket/rpcinet/socket.go | 37 +++++++++++++++++++++++++++-- pkg/sentry/socket/rpcinet/syscall_rpc.proto | 2 ++ 2 files changed, 37 insertions(+), 2 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index d8c1f2c1a..b4b380ac6 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -477,6 +477,37 @@ func rpcRecvMsg(t *kernel.Task, req *pb.SyscallRequest_Recvmsg) (*pb.RecvmsgResp return res.(*pb.RecvmsgResponse_Payload).Payload, nil } +// Because we only support SO_TIMESTAMP we will search control messages for +// that value and set it if so, all other control messages will be ignored. +func (s *socketOperations) extractControlMessages(payload *pb.RecvmsgResponse_ResultPayload) socket.ControlMessages { + c := socket.ControlMessages{} + if len(payload.GetCmsgData()) > 0 { + // Parse the control messages looking for SO_TIMESTAMP. + msgs, e := syscall.ParseSocketControlMessage(payload.GetCmsgData()) + if e != nil { + return socket.ControlMessages{} + } + for _, m := range msgs { + if m.Header.Level != linux.SOL_SOCKET || m.Header.Type != linux.SO_TIMESTAMP { + continue + } + + // Let's parse the time stamp and set it. + if len(m.Data) < linux.SizeOfTimeval { + // Give up on locating the SO_TIMESTAMP option. + return socket.ControlMessages{} + } + + var v linux.Timeval + binary.Unmarshal(m.Data[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + c.IP.HasTimestamp = true + c.IP.Timestamp = v.ToNsecCapped() + break + } + } + return c +} + // RecvMsg implements socket.Socket.RecvMsg. func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { req := &pb.SyscallRequest_Recvmsg{&pb.RecvmsgRequest{ @@ -497,7 +528,8 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags panic("CopyOut failed to copy full buffer") } } - return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) + c := s.extractControlMessages(res) + return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), c, syserr.FromError(e) } if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain || flags&linux.MSG_DONTWAIT != 0 { return 0, nil, 0, socket.ControlMessages{}, err @@ -520,7 +552,8 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags panic("CopyOut failed to copy full buffer") } } - return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), socket.ControlMessages{}, syserr.FromError(e) + c := s.extractControlMessages(res) + return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), c, syserr.FromError(e) } if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain { return 0, nil, 0, socket.ControlMessages{}, err diff --git a/pkg/sentry/socket/rpcinet/syscall_rpc.proto b/pkg/sentry/socket/rpcinet/syscall_rpc.proto index 996962aae..c056e4c9d 100644 --- a/pkg/sentry/socket/rpcinet/syscall_rpc.proto +++ b/pkg/sentry/socket/rpcinet/syscall_rpc.proto @@ -40,6 +40,7 @@ message RecvmsgRequest { bool sender = 3; bool peek = 4; bool trunc = 5; + uint32 cmsg_length = 6; } message OpenRequest { @@ -110,6 +111,7 @@ message RecvmsgResponse { bytes data = 1 [ctype = CORD]; AddressResponse address = 2; uint32 length = 3; + bytes cmsg_data = 4; } oneof result { uint32 error_number = 1; -- cgit v1.2.3 From 4fd1d40e1d874ef4eb2f6cb13de66f1b756aa92c Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Tue, 19 Jun 2018 10:42:39 -0700 Subject: Rpcinet needs to track shutdown state for blocking sockets. Because rpcinet will emulate a blocking socket backed by an rpc based non-blocking socket. In the event of a shutdown(SHUT_RD) followed by a read a non-blocking socket is allowed to return an EWOULDBLOCK however since a blocking socket knows it cannot receive anymore data it would block indefinitely and in this situation linux returns 0. We have to track this on the rpcinet sentry side so we can emulate that behavior because the remote side has no way to know if the socket is actually blocking within the sentry. PiperOrigin-RevId: 201201618 Change-Id: I4ac3a7b74b5dae471ab97c2e7d33b83f425aedac --- pkg/sentry/socket/rpcinet/BUILD | 1 + pkg/sentry/socket/rpcinet/socket.go | 64 +++++++++++++++++++++++++++++++++---- 2 files changed, 59 insertions(+), 6 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index b0351b363..8973453f9 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -34,6 +34,7 @@ go_library( "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", + "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/transport/unix", "//pkg/unet", diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index b4b380ac6..f641f25df 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -33,6 +33,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/tcpip" "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" @@ -52,6 +53,11 @@ type socketOperations struct { wq *waiter.Queue rpcConn *conn.RPCConnection notifier *notifier.Notifier + + // shState is the state of the connection with respect to shutdown. Because + // we're mixing non-blocking semantics on the other side we have to adapt for + // some strange differences between blocking and non-blocking sockets. + shState tcpip.ShutdownFlags } // Verify that we actually implement socket.Socket. @@ -96,6 +102,31 @@ func translateIOSyscallError(err error) error { return err } +// setShutdownFlags will set the shutdown flag so we can handle blocking reads +// after a read shutdown. +func (s *socketOperations) setShutdownFlags(how int) { + switch how { + case linux.SHUT_RD: + s.shState |= tcpip.ShutdownRead + case linux.SHUT_WR: + s.shState |= tcpip.ShutdownWrite + case linux.SHUT_RDWR: + s.shState |= tcpip.ShutdownWrite | tcpip.ShutdownRead + } +} + +func (s *socketOperations) resetShutdownFlags() { + s.shState = 0 +} + +func (s *socketOperations) isShutRdSet() bool { + return s.shState&tcpip.ShutdownRead != 0 +} + +func (s *socketOperations) isShutWrSet() bool { + return s.shState&tcpip.ShutdownWrite != 0 +} + // Release implements fs.FileOperations.Release. func (s *socketOperations) Release() { s.notifier.RemoveFD(s.fd) @@ -191,7 +222,12 @@ func rpcConnect(t *kernel.Task, fd uint32, sockaddr []byte) *syserr.Error { // Connect implements socket.Socket.Connect. func (s *socketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error { if !blocking { - return rpcConnect(t, s.fd, sockaddr) + e := rpcConnect(t, s.fd, sockaddr) + if e == nil { + // Reset the shutdown state on new connects. + s.resetShutdownFlags() + } + return e } // Register for notification when the endpoint becomes writable, then @@ -201,6 +237,10 @@ func (s *socketOperations) Connect(t *kernel.Task, sockaddr []byte, blocking boo defer s.EventUnregister(&e) for { if err := rpcConnect(t, s.fd, sockaddr); err == nil || err != syserr.ErrInProgress && err != syserr.ErrAlreadyInProgress { + if err == nil { + // Reset the shutdown state on new connects. + s.resetShutdownFlags() + } return err } @@ -314,6 +354,11 @@ func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { if e := stack.rpcConn.Request(id).Result.(*pb.SyscallResponse_Shutdown).Shutdown.ErrorNumber; e != 0 { return syserr.FromHost(syscall.Errno(e)) } + + // We save the shutdown state because of strange differences on linux + // related to recvs on blocking vs. non-blocking sockets after a SHUT_RD. + // We need to emulate that behavior on the blocking side. + s.setShutdownFlags(how) return nil } @@ -511,11 +556,12 @@ func (s *socketOperations) extractControlMessages(payload *pb.RecvmsgResponse_Re // RecvMsg implements socket.Socket.RecvMsg. func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { req := &pb.SyscallRequest_Recvmsg{&pb.RecvmsgRequest{ - Fd: s.fd, - Length: uint32(dst.NumBytes()), - Sender: senderRequested, - Trunc: flags&linux.MSG_TRUNC != 0, - Peek: flags&linux.MSG_PEEK != 0, + Fd: s.fd, + Length: uint32(dst.NumBytes()), + Sender: senderRequested, + Trunc: flags&linux.MSG_TRUNC != 0, + Peek: flags&linux.MSG_PEEK != 0, + CmsgLength: uint32(controlDataLen), }} res, err := rpcRecvMsg(t, req) @@ -559,6 +605,12 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags return 0, nil, 0, socket.ControlMessages{}, err } + if s.isShutRdSet() { + // Blocking would have caused us to block indefinitely so we return 0, + // this is the same behavior as Linux. + return 0, nil, 0, socket.ControlMessages{}, nil + } + if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { if err == syserror.ETIMEDOUT { return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain -- cgit v1.2.3 From bda2a1ed3503699b8cb814bb3cc7ad0b9694155b Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Tue, 19 Jun 2018 14:11:58 -0700 Subject: Rpcinet is racy around shutdown flags. Correct a data race in rpcinet where a shutdown and recvmsg can race around shutown flags. PiperOrigin-RevId: 201238366 Change-Id: I5eb06df4a2b4eba331eeb5de19076213081d581f --- pkg/sentry/socket/rpcinet/socket.go | 24 +++++++++++++++++------- 1 file changed, 17 insertions(+), 7 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index f641f25df..207123d6f 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -15,6 +15,7 @@ package rpcinet import ( + "sync/atomic" "syscall" "gvisor.googlesource.com/gvisor/pkg/abi/linux" @@ -57,7 +58,7 @@ type socketOperations struct { // shState is the state of the connection with respect to shutdown. Because // we're mixing non-blocking semantics on the other side we have to adapt for // some strange differences between blocking and non-blocking sockets. - shState tcpip.ShutdownFlags + shState int32 } // Verify that we actually implement socket.Socket. @@ -105,26 +106,35 @@ func translateIOSyscallError(err error) error { // setShutdownFlags will set the shutdown flag so we can handle blocking reads // after a read shutdown. func (s *socketOperations) setShutdownFlags(how int) { + var f tcpip.ShutdownFlags switch how { case linux.SHUT_RD: - s.shState |= tcpip.ShutdownRead + f = tcpip.ShutdownRead case linux.SHUT_WR: - s.shState |= tcpip.ShutdownWrite + f = tcpip.ShutdownWrite case linux.SHUT_RDWR: - s.shState |= tcpip.ShutdownWrite | tcpip.ShutdownRead + f = tcpip.ShutdownWrite | tcpip.ShutdownRead + } + + // Atomically update the flags. + for { + old := atomic.LoadInt32(&s.shState) + if atomic.CompareAndSwapInt32(&s.shState, old, old|int32(f)) { + break + } } } func (s *socketOperations) resetShutdownFlags() { - s.shState = 0 + atomic.StoreInt32(&s.shState, 0) } func (s *socketOperations) isShutRdSet() bool { - return s.shState&tcpip.ShutdownRead != 0 + return atomic.LoadInt32(&s.shState)&int32(tcpip.ShutdownRead) != 0 } func (s *socketOperations) isShutWrSet() bool { - return s.shState&tcpip.ShutdownWrite != 0 + return atomic.LoadInt32(&s.shState)&int32(tcpip.ShutdownWrite) != 0 } // Release implements fs.FileOperations.Release. -- cgit v1.2.3 From 7c645ac27355a9d7016e0d5c74ce70eed2add600 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Fri, 22 Jun 2018 14:47:15 -0700 Subject: Add rpcinet support for SIOCGIFCONF. The interfaces and their addresses are already available via the stack Intefaces and InterfaceAddrs. Also add some tests as we had no tests around SIOCGIFCONF. I also added the socket_netgofer lifecycle for IOCTL tests. PiperOrigin-RevId: 201744863 Change-Id: Ie0a285a2a2f859fa0cafada13201d5941b95499a --- pkg/sentry/socket/rpcinet/socket.go | 73 ++++++++++++++++++++++++++++++++++--- 1 file changed, 68 insertions(+), 5 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 207123d6f..72fa1ca8f 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -64,10 +64,6 @@ type socketOperations struct { // Verify that we actually implement socket.Socket. var _ = socket.Socket(&socketOperations{}) -const ( - sizeOfIfReq = 40 -) - // New creates a new RPC socket. func newSocketFile(ctx context.Context, stack *Stack, family int, skType int, protocol int) (*fs.File, *syserr.Error) { id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Socket{&pb.SocketRequest{Family: int64(family), Type: int64(skType | syscall.SOCK_NONBLOCK), Protocol: int64(protocol)}}}, false /* ignoreResult */) @@ -465,6 +461,55 @@ func rpcIoctl(t *kernel.Task, fd, cmd uint32, arg []byte) ([]byte, error) { return res.(*pb.IOCtlResponse_Value).Value, nil } +// ifconfIoctlFromStack populates a struct ifconf for the SIOCGIFCONF ioctl. +func ifconfIoctlFromStack(ctx context.Context, io usermem.IO, ifc *linux.IFConf) error { + // If Ptr is NULL, return the necessary buffer size via Len. + // Otherwise, write up to Len bytes starting at Ptr containing ifreq + // structs. + t := ctx.(*kernel.Task) + s := t.NetworkContext().(*Stack) + if s == nil { + return syserr.ErrNoDevice.ToError() + } + + if ifc.Ptr == 0 { + ifc.Len = int32(len(s.Interfaces())) * int32(linux.SizeOfIFReq) + return nil + } + + max := ifc.Len + ifc.Len = 0 + for key, ifaceAddrs := range s.InterfaceAddrs() { + iface := s.Interfaces()[key] + for _, ifaceAddr := range ifaceAddrs { + // Don't write past the end of the buffer. + if ifc.Len+int32(linux.SizeOfIFReq) > max { + break + } + if ifaceAddr.Family != linux.AF_INET { + continue + } + + // Populate ifr.ifr_addr. + ifr := linux.IFReq{} + ifr.SetName(iface.Name) + usermem.ByteOrder.PutUint16(ifr.Data[0:2], uint16(ifaceAddr.Family)) + usermem.ByteOrder.PutUint16(ifr.Data[2:4], 0) + copy(ifr.Data[4:8], ifaceAddr.Addr[:4]) + + // Copy the ifr to userspace. + dst := uintptr(ifc.Ptr) + uintptr(ifc.Len) + ifc.Len += int32(linux.SizeOfIFReq) + if _, err := usermem.CopyObjectOut(ctx, io, usermem.Addr(dst), ifr, usermem.IOOpts{ + AddressSpaceActive: true, + }); err != nil { + return err + } + } + } + return nil +} + // Ioctl implements fs.FileOperations.Ioctl. func (s *socketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { t := ctx.(*kernel.Task) @@ -491,7 +536,25 @@ func (s *socketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.S syscall.SIOCGIFNAME, syscall.SIOCGIFNETMASK, syscall.SIOCGIFTXQLEN: - buf = make([]byte, sizeOfIfReq) + buf = make([]byte, linux.SizeOfIFReq) + case syscall.SIOCGIFCONF: + // SIOCGIFCONF has slightly different behavior than the others, in that it + // will need to populate the array of ifreqs. + var ifc linux.IFConf + if _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &ifc, usermem.IOOpts{ + AddressSpaceActive: true, + }); err != nil { + return 0, err + } + + if err := ifconfIoctlFromStack(ctx, io, &ifc); err != nil { + return 0, err + } + _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), ifc, usermem.IOOpts{ + AddressSpaceActive: true, + }) + + return 0, err default: return 0, syserror.ENOTTY } -- cgit v1.2.3 From be7fcbc5582fe831b5ec63f773d867d7591e27a1 Mon Sep 17 00:00:00 2001 From: Zhaozhong Ni Date: Fri, 27 Jul 2018 10:16:27 -0700 Subject: stateify: support explicit annotation mode; convert refs and stack packages. We have been unnecessarily creating too many savable types implicitly. PiperOrigin-RevId: 206334201 Change-Id: Idc5a3a14bfb7ee125c4f2bb2b1c53164e46f29a8 --- pkg/abi/BUILD | 3 +- pkg/abi/linux/BUILD | 3 +- pkg/amutex/BUILD | 2 +- pkg/atomicbitops/BUILD | 2 +- pkg/binary/BUILD | 2 +- pkg/bits/BUILD | 2 +- pkg/bpf/BUILD | 3 +- pkg/compressio/BUILD | 2 +- pkg/control/client/BUILD | 2 +- pkg/control/server/BUILD | 2 +- pkg/cpuid/BUILD | 3 +- pkg/dhcp/BUILD | 2 +- pkg/eventchannel/BUILD | 2 +- pkg/fd/BUILD | 2 +- pkg/gate/BUILD | 2 +- pkg/hashio/BUILD | 2 +- pkg/ilist/BUILD | 3 +- pkg/linewriter/BUILD | 2 +- pkg/log/BUILD | 2 +- pkg/metric/BUILD | 2 +- pkg/p9/BUILD | 2 +- pkg/p9/p9test/BUILD | 2 +- pkg/rand/BUILD | 2 +- pkg/refs/BUILD | 20 +------- pkg/refs/refcounter.go | 4 ++ pkg/refs/refcounter_state.go | 1 + pkg/seccomp/BUILD | 3 +- pkg/secio/BUILD | 2 +- pkg/segment/test/BUILD | 2 +- pkg/sentry/arch/BUILD | 3 +- pkg/sentry/context/BUILD | 2 +- pkg/sentry/context/contexttest/BUILD | 3 +- pkg/sentry/control/BUILD | 2 +- pkg/sentry/device/BUILD | 2 +- pkg/sentry/fs/BUILD | 3 +- pkg/sentry/fs/anon/BUILD | 2 +- pkg/sentry/fs/ashmem/BUILD | 3 +- pkg/sentry/fs/binder/BUILD | 3 +- pkg/sentry/fs/dev/BUILD | 3 +- pkg/sentry/fs/fdpipe/BUILD | 3 +- pkg/sentry/fs/filetest/BUILD | 3 +- pkg/sentry/fs/fsutil/BUILD | 3 +- pkg/sentry/fs/gofer/BUILD | 3 +- pkg/sentry/fs/host/BUILD | 3 +- pkg/sentry/fs/lock/BUILD | 3 +- pkg/sentry/fs/proc/BUILD | 3 +- pkg/sentry/fs/proc/device/BUILD | 2 +- pkg/sentry/fs/proc/seqfile/BUILD | 3 +- pkg/sentry/fs/ramfs/BUILD | 3 +- pkg/sentry/fs/ramfs/test/BUILD | 3 +- pkg/sentry/fs/sys/BUILD | 3 +- pkg/sentry/fs/timerfd/BUILD | 3 +- pkg/sentry/fs/tmpfs/BUILD | 3 +- pkg/sentry/fs/tty/BUILD | 3 +- pkg/sentry/hostcpu/BUILD | 2 +- pkg/sentry/inet/BUILD | 4 +- pkg/sentry/kernel/BUILD | 3 +- pkg/sentry/kernel/auth/BUILD | 3 +- pkg/sentry/kernel/epoll/BUILD | 3 +- pkg/sentry/kernel/eventfd/BUILD | 3 +- pkg/sentry/kernel/fasync/BUILD | 2 +- pkg/sentry/kernel/futex/BUILD | 3 +- pkg/sentry/kernel/kdefs/BUILD | 2 +- pkg/sentry/kernel/memevent/BUILD | 2 +- pkg/sentry/kernel/pipe/BUILD | 3 +- pkg/sentry/kernel/sched/BUILD | 2 +- pkg/sentry/kernel/semaphore/BUILD | 3 +- pkg/sentry/kernel/shm/BUILD | 3 +- pkg/sentry/kernel/time/BUILD | 3 +- pkg/sentry/limits/BUILD | 3 +- pkg/sentry/loader/BUILD | 4 +- pkg/sentry/memmap/BUILD | 3 +- pkg/sentry/memutil/BUILD | 2 +- pkg/sentry/mm/BUILD | 3 +- pkg/sentry/platform/BUILD | 3 +- pkg/sentry/platform/filemem/BUILD | 3 +- pkg/sentry/platform/interrupt/BUILD | 2 +- pkg/sentry/platform/kvm/BUILD | 2 +- pkg/sentry/platform/kvm/testutil/BUILD | 2 +- pkg/sentry/platform/procid/BUILD | 2 +- pkg/sentry/platform/ptrace/BUILD | 2 +- pkg/sentry/platform/ring0/BUILD | 2 +- pkg/sentry/platform/ring0/pagetables/BUILD | 2 +- pkg/sentry/platform/safecopy/BUILD | 2 +- pkg/sentry/safemem/BUILD | 2 +- pkg/sentry/sighandling/BUILD | 2 +- pkg/sentry/socket/BUILD | 3 +- pkg/sentry/socket/control/BUILD | 3 +- pkg/sentry/socket/epsocket/BUILD | 3 +- pkg/sentry/socket/hostinet/BUILD | 3 +- pkg/sentry/socket/netlink/BUILD | 3 +- pkg/sentry/socket/netlink/port/BUILD | 3 +- pkg/sentry/socket/netlink/route/BUILD | 3 +- pkg/sentry/socket/rpcinet/BUILD | 2 +- pkg/sentry/socket/rpcinet/conn/BUILD | 2 +- pkg/sentry/socket/rpcinet/notifier/BUILD | 2 +- pkg/sentry/socket/unix/BUILD | 3 +- pkg/sentry/state/BUILD | 2 +- pkg/sentry/strace/BUILD | 2 +- pkg/sentry/syscalls/BUILD | 2 +- pkg/sentry/syscalls/linux/BUILD | 3 +- pkg/sentry/time/BUILD | 2 +- pkg/sentry/uniqueid/BUILD | 2 +- pkg/sentry/usage/BUILD | 3 +- pkg/sentry/usermem/BUILD | 3 +- pkg/sentry/watchdog/BUILD | 2 +- pkg/sleep/BUILD | 2 +- pkg/state/BUILD | 2 +- pkg/state/statefile/BUILD | 2 +- pkg/sync/BUILD | 2 +- pkg/sync/seqatomictest/BUILD | 2 +- pkg/syserr/BUILD | 2 +- pkg/syserror/BUILD | 2 +- pkg/tcpip/BUILD | 3 +- pkg/tcpip/adapters/gonet/BUILD | 2 +- pkg/tcpip/buffer/BUILD | 3 +- pkg/tcpip/checker/BUILD | 2 +- pkg/tcpip/header/BUILD | 3 +- pkg/tcpip/link/channel/BUILD | 2 +- pkg/tcpip/link/fdbased/BUILD | 2 +- pkg/tcpip/link/loopback/BUILD | 2 +- pkg/tcpip/link/rawfile/BUILD | 2 +- pkg/tcpip/link/sharedmem/BUILD | 2 +- pkg/tcpip/link/sharedmem/pipe/BUILD | 2 +- pkg/tcpip/link/sharedmem/queue/BUILD | 2 +- pkg/tcpip/link/sniffer/BUILD | 2 +- pkg/tcpip/link/tun/BUILD | 2 +- pkg/tcpip/link/waitable/BUILD | 2 +- pkg/tcpip/network/BUILD | 2 +- pkg/tcpip/network/arp/BUILD | 2 +- pkg/tcpip/network/fragmentation/BUILD | 3 +- pkg/tcpip/network/hash/BUILD | 2 +- pkg/tcpip/network/ipv4/BUILD | 2 +- pkg/tcpip/network/ipv6/BUILD | 2 +- pkg/tcpip/ports/BUILD | 2 +- pkg/tcpip/seqnum/BUILD | 3 +- pkg/tcpip/stack/BUILD | 15 +----- pkg/tcpip/stack/registration.go | 2 + pkg/tcpip/transport/ping/BUILD | 3 +- pkg/tcpip/transport/queue/BUILD | 3 +- pkg/tcpip/transport/tcp/BUILD | 3 +- pkg/tcpip/transport/tcp/testing/context/BUILD | 2 +- pkg/tcpip/transport/tcpconntrack/BUILD | 2 +- pkg/tcpip/transport/udp/BUILD | 3 +- pkg/tcpip/transport/unix/BUILD | 3 +- pkg/tmutex/BUILD | 2 +- pkg/unet/BUILD | 2 +- pkg/urpc/BUILD | 2 +- pkg/waiter/BUILD | 3 +- pkg/waiter/fdnotifier/BUILD | 2 +- tools/go_stateify/defs.bzl | 58 ++++++++++++++++++----- tools/go_stateify/main.go | 66 +++++++++++++++++++++------ 152 files changed, 255 insertions(+), 267 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/abi/BUILD b/pkg/abi/BUILD index 4d507161f..f1e6bac67 100644 --- a/pkg/abi/BUILD +++ b/pkg/abi/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "abi_state", diff --git a/pkg/abi/linux/BUILD b/pkg/abi/linux/BUILD index ae7e4378c..38b4829c9 100644 --- a/pkg/abi/linux/BUILD +++ b/pkg/abi/linux/BUILD @@ -4,8 +4,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "linux_state", diff --git a/pkg/amutex/BUILD b/pkg/amutex/BUILD index 442096319..84e6b79a5 100644 --- a/pkg/amutex/BUILD +++ b/pkg/amutex/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "amutex", diff --git a/pkg/atomicbitops/BUILD b/pkg/atomicbitops/BUILD index f20a9f855..a8dd17825 100644 --- a/pkg/atomicbitops/BUILD +++ b/pkg/atomicbitops/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "atomicbitops", diff --git a/pkg/binary/BUILD b/pkg/binary/BUILD index 16f08b13f..586d05634 100644 --- a/pkg/binary/BUILD +++ b/pkg/binary/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "binary", diff --git a/pkg/bits/BUILD b/pkg/bits/BUILD index 9897e5dc3..8c943b615 100644 --- a/pkg/bits/BUILD +++ b/pkg/bits/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_library( diff --git a/pkg/bpf/BUILD b/pkg/bpf/BUILD index d4f12f13a..403270049 100644 --- a/pkg/bpf/BUILD +++ b/pkg/bpf/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "bpf_state", diff --git a/pkg/compressio/BUILD b/pkg/compressio/BUILD index 721b2d983..d70f982c1 100644 --- a/pkg/compressio/BUILD +++ b/pkg/compressio/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "compressio", diff --git a/pkg/control/client/BUILD b/pkg/control/client/BUILD index 9e1c058e4..d58cd1b71 100644 --- a/pkg/control/client/BUILD +++ b/pkg/control/client/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "client", diff --git a/pkg/control/server/BUILD b/pkg/control/server/BUILD index 2d0fdd8b8..c3f74a532 100644 --- a/pkg/control/server/BUILD +++ b/pkg/control/server/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "server", diff --git a/pkg/cpuid/BUILD b/pkg/cpuid/BUILD index a503b7ae8..9a0ca1b33 100644 --- a/pkg/cpuid/BUILD +++ b/pkg/cpuid/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "cpuid_state", diff --git a/pkg/dhcp/BUILD b/pkg/dhcp/BUILD index f56969ad8..bd9f592b4 100644 --- a/pkg/dhcp/BUILD +++ b/pkg/dhcp/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "dhcp", diff --git a/pkg/eventchannel/BUILD b/pkg/eventchannel/BUILD index ea0c587be..ac2ea869d 100644 --- a/pkg/eventchannel/BUILD +++ b/pkg/eventchannel/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "eventchannel", diff --git a/pkg/fd/BUILD b/pkg/fd/BUILD index e69d83d06..435b6fa34 100644 --- a/pkg/fd/BUILD +++ b/pkg/fd/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "fd", diff --git a/pkg/gate/BUILD b/pkg/gate/BUILD index 0b8b01da8..872eff531 100644 --- a/pkg/gate/BUILD +++ b/pkg/gate/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "gate", diff --git a/pkg/hashio/BUILD b/pkg/hashio/BUILD index aaa58b58f..5736e2e73 100644 --- a/pkg/hashio/BUILD +++ b/pkg/hashio/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "hashio", diff --git a/pkg/ilist/BUILD b/pkg/ilist/BUILD index 16a738e89..e32f26ffa 100644 --- a/pkg/ilist/BUILD +++ b/pkg/ilist/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "list_state", diff --git a/pkg/linewriter/BUILD b/pkg/linewriter/BUILD index 4a96c6f1d..6c3795432 100644 --- a/pkg/linewriter/BUILD +++ b/pkg/linewriter/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "linewriter", diff --git a/pkg/log/BUILD b/pkg/log/BUILD index 2530cfd18..fc9281079 100644 --- a/pkg/log/BUILD +++ b/pkg/log/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "log", diff --git a/pkg/metric/BUILD b/pkg/metric/BUILD index e3f50d528..c0cd40c7b 100644 --- a/pkg/metric/BUILD +++ b/pkg/metric/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "metric", diff --git a/pkg/p9/BUILD b/pkg/p9/BUILD index f348ff2e9..1cf5c6458 100644 --- a/pkg/p9/BUILD +++ b/pkg/p9/BUILD @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//visibility:public"], diff --git a/pkg/p9/p9test/BUILD b/pkg/p9/p9test/BUILD index 339c86089..d6f428e11 100644 --- a/pkg/p9/p9test/BUILD +++ b/pkg/p9/p9test/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_test( name = "p9test_test", diff --git a/pkg/rand/BUILD b/pkg/rand/BUILD index 2bb59f895..12e6cf25a 100644 --- a/pkg/rand/BUILD +++ b/pkg/rand/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "rand", diff --git a/pkg/refs/BUILD b/pkg/refs/BUILD index 4b7c9345d..3ea877ccf 100644 --- a/pkg/refs/BUILD +++ b/pkg/refs/BUILD @@ -1,32 +1,16 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") - -go_stateify( - name = "refs_state", - srcs = [ - "refcounter.go", - "refcounter_state.go", - ], - out = "refs_state.go", - package = "refs", -) +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "refs", srcs = [ "refcounter.go", "refcounter_state.go", - "refs_state.go", ], importpath = "gvisor.googlesource.com/gvisor/pkg/refs", visibility = ["//:sandbox"], - deps = [ - "//pkg/ilist", - "//pkg/log", - "//pkg/state", - ], + deps = ["//pkg/ilist"], ) go_test( diff --git a/pkg/refs/refcounter.go b/pkg/refs/refcounter.go index 3162001e1..0d44c2499 100644 --- a/pkg/refs/refcounter.go +++ b/pkg/refs/refcounter.go @@ -58,6 +58,8 @@ type WeakRefUser interface { } // WeakRef is a weak reference. +// +// +stateify savable type WeakRef struct { ilist.Entry `state:"nosave"` @@ -177,6 +179,8 @@ func (w *WeakRef) zap() { // // N.B. To allow the zero-object to be initialized, the count is offset by // 1, that is, when refCount is n, there are really n+1 references. +// +// +stateify savable type AtomicRefCount struct { // refCount is composed of two fields: // diff --git a/pkg/refs/refcounter_state.go b/pkg/refs/refcounter_state.go index 1be67f951..093eae785 100644 --- a/pkg/refs/refcounter_state.go +++ b/pkg/refs/refcounter_state.go @@ -14,6 +14,7 @@ package refs +// +stateify savable type savedReference struct { obj interface{} } diff --git a/pkg/seccomp/BUILD b/pkg/seccomp/BUILD index cadd24505..b3e2f0b38 100644 --- a/pkg/seccomp/BUILD +++ b/pkg/seccomp/BUILD @@ -1,6 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_embed_data", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_embed_data") go_binary( name = "victim", diff --git a/pkg/secio/BUILD b/pkg/secio/BUILD index 9a28d2c1f..0ed38c64a 100644 --- a/pkg/secio/BUILD +++ b/pkg/secio/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "secio", diff --git a/pkg/segment/test/BUILD b/pkg/segment/test/BUILD index 9d398d71a..bdf53e24e 100644 --- a/pkg/segment/test/BUILD +++ b/pkg/segment/test/BUILD @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//visibility:private"], diff --git a/pkg/sentry/arch/BUILD b/pkg/sentry/arch/BUILD index a88f57ac7..0a2a35400 100644 --- a/pkg/sentry/arch/BUILD +++ b/pkg/sentry/arch/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "arch_state", diff --git a/pkg/sentry/context/BUILD b/pkg/sentry/context/BUILD index ff39f94ba..2a7a6df23 100644 --- a/pkg/sentry/context/BUILD +++ b/pkg/sentry/context/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "context", diff --git a/pkg/sentry/context/contexttest/BUILD b/pkg/sentry/context/contexttest/BUILD index 5977344de..591b11a4d 100644 --- a/pkg/sentry/context/contexttest/BUILD +++ b/pkg/sentry/context/contexttest/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "contexttest_state", diff --git a/pkg/sentry/control/BUILD b/pkg/sentry/control/BUILD index 6169891f7..fbdde0721 100644 --- a/pkg/sentry/control/BUILD +++ b/pkg/sentry/control/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "control", diff --git a/pkg/sentry/device/BUILD b/pkg/sentry/device/BUILD index 1a8b461ba..69c99b0b3 100644 --- a/pkg/sentry/device/BUILD +++ b/pkg/sentry/device/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "device", diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD index 9b7264753..e3c9a9b70 100644 --- a/pkg/sentry/fs/BUILD +++ b/pkg/sentry/fs/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "fs_state", diff --git a/pkg/sentry/fs/anon/BUILD b/pkg/sentry/fs/anon/BUILD index 6b18aee47..ff4ab850a 100644 --- a/pkg/sentry/fs/anon/BUILD +++ b/pkg/sentry/fs/anon/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "anon", diff --git a/pkg/sentry/fs/ashmem/BUILD b/pkg/sentry/fs/ashmem/BUILD index e20e22a0f..9f166799a 100644 --- a/pkg/sentry/fs/ashmem/BUILD +++ b/pkg/sentry/fs/ashmem/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_stateify( diff --git a/pkg/sentry/fs/binder/BUILD b/pkg/sentry/fs/binder/BUILD index 15f91699f..ec3928baf 100644 --- a/pkg/sentry/fs/binder/BUILD +++ b/pkg/sentry/fs/binder/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "binder_state", diff --git a/pkg/sentry/fs/dev/BUILD b/pkg/sentry/fs/dev/BUILD index d33a19c2f..ea41615fd 100644 --- a/pkg/sentry/fs/dev/BUILD +++ b/pkg/sentry/fs/dev/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "dev_state", diff --git a/pkg/sentry/fs/fdpipe/BUILD b/pkg/sentry/fs/fdpipe/BUILD index 9e1f65d3e..4fcb06f1f 100644 --- a/pkg/sentry/fs/fdpipe/BUILD +++ b/pkg/sentry/fs/fdpipe/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "pipe_state", diff --git a/pkg/sentry/fs/filetest/BUILD b/pkg/sentry/fs/filetest/BUILD index 51a390d77..f481c57fb 100644 --- a/pkg/sentry/fs/filetest/BUILD +++ b/pkg/sentry/fs/filetest/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "filetest_state", diff --git a/pkg/sentry/fs/fsutil/BUILD b/pkg/sentry/fs/fsutil/BUILD index 4fa6395f7..6eea64298 100644 --- a/pkg/sentry/fs/fsutil/BUILD +++ b/pkg/sentry/fs/fsutil/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "fsutil_state", diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD index e6f659c53..1277379e7 100644 --- a/pkg/sentry/fs/gofer/BUILD +++ b/pkg/sentry/fs/gofer/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "gofer_state", diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD index 97b64daed..23ec66f50 100644 --- a/pkg/sentry/fs/host/BUILD +++ b/pkg/sentry/fs/host/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "host_state", diff --git a/pkg/sentry/fs/lock/BUILD b/pkg/sentry/fs/lock/BUILD index c15dde800..2607d7ed3 100644 --- a/pkg/sentry/fs/lock/BUILD +++ b/pkg/sentry/fs/lock/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "lock_state", diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD index 21b5fc0c3..870df47b2 100644 --- a/pkg/sentry/fs/proc/BUILD +++ b/pkg/sentry/fs/proc/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "proc_state", diff --git a/pkg/sentry/fs/proc/device/BUILD b/pkg/sentry/fs/proc/device/BUILD index b62062bd7..34582f275 100644 --- a/pkg/sentry/fs/proc/device/BUILD +++ b/pkg/sentry/fs/proc/device/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "device", diff --git a/pkg/sentry/fs/proc/seqfile/BUILD b/pkg/sentry/fs/proc/seqfile/BUILD index 48dd25e5b..c84f7e20d 100644 --- a/pkg/sentry/fs/proc/seqfile/BUILD +++ b/pkg/sentry/fs/proc/seqfile/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "seqfile_state", diff --git a/pkg/sentry/fs/ramfs/BUILD b/pkg/sentry/fs/ramfs/BUILD index 663a1aeb9..d84f2c624 100644 --- a/pkg/sentry/fs/ramfs/BUILD +++ b/pkg/sentry/fs/ramfs/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "ramfs_state", diff --git a/pkg/sentry/fs/ramfs/test/BUILD b/pkg/sentry/fs/ramfs/test/BUILD index 074b0f5ad..57fee45e2 100644 --- a/pkg/sentry/fs/ramfs/test/BUILD +++ b/pkg/sentry/fs/ramfs/test/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "test_state", diff --git a/pkg/sentry/fs/sys/BUILD b/pkg/sentry/fs/sys/BUILD index 0ae2cbac8..095ff1f25 100644 --- a/pkg/sentry/fs/sys/BUILD +++ b/pkg/sentry/fs/sys/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "sys_state", diff --git a/pkg/sentry/fs/timerfd/BUILD b/pkg/sentry/fs/timerfd/BUILD index 7fddc29f4..8b1b7872e 100644 --- a/pkg/sentry/fs/timerfd/BUILD +++ b/pkg/sentry/fs/timerfd/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "timerfd_state", diff --git a/pkg/sentry/fs/tmpfs/BUILD b/pkg/sentry/fs/tmpfs/BUILD index be4e695d3..473ab4296 100644 --- a/pkg/sentry/fs/tmpfs/BUILD +++ b/pkg/sentry/fs/tmpfs/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tmpfs_state", diff --git a/pkg/sentry/fs/tty/BUILD b/pkg/sentry/fs/tty/BUILD index fce327dfe..363897b2c 100644 --- a/pkg/sentry/fs/tty/BUILD +++ b/pkg/sentry/fs/tty/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tty_state", diff --git a/pkg/sentry/hostcpu/BUILD b/pkg/sentry/hostcpu/BUILD index 9457618d8..f362d15c8 100644 --- a/pkg/sentry/hostcpu/BUILD +++ b/pkg/sentry/hostcpu/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "hostcpu", diff --git a/pkg/sentry/inet/BUILD b/pkg/sentry/inet/BUILD index 1150ced57..eaf8f15b2 100644 --- a/pkg/sentry/inet/BUILD +++ b/pkg/sentry/inet/BUILD @@ -1,11 +1,9 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - package( default_visibility = ["//:sandbox"], licenses = ["notice"], # Apache 2.0 ) -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "inet_state", diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD index 07568b47c..c4a7dacb2 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "kernel_state", diff --git a/pkg/sentry/kernel/auth/BUILD b/pkg/sentry/kernel/auth/BUILD index 7f0680b88..5b7b30557 100644 --- a/pkg/sentry/kernel/auth/BUILD +++ b/pkg/sentry/kernel/auth/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "auth_state", diff --git a/pkg/sentry/kernel/epoll/BUILD b/pkg/sentry/kernel/epoll/BUILD index 04651d961..7d491efbc 100644 --- a/pkg/sentry/kernel/epoll/BUILD +++ b/pkg/sentry/kernel/epoll/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "epoll_autogen_state", diff --git a/pkg/sentry/kernel/eventfd/BUILD b/pkg/sentry/kernel/eventfd/BUILD index 561ced852..7ec179bd8 100644 --- a/pkg/sentry/kernel/eventfd/BUILD +++ b/pkg/sentry/kernel/eventfd/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "eventfd_state", diff --git a/pkg/sentry/kernel/fasync/BUILD b/pkg/sentry/kernel/fasync/BUILD index 8d06e1182..17749c0de 100644 --- a/pkg/sentry/kernel/fasync/BUILD +++ b/pkg/sentry/kernel/fasync/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "fasync", diff --git a/pkg/sentry/kernel/futex/BUILD b/pkg/sentry/kernel/futex/BUILD index de9897c58..a97a43549 100644 --- a/pkg/sentry/kernel/futex/BUILD +++ b/pkg/sentry/kernel/futex/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_template_instance( name = "waiter_list", diff --git a/pkg/sentry/kernel/kdefs/BUILD b/pkg/sentry/kernel/kdefs/BUILD index b6c00042a..fe6fa2260 100644 --- a/pkg/sentry/kernel/kdefs/BUILD +++ b/pkg/sentry/kernel/kdefs/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "kdefs", diff --git a/pkg/sentry/kernel/memevent/BUILD b/pkg/sentry/kernel/memevent/BUILD index c7779e1d5..66899910c 100644 --- a/pkg/sentry/kernel/memevent/BUILD +++ b/pkg/sentry/kernel/memevent/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "memevent", diff --git a/pkg/sentry/kernel/pipe/BUILD b/pkg/sentry/kernel/pipe/BUILD index ca9825f9d..4600d19bd 100644 --- a/pkg/sentry/kernel/pipe/BUILD +++ b/pkg/sentry/kernel/pipe/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "pipe_state", diff --git a/pkg/sentry/kernel/sched/BUILD b/pkg/sentry/kernel/sched/BUILD index b533c51c4..125792f39 100644 --- a/pkg/sentry/kernel/sched/BUILD +++ b/pkg/sentry/kernel/sched/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "sched", diff --git a/pkg/sentry/kernel/semaphore/BUILD b/pkg/sentry/kernel/semaphore/BUILD index 1656ad126..969145fe1 100644 --- a/pkg/sentry/kernel/semaphore/BUILD +++ b/pkg/sentry/kernel/semaphore/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_template_instance( name = "waiter_list", diff --git a/pkg/sentry/kernel/shm/BUILD b/pkg/sentry/kernel/shm/BUILD index 182cc1c76..0f88eb0ac 100644 --- a/pkg/sentry/kernel/shm/BUILD +++ b/pkg/sentry/kernel/shm/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "shm_state", diff --git a/pkg/sentry/kernel/time/BUILD b/pkg/sentry/kernel/time/BUILD index 84f31b2dc..b3ed42aa4 100644 --- a/pkg/sentry/kernel/time/BUILD +++ b/pkg/sentry/kernel/time/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "time_state", diff --git a/pkg/sentry/limits/BUILD b/pkg/sentry/limits/BUILD index 06c3e72b0..3ce41cacc 100644 --- a/pkg/sentry/limits/BUILD +++ b/pkg/sentry/limits/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "limits_state", diff --git a/pkg/sentry/loader/BUILD b/pkg/sentry/loader/BUILD index 01a0ec426..e63052c6d 100644 --- a/pkg/sentry/loader/BUILD +++ b/pkg/sentry/loader/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_embed_data", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("@io_bazel_rules_go//go:def.bzl", "go_embed_data") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_embed_data( name = "vdso_bin", diff --git a/pkg/sentry/memmap/BUILD b/pkg/sentry/memmap/BUILD index 7525fea45..2e367e189 100644 --- a/pkg/sentry/memmap/BUILD +++ b/pkg/sentry/memmap/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "memmap_state", diff --git a/pkg/sentry/memutil/BUILD b/pkg/sentry/memutil/BUILD index a387a0c9f..341b30b98 100644 --- a/pkg/sentry/memutil/BUILD +++ b/pkg/sentry/memutil/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "memutil", diff --git a/pkg/sentry/mm/BUILD b/pkg/sentry/mm/BUILD index 258389bb2..3f396986a 100644 --- a/pkg/sentry/mm/BUILD +++ b/pkg/sentry/mm/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "mm_state", diff --git a/pkg/sentry/platform/BUILD b/pkg/sentry/platform/BUILD index d5be81f8d..15a7fbbc3 100644 --- a/pkg/sentry/platform/BUILD +++ b/pkg/sentry/platform/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "platform_state", diff --git a/pkg/sentry/platform/filemem/BUILD b/pkg/sentry/platform/filemem/BUILD index 3c4d5b0b6..dadba1d38 100644 --- a/pkg/sentry/platform/filemem/BUILD +++ b/pkg/sentry/platform/filemem/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "filemem_autogen_state", diff --git a/pkg/sentry/platform/interrupt/BUILD b/pkg/sentry/platform/interrupt/BUILD index 33dde2a31..35121321a 100644 --- a/pkg/sentry/platform/interrupt/BUILD +++ b/pkg/sentry/platform/interrupt/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "interrupt", diff --git a/pkg/sentry/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD index 673393fad..4ef9e20d7 100644 --- a/pkg/sentry/platform/kvm/BUILD +++ b/pkg/sentry/platform/kvm/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/platform/kvm/testutil/BUILD b/pkg/sentry/platform/kvm/testutil/BUILD index 8533a8d89..e779e3893 100644 --- a/pkg/sentry/platform/kvm/testutil/BUILD +++ b/pkg/sentry/platform/kvm/testutil/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "testutil", diff --git a/pkg/sentry/platform/procid/BUILD b/pkg/sentry/platform/procid/BUILD index 5db4f6261..ba68d48f4 100644 --- a/pkg/sentry/platform/procid/BUILD +++ b/pkg/sentry/platform/procid/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "procid", diff --git a/pkg/sentry/platform/ptrace/BUILD b/pkg/sentry/platform/ptrace/BUILD index 16b0b3c69..ceee895dc 100644 --- a/pkg/sentry/platform/ptrace/BUILD +++ b/pkg/sentry/platform/ptrace/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "ptrace", diff --git a/pkg/sentry/platform/ring0/BUILD b/pkg/sentry/platform/ring0/BUILD index 2df232a64..2485eb2eb 100644 --- a/pkg/sentry/platform/ring0/BUILD +++ b/pkg/sentry/platform/ring0/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template( diff --git a/pkg/sentry/platform/ring0/pagetables/BUILD b/pkg/sentry/platform/ring0/pagetables/BUILD index 023e298a0..7a86e2234 100644 --- a/pkg/sentry/platform/ring0/pagetables/BUILD +++ b/pkg/sentry/platform/ring0/pagetables/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template( diff --git a/pkg/sentry/platform/safecopy/BUILD b/pkg/sentry/platform/safecopy/BUILD index 8b9f29403..7dcf6e561 100644 --- a/pkg/sentry/platform/safecopy/BUILD +++ b/pkg/sentry/platform/safecopy/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "safecopy", diff --git a/pkg/sentry/safemem/BUILD b/pkg/sentry/safemem/BUILD index dc4cfce41..e96509ce1 100644 --- a/pkg/sentry/safemem/BUILD +++ b/pkg/sentry/safemem/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "safemem", diff --git a/pkg/sentry/sighandling/BUILD b/pkg/sentry/sighandling/BUILD index daaad7c90..f480f0735 100644 --- a/pkg/sentry/sighandling/BUILD +++ b/pkg/sentry/sighandling/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "sighandling", diff --git a/pkg/sentry/socket/BUILD b/pkg/sentry/socket/BUILD index 5500a676e..929787aa0 100644 --- a/pkg/sentry/socket/BUILD +++ b/pkg/sentry/socket/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "socket_state", diff --git a/pkg/sentry/socket/control/BUILD b/pkg/sentry/socket/control/BUILD index 25de2f655..faf2b4c27 100644 --- a/pkg/sentry/socket/control/BUILD +++ b/pkg/sentry/socket/control/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "control_state", diff --git a/pkg/sentry/socket/epsocket/BUILD b/pkg/sentry/socket/epsocket/BUILD index 8430886cb..7ad5e88c5 100644 --- a/pkg/sentry/socket/epsocket/BUILD +++ b/pkg/sentry/socket/epsocket/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "epsocket_state", diff --git a/pkg/sentry/socket/hostinet/BUILD b/pkg/sentry/socket/hostinet/BUILD index 60ec265ba..227ca3926 100644 --- a/pkg/sentry/socket/hostinet/BUILD +++ b/pkg/sentry/socket/hostinet/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "hostinet_state", diff --git a/pkg/sentry/socket/netlink/BUILD b/pkg/sentry/socket/netlink/BUILD index 9df3ab17c..b23a243f7 100644 --- a/pkg/sentry/socket/netlink/BUILD +++ b/pkg/sentry/socket/netlink/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "netlink_state", diff --git a/pkg/sentry/socket/netlink/port/BUILD b/pkg/sentry/socket/netlink/port/BUILD index 7340b95c9..ba6f686e4 100644 --- a/pkg/sentry/socket/netlink/port/BUILD +++ b/pkg/sentry/socket/netlink/port/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "port_state", diff --git a/pkg/sentry/socket/netlink/route/BUILD b/pkg/sentry/socket/netlink/route/BUILD index ff3f7b7a4..726469fc9 100644 --- a/pkg/sentry/socket/netlink/route/BUILD +++ b/pkg/sentry/socket/netlink/route/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "route_state", diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index 8973453f9..288199779 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "rpcinet", diff --git a/pkg/sentry/socket/rpcinet/conn/BUILD b/pkg/sentry/socket/rpcinet/conn/BUILD index 4923dee4b..c51ca14b1 100644 --- a/pkg/sentry/socket/rpcinet/conn/BUILD +++ b/pkg/sentry/socket/rpcinet/conn/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # BSD -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "conn", diff --git a/pkg/sentry/socket/rpcinet/notifier/BUILD b/pkg/sentry/socket/rpcinet/notifier/BUILD index 6f3b06a05..2ae902b3f 100644 --- a/pkg/sentry/socket/rpcinet/notifier/BUILD +++ b/pkg/sentry/socket/rpcinet/notifier/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # BSD -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "notifier", diff --git a/pkg/sentry/socket/unix/BUILD b/pkg/sentry/socket/unix/BUILD index 1ec6eb7ed..7d04d6b6b 100644 --- a/pkg/sentry/socket/unix/BUILD +++ b/pkg/sentry/socket/unix/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "unix_state", diff --git a/pkg/sentry/state/BUILD b/pkg/sentry/state/BUILD index 9bd98f445..a57a8298e 100644 --- a/pkg/sentry/state/BUILD +++ b/pkg/sentry/state/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "state", diff --git a/pkg/sentry/strace/BUILD b/pkg/sentry/strace/BUILD index c5946a564..e1c8db67a 100644 --- a/pkg/sentry/strace/BUILD +++ b/pkg/sentry/strace/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "strace", diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD index d667b42c8..22a757095 100644 --- a/pkg/sentry/syscalls/BUILD +++ b/pkg/sentry/syscalls/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "syscalls", diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD index d3f3cc459..574621ad2 100644 --- a/pkg/sentry/syscalls/linux/BUILD +++ b/pkg/sentry/syscalls/linux/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "linux_state", diff --git a/pkg/sentry/time/BUILD b/pkg/sentry/time/BUILD index cbcd699d5..9452787fb 100644 --- a/pkg/sentry/time/BUILD +++ b/pkg/sentry/time/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/uniqueid/BUILD b/pkg/sentry/uniqueid/BUILD index c8ab03c3d..8eba3609e 100644 --- a/pkg/sentry/uniqueid/BUILD +++ b/pkg/sentry/uniqueid/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "uniqueid", diff --git a/pkg/sentry/usage/BUILD b/pkg/sentry/usage/BUILD index a0fe0aa07..edee44d96 100644 --- a/pkg/sentry/usage/BUILD +++ b/pkg/sentry/usage/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "usage_state", diff --git a/pkg/sentry/usermem/BUILD b/pkg/sentry/usermem/BUILD index 36c0760dd..9dd1cd2b5 100644 --- a/pkg/sentry/usermem/BUILD +++ b/pkg/sentry/usermem/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "usermem_state", diff --git a/pkg/sentry/watchdog/BUILD b/pkg/sentry/watchdog/BUILD index 28fae4490..13bc33eb1 100644 --- a/pkg/sentry/watchdog/BUILD +++ b/pkg/sentry/watchdog/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "watchdog", diff --git a/pkg/sleep/BUILD b/pkg/sleep/BUILD index f2b69b225..05e4ca540 100644 --- a/pkg/sleep/BUILD +++ b/pkg/sleep/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "sleep", diff --git a/pkg/state/BUILD b/pkg/state/BUILD index bb6415d9b..012b0484e 100644 --- a/pkg/state/BUILD +++ b/pkg/state/BUILD @@ -1,7 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/state/statefile/BUILD b/pkg/state/statefile/BUILD index df2c6a578..16abe1930 100644 --- a/pkg/state/statefile/BUILD +++ b/pkg/state/statefile/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "statefile", diff --git a/pkg/sync/BUILD b/pkg/sync/BUILD index 1fc0c25b5..3959fea36 100644 --- a/pkg/sync/BUILD +++ b/pkg/sync/BUILD @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//:sandbox"], diff --git a/pkg/sync/seqatomictest/BUILD b/pkg/sync/seqatomictest/BUILD index 9d6ee2dfb..07b4f85ab 100644 --- a/pkg/sync/seqatomictest/BUILD +++ b/pkg/sync/seqatomictest/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/syserr/BUILD b/pkg/syserr/BUILD index e5ce48412..c0850f3d9 100644 --- a/pkg/syserr/BUILD +++ b/pkg/syserr/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "syserr", diff --git a/pkg/syserror/BUILD b/pkg/syserror/BUILD index 68ddec786..e050c2043 100644 --- a/pkg/syserror/BUILD +++ b/pkg/syserror/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "syserror", diff --git a/pkg/tcpip/BUILD b/pkg/tcpip/BUILD index 186a0d3bf..391d801d0 100644 --- a/pkg/tcpip/BUILD +++ b/pkg/tcpip/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tcpip_state", diff --git a/pkg/tcpip/adapters/gonet/BUILD b/pkg/tcpip/adapters/gonet/BUILD index 97da46776..bf618831a 100644 --- a/pkg/tcpip/adapters/gonet/BUILD +++ b/pkg/tcpip/adapters/gonet/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "gonet", diff --git a/pkg/tcpip/buffer/BUILD b/pkg/tcpip/buffer/BUILD index 08adf18cd..efeb6a448 100644 --- a/pkg/tcpip/buffer/BUILD +++ b/pkg/tcpip/buffer/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "buffer_state", diff --git a/pkg/tcpip/checker/BUILD b/pkg/tcpip/checker/BUILD index 5447cfbf4..e8a524918 100644 --- a/pkg/tcpip/checker/BUILD +++ b/pkg/tcpip/checker/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "checker", diff --git a/pkg/tcpip/header/BUILD b/pkg/tcpip/header/BUILD index 859c2a106..3aa2cfb24 100644 --- a/pkg/tcpip/header/BUILD +++ b/pkg/tcpip/header/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tcp_header_state", diff --git a/pkg/tcpip/link/channel/BUILD b/pkg/tcpip/link/channel/BUILD index f2f0c8b6f..9a6f49c45 100644 --- a/pkg/tcpip/link/channel/BUILD +++ b/pkg/tcpip/link/channel/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "channel", diff --git a/pkg/tcpip/link/fdbased/BUILD b/pkg/tcpip/link/fdbased/BUILD index aca3b14ca..6e75e9f47 100644 --- a/pkg/tcpip/link/fdbased/BUILD +++ b/pkg/tcpip/link/fdbased/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "fdbased", diff --git a/pkg/tcpip/link/loopback/BUILD b/pkg/tcpip/link/loopback/BUILD index 9714e93db..cc4247ffd 100644 --- a/pkg/tcpip/link/loopback/BUILD +++ b/pkg/tcpip/link/loopback/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "loopback", diff --git a/pkg/tcpip/link/rawfile/BUILD b/pkg/tcpip/link/rawfile/BUILD index 4b30c7c1c..10b35a37e 100644 --- a/pkg/tcpip/link/rawfile/BUILD +++ b/pkg/tcpip/link/rawfile/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "rawfile", diff --git a/pkg/tcpip/link/sharedmem/BUILD b/pkg/tcpip/link/sharedmem/BUILD index 1bd79a3f4..5390257c5 100644 --- a/pkg/tcpip/link/sharedmem/BUILD +++ b/pkg/tcpip/link/sharedmem/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "sharedmem", diff --git a/pkg/tcpip/link/sharedmem/pipe/BUILD b/pkg/tcpip/link/sharedmem/pipe/BUILD index e6c658071..ff798ae6f 100644 --- a/pkg/tcpip/link/sharedmem/pipe/BUILD +++ b/pkg/tcpip/link/sharedmem/pipe/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "pipe", diff --git a/pkg/tcpip/link/sharedmem/queue/BUILD b/pkg/tcpip/link/sharedmem/queue/BUILD index 80cedade1..c4a7879c4 100644 --- a/pkg/tcpip/link/sharedmem/queue/BUILD +++ b/pkg/tcpip/link/sharedmem/queue/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "queue", diff --git a/pkg/tcpip/link/sniffer/BUILD b/pkg/tcpip/link/sniffer/BUILD index d14f150d1..1e844f949 100644 --- a/pkg/tcpip/link/sniffer/BUILD +++ b/pkg/tcpip/link/sniffer/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "sniffer", diff --git a/pkg/tcpip/link/tun/BUILD b/pkg/tcpip/link/tun/BUILD index 21da7d57e..a8bb03661 100644 --- a/pkg/tcpip/link/tun/BUILD +++ b/pkg/tcpip/link/tun/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "tun", diff --git a/pkg/tcpip/link/waitable/BUILD b/pkg/tcpip/link/waitable/BUILD index 3b513383a..7582df32e 100644 --- a/pkg/tcpip/link/waitable/BUILD +++ b/pkg/tcpip/link/waitable/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "waitable", diff --git a/pkg/tcpip/network/BUILD b/pkg/tcpip/network/BUILD index 963857f51..9a26b46c4 100644 --- a/pkg/tcpip/network/BUILD +++ b/pkg/tcpip/network/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_test") +load("//tools/go_stateify:defs.bzl", "go_test") go_test( name = "ip_test", diff --git a/pkg/tcpip/network/arp/BUILD b/pkg/tcpip/network/arp/BUILD index 689f66d6e..44f2b66e5 100644 --- a/pkg/tcpip/network/arp/BUILD +++ b/pkg/tcpip/network/arp/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "arp", diff --git a/pkg/tcpip/network/fragmentation/BUILD b/pkg/tcpip/network/fragmentation/BUILD index a173f87fb..ac97ebe43 100644 --- a/pkg/tcpip/network/fragmentation/BUILD +++ b/pkg/tcpip/network/fragmentation/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "fragmentation_state", diff --git a/pkg/tcpip/network/hash/BUILD b/pkg/tcpip/network/hash/BUILD index e1b5f26c4..1c22c52fc 100644 --- a/pkg/tcpip/network/hash/BUILD +++ b/pkg/tcpip/network/hash/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "hash", diff --git a/pkg/tcpip/network/ipv4/BUILD b/pkg/tcpip/network/ipv4/BUILD index ae42b662f..19314e9bd 100644 --- a/pkg/tcpip/network/ipv4/BUILD +++ b/pkg/tcpip/network/ipv4/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "ipv4", diff --git a/pkg/tcpip/network/ipv6/BUILD b/pkg/tcpip/network/ipv6/BUILD index d008ac7fb..1c3eccae0 100644 --- a/pkg/tcpip/network/ipv6/BUILD +++ b/pkg/tcpip/network/ipv6/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "ipv6", diff --git a/pkg/tcpip/ports/BUILD b/pkg/tcpip/ports/BUILD index 710c283f7..3c3374275 100644 --- a/pkg/tcpip/ports/BUILD +++ b/pkg/tcpip/ports/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "ports", diff --git a/pkg/tcpip/seqnum/BUILD b/pkg/tcpip/seqnum/BUILD index 6d28dbc3f..a75869dac 100644 --- a/pkg/tcpip/seqnum/BUILD +++ b/pkg/tcpip/seqnum/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "seqnum_state", diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD index 6d201d0a2..5e7355135 100644 --- a/pkg/tcpip/stack/BUILD +++ b/pkg/tcpip/stack/BUILD @@ -1,17 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") - -go_stateify( - name = "stack_state", - srcs = [ - "registration.go", - "stack.go", - ], - out = "stack_state.go", - package = "stack", -) +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "stack", @@ -22,7 +11,6 @@ go_library( "route.go", "stack.go", "stack_global_state.go", - "stack_state.go", "transport_demuxer.go", ], importpath = "gvisor.googlesource.com/gvisor/pkg/tcpip/stack", @@ -32,7 +20,6 @@ go_library( deps = [ "//pkg/ilist", "//pkg/sleep", - "//pkg/state", "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/header", diff --git a/pkg/tcpip/stack/registration.go b/pkg/tcpip/stack/registration.go index e9550a062..c66f925a8 100644 --- a/pkg/tcpip/stack/registration.go +++ b/pkg/tcpip/stack/registration.go @@ -31,6 +31,8 @@ type NetworkEndpointID struct { } // TransportEndpointID is the identifier of a transport layer protocol endpoint. +// +// +stateify savable type TransportEndpointID struct { // LocalPort is the local port associated with the endpoint. LocalPort uint16 diff --git a/pkg/tcpip/transport/ping/BUILD b/pkg/tcpip/transport/ping/BUILD index 1febbf7f5..28e3e1700 100644 --- a/pkg/tcpip/transport/ping/BUILD +++ b/pkg/tcpip/transport/ping/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "ping_state", diff --git a/pkg/tcpip/transport/queue/BUILD b/pkg/tcpip/transport/queue/BUILD index 7e8ee1f66..fb878ad36 100644 --- a/pkg/tcpip/transport/queue/BUILD +++ b/pkg/tcpip/transport/queue/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "queue_state", diff --git a/pkg/tcpip/transport/tcp/BUILD b/pkg/tcpip/transport/tcp/BUILD index 53623787d..6a7153e4d 100644 --- a/pkg/tcpip/transport/tcp/BUILD +++ b/pkg/tcpip/transport/tcp/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "tcp_state", diff --git a/pkg/tcpip/transport/tcp/testing/context/BUILD b/pkg/tcpip/transport/tcp/testing/context/BUILD index 3caa38bcb..7a95594ef 100644 --- a/pkg/tcpip/transport/tcp/testing/context/BUILD +++ b/pkg/tcpip/transport/tcp/testing/context/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "context", diff --git a/pkg/tcpip/transport/tcpconntrack/BUILD b/pkg/tcpip/transport/tcpconntrack/BUILD index 3d748528e..46da3e6f1 100644 --- a/pkg/tcpip/transport/tcpconntrack/BUILD +++ b/pkg/tcpip/transport/tcpconntrack/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "tcpconntrack", diff --git a/pkg/tcpip/transport/udp/BUILD b/pkg/tcpip/transport/udp/BUILD index 4f7a47973..790dd55a3 100644 --- a/pkg/tcpip/transport/udp/BUILD +++ b/pkg/tcpip/transport/udp/BUILD @@ -1,8 +1,7 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "udp_state", diff --git a/pkg/tcpip/transport/unix/BUILD b/pkg/tcpip/transport/unix/BUILD index d58f06544..676f2cf92 100644 --- a/pkg/tcpip/transport/unix/BUILD +++ b/pkg/tcpip/transport/unix/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify") go_stateify( name = "unix_state", diff --git a/pkg/tmutex/BUILD b/pkg/tmutex/BUILD index d9a2c5ae5..d18338fff 100644 --- a/pkg/tmutex/BUILD +++ b/pkg/tmutex/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "tmutex", diff --git a/pkg/unet/BUILD b/pkg/unet/BUILD index e8e40315a..acdfd7cb6 100644 --- a/pkg/unet/BUILD +++ b/pkg/unet/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "unet", diff --git a/pkg/urpc/BUILD b/pkg/urpc/BUILD index b29b25637..d32c57d1a 100644 --- a/pkg/urpc/BUILD +++ b/pkg/urpc/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") go_library( name = "urpc", diff --git a/pkg/waiter/BUILD b/pkg/waiter/BUILD index 032ec3237..8256acdb4 100644 --- a/pkg/waiter/BUILD +++ b/pkg/waiter/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -load("//tools/go_stateify:defs.bzl", "go_stateify") +load("//tools/go_stateify:defs.bzl", "go_library", "go_stateify", "go_test") go_stateify( name = "waiter_state", diff --git a/pkg/waiter/fdnotifier/BUILD b/pkg/waiter/fdnotifier/BUILD index d5b5ee82d..4e582755d 100644 --- a/pkg/waiter/fdnotifier/BUILD +++ b/pkg/waiter/fdnotifier/BUILD @@ -1,6 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_library") go_library( name = "fdnotifier", diff --git a/tools/go_stateify/defs.bzl b/tools/go_stateify/defs.bzl index 60a9895ff..2b2582b7a 100644 --- a/tools/go_stateify/defs.bzl +++ b/tools/go_stateify/defs.bzl @@ -22,6 +22,8 @@ go_library( ) """ +load("@io_bazel_rules_go//go:def.bzl", _go_library = "go_library", _go_test = "go_test") + def _go_stateify_impl(ctx): """Implementation for the stateify tool.""" output = ctx.outputs.out @@ -33,6 +35,8 @@ def _go_stateify_impl(ctx): args += ["-statepkg=%s" % ctx.attr._statepkg] if ctx.attr.imports: args += ["-imports=%s" % ",".join(ctx.attr.imports)] + if ctx.attr.explicit: + args += ["-explicit=true"] args += ["--"] for src in ctx.attr.srcs: args += [f.path for f in src.files] @@ -45,17 +49,15 @@ def _go_stateify_impl(ctx): executable = ctx.executable._tool, ) -""" -Generates save and restore logic from a set of Go files. - - -Args: - name: the name of the rule. - srcs: the input source files. These files should include all structs in the package that need to be saved. - imports: an optional list of extra non-aliased, Go-style absolute import paths. - out: the name of the generated file output. This must not conflict with any other files and must be added to the srcs of the relevant go_library. - package: the package name for the input sources. -""" +# Generates save and restore logic from a set of Go files. +# +# Args: +# name: the name of the rule. +# srcs: the input source files. These files should include all structs in the package that need to be saved. +# imports: an optional list of extra non-aliased, Go-style absolute import paths. +# out: the name of the generated file output. This must not conflict with any other files and must be added to the srcs of the relevant go_library. +# package: the package name for the input sources. +# explicit: only generate for types explicitly annotated as savable. go_stateify = rule( implementation = _go_stateify_impl, attrs = { @@ -63,7 +65,41 @@ go_stateify = rule( "imports": attr.string_list(mandatory = False), "package": attr.string(mandatory = True), "out": attr.output(mandatory = True), + "explicit": attr.bool(default = False), "_tool": attr.label(executable = True, cfg = "host", default = Label("//tools/go_stateify:stateify")), "_statepkg": attr.string(default = "gvisor.googlesource.com/gvisor/pkg/state"), }, ) + +def go_library(name, srcs, deps = [], imports = [], **kwargs): + """wraps the standard go_library and does stateification.""" + if "encode_unsafe.go" not in srcs and (name + "_state_autogen.go") not in srcs: + # Only do stateification for non-state packages without manual autogen. + go_stateify( + name = name + "_state_autogen", + srcs = [src for src in srcs if src.endswith(".go")], + imports = imports, + package = name, + out = name + "_state_autogen.go", + explicit = True, + ) + all_srcs = srcs + [name + "_state_autogen.go"] + if "//pkg/state" not in deps: + all_deps = deps + ["//pkg/state"] + else: + all_deps = deps + else: + all_deps = deps + all_srcs = srcs + _go_library( + name = name, + srcs = all_srcs, + deps = all_deps, + **kwargs + ) + +def go_test(**kwargs): + """Wraps the standard go_test.""" + _go_test( + **kwargs + ) diff --git a/tools/go_stateify/main.go b/tools/go_stateify/main.go index 6c3583c62..231c6d80b 100644 --- a/tools/go_stateify/main.go +++ b/tools/go_stateify/main.go @@ -25,6 +25,7 @@ import ( "os" "reflect" "strings" + "sync" ) var ( @@ -32,6 +33,7 @@ var ( imports = flag.String("imports", "", "extra imports for the output file") output = flag.String("output", "", "output file") statePkg = flag.String("statepkg", "", "state import package; defaults to empty") + explicit = flag.Bool("explicit", false, "only generate for types explicitly tagged '// +stateify savable'") ) // resolveTypeName returns a qualified type name. @@ -224,16 +226,24 @@ func main() { // Emit the package name. fmt.Fprint(outputFile, "// automatically generated by stateify.\n\n") fmt.Fprintf(outputFile, "package %s\n\n", *pkg) - fmt.Fprint(outputFile, "import (\n") - if *statePkg != "" { - fmt.Fprintf(outputFile, " \"%s\"\n", *statePkg) - } - if *imports != "" { - for _, i := range strings.Split(*imports, ",") { - fmt.Fprintf(outputFile, " \"%s\"\n", i) - } + + // Emit the imports lazily. + var once sync.Once + maybeEmitImports := func() { + once.Do(func() { + // Emit the imports. + fmt.Fprint(outputFile, "import (\n") + if *statePkg != "" { + fmt.Fprintf(outputFile, " \"%s\"\n", *statePkg) + } + if *imports != "" { + for _, i := range strings.Split(*imports, ",") { + fmt.Fprintf(outputFile, " \"%s\"\n", i) + } + } + fmt.Fprint(outputFile, ")\n\n") + }) } - fmt.Fprint(outputFile, ")\n\n") files := make([]*ast.File, 0, len(flag.Args())) @@ -241,7 +251,7 @@ func main() { for _, filename := range flag.Args() { // Parse the file. fset := token.NewFileSet() - f, err := parser.ParseFile(fset, filename, nil, 0) + f, err := parser.ParseFile(fset, filename, nil, parser.ParseComments) if err != nil { // Not a valid input file? fmt.Fprintf(os.Stderr, "Input %q can't be parsed: %v\n", filename, err) @@ -308,6 +318,26 @@ func main() { continue } + if *explicit { + // In explicit mode, only generate code for + // types explicitly marked + // "// +stateify savable" in one of the + // proceeding comment lines. + if d.Doc == nil { + continue + } + savable := false + for _, l := range d.Doc.List { + if l.Text == "// +stateify savable" { + savable = true + break + } + } + if !savable { + continue + } + } + for _, gs := range d.Specs { ts := gs.(*ast.TypeSpec) switch ts.Type.(type) { @@ -315,6 +345,8 @@ func main() { // Don't register. break case *ast.StructType: + maybeEmitImports() + ss := ts.Type.(*ast.StructType) // Define beforeSave if a definition was not found. This @@ -360,6 +392,8 @@ func main() { // Add to our registration. emitRegister(ts.Name.Name) case *ast.Ident, *ast.SelectorExpr, *ast.ArrayType: + maybeEmitImports() + _, val := resolveTypeName(ts.Name.Name, ts.Type) // Dispatch directly. @@ -377,10 +411,12 @@ func main() { } } - // Emit the init() function. - fmt.Fprintf(outputFile, "func init() {\n") - for _, ic := range initCalls { - fmt.Fprintf(outputFile, " %s\n", ic) + if len(initCalls) > 0 { + // Emit the init() function. + fmt.Fprintf(outputFile, "func init() {\n") + for _, ic := range initCalls { + fmt.Fprintf(outputFile, " %s\n", ic) + } + fmt.Fprintf(outputFile, "}\n") } - fmt.Fprintf(outputFile, "}\n") } -- cgit v1.2.3 From 6cba410df0ea2eabb87bad5074a8a79ed89312b8 Mon Sep 17 00:00:00 2001 From: Ian Gudger Date: Wed, 17 Oct 2018 11:36:32 -0700 Subject: Move Unix transport out of netstack PiperOrigin-RevId: 217557656 Change-Id: I63d27635b1a6c12877279995d2d9847b6a19da9b --- pkg/sentry/fs/BUILD | 2 +- pkg/sentry/fs/ashmem/BUILD | 1 - pkg/sentry/fs/dirent.go | 4 +- pkg/sentry/fs/fsutil/BUILD | 2 +- pkg/sentry/fs/fsutil/inode.go | 6 +- pkg/sentry/fs/gofer/BUILD | 2 +- pkg/sentry/fs/gofer/path.go | 4 +- pkg/sentry/fs/gofer/session.go | 22 +- pkg/sentry/fs/gofer/socket.go | 25 +- pkg/sentry/fs/host/BUILD | 4 +- pkg/sentry/fs/host/control.go | 8 +- pkg/sentry/fs/host/inode.go | 6 +- pkg/sentry/fs/host/socket.go | 66 +- pkg/sentry/fs/host/socket_test.go | 12 +- pkg/sentry/fs/inode.go | 6 +- pkg/sentry/fs/inode_operations.go | 6 +- pkg/sentry/fs/inode_overlay.go | 6 +- pkg/sentry/fs/ramfs/BUILD | 2 +- pkg/sentry/fs/ramfs/dir.go | 6 +- pkg/sentry/fs/ramfs/ramfs.go | 4 +- pkg/sentry/fs/ramfs/socket.go | 8 +- pkg/sentry/fs/tmpfs/BUILD | 2 +- pkg/sentry/fs/tmpfs/tmpfs.go | 6 +- pkg/sentry/fs/tty/BUILD | 2 +- pkg/sentry/fs/tty/dir.go | 4 +- pkg/sentry/kernel/BUILD | 2 +- pkg/sentry/kernel/abstract_socket_namespace.go | 16 +- pkg/sentry/socket/BUILD | 2 +- pkg/sentry/socket/control/BUILD | 2 +- pkg/sentry/socket/control/control.go | 46 +- pkg/sentry/socket/epsocket/BUILD | 2 +- pkg/sentry/socket/epsocket/epsocket.go | 20 +- pkg/sentry/socket/epsocket/provider.go | 8 +- pkg/sentry/socket/hostinet/BUILD | 2 +- pkg/sentry/socket/hostinet/socket.go | 6 +- pkg/sentry/socket/netlink/BUILD | 2 +- pkg/sentry/socket/netlink/provider.go | 8 +- pkg/sentry/socket/netlink/socket.go | 20 +- pkg/sentry/socket/rpcinet/BUILD | 2 +- pkg/sentry/socket/rpcinet/socket.go | 6 +- pkg/sentry/socket/socket.go | 12 +- pkg/sentry/socket/unix/BUILD | 2 +- pkg/sentry/socket/unix/io.go | 21 +- pkg/sentry/socket/unix/transport/BUILD | 22 + pkg/sentry/socket/unix/transport/connectioned.go | 454 ++++++++++ .../socket/unix/transport/connectioned_state.go | 53 ++ pkg/sentry/socket/unix/transport/connectionless.go | 192 +++++ pkg/sentry/socket/unix/transport/queue/BUILD | 15 + pkg/sentry/socket/unix/transport/queue/queue.go | 227 +++++ pkg/sentry/socket/unix/transport/unix.go | 953 +++++++++++++++++++++ pkg/sentry/socket/unix/unix.go | 62 +- pkg/sentry/syscalls/linux/BUILD | 2 +- pkg/sentry/syscalls/linux/sys_socket.go | 8 +- pkg/sentry/uniqueid/BUILD | 2 +- pkg/sentry/uniqueid/context.go | 6 +- pkg/tcpip/transport/queue/BUILD | 15 - pkg/tcpip/transport/queue/queue.go | 227 ----- pkg/tcpip/transport/unix/BUILD | 22 - pkg/tcpip/transport/unix/connectioned.go | 454 ---------- pkg/tcpip/transport/unix/connectioned_state.go | 53 -- pkg/tcpip/transport/unix/connectionless.go | 192 ----- pkg/tcpip/transport/unix/unix.go | 953 --------------------- 62 files changed, 2154 insertions(+), 2153 deletions(-) create mode 100644 pkg/sentry/socket/unix/transport/BUILD create mode 100644 pkg/sentry/socket/unix/transport/connectioned.go create mode 100644 pkg/sentry/socket/unix/transport/connectioned_state.go create mode 100644 pkg/sentry/socket/unix/transport/connectionless.go create mode 100644 pkg/sentry/socket/unix/transport/queue/BUILD create mode 100644 pkg/sentry/socket/unix/transport/queue/queue.go create mode 100644 pkg/sentry/socket/unix/transport/unix.go delete mode 100644 pkg/tcpip/transport/queue/BUILD delete mode 100644 pkg/tcpip/transport/queue/queue.go delete mode 100644 pkg/tcpip/transport/unix/BUILD delete mode 100644 pkg/tcpip/transport/unix/connectioned.go delete mode 100644 pkg/tcpip/transport/unix/connectioned_state.go delete mode 100644 pkg/tcpip/transport/unix/connectionless.go delete mode 100644 pkg/tcpip/transport/unix/unix.go (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD index a949fffbf..548898aaa 100644 --- a/pkg/sentry/fs/BUILD +++ b/pkg/sentry/fs/BUILD @@ -59,13 +59,13 @@ go_library( "//pkg/sentry/limits", "//pkg/sentry/memmap", "//pkg/sentry/platform", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/uniqueid", "//pkg/sentry/usage", "//pkg/sentry/usermem", "//pkg/state", "//pkg/syserror", "//pkg/tcpip", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/ashmem/BUILD b/pkg/sentry/fs/ashmem/BUILD index dc893d22f..44ef82e64 100644 --- a/pkg/sentry/fs/ashmem/BUILD +++ b/pkg/sentry/fs/ashmem/BUILD @@ -28,7 +28,6 @@ go_library( "//pkg/sentry/usage", "//pkg/sentry/usermem", "//pkg/syserror", - "//pkg/tcpip/transport/unix", ], ) diff --git a/pkg/sentry/fs/dirent.go b/pkg/sentry/fs/dirent.go index a42c03e98..27fea0019 100644 --- a/pkg/sentry/fs/dirent.go +++ b/pkg/sentry/fs/dirent.go @@ -26,9 +26,9 @@ import ( "gvisor.googlesource.com/gvisor/pkg/refs" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/uniqueid" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) type globalDirentMap struct { @@ -800,7 +800,7 @@ func (d *Dirent) CreateDirectory(ctx context.Context, root *Dirent, name string, } // Bind satisfies the InodeOperations interface; otherwise same as GetFile. -func (d *Dirent) Bind(ctx context.Context, root *Dirent, name string, data unix.BoundEndpoint, perms FilePermissions) (*Dirent, error) { +func (d *Dirent) Bind(ctx context.Context, root *Dirent, name string, data transport.BoundEndpoint, perms FilePermissions) (*Dirent, error) { var childDir *Dirent err := d.genericCreate(ctx, root, name, func() error { var e error diff --git a/pkg/sentry/fs/fsutil/BUILD b/pkg/sentry/fs/fsutil/BUILD index 3512bae6f..6834e1272 100644 --- a/pkg/sentry/fs/fsutil/BUILD +++ b/pkg/sentry/fs/fsutil/BUILD @@ -87,11 +87,11 @@ go_library( "//pkg/sentry/memmap", "//pkg/sentry/platform", "//pkg/sentry/safemem", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usage", "//pkg/sentry/usermem", "//pkg/state", "//pkg/syserror", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go index 3479f2fad..3acc32752 100644 --- a/pkg/sentry/fs/fsutil/inode.go +++ b/pkg/sentry/fs/fsutil/inode.go @@ -19,9 +19,9 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs" ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -254,7 +254,7 @@ func (InodeNotDirectory) CreateDirectory(context.Context, *fs.Inode, string, fs. } // Bind implements fs.InodeOperations.Bind. -func (InodeNotDirectory) Bind(context.Context, *fs.Inode, string, unix.BoundEndpoint, fs.FilePermissions) (*fs.Dirent, error) { +func (InodeNotDirectory) Bind(context.Context, *fs.Inode, string, transport.BoundEndpoint, fs.FilePermissions) (*fs.Dirent, error) { return nil, syserror.ENOTDIR } @@ -277,7 +277,7 @@ func (InodeNotDirectory) RemoveDirectory(context.Context, *fs.Inode, string) err type InodeNotSocket struct{} // BoundEndpoint implements fs.InodeOperations.BoundEndpoint. -func (InodeNotSocket) BoundEndpoint(*fs.Inode, string) unix.BoundEndpoint { +func (InodeNotSocket) BoundEndpoint(*fs.Inode, string) transport.BoundEndpoint { return nil } diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD index cb17339c9..cef01829a 100644 --- a/pkg/sentry/fs/gofer/BUILD +++ b/pkg/sentry/fs/gofer/BUILD @@ -41,10 +41,10 @@ go_library( "//pkg/sentry/kernel/time", "//pkg/sentry/memmap", "//pkg/sentry/safemem", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserror", "//pkg/tcpip", - "//pkg/tcpip/transport/unix", "//pkg/unet", "//pkg/waiter", ], diff --git a/pkg/sentry/fs/gofer/path.go b/pkg/sentry/fs/gofer/path.go index bec9680f8..0bf7881da 100644 --- a/pkg/sentry/fs/gofer/path.go +++ b/pkg/sentry/fs/gofer/path.go @@ -22,8 +22,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/device" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) // Lookup loads an Inode at name into a Dirent based on the session's cache @@ -180,7 +180,7 @@ func (i *inodeOperations) CreateDirectory(ctx context.Context, dir *fs.Inode, s } // Bind implements InodeOperations.Bind. -func (i *inodeOperations) Bind(ctx context.Context, dir *fs.Inode, name string, ep unix.BoundEndpoint, perm fs.FilePermissions) (*fs.Dirent, error) { +func (i *inodeOperations) Bind(ctx context.Context, dir *fs.Inode, name string, ep transport.BoundEndpoint, perm fs.FilePermissions) (*fs.Dirent, error) { if i.session().endpoints == nil { return nil, syscall.EOPNOTSUPP } diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go index 49d27ee88..4e2293398 100644 --- a/pkg/sentry/fs/gofer/session.go +++ b/pkg/sentry/fs/gofer/session.go @@ -24,7 +24,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/device" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/unet" ) @@ -36,23 +36,23 @@ type endpointMaps struct { // direntMap links sockets to their dirents. // It is filled concurrently with the keyMap and is stored upon save. // Before saving, this map is used to populate the pathMap. - direntMap map[unix.BoundEndpoint]*fs.Dirent + direntMap map[transport.BoundEndpoint]*fs.Dirent // keyMap links MultiDeviceKeys (containing inode IDs) to their sockets. // It is not stored during save because the inode ID may change upon restore. - keyMap map[device.MultiDeviceKey]unix.BoundEndpoint `state:"nosave"` + keyMap map[device.MultiDeviceKey]transport.BoundEndpoint `state:"nosave"` // pathMap links the sockets to their paths. // It is filled before saving from the direntMap and is stored upon save. // Upon restore, this map is used to re-populate the keyMap. - pathMap map[unix.BoundEndpoint]string + pathMap map[transport.BoundEndpoint]string } // add adds the endpoint to the maps. // A reference is taken on the dirent argument. // // Precondition: maps must have been locked with 'lock'. -func (e *endpointMaps) add(key device.MultiDeviceKey, d *fs.Dirent, ep unix.BoundEndpoint) { +func (e *endpointMaps) add(key device.MultiDeviceKey, d *fs.Dirent, ep transport.BoundEndpoint) { e.keyMap[key] = ep d.IncRef() e.direntMap[ep] = d @@ -81,7 +81,7 @@ func (e *endpointMaps) lock() func() { // get returns the endpoint mapped to the given key. // // Precondition: maps must have been locked for reading. -func (e *endpointMaps) get(key device.MultiDeviceKey) unix.BoundEndpoint { +func (e *endpointMaps) get(key device.MultiDeviceKey) transport.BoundEndpoint { return e.keyMap[key] } @@ -285,9 +285,9 @@ func Root(ctx context.Context, dev string, filesystem fs.Filesystem, superBlockF // newEndpointMaps creates a new endpointMaps. func newEndpointMaps() *endpointMaps { return &endpointMaps{ - direntMap: make(map[unix.BoundEndpoint]*fs.Dirent), - keyMap: make(map[device.MultiDeviceKey]unix.BoundEndpoint), - pathMap: make(map[unix.BoundEndpoint]string), + direntMap: make(map[transport.BoundEndpoint]*fs.Dirent), + keyMap: make(map[device.MultiDeviceKey]transport.BoundEndpoint), + pathMap: make(map[transport.BoundEndpoint]string), } } @@ -341,7 +341,7 @@ func (s *session) fillPathMap() error { func (s *session) restoreEndpointMaps(ctx context.Context) error { // When restoring, only need to create the keyMap because the dirent and path // maps got stored through the save. - s.endpoints.keyMap = make(map[device.MultiDeviceKey]unix.BoundEndpoint) + s.endpoints.keyMap = make(map[device.MultiDeviceKey]transport.BoundEndpoint) if err := s.fillKeyMap(ctx); err != nil { return fmt.Errorf("failed to insert sockets into endpoint map: %v", err) } @@ -349,6 +349,6 @@ func (s *session) restoreEndpointMaps(ctx context.Context) error { // Re-create pathMap because it can no longer be trusted as socket paths can // change while process continues to run. Empty pathMap will be re-filled upon // next save. - s.endpoints.pathMap = make(map[unix.BoundEndpoint]string) + s.endpoints.pathMap = make(map[transport.BoundEndpoint]string) return nil } diff --git a/pkg/sentry/fs/gofer/socket.go b/pkg/sentry/fs/gofer/socket.go index 0190bc006..d072da624 100644 --- a/pkg/sentry/fs/gofer/socket.go +++ b/pkg/sentry/fs/gofer/socket.go @@ -19,13 +19,13 @@ import ( "gvisor.googlesource.com/gvisor/pkg/p9" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/host" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) -// BoundEndpoint returns a gofer-backed unix.BoundEndpoint. -func (i *inodeOperations) BoundEndpoint(inode *fs.Inode, path string) unix.BoundEndpoint { +// BoundEndpoint returns a gofer-backed transport.BoundEndpoint. +func (i *inodeOperations) BoundEndpoint(inode *fs.Inode, path string) transport.BoundEndpoint { if !fs.IsSocket(i.fileState.sattr) { return nil } @@ -45,7 +45,7 @@ func (i *inodeOperations) BoundEndpoint(inode *fs.Inode, path string) unix.Bound return &endpoint{inode, i.fileState.file.file, path} } -// endpoint is a Gofer-backed unix.BoundEndpoint. +// endpoint is a Gofer-backed transport.BoundEndpoint. // // An endpoint's lifetime is the time between when InodeOperations.BoundEndpoint() // is called and either BoundEndpoint.BidirectionalConnect or @@ -61,20 +61,20 @@ type endpoint struct { path string } -func unixSockToP9(t unix.SockType) (p9.ConnectFlags, bool) { +func unixSockToP9(t transport.SockType) (p9.ConnectFlags, bool) { switch t { - case unix.SockStream: + case transport.SockStream: return p9.StreamSocket, true - case unix.SockSeqpacket: + case transport.SockSeqpacket: return p9.SeqpacketSocket, true - case unix.SockDgram: + case transport.SockDgram: return p9.DgramSocket, true } return 0, false } // BidirectionalConnect implements ConnectableEndpoint.BidirectionalConnect. -func (e *endpoint) BidirectionalConnect(ce unix.ConnectingEndpoint, returnConnect func(unix.Receiver, unix.ConnectedEndpoint)) *tcpip.Error { +func (e *endpoint) BidirectionalConnect(ce transport.ConnectingEndpoint, returnConnect func(transport.Receiver, transport.ConnectedEndpoint)) *tcpip.Error { cf, ok := unixSockToP9(ce.Type()) if !ok { return tcpip.ErrConnectionRefused @@ -113,8 +113,9 @@ func (e *endpoint) BidirectionalConnect(ce unix.ConnectingEndpoint, returnConnec return nil } -// UnidirectionalConnect implements unix.BoundEndpoint.UnidirectionalConnect. -func (e *endpoint) UnidirectionalConnect() (unix.ConnectedEndpoint, *tcpip.Error) { +// UnidirectionalConnect implements +// transport.BoundEndpoint.UnidirectionalConnect. +func (e *endpoint) UnidirectionalConnect() (transport.ConnectedEndpoint, *tcpip.Error) { hostFile, err := e.file.Connect(p9.DgramSocket) if err != nil { return nil, tcpip.ErrConnectionRefused @@ -134,7 +135,7 @@ func (e *endpoint) UnidirectionalConnect() (unix.ConnectedEndpoint, *tcpip.Error return c, nil } -// Release implements unix.BoundEndpoint.Release. +// Release implements transport.BoundEndpoint.Release. func (e *endpoint) Release() { e.inode.DecRef() } diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD index 5ada32ee1..4f264a024 100644 --- a/pkg/sentry/fs/host/BUILD +++ b/pkg/sentry/fs/host/BUILD @@ -42,13 +42,13 @@ go_library( "//pkg/sentry/safemem", "//pkg/sentry/socket/control", "//pkg/sentry/socket/unix", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/uniqueid", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", "//pkg/tcpip", "//pkg/tcpip/link/rawfile", - "//pkg/tcpip/transport/unix", "//pkg/unet", "//pkg/waiter", "//pkg/waiter/fdnotifier", @@ -72,10 +72,10 @@ go_test( "//pkg/sentry/context/contexttest", "//pkg/sentry/fs", "//pkg/sentry/socket", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/tcpip", - "//pkg/tcpip/transport/unix", "//pkg/waiter", "//pkg/waiter/fdnotifier", ], diff --git a/pkg/sentry/fs/host/control.go b/pkg/sentry/fs/host/control.go index d2b007ab2..d2e34a69d 100644 --- a/pkg/sentry/fs/host/control.go +++ b/pkg/sentry/fs/host/control.go @@ -20,7 +20,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/control" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" ) type scmRights struct { @@ -45,13 +45,13 @@ func (c *scmRights) Files(ctx context.Context, max int) control.RightsFiles { return rf } -// Clone implements unix.RightsControlMessage.Clone. -func (c *scmRights) Clone() unix.RightsControlMessage { +// Clone implements transport.RightsControlMessage.Clone. +func (c *scmRights) Clone() transport.RightsControlMessage { // Host rights never need to be cloned. return nil } -// Release implements unix.RightsControlMessage.Release. +// Release implements transport.RightsControlMessage.Release. func (c *scmRights) Release() { for _, fd := range c.fds { syscall.Close(fd) diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go index c2e8ba62f..e32497203 100644 --- a/pkg/sentry/fs/host/inode.go +++ b/pkg/sentry/fs/host/inode.go @@ -27,8 +27,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -310,12 +310,12 @@ func (i *inodeOperations) Rename(ctx context.Context, oldParent *fs.Inode, oldNa } // Bind implements fs.InodeOperations.Bind. -func (i *inodeOperations) Bind(ctx context.Context, dir *fs.Inode, name string, data unix.BoundEndpoint, perm fs.FilePermissions) (*fs.Dirent, error) { +func (i *inodeOperations) Bind(ctx context.Context, dir *fs.Inode, name string, data transport.BoundEndpoint, perm fs.FilePermissions) (*fs.Dirent, error) { return nil, syserror.EOPNOTSUPP } // BoundEndpoint implements fs.InodeOperations.BoundEndpoint. -func (i *inodeOperations) BoundEndpoint(inode *fs.Inode, path string) unix.BoundEndpoint { +func (i *inodeOperations) BoundEndpoint(inode *fs.Inode, path string) transport.BoundEndpoint { return nil } diff --git a/pkg/sentry/fs/host/socket.go b/pkg/sentry/fs/host/socket.go index e454b6fe5..0eb267c00 100644 --- a/pkg/sentry/fs/host/socket.go +++ b/pkg/sentry/fs/host/socket.go @@ -25,12 +25,12 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/control" unixsocket "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/uniqueid" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/tcpip" "gvisor.googlesource.com/gvisor/pkg/tcpip/link/rawfile" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/unet" "gvisor.googlesource.com/gvisor/pkg/waiter" "gvisor.googlesource.com/gvisor/pkg/waiter/fdnotifier" @@ -42,7 +42,7 @@ import ( const maxSendBufferSize = 8 << 20 // ConnectedEndpoint is a host FD backed implementation of -// unix.ConnectedEndpoint and unix.Receiver. +// transport.ConnectedEndpoint and transport.Receiver. // // +stateify savable type ConnectedEndpoint struct { @@ -70,7 +70,7 @@ type ConnectedEndpoint struct { srfd int `state:"wait"` // stype is the type of Unix socket. - stype unix.SockType + stype transport.SockType // sndbuf is the size of the send buffer. // @@ -112,7 +112,7 @@ func (c *ConnectedEndpoint) init() *tcpip.Error { return tcpip.ErrInvalidEndpointState } - c.stype = unix.SockType(stype) + c.stype = transport.SockType(stype) c.sndbuf = sndbuf return nil @@ -122,8 +122,8 @@ func (c *ConnectedEndpoint) init() *tcpip.Error { // that will pretend to be bound at a given sentry path. // // The caller is responsible for calling Init(). Additionaly, Release needs to -// be called twice because ConnectedEndpoint is both a unix.Receiver and -// unix.ConnectedEndpoint. +// be called twice because ConnectedEndpoint is both a transport.Receiver and +// transport.ConnectedEndpoint. func NewConnectedEndpoint(file *fd.FD, queue *waiter.Queue, path string) (*ConnectedEndpoint, *tcpip.Error) { e := ConnectedEndpoint{ path: path, @@ -168,7 +168,7 @@ func NewSocketWithDirent(ctx context.Context, d *fs.Dirent, f *fd.FD, flags fs.F e.Init() - ep := unix.NewExternal(e.stype, uniqueid.GlobalProviderFromContext(ctx), &q, e, e) + ep := transport.NewExternal(e.stype, uniqueid.GlobalProviderFromContext(ctx), &q, e, e) return unixsocket.NewWithDirent(ctx, d, ep, flags), nil } @@ -200,13 +200,13 @@ func newSocket(ctx context.Context, orgfd int, saveable bool) (*fs.File, error) e.srfd = srfd e.Init() - ep := unix.NewExternal(e.stype, uniqueid.GlobalProviderFromContext(ctx), &q, e, e) + ep := transport.NewExternal(e.stype, uniqueid.GlobalProviderFromContext(ctx), &q, e, e) return unixsocket.New(ctx, ep), nil } -// Send implements unix.ConnectedEndpoint.Send. -func (c *ConnectedEndpoint) Send(data [][]byte, controlMessages unix.ControlMessages, from tcpip.FullAddress) (uintptr, bool, *tcpip.Error) { +// Send implements transport.ConnectedEndpoint.Send. +func (c *ConnectedEndpoint) Send(data [][]byte, controlMessages transport.ControlMessages, from tcpip.FullAddress) (uintptr, bool, *tcpip.Error) { c.mu.RLock() defer c.mu.RUnlock() if c.writeClosed { @@ -219,7 +219,7 @@ func (c *ConnectedEndpoint) Send(data [][]byte, controlMessages unix.ControlMess // Since stream sockets don't preserve message boundaries, we can write // only as much of the message as fits in the send buffer. - truncate := c.stype == unix.SockStream + truncate := c.stype == transport.SockStream n, totalLen, err := fdWriteVec(c.file.FD(), data, c.sndbuf, truncate) if n < totalLen && err == nil { @@ -239,20 +239,20 @@ func (c *ConnectedEndpoint) Send(data [][]byte, controlMessages unix.ControlMess return n, false, translateError(err) } -// SendNotify implements unix.ConnectedEndpoint.SendNotify. +// SendNotify implements transport.ConnectedEndpoint.SendNotify. func (c *ConnectedEndpoint) SendNotify() {} -// CloseSend implements unix.ConnectedEndpoint.CloseSend. +// CloseSend implements transport.ConnectedEndpoint.CloseSend. func (c *ConnectedEndpoint) CloseSend() { c.mu.Lock() c.writeClosed = true c.mu.Unlock() } -// CloseNotify implements unix.ConnectedEndpoint.CloseNotify. +// CloseNotify implements transport.ConnectedEndpoint.CloseNotify. func (c *ConnectedEndpoint) CloseNotify() {} -// Writable implements unix.ConnectedEndpoint.Writable. +// Writable implements transport.ConnectedEndpoint.Writable. func (c *ConnectedEndpoint) Writable() bool { c.mu.RLock() defer c.mu.RUnlock() @@ -262,18 +262,18 @@ func (c *ConnectedEndpoint) Writable() bool { return fdnotifier.NonBlockingPoll(int32(c.file.FD()), waiter.EventOut)&waiter.EventOut != 0 } -// Passcred implements unix.ConnectedEndpoint.Passcred. +// Passcred implements transport.ConnectedEndpoint.Passcred. func (c *ConnectedEndpoint) Passcred() bool { // We don't support credential passing for host sockets. return false } -// GetLocalAddress implements unix.ConnectedEndpoint.GetLocalAddress. +// GetLocalAddress implements transport.ConnectedEndpoint.GetLocalAddress. func (c *ConnectedEndpoint) GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) { return tcpip.FullAddress{Addr: tcpip.Address(c.path)}, nil } -// EventUpdate implements unix.ConnectedEndpoint.EventUpdate. +// EventUpdate implements transport.ConnectedEndpoint.EventUpdate. func (c *ConnectedEndpoint) EventUpdate() { c.mu.RLock() defer c.mu.RUnlock() @@ -282,12 +282,12 @@ func (c *ConnectedEndpoint) EventUpdate() { } } -// Recv implements unix.Receiver.Recv. -func (c *ConnectedEndpoint) Recv(data [][]byte, creds bool, numRights uintptr, peek bool) (uintptr, uintptr, unix.ControlMessages, tcpip.FullAddress, bool, *tcpip.Error) { +// Recv implements transport.Receiver.Recv. +func (c *ConnectedEndpoint) Recv(data [][]byte, creds bool, numRights uintptr, peek bool) (uintptr, uintptr, transport.ControlMessages, tcpip.FullAddress, bool, *tcpip.Error) { c.mu.RLock() defer c.mu.RUnlock() if c.readClosed { - return 0, 0, unix.ControlMessages{}, tcpip.FullAddress{}, false, tcpip.ErrClosedForReceive + return 0, 0, transport.ControlMessages{}, tcpip.FullAddress{}, false, tcpip.ErrClosedForReceive } var cm unet.ControlMessage @@ -305,7 +305,7 @@ func (c *ConnectedEndpoint) Recv(data [][]byte, creds bool, numRights uintptr, p err = nil } if err != nil { - return 0, 0, unix.ControlMessages{}, tcpip.FullAddress{}, false, translateError(err) + return 0, 0, transport.ControlMessages{}, tcpip.FullAddress{}, false, translateError(err) } // There is no need for the callee to call RecvNotify because fdReadVec uses @@ -318,16 +318,16 @@ func (c *ConnectedEndpoint) Recv(data [][]byte, creds bool, numRights uintptr, p // Avoid extra allocations in the case where there isn't any control data. if len(cm) == 0 { - return rl, ml, unix.ControlMessages{}, tcpip.FullAddress{Addr: tcpip.Address(c.path)}, false, nil + return rl, ml, transport.ControlMessages{}, tcpip.FullAddress{Addr: tcpip.Address(c.path)}, false, nil } fds, err := cm.ExtractFDs() if err != nil { - return 0, 0, unix.ControlMessages{}, tcpip.FullAddress{}, false, translateError(err) + return 0, 0, transport.ControlMessages{}, tcpip.FullAddress{}, false, translateError(err) } if len(fds) == 0 { - return rl, ml, unix.ControlMessages{}, tcpip.FullAddress{Addr: tcpip.Address(c.path)}, false, nil + return rl, ml, transport.ControlMessages{}, tcpip.FullAddress{Addr: tcpip.Address(c.path)}, false, nil } return rl, ml, control.New(nil, nil, newSCMRights(fds)), tcpip.FullAddress{Addr: tcpip.Address(c.path)}, false, nil } @@ -339,17 +339,17 @@ func (c *ConnectedEndpoint) close() { c.file = nil } -// RecvNotify implements unix.Receiver.RecvNotify. +// RecvNotify implements transport.Receiver.RecvNotify. func (c *ConnectedEndpoint) RecvNotify() {} -// CloseRecv implements unix.Receiver.CloseRecv. +// CloseRecv implements transport.Receiver.CloseRecv. func (c *ConnectedEndpoint) CloseRecv() { c.mu.Lock() c.readClosed = true c.mu.Unlock() } -// Readable implements unix.Receiver.Readable. +// Readable implements transport.Receiver.Readable. func (c *ConnectedEndpoint) Readable() bool { c.mu.RLock() defer c.mu.RUnlock() @@ -359,33 +359,33 @@ func (c *ConnectedEndpoint) Readable() bool { return fdnotifier.NonBlockingPoll(int32(c.file.FD()), waiter.EventIn)&waiter.EventIn != 0 } -// SendQueuedSize implements unix.Receiver.SendQueuedSize. +// SendQueuedSize implements transport.Receiver.SendQueuedSize. func (c *ConnectedEndpoint) SendQueuedSize() int64 { // SendQueuedSize isn't supported for host sockets because we don't allow the // sentry to call ioctl(2). return -1 } -// RecvQueuedSize implements unix.Receiver.RecvQueuedSize. +// RecvQueuedSize implements transport.Receiver.RecvQueuedSize. func (c *ConnectedEndpoint) RecvQueuedSize() int64 { // RecvQueuedSize isn't supported for host sockets because we don't allow the // sentry to call ioctl(2). return -1 } -// SendMaxQueueSize implements unix.Receiver.SendMaxQueueSize. +// SendMaxQueueSize implements transport.Receiver.SendMaxQueueSize. func (c *ConnectedEndpoint) SendMaxQueueSize() int64 { return int64(c.sndbuf) } -// RecvMaxQueueSize implements unix.Receiver.RecvMaxQueueSize. +// RecvMaxQueueSize implements transport.Receiver.RecvMaxQueueSize. func (c *ConnectedEndpoint) RecvMaxQueueSize() int64 { // N.B. Unix sockets don't use the receive buffer. We'll claim it is // the same size as the send buffer. return int64(c.sndbuf) } -// Release implements unix.ConnectedEndpoint.Release and unix.Receiver.Release. +// Release implements transport.ConnectedEndpoint.Release and transport.Receiver.Release. func (c *ConnectedEndpoint) Release() { c.ref.DecRefWithDestructor(c.close) } diff --git a/pkg/sentry/fs/host/socket_test.go b/pkg/sentry/fs/host/socket_test.go index 8b752737d..1c6f9ddb1 100644 --- a/pkg/sentry/fs/host/socket_test.go +++ b/pkg/sentry/fs/host/socket_test.go @@ -22,20 +22,20 @@ import ( "gvisor.googlesource.com/gvisor/pkg/fd" "gvisor.googlesource.com/gvisor/pkg/sentry/context/contexttest" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" "gvisor.googlesource.com/gvisor/pkg/waiter/fdnotifier" ) var ( - // Make sure that ConnectedEndpoint implements unix.ConnectedEndpoint. - _ = unix.ConnectedEndpoint(new(ConnectedEndpoint)) + // Make sure that ConnectedEndpoint implements transport.ConnectedEndpoint. + _ = transport.ConnectedEndpoint(new(ConnectedEndpoint)) - // Make sure that ConnectedEndpoint implements unix.Receiver. - _ = unix.Receiver(new(ConnectedEndpoint)) + // Make sure that ConnectedEndpoint implements transport.Receiver. + _ = transport.Receiver(new(ConnectedEndpoint)) ) func getFl(fd int) (uint32, error) { @@ -199,7 +199,7 @@ func TestListen(t *testing.T) { func TestSend(t *testing.T) { e := ConnectedEndpoint{writeClosed: true} - if _, _, err := e.Send(nil, unix.ControlMessages{}, tcpip.FullAddress{}); err != tcpip.ErrClosedForSend { + if _, _, err := e.Send(nil, transport.ControlMessages{}, tcpip.FullAddress{}); err != tcpip.ErrClosedForSend { t.Errorf("Got %#v.Send() = %v, want = %v", e, err, tcpip.ErrClosedForSend) } } diff --git a/pkg/sentry/fs/inode.go b/pkg/sentry/fs/inode.go index db7240dca..409c81a97 100644 --- a/pkg/sentry/fs/inode.go +++ b/pkg/sentry/fs/inode.go @@ -22,8 +22,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs/lock" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) // Inode is a file system object that can be simultaneously referenced by different @@ -223,7 +223,7 @@ func (i *Inode) Rename(ctx context.Context, oldParent *Dirent, renamed *Dirent, } // Bind calls i.InodeOperations.Bind with i as the directory. -func (i *Inode) Bind(ctx context.Context, name string, data unix.BoundEndpoint, perm FilePermissions) (*Dirent, error) { +func (i *Inode) Bind(ctx context.Context, name string, data transport.BoundEndpoint, perm FilePermissions) (*Dirent, error) { if i.overlay != nil { return overlayBind(ctx, i.overlay, name, data, perm) } @@ -231,7 +231,7 @@ func (i *Inode) Bind(ctx context.Context, name string, data unix.BoundEndpoint, } // BoundEndpoint calls i.InodeOperations.BoundEndpoint with i as the Inode. -func (i *Inode) BoundEndpoint(path string) unix.BoundEndpoint { +func (i *Inode) BoundEndpoint(path string) transport.BoundEndpoint { if i.overlay != nil { return overlayBoundEndpoint(i.overlay, path) } diff --git a/pkg/sentry/fs/inode_operations.go b/pkg/sentry/fs/inode_operations.go index 952f9704d..3ee3de10e 100644 --- a/pkg/sentry/fs/inode_operations.go +++ b/pkg/sentry/fs/inode_operations.go @@ -20,8 +20,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -146,7 +146,7 @@ type InodeOperations interface { // Implementations must ensure that name does not already exist. // // The caller must ensure that this operation is permitted. - Bind(ctx context.Context, dir *Inode, name string, data unix.BoundEndpoint, perm FilePermissions) (*Dirent, error) + Bind(ctx context.Context, dir *Inode, name string, data transport.BoundEndpoint, perm FilePermissions) (*Dirent, error) // BoundEndpoint returns the socket endpoint at path stored in // or generated by an Inode. @@ -160,7 +160,7 @@ type InodeOperations interface { // generally implies that this Inode was created via CreateSocket. // // If there is no socket endpoint available, nil will be returned. - BoundEndpoint(inode *Inode, path string) unix.BoundEndpoint + BoundEndpoint(inode *Inode, path string) transport.BoundEndpoint // GetFile returns a new open File backed by a Dirent and FileFlags. // It may block as long as it is done with ctx. diff --git a/pkg/sentry/fs/inode_overlay.go b/pkg/sentry/fs/inode_overlay.go index e18e095a0..cf698a4da 100644 --- a/pkg/sentry/fs/inode_overlay.go +++ b/pkg/sentry/fs/inode_overlay.go @@ -20,8 +20,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/log" "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) func overlayHasWhiteout(parent *Inode, name string) bool { @@ -356,7 +356,7 @@ func overlayRename(ctx context.Context, o *overlayEntry, oldParent *Dirent, rena return nil } -func overlayBind(ctx context.Context, o *overlayEntry, name string, data unix.BoundEndpoint, perm FilePermissions) (*Dirent, error) { +func overlayBind(ctx context.Context, o *overlayEntry, name string, data transport.BoundEndpoint, perm FilePermissions) (*Dirent, error) { o.copyMu.RLock() defer o.copyMu.RUnlock() // We do not support doing anything exciting with sockets unless there @@ -383,7 +383,7 @@ func overlayBind(ctx context.Context, o *overlayEntry, name string, data unix.Bo return NewDirent(newOverlayInode(ctx, entry, inode.MountSource), name), nil } -func overlayBoundEndpoint(o *overlayEntry, path string) unix.BoundEndpoint { +func overlayBoundEndpoint(o *overlayEntry, path string) transport.BoundEndpoint { o.copyMu.RLock() defer o.copyMu.RUnlock() diff --git a/pkg/sentry/fs/ramfs/BUILD b/pkg/sentry/fs/ramfs/BUILD index 5230157fe..a93ad6240 100644 --- a/pkg/sentry/fs/ramfs/BUILD +++ b/pkg/sentry/fs/ramfs/BUILD @@ -23,9 +23,9 @@ go_library( "//pkg/sentry/kernel/time", "//pkg/sentry/memmap", "//pkg/sentry/safemem", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserror", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index d8333194b..075e13b01 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -20,9 +20,9 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) // CreateOps represents operations to create different file types. @@ -37,7 +37,7 @@ type CreateOps struct { NewSymlink func(ctx context.Context, dir *fs.Inode, target string) (*fs.Inode, error) // NewBoundEndpoint creates a new socket. - NewBoundEndpoint func(ctx context.Context, dir *fs.Inode, ep unix.BoundEndpoint, perms fs.FilePermissions) (*fs.Inode, error) + NewBoundEndpoint func(ctx context.Context, dir *fs.Inode, ep transport.BoundEndpoint, perms fs.FilePermissions) (*fs.Inode, error) // NewFifo creates a new fifo. NewFifo func(ctx context.Context, dir *fs.Inode, perm fs.FilePermissions) (*fs.Inode, error) @@ -314,7 +314,7 @@ func (d *Dir) CreateDirectory(ctx context.Context, dir *fs.Inode, name string, p } // Bind implements fs.InodeOperations.Bind. -func (d *Dir) Bind(ctx context.Context, dir *fs.Inode, name string, ep unix.BoundEndpoint, perms fs.FilePermissions) (*fs.Dirent, error) { +func (d *Dir) Bind(ctx context.Context, dir *fs.Inode, name string, ep transport.BoundEndpoint, perms fs.FilePermissions) (*fs.Dirent, error) { if d.CreateOps == nil || d.CreateOps.NewBoundEndpoint == nil { return nil, ErrDenied } diff --git a/pkg/sentry/fs/ramfs/ramfs.go b/pkg/sentry/fs/ramfs/ramfs.go index 1028b5f1d..83cbcab23 100644 --- a/pkg/sentry/fs/ramfs/ramfs.go +++ b/pkg/sentry/fs/ramfs/ramfs.go @@ -26,9 +26,9 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -279,7 +279,7 @@ func (*Entry) CreateDirectory(context.Context, *fs.Inode, string, fs.FilePermiss } // Bind is not supported by default. -func (*Entry) Bind(context.Context, *fs.Inode, string, unix.BoundEndpoint, fs.FilePermissions) (*fs.Dirent, error) { +func (*Entry) Bind(context.Context, *fs.Inode, string, transport.BoundEndpoint, fs.FilePermissions) (*fs.Dirent, error) { return nil, ErrInvalidOp } diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go index 93427a1ff..9ac00eb18 100644 --- a/pkg/sentry/fs/ramfs/socket.go +++ b/pkg/sentry/fs/ramfs/socket.go @@ -17,7 +17,7 @@ package ramfs import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" ) // Socket represents a socket. @@ -27,17 +27,17 @@ type Socket struct { Entry // ep is the bound endpoint. - ep unix.BoundEndpoint + ep transport.BoundEndpoint } // InitSocket initializes a socket. -func (s *Socket) InitSocket(ctx context.Context, ep unix.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions) { +func (s *Socket) InitSocket(ctx context.Context, ep transport.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions) { s.InitEntry(ctx, owner, perms) s.ep = ep } // BoundEndpoint returns the socket data. -func (s *Socket) BoundEndpoint(*fs.Inode, string) unix.BoundEndpoint { +func (s *Socket) BoundEndpoint(*fs.Inode, string) transport.BoundEndpoint { // ramfs only supports stored sentry internal sockets. Only gofer sockets // care about the path argument. return s.ep diff --git a/pkg/sentry/fs/tmpfs/BUILD b/pkg/sentry/fs/tmpfs/BUILD index cfe11ab02..277583113 100644 --- a/pkg/sentry/fs/tmpfs/BUILD +++ b/pkg/sentry/fs/tmpfs/BUILD @@ -25,9 +25,9 @@ go_library( "//pkg/sentry/memmap", "//pkg/sentry/platform", "//pkg/sentry/safemem", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usage", "//pkg/sentry/usermem", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go index 10cb5451d..38be6db46 100644 --- a/pkg/sentry/fs/tmpfs/tmpfs.go +++ b/pkg/sentry/fs/tmpfs/tmpfs.go @@ -22,9 +22,9 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/pipe" "gvisor.googlesource.com/gvisor/pkg/sentry/platform" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usage" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) var fsInfo = fs.Info{ @@ -104,7 +104,7 @@ func (d *Dir) newCreateOps() *ramfs.CreateOps { NewSymlink: func(ctx context.Context, dir *fs.Inode, target string) (*fs.Inode, error) { return NewSymlink(ctx, target, fs.FileOwnerFromContext(ctx), dir.MountSource), nil }, - NewBoundEndpoint: func(ctx context.Context, dir *fs.Inode, socket unix.BoundEndpoint, perms fs.FilePermissions) (*fs.Inode, error) { + NewBoundEndpoint: func(ctx context.Context, dir *fs.Inode, socket transport.BoundEndpoint, perms fs.FilePermissions) (*fs.Inode, error) { return NewSocket(ctx, socket, fs.FileOwnerFromContext(ctx), perms, dir.MountSource), nil }, NewFifo: func(ctx context.Context, dir *fs.Inode, perms fs.FilePermissions) (*fs.Inode, error) { @@ -160,7 +160,7 @@ type Socket struct { } // NewSocket returns a new socket with the provided permissions. -func NewSocket(ctx context.Context, socket unix.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode { +func NewSocket(ctx context.Context, socket transport.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode { s := &Socket{} s.InitSocket(ctx, socket, owner, perms) return fs.NewInode(s, msrc, fs.StableAttr{ diff --git a/pkg/sentry/fs/tty/BUILD b/pkg/sentry/fs/tty/BUILD index 3c446eef4..d4dd20e30 100644 --- a/pkg/sentry/fs/tty/BUILD +++ b/pkg/sentry/fs/tty/BUILD @@ -26,9 +26,9 @@ go_library( "//pkg/sentry/fs/fsutil", "//pkg/sentry/kernel/auth", "//pkg/sentry/kernel/time", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserror", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index c6f39fce3..7c0c0b0c1 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -26,9 +26,9 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -215,7 +215,7 @@ func (d *dirInodeOperations) RemoveDirectory(ctx context.Context, dir *fs.Inode, } // Bind implements fs.InodeOperations.Bind. -func (d *dirInodeOperations) Bind(ctx context.Context, dir *fs.Inode, name string, data unix.BoundEndpoint, perm fs.FilePermissions) (*fs.Dirent, error) { +func (d *dirInodeOperations) Bind(ctx context.Context, dir *fs.Inode, name string, data transport.BoundEndpoint, perm fs.FilePermissions) (*fs.Dirent, error) { return nil, syserror.EPERM } diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD index 31ad96612..acc61cb09 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -156,6 +156,7 @@ go_library( "//pkg/sentry/platform", "//pkg/sentry/safemem", "//pkg/sentry/socket/netlink/port", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/time", "//pkg/sentry/uniqueid", "//pkg/sentry/usage", @@ -166,7 +167,6 @@ go_library( "//pkg/syserror", "//pkg/tcpip", "//pkg/tcpip/stack", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/kernel/abstract_socket_namespace.go b/pkg/sentry/kernel/abstract_socket_namespace.go index d6d1d341d..45088c988 100644 --- a/pkg/sentry/kernel/abstract_socket_namespace.go +++ b/pkg/sentry/kernel/abstract_socket_namespace.go @@ -19,12 +19,12 @@ import ( "syscall" "gvisor.googlesource.com/gvisor/pkg/refs" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" ) // +stateify savable type abstractEndpoint struct { - ep unix.BoundEndpoint + ep transport.BoundEndpoint wr *refs.WeakRef name string ns *AbstractSocketNamespace @@ -56,14 +56,14 @@ func NewAbstractSocketNamespace() *AbstractSocketNamespace { } } -// A boundEndpoint wraps a unix.BoundEndpoint to maintain a reference on its -// backing object. +// A boundEndpoint wraps a transport.BoundEndpoint to maintain a reference on +// its backing object. type boundEndpoint struct { - unix.BoundEndpoint + transport.BoundEndpoint rc refs.RefCounter } -// Release implements unix.BoundEndpoint.Release. +// Release implements transport.BoundEndpoint.Release. func (e *boundEndpoint) Release() { e.rc.DecRef() e.BoundEndpoint.Release() @@ -71,7 +71,7 @@ func (e *boundEndpoint) Release() { // BoundEndpoint retrieves the endpoint bound to the given name. The return // value is nil if no endpoint was bound. -func (a *AbstractSocketNamespace) BoundEndpoint(name string) unix.BoundEndpoint { +func (a *AbstractSocketNamespace) BoundEndpoint(name string) transport.BoundEndpoint { a.mu.Lock() defer a.mu.Unlock() @@ -93,7 +93,7 @@ func (a *AbstractSocketNamespace) BoundEndpoint(name string) unix.BoundEndpoint // // When the last reference managed by rc is dropped, ep may be removed from the // namespace. -func (a *AbstractSocketNamespace) Bind(name string, ep unix.BoundEndpoint, rc refs.RefCounter) error { +func (a *AbstractSocketNamespace) Bind(name string, ep transport.BoundEndpoint, rc refs.RefCounter) error { a.mu.Lock() defer a.mu.Unlock() diff --git a/pkg/sentry/socket/BUILD b/pkg/sentry/socket/BUILD index a320fca0b..3a8044b5f 100644 --- a/pkg/sentry/socket/BUILD +++ b/pkg/sentry/socket/BUILD @@ -16,9 +16,9 @@ go_library( "//pkg/sentry/kernel", "//pkg/sentry/kernel/kdefs", "//pkg/sentry/kernel/time", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/tcpip", - "//pkg/tcpip/transport/unix", ], ) diff --git a/pkg/sentry/socket/control/BUILD b/pkg/sentry/socket/control/BUILD index c4874fdfb..d3a63f15f 100644 --- a/pkg/sentry/socket/control/BUILD +++ b/pkg/sentry/socket/control/BUILD @@ -18,8 +18,8 @@ go_library( "//pkg/sentry/kernel", "//pkg/sentry/kernel/auth", "//pkg/sentry/kernel/kdefs", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserror", - "//pkg/tcpip/transport/unix", ], ) diff --git a/pkg/sentry/socket/control/control.go b/pkg/sentry/socket/control/control.go index c31182e69..db97e95f2 100644 --- a/pkg/sentry/socket/control/control.go +++ b/pkg/sentry/socket/control/control.go @@ -24,16 +24,16 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) const maxInt = int(^uint(0) >> 1) // SCMCredentials represents a SCM_CREDENTIALS socket control message. type SCMCredentials interface { - unix.CredentialsControlMessage + transport.CredentialsControlMessage // Credentials returns properly namespaced values for the sender's pid, uid // and gid. @@ -42,7 +42,7 @@ type SCMCredentials interface { // SCMRights represents a SCM_RIGHTS socket control message. type SCMRights interface { - unix.RightsControlMessage + transport.RightsControlMessage // Files returns up to max RightsFiles. Files(ctx context.Context, max int) RightsFiles @@ -81,8 +81,8 @@ func (fs *RightsFiles) Files(ctx context.Context, max int) RightsFiles { return rf } -// Clone implements unix.RightsControlMessage.Clone. -func (fs *RightsFiles) Clone() unix.RightsControlMessage { +// Clone implements transport.RightsControlMessage.Clone. +func (fs *RightsFiles) Clone() transport.RightsControlMessage { nfs := append(RightsFiles(nil), *fs...) for _, nf := range nfs { nf.IncRef() @@ -90,7 +90,7 @@ func (fs *RightsFiles) Clone() unix.RightsControlMessage { return &nfs } -// Release implements unix.RightsControlMessage.Release. +// Release implements transport.RightsControlMessage.Release. func (fs *RightsFiles) Release() { for _, f := range *fs { f.DecRef() @@ -156,8 +156,8 @@ func NewSCMCredentials(t *kernel.Task, cred linux.ControlMessageCredentials) (SC return &scmCredentials{t, kuid, kgid}, nil } -// Equals implements unix.CredentialsControlMessage.Equals. -func (c *scmCredentials) Equals(oc unix.CredentialsControlMessage) bool { +// Equals implements transport.CredentialsControlMessage.Equals. +func (c *scmCredentials) Equals(oc transport.CredentialsControlMessage) bool { if oc, _ := oc.(*scmCredentials); oc != nil && *c == *oc { return true } @@ -301,7 +301,7 @@ func PackTimestamp(t *kernel.Task, timestamp int64, buf []byte) []byte { } // Parse parses a raw socket control message into portable objects. -func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (unix.ControlMessages, error) { +func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (transport.ControlMessages, error) { var ( fds linux.ControlMessageRights @@ -311,20 +311,20 @@ func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (unix.Contr for i := 0; i < len(buf); { if i+linux.SizeOfControlMessageHeader > len(buf) { - return unix.ControlMessages{}, syserror.EINVAL + return transport.ControlMessages{}, syserror.EINVAL } var h linux.ControlMessageHeader binary.Unmarshal(buf[i:i+linux.SizeOfControlMessageHeader], usermem.ByteOrder, &h) if h.Length < uint64(linux.SizeOfControlMessageHeader) { - return unix.ControlMessages{}, syserror.EINVAL + return transport.ControlMessages{}, syserror.EINVAL } if h.Length > uint64(len(buf)-i) { - return unix.ControlMessages{}, syserror.EINVAL + return transport.ControlMessages{}, syserror.EINVAL } if h.Level != linux.SOL_SOCKET { - return unix.ControlMessages{}, syserror.EINVAL + return transport.ControlMessages{}, syserror.EINVAL } i += linux.SizeOfControlMessageHeader @@ -340,7 +340,7 @@ func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (unix.Contr numRights := rightsSize / linux.SizeOfControlMessageRight if len(fds)+numRights > linux.SCM_MAX_FD { - return unix.ControlMessages{}, syserror.EINVAL + return transport.ControlMessages{}, syserror.EINVAL } for j := i; j < i+rightsSize; j += linux.SizeOfControlMessageRight { @@ -351,7 +351,7 @@ func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (unix.Contr case linux.SCM_CREDENTIALS: if length < linux.SizeOfControlMessageCredentials { - return unix.ControlMessages{}, syserror.EINVAL + return transport.ControlMessages{}, syserror.EINVAL } binary.Unmarshal(buf[i:i+linux.SizeOfControlMessageCredentials], usermem.ByteOrder, &creds) @@ -360,7 +360,7 @@ func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (unix.Contr default: // Unknown message type. - return unix.ControlMessages{}, syserror.EINVAL + return transport.ControlMessages{}, syserror.EINVAL } } @@ -368,7 +368,7 @@ func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (unix.Contr if haveCreds { var err error if credentials, err = NewSCMCredentials(t, creds); err != nil { - return unix.ControlMessages{}, err + return transport.ControlMessages{}, err } } else { credentials = makeCreds(t, socketOrEndpoint) @@ -378,22 +378,22 @@ func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte) (unix.Contr if len(fds) > 0 { var err error if rights, err = NewSCMRights(t, fds); err != nil { - return unix.ControlMessages{}, err + return transport.ControlMessages{}, err } } if credentials == nil && rights == nil { - return unix.ControlMessages{}, nil + return transport.ControlMessages{}, nil } - return unix.ControlMessages{Credentials: credentials, Rights: rights}, nil + return transport.ControlMessages{Credentials: credentials, Rights: rights}, nil } func makeCreds(t *kernel.Task, socketOrEndpoint interface{}) SCMCredentials { if t == nil || socketOrEndpoint == nil { return nil } - if cr, ok := socketOrEndpoint.(unix.Credentialer); ok && (cr.Passcred() || cr.ConnectedPasscred()) { + if cr, ok := socketOrEndpoint.(transport.Credentialer); ok && (cr.Passcred() || cr.ConnectedPasscred()) { tcred := t.Credentials() return &scmCredentials{t, tcred.EffectiveKUID, tcred.EffectiveKGID} } @@ -401,8 +401,8 @@ func makeCreds(t *kernel.Task, socketOrEndpoint interface{}) SCMCredentials { } // New creates default control messages if needed. -func New(t *kernel.Task, socketOrEndpoint interface{}, rights SCMRights) unix.ControlMessages { - return unix.ControlMessages{ +func New(t *kernel.Task, socketOrEndpoint interface{}, rights SCMRights) transport.ControlMessages { + return transport.ControlMessages{ Credentials: makeCreds(t, socketOrEndpoint), Rights: rights, } diff --git a/pkg/sentry/socket/epsocket/BUILD b/pkg/sentry/socket/epsocket/BUILD index 7f9ea9edc..dbabc931c 100644 --- a/pkg/sentry/socket/epsocket/BUILD +++ b/pkg/sentry/socket/epsocket/BUILD @@ -31,6 +31,7 @@ go_library( "//pkg/sentry/kernel/time", "//pkg/sentry/safemem", "//pkg/sentry/socket", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", @@ -42,7 +43,6 @@ go_library( "//pkg/tcpip/stack", "//pkg/tcpip/transport/tcp", "//pkg/tcpip/transport/udp", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index a44679f0b..47c575e7b 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -44,13 +44,13 @@ import ( ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/tcpip" "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" "gvisor.googlesource.com/gvisor/pkg/tcpip/stack" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -108,26 +108,26 @@ func htons(v uint16) uint16 { } // commonEndpoint represents the intersection of a tcpip.Endpoint and a -// unix.Endpoint. +// transport.Endpoint. type commonEndpoint interface { // GetLocalAddress implements tcpip.Endpoint.GetLocalAddress and - // unix.Endpoint.GetLocalAddress. + // transport.Endpoint.GetLocalAddress. GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) // GetRemoteAddress implements tcpip.Endpoint.GetRemoteAddress and - // unix.Endpoint.GetRemoteAddress. + // transport.Endpoint.GetRemoteAddress. GetRemoteAddress() (tcpip.FullAddress, *tcpip.Error) // Readiness implements tcpip.Endpoint.Readiness and - // unix.Endpoint.Readiness. + // transport.Endpoint.Readiness. Readiness(mask waiter.EventMask) waiter.EventMask // SetSockOpt implements tcpip.Endpoint.SetSockOpt and - // unix.Endpoint.SetSockOpt. + // transport.Endpoint.SetSockOpt. SetSockOpt(interface{}) *tcpip.Error // GetSockOpt implements tcpip.Endpoint.GetSockOpt and - // unix.Endpoint.GetSockOpt. + // transport.Endpoint.GetSockOpt. GetSockOpt(interface{}) *tcpip.Error } @@ -146,7 +146,7 @@ type SocketOperations struct { family int Endpoint tcpip.Endpoint - skType unix.SockType + skType transport.SockType // readMu protects access to readView, control, and sender. readMu sync.Mutex `state:"nosave"` @@ -156,7 +156,7 @@ type SocketOperations struct { } // New creates a new endpoint socket. -func New(t *kernel.Task, family int, skType unix.SockType, queue *waiter.Queue, endpoint tcpip.Endpoint) *fs.File { +func New(t *kernel.Task, family int, skType transport.SockType, queue *waiter.Queue, endpoint tcpip.Endpoint) *fs.File { dirent := socket.NewDirent(t, epsocketDevice) defer dirent.DecRef() return fs.NewFile(t, dirent, fs.FileFlags{Read: true, Write: true}, &SocketOperations{ @@ -502,7 +502,7 @@ func (s *SocketOperations) GetSockOpt(t *kernel.Task, level, name, outLen int) ( // GetSockOpt can be used to implement the linux syscall getsockopt(2) for // sockets backed by a commonEndpoint. -func GetSockOpt(t *kernel.Task, s socket.Socket, ep commonEndpoint, family int, skType unix.SockType, level, name, outLen int) (interface{}, *syserr.Error) { +func GetSockOpt(t *kernel.Task, s socket.Socket, ep commonEndpoint, family int, skType transport.SockType, level, name, outLen int) (interface{}, *syserr.Error) { switch level { case linux.SOL_SOCKET: switch name { diff --git a/pkg/sentry/socket/epsocket/provider.go b/pkg/sentry/socket/epsocket/provider.go index 6c1e3b6b9..dbc232d26 100644 --- a/pkg/sentry/socket/epsocket/provider.go +++ b/pkg/sentry/socket/epsocket/provider.go @@ -21,6 +21,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/tcpip" "gvisor.googlesource.com/gvisor/pkg/tcpip/header" @@ -28,7 +29,6 @@ import ( "gvisor.googlesource.com/gvisor/pkg/tcpip/network/ipv6" "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/tcp" "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/udp" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -40,7 +40,7 @@ type provider struct { // GetTransportProtocol figures out transport protocol. Currently only TCP, // UDP, and ICMP are supported. -func GetTransportProtocol(stype unix.SockType, protocol int) (tcpip.TransportProtocolNumber, *syserr.Error) { +func GetTransportProtocol(stype transport.SockType, protocol int) (tcpip.TransportProtocolNumber, *syserr.Error) { switch stype { case linux.SOCK_STREAM: if protocol != 0 && protocol != syscall.IPPROTO_TCP { @@ -62,7 +62,7 @@ func GetTransportProtocol(stype unix.SockType, protocol int) (tcpip.TransportPro } // Socket creates a new socket object for the AF_INET or AF_INET6 family. -func (p *provider) Socket(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *syserr.Error) { +func (p *provider) Socket(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *syserr.Error) { // Fail right away if we don't have a stack. stack := t.NetworkContext() if stack == nil { @@ -92,7 +92,7 @@ func (p *provider) Socket(t *kernel.Task, stype unix.SockType, protocol int) (*f } // Pair just returns nil sockets (not supported). -func (*provider) Pair(*kernel.Task, unix.SockType, int) (*fs.File, *fs.File, *syserr.Error) { +func (*provider) Pair(*kernel.Task, transport.SockType, int) (*fs.File, *fs.File, *syserr.Error) { return nil, nil, nil } diff --git a/pkg/sentry/socket/hostinet/BUILD b/pkg/sentry/socket/hostinet/BUILD index d623718b3..c30220a46 100644 --- a/pkg/sentry/socket/hostinet/BUILD +++ b/pkg/sentry/socket/hostinet/BUILD @@ -29,10 +29,10 @@ go_library( "//pkg/sentry/kernel/time", "//pkg/sentry/safemem", "//pkg/sentry/socket", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", - "//pkg/tcpip/transport/unix", "//pkg/waiter", "//pkg/waiter/fdnotifier", ], diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index d0f3054dc..e82624b44 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -27,10 +27,10 @@ import ( ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" "gvisor.googlesource.com/gvisor/pkg/waiter/fdnotifier" ) @@ -511,7 +511,7 @@ type socketProvider struct { } // Socket implements socket.Provider.Socket. -func (p *socketProvider) Socket(t *kernel.Task, stypeflags unix.SockType, protocol int) (*fs.File, *syserr.Error) { +func (p *socketProvider) Socket(t *kernel.Task, stypeflags transport.SockType, protocol int) (*fs.File, *syserr.Error) { // Check that we are using the host network stack. stack := t.NetworkContext() if stack == nil { @@ -553,7 +553,7 @@ func (p *socketProvider) Socket(t *kernel.Task, stypeflags unix.SockType, protoc } // Pair implements socket.Provider.Pair. -func (p *socketProvider) Pair(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { +func (p *socketProvider) Pair(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { // Not supported by AF_INET/AF_INET6. return nil, nil, nil } diff --git a/pkg/sentry/socket/netlink/BUILD b/pkg/sentry/socket/netlink/BUILD index b852165f7..cff922cb8 100644 --- a/pkg/sentry/socket/netlink/BUILD +++ b/pkg/sentry/socket/netlink/BUILD @@ -25,11 +25,11 @@ go_library( "//pkg/sentry/socket", "//pkg/sentry/socket/netlink/port", "//pkg/sentry/socket/unix", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", "//pkg/tcpip", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/socket/netlink/provider.go b/pkg/sentry/socket/netlink/provider.go index e874216f4..5d0a04a07 100644 --- a/pkg/sentry/socket/netlink/provider.go +++ b/pkg/sentry/socket/netlink/provider.go @@ -22,8 +22,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/syserr" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) // Protocol is the implementation of a netlink socket protocol. @@ -66,10 +66,10 @@ type socketProvider struct { } // Socket implements socket.Provider.Socket. -func (*socketProvider) Socket(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *syserr.Error) { +func (*socketProvider) Socket(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *syserr.Error) { // Netlink sockets must be specified as datagram or raw, but they // behave the same regardless of type. - if stype != unix.SockDgram && stype != unix.SockRaw { + if stype != transport.SockDgram && stype != transport.SockRaw { return nil, syserr.ErrSocketNotSupported } @@ -94,7 +94,7 @@ func (*socketProvider) Socket(t *kernel.Task, stype unix.SockType, protocol int) } // Pair implements socket.Provider.Pair by returning an error. -func (*socketProvider) Pair(*kernel.Task, unix.SockType, int) (*fs.File, *fs.File, *syserr.Error) { +func (*socketProvider) Pair(*kernel.Task, transport.SockType, int) (*fs.File, *fs.File, *syserr.Error) { // Netlink sockets never supports creating socket pairs. return nil, nil, syserr.ErrNotSupported } diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index f3b2c7256..0c03997f2 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -31,12 +31,12 @@ import ( ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink/port" - sunix "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -80,11 +80,11 @@ type Socket struct { // ep is a datagram unix endpoint used to buffer messages sent from the // kernel to userspace. RecvMsg reads messages from this endpoint. - ep unix.Endpoint + ep transport.Endpoint // connection is the kernel's connection to ep, used to write messages // sent to userspace. - connection unix.ConnectedEndpoint + connection transport.ConnectedEndpoint // mu protects the fields below. mu sync.Mutex `state:"nosave"` @@ -105,7 +105,7 @@ var _ socket.Socket = (*Socket)(nil) // NewSocket creates a new Socket. func NewSocket(t *kernel.Task, protocol Protocol) (*Socket, *syserr.Error) { // Datagram endpoint used to buffer kernel -> user messages. - ep := unix.NewConnectionless() + ep := transport.NewConnectionless() // Bind the endpoint for good measure so we can connect to it. The // bound address will never be exposed. @@ -115,7 +115,7 @@ func NewSocket(t *kernel.Task, protocol Protocol) (*Socket, *syserr.Error) { } // Create a connection from which the kernel can write messages. - connection, terr := ep.(unix.BoundEndpoint).UnidirectionalConnect() + connection, terr := ep.(transport.BoundEndpoint).UnidirectionalConnect() if terr != nil { ep.Close() return nil, syserr.TranslateNetstackError(terr) @@ -368,7 +368,7 @@ func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, have trunc := flags&linux.MSG_TRUNC != 0 - r := sunix.EndpointReader{ + r := unix.EndpointReader{ Endpoint: s.ep, Peek: flags&linux.MSG_PEEK != 0, } @@ -408,7 +408,7 @@ func (s *Socket) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, _ if dst.NumBytes() == 0 { return 0, nil } - return dst.CopyOutFrom(ctx, &sunix.EndpointReader{ + return dst.CopyOutFrom(ctx, &unix.EndpointReader{ Endpoint: s.ep, }) } @@ -424,7 +424,7 @@ func (s *Socket) sendResponse(ctx context.Context, ms *MessageSet) *syserr.Error if len(bufs) > 0 { // RecvMsg never receives the address, so we don't need to send // one. - _, notify, terr := s.connection.Send(bufs, unix.ControlMessages{}, tcpip.FullAddress{}) + _, notify, terr := s.connection.Send(bufs, transport.ControlMessages{}, tcpip.FullAddress{}) // If the buffer is full, we simply drop messages, just like // Linux. if terr != nil && terr != tcpip.ErrWouldBlock { @@ -448,7 +448,7 @@ func (s *Socket) sendResponse(ctx context.Context, ms *MessageSet) *syserr.Error PortID: uint32(ms.PortID), }) - _, notify, terr := s.connection.Send([][]byte{m.Finalize()}, unix.ControlMessages{}, tcpip.FullAddress{}) + _, notify, terr := s.connection.Send([][]byte{m.Finalize()}, transport.ControlMessages{}, tcpip.FullAddress{}) if terr != nil && terr != tcpip.ErrWouldBlock { return syserr.TranslateNetstackError(terr) } diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index 288199779..3ea433360 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -31,12 +31,12 @@ go_library( "//pkg/sentry/socket/hostinet", "//pkg/sentry/socket/rpcinet/conn", "//pkg/sentry/socket/rpcinet/notifier", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", "//pkg/tcpip", "//pkg/tcpip/buffer", - "//pkg/tcpip/transport/unix", "//pkg/unet", "//pkg/waiter", ], diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 72fa1ca8f..c7e761d54 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -31,12 +31,12 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/conn" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/notifier" pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/tcpip" "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) @@ -763,7 +763,7 @@ type socketProvider struct { } // Socket implements socket.Provider.Socket. -func (p *socketProvider) Socket(t *kernel.Task, stypeflags unix.SockType, protocol int) (*fs.File, *syserr.Error) { +func (p *socketProvider) Socket(t *kernel.Task, stypeflags transport.SockType, protocol int) (*fs.File, *syserr.Error) { // Check that we are using the RPC network stack. stack := t.NetworkContext() if stack == nil { @@ -803,7 +803,7 @@ func (p *socketProvider) Socket(t *kernel.Task, stypeflags unix.SockType, protoc } // Pair implements socket.Provider.Pair. -func (p *socketProvider) Pair(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { +func (p *socketProvider) Pair(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { // Not supported by AF_INET/AF_INET6. return nil, nil, nil } diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index 54fe64595..31f8d42d7 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -29,16 +29,16 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) // ControlMessages represents the union of unix control messages and tcpip // control messages. type ControlMessages struct { - Unix unix.ControlMessages + Unix transport.ControlMessages IP tcpip.ControlMessages } @@ -109,12 +109,12 @@ type Provider interface { // If a nil Socket _and_ a nil error is returned, it means that the // protocol is not supported. A non-nil error should only be returned // if the protocol is supported, but an error occurs during creation. - Socket(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *syserr.Error) + Socket(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *syserr.Error) // Pair creates a pair of connected sockets. // // See Socket for error information. - Pair(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) + Pair(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) } // families holds a map of all known address families and their providers. @@ -128,7 +128,7 @@ func RegisterProvider(family int, provider Provider) { } // New creates a new socket with the given family, type and protocol. -func New(t *kernel.Task, family int, stype unix.SockType, protocol int) (*fs.File, *syserr.Error) { +func New(t *kernel.Task, family int, stype transport.SockType, protocol int) (*fs.File, *syserr.Error) { for _, p := range families[family] { s, err := p.Socket(t, stype, protocol) if err != nil { @@ -144,7 +144,7 @@ func New(t *kernel.Task, family int, stype unix.SockType, protocol int) (*fs.Fil // Pair creates a new connected socket pair with the given family, type and // protocol. -func Pair(t *kernel.Task, family int, stype unix.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { +func Pair(t *kernel.Task, family int, stype transport.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { providers, ok := families[family] if !ok { return nil, nil, syserr.ErrAddressFamilyNotSupported diff --git a/pkg/sentry/socket/unix/BUILD b/pkg/sentry/socket/unix/BUILD index 9fe681e9a..a12fa93db 100644 --- a/pkg/sentry/socket/unix/BUILD +++ b/pkg/sentry/socket/unix/BUILD @@ -26,11 +26,11 @@ go_library( "//pkg/sentry/socket", "//pkg/sentry/socket/control", "//pkg/sentry/socket/epsocket", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", "//pkg/tcpip", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/socket/unix/io.go b/pkg/sentry/socket/unix/io.go index 0ca2e35d0..06333e14b 100644 --- a/pkg/sentry/socket/unix/io.go +++ b/pkg/sentry/socket/unix/io.go @@ -16,23 +16,23 @@ package unix import ( "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) -// EndpointWriter implements safemem.Writer that writes to a unix.Endpoint. +// EndpointWriter implements safemem.Writer that writes to a transport.Endpoint. // // EndpointWriter is not thread-safe. type EndpointWriter struct { - // Endpoint is the unix.Endpoint to write to. - Endpoint unix.Endpoint + // Endpoint is the transport.Endpoint to write to. + Endpoint transport.Endpoint // Control is the control messages to send. - Control unix.ControlMessages + Control transport.ControlMessages // To is the endpoint to send to. May be nil. - To unix.BoundEndpoint + To transport.BoundEndpoint } // WriteFromBlocks implements safemem.Writer.WriteFromBlocks. @@ -46,12 +46,13 @@ func (w *EndpointWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) }}.WriteFromBlocks(srcs) } -// EndpointReader implements safemem.Reader that reads from a unix.Endpoint. +// EndpointReader implements safemem.Reader that reads from a +// transport.Endpoint. // // EndpointReader is not thread-safe. type EndpointReader struct { - // Endpoint is the unix.Endpoint to read from. - Endpoint unix.Endpoint + // Endpoint is the transport.Endpoint to read from. + Endpoint transport.Endpoint // Creds indicates if credential control messages are requested. Creds bool @@ -71,7 +72,7 @@ type EndpointReader struct { From *tcpip.FullAddress // Control contains the received control messages. - Control unix.ControlMessages + Control transport.ControlMessages } // ReadToBlocks implements safemem.Reader.ReadToBlocks. diff --git a/pkg/sentry/socket/unix/transport/BUILD b/pkg/sentry/socket/unix/transport/BUILD new file mode 100644 index 000000000..04ef0d438 --- /dev/null +++ b/pkg/sentry/socket/unix/transport/BUILD @@ -0,0 +1,22 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("//tools/go_stateify:defs.bzl", "go_library") + +go_library( + name = "transport", + srcs = [ + "connectioned.go", + "connectioned_state.go", + "connectionless.go", + "unix.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport", + visibility = ["//:sandbox"], + deps = [ + "//pkg/ilist", + "//pkg/sentry/socket/unix/transport/queue", + "//pkg/tcpip", + "//pkg/tcpip/buffer", + "//pkg/waiter", + ], +) diff --git a/pkg/sentry/socket/unix/transport/connectioned.go b/pkg/sentry/socket/unix/transport/connectioned.go new file mode 100644 index 000000000..f09935765 --- /dev/null +++ b/pkg/sentry/socket/unix/transport/connectioned.go @@ -0,0 +1,454 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package transport + +import ( + "sync" + + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport/queue" + "gvisor.googlesource.com/gvisor/pkg/tcpip" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// UniqueIDProvider generates a sequence of unique identifiers useful for, +// among other things, lock ordering. +type UniqueIDProvider interface { + // UniqueID returns a new unique identifier. + UniqueID() uint64 +} + +// A ConnectingEndpoint is a connectioned unix endpoint that is attempting to +// establish a bidirectional connection with a BoundEndpoint. +type ConnectingEndpoint interface { + // ID returns the endpoint's globally unique identifier. This identifier + // must be used to determine locking order if more than one endpoint is + // to be locked in the same codepath. The endpoint with the smaller + // identifier must be locked before endpoints with larger identifiers. + ID() uint64 + + // Passcred implements socket.Credentialer.Passcred. + Passcred() bool + + // Type returns the socket type, typically either SockStream or + // SockSeqpacket. The connection attempt must be aborted if this + // value doesn't match the ConnectableEndpoint's type. + Type() SockType + + // GetLocalAddress returns the bound path. + GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) + + // Locker protects the following methods. While locked, only the holder of + // the lock can change the return value of the protected methods. + sync.Locker + + // Connected returns true iff the ConnectingEndpoint is in the connected + // state. ConnectingEndpoints can only be connected to a single endpoint, + // so the connection attempt must be aborted if this returns true. + Connected() bool + + // Listening returns true iff the ConnectingEndpoint is in the listening + // state. ConnectingEndpoints cannot make connections while listening, so + // the connection attempt must be aborted if this returns true. + Listening() bool + + // WaiterQueue returns a pointer to the endpoint's waiter queue. + WaiterQueue() *waiter.Queue +} + +// connectionedEndpoint is a Unix-domain connected or connectable endpoint and implements +// ConnectingEndpoint, ConnectableEndpoint and tcpip.Endpoint. +// +// connectionedEndpoints must be in connected state in order to transfer data. +// +// This implementation includes STREAM and SEQPACKET Unix sockets created with +// socket(2), accept(2) or socketpair(2) and dgram unix sockets created with +// socketpair(2). See unix_connectionless.go for the implementation of DGRAM +// Unix sockets created with socket(2). +// +// The state is much simpler than a TCP endpoint, so it is not encoded +// explicitly. Instead we enforce the following invariants: +// +// receiver != nil, connected != nil => connected. +// path != "" && acceptedChan == nil => bound, not listening. +// path != "" && acceptedChan != nil => bound and listening. +// +// Only one of these will be true at any moment. +// +// +stateify savable +type connectionedEndpoint struct { + baseEndpoint + + // id is the unique endpoint identifier. This is used exclusively for + // lock ordering within connect. + id uint64 + + // idGenerator is used to generate new unique endpoint identifiers. + idGenerator UniqueIDProvider + + // stype is used by connecting sockets to ensure that they are the + // same type. The value is typically either tcpip.SockSeqpacket or + // tcpip.SockStream. + stype SockType + + // acceptedChan is per the TCP endpoint implementation. Note that the + // sockets in this channel are _already in the connected state_, and + // have another associated connectionedEndpoint. + // + // If nil, then no listen call has been made. + acceptedChan chan *connectionedEndpoint `state:".([]*connectionedEndpoint)"` +} + +// NewConnectioned creates a new unbound connectionedEndpoint. +func NewConnectioned(stype SockType, uid UniqueIDProvider) Endpoint { + return &connectionedEndpoint{ + baseEndpoint: baseEndpoint{Queue: &waiter.Queue{}}, + id: uid.UniqueID(), + idGenerator: uid, + stype: stype, + } +} + +// NewPair allocates a new pair of connected unix-domain connectionedEndpoints. +func NewPair(stype SockType, uid UniqueIDProvider) (Endpoint, Endpoint) { + a := &connectionedEndpoint{ + baseEndpoint: baseEndpoint{Queue: &waiter.Queue{}}, + id: uid.UniqueID(), + idGenerator: uid, + stype: stype, + } + b := &connectionedEndpoint{ + baseEndpoint: baseEndpoint{Queue: &waiter.Queue{}}, + id: uid.UniqueID(), + idGenerator: uid, + stype: stype, + } + + q1 := queue.New(a.Queue, b.Queue, initialLimit) + q2 := queue.New(b.Queue, a.Queue, initialLimit) + + if stype == SockStream { + a.receiver = &streamQueueReceiver{queueReceiver: queueReceiver{q1}} + b.receiver = &streamQueueReceiver{queueReceiver: queueReceiver{q2}} + } else { + a.receiver = &queueReceiver{q1} + b.receiver = &queueReceiver{q2} + } + + a.connected = &connectedEndpoint{ + endpoint: b, + writeQueue: q2, + } + b.connected = &connectedEndpoint{ + endpoint: a, + writeQueue: q1, + } + + return a, b +} + +// NewExternal creates a new externally backed Endpoint. It behaves like a +// socketpair. +func NewExternal(stype SockType, uid UniqueIDProvider, queue *waiter.Queue, receiver Receiver, connected ConnectedEndpoint) Endpoint { + return &connectionedEndpoint{ + baseEndpoint: baseEndpoint{Queue: queue, receiver: receiver, connected: connected}, + id: uid.UniqueID(), + idGenerator: uid, + stype: stype, + } +} + +// ID implements ConnectingEndpoint.ID. +func (e *connectionedEndpoint) ID() uint64 { + return e.id +} + +// Type implements ConnectingEndpoint.Type and Endpoint.Type. +func (e *connectionedEndpoint) Type() SockType { + return e.stype +} + +// WaiterQueue implements ConnectingEndpoint.WaiterQueue. +func (e *connectionedEndpoint) WaiterQueue() *waiter.Queue { + return e.Queue +} + +// isBound returns true iff the connectionedEndpoint is bound (but not +// listening). +func (e *connectionedEndpoint) isBound() bool { + return e.path != "" && e.acceptedChan == nil +} + +// Listening implements ConnectingEndpoint.Listening. +func (e *connectionedEndpoint) Listening() bool { + return e.acceptedChan != nil +} + +// Close puts the connectionedEndpoint in a closed state and frees all +// resources associated with it. +// +// The socket will be a fresh state after a call to close and may be reused. +// That is, close may be used to "unbind" or "disconnect" the socket in error +// paths. +func (e *connectionedEndpoint) Close() { + e.Lock() + var c ConnectedEndpoint + var r Receiver + switch { + case e.Connected(): + e.connected.CloseSend() + e.receiver.CloseRecv() + c = e.connected + r = e.receiver + e.connected = nil + e.receiver = nil + case e.isBound(): + e.path = "" + case e.Listening(): + close(e.acceptedChan) + for n := range e.acceptedChan { + n.Close() + } + e.acceptedChan = nil + e.path = "" + } + e.Unlock() + if c != nil { + c.CloseNotify() + c.Release() + } + if r != nil { + r.CloseNotify() + r.Release() + } +} + +// BidirectionalConnect implements BoundEndpoint.BidirectionalConnect. +func (e *connectionedEndpoint) BidirectionalConnect(ce ConnectingEndpoint, returnConnect func(Receiver, ConnectedEndpoint)) *tcpip.Error { + if ce.Type() != e.stype { + return tcpip.ErrConnectionRefused + } + + // Check if ce is e to avoid a deadlock. + if ce, ok := ce.(*connectionedEndpoint); ok && ce == e { + return tcpip.ErrInvalidEndpointState + } + + // Do a dance to safely acquire locks on both endpoints. + if e.id < ce.ID() { + e.Lock() + ce.Lock() + } else { + ce.Lock() + e.Lock() + } + + // Check connecting state. + if ce.Connected() { + e.Unlock() + ce.Unlock() + return tcpip.ErrAlreadyConnected + } + if ce.Listening() { + e.Unlock() + ce.Unlock() + return tcpip.ErrInvalidEndpointState + } + + // Check bound state. + if !e.Listening() { + e.Unlock() + ce.Unlock() + return tcpip.ErrConnectionRefused + } + + // Create a newly bound connectionedEndpoint. + ne := &connectionedEndpoint{ + baseEndpoint: baseEndpoint{ + path: e.path, + Queue: &waiter.Queue{}, + }, + id: e.idGenerator.UniqueID(), + idGenerator: e.idGenerator, + stype: e.stype, + } + readQueue := queue.New(ce.WaiterQueue(), ne.Queue, initialLimit) + writeQueue := queue.New(ne.Queue, ce.WaiterQueue(), initialLimit) + ne.connected = &connectedEndpoint{ + endpoint: ce, + writeQueue: readQueue, + } + if e.stype == SockStream { + ne.receiver = &streamQueueReceiver{queueReceiver: queueReceiver{readQueue: writeQueue}} + } else { + ne.receiver = &queueReceiver{readQueue: writeQueue} + } + + select { + case e.acceptedChan <- ne: + // Commit state. + connected := &connectedEndpoint{ + endpoint: ne, + writeQueue: writeQueue, + } + if e.stype == SockStream { + returnConnect(&streamQueueReceiver{queueReceiver: queueReceiver{readQueue: readQueue}}, connected) + } else { + returnConnect(&queueReceiver{readQueue: readQueue}, connected) + } + + // Notify can deadlock if we are holding these locks. + e.Unlock() + ce.Unlock() + + // Notify on both ends. + e.Notify(waiter.EventIn) + ce.WaiterQueue().Notify(waiter.EventOut) + + return nil + default: + // Busy; return ECONNREFUSED per spec. + ne.Close() + e.Unlock() + ce.Unlock() + return tcpip.ErrConnectionRefused + } +} + +// UnidirectionalConnect implements BoundEndpoint.UnidirectionalConnect. +func (e *connectionedEndpoint) UnidirectionalConnect() (ConnectedEndpoint, *tcpip.Error) { + return nil, tcpip.ErrConnectionRefused +} + +// Connect attempts to directly connect to another Endpoint. +// Implements Endpoint.Connect. +func (e *connectionedEndpoint) Connect(server BoundEndpoint) *tcpip.Error { + returnConnect := func(r Receiver, ce ConnectedEndpoint) { + e.receiver = r + e.connected = ce + } + + return server.BidirectionalConnect(e, returnConnect) +} + +// Listen starts listening on the connection. +func (e *connectionedEndpoint) Listen(backlog int) *tcpip.Error { + e.Lock() + defer e.Unlock() + if e.Listening() { + // Adjust the size of the channel iff we can fix existing + // pending connections into the new one. + if len(e.acceptedChan) > backlog { + return tcpip.ErrInvalidEndpointState + } + origChan := e.acceptedChan + e.acceptedChan = make(chan *connectionedEndpoint, backlog) + close(origChan) + for ep := range origChan { + e.acceptedChan <- ep + } + return nil + } + if !e.isBound() { + return tcpip.ErrInvalidEndpointState + } + + // Normal case. + e.acceptedChan = make(chan *connectionedEndpoint, backlog) + return nil +} + +// Accept accepts a new connection. +func (e *connectionedEndpoint) Accept() (Endpoint, *tcpip.Error) { + e.Lock() + defer e.Unlock() + + if !e.Listening() { + return nil, tcpip.ErrInvalidEndpointState + } + + select { + case ne := <-e.acceptedChan: + return ne, nil + + default: + // Nothing left. + return nil, tcpip.ErrWouldBlock + } +} + +// Bind binds the connection. +// +// For Unix connectionedEndpoints, this _only sets the address associated with +// the socket_. Work associated with sockets in the filesystem or finding those +// sockets must be done by a higher level. +// +// Bind will fail only if the socket is connected, bound or the passed address +// is invalid (the empty string). +func (e *connectionedEndpoint) Bind(addr tcpip.FullAddress, commit func() *tcpip.Error) *tcpip.Error { + e.Lock() + defer e.Unlock() + if e.isBound() || e.Listening() { + return tcpip.ErrAlreadyBound + } + if addr.Addr == "" { + // The empty string is not permitted. + return tcpip.ErrBadLocalAddress + } + if commit != nil { + if err := commit(); err != nil { + return err + } + } + + // Save the bound address. + e.path = string(addr.Addr) + return nil +} + +// SendMsg writes data and a control message to the endpoint's peer. +// This method does not block if the data cannot be written. +func (e *connectionedEndpoint) SendMsg(data [][]byte, c ControlMessages, to BoundEndpoint) (uintptr, *tcpip.Error) { + // Stream sockets do not support specifying the endpoint. Seqpacket + // sockets ignore the passed endpoint. + if e.stype == SockStream && to != nil { + return 0, tcpip.ErrNotSupported + } + return e.baseEndpoint.SendMsg(data, c, to) +} + +// Readiness returns the current readiness of the connectionedEndpoint. For +// example, if waiter.EventIn is set, the connectionedEndpoint is immediately +// readable. +func (e *connectionedEndpoint) Readiness(mask waiter.EventMask) waiter.EventMask { + e.Lock() + defer e.Unlock() + + ready := waiter.EventMask(0) + switch { + case e.Connected(): + if mask&waiter.EventIn != 0 && e.receiver.Readable() { + ready |= waiter.EventIn + } + if mask&waiter.EventOut != 0 && e.connected.Writable() { + ready |= waiter.EventOut + } + case e.Listening(): + if mask&waiter.EventIn != 0 && len(e.acceptedChan) > 0 { + ready |= waiter.EventIn + } + } + + return ready +} diff --git a/pkg/sentry/socket/unix/transport/connectioned_state.go b/pkg/sentry/socket/unix/transport/connectioned_state.go new file mode 100644 index 000000000..7e6c73dcc --- /dev/null +++ b/pkg/sentry/socket/unix/transport/connectioned_state.go @@ -0,0 +1,53 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package transport + +// saveAcceptedChan is invoked by stateify. +func (e *connectionedEndpoint) saveAcceptedChan() []*connectionedEndpoint { + // If acceptedChan is nil (i.e. we are not listening) then we will save nil. + // Otherwise we create a (possibly empty) slice of the values in acceptedChan and + // save that. + var acceptedSlice []*connectionedEndpoint + if e.acceptedChan != nil { + // Swap out acceptedChan with a new empty channel of the same capacity. + saveChan := e.acceptedChan + e.acceptedChan = make(chan *connectionedEndpoint, cap(saveChan)) + + // Create a new slice with the same len and capacity as the channel. + acceptedSlice = make([]*connectionedEndpoint, len(saveChan), cap(saveChan)) + // Drain acceptedChan into saveSlice, and fill up the new acceptChan at the + // same time. + for i := range acceptedSlice { + ep := <-saveChan + acceptedSlice[i] = ep + e.acceptedChan <- ep + } + close(saveChan) + } + return acceptedSlice +} + +// loadAcceptedChan is invoked by stateify. +func (e *connectionedEndpoint) loadAcceptedChan(acceptedSlice []*connectionedEndpoint) { + // If acceptedSlice is nil, then acceptedChan should also be nil. + if acceptedSlice != nil { + // Otherwise, create a new channel with the same capacity as acceptedSlice. + e.acceptedChan = make(chan *connectionedEndpoint, cap(acceptedSlice)) + // Seed the channel with values from acceptedSlice. + for _, ep := range acceptedSlice { + e.acceptedChan <- ep + } + } +} diff --git a/pkg/sentry/socket/unix/transport/connectionless.go b/pkg/sentry/socket/unix/transport/connectionless.go new file mode 100644 index 000000000..fb2728010 --- /dev/null +++ b/pkg/sentry/socket/unix/transport/connectionless.go @@ -0,0 +1,192 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package transport + +import ( + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport/queue" + "gvisor.googlesource.com/gvisor/pkg/tcpip" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// connectionlessEndpoint is a unix endpoint for unix sockets that support operating in +// a conectionless fashon. +// +// Specifically, this means datagram unix sockets not created with +// socketpair(2). +// +// +stateify savable +type connectionlessEndpoint struct { + baseEndpoint +} + +// NewConnectionless creates a new unbound dgram endpoint. +func NewConnectionless() Endpoint { + ep := &connectionlessEndpoint{baseEndpoint{Queue: &waiter.Queue{}}} + ep.receiver = &queueReceiver{readQueue: queue.New(&waiter.Queue{}, ep.Queue, initialLimit)} + return ep +} + +// isBound returns true iff the endpoint is bound. +func (e *connectionlessEndpoint) isBound() bool { + return e.path != "" +} + +// Close puts the endpoint in a closed state and frees all resources associated +// with it. +// +// The socket will be a fresh state after a call to close and may be reused. +// That is, close may be used to "unbind" or "disconnect" the socket in error +// paths. +func (e *connectionlessEndpoint) Close() { + e.Lock() + var r Receiver + if e.Connected() { + e.receiver.CloseRecv() + r = e.receiver + e.receiver = nil + + e.connected.Release() + e.connected = nil + } + if e.isBound() { + e.path = "" + } + e.Unlock() + if r != nil { + r.CloseNotify() + r.Release() + } +} + +// BidirectionalConnect implements BoundEndpoint.BidirectionalConnect. +func (e *connectionlessEndpoint) BidirectionalConnect(ce ConnectingEndpoint, returnConnect func(Receiver, ConnectedEndpoint)) *tcpip.Error { + return tcpip.ErrConnectionRefused +} + +// UnidirectionalConnect implements BoundEndpoint.UnidirectionalConnect. +func (e *connectionlessEndpoint) UnidirectionalConnect() (ConnectedEndpoint, *tcpip.Error) { + e.Lock() + r := e.receiver + e.Unlock() + if r == nil { + return nil, tcpip.ErrConnectionRefused + } + return &connectedEndpoint{ + endpoint: e, + writeQueue: r.(*queueReceiver).readQueue, + }, nil +} + +// SendMsg writes data and a control message to the specified endpoint. +// This method does not block if the data cannot be written. +func (e *connectionlessEndpoint) SendMsg(data [][]byte, c ControlMessages, to BoundEndpoint) (uintptr, *tcpip.Error) { + if to == nil { + return e.baseEndpoint.SendMsg(data, c, nil) + } + + connected, err := to.UnidirectionalConnect() + if err != nil { + return 0, tcpip.ErrInvalidEndpointState + } + defer connected.Release() + + e.Lock() + n, notify, err := connected.Send(data, c, tcpip.FullAddress{Addr: tcpip.Address(e.path)}) + e.Unlock() + + if notify { + connected.SendNotify() + } + + return n, err +} + +// Type implements Endpoint.Type. +func (e *connectionlessEndpoint) Type() SockType { + return SockDgram +} + +// Connect attempts to connect directly to server. +func (e *connectionlessEndpoint) Connect(server BoundEndpoint) *tcpip.Error { + connected, err := server.UnidirectionalConnect() + if err != nil { + return err + } + + e.Lock() + e.connected = connected + e.Unlock() + + return nil +} + +// Listen starts listening on the connection. +func (e *connectionlessEndpoint) Listen(int) *tcpip.Error { + return tcpip.ErrNotSupported +} + +// Accept accepts a new connection. +func (e *connectionlessEndpoint) Accept() (Endpoint, *tcpip.Error) { + return nil, tcpip.ErrNotSupported +} + +// Bind binds the connection. +// +// For Unix endpoints, this _only sets the address associated with the socket_. +// Work associated with sockets in the filesystem or finding those sockets must +// be done by a higher level. +// +// Bind will fail only if the socket is connected, bound or the passed address +// is invalid (the empty string). +func (e *connectionlessEndpoint) Bind(addr tcpip.FullAddress, commit func() *tcpip.Error) *tcpip.Error { + e.Lock() + defer e.Unlock() + if e.isBound() { + return tcpip.ErrAlreadyBound + } + if addr.Addr == "" { + // The empty string is not permitted. + return tcpip.ErrBadLocalAddress + } + if commit != nil { + if err := commit(); err != nil { + return err + } + } + + // Save the bound address. + e.path = string(addr.Addr) + return nil +} + +// Readiness returns the current readiness of the endpoint. For example, if +// waiter.EventIn is set, the endpoint is immediately readable. +func (e *connectionlessEndpoint) Readiness(mask waiter.EventMask) waiter.EventMask { + e.Lock() + defer e.Unlock() + + ready := waiter.EventMask(0) + if mask&waiter.EventIn != 0 && e.receiver.Readable() { + ready |= waiter.EventIn + } + + if e.Connected() { + if mask&waiter.EventOut != 0 && e.connected.Writable() { + ready |= waiter.EventOut + } + } + + return ready +} diff --git a/pkg/sentry/socket/unix/transport/queue/BUILD b/pkg/sentry/socket/unix/transport/queue/BUILD new file mode 100644 index 000000000..d914ecc23 --- /dev/null +++ b/pkg/sentry/socket/unix/transport/queue/BUILD @@ -0,0 +1,15 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("//tools/go_stateify:defs.bzl", "go_library") + +go_library( + name = "queue", + srcs = ["queue.go"], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport/queue", + visibility = ["//:sandbox"], + deps = [ + "//pkg/ilist", + "//pkg/tcpip", + "//pkg/waiter", + ], +) diff --git a/pkg/sentry/socket/unix/transport/queue/queue.go b/pkg/sentry/socket/unix/transport/queue/queue.go new file mode 100644 index 000000000..b3d2ea68b --- /dev/null +++ b/pkg/sentry/socket/unix/transport/queue/queue.go @@ -0,0 +1,227 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package queue provides the implementation of buffer queue +// and interface of queue entry with Length method. +package queue + +import ( + "sync" + + "gvisor.googlesource.com/gvisor/pkg/ilist" + "gvisor.googlesource.com/gvisor/pkg/tcpip" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// Entry implements Linker interface and has additional required methods. +type Entry interface { + ilist.Linker + + // Length returns the number of bytes stored in the entry. + Length() int64 + + // Release releases any resources held by the entry. + Release() + + // Peek returns a copy of the entry. It must be Released separately. + Peek() Entry + + // Truncate reduces the number of bytes stored in the entry to n bytes. + // + // Preconditions: n <= Length(). + Truncate(n int64) +} + +// Queue is a buffer queue. +// +// +stateify savable +type Queue struct { + ReaderQueue *waiter.Queue + WriterQueue *waiter.Queue + + mu sync.Mutex `state:"nosave"` + closed bool + used int64 + limit int64 + dataList ilist.List +} + +// New allocates and initializes a new queue. +func New(ReaderQueue *waiter.Queue, WriterQueue *waiter.Queue, limit int64) *Queue { + return &Queue{ReaderQueue: ReaderQueue, WriterQueue: WriterQueue, limit: limit} +} + +// Close closes q for reading and writing. It is immediately not writable and +// will become unreadable when no more data is pending. +// +// Both the read and write queues must be notified after closing: +// q.ReaderQueue.Notify(waiter.EventIn) +// q.WriterQueue.Notify(waiter.EventOut) +func (q *Queue) Close() { + q.mu.Lock() + q.closed = true + q.mu.Unlock() +} + +// Reset empties the queue and Releases all of the Entries. +// +// Both the read and write queues must be notified after resetting: +// q.ReaderQueue.Notify(waiter.EventIn) +// q.WriterQueue.Notify(waiter.EventOut) +func (q *Queue) Reset() { + q.mu.Lock() + for cur := q.dataList.Front(); cur != nil; cur = cur.Next() { + cur.(Entry).Release() + } + q.dataList.Reset() + q.used = 0 + q.mu.Unlock() +} + +// IsReadable determines if q is currently readable. +func (q *Queue) IsReadable() bool { + q.mu.Lock() + defer q.mu.Unlock() + + return q.closed || q.dataList.Front() != nil +} + +// bufWritable returns true if there is space for writing. +// +// N.B. Linux only considers a unix socket "writable" if >75% of the buffer is +// free. +// +// See net/unix/af_unix.c:unix_writeable. +func (q *Queue) bufWritable() bool { + return 4*q.used < q.limit +} + +// IsWritable determines if q is currently writable. +func (q *Queue) IsWritable() bool { + q.mu.Lock() + defer q.mu.Unlock() + + return q.closed || q.bufWritable() +} + +// Enqueue adds an entry to the data queue if room is available. +// +// If truncate is true, Enqueue may truncate the message beforing enqueuing it. +// Otherwise, the entire message must fit. If n < e.Length(), err indicates why. +// +// If notify is true, ReaderQueue.Notify must be called: +// q.ReaderQueue.Notify(waiter.EventIn) +func (q *Queue) Enqueue(e Entry, truncate bool) (l int64, notify bool, err *tcpip.Error) { + q.mu.Lock() + + if q.closed { + q.mu.Unlock() + return 0, false, tcpip.ErrClosedForSend + } + + free := q.limit - q.used + + l = e.Length() + + if l > free && truncate { + if free == 0 { + // Message can't fit right now. + q.mu.Unlock() + return 0, false, tcpip.ErrWouldBlock + } + + e.Truncate(free) + l = e.Length() + err = tcpip.ErrWouldBlock + } + + if l > q.limit { + // Message is too big to ever fit. + q.mu.Unlock() + return 0, false, tcpip.ErrMessageTooLong + } + + if l > free { + // Message can't fit right now. + q.mu.Unlock() + return 0, false, tcpip.ErrWouldBlock + } + + notify = q.dataList.Front() == nil + q.used += l + q.dataList.PushBack(e) + + q.mu.Unlock() + + return l, notify, err +} + +// Dequeue removes the first entry in the data queue, if one exists. +// +// If notify is true, WriterQueue.Notify must be called: +// q.WriterQueue.Notify(waiter.EventOut) +func (q *Queue) Dequeue() (e Entry, notify bool, err *tcpip.Error) { + q.mu.Lock() + + if q.dataList.Front() == nil { + err := tcpip.ErrWouldBlock + if q.closed { + err = tcpip.ErrClosedForReceive + } + q.mu.Unlock() + + return nil, false, err + } + + notify = !q.bufWritable() + + e = q.dataList.Front().(Entry) + q.dataList.Remove(e) + q.used -= e.Length() + + notify = notify && q.bufWritable() + + q.mu.Unlock() + + return e, notify, nil +} + +// Peek returns the first entry in the data queue, if one exists. +func (q *Queue) Peek() (Entry, *tcpip.Error) { + q.mu.Lock() + defer q.mu.Unlock() + + if q.dataList.Front() == nil { + err := tcpip.ErrWouldBlock + if q.closed { + err = tcpip.ErrClosedForReceive + } + return nil, err + } + + return q.dataList.Front().(Entry).Peek(), nil +} + +// QueuedSize returns the number of bytes currently in the queue, that is, the +// number of readable bytes. +func (q *Queue) QueuedSize() int64 { + q.mu.Lock() + defer q.mu.Unlock() + return q.used +} + +// MaxQueueSize returns the maximum number of bytes storable in the queue. +func (q *Queue) MaxQueueSize() int64 { + return q.limit +} diff --git a/pkg/sentry/socket/unix/transport/unix.go b/pkg/sentry/socket/unix/transport/unix.go new file mode 100644 index 000000000..577aa87d5 --- /dev/null +++ b/pkg/sentry/socket/unix/transport/unix.go @@ -0,0 +1,953 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package transport contains the implementation of Unix endpoints. +package transport + +import ( + "sync" + "sync/atomic" + + "gvisor.googlesource.com/gvisor/pkg/ilist" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport/queue" + "gvisor.googlesource.com/gvisor/pkg/tcpip" + "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// initialLimit is the starting limit for the socket buffers. +const initialLimit = 16 * 1024 + +// A SockType is a type (as opposed to family) of sockets. These are enumerated +// in the syscall package as syscall.SOCK_* constants. +type SockType int + +const ( + // SockStream corresponds to syscall.SOCK_STREAM. + SockStream SockType = 1 + // SockDgram corresponds to syscall.SOCK_DGRAM. + SockDgram SockType = 2 + // SockRaw corresponds to syscall.SOCK_RAW. + SockRaw SockType = 3 + // SockSeqpacket corresponds to syscall.SOCK_SEQPACKET. + SockSeqpacket SockType = 5 +) + +// A RightsControlMessage is a control message containing FDs. +type RightsControlMessage interface { + // Clone returns a copy of the RightsControlMessage. + Clone() RightsControlMessage + + // Release releases any resources owned by the RightsControlMessage. + Release() +} + +// A CredentialsControlMessage is a control message containing Unix credentials. +type CredentialsControlMessage interface { + // Equals returns true iff the two messages are equal. + Equals(CredentialsControlMessage) bool +} + +// A ControlMessages represents a collection of socket control messages. +// +// +stateify savable +type ControlMessages struct { + // Rights is a control message containing FDs. + Rights RightsControlMessage + + // Credentials is a control message containing Unix credentials. + Credentials CredentialsControlMessage +} + +// Empty returns true iff the ControlMessages does not contain either +// credentials or rights. +func (c *ControlMessages) Empty() bool { + return c.Rights == nil && c.Credentials == nil +} + +// Clone clones both the credentials and the rights. +func (c *ControlMessages) Clone() ControlMessages { + cm := ControlMessages{} + if c.Rights != nil { + cm.Rights = c.Rights.Clone() + } + cm.Credentials = c.Credentials + return cm +} + +// Release releases both the credentials and the rights. +func (c *ControlMessages) Release() { + if c.Rights != nil { + c.Rights.Release() + } + *c = ControlMessages{} +} + +// Endpoint is the interface implemented by Unix transport protocol +// implementations that expose functionality like sendmsg, recvmsg, connect, +// etc. to Unix socket implementations. +type Endpoint interface { + Credentialer + waiter.Waitable + + // Close puts the endpoint in a closed state and frees all resources + // associated with it. + Close() + + // RecvMsg reads data and a control message from the endpoint. This method + // does not block if there is no data pending. + // + // creds indicates if credential control messages are requested by the + // caller. This is useful for determining if control messages can be + // coalesced. creds is a hint and can be safely ignored by the + // implementation if no coalescing is possible. It is fine to return + // credential control messages when none were requested or to not return + // credential control messages when they were requested. + // + // numRights is the number of SCM_RIGHTS FDs requested by the caller. This + // is useful if one must allocate a buffer to receive a SCM_RIGHTS message + // or determine if control messages can be coalesced. numRights is a hint + // and can be safely ignored by the implementation if the number of + // available SCM_RIGHTS FDs is known and no coalescing is possible. It is + // fine for the returned number of SCM_RIGHTS FDs to be either higher or + // lower than the requested number. + // + // If peek is true, no data should be consumed from the Endpoint. Any and + // all data returned from a peek should be available in the next call to + // RecvMsg. + // + // recvLen is the number of bytes copied into data. + // + // msgLen is the length of the read message consumed for datagram Endpoints. + // msgLen is always the same as recvLen for stream Endpoints. + RecvMsg(data [][]byte, creds bool, numRights uintptr, peek bool, addr *tcpip.FullAddress) (recvLen, msgLen uintptr, cm ControlMessages, err *tcpip.Error) + + // SendMsg writes data and a control message to the endpoint's peer. + // This method does not block if the data cannot be written. + // + // SendMsg does not take ownership of any of its arguments on error. + SendMsg([][]byte, ControlMessages, BoundEndpoint) (uintptr, *tcpip.Error) + + // Connect connects this endpoint directly to another. + // + // This should be called on the client endpoint, and the (bound) + // endpoint passed in as a parameter. + // + // The error codes are the same as Connect. + Connect(server BoundEndpoint) *tcpip.Error + + // Shutdown closes the read and/or write end of the endpoint connection + // to its peer. + Shutdown(flags tcpip.ShutdownFlags) *tcpip.Error + + // Listen puts the endpoint in "listen" mode, which allows it to accept + // new connections. + Listen(backlog int) *tcpip.Error + + // Accept returns a new endpoint if a peer has established a connection + // to an endpoint previously set to listen mode. This method does not + // block if no new connections are available. + // + // The returned Queue is the wait queue for the newly created endpoint. + Accept() (Endpoint, *tcpip.Error) + + // Bind binds the endpoint to a specific local address and port. + // Specifying a NIC is optional. + // + // An optional commit function will be executed atomically with respect + // to binding the endpoint. If this returns an error, the bind will not + // occur and the error will be propagated back to the caller. + Bind(address tcpip.FullAddress, commit func() *tcpip.Error) *tcpip.Error + + // Type return the socket type, typically either SockStream, SockDgram + // or SockSeqpacket. + Type() SockType + + // GetLocalAddress returns the address to which the endpoint is bound. + GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) + + // GetRemoteAddress returns the address to which the endpoint is + // connected. + GetRemoteAddress() (tcpip.FullAddress, *tcpip.Error) + + // SetSockOpt sets a socket option. opt should be one of the tcpip.*Option + // types. + SetSockOpt(opt interface{}) *tcpip.Error + + // GetSockOpt gets a socket option. opt should be a pointer to one of the + // tcpip.*Option types. + GetSockOpt(opt interface{}) *tcpip.Error +} + +// A Credentialer is a socket or endpoint that supports the SO_PASSCRED socket +// option. +type Credentialer interface { + // Passcred returns whether or not the SO_PASSCRED socket option is + // enabled on this end. + Passcred() bool + + // ConnectedPasscred returns whether or not the SO_PASSCRED socket option + // is enabled on the connected end. + ConnectedPasscred() bool +} + +// A BoundEndpoint is a unix endpoint that can be connected to. +type BoundEndpoint interface { + // BidirectionalConnect establishes a bi-directional connection between two + // unix endpoints in an all-or-nothing manner. If an error occurs during + // connecting, the state of neither endpoint should be modified. + // + // In order for an endpoint to establish such a bidirectional connection + // with a BoundEndpoint, the endpoint calls the BidirectionalConnect method + // on the BoundEndpoint and sends a representation of itself (the + // ConnectingEndpoint) and a callback (returnConnect) to receive the + // connection information (Receiver and ConnectedEndpoint) upon a + // successful connect. The callback should only be called on a successful + // connect. + // + // For a connection attempt to be successful, the ConnectingEndpoint must + // be unconnected and not listening and the BoundEndpoint whose + // BidirectionalConnect method is being called must be listening. + // + // This method will return tcpip.ErrConnectionRefused on endpoints with a + // type that isn't SockStream or SockSeqpacket. + BidirectionalConnect(ep ConnectingEndpoint, returnConnect func(Receiver, ConnectedEndpoint)) *tcpip.Error + + // UnidirectionalConnect establishes a write-only connection to a unix + // endpoint. + // + // An endpoint which calls UnidirectionalConnect and supports it itself must + // not hold its own lock when calling UnidirectionalConnect. + // + // This method will return tcpip.ErrConnectionRefused on a non-SockDgram + // endpoint. + UnidirectionalConnect() (ConnectedEndpoint, *tcpip.Error) + + // Release releases any resources held by the BoundEndpoint. It must be + // called before dropping all references to a BoundEndpoint returned by a + // function. + Release() +} + +// message represents a message passed over a Unix domain socket. +// +// +stateify savable +type message struct { + ilist.Entry + + // Data is the Message payload. + Data buffer.View + + // Control is auxiliary control message data that goes along with the + // data. + Control ControlMessages + + // Address is the bound address of the endpoint that sent the message. + // + // If the endpoint that sent the message is not bound, the Address is + // the empty string. + Address tcpip.FullAddress +} + +// Length returns number of bytes stored in the message. +func (m *message) Length() int64 { + return int64(len(m.Data)) +} + +// Release releases any resources held by the message. +func (m *message) Release() { + m.Control.Release() +} + +// Peek returns a copy of the message. +func (m *message) Peek() queue.Entry { + return &message{Data: m.Data, Control: m.Control.Clone(), Address: m.Address} +} + +// Truncate reduces the length of the message payload to n bytes. +// +// Preconditions: n <= m.Length(). +func (m *message) Truncate(n int64) { + m.Data.CapLength(int(n)) +} + +// A Receiver can be used to receive Messages. +type Receiver interface { + // Recv receives a single message. This method does not block. + // + // See Endpoint.RecvMsg for documentation on shared arguments. + // + // notify indicates if RecvNotify should be called. + Recv(data [][]byte, creds bool, numRights uintptr, peek bool) (recvLen, msgLen uintptr, cm ControlMessages, source tcpip.FullAddress, notify bool, err *tcpip.Error) + + // RecvNotify notifies the Receiver of a successful Recv. This must not be + // called while holding any endpoint locks. + RecvNotify() + + // CloseRecv prevents the receiving of additional Messages. + // + // After CloseRecv is called, CloseNotify must also be called. + CloseRecv() + + // CloseNotify notifies the Receiver of recv being closed. This must not be + // called while holding any endpoint locks. + CloseNotify() + + // Readable returns if messages should be attempted to be received. This + // includes when read has been shutdown. + Readable() bool + + // RecvQueuedSize returns the total amount of data currently receivable. + // RecvQueuedSize should return -1 if the operation isn't supported. + RecvQueuedSize() int64 + + // RecvMaxQueueSize returns maximum value for RecvQueuedSize. + // RecvMaxQueueSize should return -1 if the operation isn't supported. + RecvMaxQueueSize() int64 + + // Release releases any resources owned by the Receiver. It should be + // called before droping all references to a Receiver. + Release() +} + +// queueReceiver implements Receiver for datagram sockets. +// +// +stateify savable +type queueReceiver struct { + readQueue *queue.Queue +} + +// Recv implements Receiver.Recv. +func (q *queueReceiver) Recv(data [][]byte, creds bool, numRights uintptr, peek bool) (uintptr, uintptr, ControlMessages, tcpip.FullAddress, bool, *tcpip.Error) { + var m queue.Entry + var notify bool + var err *tcpip.Error + if peek { + m, err = q.readQueue.Peek() + } else { + m, notify, err = q.readQueue.Dequeue() + } + if err != nil { + return 0, 0, ControlMessages{}, tcpip.FullAddress{}, false, err + } + msg := m.(*message) + src := []byte(msg.Data) + var copied uintptr + for i := 0; i < len(data) && len(src) > 0; i++ { + n := copy(data[i], src) + copied += uintptr(n) + src = src[n:] + } + return copied, uintptr(len(msg.Data)), msg.Control, msg.Address, notify, nil +} + +// RecvNotify implements Receiver.RecvNotify. +func (q *queueReceiver) RecvNotify() { + q.readQueue.WriterQueue.Notify(waiter.EventOut) +} + +// CloseNotify implements Receiver.CloseNotify. +func (q *queueReceiver) CloseNotify() { + q.readQueue.ReaderQueue.Notify(waiter.EventIn) + q.readQueue.WriterQueue.Notify(waiter.EventOut) +} + +// CloseRecv implements Receiver.CloseRecv. +func (q *queueReceiver) CloseRecv() { + q.readQueue.Close() +} + +// Readable implements Receiver.Readable. +func (q *queueReceiver) Readable() bool { + return q.readQueue.IsReadable() +} + +// RecvQueuedSize implements Receiver.RecvQueuedSize. +func (q *queueReceiver) RecvQueuedSize() int64 { + return q.readQueue.QueuedSize() +} + +// RecvMaxQueueSize implements Receiver.RecvMaxQueueSize. +func (q *queueReceiver) RecvMaxQueueSize() int64 { + return q.readQueue.MaxQueueSize() +} + +// Release implements Receiver.Release. +func (*queueReceiver) Release() {} + +// streamQueueReceiver implements Receiver for stream sockets. +// +// +stateify savable +type streamQueueReceiver struct { + queueReceiver + + mu sync.Mutex `state:"nosave"` + buffer []byte + control ControlMessages + addr tcpip.FullAddress +} + +func vecCopy(data [][]byte, buf []byte) (uintptr, [][]byte, []byte) { + var copied uintptr + for len(data) > 0 && len(buf) > 0 { + n := copy(data[0], buf) + copied += uintptr(n) + buf = buf[n:] + data[0] = data[0][n:] + if len(data[0]) == 0 { + data = data[1:] + } + } + return copied, data, buf +} + +// Readable implements Receiver.Readable. +func (q *streamQueueReceiver) Readable() bool { + q.mu.Lock() + bl := len(q.buffer) + r := q.readQueue.IsReadable() + q.mu.Unlock() + // We're readable if we have data in our buffer or if the queue receiver is + // readable. + return bl > 0 || r +} + +// RecvQueuedSize implements Receiver.RecvQueuedSize. +func (q *streamQueueReceiver) RecvQueuedSize() int64 { + q.mu.Lock() + bl := len(q.buffer) + qs := q.readQueue.QueuedSize() + q.mu.Unlock() + return int64(bl) + qs +} + +// RecvMaxQueueSize implements Receiver.RecvMaxQueueSize. +func (q *streamQueueReceiver) RecvMaxQueueSize() int64 { + // The RecvMaxQueueSize() is the readQueue's MaxQueueSize() plus the largest + // message we can buffer which is also the largest message we can receive. + return 2 * q.readQueue.MaxQueueSize() +} + +// Recv implements Receiver.Recv. +func (q *streamQueueReceiver) Recv(data [][]byte, wantCreds bool, numRights uintptr, peek bool) (uintptr, uintptr, ControlMessages, tcpip.FullAddress, bool, *tcpip.Error) { + q.mu.Lock() + defer q.mu.Unlock() + + var notify bool + + // If we have no data in the endpoint, we need to get some. + if len(q.buffer) == 0 { + // Load the next message into a buffer, even if we are peeking. Peeking + // won't consume the message, so it will be still available to be read + // the next time Recv() is called. + m, n, err := q.readQueue.Dequeue() + if err != nil { + return 0, 0, ControlMessages{}, tcpip.FullAddress{}, false, err + } + notify = n + msg := m.(*message) + q.buffer = []byte(msg.Data) + q.control = msg.Control + q.addr = msg.Address + } + + var copied uintptr + if peek { + // Don't consume control message if we are peeking. + c := q.control.Clone() + + // Don't consume data since we are peeking. + copied, data, _ = vecCopy(data, q.buffer) + + return copied, copied, c, q.addr, notify, nil + } + + // Consume data and control message since we are not peeking. + copied, data, q.buffer = vecCopy(data, q.buffer) + + // Save the original state of q.control. + c := q.control + + // Remove rights from q.control and leave behind just the creds. + q.control.Rights = nil + if !wantCreds { + c.Credentials = nil + } + + if c.Rights != nil && numRights == 0 { + c.Rights.Release() + c.Rights = nil + } + + haveRights := c.Rights != nil + + // If we have more capacity for data and haven't received any usable + // rights. + // + // Linux never coalesces rights control messages. + for !haveRights && len(data) > 0 { + // Get a message from the readQueue. + m, n, err := q.readQueue.Dequeue() + if err != nil { + // We already got some data, so ignore this error. This will + // manifest as a short read to the user, which is what Linux + // does. + break + } + notify = notify || n + msg := m.(*message) + q.buffer = []byte(msg.Data) + q.control = msg.Control + q.addr = msg.Address + + if wantCreds { + if (q.control.Credentials == nil) != (c.Credentials == nil) { + // One message has credentials, the other does not. + break + } + + if q.control.Credentials != nil && c.Credentials != nil && !q.control.Credentials.Equals(c.Credentials) { + // Both messages have credentials, but they don't match. + break + } + } + + if numRights != 0 && c.Rights != nil && q.control.Rights != nil { + // Both messages have rights. + break + } + + var cpd uintptr + cpd, data, q.buffer = vecCopy(data, q.buffer) + copied += cpd + + if cpd == 0 { + // data was actually full. + break + } + + if q.control.Rights != nil { + // Consume rights. + if numRights == 0 { + q.control.Rights.Release() + } else { + c.Rights = q.control.Rights + haveRights = true + } + q.control.Rights = nil + } + } + return copied, copied, c, q.addr, notify, nil +} + +// A ConnectedEndpoint is an Endpoint that can be used to send Messages. +type ConnectedEndpoint interface { + // Passcred implements Endpoint.Passcred. + Passcred() bool + + // GetLocalAddress implements Endpoint.GetLocalAddress. + GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) + + // Send sends a single message. This method does not block. + // + // notify indicates if SendNotify should be called. + // + // tcpip.ErrWouldBlock can be returned along with a partial write if + // the caller should block to send the rest of the data. + Send(data [][]byte, controlMessages ControlMessages, from tcpip.FullAddress) (n uintptr, notify bool, err *tcpip.Error) + + // SendNotify notifies the ConnectedEndpoint of a successful Send. This + // must not be called while holding any endpoint locks. + SendNotify() + + // CloseSend prevents the sending of additional Messages. + // + // After CloseSend is call, CloseNotify must also be called. + CloseSend() + + // CloseNotify notifies the ConnectedEndpoint of send being closed. This + // must not be called while holding any endpoint locks. + CloseNotify() + + // Writable returns if messages should be attempted to be sent. This + // includes when write has been shutdown. + Writable() bool + + // EventUpdate lets the ConnectedEndpoint know that event registrations + // have changed. + EventUpdate() + + // SendQueuedSize returns the total amount of data currently queued for + // sending. SendQueuedSize should return -1 if the operation isn't + // supported. + SendQueuedSize() int64 + + // SendMaxQueueSize returns maximum value for SendQueuedSize. + // SendMaxQueueSize should return -1 if the operation isn't supported. + SendMaxQueueSize() int64 + + // Release releases any resources owned by the ConnectedEndpoint. It should + // be called before droping all references to a ConnectedEndpoint. + Release() +} + +// +stateify savable +type connectedEndpoint struct { + // endpoint represents the subset of the Endpoint functionality needed by + // the connectedEndpoint. It is implemented by both connectionedEndpoint + // and connectionlessEndpoint and allows the use of types which don't + // fully implement Endpoint. + endpoint interface { + // Passcred implements Endpoint.Passcred. + Passcred() bool + + // GetLocalAddress implements Endpoint.GetLocalAddress. + GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) + + // Type implements Endpoint.Type. + Type() SockType + } + + writeQueue *queue.Queue +} + +// Passcred implements ConnectedEndpoint.Passcred. +func (e *connectedEndpoint) Passcred() bool { + return e.endpoint.Passcred() +} + +// GetLocalAddress implements ConnectedEndpoint.GetLocalAddress. +func (e *connectedEndpoint) GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) { + return e.endpoint.GetLocalAddress() +} + +// Send implements ConnectedEndpoint.Send. +func (e *connectedEndpoint) Send(data [][]byte, controlMessages ControlMessages, from tcpip.FullAddress) (uintptr, bool, *tcpip.Error) { + var l int64 + for _, d := range data { + l += int64(len(d)) + } + + truncate := false + if e.endpoint.Type() == SockStream { + // Since stream sockets don't preserve message boundaries, we + // can write only as much of the message as fits in the queue. + truncate = true + + // Discard empty stream packets. Since stream sockets don't + // preserve message boundaries, sending zero bytes is a no-op. + // In Linux, the receiver actually uses a zero-length receive + // as an indication that the stream was closed. + if l == 0 { + controlMessages.Release() + return 0, false, nil + } + } + + v := make([]byte, 0, l) + for _, d := range data { + v = append(v, d...) + } + + l, notify, err := e.writeQueue.Enqueue(&message{Data: buffer.View(v), Control: controlMessages, Address: from}, truncate) + return uintptr(l), notify, err +} + +// SendNotify implements ConnectedEndpoint.SendNotify. +func (e *connectedEndpoint) SendNotify() { + e.writeQueue.ReaderQueue.Notify(waiter.EventIn) +} + +// CloseNotify implements ConnectedEndpoint.CloseNotify. +func (e *connectedEndpoint) CloseNotify() { + e.writeQueue.ReaderQueue.Notify(waiter.EventIn) + e.writeQueue.WriterQueue.Notify(waiter.EventOut) +} + +// CloseSend implements ConnectedEndpoint.CloseSend. +func (e *connectedEndpoint) CloseSend() { + e.writeQueue.Close() +} + +// Writable implements ConnectedEndpoint.Writable. +func (e *connectedEndpoint) Writable() bool { + return e.writeQueue.IsWritable() +} + +// EventUpdate implements ConnectedEndpoint.EventUpdate. +func (*connectedEndpoint) EventUpdate() {} + +// SendQueuedSize implements ConnectedEndpoint.SendQueuedSize. +func (e *connectedEndpoint) SendQueuedSize() int64 { + return e.writeQueue.QueuedSize() +} + +// SendMaxQueueSize implements ConnectedEndpoint.SendMaxQueueSize. +func (e *connectedEndpoint) SendMaxQueueSize() int64 { + return e.writeQueue.MaxQueueSize() +} + +// Release implements ConnectedEndpoint.Release. +func (*connectedEndpoint) Release() {} + +// baseEndpoint is an embeddable unix endpoint base used in both the connected and connectionless +// unix domain socket Endpoint implementations. +// +// Not to be used on its own. +// +// +stateify savable +type baseEndpoint struct { + *waiter.Queue + + // passcred specifies whether SCM_CREDENTIALS socket control messages are + // enabled on this endpoint. Must be accessed atomically. + passcred int32 + + // Mutex protects the below fields. + sync.Mutex `state:"nosave"` + + // receiver allows Messages to be received. + receiver Receiver + + // connected allows messages to be sent and state information about the + // connected endpoint to be read. + connected ConnectedEndpoint + + // path is not empty if the endpoint has been bound, + // or may be used if the endpoint is connected. + path string +} + +// EventRegister implements waiter.Waitable.EventRegister. +func (e *baseEndpoint) EventRegister(we *waiter.Entry, mask waiter.EventMask) { + e.Queue.EventRegister(we, mask) + e.Lock() + if e.connected != nil { + e.connected.EventUpdate() + } + e.Unlock() +} + +// EventUnregister implements waiter.Waitable.EventUnregister. +func (e *baseEndpoint) EventUnregister(we *waiter.Entry) { + e.Queue.EventUnregister(we) + e.Lock() + if e.connected != nil { + e.connected.EventUpdate() + } + e.Unlock() +} + +// Passcred implements Credentialer.Passcred. +func (e *baseEndpoint) Passcred() bool { + return atomic.LoadInt32(&e.passcred) != 0 +} + +// ConnectedPasscred implements Credentialer.ConnectedPasscred. +func (e *baseEndpoint) ConnectedPasscred() bool { + e.Lock() + defer e.Unlock() + return e.connected != nil && e.connected.Passcred() +} + +func (e *baseEndpoint) setPasscred(pc bool) { + if pc { + atomic.StoreInt32(&e.passcred, 1) + } else { + atomic.StoreInt32(&e.passcred, 0) + } +} + +// Connected implements ConnectingEndpoint.Connected. +func (e *baseEndpoint) Connected() bool { + return e.receiver != nil && e.connected != nil +} + +// RecvMsg reads data and a control message from the endpoint. +func (e *baseEndpoint) RecvMsg(data [][]byte, creds bool, numRights uintptr, peek bool, addr *tcpip.FullAddress) (uintptr, uintptr, ControlMessages, *tcpip.Error) { + e.Lock() + + if e.receiver == nil { + e.Unlock() + return 0, 0, ControlMessages{}, tcpip.ErrNotConnected + } + + recvLen, msgLen, cms, a, notify, err := e.receiver.Recv(data, creds, numRights, peek) + e.Unlock() + if err != nil { + return 0, 0, ControlMessages{}, err + } + + if notify { + e.receiver.RecvNotify() + } + + if addr != nil { + *addr = a + } + return recvLen, msgLen, cms, nil +} + +// SendMsg writes data and a control message to the endpoint's peer. +// This method does not block if the data cannot be written. +func (e *baseEndpoint) SendMsg(data [][]byte, c ControlMessages, to BoundEndpoint) (uintptr, *tcpip.Error) { + e.Lock() + if !e.Connected() { + e.Unlock() + return 0, tcpip.ErrNotConnected + } + if to != nil { + e.Unlock() + return 0, tcpip.ErrAlreadyConnected + } + + n, notify, err := e.connected.Send(data, c, tcpip.FullAddress{Addr: tcpip.Address(e.path)}) + e.Unlock() + + if notify { + e.connected.SendNotify() + } + + return n, err +} + +// SetSockOpt sets a socket option. Currently not supported. +func (e *baseEndpoint) SetSockOpt(opt interface{}) *tcpip.Error { + switch v := opt.(type) { + case tcpip.PasscredOption: + e.setPasscred(v != 0) + return nil + } + return nil +} + +// GetSockOpt implements tcpip.Endpoint.GetSockOpt. +func (e *baseEndpoint) GetSockOpt(opt interface{}) *tcpip.Error { + switch o := opt.(type) { + case tcpip.ErrorOption: + return nil + case *tcpip.SendQueueSizeOption: + e.Lock() + if !e.Connected() { + e.Unlock() + return tcpip.ErrNotConnected + } + qs := tcpip.SendQueueSizeOption(e.connected.SendQueuedSize()) + e.Unlock() + if qs < 0 { + return tcpip.ErrQueueSizeNotSupported + } + *o = qs + return nil + case *tcpip.ReceiveQueueSizeOption: + e.Lock() + if !e.Connected() { + e.Unlock() + return tcpip.ErrNotConnected + } + qs := tcpip.ReceiveQueueSizeOption(e.receiver.RecvQueuedSize()) + e.Unlock() + if qs < 0 { + return tcpip.ErrQueueSizeNotSupported + } + *o = qs + return nil + case *tcpip.PasscredOption: + if e.Passcred() { + *o = tcpip.PasscredOption(1) + } else { + *o = tcpip.PasscredOption(0) + } + return nil + case *tcpip.SendBufferSizeOption: + e.Lock() + if !e.Connected() { + e.Unlock() + return tcpip.ErrNotConnected + } + qs := tcpip.SendBufferSizeOption(e.connected.SendMaxQueueSize()) + e.Unlock() + if qs < 0 { + return tcpip.ErrQueueSizeNotSupported + } + *o = qs + return nil + case *tcpip.ReceiveBufferSizeOption: + e.Lock() + if e.receiver == nil { + e.Unlock() + return tcpip.ErrNotConnected + } + qs := tcpip.ReceiveBufferSizeOption(e.receiver.RecvMaxQueueSize()) + e.Unlock() + if qs < 0 { + return tcpip.ErrQueueSizeNotSupported + } + *o = qs + return nil + } + return tcpip.ErrUnknownProtocolOption +} + +// Shutdown closes the read and/or write end of the endpoint connection to its +// peer. +func (e *baseEndpoint) Shutdown(flags tcpip.ShutdownFlags) *tcpip.Error { + e.Lock() + if !e.Connected() { + e.Unlock() + return tcpip.ErrNotConnected + } + + if flags&tcpip.ShutdownRead != 0 { + e.receiver.CloseRecv() + } + + if flags&tcpip.ShutdownWrite != 0 { + e.connected.CloseSend() + } + + e.Unlock() + + if flags&tcpip.ShutdownRead != 0 { + e.receiver.CloseNotify() + } + + if flags&tcpip.ShutdownWrite != 0 { + e.connected.CloseNotify() + } + + return nil +} + +// GetLocalAddress returns the bound path. +func (e *baseEndpoint) GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) { + e.Lock() + defer e.Unlock() + return tcpip.FullAddress{Addr: tcpip.Address(e.path)}, nil +} + +// GetRemoteAddress returns the local address of the connected endpoint (if +// available). +func (e *baseEndpoint) GetRemoteAddress() (tcpip.FullAddress, *tcpip.Error) { + e.Lock() + c := e.connected + e.Unlock() + if c != nil { + return c.GetLocalAddress() + } + return tcpip.FullAddress{}, tcpip.ErrNotConnected +} + +// Release implements BoundEndpoint.Release. +func (*baseEndpoint) Release() {} diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index e30378e60..668363864 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -32,16 +32,16 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/socket" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/control" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/epsocket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" "gvisor.googlesource.com/gvisor/pkg/waiter" ) -// SocketOperations is a Unix socket. It is similar to an epsocket, except it is backed -// by a unix.Endpoint instead of a tcpip.Endpoint. +// SocketOperations is a Unix socket. It is similar to an epsocket, except it +// is backed by a transport.Endpoint instead of a tcpip.Endpoint. // // +stateify savable type SocketOperations struct { @@ -52,18 +52,18 @@ type SocketOperations struct { fsutil.NoFsync `state:"nosave"` fsutil.NoopFlush `state:"nosave"` fsutil.NoMMap `state:"nosave"` - ep unix.Endpoint + ep transport.Endpoint } // New creates a new unix socket. -func New(ctx context.Context, endpoint unix.Endpoint) *fs.File { +func New(ctx context.Context, endpoint transport.Endpoint) *fs.File { dirent := socket.NewDirent(ctx, unixSocketDevice) defer dirent.DecRef() return NewWithDirent(ctx, dirent, endpoint, fs.FileFlags{Read: true, Write: true}) } // NewWithDirent creates a new unix socket using an existing dirent. -func NewWithDirent(ctx context.Context, d *fs.Dirent, ep unix.Endpoint, flags fs.FileFlags) *fs.File { +func NewWithDirent(ctx context.Context, d *fs.Dirent, ep transport.Endpoint, flags fs.FileFlags) *fs.File { return fs.NewFile(ctx, d, flags, &SocketOperations{ ep: ep, }) @@ -83,8 +83,8 @@ func (s *SocketOperations) Release() { s.DecRef() } -// Endpoint extracts the unix.Endpoint. -func (s *SocketOperations) Endpoint() unix.Endpoint { +// Endpoint extracts the transport.Endpoint. +func (s *SocketOperations) Endpoint() transport.Endpoint { return s.ep } @@ -110,7 +110,7 @@ func extractPath(sockaddr []byte) (string, *syserr.Error) { } // GetPeerName implements the linux syscall getpeername(2) for sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) GetPeerName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { addr, err := s.ep.GetRemoteAddress() if err != nil { @@ -122,7 +122,7 @@ func (s *SocketOperations) GetPeerName(t *kernel.Task) (interface{}, uint32, *sy } // GetSockName implements the linux syscall getsockname(2) for sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) GetSockName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { addr, err := s.ep.GetLocalAddress() if err != nil { @@ -139,20 +139,20 @@ func (s *SocketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.S } // GetSockOpt implements the linux syscall getsockopt(2) for sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) GetSockOpt(t *kernel.Task, level, name, outLen int) (interface{}, *syserr.Error) { return epsocket.GetSockOpt(t, s, s.ep, linux.AF_UNIX, s.ep.Type(), level, name, outLen) } // Listen implements the linux syscall listen(2) for sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) Listen(t *kernel.Task, backlog int) *syserr.Error { return syserr.TranslateNetstackError(s.ep.Listen(backlog)) } // blockingAccept implements a blocking version of accept(2), that is, if no // connections are ready to be accept, it will block until one becomes ready. -func (s *SocketOperations) blockingAccept(t *kernel.Task) (unix.Endpoint, *syserr.Error) { +func (s *SocketOperations) blockingAccept(t *kernel.Task) (transport.Endpoint, *syserr.Error) { // Register for notifications. e, ch := waiter.NewChannelEntry(nil) s.EventRegister(&e, waiter.EventIn) @@ -172,7 +172,7 @@ func (s *SocketOperations) blockingAccept(t *kernel.Task) (unix.Endpoint, *syser } // Accept implements the linux syscall accept(2) for sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (kdefs.FD, interface{}, uint32, *syserr.Error) { // Issue the accept request to get the new endpoint. ep, err := s.ep.Accept() @@ -226,7 +226,7 @@ func (s *SocketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error { return e } - bep, ok := s.ep.(unix.BoundEndpoint) + bep, ok := s.ep.(transport.BoundEndpoint) if !ok { // This socket can't be bound. return syserr.ErrInvalidArgument @@ -287,10 +287,10 @@ func (s *SocketOperations) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error { })) } -// extractEndpoint retrieves the unix.BoundEndpoint associated with a Unix -// socket path. The Release must be called on the unix.BoundEndpoint when the -// caller is done with it. -func extractEndpoint(t *kernel.Task, sockaddr []byte) (unix.BoundEndpoint, *syserr.Error) { +// extractEndpoint retrieves the transport.BoundEndpoint associated with a Unix +// socket path. The Release must be called on the transport.BoundEndpoint when +// the caller is done with it. +func extractEndpoint(t *kernel.Task, sockaddr []byte) (transport.BoundEndpoint, *syserr.Error) { path, err := extractPath(sockaddr) if err != nil { return nil, err @@ -362,7 +362,7 @@ func (s *SocketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IO } // SendMsg implements the linux syscall sendmsg(2) for unix sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { w := EndpointWriter{ Endpoint: s.ep, @@ -408,12 +408,12 @@ func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] return int(total), syserr.FromError(err) } -// Passcred implements unix.Credentialer.Passcred. +// Passcred implements transport.Credentialer.Passcred. func (s *SocketOperations) Passcred() bool { return s.ep.Passcred() } -// ConnectedPasscred implements unix.Credentialer.ConnectedPasscred. +// ConnectedPasscred implements transport.Credentialer.ConnectedPasscred. func (s *SocketOperations) ConnectedPasscred() bool { return s.ep.ConnectedPasscred() } @@ -434,13 +434,13 @@ func (s *SocketOperations) EventUnregister(e *waiter.Entry) { } // SetSockOpt implements the linux syscall setsockopt(2) for sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) SetSockOpt(t *kernel.Task, level int, name int, optVal []byte) *syserr.Error { return epsocket.SetSockOpt(t, s, s.ep, level, name, optVal) } // Shutdown implements the linux syscall shutdown(2) for sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { f, err := epsocket.ConvertShutdown(how) if err != nil { @@ -465,7 +465,7 @@ func (s *SocketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOS } // RecvMsg implements the linux syscall recvmsg(2) for sockets backed by -// a unix.Endpoint. +// a transport.Endpoint. func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) { trunc := flags&linux.MSG_TRUNC != 0 peek := flags&linux.MSG_PEEK != 0 @@ -539,19 +539,19 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags type provider struct{} // Socket returns a new unix domain socket. -func (*provider) Socket(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *syserr.Error) { +func (*provider) Socket(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *syserr.Error) { // Check arguments. if protocol != 0 { return nil, syserr.ErrInvalidArgument } // Create the endpoint and socket. - var ep unix.Endpoint + var ep transport.Endpoint switch stype { case linux.SOCK_DGRAM: - ep = unix.NewConnectionless() + ep = transport.NewConnectionless() case linux.SOCK_STREAM, linux.SOCK_SEQPACKET: - ep = unix.NewConnectioned(stype, t.Kernel()) + ep = transport.NewConnectioned(stype, t.Kernel()) default: return nil, syserr.ErrInvalidArgument } @@ -560,7 +560,7 @@ func (*provider) Socket(t *kernel.Task, stype unix.SockType, protocol int) (*fs. } // Pair creates a new pair of AF_UNIX connected sockets. -func (*provider) Pair(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { +func (*provider) Pair(t *kernel.Task, stype transport.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) { // Check arguments. if protocol != 0 { return nil, nil, syserr.ErrInvalidArgument @@ -573,7 +573,7 @@ func (*provider) Pair(t *kernel.Task, stype unix.SockType, protocol int) (*fs.Fi } // Create the endpoints and sockets. - ep1, ep2 := unix.NewPair(stype, t.Kernel()) + ep1, ep2 := transport.NewPair(stype, t.Kernel()) s1 := New(t, ep1) s2 := New(t, ep2) diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD index bbdfad9da..7621bfdbd 100644 --- a/pkg/sentry/syscalls/linux/BUILD +++ b/pkg/sentry/syscalls/linux/BUILD @@ -79,11 +79,11 @@ go_library( "//pkg/sentry/safemem", "//pkg/sentry/socket", "//pkg/sentry/socket/control", + "//pkg/sentry/socket/unix/transport", "//pkg/sentry/syscalls", "//pkg/sentry/usage", "//pkg/sentry/usermem", "//pkg/syserror", - "//pkg/tcpip/transport/unix", "//pkg/waiter", ], ) diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index 867fec468..5fa5ddce6 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -27,9 +27,9 @@ import ( ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/control" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" ) // minListenBacklog is the minimum reasonable backlog for listening sockets. @@ -180,7 +180,7 @@ func Socket(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscal } // Create the new socket. - s, e := socket.New(t, domain, unix.SockType(stype&0xf), protocol) + s, e := socket.New(t, domain, transport.SockType(stype&0xf), protocol) if e != nil { return 0, nil, e.ToError() } @@ -219,7 +219,7 @@ func SocketPair(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sy } // Create the socket pair. - s1, s2, e := socket.Pair(t, domain, unix.SockType(stype&0xf), protocol) + s1, s2, e := socket.Pair(t, domain, transport.SockType(stype&0xf), protocol) if e != nil { return 0, nil, e.ToError() } @@ -750,7 +750,7 @@ func recvSingleMsg(t *kernel.Task, s socket.Socket, msgPtr usermem.Addr, flags i controlData := make([]byte, 0, msg.ControlLen) - if cr, ok := s.(unix.Credentialer); ok && cr.Passcred() { + if cr, ok := s.(transport.Credentialer); ok && cr.Passcred() { creds, _ := cms.Unix.Credentials.(control.SCMCredentials) controlData = control.PackCredentials(t, creds, controlData) } diff --git a/pkg/sentry/uniqueid/BUILD b/pkg/sentry/uniqueid/BUILD index ff50b9925..68b82af47 100644 --- a/pkg/sentry/uniqueid/BUILD +++ b/pkg/sentry/uniqueid/BUILD @@ -9,6 +9,6 @@ go_library( visibility = ["//pkg/sentry:internal"], deps = [ "//pkg/sentry/context", - "//pkg/tcpip/transport/unix", + "//pkg/sentry/socket/unix/transport", ], ) diff --git a/pkg/sentry/uniqueid/context.go b/pkg/sentry/uniqueid/context.go index 541e0611d..e48fabc2d 100644 --- a/pkg/sentry/uniqueid/context.go +++ b/pkg/sentry/uniqueid/context.go @@ -18,7 +18,7 @@ package uniqueid import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" ) // contextID is the kernel package's type for context.Context.Value keys. @@ -44,8 +44,8 @@ func GlobalFromContext(ctx context.Context) uint64 { } // GlobalProviderFromContext returns a system-wide unique identifier from ctx. -func GlobalProviderFromContext(ctx context.Context) unix.UniqueIDProvider { - return ctx.Value(CtxGlobalUniqueIDProvider).(unix.UniqueIDProvider) +func GlobalProviderFromContext(ctx context.Context) transport.UniqueIDProvider { + return ctx.Value(CtxGlobalUniqueIDProvider).(transport.UniqueIDProvider) } // InotifyCookie generates a unique inotify event cookie from ctx. diff --git a/pkg/tcpip/transport/queue/BUILD b/pkg/tcpip/transport/queue/BUILD deleted file mode 100644 index 6dcec312e..000000000 --- a/pkg/tcpip/transport/queue/BUILD +++ /dev/null @@ -1,15 +0,0 @@ -package(licenses = ["notice"]) # Apache 2.0 - -load("//tools/go_stateify:defs.bzl", "go_library") - -go_library( - name = "queue", - srcs = ["queue.go"], - importpath = "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/queue", - visibility = ["//:sandbox"], - deps = [ - "//pkg/ilist", - "//pkg/tcpip", - "//pkg/waiter", - ], -) diff --git a/pkg/tcpip/transport/queue/queue.go b/pkg/tcpip/transport/queue/queue.go deleted file mode 100644 index b3d2ea68b..000000000 --- a/pkg/tcpip/transport/queue/queue.go +++ /dev/null @@ -1,227 +0,0 @@ -// Copyright 2018 Google Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package queue provides the implementation of buffer queue -// and interface of queue entry with Length method. -package queue - -import ( - "sync" - - "gvisor.googlesource.com/gvisor/pkg/ilist" - "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/waiter" -) - -// Entry implements Linker interface and has additional required methods. -type Entry interface { - ilist.Linker - - // Length returns the number of bytes stored in the entry. - Length() int64 - - // Release releases any resources held by the entry. - Release() - - // Peek returns a copy of the entry. It must be Released separately. - Peek() Entry - - // Truncate reduces the number of bytes stored in the entry to n bytes. - // - // Preconditions: n <= Length(). - Truncate(n int64) -} - -// Queue is a buffer queue. -// -// +stateify savable -type Queue struct { - ReaderQueue *waiter.Queue - WriterQueue *waiter.Queue - - mu sync.Mutex `state:"nosave"` - closed bool - used int64 - limit int64 - dataList ilist.List -} - -// New allocates and initializes a new queue. -func New(ReaderQueue *waiter.Queue, WriterQueue *waiter.Queue, limit int64) *Queue { - return &Queue{ReaderQueue: ReaderQueue, WriterQueue: WriterQueue, limit: limit} -} - -// Close closes q for reading and writing. It is immediately not writable and -// will become unreadable when no more data is pending. -// -// Both the read and write queues must be notified after closing: -// q.ReaderQueue.Notify(waiter.EventIn) -// q.WriterQueue.Notify(waiter.EventOut) -func (q *Queue) Close() { - q.mu.Lock() - q.closed = true - q.mu.Unlock() -} - -// Reset empties the queue and Releases all of the Entries. -// -// Both the read and write queues must be notified after resetting: -// q.ReaderQueue.Notify(waiter.EventIn) -// q.WriterQueue.Notify(waiter.EventOut) -func (q *Queue) Reset() { - q.mu.Lock() - for cur := q.dataList.Front(); cur != nil; cur = cur.Next() { - cur.(Entry).Release() - } - q.dataList.Reset() - q.used = 0 - q.mu.Unlock() -} - -// IsReadable determines if q is currently readable. -func (q *Queue) IsReadable() bool { - q.mu.Lock() - defer q.mu.Unlock() - - return q.closed || q.dataList.Front() != nil -} - -// bufWritable returns true if there is space for writing. -// -// N.B. Linux only considers a unix socket "writable" if >75% of the buffer is -// free. -// -// See net/unix/af_unix.c:unix_writeable. -func (q *Queue) bufWritable() bool { - return 4*q.used < q.limit -} - -// IsWritable determines if q is currently writable. -func (q *Queue) IsWritable() bool { - q.mu.Lock() - defer q.mu.Unlock() - - return q.closed || q.bufWritable() -} - -// Enqueue adds an entry to the data queue if room is available. -// -// If truncate is true, Enqueue may truncate the message beforing enqueuing it. -// Otherwise, the entire message must fit. If n < e.Length(), err indicates why. -// -// If notify is true, ReaderQueue.Notify must be called: -// q.ReaderQueue.Notify(waiter.EventIn) -func (q *Queue) Enqueue(e Entry, truncate bool) (l int64, notify bool, err *tcpip.Error) { - q.mu.Lock() - - if q.closed { - q.mu.Unlock() - return 0, false, tcpip.ErrClosedForSend - } - - free := q.limit - q.used - - l = e.Length() - - if l > free && truncate { - if free == 0 { - // Message can't fit right now. - q.mu.Unlock() - return 0, false, tcpip.ErrWouldBlock - } - - e.Truncate(free) - l = e.Length() - err = tcpip.ErrWouldBlock - } - - if l > q.limit { - // Message is too big to ever fit. - q.mu.Unlock() - return 0, false, tcpip.ErrMessageTooLong - } - - if l > free { - // Message can't fit right now. - q.mu.Unlock() - return 0, false, tcpip.ErrWouldBlock - } - - notify = q.dataList.Front() == nil - q.used += l - q.dataList.PushBack(e) - - q.mu.Unlock() - - return l, notify, err -} - -// Dequeue removes the first entry in the data queue, if one exists. -// -// If notify is true, WriterQueue.Notify must be called: -// q.WriterQueue.Notify(waiter.EventOut) -func (q *Queue) Dequeue() (e Entry, notify bool, err *tcpip.Error) { - q.mu.Lock() - - if q.dataList.Front() == nil { - err := tcpip.ErrWouldBlock - if q.closed { - err = tcpip.ErrClosedForReceive - } - q.mu.Unlock() - - return nil, false, err - } - - notify = !q.bufWritable() - - e = q.dataList.Front().(Entry) - q.dataList.Remove(e) - q.used -= e.Length() - - notify = notify && q.bufWritable() - - q.mu.Unlock() - - return e, notify, nil -} - -// Peek returns the first entry in the data queue, if one exists. -func (q *Queue) Peek() (Entry, *tcpip.Error) { - q.mu.Lock() - defer q.mu.Unlock() - - if q.dataList.Front() == nil { - err := tcpip.ErrWouldBlock - if q.closed { - err = tcpip.ErrClosedForReceive - } - return nil, err - } - - return q.dataList.Front().(Entry).Peek(), nil -} - -// QueuedSize returns the number of bytes currently in the queue, that is, the -// number of readable bytes. -func (q *Queue) QueuedSize() int64 { - q.mu.Lock() - defer q.mu.Unlock() - return q.used -} - -// MaxQueueSize returns the maximum number of bytes storable in the queue. -func (q *Queue) MaxQueueSize() int64 { - return q.limit -} diff --git a/pkg/tcpip/transport/unix/BUILD b/pkg/tcpip/transport/unix/BUILD deleted file mode 100644 index dae0bd079..000000000 --- a/pkg/tcpip/transport/unix/BUILD +++ /dev/null @@ -1,22 +0,0 @@ -package(licenses = ["notice"]) # Apache 2.0 - -load("//tools/go_stateify:defs.bzl", "go_library") - -go_library( - name = "unix", - srcs = [ - "connectioned.go", - "connectioned_state.go", - "connectionless.go", - "unix.go", - ], - importpath = "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix", - visibility = ["//:sandbox"], - deps = [ - "//pkg/ilist", - "//pkg/tcpip", - "//pkg/tcpip/buffer", - "//pkg/tcpip/transport/queue", - "//pkg/waiter", - ], -) diff --git a/pkg/tcpip/transport/unix/connectioned.go b/pkg/tcpip/transport/unix/connectioned.go deleted file mode 100644 index e319b3bb8..000000000 --- a/pkg/tcpip/transport/unix/connectioned.go +++ /dev/null @@ -1,454 +0,0 @@ -// Copyright 2018 Google Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package unix - -import ( - "sync" - - "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/queue" - "gvisor.googlesource.com/gvisor/pkg/waiter" -) - -// UniqueIDProvider generates a sequence of unique identifiers useful for, -// among other things, lock ordering. -type UniqueIDProvider interface { - // UniqueID returns a new unique identifier. - UniqueID() uint64 -} - -// A ConnectingEndpoint is a connectioned unix endpoint that is attempting to -// establish a bidirectional connection with a BoundEndpoint. -type ConnectingEndpoint interface { - // ID returns the endpoint's globally unique identifier. This identifier - // must be used to determine locking order if more than one endpoint is - // to be locked in the same codepath. The endpoint with the smaller - // identifier must be locked before endpoints with larger identifiers. - ID() uint64 - - // Passcred implements socket.Credentialer.Passcred. - Passcred() bool - - // Type returns the socket type, typically either SockStream or - // SockSeqpacket. The connection attempt must be aborted if this - // value doesn't match the ConnectableEndpoint's type. - Type() SockType - - // GetLocalAddress returns the bound path. - GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) - - // Locker protects the following methods. While locked, only the holder of - // the lock can change the return value of the protected methods. - sync.Locker - - // Connected returns true iff the ConnectingEndpoint is in the connected - // state. ConnectingEndpoints can only be connected to a single endpoint, - // so the connection attempt must be aborted if this returns true. - Connected() bool - - // Listening returns true iff the ConnectingEndpoint is in the listening - // state. ConnectingEndpoints cannot make connections while listening, so - // the connection attempt must be aborted if this returns true. - Listening() bool - - // WaiterQueue returns a pointer to the endpoint's waiter queue. - WaiterQueue() *waiter.Queue -} - -// connectionedEndpoint is a Unix-domain connected or connectable endpoint and implements -// ConnectingEndpoint, ConnectableEndpoint and tcpip.Endpoint. -// -// connectionedEndpoints must be in connected state in order to transfer data. -// -// This implementation includes STREAM and SEQPACKET Unix sockets created with -// socket(2), accept(2) or socketpair(2) and dgram unix sockets created with -// socketpair(2). See unix_connectionless.go for the implementation of DGRAM -// Unix sockets created with socket(2). -// -// The state is much simpler than a TCP endpoint, so it is not encoded -// explicitly. Instead we enforce the following invariants: -// -// receiver != nil, connected != nil => connected. -// path != "" && acceptedChan == nil => bound, not listening. -// path != "" && acceptedChan != nil => bound and listening. -// -// Only one of these will be true at any moment. -// -// +stateify savable -type connectionedEndpoint struct { - baseEndpoint - - // id is the unique endpoint identifier. This is used exclusively for - // lock ordering within connect. - id uint64 - - // idGenerator is used to generate new unique endpoint identifiers. - idGenerator UniqueIDProvider - - // stype is used by connecting sockets to ensure that they are the - // same type. The value is typically either tcpip.SockSeqpacket or - // tcpip.SockStream. - stype SockType - - // acceptedChan is per the TCP endpoint implementation. Note that the - // sockets in this channel are _already in the connected state_, and - // have another associated connectionedEndpoint. - // - // If nil, then no listen call has been made. - acceptedChan chan *connectionedEndpoint `state:".([]*connectionedEndpoint)"` -} - -// NewConnectioned creates a new unbound connectionedEndpoint. -func NewConnectioned(stype SockType, uid UniqueIDProvider) Endpoint { - return &connectionedEndpoint{ - baseEndpoint: baseEndpoint{Queue: &waiter.Queue{}}, - id: uid.UniqueID(), - idGenerator: uid, - stype: stype, - } -} - -// NewPair allocates a new pair of connected unix-domain connectionedEndpoints. -func NewPair(stype SockType, uid UniqueIDProvider) (Endpoint, Endpoint) { - a := &connectionedEndpoint{ - baseEndpoint: baseEndpoint{Queue: &waiter.Queue{}}, - id: uid.UniqueID(), - idGenerator: uid, - stype: stype, - } - b := &connectionedEndpoint{ - baseEndpoint: baseEndpoint{Queue: &waiter.Queue{}}, - id: uid.UniqueID(), - idGenerator: uid, - stype: stype, - } - - q1 := queue.New(a.Queue, b.Queue, initialLimit) - q2 := queue.New(b.Queue, a.Queue, initialLimit) - - if stype == SockStream { - a.receiver = &streamQueueReceiver{queueReceiver: queueReceiver{q1}} - b.receiver = &streamQueueReceiver{queueReceiver: queueReceiver{q2}} - } else { - a.receiver = &queueReceiver{q1} - b.receiver = &queueReceiver{q2} - } - - a.connected = &connectedEndpoint{ - endpoint: b, - writeQueue: q2, - } - b.connected = &connectedEndpoint{ - endpoint: a, - writeQueue: q1, - } - - return a, b -} - -// NewExternal creates a new externally backed Endpoint. It behaves like a -// socketpair. -func NewExternal(stype SockType, uid UniqueIDProvider, queue *waiter.Queue, receiver Receiver, connected ConnectedEndpoint) Endpoint { - return &connectionedEndpoint{ - baseEndpoint: baseEndpoint{Queue: queue, receiver: receiver, connected: connected}, - id: uid.UniqueID(), - idGenerator: uid, - stype: stype, - } -} - -// ID implements ConnectingEndpoint.ID. -func (e *connectionedEndpoint) ID() uint64 { - return e.id -} - -// Type implements ConnectingEndpoint.Type and Endpoint.Type. -func (e *connectionedEndpoint) Type() SockType { - return e.stype -} - -// WaiterQueue implements ConnectingEndpoint.WaiterQueue. -func (e *connectionedEndpoint) WaiterQueue() *waiter.Queue { - return e.Queue -} - -// isBound returns true iff the connectionedEndpoint is bound (but not -// listening). -func (e *connectionedEndpoint) isBound() bool { - return e.path != "" && e.acceptedChan == nil -} - -// Listening implements ConnectingEndpoint.Listening. -func (e *connectionedEndpoint) Listening() bool { - return e.acceptedChan != nil -} - -// Close puts the connectionedEndpoint in a closed state and frees all -// resources associated with it. -// -// The socket will be a fresh state after a call to close and may be reused. -// That is, close may be used to "unbind" or "disconnect" the socket in error -// paths. -func (e *connectionedEndpoint) Close() { - e.Lock() - var c ConnectedEndpoint - var r Receiver - switch { - case e.Connected(): - e.connected.CloseSend() - e.receiver.CloseRecv() - c = e.connected - r = e.receiver - e.connected = nil - e.receiver = nil - case e.isBound(): - e.path = "" - case e.Listening(): - close(e.acceptedChan) - for n := range e.acceptedChan { - n.Close() - } - e.acceptedChan = nil - e.path = "" - } - e.Unlock() - if c != nil { - c.CloseNotify() - c.Release() - } - if r != nil { - r.CloseNotify() - r.Release() - } -} - -// BidirectionalConnect implements BoundEndpoint.BidirectionalConnect. -func (e *connectionedEndpoint) BidirectionalConnect(ce ConnectingEndpoint, returnConnect func(Receiver, ConnectedEndpoint)) *tcpip.Error { - if ce.Type() != e.stype { - return tcpip.ErrConnectionRefused - } - - // Check if ce is e to avoid a deadlock. - if ce, ok := ce.(*connectionedEndpoint); ok && ce == e { - return tcpip.ErrInvalidEndpointState - } - - // Do a dance to safely acquire locks on both endpoints. - if e.id < ce.ID() { - e.Lock() - ce.Lock() - } else { - ce.Lock() - e.Lock() - } - - // Check connecting state. - if ce.Connected() { - e.Unlock() - ce.Unlock() - return tcpip.ErrAlreadyConnected - } - if ce.Listening() { - e.Unlock() - ce.Unlock() - return tcpip.ErrInvalidEndpointState - } - - // Check bound state. - if !e.Listening() { - e.Unlock() - ce.Unlock() - return tcpip.ErrConnectionRefused - } - - // Create a newly bound connectionedEndpoint. - ne := &connectionedEndpoint{ - baseEndpoint: baseEndpoint{ - path: e.path, - Queue: &waiter.Queue{}, - }, - id: e.idGenerator.UniqueID(), - idGenerator: e.idGenerator, - stype: e.stype, - } - readQueue := queue.New(ce.WaiterQueue(), ne.Queue, initialLimit) - writeQueue := queue.New(ne.Queue, ce.WaiterQueue(), initialLimit) - ne.connected = &connectedEndpoint{ - endpoint: ce, - writeQueue: readQueue, - } - if e.stype == SockStream { - ne.receiver = &streamQueueReceiver{queueReceiver: queueReceiver{readQueue: writeQueue}} - } else { - ne.receiver = &queueReceiver{readQueue: writeQueue} - } - - select { - case e.acceptedChan <- ne: - // Commit state. - connected := &connectedEndpoint{ - endpoint: ne, - writeQueue: writeQueue, - } - if e.stype == SockStream { - returnConnect(&streamQueueReceiver{queueReceiver: queueReceiver{readQueue: readQueue}}, connected) - } else { - returnConnect(&queueReceiver{readQueue: readQueue}, connected) - } - - // Notify can deadlock if we are holding these locks. - e.Unlock() - ce.Unlock() - - // Notify on both ends. - e.Notify(waiter.EventIn) - ce.WaiterQueue().Notify(waiter.EventOut) - - return nil - default: - // Busy; return ECONNREFUSED per spec. - ne.Close() - e.Unlock() - ce.Unlock() - return tcpip.ErrConnectionRefused - } -} - -// UnidirectionalConnect implements BoundEndpoint.UnidirectionalConnect. -func (e *connectionedEndpoint) UnidirectionalConnect() (ConnectedEndpoint, *tcpip.Error) { - return nil, tcpip.ErrConnectionRefused -} - -// Connect attempts to directly connect to another Endpoint. -// Implements Endpoint.Connect. -func (e *connectionedEndpoint) Connect(server BoundEndpoint) *tcpip.Error { - returnConnect := func(r Receiver, ce ConnectedEndpoint) { - e.receiver = r - e.connected = ce - } - - return server.BidirectionalConnect(e, returnConnect) -} - -// Listen starts listening on the connection. -func (e *connectionedEndpoint) Listen(backlog int) *tcpip.Error { - e.Lock() - defer e.Unlock() - if e.Listening() { - // Adjust the size of the channel iff we can fix existing - // pending connections into the new one. - if len(e.acceptedChan) > backlog { - return tcpip.ErrInvalidEndpointState - } - origChan := e.acceptedChan - e.acceptedChan = make(chan *connectionedEndpoint, backlog) - close(origChan) - for ep := range origChan { - e.acceptedChan <- ep - } - return nil - } - if !e.isBound() { - return tcpip.ErrInvalidEndpointState - } - - // Normal case. - e.acceptedChan = make(chan *connectionedEndpoint, backlog) - return nil -} - -// Accept accepts a new connection. -func (e *connectionedEndpoint) Accept() (Endpoint, *tcpip.Error) { - e.Lock() - defer e.Unlock() - - if !e.Listening() { - return nil, tcpip.ErrInvalidEndpointState - } - - select { - case ne := <-e.acceptedChan: - return ne, nil - - default: - // Nothing left. - return nil, tcpip.ErrWouldBlock - } -} - -// Bind binds the connection. -// -// For Unix connectionedEndpoints, this _only sets the address associated with -// the socket_. Work associated with sockets in the filesystem or finding those -// sockets must be done by a higher level. -// -// Bind will fail only if the socket is connected, bound or the passed address -// is invalid (the empty string). -func (e *connectionedEndpoint) Bind(addr tcpip.FullAddress, commit func() *tcpip.Error) *tcpip.Error { - e.Lock() - defer e.Unlock() - if e.isBound() || e.Listening() { - return tcpip.ErrAlreadyBound - } - if addr.Addr == "" { - // The empty string is not permitted. - return tcpip.ErrBadLocalAddress - } - if commit != nil { - if err := commit(); err != nil { - return err - } - } - - // Save the bound address. - e.path = string(addr.Addr) - return nil -} - -// SendMsg writes data and a control message to the endpoint's peer. -// This method does not block if the data cannot be written. -func (e *connectionedEndpoint) SendMsg(data [][]byte, c ControlMessages, to BoundEndpoint) (uintptr, *tcpip.Error) { - // Stream sockets do not support specifying the endpoint. Seqpacket - // sockets ignore the passed endpoint. - if e.stype == SockStream && to != nil { - return 0, tcpip.ErrNotSupported - } - return e.baseEndpoint.SendMsg(data, c, to) -} - -// Readiness returns the current readiness of the connectionedEndpoint. For -// example, if waiter.EventIn is set, the connectionedEndpoint is immediately -// readable. -func (e *connectionedEndpoint) Readiness(mask waiter.EventMask) waiter.EventMask { - e.Lock() - defer e.Unlock() - - ready := waiter.EventMask(0) - switch { - case e.Connected(): - if mask&waiter.EventIn != 0 && e.receiver.Readable() { - ready |= waiter.EventIn - } - if mask&waiter.EventOut != 0 && e.connected.Writable() { - ready |= waiter.EventOut - } - case e.Listening(): - if mask&waiter.EventIn != 0 && len(e.acceptedChan) > 0 { - ready |= waiter.EventIn - } - } - - return ready -} diff --git a/pkg/tcpip/transport/unix/connectioned_state.go b/pkg/tcpip/transport/unix/connectioned_state.go deleted file mode 100644 index 39e0ca2d6..000000000 --- a/pkg/tcpip/transport/unix/connectioned_state.go +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright 2018 Google Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package unix - -// saveAcceptedChan is invoked by stateify. -func (e *connectionedEndpoint) saveAcceptedChan() []*connectionedEndpoint { - // If acceptedChan is nil (i.e. we are not listening) then we will save nil. - // Otherwise we create a (possibly empty) slice of the values in acceptedChan and - // save that. - var acceptedSlice []*connectionedEndpoint - if e.acceptedChan != nil { - // Swap out acceptedChan with a new empty channel of the same capacity. - saveChan := e.acceptedChan - e.acceptedChan = make(chan *connectionedEndpoint, cap(saveChan)) - - // Create a new slice with the same len and capacity as the channel. - acceptedSlice = make([]*connectionedEndpoint, len(saveChan), cap(saveChan)) - // Drain acceptedChan into saveSlice, and fill up the new acceptChan at the - // same time. - for i := range acceptedSlice { - ep := <-saveChan - acceptedSlice[i] = ep - e.acceptedChan <- ep - } - close(saveChan) - } - return acceptedSlice -} - -// loadAcceptedChan is invoked by stateify. -func (e *connectionedEndpoint) loadAcceptedChan(acceptedSlice []*connectionedEndpoint) { - // If acceptedSlice is nil, then acceptedChan should also be nil. - if acceptedSlice != nil { - // Otherwise, create a new channel with the same capacity as acceptedSlice. - e.acceptedChan = make(chan *connectionedEndpoint, cap(acceptedSlice)) - // Seed the channel with values from acceptedSlice. - for _, ep := range acceptedSlice { - e.acceptedChan <- ep - } - } -} diff --git a/pkg/tcpip/transport/unix/connectionless.go b/pkg/tcpip/transport/unix/connectionless.go deleted file mode 100644 index ae93c61d7..000000000 --- a/pkg/tcpip/transport/unix/connectionless.go +++ /dev/null @@ -1,192 +0,0 @@ -// Copyright 2018 Google Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package unix - -import ( - "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/queue" - "gvisor.googlesource.com/gvisor/pkg/waiter" -) - -// connectionlessEndpoint is a unix endpoint for unix sockets that support operating in -// a conectionless fashon. -// -// Specifically, this means datagram unix sockets not created with -// socketpair(2). -// -// +stateify savable -type connectionlessEndpoint struct { - baseEndpoint -} - -// NewConnectionless creates a new unbound dgram endpoint. -func NewConnectionless() Endpoint { - ep := &connectionlessEndpoint{baseEndpoint{Queue: &waiter.Queue{}}} - ep.receiver = &queueReceiver{readQueue: queue.New(&waiter.Queue{}, ep.Queue, initialLimit)} - return ep -} - -// isBound returns true iff the endpoint is bound. -func (e *connectionlessEndpoint) isBound() bool { - return e.path != "" -} - -// Close puts the endpoint in a closed state and frees all resources associated -// with it. -// -// The socket will be a fresh state after a call to close and may be reused. -// That is, close may be used to "unbind" or "disconnect" the socket in error -// paths. -func (e *connectionlessEndpoint) Close() { - e.Lock() - var r Receiver - if e.Connected() { - e.receiver.CloseRecv() - r = e.receiver - e.receiver = nil - - e.connected.Release() - e.connected = nil - } - if e.isBound() { - e.path = "" - } - e.Unlock() - if r != nil { - r.CloseNotify() - r.Release() - } -} - -// BidirectionalConnect implements BoundEndpoint.BidirectionalConnect. -func (e *connectionlessEndpoint) BidirectionalConnect(ce ConnectingEndpoint, returnConnect func(Receiver, ConnectedEndpoint)) *tcpip.Error { - return tcpip.ErrConnectionRefused -} - -// UnidirectionalConnect implements BoundEndpoint.UnidirectionalConnect. -func (e *connectionlessEndpoint) UnidirectionalConnect() (ConnectedEndpoint, *tcpip.Error) { - e.Lock() - r := e.receiver - e.Unlock() - if r == nil { - return nil, tcpip.ErrConnectionRefused - } - return &connectedEndpoint{ - endpoint: e, - writeQueue: r.(*queueReceiver).readQueue, - }, nil -} - -// SendMsg writes data and a control message to the specified endpoint. -// This method does not block if the data cannot be written. -func (e *connectionlessEndpoint) SendMsg(data [][]byte, c ControlMessages, to BoundEndpoint) (uintptr, *tcpip.Error) { - if to == nil { - return e.baseEndpoint.SendMsg(data, c, nil) - } - - connected, err := to.UnidirectionalConnect() - if err != nil { - return 0, tcpip.ErrInvalidEndpointState - } - defer connected.Release() - - e.Lock() - n, notify, err := connected.Send(data, c, tcpip.FullAddress{Addr: tcpip.Address(e.path)}) - e.Unlock() - - if notify { - connected.SendNotify() - } - - return n, err -} - -// Type implements Endpoint.Type. -func (e *connectionlessEndpoint) Type() SockType { - return SockDgram -} - -// Connect attempts to connect directly to server. -func (e *connectionlessEndpoint) Connect(server BoundEndpoint) *tcpip.Error { - connected, err := server.UnidirectionalConnect() - if err != nil { - return err - } - - e.Lock() - e.connected = connected - e.Unlock() - - return nil -} - -// Listen starts listening on the connection. -func (e *connectionlessEndpoint) Listen(int) *tcpip.Error { - return tcpip.ErrNotSupported -} - -// Accept accepts a new connection. -func (e *connectionlessEndpoint) Accept() (Endpoint, *tcpip.Error) { - return nil, tcpip.ErrNotSupported -} - -// Bind binds the connection. -// -// For Unix endpoints, this _only sets the address associated with the socket_. -// Work associated with sockets in the filesystem or finding those sockets must -// be done by a higher level. -// -// Bind will fail only if the socket is connected, bound or the passed address -// is invalid (the empty string). -func (e *connectionlessEndpoint) Bind(addr tcpip.FullAddress, commit func() *tcpip.Error) *tcpip.Error { - e.Lock() - defer e.Unlock() - if e.isBound() { - return tcpip.ErrAlreadyBound - } - if addr.Addr == "" { - // The empty string is not permitted. - return tcpip.ErrBadLocalAddress - } - if commit != nil { - if err := commit(); err != nil { - return err - } - } - - // Save the bound address. - e.path = string(addr.Addr) - return nil -} - -// Readiness returns the current readiness of the endpoint. For example, if -// waiter.EventIn is set, the endpoint is immediately readable. -func (e *connectionlessEndpoint) Readiness(mask waiter.EventMask) waiter.EventMask { - e.Lock() - defer e.Unlock() - - ready := waiter.EventMask(0) - if mask&waiter.EventIn != 0 && e.receiver.Readable() { - ready |= waiter.EventIn - } - - if e.Connected() { - if mask&waiter.EventOut != 0 && e.connected.Writable() { - ready |= waiter.EventOut - } - } - - return ready -} diff --git a/pkg/tcpip/transport/unix/unix.go b/pkg/tcpip/transport/unix/unix.go deleted file mode 100644 index 1bca4b0b4..000000000 --- a/pkg/tcpip/transport/unix/unix.go +++ /dev/null @@ -1,953 +0,0 @@ -// Copyright 2018 Google Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package unix contains the implementation of Unix endpoints. -package unix - -import ( - "sync" - "sync/atomic" - - "gvisor.googlesource.com/gvisor/pkg/ilist" - "gvisor.googlesource.com/gvisor/pkg/tcpip" - "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" - "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/queue" - "gvisor.googlesource.com/gvisor/pkg/waiter" -) - -// initialLimit is the starting limit for the socket buffers. -const initialLimit = 16 * 1024 - -// A SockType is a type (as opposed to family) of sockets. These are enumerated -// in the syscall package as syscall.SOCK_* constants. -type SockType int - -const ( - // SockStream corresponds to syscall.SOCK_STREAM. - SockStream SockType = 1 - // SockDgram corresponds to syscall.SOCK_DGRAM. - SockDgram SockType = 2 - // SockRaw corresponds to syscall.SOCK_RAW. - SockRaw SockType = 3 - // SockSeqpacket corresponds to syscall.SOCK_SEQPACKET. - SockSeqpacket SockType = 5 -) - -// A RightsControlMessage is a control message containing FDs. -type RightsControlMessage interface { - // Clone returns a copy of the RightsControlMessage. - Clone() RightsControlMessage - - // Release releases any resources owned by the RightsControlMessage. - Release() -} - -// A CredentialsControlMessage is a control message containing Unix credentials. -type CredentialsControlMessage interface { - // Equals returns true iff the two messages are equal. - Equals(CredentialsControlMessage) bool -} - -// A ControlMessages represents a collection of socket control messages. -// -// +stateify savable -type ControlMessages struct { - // Rights is a control message containing FDs. - Rights RightsControlMessage - - // Credentials is a control message containing Unix credentials. - Credentials CredentialsControlMessage -} - -// Empty returns true iff the ControlMessages does not contain either -// credentials or rights. -func (c *ControlMessages) Empty() bool { - return c.Rights == nil && c.Credentials == nil -} - -// Clone clones both the credentials and the rights. -func (c *ControlMessages) Clone() ControlMessages { - cm := ControlMessages{} - if c.Rights != nil { - cm.Rights = c.Rights.Clone() - } - cm.Credentials = c.Credentials - return cm -} - -// Release releases both the credentials and the rights. -func (c *ControlMessages) Release() { - if c.Rights != nil { - c.Rights.Release() - } - *c = ControlMessages{} -} - -// Endpoint is the interface implemented by Unix transport protocol -// implementations that expose functionality like sendmsg, recvmsg, connect, -// etc. to Unix socket implementations. -type Endpoint interface { - Credentialer - waiter.Waitable - - // Close puts the endpoint in a closed state and frees all resources - // associated with it. - Close() - - // RecvMsg reads data and a control message from the endpoint. This method - // does not block if there is no data pending. - // - // creds indicates if credential control messages are requested by the - // caller. This is useful for determining if control messages can be - // coalesced. creds is a hint and can be safely ignored by the - // implementation if no coalescing is possible. It is fine to return - // credential control messages when none were requested or to not return - // credential control messages when they were requested. - // - // numRights is the number of SCM_RIGHTS FDs requested by the caller. This - // is useful if one must allocate a buffer to receive a SCM_RIGHTS message - // or determine if control messages can be coalesced. numRights is a hint - // and can be safely ignored by the implementation if the number of - // available SCM_RIGHTS FDs is known and no coalescing is possible. It is - // fine for the returned number of SCM_RIGHTS FDs to be either higher or - // lower than the requested number. - // - // If peek is true, no data should be consumed from the Endpoint. Any and - // all data returned from a peek should be available in the next call to - // RecvMsg. - // - // recvLen is the number of bytes copied into data. - // - // msgLen is the length of the read message consumed for datagram Endpoints. - // msgLen is always the same as recvLen for stream Endpoints. - RecvMsg(data [][]byte, creds bool, numRights uintptr, peek bool, addr *tcpip.FullAddress) (recvLen, msgLen uintptr, cm ControlMessages, err *tcpip.Error) - - // SendMsg writes data and a control message to the endpoint's peer. - // This method does not block if the data cannot be written. - // - // SendMsg does not take ownership of any of its arguments on error. - SendMsg([][]byte, ControlMessages, BoundEndpoint) (uintptr, *tcpip.Error) - - // Connect connects this endpoint directly to another. - // - // This should be called on the client endpoint, and the (bound) - // endpoint passed in as a parameter. - // - // The error codes are the same as Connect. - Connect(server BoundEndpoint) *tcpip.Error - - // Shutdown closes the read and/or write end of the endpoint connection - // to its peer. - Shutdown(flags tcpip.ShutdownFlags) *tcpip.Error - - // Listen puts the endpoint in "listen" mode, which allows it to accept - // new connections. - Listen(backlog int) *tcpip.Error - - // Accept returns a new endpoint if a peer has established a connection - // to an endpoint previously set to listen mode. This method does not - // block if no new connections are available. - // - // The returned Queue is the wait queue for the newly created endpoint. - Accept() (Endpoint, *tcpip.Error) - - // Bind binds the endpoint to a specific local address and port. - // Specifying a NIC is optional. - // - // An optional commit function will be executed atomically with respect - // to binding the endpoint. If this returns an error, the bind will not - // occur and the error will be propagated back to the caller. - Bind(address tcpip.FullAddress, commit func() *tcpip.Error) *tcpip.Error - - // Type return the socket type, typically either SockStream, SockDgram - // or SockSeqpacket. - Type() SockType - - // GetLocalAddress returns the address to which the endpoint is bound. - GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) - - // GetRemoteAddress returns the address to which the endpoint is - // connected. - GetRemoteAddress() (tcpip.FullAddress, *tcpip.Error) - - // SetSockOpt sets a socket option. opt should be one of the tcpip.*Option - // types. - SetSockOpt(opt interface{}) *tcpip.Error - - // GetSockOpt gets a socket option. opt should be a pointer to one of the - // tcpip.*Option types. - GetSockOpt(opt interface{}) *tcpip.Error -} - -// A Credentialer is a socket or endpoint that supports the SO_PASSCRED socket -// option. -type Credentialer interface { - // Passcred returns whether or not the SO_PASSCRED socket option is - // enabled on this end. - Passcred() bool - - // ConnectedPasscred returns whether or not the SO_PASSCRED socket option - // is enabled on the connected end. - ConnectedPasscred() bool -} - -// A BoundEndpoint is a unix endpoint that can be connected to. -type BoundEndpoint interface { - // BidirectionalConnect establishes a bi-directional connection between two - // unix endpoints in an all-or-nothing manner. If an error occurs during - // connecting, the state of neither endpoint should be modified. - // - // In order for an endpoint to establish such a bidirectional connection - // with a BoundEndpoint, the endpoint calls the BidirectionalConnect method - // on the BoundEndpoint and sends a representation of itself (the - // ConnectingEndpoint) and a callback (returnConnect) to receive the - // connection information (Receiver and ConnectedEndpoint) upon a - // successful connect. The callback should only be called on a successful - // connect. - // - // For a connection attempt to be successful, the ConnectingEndpoint must - // be unconnected and not listening and the BoundEndpoint whose - // BidirectionalConnect method is being called must be listening. - // - // This method will return tcpip.ErrConnectionRefused on endpoints with a - // type that isn't SockStream or SockSeqpacket. - BidirectionalConnect(ep ConnectingEndpoint, returnConnect func(Receiver, ConnectedEndpoint)) *tcpip.Error - - // UnidirectionalConnect establishes a write-only connection to a unix - // endpoint. - // - // An endpoint which calls UnidirectionalConnect and supports it itself must - // not hold its own lock when calling UnidirectionalConnect. - // - // This method will return tcpip.ErrConnectionRefused on a non-SockDgram - // endpoint. - UnidirectionalConnect() (ConnectedEndpoint, *tcpip.Error) - - // Release releases any resources held by the BoundEndpoint. It must be - // called before dropping all references to a BoundEndpoint returned by a - // function. - Release() -} - -// message represents a message passed over a Unix domain socket. -// -// +stateify savable -type message struct { - ilist.Entry - - // Data is the Message payload. - Data buffer.View - - // Control is auxiliary control message data that goes along with the - // data. - Control ControlMessages - - // Address is the bound address of the endpoint that sent the message. - // - // If the endpoint that sent the message is not bound, the Address is - // the empty string. - Address tcpip.FullAddress -} - -// Length returns number of bytes stored in the message. -func (m *message) Length() int64 { - return int64(len(m.Data)) -} - -// Release releases any resources held by the message. -func (m *message) Release() { - m.Control.Release() -} - -// Peek returns a copy of the message. -func (m *message) Peek() queue.Entry { - return &message{Data: m.Data, Control: m.Control.Clone(), Address: m.Address} -} - -// Truncate reduces the length of the message payload to n bytes. -// -// Preconditions: n <= m.Length(). -func (m *message) Truncate(n int64) { - m.Data.CapLength(int(n)) -} - -// A Receiver can be used to receive Messages. -type Receiver interface { - // Recv receives a single message. This method does not block. - // - // See Endpoint.RecvMsg for documentation on shared arguments. - // - // notify indicates if RecvNotify should be called. - Recv(data [][]byte, creds bool, numRights uintptr, peek bool) (recvLen, msgLen uintptr, cm ControlMessages, source tcpip.FullAddress, notify bool, err *tcpip.Error) - - // RecvNotify notifies the Receiver of a successful Recv. This must not be - // called while holding any endpoint locks. - RecvNotify() - - // CloseRecv prevents the receiving of additional Messages. - // - // After CloseRecv is called, CloseNotify must also be called. - CloseRecv() - - // CloseNotify notifies the Receiver of recv being closed. This must not be - // called while holding any endpoint locks. - CloseNotify() - - // Readable returns if messages should be attempted to be received. This - // includes when read has been shutdown. - Readable() bool - - // RecvQueuedSize returns the total amount of data currently receivable. - // RecvQueuedSize should return -1 if the operation isn't supported. - RecvQueuedSize() int64 - - // RecvMaxQueueSize returns maximum value for RecvQueuedSize. - // RecvMaxQueueSize should return -1 if the operation isn't supported. - RecvMaxQueueSize() int64 - - // Release releases any resources owned by the Receiver. It should be - // called before droping all references to a Receiver. - Release() -} - -// queueReceiver implements Receiver for datagram sockets. -// -// +stateify savable -type queueReceiver struct { - readQueue *queue.Queue -} - -// Recv implements Receiver.Recv. -func (q *queueReceiver) Recv(data [][]byte, creds bool, numRights uintptr, peek bool) (uintptr, uintptr, ControlMessages, tcpip.FullAddress, bool, *tcpip.Error) { - var m queue.Entry - var notify bool - var err *tcpip.Error - if peek { - m, err = q.readQueue.Peek() - } else { - m, notify, err = q.readQueue.Dequeue() - } - if err != nil { - return 0, 0, ControlMessages{}, tcpip.FullAddress{}, false, err - } - msg := m.(*message) - src := []byte(msg.Data) - var copied uintptr - for i := 0; i < len(data) && len(src) > 0; i++ { - n := copy(data[i], src) - copied += uintptr(n) - src = src[n:] - } - return copied, uintptr(len(msg.Data)), msg.Control, msg.Address, notify, nil -} - -// RecvNotify implements Receiver.RecvNotify. -func (q *queueReceiver) RecvNotify() { - q.readQueue.WriterQueue.Notify(waiter.EventOut) -} - -// CloseNotify implements Receiver.CloseNotify. -func (q *queueReceiver) CloseNotify() { - q.readQueue.ReaderQueue.Notify(waiter.EventIn) - q.readQueue.WriterQueue.Notify(waiter.EventOut) -} - -// CloseRecv implements Receiver.CloseRecv. -func (q *queueReceiver) CloseRecv() { - q.readQueue.Close() -} - -// Readable implements Receiver.Readable. -func (q *queueReceiver) Readable() bool { - return q.readQueue.IsReadable() -} - -// RecvQueuedSize implements Receiver.RecvQueuedSize. -func (q *queueReceiver) RecvQueuedSize() int64 { - return q.readQueue.QueuedSize() -} - -// RecvMaxQueueSize implements Receiver.RecvMaxQueueSize. -func (q *queueReceiver) RecvMaxQueueSize() int64 { - return q.readQueue.MaxQueueSize() -} - -// Release implements Receiver.Release. -func (*queueReceiver) Release() {} - -// streamQueueReceiver implements Receiver for stream sockets. -// -// +stateify savable -type streamQueueReceiver struct { - queueReceiver - - mu sync.Mutex `state:"nosave"` - buffer []byte - control ControlMessages - addr tcpip.FullAddress -} - -func vecCopy(data [][]byte, buf []byte) (uintptr, [][]byte, []byte) { - var copied uintptr - for len(data) > 0 && len(buf) > 0 { - n := copy(data[0], buf) - copied += uintptr(n) - buf = buf[n:] - data[0] = data[0][n:] - if len(data[0]) == 0 { - data = data[1:] - } - } - return copied, data, buf -} - -// Readable implements Receiver.Readable. -func (q *streamQueueReceiver) Readable() bool { - q.mu.Lock() - bl := len(q.buffer) - r := q.readQueue.IsReadable() - q.mu.Unlock() - // We're readable if we have data in our buffer or if the queue receiver is - // readable. - return bl > 0 || r -} - -// RecvQueuedSize implements Receiver.RecvQueuedSize. -func (q *streamQueueReceiver) RecvQueuedSize() int64 { - q.mu.Lock() - bl := len(q.buffer) - qs := q.readQueue.QueuedSize() - q.mu.Unlock() - return int64(bl) + qs -} - -// RecvMaxQueueSize implements Receiver.RecvMaxQueueSize. -func (q *streamQueueReceiver) RecvMaxQueueSize() int64 { - // The RecvMaxQueueSize() is the readQueue's MaxQueueSize() plus the largest - // message we can buffer which is also the largest message we can receive. - return 2 * q.readQueue.MaxQueueSize() -} - -// Recv implements Receiver.Recv. -func (q *streamQueueReceiver) Recv(data [][]byte, wantCreds bool, numRights uintptr, peek bool) (uintptr, uintptr, ControlMessages, tcpip.FullAddress, bool, *tcpip.Error) { - q.mu.Lock() - defer q.mu.Unlock() - - var notify bool - - // If we have no data in the endpoint, we need to get some. - if len(q.buffer) == 0 { - // Load the next message into a buffer, even if we are peeking. Peeking - // won't consume the message, so it will be still available to be read - // the next time Recv() is called. - m, n, err := q.readQueue.Dequeue() - if err != nil { - return 0, 0, ControlMessages{}, tcpip.FullAddress{}, false, err - } - notify = n - msg := m.(*message) - q.buffer = []byte(msg.Data) - q.control = msg.Control - q.addr = msg.Address - } - - var copied uintptr - if peek { - // Don't consume control message if we are peeking. - c := q.control.Clone() - - // Don't consume data since we are peeking. - copied, data, _ = vecCopy(data, q.buffer) - - return copied, copied, c, q.addr, notify, nil - } - - // Consume data and control message since we are not peeking. - copied, data, q.buffer = vecCopy(data, q.buffer) - - // Save the original state of q.control. - c := q.control - - // Remove rights from q.control and leave behind just the creds. - q.control.Rights = nil - if !wantCreds { - c.Credentials = nil - } - - if c.Rights != nil && numRights == 0 { - c.Rights.Release() - c.Rights = nil - } - - haveRights := c.Rights != nil - - // If we have more capacity for data and haven't received any usable - // rights. - // - // Linux never coalesces rights control messages. - for !haveRights && len(data) > 0 { - // Get a message from the readQueue. - m, n, err := q.readQueue.Dequeue() - if err != nil { - // We already got some data, so ignore this error. This will - // manifest as a short read to the user, which is what Linux - // does. - break - } - notify = notify || n - msg := m.(*message) - q.buffer = []byte(msg.Data) - q.control = msg.Control - q.addr = msg.Address - - if wantCreds { - if (q.control.Credentials == nil) != (c.Credentials == nil) { - // One message has credentials, the other does not. - break - } - - if q.control.Credentials != nil && c.Credentials != nil && !q.control.Credentials.Equals(c.Credentials) { - // Both messages have credentials, but they don't match. - break - } - } - - if numRights != 0 && c.Rights != nil && q.control.Rights != nil { - // Both messages have rights. - break - } - - var cpd uintptr - cpd, data, q.buffer = vecCopy(data, q.buffer) - copied += cpd - - if cpd == 0 { - // data was actually full. - break - } - - if q.control.Rights != nil { - // Consume rights. - if numRights == 0 { - q.control.Rights.Release() - } else { - c.Rights = q.control.Rights - haveRights = true - } - q.control.Rights = nil - } - } - return copied, copied, c, q.addr, notify, nil -} - -// A ConnectedEndpoint is an Endpoint that can be used to send Messages. -type ConnectedEndpoint interface { - // Passcred implements Endpoint.Passcred. - Passcred() bool - - // GetLocalAddress implements Endpoint.GetLocalAddress. - GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) - - // Send sends a single message. This method does not block. - // - // notify indicates if SendNotify should be called. - // - // tcpip.ErrWouldBlock can be returned along with a partial write if - // the caller should block to send the rest of the data. - Send(data [][]byte, controlMessages ControlMessages, from tcpip.FullAddress) (n uintptr, notify bool, err *tcpip.Error) - - // SendNotify notifies the ConnectedEndpoint of a successful Send. This - // must not be called while holding any endpoint locks. - SendNotify() - - // CloseSend prevents the sending of additional Messages. - // - // After CloseSend is call, CloseNotify must also be called. - CloseSend() - - // CloseNotify notifies the ConnectedEndpoint of send being closed. This - // must not be called while holding any endpoint locks. - CloseNotify() - - // Writable returns if messages should be attempted to be sent. This - // includes when write has been shutdown. - Writable() bool - - // EventUpdate lets the ConnectedEndpoint know that event registrations - // have changed. - EventUpdate() - - // SendQueuedSize returns the total amount of data currently queued for - // sending. SendQueuedSize should return -1 if the operation isn't - // supported. - SendQueuedSize() int64 - - // SendMaxQueueSize returns maximum value for SendQueuedSize. - // SendMaxQueueSize should return -1 if the operation isn't supported. - SendMaxQueueSize() int64 - - // Release releases any resources owned by the ConnectedEndpoint. It should - // be called before droping all references to a ConnectedEndpoint. - Release() -} - -// +stateify savable -type connectedEndpoint struct { - // endpoint represents the subset of the Endpoint functionality needed by - // the connectedEndpoint. It is implemented by both connectionedEndpoint - // and connectionlessEndpoint and allows the use of types which don't - // fully implement Endpoint. - endpoint interface { - // Passcred implements Endpoint.Passcred. - Passcred() bool - - // GetLocalAddress implements Endpoint.GetLocalAddress. - GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) - - // Type implements Endpoint.Type. - Type() SockType - } - - writeQueue *queue.Queue -} - -// Passcred implements ConnectedEndpoint.Passcred. -func (e *connectedEndpoint) Passcred() bool { - return e.endpoint.Passcred() -} - -// GetLocalAddress implements ConnectedEndpoint.GetLocalAddress. -func (e *connectedEndpoint) GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) { - return e.endpoint.GetLocalAddress() -} - -// Send implements ConnectedEndpoint.Send. -func (e *connectedEndpoint) Send(data [][]byte, controlMessages ControlMessages, from tcpip.FullAddress) (uintptr, bool, *tcpip.Error) { - var l int64 - for _, d := range data { - l += int64(len(d)) - } - - truncate := false - if e.endpoint.Type() == SockStream { - // Since stream sockets don't preserve message boundaries, we - // can write only as much of the message as fits in the queue. - truncate = true - - // Discard empty stream packets. Since stream sockets don't - // preserve message boundaries, sending zero bytes is a no-op. - // In Linux, the receiver actually uses a zero-length receive - // as an indication that the stream was closed. - if l == 0 { - controlMessages.Release() - return 0, false, nil - } - } - - v := make([]byte, 0, l) - for _, d := range data { - v = append(v, d...) - } - - l, notify, err := e.writeQueue.Enqueue(&message{Data: buffer.View(v), Control: controlMessages, Address: from}, truncate) - return uintptr(l), notify, err -} - -// SendNotify implements ConnectedEndpoint.SendNotify. -func (e *connectedEndpoint) SendNotify() { - e.writeQueue.ReaderQueue.Notify(waiter.EventIn) -} - -// CloseNotify implements ConnectedEndpoint.CloseNotify. -func (e *connectedEndpoint) CloseNotify() { - e.writeQueue.ReaderQueue.Notify(waiter.EventIn) - e.writeQueue.WriterQueue.Notify(waiter.EventOut) -} - -// CloseSend implements ConnectedEndpoint.CloseSend. -func (e *connectedEndpoint) CloseSend() { - e.writeQueue.Close() -} - -// Writable implements ConnectedEndpoint.Writable. -func (e *connectedEndpoint) Writable() bool { - return e.writeQueue.IsWritable() -} - -// EventUpdate implements ConnectedEndpoint.EventUpdate. -func (*connectedEndpoint) EventUpdate() {} - -// SendQueuedSize implements ConnectedEndpoint.SendQueuedSize. -func (e *connectedEndpoint) SendQueuedSize() int64 { - return e.writeQueue.QueuedSize() -} - -// SendMaxQueueSize implements ConnectedEndpoint.SendMaxQueueSize. -func (e *connectedEndpoint) SendMaxQueueSize() int64 { - return e.writeQueue.MaxQueueSize() -} - -// Release implements ConnectedEndpoint.Release. -func (*connectedEndpoint) Release() {} - -// baseEndpoint is an embeddable unix endpoint base used in both the connected and connectionless -// unix domain socket Endpoint implementations. -// -// Not to be used on its own. -// -// +stateify savable -type baseEndpoint struct { - *waiter.Queue - - // passcred specifies whether SCM_CREDENTIALS socket control messages are - // enabled on this endpoint. Must be accessed atomically. - passcred int32 - - // Mutex protects the below fields. - sync.Mutex `state:"nosave"` - - // receiver allows Messages to be received. - receiver Receiver - - // connected allows messages to be sent and state information about the - // connected endpoint to be read. - connected ConnectedEndpoint - - // path is not empty if the endpoint has been bound, - // or may be used if the endpoint is connected. - path string -} - -// EventRegister implements waiter.Waitable.EventRegister. -func (e *baseEndpoint) EventRegister(we *waiter.Entry, mask waiter.EventMask) { - e.Queue.EventRegister(we, mask) - e.Lock() - if e.connected != nil { - e.connected.EventUpdate() - } - e.Unlock() -} - -// EventUnregister implements waiter.Waitable.EventUnregister. -func (e *baseEndpoint) EventUnregister(we *waiter.Entry) { - e.Queue.EventUnregister(we) - e.Lock() - if e.connected != nil { - e.connected.EventUpdate() - } - e.Unlock() -} - -// Passcred implements Credentialer.Passcred. -func (e *baseEndpoint) Passcred() bool { - return atomic.LoadInt32(&e.passcred) != 0 -} - -// ConnectedPasscred implements Credentialer.ConnectedPasscred. -func (e *baseEndpoint) ConnectedPasscred() bool { - e.Lock() - defer e.Unlock() - return e.connected != nil && e.connected.Passcred() -} - -func (e *baseEndpoint) setPasscred(pc bool) { - if pc { - atomic.StoreInt32(&e.passcred, 1) - } else { - atomic.StoreInt32(&e.passcred, 0) - } -} - -// Connected implements ConnectingEndpoint.Connected. -func (e *baseEndpoint) Connected() bool { - return e.receiver != nil && e.connected != nil -} - -// RecvMsg reads data and a control message from the endpoint. -func (e *baseEndpoint) RecvMsg(data [][]byte, creds bool, numRights uintptr, peek bool, addr *tcpip.FullAddress) (uintptr, uintptr, ControlMessages, *tcpip.Error) { - e.Lock() - - if e.receiver == nil { - e.Unlock() - return 0, 0, ControlMessages{}, tcpip.ErrNotConnected - } - - recvLen, msgLen, cms, a, notify, err := e.receiver.Recv(data, creds, numRights, peek) - e.Unlock() - if err != nil { - return 0, 0, ControlMessages{}, err - } - - if notify { - e.receiver.RecvNotify() - } - - if addr != nil { - *addr = a - } - return recvLen, msgLen, cms, nil -} - -// SendMsg writes data and a control message to the endpoint's peer. -// This method does not block if the data cannot be written. -func (e *baseEndpoint) SendMsg(data [][]byte, c ControlMessages, to BoundEndpoint) (uintptr, *tcpip.Error) { - e.Lock() - if !e.Connected() { - e.Unlock() - return 0, tcpip.ErrNotConnected - } - if to != nil { - e.Unlock() - return 0, tcpip.ErrAlreadyConnected - } - - n, notify, err := e.connected.Send(data, c, tcpip.FullAddress{Addr: tcpip.Address(e.path)}) - e.Unlock() - - if notify { - e.connected.SendNotify() - } - - return n, err -} - -// SetSockOpt sets a socket option. Currently not supported. -func (e *baseEndpoint) SetSockOpt(opt interface{}) *tcpip.Error { - switch v := opt.(type) { - case tcpip.PasscredOption: - e.setPasscred(v != 0) - return nil - } - return nil -} - -// GetSockOpt implements tcpip.Endpoint.GetSockOpt. -func (e *baseEndpoint) GetSockOpt(opt interface{}) *tcpip.Error { - switch o := opt.(type) { - case tcpip.ErrorOption: - return nil - case *tcpip.SendQueueSizeOption: - e.Lock() - if !e.Connected() { - e.Unlock() - return tcpip.ErrNotConnected - } - qs := tcpip.SendQueueSizeOption(e.connected.SendQueuedSize()) - e.Unlock() - if qs < 0 { - return tcpip.ErrQueueSizeNotSupported - } - *o = qs - return nil - case *tcpip.ReceiveQueueSizeOption: - e.Lock() - if !e.Connected() { - e.Unlock() - return tcpip.ErrNotConnected - } - qs := tcpip.ReceiveQueueSizeOption(e.receiver.RecvQueuedSize()) - e.Unlock() - if qs < 0 { - return tcpip.ErrQueueSizeNotSupported - } - *o = qs - return nil - case *tcpip.PasscredOption: - if e.Passcred() { - *o = tcpip.PasscredOption(1) - } else { - *o = tcpip.PasscredOption(0) - } - return nil - case *tcpip.SendBufferSizeOption: - e.Lock() - if !e.Connected() { - e.Unlock() - return tcpip.ErrNotConnected - } - qs := tcpip.SendBufferSizeOption(e.connected.SendMaxQueueSize()) - e.Unlock() - if qs < 0 { - return tcpip.ErrQueueSizeNotSupported - } - *o = qs - return nil - case *tcpip.ReceiveBufferSizeOption: - e.Lock() - if e.receiver == nil { - e.Unlock() - return tcpip.ErrNotConnected - } - qs := tcpip.ReceiveBufferSizeOption(e.receiver.RecvMaxQueueSize()) - e.Unlock() - if qs < 0 { - return tcpip.ErrQueueSizeNotSupported - } - *o = qs - return nil - } - return tcpip.ErrUnknownProtocolOption -} - -// Shutdown closes the read and/or write end of the endpoint connection to its -// peer. -func (e *baseEndpoint) Shutdown(flags tcpip.ShutdownFlags) *tcpip.Error { - e.Lock() - if !e.Connected() { - e.Unlock() - return tcpip.ErrNotConnected - } - - if flags&tcpip.ShutdownRead != 0 { - e.receiver.CloseRecv() - } - - if flags&tcpip.ShutdownWrite != 0 { - e.connected.CloseSend() - } - - e.Unlock() - - if flags&tcpip.ShutdownRead != 0 { - e.receiver.CloseNotify() - } - - if flags&tcpip.ShutdownWrite != 0 { - e.connected.CloseNotify() - } - - return nil -} - -// GetLocalAddress returns the bound path. -func (e *baseEndpoint) GetLocalAddress() (tcpip.FullAddress, *tcpip.Error) { - e.Lock() - defer e.Unlock() - return tcpip.FullAddress{Addr: tcpip.Address(e.path)}, nil -} - -// GetRemoteAddress returns the local address of the connected endpoint (if -// available). -func (e *baseEndpoint) GetRemoteAddress() (tcpip.FullAddress, *tcpip.Error) { - e.Lock() - c := e.connected - e.Unlock() - if c != nil { - return c.GetLocalAddress() - } - return tcpip.FullAddress{}, tcpip.ErrNotConnected -} - -// Release implements BoundEndpoint.Release. -func (*baseEndpoint) Release() {} -- cgit v1.2.3 From 8fce67af24945f82378b4c2731cca1788936d074 Mon Sep 17 00:00:00 2001 From: Ian Gudger Date: Fri, 19 Oct 2018 16:34:09 -0700 Subject: Use correct company name in copyright header PiperOrigin-RevId: 217951017 Change-Id: Ie08bf6987f98467d07457bcf35b5f1ff6e43c035 --- kokoro/run_build.sh | 2 +- kokoro/run_tests.sh | 2 +- pkg/abi/abi.go | 2 +- pkg/abi/abi_linux.go | 2 +- pkg/abi/flag.go | 2 +- pkg/abi/linux/aio.go | 2 +- pkg/abi/linux/ashmem.go | 2 +- pkg/abi/linux/binder.go | 2 +- pkg/abi/linux/bpf.go | 2 +- pkg/abi/linux/capability.go | 2 +- pkg/abi/linux/dev.go | 2 +- pkg/abi/linux/elf.go | 2 +- pkg/abi/linux/errors.go | 2 +- pkg/abi/linux/eventfd.go | 2 +- pkg/abi/linux/exec.go | 2 +- pkg/abi/linux/fcntl.go | 2 +- pkg/abi/linux/file.go | 2 +- pkg/abi/linux/fs.go | 2 +- pkg/abi/linux/futex.go | 2 +- pkg/abi/linux/inotify.go | 2 +- pkg/abi/linux/ioctl.go | 2 +- pkg/abi/linux/ip.go | 2 +- pkg/abi/linux/ipc.go | 2 +- pkg/abi/linux/limits.go | 2 +- pkg/abi/linux/linux.go | 2 +- pkg/abi/linux/mm.go | 2 +- pkg/abi/linux/netdevice.go | 2 +- pkg/abi/linux/netlink.go | 2 +- pkg/abi/linux/netlink_route.go | 2 +- pkg/abi/linux/poll.go | 2 +- pkg/abi/linux/prctl.go | 2 +- pkg/abi/linux/ptrace.go | 2 +- pkg/abi/linux/rusage.go | 2 +- pkg/abi/linux/sched.go | 2 +- pkg/abi/linux/seccomp.go | 2 +- pkg/abi/linux/sem.go | 2 +- pkg/abi/linux/shm.go | 2 +- pkg/abi/linux/signal.go | 2 +- pkg/abi/linux/socket.go | 2 +- pkg/abi/linux/time.go | 2 +- pkg/abi/linux/timer.go | 2 +- pkg/abi/linux/tty.go | 2 +- pkg/abi/linux/uio.go | 2 +- pkg/abi/linux/utsname.go | 2 +- pkg/amutex/amutex.go | 2 +- pkg/amutex/amutex_test.go | 2 +- pkg/atomicbitops/atomic_bitops.go | 2 +- pkg/atomicbitops/atomic_bitops_amd64.s | 2 +- pkg/atomicbitops/atomic_bitops_common.go | 2 +- pkg/atomicbitops/atomic_bitops_test.go | 2 +- pkg/binary/binary.go | 2 +- pkg/binary/binary_test.go | 2 +- pkg/bits/bits.go | 2 +- pkg/bits/bits_template.go | 2 +- pkg/bits/uint64_arch_amd64.go | 2 +- pkg/bits/uint64_arch_amd64_asm.s | 2 +- pkg/bits/uint64_arch_generic.go | 2 +- pkg/bits/uint64_test.go | 2 +- pkg/bpf/bpf.go | 2 +- pkg/bpf/decoder.go | 2 +- pkg/bpf/decoder_test.go | 2 +- pkg/bpf/input_bytes.go | 2 +- pkg/bpf/interpreter.go | 2 +- pkg/bpf/interpreter_test.go | 2 +- pkg/bpf/program_builder.go | 2 +- pkg/bpf/program_builder_test.go | 2 +- pkg/compressio/compressio.go | 2 +- pkg/compressio/compressio_test.go | 2 +- pkg/control/client/client.go | 2 +- pkg/control/server/server.go | 2 +- pkg/cpuid/cpu_amd64.s | 2 +- pkg/cpuid/cpuid.go | 2 +- pkg/cpuid/cpuid_parse_test.go | 2 +- pkg/cpuid/cpuid_test.go | 2 +- pkg/dhcp/client.go | 2 +- pkg/dhcp/dhcp.go | 2 +- pkg/dhcp/dhcp_string.go | 2 +- pkg/dhcp/dhcp_test.go | 2 +- pkg/dhcp/server.go | 2 +- pkg/eventchannel/event.go | 2 +- pkg/eventchannel/event.proto | 2 +- pkg/fd/fd.go | 2 +- pkg/fd/fd_test.go | 2 +- pkg/gate/gate.go | 2 +- pkg/gate/gate_test.go | 2 +- pkg/ilist/list.go | 2 +- pkg/ilist/list_test.go | 2 +- pkg/linewriter/linewriter.go | 2 +- pkg/linewriter/linewriter_test.go | 2 +- pkg/log/glog.go | 2 +- pkg/log/glog_unsafe.go | 2 +- pkg/log/json.go | 2 +- pkg/log/json_test.go | 2 +- pkg/log/log.go | 2 +- pkg/log/log_test.go | 2 +- pkg/metric/metric.go | 2 +- pkg/metric/metric.proto | 2 +- pkg/metric/metric_test.go | 2 +- pkg/p9/buffer.go | 2 +- pkg/p9/client.go | 2 +- pkg/p9/client_file.go | 2 +- pkg/p9/client_test.go | 2 +- pkg/p9/file.go | 2 +- pkg/p9/handlers.go | 2 +- pkg/p9/local_server/local_server.go | 2 +- pkg/p9/messages.go | 2 +- pkg/p9/messages_test.go | 2 +- pkg/p9/p9.go | 2 +- pkg/p9/p9_test.go | 2 +- pkg/p9/p9test/client_test.go | 2 +- pkg/p9/p9test/mocks.go | 2 +- pkg/p9/pool.go | 2 +- pkg/p9/pool_test.go | 2 +- pkg/p9/server.go | 2 +- pkg/p9/transport.go | 2 +- pkg/p9/transport_test.go | 2 +- pkg/p9/version.go | 2 +- pkg/p9/version_test.go | 2 +- pkg/rand/rand.go | 2 +- pkg/rand/rand_linux.go | 2 +- pkg/refs/refcounter.go | 2 +- pkg/refs/refcounter_state.go | 2 +- pkg/refs/refcounter_test.go | 2 +- pkg/seccomp/seccomp.go | 2 +- pkg/seccomp/seccomp_rules.go | 2 +- pkg/seccomp/seccomp_test.go | 2 +- pkg/seccomp/seccomp_test_victim.go | 2 +- pkg/seccomp/seccomp_unsafe.go | 2 +- pkg/secio/full_reader.go | 2 +- pkg/secio/secio.go | 2 +- pkg/secio/secio_test.go | 2 +- pkg/segment/range.go | 2 +- pkg/segment/set.go | 2 +- pkg/segment/set_state.go | 2 +- pkg/segment/test/segment_test.go | 2 +- pkg/segment/test/set_functions.go | 2 +- pkg/sentry/arch/aligned.go | 2 +- pkg/sentry/arch/arch.go | 2 +- pkg/sentry/arch/arch_amd64.go | 2 +- pkg/sentry/arch/arch_amd64.s | 2 +- pkg/sentry/arch/arch_state_x86.go | 2 +- pkg/sentry/arch/arch_x86.go | 2 +- pkg/sentry/arch/auxv.go | 2 +- pkg/sentry/arch/registers.proto | 2 +- pkg/sentry/arch/signal_act.go | 2 +- pkg/sentry/arch/signal_amd64.go | 2 +- pkg/sentry/arch/signal_info.go | 2 +- pkg/sentry/arch/signal_stack.go | 2 +- pkg/sentry/arch/stack.go | 2 +- pkg/sentry/arch/syscalls_amd64.go | 2 +- pkg/sentry/context/context.go | 2 +- pkg/sentry/context/contexttest/contexttest.go | 2 +- pkg/sentry/control/control.go | 2 +- pkg/sentry/control/proc.go | 2 +- pkg/sentry/control/proc_test.go | 2 +- pkg/sentry/control/state.go | 2 +- pkg/sentry/device/device.go | 2 +- pkg/sentry/device/device_test.go | 2 +- pkg/sentry/fs/anon/anon.go | 2 +- pkg/sentry/fs/anon/device.go | 2 +- pkg/sentry/fs/ashmem/area.go | 2 +- pkg/sentry/fs/ashmem/device.go | 2 +- pkg/sentry/fs/ashmem/pin_board.go | 2 +- pkg/sentry/fs/ashmem/pin_board_test.go | 2 +- pkg/sentry/fs/attr.go | 2 +- pkg/sentry/fs/binder/binder.go | 2 +- pkg/sentry/fs/context.go | 2 +- pkg/sentry/fs/copy_up.go | 2 +- pkg/sentry/fs/copy_up_test.go | 2 +- pkg/sentry/fs/dentry.go | 2 +- pkg/sentry/fs/dev/dev.go | 2 +- pkg/sentry/fs/dev/device.go | 2 +- pkg/sentry/fs/dev/fs.go | 2 +- pkg/sentry/fs/dev/full.go | 2 +- pkg/sentry/fs/dev/null.go | 2 +- pkg/sentry/fs/dev/random.go | 2 +- pkg/sentry/fs/dirent.go | 2 +- pkg/sentry/fs/dirent_cache.go | 2 +- pkg/sentry/fs/dirent_cache_test.go | 2 +- pkg/sentry/fs/dirent_refs_test.go | 2 +- pkg/sentry/fs/dirent_state.go | 2 +- pkg/sentry/fs/fdpipe/pipe.go | 2 +- pkg/sentry/fs/fdpipe/pipe_opener.go | 2 +- pkg/sentry/fs/fdpipe/pipe_opener_test.go | 2 +- pkg/sentry/fs/fdpipe/pipe_state.go | 2 +- pkg/sentry/fs/fdpipe/pipe_test.go | 2 +- pkg/sentry/fs/file.go | 2 +- pkg/sentry/fs/file_operations.go | 2 +- pkg/sentry/fs/file_overlay.go | 2 +- pkg/sentry/fs/file_overlay_test.go | 2 +- pkg/sentry/fs/file_state.go | 2 +- pkg/sentry/fs/file_test.go | 2 +- pkg/sentry/fs/filesystems.go | 2 +- pkg/sentry/fs/filetest/filetest.go | 2 +- pkg/sentry/fs/flags.go | 2 +- pkg/sentry/fs/fs.go | 2 +- pkg/sentry/fs/fsutil/dirty_set.go | 2 +- pkg/sentry/fs/fsutil/dirty_set_test.go | 2 +- pkg/sentry/fs/fsutil/file.go | 2 +- pkg/sentry/fs/fsutil/file_range_set.go | 2 +- pkg/sentry/fs/fsutil/frame_ref_set.go | 2 +- pkg/sentry/fs/fsutil/fsutil.go | 2 +- pkg/sentry/fs/fsutil/handle.go | 2 +- pkg/sentry/fs/fsutil/handle_test.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper_state.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go | 2 +- pkg/sentry/fs/fsutil/inode.go | 2 +- pkg/sentry/fs/fsutil/inode_cached.go | 2 +- pkg/sentry/fs/fsutil/inode_cached_test.go | 2 +- pkg/sentry/fs/gofer/attr.go | 2 +- pkg/sentry/fs/gofer/cache_policy.go | 2 +- pkg/sentry/fs/gofer/context_file.go | 2 +- pkg/sentry/fs/gofer/device.go | 2 +- pkg/sentry/fs/gofer/file.go | 2 +- pkg/sentry/fs/gofer/file_state.go | 2 +- pkg/sentry/fs/gofer/fs.go | 2 +- pkg/sentry/fs/gofer/gofer_test.go | 2 +- pkg/sentry/fs/gofer/handles.go | 2 +- pkg/sentry/fs/gofer/inode.go | 2 +- pkg/sentry/fs/gofer/inode_state.go | 2 +- pkg/sentry/fs/gofer/path.go | 2 +- pkg/sentry/fs/gofer/session.go | 2 +- pkg/sentry/fs/gofer/session_state.go | 2 +- pkg/sentry/fs/gofer/socket.go | 2 +- pkg/sentry/fs/gofer/util.go | 2 +- pkg/sentry/fs/host/control.go | 2 +- pkg/sentry/fs/host/descriptor.go | 2 +- pkg/sentry/fs/host/descriptor_state.go | 2 +- pkg/sentry/fs/host/descriptor_test.go | 2 +- pkg/sentry/fs/host/device.go | 2 +- pkg/sentry/fs/host/file.go | 2 +- pkg/sentry/fs/host/fs.go | 2 +- pkg/sentry/fs/host/fs_test.go | 2 +- pkg/sentry/fs/host/inode.go | 2 +- pkg/sentry/fs/host/inode_state.go | 2 +- pkg/sentry/fs/host/inode_test.go | 2 +- pkg/sentry/fs/host/ioctl_unsafe.go | 2 +- pkg/sentry/fs/host/socket.go | 2 +- pkg/sentry/fs/host/socket_iovec.go | 2 +- pkg/sentry/fs/host/socket_state.go | 2 +- pkg/sentry/fs/host/socket_test.go | 2 +- pkg/sentry/fs/host/socket_unsafe.go | 2 +- pkg/sentry/fs/host/tty.go | 2 +- pkg/sentry/fs/host/util.go | 2 +- pkg/sentry/fs/host/util_unsafe.go | 2 +- pkg/sentry/fs/host/wait_test.go | 2 +- pkg/sentry/fs/inode.go | 2 +- pkg/sentry/fs/inode_inotify.go | 2 +- pkg/sentry/fs/inode_operations.go | 2 +- pkg/sentry/fs/inode_overlay.go | 2 +- pkg/sentry/fs/inode_overlay_test.go | 2 +- pkg/sentry/fs/inotify.go | 2 +- pkg/sentry/fs/inotify_event.go | 2 +- pkg/sentry/fs/inotify_watch.go | 2 +- pkg/sentry/fs/lock/lock.go | 2 +- pkg/sentry/fs/lock/lock_range_test.go | 2 +- pkg/sentry/fs/lock/lock_set_functions.go | 2 +- pkg/sentry/fs/lock/lock_test.go | 2 +- pkg/sentry/fs/mock.go | 2 +- pkg/sentry/fs/mount.go | 2 +- pkg/sentry/fs/mount_overlay.go | 2 +- pkg/sentry/fs/mount_state.go | 2 +- pkg/sentry/fs/mount_test.go | 2 +- pkg/sentry/fs/mounts.go | 2 +- pkg/sentry/fs/mounts_test.go | 2 +- pkg/sentry/fs/offset.go | 2 +- pkg/sentry/fs/overlay.go | 2 +- pkg/sentry/fs/path.go | 2 +- pkg/sentry/fs/path_test.go | 2 +- pkg/sentry/fs/proc/cpuinfo.go | 2 +- pkg/sentry/fs/proc/device/device.go | 2 +- pkg/sentry/fs/proc/exec_args.go | 2 +- pkg/sentry/fs/proc/fds.go | 2 +- pkg/sentry/fs/proc/file.go | 2 +- pkg/sentry/fs/proc/filesystems.go | 2 +- pkg/sentry/fs/proc/fs.go | 2 +- pkg/sentry/fs/proc/loadavg.go | 2 +- pkg/sentry/fs/proc/meminfo.go | 2 +- pkg/sentry/fs/proc/mounts.go | 2 +- pkg/sentry/fs/proc/net.go | 2 +- pkg/sentry/fs/proc/net_test.go | 2 +- pkg/sentry/fs/proc/proc.go | 2 +- pkg/sentry/fs/proc/rpcinet_proc.go | 2 +- pkg/sentry/fs/proc/seqfile/seqfile.go | 2 +- pkg/sentry/fs/proc/seqfile/seqfile_test.go | 2 +- pkg/sentry/fs/proc/stat.go | 2 +- pkg/sentry/fs/proc/sys.go | 2 +- pkg/sentry/fs/proc/sys_net.go | 2 +- pkg/sentry/fs/proc/sys_net_test.go | 2 +- pkg/sentry/fs/proc/task.go | 2 +- pkg/sentry/fs/proc/uid_gid_map.go | 2 +- pkg/sentry/fs/proc/uptime.go | 2 +- pkg/sentry/fs/proc/version.go | 2 +- pkg/sentry/fs/ramfs/dir.go | 2 +- pkg/sentry/fs/ramfs/file.go | 2 +- pkg/sentry/fs/ramfs/ramfs.go | 2 +- pkg/sentry/fs/ramfs/socket.go | 2 +- pkg/sentry/fs/ramfs/symlink.go | 2 +- pkg/sentry/fs/ramfs/test/test.go | 2 +- pkg/sentry/fs/ramfs/tree.go | 2 +- pkg/sentry/fs/ramfs/tree_test.go | 2 +- pkg/sentry/fs/restore.go | 2 +- pkg/sentry/fs/save.go | 2 +- pkg/sentry/fs/seek.go | 2 +- pkg/sentry/fs/sync.go | 2 +- pkg/sentry/fs/sys/device.go | 2 +- pkg/sentry/fs/sys/devices.go | 2 +- pkg/sentry/fs/sys/fs.go | 2 +- pkg/sentry/fs/sys/sys.go | 2 +- pkg/sentry/fs/timerfd/timerfd.go | 2 +- pkg/sentry/fs/tmpfs/device.go | 2 +- pkg/sentry/fs/tmpfs/file_regular.go | 2 +- pkg/sentry/fs/tmpfs/file_test.go | 2 +- pkg/sentry/fs/tmpfs/fs.go | 2 +- pkg/sentry/fs/tmpfs/inode_file.go | 2 +- pkg/sentry/fs/tmpfs/tmpfs.go | 2 +- pkg/sentry/fs/tty/dir.go | 2 +- pkg/sentry/fs/tty/fs.go | 2 +- pkg/sentry/fs/tty/inode.go | 2 +- pkg/sentry/fs/tty/line_discipline.go | 2 +- pkg/sentry/fs/tty/master.go | 2 +- pkg/sentry/fs/tty/queue.go | 2 +- pkg/sentry/fs/tty/slave.go | 2 +- pkg/sentry/fs/tty/terminal.go | 2 +- pkg/sentry/fs/tty/tty_test.go | 2 +- pkg/sentry/hostcpu/getcpu_amd64.s | 2 +- pkg/sentry/hostcpu/hostcpu.go | 2 +- pkg/sentry/hostcpu/hostcpu_test.go | 2 +- pkg/sentry/inet/context.go | 2 +- pkg/sentry/inet/inet.go | 2 +- pkg/sentry/inet/test_stack.go | 2 +- pkg/sentry/kernel/abstract_socket_namespace.go | 2 +- pkg/sentry/kernel/auth/auth.go | 2 +- pkg/sentry/kernel/auth/capability_set.go | 2 +- pkg/sentry/kernel/auth/context.go | 2 +- pkg/sentry/kernel/auth/credentials.go | 2 +- pkg/sentry/kernel/auth/id.go | 2 +- pkg/sentry/kernel/auth/id_map.go | 2 +- pkg/sentry/kernel/auth/id_map_functions.go | 2 +- pkg/sentry/kernel/auth/user_namespace.go | 2 +- pkg/sentry/kernel/context.go | 2 +- pkg/sentry/kernel/epoll/epoll.go | 2 +- pkg/sentry/kernel/epoll/epoll_state.go | 2 +- pkg/sentry/kernel/epoll/epoll_test.go | 2 +- pkg/sentry/kernel/eventfd/eventfd.go | 2 +- pkg/sentry/kernel/eventfd/eventfd_test.go | 2 +- pkg/sentry/kernel/fasync/fasync.go | 2 +- pkg/sentry/kernel/fd_map.go | 2 +- pkg/sentry/kernel/fd_map_test.go | 2 +- pkg/sentry/kernel/fs_context.go | 2 +- pkg/sentry/kernel/futex/futex.go | 2 +- pkg/sentry/kernel/futex/futex_test.go | 2 +- pkg/sentry/kernel/ipc_namespace.go | 2 +- pkg/sentry/kernel/kdefs/kdefs.go | 2 +- pkg/sentry/kernel/kernel.go | 2 +- pkg/sentry/kernel/kernel_state.go | 2 +- pkg/sentry/kernel/memevent/memory_events.go | 2 +- pkg/sentry/kernel/memevent/memory_events.proto | 2 +- pkg/sentry/kernel/pending_signals.go | 2 +- pkg/sentry/kernel/pending_signals_state.go | 2 +- pkg/sentry/kernel/pipe/buffers.go | 2 +- pkg/sentry/kernel/pipe/device.go | 2 +- pkg/sentry/kernel/pipe/node.go | 2 +- pkg/sentry/kernel/pipe/node_test.go | 2 +- pkg/sentry/kernel/pipe/pipe.go | 2 +- pkg/sentry/kernel/pipe/pipe_test.go | 2 +- pkg/sentry/kernel/pipe/reader.go | 2 +- pkg/sentry/kernel/pipe/reader_writer.go | 2 +- pkg/sentry/kernel/pipe/writer.go | 2 +- pkg/sentry/kernel/posixtimer.go | 2 +- pkg/sentry/kernel/ptrace.go | 2 +- pkg/sentry/kernel/rseq.go | 2 +- pkg/sentry/kernel/sched/cpuset.go | 2 +- pkg/sentry/kernel/sched/cpuset_test.go | 2 +- pkg/sentry/kernel/sched/sched.go | 2 +- pkg/sentry/kernel/seccomp.go | 2 +- pkg/sentry/kernel/semaphore/semaphore.go | 2 +- pkg/sentry/kernel/semaphore/semaphore_test.go | 2 +- pkg/sentry/kernel/sessions.go | 2 +- pkg/sentry/kernel/shm/device.go | 2 +- pkg/sentry/kernel/shm/shm.go | 2 +- pkg/sentry/kernel/signal.go | 2 +- pkg/sentry/kernel/signal_handlers.go | 2 +- pkg/sentry/kernel/syscalls.go | 2 +- pkg/sentry/kernel/syscalls_state.go | 2 +- pkg/sentry/kernel/syslog.go | 2 +- pkg/sentry/kernel/table_test.go | 2 +- pkg/sentry/kernel/task.go | 2 +- pkg/sentry/kernel/task_acct.go | 2 +- pkg/sentry/kernel/task_block.go | 2 +- pkg/sentry/kernel/task_clone.go | 2 +- pkg/sentry/kernel/task_context.go | 2 +- pkg/sentry/kernel/task_exec.go | 2 +- pkg/sentry/kernel/task_exit.go | 2 +- pkg/sentry/kernel/task_futex.go | 2 +- pkg/sentry/kernel/task_identity.go | 2 +- pkg/sentry/kernel/task_log.go | 2 +- pkg/sentry/kernel/task_net.go | 2 +- pkg/sentry/kernel/task_run.go | 2 +- pkg/sentry/kernel/task_sched.go | 2 +- pkg/sentry/kernel/task_signals.go | 2 +- pkg/sentry/kernel/task_start.go | 2 +- pkg/sentry/kernel/task_stop.go | 2 +- pkg/sentry/kernel/task_syscall.go | 2 +- pkg/sentry/kernel/task_test.go | 2 +- pkg/sentry/kernel/task_usermem.go | 2 +- pkg/sentry/kernel/thread_group.go | 2 +- pkg/sentry/kernel/threads.go | 2 +- pkg/sentry/kernel/time/context.go | 2 +- pkg/sentry/kernel/time/time.go | 2 +- pkg/sentry/kernel/timekeeper.go | 2 +- pkg/sentry/kernel/timekeeper_state.go | 2 +- pkg/sentry/kernel/timekeeper_test.go | 2 +- pkg/sentry/kernel/uts_namespace.go | 2 +- pkg/sentry/kernel/vdso.go | 2 +- pkg/sentry/kernel/version.go | 2 +- pkg/sentry/limits/context.go | 2 +- pkg/sentry/limits/limits.go | 2 +- pkg/sentry/limits/limits_test.go | 2 +- pkg/sentry/limits/linux.go | 2 +- pkg/sentry/loader/elf.go | 2 +- pkg/sentry/loader/interpreter.go | 2 +- pkg/sentry/loader/loader.go | 2 +- pkg/sentry/loader/vdso.go | 2 +- pkg/sentry/loader/vdso_state.go | 2 +- pkg/sentry/memmap/mapping_set.go | 2 +- pkg/sentry/memmap/mapping_set_test.go | 2 +- pkg/sentry/memmap/memmap.go | 2 +- pkg/sentry/memutil/memutil.go | 2 +- pkg/sentry/memutil/memutil_unsafe.go | 2 +- pkg/sentry/mm/address_space.go | 2 +- pkg/sentry/mm/aio_context.go | 2 +- pkg/sentry/mm/aio_context_state.go | 2 +- pkg/sentry/mm/debug.go | 2 +- pkg/sentry/mm/io.go | 2 +- pkg/sentry/mm/lifecycle.go | 2 +- pkg/sentry/mm/metadata.go | 2 +- pkg/sentry/mm/mm.go | 2 +- pkg/sentry/mm/mm_test.go | 2 +- pkg/sentry/mm/pma.go | 2 +- pkg/sentry/mm/proc_pid_maps.go | 2 +- pkg/sentry/mm/save_restore.go | 2 +- pkg/sentry/mm/shm.go | 2 +- pkg/sentry/mm/special_mappable.go | 2 +- pkg/sentry/mm/syscalls.go | 2 +- pkg/sentry/mm/vma.go | 2 +- pkg/sentry/platform/context.go | 2 +- pkg/sentry/platform/filemem/filemem.go | 2 +- pkg/sentry/platform/filemem/filemem_state.go | 2 +- pkg/sentry/platform/filemem/filemem_test.go | 2 +- pkg/sentry/platform/filemem/filemem_unsafe.go | 2 +- pkg/sentry/platform/interrupt/interrupt.go | 2 +- pkg/sentry/platform/interrupt/interrupt_test.go | 2 +- pkg/sentry/platform/kvm/address_space.go | 2 +- pkg/sentry/platform/kvm/allocator.go | 2 +- pkg/sentry/platform/kvm/bluepill.go | 2 +- pkg/sentry/platform/kvm/bluepill_amd64.go | 2 +- pkg/sentry/platform/kvm/bluepill_amd64.s | 2 +- pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/bluepill_fault.go | 2 +- pkg/sentry/platform/kvm/bluepill_unsafe.go | 2 +- pkg/sentry/platform/kvm/context.go | 2 +- pkg/sentry/platform/kvm/host_map.go | 2 +- pkg/sentry/platform/kvm/kvm.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/kvm_const.go | 2 +- pkg/sentry/platform/kvm/kvm_test.go | 2 +- pkg/sentry/platform/kvm/machine.go | 2 +- pkg/sentry/platform/kvm/machine_amd64.go | 2 +- pkg/sentry/platform/kvm/machine_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/machine_unsafe.go | 2 +- pkg/sentry/platform/kvm/physical_map.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil_amd64.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil_amd64.s | 2 +- pkg/sentry/platform/kvm/virtual_map.go | 2 +- pkg/sentry/platform/kvm/virtual_map_test.go | 2 +- pkg/sentry/platform/mmap_min_addr.go | 2 +- pkg/sentry/platform/platform.go | 2 +- pkg/sentry/platform/procid/procid.go | 2 +- pkg/sentry/platform/procid/procid_amd64.s | 2 +- pkg/sentry/platform/procid/procid_net_test.go | 2 +- pkg/sentry/platform/procid/procid_test.go | 2 +- pkg/sentry/platform/ptrace/ptrace.go | 2 +- pkg/sentry/platform/ptrace/ptrace_unsafe.go | 2 +- pkg/sentry/platform/ptrace/stub_amd64.s | 2 +- pkg/sentry/platform/ptrace/stub_unsafe.go | 2 +- pkg/sentry/platform/ptrace/subprocess.go | 2 +- pkg/sentry/platform/ptrace/subprocess_amd64.go | 2 +- pkg/sentry/platform/ptrace/subprocess_linux.go | 2 +- pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go | 2 +- pkg/sentry/platform/ptrace/subprocess_unsafe.go | 2 +- pkg/sentry/platform/ring0/defs.go | 2 +- pkg/sentry/platform/ring0/defs_amd64.go | 2 +- pkg/sentry/platform/ring0/entry_amd64.go | 2 +- pkg/sentry/platform/ring0/entry_amd64.s | 2 +- pkg/sentry/platform/ring0/gen_offsets/main.go | 2 +- pkg/sentry/platform/ring0/kernel.go | 2 +- pkg/sentry/platform/ring0/kernel_amd64.go | 2 +- pkg/sentry/platform/ring0/kernel_unsafe.go | 2 +- pkg/sentry/platform/ring0/lib_amd64.go | 2 +- pkg/sentry/platform/ring0/lib_amd64.s | 2 +- pkg/sentry/platform/ring0/offsets_amd64.go | 2 +- pkg/sentry/platform/ring0/pagetables/allocator.go | 2 +- pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_test.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_x86.go | 2 +- pkg/sentry/platform/ring0/pagetables/pcids_x86.go | 2 +- pkg/sentry/platform/ring0/pagetables/walker_amd64.go | 2 +- pkg/sentry/platform/ring0/ring0.go | 2 +- pkg/sentry/platform/ring0/x86.go | 2 +- pkg/sentry/platform/safecopy/atomic_amd64.s | 2 +- pkg/sentry/platform/safecopy/memclr_amd64.s | 2 +- pkg/sentry/platform/safecopy/memcpy_amd64.s | 2 +- pkg/sentry/platform/safecopy/safecopy.go | 2 +- pkg/sentry/platform/safecopy/safecopy_test.go | 2 +- pkg/sentry/platform/safecopy/safecopy_unsafe.go | 2 +- pkg/sentry/platform/safecopy/sighandler_amd64.s | 2 +- pkg/sentry/safemem/block_unsafe.go | 2 +- pkg/sentry/safemem/io.go | 2 +- pkg/sentry/safemem/io_test.go | 2 +- pkg/sentry/safemem/safemem.go | 2 +- pkg/sentry/safemem/seq_test.go | 2 +- pkg/sentry/safemem/seq_unsafe.go | 2 +- pkg/sentry/sighandling/sighandling.go | 2 +- pkg/sentry/sighandling/sighandling_unsafe.go | 2 +- pkg/sentry/socket/control/control.go | 2 +- pkg/sentry/socket/epsocket/device.go | 2 +- pkg/sentry/socket/epsocket/epsocket.go | 2 +- pkg/sentry/socket/epsocket/provider.go | 2 +- pkg/sentry/socket/epsocket/save_restore.go | 2 +- pkg/sentry/socket/epsocket/stack.go | 2 +- pkg/sentry/socket/hostinet/device.go | 2 +- pkg/sentry/socket/hostinet/hostinet.go | 2 +- pkg/sentry/socket/hostinet/save_restore.go | 2 +- pkg/sentry/socket/hostinet/socket.go | 2 +- pkg/sentry/socket/hostinet/socket_unsafe.go | 2 +- pkg/sentry/socket/hostinet/stack.go | 2 +- pkg/sentry/socket/netlink/message.go | 2 +- pkg/sentry/socket/netlink/port/port.go | 2 +- pkg/sentry/socket/netlink/port/port_test.go | 2 +- pkg/sentry/socket/netlink/provider.go | 2 +- pkg/sentry/socket/netlink/route/protocol.go | 2 +- pkg/sentry/socket/netlink/socket.go | 2 +- pkg/sentry/socket/rpcinet/conn/conn.go | 2 +- pkg/sentry/socket/rpcinet/device.go | 2 +- pkg/sentry/socket/rpcinet/notifier/notifier.go | 2 +- pkg/sentry/socket/rpcinet/rpcinet.go | 2 +- pkg/sentry/socket/rpcinet/socket.go | 2 +- pkg/sentry/socket/rpcinet/stack.go | 2 +- pkg/sentry/socket/rpcinet/stack_unsafe.go | 2 +- pkg/sentry/socket/socket.go | 2 +- pkg/sentry/socket/unix/device.go | 2 +- pkg/sentry/socket/unix/io.go | 2 +- pkg/sentry/socket/unix/transport/connectioned.go | 2 +- pkg/sentry/socket/unix/transport/connectioned_state.go | 2 +- pkg/sentry/socket/unix/transport/connectionless.go | 2 +- pkg/sentry/socket/unix/transport/queue.go | 2 +- pkg/sentry/socket/unix/transport/unix.go | 2 +- pkg/sentry/socket/unix/unix.go | 2 +- pkg/sentry/state/state.go | 2 +- pkg/sentry/state/state_metadata.go | 2 +- pkg/sentry/state/state_unsafe.go | 2 +- pkg/sentry/strace/clone.go | 2 +- pkg/sentry/strace/futex.go | 2 +- pkg/sentry/strace/linux64.go | 2 +- pkg/sentry/strace/open.go | 2 +- pkg/sentry/strace/ptrace.go | 2 +- pkg/sentry/strace/socket.go | 2 +- pkg/sentry/strace/strace.go | 2 +- pkg/sentry/strace/strace.proto | 2 +- pkg/sentry/strace/syscalls.go | 2 +- pkg/sentry/syscalls/epoll.go | 2 +- pkg/sentry/syscalls/linux/error.go | 2 +- pkg/sentry/syscalls/linux/flags.go | 2 +- pkg/sentry/syscalls/linux/linux64.go | 2 +- pkg/sentry/syscalls/linux/sigset.go | 2 +- pkg/sentry/syscalls/linux/sys_aio.go | 2 +- pkg/sentry/syscalls/linux/sys_capability.go | 2 +- pkg/sentry/syscalls/linux/sys_epoll.go | 2 +- pkg/sentry/syscalls/linux/sys_eventfd.go | 2 +- pkg/sentry/syscalls/linux/sys_file.go | 2 +- pkg/sentry/syscalls/linux/sys_futex.go | 2 +- pkg/sentry/syscalls/linux/sys_getdents.go | 2 +- pkg/sentry/syscalls/linux/sys_identity.go | 2 +- pkg/sentry/syscalls/linux/sys_inotify.go | 2 +- pkg/sentry/syscalls/linux/sys_lseek.go | 2 +- pkg/sentry/syscalls/linux/sys_mmap.go | 2 +- pkg/sentry/syscalls/linux/sys_mount.go | 2 +- pkg/sentry/syscalls/linux/sys_pipe.go | 2 +- pkg/sentry/syscalls/linux/sys_poll.go | 2 +- pkg/sentry/syscalls/linux/sys_prctl.go | 2 +- pkg/sentry/syscalls/linux/sys_random.go | 2 +- pkg/sentry/syscalls/linux/sys_read.go | 2 +- pkg/sentry/syscalls/linux/sys_rlimit.go | 2 +- pkg/sentry/syscalls/linux/sys_rusage.go | 2 +- pkg/sentry/syscalls/linux/sys_sched.go | 2 +- pkg/sentry/syscalls/linux/sys_seccomp.go | 2 +- pkg/sentry/syscalls/linux/sys_sem.go | 2 +- pkg/sentry/syscalls/linux/sys_shm.go | 2 +- pkg/sentry/syscalls/linux/sys_signal.go | 2 +- pkg/sentry/syscalls/linux/sys_socket.go | 2 +- pkg/sentry/syscalls/linux/sys_stat.go | 2 +- pkg/sentry/syscalls/linux/sys_sync.go | 2 +- pkg/sentry/syscalls/linux/sys_sysinfo.go | 2 +- pkg/sentry/syscalls/linux/sys_syslog.go | 2 +- pkg/sentry/syscalls/linux/sys_thread.go | 2 +- pkg/sentry/syscalls/linux/sys_time.go | 2 +- pkg/sentry/syscalls/linux/sys_timer.go | 2 +- pkg/sentry/syscalls/linux/sys_timerfd.go | 2 +- pkg/sentry/syscalls/linux/sys_tls.go | 2 +- pkg/sentry/syscalls/linux/sys_utsname.go | 2 +- pkg/sentry/syscalls/linux/sys_write.go | 2 +- pkg/sentry/syscalls/linux/timespec.go | 2 +- pkg/sentry/syscalls/polling.go | 2 +- pkg/sentry/syscalls/syscalls.go | 2 +- pkg/sentry/syscalls/unimplemented_syscall.proto | 2 +- pkg/sentry/time/calibrated_clock.go | 2 +- pkg/sentry/time/calibrated_clock_test.go | 2 +- pkg/sentry/time/clock_id.go | 2 +- pkg/sentry/time/clocks.go | 2 +- pkg/sentry/time/muldiv_amd64.s | 2 +- pkg/sentry/time/parameters.go | 2 +- pkg/sentry/time/parameters_test.go | 2 +- pkg/sentry/time/sampler.go | 2 +- pkg/sentry/time/sampler_test.go | 2 +- pkg/sentry/time/sampler_unsafe.go | 2 +- pkg/sentry/time/tsc_amd64.s | 2 +- pkg/sentry/uniqueid/context.go | 2 +- pkg/sentry/usage/cpu.go | 2 +- pkg/sentry/usage/io.go | 2 +- pkg/sentry/usage/memory.go | 2 +- pkg/sentry/usage/memory_unsafe.go | 2 +- pkg/sentry/usage/usage.go | 2 +- pkg/sentry/usermem/access_type.go | 2 +- pkg/sentry/usermem/addr.go | 2 +- pkg/sentry/usermem/addr_range_seq_test.go | 2 +- pkg/sentry/usermem/addr_range_seq_unsafe.go | 2 +- pkg/sentry/usermem/bytes_io.go | 2 +- pkg/sentry/usermem/bytes_io_unsafe.go | 2 +- pkg/sentry/usermem/usermem.go | 2 +- pkg/sentry/usermem/usermem_test.go | 2 +- pkg/sentry/usermem/usermem_x86.go | 2 +- pkg/sentry/watchdog/watchdog.go | 2 +- pkg/sleep/commit_amd64.s | 2 +- pkg/sleep/commit_asm.go | 2 +- pkg/sleep/commit_noasm.go | 2 +- pkg/sleep/empty.s | 2 +- pkg/sleep/sleep_test.go | 2 +- pkg/sleep/sleep_unsafe.go | 2 +- pkg/state/decode.go | 2 +- pkg/state/encode.go | 2 +- pkg/state/encode_unsafe.go | 2 +- pkg/state/map.go | 2 +- pkg/state/object.proto | 2 +- pkg/state/printer.go | 2 +- pkg/state/state.go | 2 +- pkg/state/state_test.go | 2 +- pkg/state/statefile/statefile.go | 2 +- pkg/state/statefile/statefile_test.go | 2 +- pkg/state/stats.go | 2 +- pkg/sync/atomicptr_unsafe.go | 2 +- pkg/sync/atomicptrtest/atomicptr_test.go | 2 +- pkg/sync/memmove_unsafe.go | 2 +- pkg/sync/norace_unsafe.go | 2 +- pkg/sync/race_unsafe.go | 2 +- pkg/sync/seqatomic_unsafe.go | 2 +- pkg/sync/seqatomictest/seqatomic_test.go | 2 +- pkg/sync/seqcount.go | 2 +- pkg/sync/seqcount_test.go | 2 +- pkg/sync/sync.go | 2 +- pkg/syserr/host_linux.go | 2 +- pkg/syserr/netstack.go | 2 +- pkg/syserr/syserr.go | 2 +- pkg/syserror/syserror.go | 2 +- pkg/syserror/syserror_test.go | 2 +- pkg/tcpip/adapters/gonet/gonet.go | 2 +- pkg/tcpip/adapters/gonet/gonet_test.go | 2 +- pkg/tcpip/buffer/prependable.go | 2 +- pkg/tcpip/buffer/view.go | 2 +- pkg/tcpip/buffer/view_test.go | 2 +- pkg/tcpip/checker/checker.go | 2 +- pkg/tcpip/header/arp.go | 2 +- pkg/tcpip/header/checksum.go | 2 +- pkg/tcpip/header/eth.go | 2 +- pkg/tcpip/header/gue.go | 2 +- pkg/tcpip/header/icmpv4.go | 2 +- pkg/tcpip/header/icmpv6.go | 2 +- pkg/tcpip/header/interfaces.go | 2 +- pkg/tcpip/header/ipv4.go | 2 +- pkg/tcpip/header/ipv6.go | 2 +- pkg/tcpip/header/ipv6_fragment.go | 2 +- pkg/tcpip/header/ipversion_test.go | 2 +- pkg/tcpip/header/tcp.go | 2 +- pkg/tcpip/header/tcp_test.go | 2 +- pkg/tcpip/header/udp.go | 2 +- pkg/tcpip/link/channel/channel.go | 2 +- pkg/tcpip/link/fdbased/endpoint.go | 2 +- pkg/tcpip/link/fdbased/endpoint_test.go | 2 +- pkg/tcpip/link/loopback/loopback.go | 2 +- pkg/tcpip/link/rawfile/blockingpoll_amd64.s | 2 +- pkg/tcpip/link/rawfile/blockingpoll_unsafe.go | 2 +- pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go | 2 +- pkg/tcpip/link/rawfile/errors.go | 2 +- pkg/tcpip/link/rawfile/rawfile_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe_test.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/rx.go | 2 +- pkg/tcpip/link/sharedmem/pipe/tx.go | 2 +- pkg/tcpip/link/sharedmem/queue/queue_test.go | 2 +- pkg/tcpip/link/sharedmem/queue/rx.go | 2 +- pkg/tcpip/link/sharedmem/queue/tx.go | 2 +- pkg/tcpip/link/sharedmem/rx.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem_test.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/tx.go | 2 +- pkg/tcpip/link/sniffer/pcap.go | 2 +- pkg/tcpip/link/sniffer/sniffer.go | 2 +- pkg/tcpip/link/tun/tun_unsafe.go | 2 +- pkg/tcpip/link/waitable/waitable.go | 2 +- pkg/tcpip/link/waitable/waitable_test.go | 2 +- pkg/tcpip/network/arp/arp.go | 2 +- pkg/tcpip/network/arp/arp_test.go | 2 +- pkg/tcpip/network/fragmentation/frag_heap.go | 2 +- pkg/tcpip/network/fragmentation/frag_heap_test.go | 2 +- pkg/tcpip/network/fragmentation/fragmentation.go | 2 +- pkg/tcpip/network/fragmentation/fragmentation_test.go | 2 +- pkg/tcpip/network/fragmentation/reassembler.go | 2 +- pkg/tcpip/network/fragmentation/reassembler_test.go | 2 +- pkg/tcpip/network/hash/hash.go | 2 +- pkg/tcpip/network/ip_test.go | 2 +- pkg/tcpip/network/ipv4/icmp.go | 2 +- pkg/tcpip/network/ipv4/ipv4.go | 2 +- pkg/tcpip/network/ipv4/ipv4_test.go | 2 +- pkg/tcpip/network/ipv6/icmp.go | 2 +- pkg/tcpip/network/ipv6/icmp_test.go | 2 +- pkg/tcpip/network/ipv6/ipv6.go | 2 +- pkg/tcpip/ports/ports.go | 2 +- pkg/tcpip/ports/ports_test.go | 2 +- pkg/tcpip/sample/tun_tcp_connect/main.go | 2 +- pkg/tcpip/sample/tun_tcp_echo/main.go | 2 +- pkg/tcpip/seqnum/seqnum.go | 2 +- pkg/tcpip/stack/linkaddrcache.go | 2 +- pkg/tcpip/stack/linkaddrcache_test.go | 2 +- pkg/tcpip/stack/nic.go | 2 +- pkg/tcpip/stack/registration.go | 2 +- pkg/tcpip/stack/route.go | 2 +- pkg/tcpip/stack/stack.go | 2 +- pkg/tcpip/stack/stack_global_state.go | 2 +- pkg/tcpip/stack/stack_test.go | 2 +- pkg/tcpip/stack/transport_demuxer.go | 2 +- pkg/tcpip/stack/transport_test.go | 2 +- pkg/tcpip/tcpip.go | 2 +- pkg/tcpip/tcpip_test.go | 2 +- pkg/tcpip/time.s | 2 +- pkg/tcpip/time_unsafe.go | 2 +- pkg/tcpip/transport/ping/endpoint.go | 2 +- pkg/tcpip/transport/ping/endpoint_state.go | 2 +- pkg/tcpip/transport/ping/protocol.go | 2 +- pkg/tcpip/transport/tcp/accept.go | 2 +- pkg/tcpip/transport/tcp/connect.go | 2 +- pkg/tcpip/transport/tcp/cubic.go | 2 +- pkg/tcpip/transport/tcp/dual_stack_test.go | 2 +- pkg/tcpip/transport/tcp/endpoint.go | 2 +- pkg/tcpip/transport/tcp/endpoint_state.go | 2 +- pkg/tcpip/transport/tcp/forwarder.go | 2 +- pkg/tcpip/transport/tcp/protocol.go | 2 +- pkg/tcpip/transport/tcp/rcv.go | 2 +- pkg/tcpip/transport/tcp/reno.go | 2 +- pkg/tcpip/transport/tcp/sack.go | 2 +- pkg/tcpip/transport/tcp/segment.go | 2 +- pkg/tcpip/transport/tcp/segment_heap.go | 2 +- pkg/tcpip/transport/tcp/segment_queue.go | 2 +- pkg/tcpip/transport/tcp/segment_state.go | 2 +- pkg/tcpip/transport/tcp/snd.go | 2 +- pkg/tcpip/transport/tcp/snd_state.go | 2 +- pkg/tcpip/transport/tcp/tcp_sack_test.go | 2 +- pkg/tcpip/transport/tcp/tcp_test.go | 2 +- pkg/tcpip/transport/tcp/tcp_timestamp_test.go | 2 +- pkg/tcpip/transport/tcp/testing/context/context.go | 2 +- pkg/tcpip/transport/tcp/timer.go | 2 +- pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go | 2 +- pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go | 2 +- pkg/tcpip/transport/udp/endpoint.go | 2 +- pkg/tcpip/transport/udp/endpoint_state.go | 2 +- pkg/tcpip/transport/udp/protocol.go | 2 +- pkg/tcpip/transport/udp/udp_test.go | 2 +- pkg/tmutex/tmutex.go | 2 +- pkg/tmutex/tmutex_test.go | 2 +- pkg/unet/unet.go | 2 +- pkg/unet/unet_test.go | 2 +- pkg/unet/unet_unsafe.go | 2 +- pkg/urpc/urpc.go | 2 +- pkg/urpc/urpc_test.go | 2 +- pkg/waiter/fdnotifier/fdnotifier.go | 2 +- pkg/waiter/fdnotifier/poll_unsafe.go | 2 +- pkg/waiter/waiter.go | 2 +- pkg/waiter/waiter_test.go | 2 +- runsc/boot/compat.go | 2 +- runsc/boot/config.go | 2 +- runsc/boot/controller.go | 2 +- runsc/boot/debug.go | 2 +- runsc/boot/events.go | 2 +- runsc/boot/fds.go | 2 +- runsc/boot/filter/config.go | 2 +- runsc/boot/filter/extra_filters.go | 2 +- runsc/boot/filter/extra_filters_msan.go | 2 +- runsc/boot/filter/extra_filters_race.go | 2 +- runsc/boot/filter/filter.go | 2 +- runsc/boot/fs.go | 2 +- runsc/boot/limits.go | 2 +- runsc/boot/loader.go | 2 +- runsc/boot/loader_test.go | 2 +- runsc/boot/network.go | 2 +- runsc/boot/strace.go | 2 +- runsc/cgroup/cgroup.go | 2 +- runsc/cgroup/cgroup_test.go | 2 +- runsc/cmd/boot.go | 2 +- runsc/cmd/capability.go | 2 +- runsc/cmd/capability_test.go | 2 +- runsc/cmd/checkpoint.go | 2 +- runsc/cmd/cmd.go | 2 +- runsc/cmd/create.go | 2 +- runsc/cmd/debug.go | 2 +- runsc/cmd/delete.go | 2 +- runsc/cmd/delete_test.go | 2 +- runsc/cmd/events.go | 2 +- runsc/cmd/exec.go | 2 +- runsc/cmd/exec_test.go | 2 +- runsc/cmd/gofer.go | 2 +- runsc/cmd/kill.go | 2 +- runsc/cmd/list.go | 2 +- runsc/cmd/path.go | 2 +- runsc/cmd/pause.go | 2 +- runsc/cmd/ps.go | 2 +- runsc/cmd/restore.go | 2 +- runsc/cmd/resume.go | 2 +- runsc/cmd/run.go | 2 +- runsc/cmd/spec.go | 2 +- runsc/cmd/start.go | 2 +- runsc/cmd/state.go | 2 +- runsc/cmd/wait.go | 2 +- runsc/console/console.go | 2 +- runsc/container/console_test.go | 2 +- runsc/container/container.go | 2 +- runsc/container/container_test.go | 2 +- runsc/container/fs.go | 2 +- runsc/container/fs_test.go | 2 +- runsc/container/hook.go | 2 +- runsc/container/multi_container_test.go | 2 +- runsc/container/status.go | 2 +- runsc/container/test_app.go | 2 +- runsc/fsgofer/filter/config.go | 2 +- runsc/fsgofer/filter/extra_filters.go | 2 +- runsc/fsgofer/filter/extra_filters_msan.go | 2 +- runsc/fsgofer/filter/extra_filters_race.go | 2 +- runsc/fsgofer/filter/filter.go | 2 +- runsc/fsgofer/fsgofer.go | 2 +- runsc/fsgofer/fsgofer_test.go | 2 +- runsc/fsgofer/fsgofer_unsafe.go | 2 +- runsc/main.go | 2 +- runsc/sandbox/chroot.go | 2 +- runsc/sandbox/network.go | 2 +- runsc/sandbox/sandbox.go | 2 +- runsc/specutils/namespace.go | 2 +- runsc/specutils/specutils.go | 2 +- runsc/specutils/specutils_test.go | 2 +- runsc/test/image/image.go | 2 +- runsc/test/image/image_test.go | 2 +- runsc/test/image/mysql.sql | 2 +- runsc/test/image/ruby.rb | 2 +- runsc/test/image/ruby.sh | 2 +- runsc/test/install.sh | 2 +- runsc/test/integration/exec_test.go | 2 +- runsc/test/integration/integration.go | 2 +- runsc/test/integration/integration_test.go | 2 +- runsc/test/root/cgroup_test.go | 2 +- runsc/test/root/chroot_test.go | 2 +- runsc/test/root/root.go | 2 +- runsc/test/testutil/docker.go | 2 +- runsc/test/testutil/testutil.go | 2 +- runsc/test/testutil/testutil_race.go | 2 +- runsc/tools/dockercfg/dockercfg.go | 2 +- tools/go_generics/generics.go | 2 +- tools/go_generics/generics_tests/all_stmts/input.go | 2 +- tools/go_generics/generics_tests/all_stmts/output/output.go | 2 +- tools/go_generics/generics_tests/all_types/input.go | 2 +- tools/go_generics/generics_tests/all_types/lib/lib.go | 2 +- tools/go_generics/generics_tests/all_types/output/output.go | 2 +- tools/go_generics/generics_tests/consts/input.go | 2 +- tools/go_generics/generics_tests/consts/output/output.go | 2 +- tools/go_generics/generics_tests/imports/input.go | 2 +- tools/go_generics/generics_tests/imports/output/output.go | 2 +- tools/go_generics/generics_tests/remove_typedef/input.go | 2 +- tools/go_generics/generics_tests/remove_typedef/output/output.go | 2 +- tools/go_generics/generics_tests/simple/input.go | 2 +- tools/go_generics/generics_tests/simple/output/output.go | 2 +- tools/go_generics/globals/globals_visitor.go | 2 +- tools/go_generics/globals/scope.go | 2 +- tools/go_generics/go_generics_unittest.sh | 2 +- tools/go_generics/imports.go | 2 +- tools/go_generics/merge.go | 2 +- tools/go_generics/remove.go | 2 +- tools/go_generics/rules_tests/template.go | 2 +- tools/go_generics/rules_tests/template_test.go | 2 +- tools/go_stateify/main.go | 2 +- tools/workspace_status.sh | 2 +- vdso/barrier.h | 2 +- vdso/check_vdso.py | 2 +- vdso/compiler.h | 2 +- vdso/cycle_clock.h | 2 +- vdso/seqlock.h | 2 +- vdso/syscalls.h | 2 +- vdso/vdso.cc | 2 +- vdso/vdso_time.cc | 2 +- vdso/vdso_time.h | 2 +- 923 files changed, 923 insertions(+), 923 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/kokoro/run_build.sh b/kokoro/run_build.sh index f2b719f52..89e24b037 100755 --- a/kokoro/run_build.sh +++ b/kokoro/run_build.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/kokoro/run_tests.sh b/kokoro/run_tests.sh index 3f8841cee..0a0d73d29 100755 --- a/kokoro/run_tests.sh +++ b/kokoro/run_tests.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/pkg/abi/abi.go b/pkg/abi/abi.go index a53c2747b..7770f0405 100644 --- a/pkg/abi/abi.go +++ b/pkg/abi/abi.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/abi_linux.go b/pkg/abi/abi_linux.go index dd5d67b51..9d9f361a4 100644 --- a/pkg/abi/abi_linux.go +++ b/pkg/abi/abi_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/flag.go b/pkg/abi/flag.go index 0391ccf37..0698e410f 100644 --- a/pkg/abi/flag.go +++ b/pkg/abi/flag.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/aio.go b/pkg/abi/linux/aio.go index 9c39ca2ef..1b7ca714a 100644 --- a/pkg/abi/linux/aio.go +++ b/pkg/abi/linux/aio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ashmem.go b/pkg/abi/linux/ashmem.go index 7fbfd2e68..ced1e44d4 100644 --- a/pkg/abi/linux/ashmem.go +++ b/pkg/abi/linux/ashmem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/binder.go b/pkg/abi/linux/binder.go index b228898f9..522dc6f53 100644 --- a/pkg/abi/linux/binder.go +++ b/pkg/abi/linux/binder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/bpf.go b/pkg/abi/linux/bpf.go index 80e5b1af1..d9cd09948 100644 --- a/pkg/abi/linux/bpf.go +++ b/pkg/abi/linux/bpf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/capability.go b/pkg/abi/linux/capability.go index b470ce0a5..7d96f013e 100644 --- a/pkg/abi/linux/capability.go +++ b/pkg/abi/linux/capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/dev.go b/pkg/abi/linux/dev.go index ea5b16b7b..5b1199aac 100644 --- a/pkg/abi/linux/dev.go +++ b/pkg/abi/linux/dev.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/elf.go b/pkg/abi/linux/elf.go index 76c13b677..928067c04 100644 --- a/pkg/abi/linux/elf.go +++ b/pkg/abi/linux/elf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/errors.go b/pkg/abi/linux/errors.go index b5ddb2b2f..01e4095b8 100644 --- a/pkg/abi/linux/errors.go +++ b/pkg/abi/linux/errors.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/eventfd.go b/pkg/abi/linux/eventfd.go index bc0fb44d2..5614f5cf1 100644 --- a/pkg/abi/linux/eventfd.go +++ b/pkg/abi/linux/eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/exec.go b/pkg/abi/linux/exec.go index 4d81eca54..a07c29243 100644 --- a/pkg/abi/linux/exec.go +++ b/pkg/abi/linux/exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/fcntl.go b/pkg/abi/linux/fcntl.go index 2a5ad6ed7..c8558933a 100644 --- a/pkg/abi/linux/fcntl.go +++ b/pkg/abi/linux/fcntl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/file.go b/pkg/abi/linux/file.go index 9bf229a57..72e5c6f83 100644 --- a/pkg/abi/linux/file.go +++ b/pkg/abi/linux/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/fs.go b/pkg/abi/linux/fs.go index 32a0812b4..7817bfb52 100644 --- a/pkg/abi/linux/fs.go +++ b/pkg/abi/linux/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/futex.go b/pkg/abi/linux/futex.go index f63f5200c..5dff01fba 100644 --- a/pkg/abi/linux/futex.go +++ b/pkg/abi/linux/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/inotify.go b/pkg/abi/linux/inotify.go index 072a2d146..79c5d3593 100644 --- a/pkg/abi/linux/inotify.go +++ b/pkg/abi/linux/inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ioctl.go b/pkg/abi/linux/ioctl.go index afd9ee82b..9afc3d1ef 100644 --- a/pkg/abi/linux/ioctl.go +++ b/pkg/abi/linux/ioctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ip.go b/pkg/abi/linux/ip.go index 6b68999ab..fcec16965 100644 --- a/pkg/abi/linux/ip.go +++ b/pkg/abi/linux/ip.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ipc.go b/pkg/abi/linux/ipc.go index 81e9904dd..10681768b 100644 --- a/pkg/abi/linux/ipc.go +++ b/pkg/abi/linux/ipc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/limits.go b/pkg/abi/linux/limits.go index e1f0932ec..b2e51b9bd 100644 --- a/pkg/abi/linux/limits.go +++ b/pkg/abi/linux/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/linux.go b/pkg/abi/linux/linux.go index de2af80dc..d365f693d 100644 --- a/pkg/abi/linux/linux.go +++ b/pkg/abi/linux/linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/mm.go b/pkg/abi/linux/mm.go index b48e1d18a..3fcdf8235 100644 --- a/pkg/abi/linux/mm.go +++ b/pkg/abi/linux/mm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netdevice.go b/pkg/abi/linux/netdevice.go index 88654a1b3..e3b6b1e40 100644 --- a/pkg/abi/linux/netdevice.go +++ b/pkg/abi/linux/netdevice.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netlink.go b/pkg/abi/linux/netlink.go index e823ffa7e..10ceb5bf2 100644 --- a/pkg/abi/linux/netlink.go +++ b/pkg/abi/linux/netlink.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/netlink_route.go b/pkg/abi/linux/netlink_route.go index a5d778748..4200b6506 100644 --- a/pkg/abi/linux/netlink_route.go +++ b/pkg/abi/linux/netlink_route.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/poll.go b/pkg/abi/linux/poll.go index f373cfca1..9f0b15d1c 100644 --- a/pkg/abi/linux/poll.go +++ b/pkg/abi/linux/poll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/prctl.go b/pkg/abi/linux/prctl.go index 074ec03f0..e152c4c27 100644 --- a/pkg/abi/linux/prctl.go +++ b/pkg/abi/linux/prctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/ptrace.go b/pkg/abi/linux/ptrace.go index ba48d4d6d..7db4f5464 100644 --- a/pkg/abi/linux/ptrace.go +++ b/pkg/abi/linux/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/rusage.go b/pkg/abi/linux/rusage.go index a4a89abda..7fea4b589 100644 --- a/pkg/abi/linux/rusage.go +++ b/pkg/abi/linux/rusage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/sched.go b/pkg/abi/linux/sched.go index 05fda1604..ef96a3801 100644 --- a/pkg/abi/linux/sched.go +++ b/pkg/abi/linux/sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/seccomp.go b/pkg/abi/linux/seccomp.go index a8de9d3d0..9963ceeba 100644 --- a/pkg/abi/linux/seccomp.go +++ b/pkg/abi/linux/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/sem.go b/pkg/abi/linux/sem.go index 3495f5cd0..d1a0bdb32 100644 --- a/pkg/abi/linux/sem.go +++ b/pkg/abi/linux/sem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/shm.go b/pkg/abi/linux/shm.go index f50b3c2e2..82a80e609 100644 --- a/pkg/abi/linux/shm.go +++ b/pkg/abi/linux/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/signal.go b/pkg/abi/linux/signal.go index b2c7230c4..bf9bce6ed 100644 --- a/pkg/abi/linux/signal.go +++ b/pkg/abi/linux/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/socket.go b/pkg/abi/linux/socket.go index 19b5fa212..af0761a3b 100644 --- a/pkg/abi/linux/socket.go +++ b/pkg/abi/linux/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/time.go b/pkg/abi/linux/time.go index 4569f4208..bbd21e726 100644 --- a/pkg/abi/linux/time.go +++ b/pkg/abi/linux/time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/timer.go b/pkg/abi/linux/timer.go index 6c4675c35..a6f420bdb 100644 --- a/pkg/abi/linux/timer.go +++ b/pkg/abi/linux/timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/tty.go b/pkg/abi/linux/tty.go index f63dc52aa..e6f7c5b2a 100644 --- a/pkg/abi/linux/tty.go +++ b/pkg/abi/linux/tty.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/uio.go b/pkg/abi/linux/uio.go index 93c972774..7e00d9959 100644 --- a/pkg/abi/linux/uio.go +++ b/pkg/abi/linux/uio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/abi/linux/utsname.go b/pkg/abi/linux/utsname.go index 7d33d20de..f80ed7d4a 100644 --- a/pkg/abi/linux/utsname.go +++ b/pkg/abi/linux/utsname.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/amutex/amutex.go b/pkg/amutex/amutex.go index 1cb73359a..26b674435 100644 --- a/pkg/amutex/amutex.go +++ b/pkg/amutex/amutex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/amutex/amutex_test.go b/pkg/amutex/amutex_test.go index 876e47b19..104e0dab1 100644 --- a/pkg/amutex/amutex_test.go +++ b/pkg/amutex/amutex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops.go b/pkg/atomicbitops/atomic_bitops.go index 6635ea0d2..9a57f9599 100644 --- a/pkg/atomicbitops/atomic_bitops.go +++ b/pkg/atomicbitops/atomic_bitops.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_amd64.s b/pkg/atomicbitops/atomic_bitops_amd64.s index 542452bec..b37e3aad3 100644 --- a/pkg/atomicbitops/atomic_bitops_amd64.s +++ b/pkg/atomicbitops/atomic_bitops_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_common.go b/pkg/atomicbitops/atomic_bitops_common.go index 542ff4e83..b03242baa 100644 --- a/pkg/atomicbitops/atomic_bitops_common.go +++ b/pkg/atomicbitops/atomic_bitops_common.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/atomicbitops/atomic_bitops_test.go b/pkg/atomicbitops/atomic_bitops_test.go index ec0c07ee2..ee6207cb3 100644 --- a/pkg/atomicbitops/atomic_bitops_test.go +++ b/pkg/atomicbitops/atomic_bitops_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/binary/binary.go b/pkg/binary/binary.go index 3b18a86ee..02f7e9fb8 100644 --- a/pkg/binary/binary.go +++ b/pkg/binary/binary.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/binary/binary_test.go b/pkg/binary/binary_test.go index 921a0369a..d8d481f32 100644 --- a/pkg/binary/binary_test.go +++ b/pkg/binary/binary_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/bits.go b/pkg/bits/bits.go index 50ca4bff7..eb3c80f49 100644 --- a/pkg/bits/bits.go +++ b/pkg/bits/bits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/bits_template.go b/pkg/bits/bits_template.go index 0a01f29c2..8c578cca2 100644 --- a/pkg/bits/bits_template.go +++ b/pkg/bits/bits_template.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_amd64.go b/pkg/bits/uint64_arch_amd64.go index 068597f68..1fef89394 100644 --- a/pkg/bits/uint64_arch_amd64.go +++ b/pkg/bits/uint64_arch_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_amd64_asm.s b/pkg/bits/uint64_arch_amd64_asm.s index 33885641a..8c7322f0f 100644 --- a/pkg/bits/uint64_arch_amd64_asm.s +++ b/pkg/bits/uint64_arch_amd64_asm.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_arch_generic.go b/pkg/bits/uint64_arch_generic.go index 862033a4b..cfb47400b 100644 --- a/pkg/bits/uint64_arch_generic.go +++ b/pkg/bits/uint64_arch_generic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bits/uint64_test.go b/pkg/bits/uint64_test.go index 906017e1a..d6dbaf602 100644 --- a/pkg/bits/uint64_test.go +++ b/pkg/bits/uint64_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/bpf.go b/pkg/bpf/bpf.go index 757744090..98d44d911 100644 --- a/pkg/bpf/bpf.go +++ b/pkg/bpf/bpf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/decoder.go b/pkg/bpf/decoder.go index ef41e9edc..ae6b8839a 100644 --- a/pkg/bpf/decoder.go +++ b/pkg/bpf/decoder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/decoder_test.go b/pkg/bpf/decoder_test.go index 18709b944..f093e1e41 100644 --- a/pkg/bpf/decoder_test.go +++ b/pkg/bpf/decoder_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/input_bytes.go b/pkg/bpf/input_bytes.go index 74af038eb..745c0749b 100644 --- a/pkg/bpf/input_bytes.go +++ b/pkg/bpf/input_bytes.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/interpreter.go b/pkg/bpf/interpreter.go index 111ada9d1..86c7add4d 100644 --- a/pkg/bpf/interpreter.go +++ b/pkg/bpf/interpreter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/interpreter_test.go b/pkg/bpf/interpreter_test.go index 9e5e33228..c46a43991 100644 --- a/pkg/bpf/interpreter_test.go +++ b/pkg/bpf/interpreter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/program_builder.go b/pkg/bpf/program_builder.go index bad56d7ac..b4ce228e1 100644 --- a/pkg/bpf/program_builder.go +++ b/pkg/bpf/program_builder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/bpf/program_builder_test.go b/pkg/bpf/program_builder_test.go index 7e4f06584..0e0b79d88 100644 --- a/pkg/bpf/program_builder_test.go +++ b/pkg/bpf/program_builder_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/compressio/compressio.go b/pkg/compressio/compressio.go index 667f17c5c..205536812 100644 --- a/pkg/compressio/compressio.go +++ b/pkg/compressio/compressio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/compressio/compressio_test.go b/pkg/compressio/compressio_test.go index 7cb5f8dc4..1bbabee79 100644 --- a/pkg/compressio/compressio_test.go +++ b/pkg/compressio/compressio_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/control/client/client.go b/pkg/control/client/client.go index f7c2e8776..0d0c9f148 100644 --- a/pkg/control/client/client.go +++ b/pkg/control/client/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/control/server/server.go b/pkg/control/server/server.go index d00061ce3..c46b5d70b 100644 --- a/pkg/control/server/server.go +++ b/pkg/control/server/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpu_amd64.s b/pkg/cpuid/cpu_amd64.s index 48a13c6fd..905c1d12e 100644 --- a/pkg/cpuid/cpu_amd64.s +++ b/pkg/cpuid/cpu_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid.go b/pkg/cpuid/cpuid.go index e91e34dc7..5b083a5fb 100644 --- a/pkg/cpuid/cpuid.go +++ b/pkg/cpuid/cpuid.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid_parse_test.go b/pkg/cpuid/cpuid_parse_test.go index c4f52818c..81b06f48c 100644 --- a/pkg/cpuid/cpuid_parse_test.go +++ b/pkg/cpuid/cpuid_parse_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/cpuid/cpuid_test.go b/pkg/cpuid/cpuid_test.go index 02f732f85..0decd8f08 100644 --- a/pkg/cpuid/cpuid_test.go +++ b/pkg/cpuid/cpuid_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/client.go b/pkg/dhcp/client.go index 92c634a14..3330c4998 100644 --- a/pkg/dhcp/client.go +++ b/pkg/dhcp/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp.go b/pkg/dhcp/dhcp.go index ceaba34c3..ad11e178a 100644 --- a/pkg/dhcp/dhcp.go +++ b/pkg/dhcp/dhcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp_string.go b/pkg/dhcp/dhcp_string.go index 7cabed29e..8533895bd 100644 --- a/pkg/dhcp/dhcp_string.go +++ b/pkg/dhcp/dhcp_string.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/dhcp_test.go b/pkg/dhcp/dhcp_test.go index d60e3752b..a21dce6bc 100644 --- a/pkg/dhcp/dhcp_test.go +++ b/pkg/dhcp/dhcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/dhcp/server.go b/pkg/dhcp/server.go index 26700bdbc..3e06ab4c7 100644 --- a/pkg/dhcp/server.go +++ b/pkg/dhcp/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/eventchannel/event.go b/pkg/eventchannel/event.go index bfd28256e..41a7b5ed3 100644 --- a/pkg/eventchannel/event.go +++ b/pkg/eventchannel/event.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/eventchannel/event.proto b/pkg/eventchannel/event.proto index 455f03658..c1679c7e7 100644 --- a/pkg/eventchannel/event.proto +++ b/pkg/eventchannel/event.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/fd/fd.go b/pkg/fd/fd.go index 32d24c41b..f6656ffa1 100644 --- a/pkg/fd/fd.go +++ b/pkg/fd/fd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/fd/fd_test.go b/pkg/fd/fd_test.go index 94b3eb7cc..42bb3ef6c 100644 --- a/pkg/fd/fd_test.go +++ b/pkg/fd/fd_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/gate/gate.go b/pkg/gate/gate.go index 93808c9dd..48122bf5a 100644 --- a/pkg/gate/gate.go +++ b/pkg/gate/gate.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/gate/gate_test.go b/pkg/gate/gate_test.go index 06587339b..95620fa8e 100644 --- a/pkg/gate/gate_test.go +++ b/pkg/gate/gate_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/ilist/list.go b/pkg/ilist/list.go index 4ae02eee9..51c9b6df3 100644 --- a/pkg/ilist/list.go +++ b/pkg/ilist/list.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/ilist/list_test.go b/pkg/ilist/list_test.go index 2c56280f6..4bda570b6 100644 --- a/pkg/ilist/list_test.go +++ b/pkg/ilist/list_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/linewriter/linewriter.go b/pkg/linewriter/linewriter.go index 98f974410..5fbd4e779 100644 --- a/pkg/linewriter/linewriter.go +++ b/pkg/linewriter/linewriter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/linewriter/linewriter_test.go b/pkg/linewriter/linewriter_test.go index ce97cca05..9140ee6af 100644 --- a/pkg/linewriter/linewriter_test.go +++ b/pkg/linewriter/linewriter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/glog.go b/pkg/log/glog.go index 58b4052e6..fbb58501b 100644 --- a/pkg/log/glog.go +++ b/pkg/log/glog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/glog_unsafe.go b/pkg/log/glog_unsafe.go index c320190b8..bb06aa7d3 100644 --- a/pkg/log/glog_unsafe.go +++ b/pkg/log/glog_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/json.go b/pkg/log/json.go index 3887f1cd5..96bd13d87 100644 --- a/pkg/log/json.go +++ b/pkg/log/json.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/json_test.go b/pkg/log/json_test.go index 3b167dab0..b8c7a795e 100644 --- a/pkg/log/json_test.go +++ b/pkg/log/json_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/log.go b/pkg/log/log.go index c496e86e4..b8d456aae 100644 --- a/pkg/log/log.go +++ b/pkg/log/log.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/log/log_test.go b/pkg/log/log_test.go index d93e989dc..a59d457dd 100644 --- a/pkg/log/log_test.go +++ b/pkg/log/log_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric.go b/pkg/metric/metric.go index 763cd6bc2..02af75974 100644 --- a/pkg/metric/metric.go +++ b/pkg/metric/metric.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric.proto b/pkg/metric/metric.proto index 6108cb7c0..917fda1ac 100644 --- a/pkg/metric/metric.proto +++ b/pkg/metric/metric.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/metric/metric_test.go b/pkg/metric/metric_test.go index 7d156e4a5..40034a589 100644 --- a/pkg/metric/metric_test.go +++ b/pkg/metric/metric_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/buffer.go b/pkg/p9/buffer.go index fc65d2c5f..9575ddf12 100644 --- a/pkg/p9/buffer.go +++ b/pkg/p9/buffer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client.go b/pkg/p9/client.go index 5fa231bc5..3ebfab82a 100644 --- a/pkg/p9/client.go +++ b/pkg/p9/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go index a46efd27f..066639fda 100644 --- a/pkg/p9/client_file.go +++ b/pkg/p9/client_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/client_test.go b/pkg/p9/client_test.go index 06302a76a..f7145452d 100644 --- a/pkg/p9/client_test.go +++ b/pkg/p9/client_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/file.go b/pkg/p9/file.go index 9723fa24d..d2e89e373 100644 --- a/pkg/p9/file.go +++ b/pkg/p9/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go index ea41f97c7..959dff31d 100644 --- a/pkg/p9/handlers.go +++ b/pkg/p9/handlers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/local_server/local_server.go b/pkg/p9/local_server/local_server.go index cef3701a7..1e6aaa762 100644 --- a/pkg/p9/local_server/local_server.go +++ b/pkg/p9/local_server/local_server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/messages.go b/pkg/p9/messages.go index b3d76801b..972c37344 100644 --- a/pkg/p9/messages.go +++ b/pkg/p9/messages.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/messages_test.go b/pkg/p9/messages_test.go index f353755f1..dfb41bb76 100644 --- a/pkg/p9/messages_test.go +++ b/pkg/p9/messages_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9.go b/pkg/p9/p9.go index c6899c3ce..3b0993ecd 100644 --- a/pkg/p9/p9.go +++ b/pkg/p9/p9.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9_test.go b/pkg/p9/p9_test.go index a50ac80a4..02498346c 100644 --- a/pkg/p9/p9_test.go +++ b/pkg/p9/p9_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9test/client_test.go b/pkg/p9/p9test/client_test.go index 34ddccd8b..db562b9ba 100644 --- a/pkg/p9/p9test/client_test.go +++ b/pkg/p9/p9test/client_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/p9test/mocks.go b/pkg/p9/p9test/mocks.go index 9d039ac63..9a8c14975 100644 --- a/pkg/p9/p9test/mocks.go +++ b/pkg/p9/p9test/mocks.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/pool.go b/pkg/p9/pool.go index 9a508b898..34ed898e8 100644 --- a/pkg/p9/pool.go +++ b/pkg/p9/pool.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/pool_test.go b/pkg/p9/pool_test.go index 96be2c8bd..71052d8c4 100644 --- a/pkg/p9/pool_test.go +++ b/pkg/p9/pool_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/server.go b/pkg/p9/server.go index 28a273ac6..5c7cb18c8 100644 --- a/pkg/p9/server.go +++ b/pkg/p9/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/transport.go b/pkg/p9/transport.go index b5df29961..97396806c 100644 --- a/pkg/p9/transport.go +++ b/pkg/p9/transport.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/transport_test.go b/pkg/p9/transport_test.go index d6d4b6365..3352a5205 100644 --- a/pkg/p9/transport_test.go +++ b/pkg/p9/transport_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/version.go b/pkg/p9/version.go index 8783eaa7e..ceb6fabbf 100644 --- a/pkg/p9/version.go +++ b/pkg/p9/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/p9/version_test.go b/pkg/p9/version_test.go index 634ac3ca5..c053614c9 100644 --- a/pkg/p9/version_test.go +++ b/pkg/p9/version_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/rand/rand.go b/pkg/rand/rand.go index e81f0f5db..593a14380 100644 --- a/pkg/rand/rand.go +++ b/pkg/rand/rand.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/rand/rand_linux.go b/pkg/rand/rand_linux.go index a2be66b3b..7ebe8f3b0 100644 --- a/pkg/rand/rand_linux.go +++ b/pkg/rand/rand_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter.go b/pkg/refs/refcounter.go index 638a93bab..8f08c74c7 100644 --- a/pkg/refs/refcounter.go +++ b/pkg/refs/refcounter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter_state.go b/pkg/refs/refcounter_state.go index 093eae785..136f06fbf 100644 --- a/pkg/refs/refcounter_state.go +++ b/pkg/refs/refcounter_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/refs/refcounter_test.go b/pkg/refs/refcounter_test.go index cc11bcd71..abaa87453 100644 --- a/pkg/refs/refcounter_test.go +++ b/pkg/refs/refcounter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp.go b/pkg/seccomp/seccomp.go index a746dc9b3..1dfbf749e 100644 --- a/pkg/seccomp/seccomp.go +++ b/pkg/seccomp/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_rules.go b/pkg/seccomp/seccomp_rules.go index 6b707f195..a9278c64b 100644 --- a/pkg/seccomp/seccomp_rules.go +++ b/pkg/seccomp/seccomp_rules.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_test.go b/pkg/seccomp/seccomp_test.go index 0188ad4f3..226f30b7b 100644 --- a/pkg/seccomp/seccomp_test.go +++ b/pkg/seccomp/seccomp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_test_victim.go b/pkg/seccomp/seccomp_test_victim.go index 4f2ae4dac..007038273 100644 --- a/pkg/seccomp/seccomp_test_victim.go +++ b/pkg/seccomp/seccomp_test_victim.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/seccomp/seccomp_unsafe.go b/pkg/seccomp/seccomp_unsafe.go index ae18534bf..dd009221a 100644 --- a/pkg/seccomp/seccomp_unsafe.go +++ b/pkg/seccomp/seccomp_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/full_reader.go b/pkg/secio/full_reader.go index b2dbb8615..90b1772a7 100644 --- a/pkg/secio/full_reader.go +++ b/pkg/secio/full_reader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/secio.go b/pkg/secio/secio.go index fc625efb8..e5f74a497 100644 --- a/pkg/secio/secio.go +++ b/pkg/secio/secio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/secio/secio_test.go b/pkg/secio/secio_test.go index 64b4cc17d..8304c4f74 100644 --- a/pkg/secio/secio_test.go +++ b/pkg/secio/secio_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/range.go b/pkg/segment/range.go index 34c067265..057bcd7ff 100644 --- a/pkg/segment/range.go +++ b/pkg/segment/range.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/set.go b/pkg/segment/set.go index cffec2a2c..a9a3b8875 100644 --- a/pkg/segment/set.go +++ b/pkg/segment/set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/set_state.go b/pkg/segment/set_state.go index a763d1915..b86e1b75f 100644 --- a/pkg/segment/set_state.go +++ b/pkg/segment/set_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/test/segment_test.go b/pkg/segment/test/segment_test.go index 7ea24b177..0825105db 100644 --- a/pkg/segment/test/segment_test.go +++ b/pkg/segment/test/segment_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/segment/test/set_functions.go b/pkg/segment/test/set_functions.go index 37c196ea1..05ba5fbb9 100644 --- a/pkg/segment/test/set_functions.go +++ b/pkg/segment/test/set_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/aligned.go b/pkg/sentry/arch/aligned.go index 193232e27..c88c034f6 100644 --- a/pkg/sentry/arch/aligned.go +++ b/pkg/sentry/arch/aligned.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch.go b/pkg/sentry/arch/arch.go index 21cb84502..575b7ba66 100644 --- a/pkg/sentry/arch/arch.go +++ b/pkg/sentry/arch/arch.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_amd64.go b/pkg/sentry/arch/arch_amd64.go index 5ba6c19ea..bb80a7bed 100644 --- a/pkg/sentry/arch/arch_amd64.go +++ b/pkg/sentry/arch/arch_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_amd64.s b/pkg/sentry/arch/arch_amd64.s index 10d621b6d..fa9857df7 100644 --- a/pkg/sentry/arch/arch_amd64.s +++ b/pkg/sentry/arch/arch_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_state_x86.go b/pkg/sentry/arch/arch_state_x86.go index e9c23a06b..604bd08a6 100644 --- a/pkg/sentry/arch/arch_state_x86.go +++ b/pkg/sentry/arch/arch_state_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/arch_x86.go b/pkg/sentry/arch/arch_x86.go index b35eec53c..59bf89d99 100644 --- a/pkg/sentry/arch/arch_x86.go +++ b/pkg/sentry/arch/arch_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/auxv.go b/pkg/sentry/arch/auxv.go index 81cfb4a01..5df65a691 100644 --- a/pkg/sentry/arch/auxv.go +++ b/pkg/sentry/arch/auxv.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/registers.proto b/pkg/sentry/arch/registers.proto index 437ff44ca..f4c2f7043 100644 --- a/pkg/sentry/arch/registers.proto +++ b/pkg/sentry/arch/registers.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_act.go b/pkg/sentry/arch/signal_act.go index 36437b965..ad098c746 100644 --- a/pkg/sentry/arch/signal_act.go +++ b/pkg/sentry/arch/signal_act.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_amd64.go b/pkg/sentry/arch/signal_amd64.go index 9ca4c8ed1..f7f054b0b 100644 --- a/pkg/sentry/arch/signal_amd64.go +++ b/pkg/sentry/arch/signal_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_info.go b/pkg/sentry/arch/signal_info.go index ec004ae75..fa0ecbec5 100644 --- a/pkg/sentry/arch/signal_info.go +++ b/pkg/sentry/arch/signal_info.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/signal_stack.go b/pkg/sentry/arch/signal_stack.go index ba43dd1d4..c02ae3b7c 100644 --- a/pkg/sentry/arch/signal_stack.go +++ b/pkg/sentry/arch/signal_stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/stack.go b/pkg/sentry/arch/stack.go index 6c1b9be82..716a3574d 100644 --- a/pkg/sentry/arch/stack.go +++ b/pkg/sentry/arch/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/arch/syscalls_amd64.go b/pkg/sentry/arch/syscalls_amd64.go index 41d8ba0d1..47c31d4b9 100644 --- a/pkg/sentry/arch/syscalls_amd64.go +++ b/pkg/sentry/arch/syscalls_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/context/context.go b/pkg/sentry/context/context.go index 598c5b4ff..12bdcef85 100644 --- a/pkg/sentry/context/context.go +++ b/pkg/sentry/context/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/context/contexttest/contexttest.go b/pkg/sentry/context/contexttest/contexttest.go index b3c6a566b..d2f084ed7 100644 --- a/pkg/sentry/context/contexttest/contexttest.go +++ b/pkg/sentry/context/contexttest/contexttest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/control.go b/pkg/sentry/control/control.go index a6ee6e649..32d30b6ea 100644 --- a/pkg/sentry/control/control.go +++ b/pkg/sentry/control/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/proc.go b/pkg/sentry/control/proc.go index 0ba730c1e..b6ac2f312 100644 --- a/pkg/sentry/control/proc.go +++ b/pkg/sentry/control/proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/proc_test.go b/pkg/sentry/control/proc_test.go index 22c826236..5d52cd829 100644 --- a/pkg/sentry/control/proc_test.go +++ b/pkg/sentry/control/proc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/state.go b/pkg/sentry/control/state.go index cee4db636..0a480c84a 100644 --- a/pkg/sentry/control/state.go +++ b/pkg/sentry/control/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/device/device.go b/pkg/sentry/device/device.go index 21fee8f8a..27e4eb258 100644 --- a/pkg/sentry/device/device.go +++ b/pkg/sentry/device/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/device/device_test.go b/pkg/sentry/device/device_test.go index dfec45046..5d8805c2f 100644 --- a/pkg/sentry/device/device_test.go +++ b/pkg/sentry/device/device_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/anon/anon.go b/pkg/sentry/fs/anon/anon.go index ddc2c0985..743cf511f 100644 --- a/pkg/sentry/fs/anon/anon.go +++ b/pkg/sentry/fs/anon/anon.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/anon/device.go b/pkg/sentry/fs/anon/device.go index 1c666729c..2d1249299 100644 --- a/pkg/sentry/fs/anon/device.go +++ b/pkg/sentry/fs/anon/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go index bfd7f2762..5372875ac 100644 --- a/pkg/sentry/fs/ashmem/area.go +++ b/pkg/sentry/fs/ashmem/area.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/device.go b/pkg/sentry/fs/ashmem/device.go index d0986fa11..962da141b 100644 --- a/pkg/sentry/fs/ashmem/device.go +++ b/pkg/sentry/fs/ashmem/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/pin_board.go b/pkg/sentry/fs/ashmem/pin_board.go index ecba395a0..7c997f533 100644 --- a/pkg/sentry/fs/ashmem/pin_board.go +++ b/pkg/sentry/fs/ashmem/pin_board.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ashmem/pin_board_test.go b/pkg/sentry/fs/ashmem/pin_board_test.go index f4ea5de6d..736e628dc 100644 --- a/pkg/sentry/fs/ashmem/pin_board_test.go +++ b/pkg/sentry/fs/ashmem/pin_board_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/attr.go b/pkg/sentry/fs/attr.go index 091f4ac63..59e060e3c 100644 --- a/pkg/sentry/fs/attr.go +++ b/pkg/sentry/fs/attr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go index 502a262dd..42b9e8b26 100644 --- a/pkg/sentry/fs/binder/binder.go +++ b/pkg/sentry/fs/binder/binder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/context.go b/pkg/sentry/fs/context.go index da46ad77f..1775d3486 100644 --- a/pkg/sentry/fs/context.go +++ b/pkg/sentry/fs/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/copy_up.go b/pkg/sentry/fs/copy_up.go index 8c949b176..d65dc74bf 100644 --- a/pkg/sentry/fs/copy_up.go +++ b/pkg/sentry/fs/copy_up.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/copy_up_test.go b/pkg/sentry/fs/copy_up_test.go index c3c9d963d..64f030f72 100644 --- a/pkg/sentry/fs/copy_up_test.go +++ b/pkg/sentry/fs/copy_up_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dentry.go b/pkg/sentry/fs/dentry.go index b347468ff..ef6d1a870 100644 --- a/pkg/sentry/fs/dentry.go +++ b/pkg/sentry/fs/dentry.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/dev.go b/pkg/sentry/fs/dev/dev.go index 3f4f2a40a..05a5005ad 100644 --- a/pkg/sentry/fs/dev/dev.go +++ b/pkg/sentry/fs/dev/dev.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/device.go b/pkg/sentry/fs/dev/device.go index 9d935e008..3cecdf6e2 100644 --- a/pkg/sentry/fs/dev/device.go +++ b/pkg/sentry/fs/dev/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/fs.go b/pkg/sentry/fs/dev/fs.go index 2ae49be4e..d96f4f423 100644 --- a/pkg/sentry/fs/dev/fs.go +++ b/pkg/sentry/fs/dev/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/full.go b/pkg/sentry/fs/dev/full.go index 492b8eb3a..eeda646ab 100644 --- a/pkg/sentry/fs/dev/full.go +++ b/pkg/sentry/fs/dev/full.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/null.go b/pkg/sentry/fs/dev/null.go index 2977c8670..68090f353 100644 --- a/pkg/sentry/fs/dev/null.go +++ b/pkg/sentry/fs/dev/null.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dev/random.go b/pkg/sentry/fs/dev/random.go index 47b76218f..33e4913e4 100644 --- a/pkg/sentry/fs/dev/random.go +++ b/pkg/sentry/fs/dev/random.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent.go b/pkg/sentry/fs/dirent.go index 27fea0019..2c01485a8 100644 --- a/pkg/sentry/fs/dirent.go +++ b/pkg/sentry/fs/dirent.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_cache.go b/pkg/sentry/fs/dirent_cache.go index c680e4828..502b0a09b 100644 --- a/pkg/sentry/fs/dirent_cache.go +++ b/pkg/sentry/fs/dirent_cache.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_cache_test.go b/pkg/sentry/fs/dirent_cache_test.go index 82b7f6bd5..5d0e9d91c 100644 --- a/pkg/sentry/fs/dirent_cache_test.go +++ b/pkg/sentry/fs/dirent_cache_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_refs_test.go b/pkg/sentry/fs/dirent_refs_test.go index f9dcba316..325404e27 100644 --- a/pkg/sentry/fs/dirent_refs_test.go +++ b/pkg/sentry/fs/dirent_refs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/dirent_state.go b/pkg/sentry/fs/dirent_state.go index 04ab197b9..5cf151dab 100644 --- a/pkg/sentry/fs/dirent_state.go +++ b/pkg/sentry/fs/dirent_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go index 2e34604e6..bfafff5ec 100644 --- a/pkg/sentry/fs/fdpipe/pipe.go +++ b/pkg/sentry/fs/fdpipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_opener.go b/pkg/sentry/fs/fdpipe/pipe_opener.go index 945cfaf08..92ab6ff0e 100644 --- a/pkg/sentry/fs/fdpipe/pipe_opener.go +++ b/pkg/sentry/fs/fdpipe/pipe_opener.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_opener_test.go b/pkg/sentry/fs/fdpipe/pipe_opener_test.go index 83f6c1986..69516e048 100644 --- a/pkg/sentry/fs/fdpipe/pipe_opener_test.go +++ b/pkg/sentry/fs/fdpipe/pipe_opener_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_state.go b/pkg/sentry/fs/fdpipe/pipe_state.go index 99c40d8ed..4395666ad 100644 --- a/pkg/sentry/fs/fdpipe/pipe_state.go +++ b/pkg/sentry/fs/fdpipe/pipe_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fdpipe/pipe_test.go b/pkg/sentry/fs/fdpipe/pipe_test.go index 6cd314f5b..d3f15be6b 100644 --- a/pkg/sentry/fs/fdpipe/pipe_test.go +++ b/pkg/sentry/fs/fdpipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go index 36794d378..d6752ed1b 100644 --- a/pkg/sentry/fs/file.go +++ b/pkg/sentry/fs/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_operations.go b/pkg/sentry/fs/file_operations.go index d223bb5c7..28e8e233d 100644 --- a/pkg/sentry/fs/file_operations.go +++ b/pkg/sentry/fs/file_operations.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go index 41e646ee8..9b958b64b 100644 --- a/pkg/sentry/fs/file_overlay.go +++ b/pkg/sentry/fs/file_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_overlay_test.go b/pkg/sentry/fs/file_overlay_test.go index 830458ff9..11e4f7203 100644 --- a/pkg/sentry/fs/file_overlay_test.go +++ b/pkg/sentry/fs/file_overlay_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_state.go b/pkg/sentry/fs/file_state.go index f848d1b79..1c3bae3e8 100644 --- a/pkg/sentry/fs/file_state.go +++ b/pkg/sentry/fs/file_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/file_test.go b/pkg/sentry/fs/file_test.go index 18aee7101..f3ed9a70b 100644 --- a/pkg/sentry/fs/file_test.go +++ b/pkg/sentry/fs/file_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/filesystems.go b/pkg/sentry/fs/filesystems.go index 5a1e7a270..ba8be85e4 100644 --- a/pkg/sentry/fs/filesystems.go +++ b/pkg/sentry/fs/filesystems.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/filetest/filetest.go b/pkg/sentry/fs/filetest/filetest.go index 1831aa82f..65ca196d9 100644 --- a/pkg/sentry/fs/filetest/filetest.go +++ b/pkg/sentry/fs/filetest/filetest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/flags.go b/pkg/sentry/fs/flags.go index 1aa271560..bf2a20b33 100644 --- a/pkg/sentry/fs/flags.go +++ b/pkg/sentry/fs/flags.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fs.go b/pkg/sentry/fs/fs.go index 6ec9ff446..b5c72990e 100644 --- a/pkg/sentry/fs/fs.go +++ b/pkg/sentry/fs/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/dirty_set.go b/pkg/sentry/fs/fsutil/dirty_set.go index 8e31e48fd..5add16ac4 100644 --- a/pkg/sentry/fs/fsutil/dirty_set.go +++ b/pkg/sentry/fs/fsutil/dirty_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/dirty_set_test.go b/pkg/sentry/fs/fsutil/dirty_set_test.go index f7693cb19..f5c9d9215 100644 --- a/pkg/sentry/fs/fsutil/dirty_set_test.go +++ b/pkg/sentry/fs/fsutil/dirty_set_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index d5881613b..46db2e51c 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/file_range_set.go b/pkg/sentry/fs/fsutil/file_range_set.go index da6949ccb..dd7ab4b4a 100644 --- a/pkg/sentry/fs/fsutil/file_range_set.go +++ b/pkg/sentry/fs/fsutil/file_range_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/frame_ref_set.go b/pkg/sentry/fs/fsutil/frame_ref_set.go index 14dece315..b6e783614 100644 --- a/pkg/sentry/fs/fsutil/frame_ref_set.go +++ b/pkg/sentry/fs/fsutil/frame_ref_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/fsutil.go b/pkg/sentry/fs/fsutil/fsutil.go index 6fe4ef13d..3d7f3732d 100644 --- a/pkg/sentry/fs/fsutil/fsutil.go +++ b/pkg/sentry/fs/fsutil/fsutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/handle.go b/pkg/sentry/fs/fsutil/handle.go index e7efd3c0f..8920b72ee 100644 --- a/pkg/sentry/fs/fsutil/handle.go +++ b/pkg/sentry/fs/fsutil/handle.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/handle_test.go b/pkg/sentry/fs/fsutil/handle_test.go index d94c3eb0d..43e1a3bdf 100644 --- a/pkg/sentry/fs/fsutil/handle_test.go +++ b/pkg/sentry/fs/fsutil/handle_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper.go b/pkg/sentry/fs/fsutil/host_file_mapper.go index 9c1e2f76f..9599665f0 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper_state.go b/pkg/sentry/fs/fsutil/host_file_mapper_state.go index 57705decd..bbd15b30b 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper_state.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go b/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go index 790f3a5a6..86df76822 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go index 3acc32752..d4db1c2de 100644 --- a/pkg/sentry/fs/fsutil/inode.go +++ b/pkg/sentry/fs/fsutil/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode_cached.go b/pkg/sentry/fs/fsutil/inode_cached.go index 6777c8bf7..b0af44ddd 100644 --- a/pkg/sentry/fs/fsutil/inode_cached.go +++ b/pkg/sentry/fs/fsutil/inode_cached.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode_cached_test.go b/pkg/sentry/fs/fsutil/inode_cached_test.go index 996c91849..e388ec3d7 100644 --- a/pkg/sentry/fs/fsutil/inode_cached_test.go +++ b/pkg/sentry/fs/fsutil/inode_cached_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/attr.go b/pkg/sentry/fs/gofer/attr.go index 5e24767f9..98700d014 100644 --- a/pkg/sentry/fs/gofer/attr.go +++ b/pkg/sentry/fs/gofer/attr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/cache_policy.go b/pkg/sentry/fs/gofer/cache_policy.go index 98f43c578..3d380f0e8 100644 --- a/pkg/sentry/fs/gofer/cache_policy.go +++ b/pkg/sentry/fs/gofer/cache_policy.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/context_file.go b/pkg/sentry/fs/gofer/context_file.go index d4b6f6eb7..a0265c2aa 100644 --- a/pkg/sentry/fs/gofer/context_file.go +++ b/pkg/sentry/fs/gofer/context_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/device.go b/pkg/sentry/fs/gofer/device.go index fac7306d4..52c5acf48 100644 --- a/pkg/sentry/fs/gofer/device.go +++ b/pkg/sentry/fs/gofer/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go index c4a210656..6d961813d 100644 --- a/pkg/sentry/fs/gofer/file.go +++ b/pkg/sentry/fs/gofer/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/file_state.go b/pkg/sentry/fs/gofer/file_state.go index 715af8f16..dd4f817bf 100644 --- a/pkg/sentry/fs/gofer/file_state.go +++ b/pkg/sentry/fs/gofer/file_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/fs.go b/pkg/sentry/fs/gofer/fs.go index 3ae93f059..ed30cb1f1 100644 --- a/pkg/sentry/fs/gofer/fs.go +++ b/pkg/sentry/fs/gofer/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/gofer_test.go b/pkg/sentry/fs/gofer/gofer_test.go index c8d7bd773..3190d1e18 100644 --- a/pkg/sentry/fs/gofer/gofer_test.go +++ b/pkg/sentry/fs/gofer/gofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/handles.go b/pkg/sentry/fs/gofer/handles.go index a3e52aad6..f32e99ce0 100644 --- a/pkg/sentry/fs/gofer/handles.go +++ b/pkg/sentry/fs/gofer/handles.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/inode.go b/pkg/sentry/fs/gofer/inode.go index 7fc8f77b0..5811b8b12 100644 --- a/pkg/sentry/fs/gofer/inode.go +++ b/pkg/sentry/fs/gofer/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/inode_state.go b/pkg/sentry/fs/gofer/inode_state.go index ad11034f9..ad4d3df58 100644 --- a/pkg/sentry/fs/gofer/inode_state.go +++ b/pkg/sentry/fs/gofer/inode_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/path.go b/pkg/sentry/fs/gofer/path.go index 0bf7881da..a324dc990 100644 --- a/pkg/sentry/fs/gofer/path.go +++ b/pkg/sentry/fs/gofer/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go index 4e2293398..7552216f3 100644 --- a/pkg/sentry/fs/gofer/session.go +++ b/pkg/sentry/fs/gofer/session.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/session_state.go b/pkg/sentry/fs/gofer/session_state.go index 8e6424492..f657135fc 100644 --- a/pkg/sentry/fs/gofer/session_state.go +++ b/pkg/sentry/fs/gofer/session_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/socket.go b/pkg/sentry/fs/gofer/socket.go index d072da624..76ce58810 100644 --- a/pkg/sentry/fs/gofer/socket.go +++ b/pkg/sentry/fs/gofer/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/gofer/util.go b/pkg/sentry/fs/gofer/util.go index d9ed8c81e..1a759370d 100644 --- a/pkg/sentry/fs/gofer/util.go +++ b/pkg/sentry/fs/gofer/util.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/control.go b/pkg/sentry/fs/host/control.go index d2e34a69d..0753640a2 100644 --- a/pkg/sentry/fs/host/control.go +++ b/pkg/sentry/fs/host/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor.go b/pkg/sentry/fs/host/descriptor.go index 148291ba6..7c9d2b299 100644 --- a/pkg/sentry/fs/host/descriptor.go +++ b/pkg/sentry/fs/host/descriptor.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor_state.go b/pkg/sentry/fs/host/descriptor_state.go index 7fb274451..530c0109f 100644 --- a/pkg/sentry/fs/host/descriptor_state.go +++ b/pkg/sentry/fs/host/descriptor_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/descriptor_test.go b/pkg/sentry/fs/host/descriptor_test.go index f393a8b54..6bc1bd2ae 100644 --- a/pkg/sentry/fs/host/descriptor_test.go +++ b/pkg/sentry/fs/host/descriptor_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/device.go b/pkg/sentry/fs/host/device.go index f2a0b6b15..b5adedf44 100644 --- a/pkg/sentry/fs/host/device.go +++ b/pkg/sentry/fs/host/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go index 22a5d9f12..975084c86 100644 --- a/pkg/sentry/fs/host/file.go +++ b/pkg/sentry/fs/host/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/fs.go b/pkg/sentry/fs/host/fs.go index e46ae433c..fec890964 100644 --- a/pkg/sentry/fs/host/fs.go +++ b/pkg/sentry/fs/host/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/fs_test.go b/pkg/sentry/fs/host/fs_test.go index b08125ca8..e69559aac 100644 --- a/pkg/sentry/fs/host/fs_test.go +++ b/pkg/sentry/fs/host/fs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go index e32497203..08754bd6b 100644 --- a/pkg/sentry/fs/host/inode.go +++ b/pkg/sentry/fs/host/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode_state.go b/pkg/sentry/fs/host/inode_state.go index 8bc99d94b..b7c1a9581 100644 --- a/pkg/sentry/fs/host/inode_state.go +++ b/pkg/sentry/fs/host/inode_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/inode_test.go b/pkg/sentry/fs/host/inode_test.go index 0ff87c418..9f1561bd5 100644 --- a/pkg/sentry/fs/host/inode_test.go +++ b/pkg/sentry/fs/host/inode_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/ioctl_unsafe.go b/pkg/sentry/fs/host/ioctl_unsafe.go index bc965a1c2..175dca613 100644 --- a/pkg/sentry/fs/host/ioctl_unsafe.go +++ b/pkg/sentry/fs/host/ioctl_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket.go b/pkg/sentry/fs/host/socket.go index 0eb267c00..af53bf533 100644 --- a/pkg/sentry/fs/host/socket.go +++ b/pkg/sentry/fs/host/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_iovec.go b/pkg/sentry/fs/host/socket_iovec.go index 1a9587b90..d4ce4a8c1 100644 --- a/pkg/sentry/fs/host/socket_iovec.go +++ b/pkg/sentry/fs/host/socket_iovec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_state.go b/pkg/sentry/fs/host/socket_state.go index 7fa500bfb..2932c1f16 100644 --- a/pkg/sentry/fs/host/socket_state.go +++ b/pkg/sentry/fs/host/socket_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_test.go b/pkg/sentry/fs/host/socket_test.go index 483e99dd6..e9a88b124 100644 --- a/pkg/sentry/fs/host/socket_test.go +++ b/pkg/sentry/fs/host/socket_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/socket_unsafe.go b/pkg/sentry/fs/host/socket_unsafe.go index 5e4c5feed..f35e2492d 100644 --- a/pkg/sentry/fs/host/socket_unsafe.go +++ b/pkg/sentry/fs/host/socket_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/tty.go b/pkg/sentry/fs/host/tty.go index ad1323610..cf3639c46 100644 --- a/pkg/sentry/fs/host/tty.go +++ b/pkg/sentry/fs/host/tty.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/util.go b/pkg/sentry/fs/host/util.go index 74c703eb7..40c450660 100644 --- a/pkg/sentry/fs/host/util.go +++ b/pkg/sentry/fs/host/util.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/util_unsafe.go b/pkg/sentry/fs/host/util_unsafe.go index 2ecb54319..d00da89d6 100644 --- a/pkg/sentry/fs/host/util_unsafe.go +++ b/pkg/sentry/fs/host/util_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/host/wait_test.go b/pkg/sentry/fs/host/wait_test.go index c5f5c9c0d..9ca8c399f 100644 --- a/pkg/sentry/fs/host/wait_test.go +++ b/pkg/sentry/fs/host/wait_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode.go b/pkg/sentry/fs/inode.go index 409c81a97..95769ccf8 100644 --- a/pkg/sentry/fs/inode.go +++ b/pkg/sentry/fs/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_inotify.go b/pkg/sentry/fs/inode_inotify.go index 683140afe..e213df924 100644 --- a/pkg/sentry/fs/inode_inotify.go +++ b/pkg/sentry/fs/inode_inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_operations.go b/pkg/sentry/fs/inode_operations.go index 3ee3de10e..77973ce79 100644 --- a/pkg/sentry/fs/inode_operations.go +++ b/pkg/sentry/fs/inode_operations.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_overlay.go b/pkg/sentry/fs/inode_overlay.go index cf698a4da..78923fb5b 100644 --- a/pkg/sentry/fs/inode_overlay.go +++ b/pkg/sentry/fs/inode_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inode_overlay_test.go b/pkg/sentry/fs/inode_overlay_test.go index 23e5635a4..bba20da14 100644 --- a/pkg/sentry/fs/inode_overlay_test.go +++ b/pkg/sentry/fs/inode_overlay_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify.go b/pkg/sentry/fs/inotify.go index 2aabdded8..f251df0d1 100644 --- a/pkg/sentry/fs/inotify.go +++ b/pkg/sentry/fs/inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify_event.go b/pkg/sentry/fs/inotify_event.go index e9b5e0f56..9e3e9d816 100644 --- a/pkg/sentry/fs/inotify_event.go +++ b/pkg/sentry/fs/inotify_event.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/inotify_watch.go b/pkg/sentry/fs/inotify_watch.go index 3e1959e83..b83544c9f 100644 --- a/pkg/sentry/fs/inotify_watch.go +++ b/pkg/sentry/fs/inotify_watch.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock.go b/pkg/sentry/fs/lock/lock.go index 439e645db..5ff800d2d 100644 --- a/pkg/sentry/fs/lock/lock.go +++ b/pkg/sentry/fs/lock/lock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_range_test.go b/pkg/sentry/fs/lock/lock_range_test.go index 06a37c701..b0ab882b9 100644 --- a/pkg/sentry/fs/lock/lock_range_test.go +++ b/pkg/sentry/fs/lock/lock_range_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_set_functions.go b/pkg/sentry/fs/lock/lock_set_functions.go index e16f485be..395592a4b 100644 --- a/pkg/sentry/fs/lock/lock_set_functions.go +++ b/pkg/sentry/fs/lock/lock_set_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/lock/lock_test.go b/pkg/sentry/fs/lock/lock_test.go index c60f5f7a2..67fa4b1dd 100644 --- a/pkg/sentry/fs/lock/lock_test.go +++ b/pkg/sentry/fs/lock/lock_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mock.go b/pkg/sentry/fs/mock.go index 846b6e8bb..6bfcda6bb 100644 --- a/pkg/sentry/fs/mock.go +++ b/pkg/sentry/fs/mock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount.go b/pkg/sentry/fs/mount.go index 8345876fc..24e28ddb2 100644 --- a/pkg/sentry/fs/mount.go +++ b/pkg/sentry/fs/mount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount_overlay.go b/pkg/sentry/fs/mount_overlay.go index dbc608c7e..fb91635bc 100644 --- a/pkg/sentry/fs/mount_overlay.go +++ b/pkg/sentry/fs/mount_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount_state.go b/pkg/sentry/fs/mount_state.go index f5ed1dd8d..6344d5160 100644 --- a/pkg/sentry/fs/mount_state.go +++ b/pkg/sentry/fs/mount_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mount_test.go b/pkg/sentry/fs/mount_test.go index 968b435ab..a1c9f4f79 100644 --- a/pkg/sentry/fs/mount_test.go +++ b/pkg/sentry/fs/mount_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mounts.go b/pkg/sentry/fs/mounts.go index c0a803b2d..7c5348cce 100644 --- a/pkg/sentry/fs/mounts.go +++ b/pkg/sentry/fs/mounts.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/mounts_test.go b/pkg/sentry/fs/mounts_test.go index 8669f3a38..cc7c32c9b 100644 --- a/pkg/sentry/fs/mounts_test.go +++ b/pkg/sentry/fs/mounts_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/offset.go b/pkg/sentry/fs/offset.go index 7cc8398e6..38aee765a 100644 --- a/pkg/sentry/fs/offset.go +++ b/pkg/sentry/fs/offset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/overlay.go b/pkg/sentry/fs/overlay.go index 5a30af419..036c0f733 100644 --- a/pkg/sentry/fs/overlay.go +++ b/pkg/sentry/fs/overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/path.go b/pkg/sentry/fs/path.go index b74f6ed8c..91a9a8ffd 100644 --- a/pkg/sentry/fs/path.go +++ b/pkg/sentry/fs/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/path_test.go b/pkg/sentry/fs/path_test.go index 7ab070855..391b010a7 100644 --- a/pkg/sentry/fs/path_test.go +++ b/pkg/sentry/fs/path_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/cpuinfo.go b/pkg/sentry/fs/proc/cpuinfo.go index 4dfec03a4..f8be06dc3 100644 --- a/pkg/sentry/fs/proc/cpuinfo.go +++ b/pkg/sentry/fs/proc/cpuinfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/device/device.go b/pkg/sentry/fs/proc/device/device.go index 6194afe88..04b687bcf 100644 --- a/pkg/sentry/fs/proc/device/device.go +++ b/pkg/sentry/fs/proc/device/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/exec_args.go b/pkg/sentry/fs/proc/exec_args.go index a69cbaa0e..b4896053f 100644 --- a/pkg/sentry/fs/proc/exec_args.go +++ b/pkg/sentry/fs/proc/exec_args.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/fds.go b/pkg/sentry/fs/proc/fds.go index dada8f982..5ebb33703 100644 --- a/pkg/sentry/fs/proc/fds.go +++ b/pkg/sentry/fs/proc/fds.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/file.go b/pkg/sentry/fs/proc/file.go index 4b3448245..f659e590a 100644 --- a/pkg/sentry/fs/proc/file.go +++ b/pkg/sentry/fs/proc/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/filesystems.go b/pkg/sentry/fs/proc/filesystems.go index 49b92fd8a..c050a00be 100644 --- a/pkg/sentry/fs/proc/filesystems.go +++ b/pkg/sentry/fs/proc/filesystems.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/fs.go b/pkg/sentry/fs/proc/fs.go index 061824b8c..63f737ff4 100644 --- a/pkg/sentry/fs/proc/fs.go +++ b/pkg/sentry/fs/proc/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/loadavg.go b/pkg/sentry/fs/proc/loadavg.go index 6fac251d2..78f3a1dc0 100644 --- a/pkg/sentry/fs/proc/loadavg.go +++ b/pkg/sentry/fs/proc/loadavg.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/meminfo.go b/pkg/sentry/fs/proc/meminfo.go index 53dfd59ef..b31258eed 100644 --- a/pkg/sentry/fs/proc/meminfo.go +++ b/pkg/sentry/fs/proc/meminfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/mounts.go b/pkg/sentry/fs/proc/mounts.go index 81dcc153a..0b0e87528 100644 --- a/pkg/sentry/fs/proc/mounts.go +++ b/pkg/sentry/fs/proc/mounts.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go index 8cd6fe9d3..45f2a1211 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/net_test.go b/pkg/sentry/fs/proc/net_test.go index a31a20494..94677cc1d 100644 --- a/pkg/sentry/fs/proc/net_test.go +++ b/pkg/sentry/fs/proc/net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/proc.go b/pkg/sentry/fs/proc/proc.go index 07029a7bb..33030bebf 100644 --- a/pkg/sentry/fs/proc/proc.go +++ b/pkg/sentry/fs/proc/proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go index 50d0271f9..d025069df 100644 --- a/pkg/sentry/fs/proc/rpcinet_proc.go +++ b/pkg/sentry/fs/proc/rpcinet_proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/seqfile/seqfile.go b/pkg/sentry/fs/proc/seqfile/seqfile.go index 51cae5e37..0499ba65b 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/seqfile/seqfile_test.go b/pkg/sentry/fs/proc/seqfile/seqfile_test.go index d90e3e736..f9a2ca38e 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile_test.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/stat.go b/pkg/sentry/fs/proc/stat.go index bf7650211..f2bbef375 100644 --- a/pkg/sentry/fs/proc/stat.go +++ b/pkg/sentry/fs/proc/stat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go index 384b4ffe1..54562508d 100644 --- a/pkg/sentry/fs/proc/sys.go +++ b/pkg/sentry/fs/proc/sys.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index beb25be20..801eb6a1e 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/sys_net_test.go b/pkg/sentry/fs/proc/sys_net_test.go index 7ba392346..0ce9d30f1 100644 --- a/pkg/sentry/fs/proc/sys_net_test.go +++ b/pkg/sentry/fs/proc/sys_net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go index 748ca4320..404faea0a 100644 --- a/pkg/sentry/fs/proc/task.go +++ b/pkg/sentry/fs/proc/task.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/uid_gid_map.go b/pkg/sentry/fs/proc/uid_gid_map.go index a7e4cf0a6..f70399686 100644 --- a/pkg/sentry/fs/proc/uid_gid_map.go +++ b/pkg/sentry/fs/proc/uid_gid_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/uptime.go b/pkg/sentry/fs/proc/uptime.go index f3a9b81df..80c7ce0b4 100644 --- a/pkg/sentry/fs/proc/uptime.go +++ b/pkg/sentry/fs/proc/uptime.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/proc/version.go b/pkg/sentry/fs/proc/version.go index 00f6a2afd..b6d49d5e9 100644 --- a/pkg/sentry/fs/proc/version.go +++ b/pkg/sentry/fs/proc/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index 075e13b01..0a911b155 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/file.go b/pkg/sentry/fs/ramfs/file.go index 0b94d92a1..b7fc98ffc 100644 --- a/pkg/sentry/fs/ramfs/file.go +++ b/pkg/sentry/fs/ramfs/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/ramfs.go b/pkg/sentry/fs/ramfs/ramfs.go index 83cbcab23..d77688a34 100644 --- a/pkg/sentry/fs/ramfs/ramfs.go +++ b/pkg/sentry/fs/ramfs/ramfs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go index 9ac00eb18..8c81478c8 100644 --- a/pkg/sentry/fs/ramfs/socket.go +++ b/pkg/sentry/fs/ramfs/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/symlink.go b/pkg/sentry/fs/ramfs/symlink.go index 1c54d9991..a21fac2c7 100644 --- a/pkg/sentry/fs/ramfs/symlink.go +++ b/pkg/sentry/fs/ramfs/symlink.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/test/test.go b/pkg/sentry/fs/ramfs/test/test.go index fb669558f..11bff7729 100644 --- a/pkg/sentry/fs/ramfs/test/test.go +++ b/pkg/sentry/fs/ramfs/test/test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/tree.go b/pkg/sentry/fs/ramfs/tree.go index 1fb335f74..29a70f698 100644 --- a/pkg/sentry/fs/ramfs/tree.go +++ b/pkg/sentry/fs/ramfs/tree.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/ramfs/tree_test.go b/pkg/sentry/fs/ramfs/tree_test.go index 68e2929d5..d5567d9e1 100644 --- a/pkg/sentry/fs/ramfs/tree_test.go +++ b/pkg/sentry/fs/ramfs/tree_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/restore.go b/pkg/sentry/fs/restore.go index b4ac85a27..da2df7e1d 100644 --- a/pkg/sentry/fs/restore.go +++ b/pkg/sentry/fs/restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/save.go b/pkg/sentry/fs/save.go index bf2a85143..90988d385 100644 --- a/pkg/sentry/fs/save.go +++ b/pkg/sentry/fs/save.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/seek.go b/pkg/sentry/fs/seek.go index 1268726c2..72f3fb632 100644 --- a/pkg/sentry/fs/seek.go +++ b/pkg/sentry/fs/seek.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sync.go b/pkg/sentry/fs/sync.go index 9738a8f22..6dcc2fe8d 100644 --- a/pkg/sentry/fs/sync.go +++ b/pkg/sentry/fs/sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/device.go b/pkg/sentry/fs/sys/device.go index 54e414d1b..38ecd0c18 100644 --- a/pkg/sentry/fs/sys/device.go +++ b/pkg/sentry/fs/sys/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/devices.go b/pkg/sentry/fs/sys/devices.go index 2cf3a6f98..e64aa0edc 100644 --- a/pkg/sentry/fs/sys/devices.go +++ b/pkg/sentry/fs/sys/devices.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/fs.go b/pkg/sentry/fs/sys/fs.go index 625525540..5ce33f87f 100644 --- a/pkg/sentry/fs/sys/fs.go +++ b/pkg/sentry/fs/sys/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/sys/sys.go b/pkg/sentry/fs/sys/sys.go index 7b9697668..7cc1942c7 100644 --- a/pkg/sentry/fs/sys/sys.go +++ b/pkg/sentry/fs/sys/sys.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go index 767db95a0..7423e816c 100644 --- a/pkg/sentry/fs/timerfd/timerfd.go +++ b/pkg/sentry/fs/timerfd/timerfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/device.go b/pkg/sentry/fs/tmpfs/device.go index e588b3440..aade93c26 100644 --- a/pkg/sentry/fs/tmpfs/device.go +++ b/pkg/sentry/fs/tmpfs/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go index 342688f81..1f9d69909 100644 --- a/pkg/sentry/fs/tmpfs/file_regular.go +++ b/pkg/sentry/fs/tmpfs/file_regular.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/file_test.go b/pkg/sentry/fs/tmpfs/file_test.go index f064eb1ac..b5830d3df 100644 --- a/pkg/sentry/fs/tmpfs/file_test.go +++ b/pkg/sentry/fs/tmpfs/file_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/fs.go b/pkg/sentry/fs/tmpfs/fs.go index ca620e65e..7c91e248b 100644 --- a/pkg/sentry/fs/tmpfs/fs.go +++ b/pkg/sentry/fs/tmpfs/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/inode_file.go b/pkg/sentry/fs/tmpfs/inode_file.go index 1e4fe47d2..42a7d7b9c 100644 --- a/pkg/sentry/fs/tmpfs/inode_file.go +++ b/pkg/sentry/fs/tmpfs/inode_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go index 38be6db46..91b782540 100644 --- a/pkg/sentry/fs/tmpfs/tmpfs.go +++ b/pkg/sentry/fs/tmpfs/tmpfs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index 7c0c0b0c1..e32b05c1d 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/fs.go b/pkg/sentry/fs/tty/fs.go index d9f8f02f3..0c412eb21 100644 --- a/pkg/sentry/fs/tty/fs.go +++ b/pkg/sentry/fs/tty/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/inode.go b/pkg/sentry/fs/tty/inode.go index c0fa2b407..d5d1caafc 100644 --- a/pkg/sentry/fs/tty/inode.go +++ b/pkg/sentry/fs/tty/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/line_discipline.go b/pkg/sentry/fs/tty/line_discipline.go index 31804571e..484366f85 100644 --- a/pkg/sentry/fs/tty/line_discipline.go +++ b/pkg/sentry/fs/tty/line_discipline.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index ae7540eff..dad0cad79 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/queue.go b/pkg/sentry/fs/tty/queue.go index 01dc8d1ac..a09ca0119 100644 --- a/pkg/sentry/fs/tty/queue.go +++ b/pkg/sentry/fs/tty/queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index 4a0d4fdb9..9de3168bf 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/terminal.go b/pkg/sentry/fs/tty/terminal.go index 3cb135124..79f9d76d7 100644 --- a/pkg/sentry/fs/tty/terminal.go +++ b/pkg/sentry/fs/tty/terminal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/tty/tty_test.go b/pkg/sentry/fs/tty/tty_test.go index 32e1b1556..ad535838f 100644 --- a/pkg/sentry/fs/tty/tty_test.go +++ b/pkg/sentry/fs/tty/tty_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/getcpu_amd64.s b/pkg/sentry/hostcpu/getcpu_amd64.s index 7f6247d81..409db1450 100644 --- a/pkg/sentry/hostcpu/getcpu_amd64.s +++ b/pkg/sentry/hostcpu/getcpu_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/hostcpu.go b/pkg/sentry/hostcpu/hostcpu.go index fa46499ad..3adc847bb 100644 --- a/pkg/sentry/hostcpu/hostcpu.go +++ b/pkg/sentry/hostcpu/hostcpu.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/hostcpu/hostcpu_test.go b/pkg/sentry/hostcpu/hostcpu_test.go index a82e1a271..38de0e1f6 100644 --- a/pkg/sentry/hostcpu/hostcpu_test.go +++ b/pkg/sentry/hostcpu/hostcpu_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/context.go b/pkg/sentry/inet/context.go index 370381f41..d05e96f15 100644 --- a/pkg/sentry/inet/context.go +++ b/pkg/sentry/inet/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/inet.go b/pkg/sentry/inet/inet.go index 30ca4e0c0..8206377cc 100644 --- a/pkg/sentry/inet/inet.go +++ b/pkg/sentry/inet/inet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/inet/test_stack.go b/pkg/sentry/inet/test_stack.go index bc10926ee..05c1a1792 100644 --- a/pkg/sentry/inet/test_stack.go +++ b/pkg/sentry/inet/test_stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/abstract_socket_namespace.go b/pkg/sentry/kernel/abstract_socket_namespace.go index 45088c988..1ea2cee36 100644 --- a/pkg/sentry/kernel/abstract_socket_namespace.go +++ b/pkg/sentry/kernel/abstract_socket_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/auth.go b/pkg/sentry/kernel/auth/auth.go index c49a6b852..19f15fd36 100644 --- a/pkg/sentry/kernel/auth/auth.go +++ b/pkg/sentry/kernel/auth/auth.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/capability_set.go b/pkg/sentry/kernel/auth/capability_set.go index 5b8164c49..88d6243aa 100644 --- a/pkg/sentry/kernel/auth/capability_set.go +++ b/pkg/sentry/kernel/auth/capability_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/context.go b/pkg/sentry/kernel/auth/context.go index 914589b28..f7e945599 100644 --- a/pkg/sentry/kernel/auth/context.go +++ b/pkg/sentry/kernel/auth/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/credentials.go b/pkg/sentry/kernel/auth/credentials.go index f18f7dac9..de33f1953 100644 --- a/pkg/sentry/kernel/auth/credentials.go +++ b/pkg/sentry/kernel/auth/credentials.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id.go b/pkg/sentry/kernel/auth/id.go index 37522b018..e5bed44d7 100644 --- a/pkg/sentry/kernel/auth/id.go +++ b/pkg/sentry/kernel/auth/id.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id_map.go b/pkg/sentry/kernel/auth/id_map.go index bd0090e0f..43f439825 100644 --- a/pkg/sentry/kernel/auth/id_map.go +++ b/pkg/sentry/kernel/auth/id_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/id_map_functions.go b/pkg/sentry/kernel/auth/id_map_functions.go index 889291d96..8f1a189ec 100644 --- a/pkg/sentry/kernel/auth/id_map_functions.go +++ b/pkg/sentry/kernel/auth/id_map_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/auth/user_namespace.go b/pkg/sentry/kernel/auth/user_namespace.go index d359f3f31..5bb9c44c0 100644 --- a/pkg/sentry/kernel/auth/user_namespace.go +++ b/pkg/sentry/kernel/auth/user_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/context.go b/pkg/sentry/kernel/context.go index 261ca6f7a..b629521eb 100644 --- a/pkg/sentry/kernel/context.go +++ b/pkg/sentry/kernel/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go index a8eb114c0..9c13ecfcc 100644 --- a/pkg/sentry/kernel/epoll/epoll.go +++ b/pkg/sentry/kernel/epoll/epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll_state.go b/pkg/sentry/kernel/epoll/epoll_state.go index dabb32f49..7f3e2004a 100644 --- a/pkg/sentry/kernel/epoll/epoll_state.go +++ b/pkg/sentry/kernel/epoll/epoll_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/epoll/epoll_test.go b/pkg/sentry/kernel/epoll/epoll_test.go index bc869fc13..d89c1b745 100644 --- a/pkg/sentry/kernel/epoll/epoll_test.go +++ b/pkg/sentry/kernel/epoll/epoll_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/eventfd/eventfd.go b/pkg/sentry/kernel/eventfd/eventfd.go index a4ada0e78..26dc59a85 100644 --- a/pkg/sentry/kernel/eventfd/eventfd.go +++ b/pkg/sentry/kernel/eventfd/eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/eventfd/eventfd_test.go b/pkg/sentry/kernel/eventfd/eventfd_test.go index 71326b62f..14e8996d9 100644 --- a/pkg/sentry/kernel/eventfd/eventfd_test.go +++ b/pkg/sentry/kernel/eventfd/eventfd_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fasync/fasync.go b/pkg/sentry/kernel/fasync/fasync.go index f77339cae..aa4aac109 100644 --- a/pkg/sentry/kernel/fasync/fasync.go +++ b/pkg/sentry/kernel/fasync/fasync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fd_map.go b/pkg/sentry/kernel/fd_map.go index cad0b0a20..715f4714d 100644 --- a/pkg/sentry/kernel/fd_map.go +++ b/pkg/sentry/kernel/fd_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fd_map_test.go b/pkg/sentry/kernel/fd_map_test.go index 95123aef3..b49996137 100644 --- a/pkg/sentry/kernel/fd_map_test.go +++ b/pkg/sentry/kernel/fd_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/fs_context.go b/pkg/sentry/kernel/fs_context.go index f3f05e8f5..3cf0db280 100644 --- a/pkg/sentry/kernel/fs_context.go +++ b/pkg/sentry/kernel/fs_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/futex/futex.go b/pkg/sentry/kernel/futex/futex.go index 54b1982a0..ea69d433b 100644 --- a/pkg/sentry/kernel/futex/futex.go +++ b/pkg/sentry/kernel/futex/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/futex/futex_test.go b/pkg/sentry/kernel/futex/futex_test.go index 726c26990..ea506a29b 100644 --- a/pkg/sentry/kernel/futex/futex_test.go +++ b/pkg/sentry/kernel/futex/futex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ipc_namespace.go b/pkg/sentry/kernel/ipc_namespace.go index 5eef49f59..9ceb9bd92 100644 --- a/pkg/sentry/kernel/ipc_namespace.go +++ b/pkg/sentry/kernel/ipc_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kdefs/kdefs.go b/pkg/sentry/kernel/kdefs/kdefs.go index bbb476544..8eafe810b 100644 --- a/pkg/sentry/kernel/kdefs/kdefs.go +++ b/pkg/sentry/kernel/kdefs/kdefs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index 5d6856f3c..bad558d48 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/kernel_state.go b/pkg/sentry/kernel/kernel_state.go index bb2d5102d..a0a69b498 100644 --- a/pkg/sentry/kernel/kernel_state.go +++ b/pkg/sentry/kernel/kernel_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/memevent/memory_events.go b/pkg/sentry/kernel/memevent/memory_events.go index f7a183a1d..f05ef1b64 100644 --- a/pkg/sentry/kernel/memevent/memory_events.go +++ b/pkg/sentry/kernel/memevent/memory_events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/memevent/memory_events.proto b/pkg/sentry/kernel/memevent/memory_events.proto index abc565054..43b8deb76 100644 --- a/pkg/sentry/kernel/memevent/memory_events.proto +++ b/pkg/sentry/kernel/memevent/memory_events.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pending_signals.go b/pkg/sentry/kernel/pending_signals.go index bb5db0309..373e11772 100644 --- a/pkg/sentry/kernel/pending_signals.go +++ b/pkg/sentry/kernel/pending_signals.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pending_signals_state.go b/pkg/sentry/kernel/pending_signals_state.go index 6d90ed033..72be6702f 100644 --- a/pkg/sentry/kernel/pending_signals_state.go +++ b/pkg/sentry/kernel/pending_signals_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/buffers.go b/pkg/sentry/kernel/pipe/buffers.go index a82e45c3f..fa8045910 100644 --- a/pkg/sentry/kernel/pipe/buffers.go +++ b/pkg/sentry/kernel/pipe/buffers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/device.go b/pkg/sentry/kernel/pipe/device.go index 8d383577a..eec5c5de8 100644 --- a/pkg/sentry/kernel/pipe/device.go +++ b/pkg/sentry/kernel/pipe/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/node.go b/pkg/sentry/kernel/pipe/node.go index 23d692da1..4b0e00b85 100644 --- a/pkg/sentry/kernel/pipe/node.go +++ b/pkg/sentry/kernel/pipe/node.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/node_test.go b/pkg/sentry/kernel/pipe/node_test.go index cc1ebf4f6..eda551594 100644 --- a/pkg/sentry/kernel/pipe/node_test.go +++ b/pkg/sentry/kernel/pipe/node_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/pipe.go b/pkg/sentry/kernel/pipe/pipe.go index ced2559a7..126054826 100644 --- a/pkg/sentry/kernel/pipe/pipe.go +++ b/pkg/sentry/kernel/pipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/pipe_test.go b/pkg/sentry/kernel/pipe/pipe_test.go index 49ef8c8ac..3b9895927 100644 --- a/pkg/sentry/kernel/pipe/pipe_test.go +++ b/pkg/sentry/kernel/pipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/reader.go b/pkg/sentry/kernel/pipe/reader.go index 1fa5e9a32..f27379969 100644 --- a/pkg/sentry/kernel/pipe/reader.go +++ b/pkg/sentry/kernel/pipe/reader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/reader_writer.go b/pkg/sentry/kernel/pipe/reader_writer.go index 82607367b..63efc5bbe 100644 --- a/pkg/sentry/kernel/pipe/reader_writer.go +++ b/pkg/sentry/kernel/pipe/reader_writer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/pipe/writer.go b/pkg/sentry/kernel/pipe/writer.go index d93324b53..6fea9769c 100644 --- a/pkg/sentry/kernel/pipe/writer.go +++ b/pkg/sentry/kernel/pipe/writer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/posixtimer.go b/pkg/sentry/kernel/posixtimer.go index 0ab958529..40b5acca3 100644 --- a/pkg/sentry/kernel/posixtimer.go +++ b/pkg/sentry/kernel/posixtimer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ptrace.go b/pkg/sentry/kernel/ptrace.go index 9fe28f435..20bac2b70 100644 --- a/pkg/sentry/kernel/ptrace.go +++ b/pkg/sentry/kernel/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/rseq.go b/pkg/sentry/kernel/rseq.go index 1f3de58e3..46b03c700 100644 --- a/pkg/sentry/kernel/rseq.go +++ b/pkg/sentry/kernel/rseq.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/cpuset.go b/pkg/sentry/kernel/sched/cpuset.go index 0a97603f0..69aee9127 100644 --- a/pkg/sentry/kernel/sched/cpuset.go +++ b/pkg/sentry/kernel/sched/cpuset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/cpuset_test.go b/pkg/sentry/kernel/sched/cpuset_test.go index 8a6e12958..a036ed513 100644 --- a/pkg/sentry/kernel/sched/cpuset_test.go +++ b/pkg/sentry/kernel/sched/cpuset_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sched/sched.go b/pkg/sentry/kernel/sched/sched.go index f1de1da60..e59909baf 100644 --- a/pkg/sentry/kernel/sched/sched.go +++ b/pkg/sentry/kernel/sched/sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/seccomp.go b/pkg/sentry/kernel/seccomp.go index d77c05e2f..37dd3e4c9 100644 --- a/pkg/sentry/kernel/seccomp.go +++ b/pkg/sentry/kernel/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/semaphore/semaphore.go b/pkg/sentry/kernel/semaphore/semaphore.go index aa07946cf..232a276dc 100644 --- a/pkg/sentry/kernel/semaphore/semaphore.go +++ b/pkg/sentry/kernel/semaphore/semaphore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/semaphore/semaphore_test.go b/pkg/sentry/kernel/semaphore/semaphore_test.go index f9eb382e9..5f886bf31 100644 --- a/pkg/sentry/kernel/semaphore/semaphore_test.go +++ b/pkg/sentry/kernel/semaphore/semaphore_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/sessions.go b/pkg/sentry/kernel/sessions.go index a9b4e7647..78a5b4063 100644 --- a/pkg/sentry/kernel/sessions.go +++ b/pkg/sentry/kernel/sessions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/shm/device.go b/pkg/sentry/kernel/shm/device.go index b0dacdbe0..bbc653ed8 100644 --- a/pkg/sentry/kernel/shm/device.go +++ b/pkg/sentry/kernel/shm/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/shm/shm.go b/pkg/sentry/kernel/shm/shm.go index 77973951e..8d0d14e45 100644 --- a/pkg/sentry/kernel/shm/shm.go +++ b/pkg/sentry/kernel/shm/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/signal.go b/pkg/sentry/kernel/signal.go index e3a2a777a..b066df132 100644 --- a/pkg/sentry/kernel/signal.go +++ b/pkg/sentry/kernel/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/signal_handlers.go b/pkg/sentry/kernel/signal_handlers.go index 3649f5e4d..3f1ac9898 100644 --- a/pkg/sentry/kernel/signal_handlers.go +++ b/pkg/sentry/kernel/signal_handlers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syscalls.go b/pkg/sentry/kernel/syscalls.go index 4c7811b6c..19b711e9c 100644 --- a/pkg/sentry/kernel/syscalls.go +++ b/pkg/sentry/kernel/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syscalls_state.go b/pkg/sentry/kernel/syscalls_state.go index 826809a70..981455d46 100644 --- a/pkg/sentry/kernel/syscalls_state.go +++ b/pkg/sentry/kernel/syscalls_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/syslog.go b/pkg/sentry/kernel/syslog.go index 6531bd5d2..2aecf3eea 100644 --- a/pkg/sentry/kernel/syslog.go +++ b/pkg/sentry/kernel/syslog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/table_test.go b/pkg/sentry/kernel/table_test.go index 71ca75555..3b29d3c6a 100644 --- a/pkg/sentry/kernel/table_test.go +++ b/pkg/sentry/kernel/table_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go index 4f0b7fe3f..e22ec768d 100644 --- a/pkg/sentry/kernel/task.go +++ b/pkg/sentry/kernel/task.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_acct.go b/pkg/sentry/kernel/task_acct.go index d2052921e..24230af89 100644 --- a/pkg/sentry/kernel/task_acct.go +++ b/pkg/sentry/kernel/task_acct.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_block.go b/pkg/sentry/kernel/task_block.go index 6dc7b938e..e5027e551 100644 --- a/pkg/sentry/kernel/task_block.go +++ b/pkg/sentry/kernel/task_block.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go index de3aef40d..755fe0370 100644 --- a/pkg/sentry/kernel/task_clone.go +++ b/pkg/sentry/kernel/task_clone.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_context.go b/pkg/sentry/kernel/task_context.go index d2df7e9d1..45b8d2b04 100644 --- a/pkg/sentry/kernel/task_context.go +++ b/pkg/sentry/kernel/task_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_exec.go b/pkg/sentry/kernel/task_exec.go index 1b760aba4..a9b74da8e 100644 --- a/pkg/sentry/kernel/task_exec.go +++ b/pkg/sentry/kernel/task_exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index 65969ca9b..44fbb487c 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_futex.go b/pkg/sentry/kernel/task_futex.go index 62ebbcb0d..5a11ca3df 100644 --- a/pkg/sentry/kernel/task_futex.go +++ b/pkg/sentry/kernel/task_futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_identity.go b/pkg/sentry/kernel/task_identity.go index b0921b2eb..8f90ed786 100644 --- a/pkg/sentry/kernel/task_identity.go +++ b/pkg/sentry/kernel/task_identity.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_log.go b/pkg/sentry/kernel/task_log.go index 1769da210..f4c881c2d 100644 --- a/pkg/sentry/kernel/task_log.go +++ b/pkg/sentry/kernel/task_log.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_net.go b/pkg/sentry/kernel/task_net.go index 4df2e53d3..fc7cefc1f 100644 --- a/pkg/sentry/kernel/task_net.go +++ b/pkg/sentry/kernel/task_net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_run.go b/pkg/sentry/kernel/task_run.go index 49ac933b7..596b9aa16 100644 --- a/pkg/sentry/kernel/task_run.go +++ b/pkg/sentry/kernel/task_run.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_sched.go b/pkg/sentry/kernel/task_sched.go index 19dcc963a..3b3cdc24a 100644 --- a/pkg/sentry/kernel/task_sched.go +++ b/pkg/sentry/kernel/task_sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go index e2925a708..fe24f7542 100644 --- a/pkg/sentry/kernel/task_signals.go +++ b/pkg/sentry/kernel/task_signals.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go index 6c8d7d316..c82a32c78 100644 --- a/pkg/sentry/kernel/task_start.go +++ b/pkg/sentry/kernel/task_start.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_stop.go b/pkg/sentry/kernel/task_stop.go index feaf6cae4..36846484c 100644 --- a/pkg/sentry/kernel/task_stop.go +++ b/pkg/sentry/kernel/task_stop.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_syscall.go b/pkg/sentry/kernel/task_syscall.go index f0373c375..0318adb35 100644 --- a/pkg/sentry/kernel/task_syscall.go +++ b/pkg/sentry/kernel/task_syscall.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_test.go b/pkg/sentry/kernel/task_test.go index 82ef858a1..3f37f505d 100644 --- a/pkg/sentry/kernel/task_test.go +++ b/pkg/sentry/kernel/task_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/task_usermem.go b/pkg/sentry/kernel/task_usermem.go index 2b4954869..c8e973bd5 100644 --- a/pkg/sentry/kernel/task_usermem.go +++ b/pkg/sentry/kernel/task_usermem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/thread_group.go b/pkg/sentry/kernel/thread_group.go index dfff7b52d..d7652f57c 100644 --- a/pkg/sentry/kernel/thread_group.go +++ b/pkg/sentry/kernel/thread_group.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/threads.go b/pkg/sentry/kernel/threads.go index 4e3d19e97..bdb907905 100644 --- a/pkg/sentry/kernel/threads.go +++ b/pkg/sentry/kernel/threads.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/time/context.go b/pkg/sentry/kernel/time/context.go index ac4dc01d8..3675ea20d 100644 --- a/pkg/sentry/kernel/time/context.go +++ b/pkg/sentry/kernel/time/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/time/time.go b/pkg/sentry/kernel/time/time.go index 52e0dfba1..ca0f4ba2e 100644 --- a/pkg/sentry/kernel/time/time.go +++ b/pkg/sentry/kernel/time/time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper.go b/pkg/sentry/kernel/timekeeper.go index 2167f3efe..6bff80f13 100644 --- a/pkg/sentry/kernel/timekeeper.go +++ b/pkg/sentry/kernel/timekeeper.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper_state.go b/pkg/sentry/kernel/timekeeper_state.go index 2e7fed4d8..f3a3ed543 100644 --- a/pkg/sentry/kernel/timekeeper_state.go +++ b/pkg/sentry/kernel/timekeeper_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/timekeeper_test.go b/pkg/sentry/kernel/timekeeper_test.go index 34a5cec27..71674c21c 100644 --- a/pkg/sentry/kernel/timekeeper_test.go +++ b/pkg/sentry/kernel/timekeeper_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/uts_namespace.go b/pkg/sentry/kernel/uts_namespace.go index 7e0fe0d21..ed5f0c031 100644 --- a/pkg/sentry/kernel/uts_namespace.go +++ b/pkg/sentry/kernel/uts_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/vdso.go b/pkg/sentry/kernel/vdso.go index 971e8bc59..0ec858a4a 100644 --- a/pkg/sentry/kernel/vdso.go +++ b/pkg/sentry/kernel/vdso.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/version.go b/pkg/sentry/kernel/version.go index a9e84673f..72bb0f93c 100644 --- a/pkg/sentry/kernel/version.go +++ b/pkg/sentry/kernel/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/context.go b/pkg/sentry/limits/context.go index 75e97bf92..bf413eb7d 100644 --- a/pkg/sentry/limits/context.go +++ b/pkg/sentry/limits/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/limits.go b/pkg/sentry/limits/limits.go index 02c8b60e3..ba0b7d4fd 100644 --- a/pkg/sentry/limits/limits.go +++ b/pkg/sentry/limits/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/limits_test.go b/pkg/sentry/limits/limits_test.go index dd6f80750..d41f62554 100644 --- a/pkg/sentry/limits/limits_test.go +++ b/pkg/sentry/limits/limits_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/limits/linux.go b/pkg/sentry/limits/linux.go index 8e6a24341..511db6733 100644 --- a/pkg/sentry/limits/linux.go +++ b/pkg/sentry/limits/linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/elf.go b/pkg/sentry/loader/elf.go index 849be5a3d..9b1e81dc9 100644 --- a/pkg/sentry/loader/elf.go +++ b/pkg/sentry/loader/elf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/interpreter.go b/pkg/sentry/loader/interpreter.go index 54534952b..06a3c7156 100644 --- a/pkg/sentry/loader/interpreter.go +++ b/pkg/sentry/loader/interpreter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/loader.go b/pkg/sentry/loader/loader.go index 62b39e52b..d1417c4f1 100644 --- a/pkg/sentry/loader/loader.go +++ b/pkg/sentry/loader/loader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/vdso.go b/pkg/sentry/loader/vdso.go index a06e27ac9..437cc5da1 100644 --- a/pkg/sentry/loader/vdso.go +++ b/pkg/sentry/loader/vdso.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/loader/vdso_state.go b/pkg/sentry/loader/vdso_state.go index dc71e1c2d..b327f0e1e 100644 --- a/pkg/sentry/loader/vdso_state.go +++ b/pkg/sentry/loader/vdso_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/mapping_set.go b/pkg/sentry/memmap/mapping_set.go index c9483905d..33cf16f91 100644 --- a/pkg/sentry/memmap/mapping_set.go +++ b/pkg/sentry/memmap/mapping_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/mapping_set_test.go b/pkg/sentry/memmap/mapping_set_test.go index 10668d404..49ee34548 100644 --- a/pkg/sentry/memmap/mapping_set_test.go +++ b/pkg/sentry/memmap/mapping_set_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memmap/memmap.go b/pkg/sentry/memmap/memmap.go index cdc5f2b27..05349a77f 100644 --- a/pkg/sentry/memmap/memmap.go +++ b/pkg/sentry/memmap/memmap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memutil/memutil.go b/pkg/sentry/memutil/memutil.go index 4f245cf3c..286d50ca4 100644 --- a/pkg/sentry/memutil/memutil.go +++ b/pkg/sentry/memutil/memutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/memutil/memutil_unsafe.go b/pkg/sentry/memutil/memutil_unsafe.go index 32c27eb2f..8d9fc64fb 100644 --- a/pkg/sentry/memutil/memutil_unsafe.go +++ b/pkg/sentry/memutil/memutil_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/address_space.go b/pkg/sentry/mm/address_space.go index 27554f163..7488f7c4a 100644 --- a/pkg/sentry/mm/address_space.go +++ b/pkg/sentry/mm/address_space.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/aio_context.go b/pkg/sentry/mm/aio_context.go index b42156d45..87942af0e 100644 --- a/pkg/sentry/mm/aio_context.go +++ b/pkg/sentry/mm/aio_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/aio_context_state.go b/pkg/sentry/mm/aio_context_state.go index 1a5e56f8e..192a6f744 100644 --- a/pkg/sentry/mm/aio_context_state.go +++ b/pkg/sentry/mm/aio_context_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/debug.go b/pkg/sentry/mm/debug.go index 56d0490f0..d341b9c07 100644 --- a/pkg/sentry/mm/debug.go +++ b/pkg/sentry/mm/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/io.go b/pkg/sentry/mm/io.go index 6741db594..6600ddd78 100644 --- a/pkg/sentry/mm/io.go +++ b/pkg/sentry/mm/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/lifecycle.go b/pkg/sentry/mm/lifecycle.go index a4b5cb443..b248b76e7 100644 --- a/pkg/sentry/mm/lifecycle.go +++ b/pkg/sentry/mm/lifecycle.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/metadata.go b/pkg/sentry/mm/metadata.go index 32d5e2ff6..5ef1ba0b1 100644 --- a/pkg/sentry/mm/metadata.go +++ b/pkg/sentry/mm/metadata.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/mm.go b/pkg/sentry/mm/mm.go index 3299ae164..aab697f9e 100644 --- a/pkg/sentry/mm/mm.go +++ b/pkg/sentry/mm/mm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/mm_test.go b/pkg/sentry/mm/mm_test.go index b47aa7263..f2db43196 100644 --- a/pkg/sentry/mm/mm_test.go +++ b/pkg/sentry/mm/mm_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/pma.go b/pkg/sentry/mm/pma.go index 9febb25ac..5690fe6b4 100644 --- a/pkg/sentry/mm/pma.go +++ b/pkg/sentry/mm/pma.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/proc_pid_maps.go b/pkg/sentry/mm/proc_pid_maps.go index 5840b257c..0bf1cdb51 100644 --- a/pkg/sentry/mm/proc_pid_maps.go +++ b/pkg/sentry/mm/proc_pid_maps.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/save_restore.go b/pkg/sentry/mm/save_restore.go index 36fed8f1c..6e7080a84 100644 --- a/pkg/sentry/mm/save_restore.go +++ b/pkg/sentry/mm/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/shm.go b/pkg/sentry/mm/shm.go index bab137a5a..3bc48c7e7 100644 --- a/pkg/sentry/mm/shm.go +++ b/pkg/sentry/mm/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/special_mappable.go b/pkg/sentry/mm/special_mappable.go index 5d7bd33bd..e511472f4 100644 --- a/pkg/sentry/mm/special_mappable.go +++ b/pkg/sentry/mm/special_mappable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/syscalls.go b/pkg/sentry/mm/syscalls.go index b0622b0c3..a721cc456 100644 --- a/pkg/sentry/mm/syscalls.go +++ b/pkg/sentry/mm/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/mm/vma.go b/pkg/sentry/mm/vma.go index b81e861f1..dafdbd0e4 100644 --- a/pkg/sentry/mm/vma.go +++ b/pkg/sentry/mm/vma.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/context.go b/pkg/sentry/platform/context.go index 0d200a5e2..cca21a23e 100644 --- a/pkg/sentry/platform/context.go +++ b/pkg/sentry/platform/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/filemem/filemem.go b/pkg/sentry/platform/filemem/filemem.go index f278c8d63..97da31e70 100644 --- a/pkg/sentry/platform/filemem/filemem.go +++ b/pkg/sentry/platform/filemem/filemem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/filemem/filemem_state.go b/pkg/sentry/platform/filemem/filemem_state.go index e28e021c9..964e2aaaa 100644 --- a/pkg/sentry/platform/filemem/filemem_state.go +++ b/pkg/sentry/platform/filemem/filemem_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/filemem/filemem_test.go b/pkg/sentry/platform/filemem/filemem_test.go index 4b165dc48..9becec25f 100644 --- a/pkg/sentry/platform/filemem/filemem_test.go +++ b/pkg/sentry/platform/filemem/filemem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/filemem/filemem_unsafe.go b/pkg/sentry/platform/filemem/filemem_unsafe.go index a23b9825a..776aed74d 100644 --- a/pkg/sentry/platform/filemem/filemem_unsafe.go +++ b/pkg/sentry/platform/filemem/filemem_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/interrupt/interrupt.go b/pkg/sentry/platform/interrupt/interrupt.go index ca4f42087..9c83f41eb 100644 --- a/pkg/sentry/platform/interrupt/interrupt.go +++ b/pkg/sentry/platform/interrupt/interrupt.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/interrupt/interrupt_test.go b/pkg/sentry/platform/interrupt/interrupt_test.go index 7c49eeea6..fb3284395 100644 --- a/pkg/sentry/platform/interrupt/interrupt_test.go +++ b/pkg/sentry/platform/interrupt/interrupt_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/address_space.go b/pkg/sentry/platform/kvm/address_space.go index c4293c517..72e897a9a 100644 --- a/pkg/sentry/platform/kvm/address_space.go +++ b/pkg/sentry/platform/kvm/address_space.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/allocator.go b/pkg/sentry/platform/kvm/allocator.go index f5cebd5b3..b25cad155 100644 --- a/pkg/sentry/platform/kvm/allocator.go +++ b/pkg/sentry/platform/kvm/allocator.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill.go b/pkg/sentry/platform/kvm/bluepill.go index ecc33d7dd..9f1c9510b 100644 --- a/pkg/sentry/platform/kvm/bluepill.go +++ b/pkg/sentry/platform/kvm/bluepill.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.go b/pkg/sentry/platform/kvm/bluepill_amd64.go index b364e3ef7..f013d1dc9 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64.go +++ b/pkg/sentry/platform/kvm/bluepill_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.s b/pkg/sentry/platform/kvm/bluepill_amd64.s index 0881bd5f5..ec017f6c2 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64.s +++ b/pkg/sentry/platform/kvm/bluepill_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go b/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go index 61ca61dcb..cd00a47f2 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_fault.go b/pkg/sentry/platform/kvm/bluepill_fault.go index 8650cd78f..e79a30ef2 100644 --- a/pkg/sentry/platform/kvm/bluepill_fault.go +++ b/pkg/sentry/platform/kvm/bluepill_fault.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/bluepill_unsafe.go b/pkg/sentry/platform/kvm/bluepill_unsafe.go index 216d4b4b6..747a95997 100644 --- a/pkg/sentry/platform/kvm/bluepill_unsafe.go +++ b/pkg/sentry/platform/kvm/bluepill_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/context.go b/pkg/sentry/platform/kvm/context.go index aac84febf..be902be88 100644 --- a/pkg/sentry/platform/kvm/context.go +++ b/pkg/sentry/platform/kvm/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/host_map.go b/pkg/sentry/platform/kvm/host_map.go index fc16ad2de..ee6a1a42d 100644 --- a/pkg/sentry/platform/kvm/host_map.go +++ b/pkg/sentry/platform/kvm/host_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm.go b/pkg/sentry/platform/kvm/kvm.go index 0c4dff308..d4f50024d 100644 --- a/pkg/sentry/platform/kvm/kvm.go +++ b/pkg/sentry/platform/kvm/kvm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_amd64.go b/pkg/sentry/platform/kvm/kvm_amd64.go index 3d56ed895..70d0ac63b 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64.go +++ b/pkg/sentry/platform/kvm/kvm_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go index 476e783a0..c0a0af92d 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_const.go b/pkg/sentry/platform/kvm/kvm_const.go index ca44c31b3..8c53c6f06 100644 --- a/pkg/sentry/platform/kvm/kvm_const.go +++ b/pkg/sentry/platform/kvm/kvm_const.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/kvm_test.go b/pkg/sentry/platform/kvm/kvm_test.go index 52448839f..45eeb96ff 100644 --- a/pkg/sentry/platform/kvm/kvm_test.go +++ b/pkg/sentry/platform/kvm/kvm_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine.go b/pkg/sentry/platform/kvm/machine.go index 9f60b6b31..fc7ad258f 100644 --- a/pkg/sentry/platform/kvm/machine.go +++ b/pkg/sentry/platform/kvm/machine.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_amd64.go b/pkg/sentry/platform/kvm/machine_amd64.go index bcd29a947..e0aec42b8 100644 --- a/pkg/sentry/platform/kvm/machine_amd64.go +++ b/pkg/sentry/platform/kvm/machine_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go index 8b9041f13..50e513f3b 100644 --- a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/machine_unsafe.go b/pkg/sentry/platform/kvm/machine_unsafe.go index 86323c891..4f5b01321 100644 --- a/pkg/sentry/platform/kvm/machine_unsafe.go +++ b/pkg/sentry/platform/kvm/machine_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/physical_map.go b/pkg/sentry/platform/kvm/physical_map.go index 81a98656d..b908cae6a 100644 --- a/pkg/sentry/platform/kvm/physical_map.go +++ b/pkg/sentry/platform/kvm/physical_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil.go b/pkg/sentry/platform/kvm/testutil/testutil.go index 8a614e25d..0d496561d 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil.go +++ b/pkg/sentry/platform/kvm/testutil/testutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil_amd64.go b/pkg/sentry/platform/kvm/testutil/testutil_amd64.go index 39286a0af..fcba33813 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil_amd64.go +++ b/pkg/sentry/platform/kvm/testutil/testutil_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/testutil/testutil_amd64.s b/pkg/sentry/platform/kvm/testutil/testutil_amd64.s index 3b5ad8817..f1da41a44 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil_amd64.s +++ b/pkg/sentry/platform/kvm/testutil/testutil_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/virtual_map.go b/pkg/sentry/platform/kvm/virtual_map.go index 0d3fbe043..0343e9267 100644 --- a/pkg/sentry/platform/kvm/virtual_map.go +++ b/pkg/sentry/platform/kvm/virtual_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/kvm/virtual_map_test.go b/pkg/sentry/platform/kvm/virtual_map_test.go index 7875bd3e9..935e0eb93 100644 --- a/pkg/sentry/platform/kvm/virtual_map_test.go +++ b/pkg/sentry/platform/kvm/virtual_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/mmap_min_addr.go b/pkg/sentry/platform/mmap_min_addr.go index 6398e5e01..1bcc1f8e9 100644 --- a/pkg/sentry/platform/mmap_min_addr.go +++ b/pkg/sentry/platform/mmap_min_addr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/platform.go b/pkg/sentry/platform/platform.go index 8a1620d93..f16588e6e 100644 --- a/pkg/sentry/platform/platform.go +++ b/pkg/sentry/platform/platform.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid.go b/pkg/sentry/platform/procid/procid.go index 5f861908f..3f49ab093 100644 --- a/pkg/sentry/platform/procid/procid.go +++ b/pkg/sentry/platform/procid/procid.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_amd64.s b/pkg/sentry/platform/procid/procid_amd64.s index 5b1ba1f24..fd88ce82e 100644 --- a/pkg/sentry/platform/procid/procid_amd64.s +++ b/pkg/sentry/platform/procid/procid_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_net_test.go b/pkg/sentry/platform/procid/procid_net_test.go index 2d1605a08..e8dcc479d 100644 --- a/pkg/sentry/platform/procid/procid_net_test.go +++ b/pkg/sentry/platform/procid/procid_net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/procid/procid_test.go b/pkg/sentry/platform/procid/procid_test.go index 5e44da36f..7a57c7cdc 100644 --- a/pkg/sentry/platform/procid/procid_test.go +++ b/pkg/sentry/platform/procid/procid_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/ptrace.go b/pkg/sentry/platform/ptrace/ptrace.go index 4f20716f7..00d92b092 100644 --- a/pkg/sentry/platform/ptrace/ptrace.go +++ b/pkg/sentry/platform/ptrace/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/ptrace_unsafe.go b/pkg/sentry/platform/ptrace/ptrace_unsafe.go index 46a8bda8e..7a3cb8f49 100644 --- a/pkg/sentry/platform/ptrace/ptrace_unsafe.go +++ b/pkg/sentry/platform/ptrace/ptrace_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/stub_amd64.s b/pkg/sentry/platform/ptrace/stub_amd64.s index 9bf87b6f6..63f98e40d 100644 --- a/pkg/sentry/platform/ptrace/stub_amd64.s +++ b/pkg/sentry/platform/ptrace/stub_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/stub_unsafe.go b/pkg/sentry/platform/ptrace/stub_unsafe.go index c868a2d68..48c16c4a1 100644 --- a/pkg/sentry/platform/ptrace/stub_unsafe.go +++ b/pkg/sentry/platform/ptrace/stub_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess.go b/pkg/sentry/platform/ptrace/subprocess.go index 6d5ad6b71..6a9da5db8 100644 --- a/pkg/sentry/platform/ptrace/subprocess.go +++ b/pkg/sentry/platform/ptrace/subprocess.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_amd64.go b/pkg/sentry/platform/ptrace/subprocess_amd64.go index c38dc1ff8..d23a1133e 100644 --- a/pkg/sentry/platform/ptrace/subprocess_amd64.go +++ b/pkg/sentry/platform/ptrace/subprocess_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_linux.go b/pkg/sentry/platform/ptrace/subprocess_linux.go index 53adadadd..7523487e7 100644 --- a/pkg/sentry/platform/ptrace/subprocess_linux.go +++ b/pkg/sentry/platform/ptrace/subprocess_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go index 697431472..0c9263060 100644 --- a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go +++ b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ptrace/subprocess_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_unsafe.go index fe41641d3..ca6c4ac97 100644 --- a/pkg/sentry/platform/ptrace/subprocess_unsafe.go +++ b/pkg/sentry/platform/ptrace/subprocess_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/defs.go b/pkg/sentry/platform/ring0/defs.go index f09d045eb..18137e55d 100644 --- a/pkg/sentry/platform/ring0/defs.go +++ b/pkg/sentry/platform/ring0/defs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/defs_amd64.go b/pkg/sentry/platform/ring0/defs_amd64.go index 84819f132..67242b92b 100644 --- a/pkg/sentry/platform/ring0/defs_amd64.go +++ b/pkg/sentry/platform/ring0/defs_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/entry_amd64.go b/pkg/sentry/platform/ring0/entry_amd64.go index a3e992e0d..4a9affe64 100644 --- a/pkg/sentry/platform/ring0/entry_amd64.go +++ b/pkg/sentry/platform/ring0/entry_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/entry_amd64.s b/pkg/sentry/platform/ring0/entry_amd64.s index 08c15ad65..d48fbd2d1 100644 --- a/pkg/sentry/platform/ring0/entry_amd64.s +++ b/pkg/sentry/platform/ring0/entry_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/gen_offsets/main.go b/pkg/sentry/platform/ring0/gen_offsets/main.go index ffa7eaf77..11c49855f 100644 --- a/pkg/sentry/platform/ring0/gen_offsets/main.go +++ b/pkg/sentry/platform/ring0/gen_offsets/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel.go b/pkg/sentry/platform/ring0/kernel.go index 62e67005e..e70eafde2 100644 --- a/pkg/sentry/platform/ring0/kernel.go +++ b/pkg/sentry/platform/ring0/kernel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel_amd64.go b/pkg/sentry/platform/ring0/kernel_amd64.go index 0d2b0f7dc..ab562bca7 100644 --- a/pkg/sentry/platform/ring0/kernel_amd64.go +++ b/pkg/sentry/platform/ring0/kernel_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/kernel_unsafe.go b/pkg/sentry/platform/ring0/kernel_unsafe.go index cfb3ad853..faf4240e5 100644 --- a/pkg/sentry/platform/ring0/kernel_unsafe.go +++ b/pkg/sentry/platform/ring0/kernel_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/lib_amd64.go b/pkg/sentry/platform/ring0/lib_amd64.go index 989e3e383..2b95a0141 100644 --- a/pkg/sentry/platform/ring0/lib_amd64.go +++ b/pkg/sentry/platform/ring0/lib_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/lib_amd64.s b/pkg/sentry/platform/ring0/lib_amd64.s index 6f143ea5a..98a130525 100644 --- a/pkg/sentry/platform/ring0/lib_amd64.s +++ b/pkg/sentry/platform/ring0/lib_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/offsets_amd64.go b/pkg/sentry/platform/ring0/offsets_amd64.go index ca5fd456b..753d31ef8 100644 --- a/pkg/sentry/platform/ring0/offsets_amd64.go +++ b/pkg/sentry/platform/ring0/offsets_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/allocator.go b/pkg/sentry/platform/ring0/pagetables/allocator.go index 049fd0247..ee6e90a11 100644 --- a/pkg/sentry/platform/ring0/pagetables/allocator.go +++ b/pkg/sentry/platform/ring0/pagetables/allocator.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go b/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go index aca778913..f48647b3a 100644 --- a/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go +++ b/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables.go b/pkg/sentry/platform/ring0/pagetables/pagetables.go index ff5787f89..c7207ec18 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go index 878463018..746f614e5 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go index a7f2ad9a4..2f82c4353 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_test.go b/pkg/sentry/platform/ring0/pagetables/pagetables_test.go index dca3f69ef..3e5dc7dc7 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_test.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go b/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go index ca49d20f8..6bd8c3584 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/pcids_x86.go b/pkg/sentry/platform/ring0/pagetables/pcids_x86.go index fa068e35e..0d9a51aa5 100644 --- a/pkg/sentry/platform/ring0/pagetables/pcids_x86.go +++ b/pkg/sentry/platform/ring0/pagetables/pcids_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/pagetables/walker_amd64.go b/pkg/sentry/platform/ring0/pagetables/walker_amd64.go index afa4d473a..c4c71d23e 100644 --- a/pkg/sentry/platform/ring0/pagetables/walker_amd64.go +++ b/pkg/sentry/platform/ring0/pagetables/walker_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/ring0.go b/pkg/sentry/platform/ring0/ring0.go index 4991031c5..10c51e88d 100644 --- a/pkg/sentry/platform/ring0/ring0.go +++ b/pkg/sentry/platform/ring0/ring0.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/ring0/x86.go b/pkg/sentry/platform/ring0/x86.go index f489fcecb..7c88010d8 100644 --- a/pkg/sentry/platform/ring0/x86.go +++ b/pkg/sentry/platform/ring0/x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/atomic_amd64.s b/pkg/sentry/platform/safecopy/atomic_amd64.s index 69947dec3..873ffa046 100644 --- a/pkg/sentry/platform/safecopy/atomic_amd64.s +++ b/pkg/sentry/platform/safecopy/atomic_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/memclr_amd64.s b/pkg/sentry/platform/safecopy/memclr_amd64.s index 7d1019f60..488b6e666 100644 --- a/pkg/sentry/platform/safecopy/memclr_amd64.s +++ b/pkg/sentry/platform/safecopy/memclr_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/memcpy_amd64.s b/pkg/sentry/platform/safecopy/memcpy_amd64.s index 96ef2eefc..0bf26fd7b 100644 --- a/pkg/sentry/platform/safecopy/memcpy_amd64.s +++ b/pkg/sentry/platform/safecopy/memcpy_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy.go b/pkg/sentry/platform/safecopy/safecopy.go index 90a2aad7b..c60f73103 100644 --- a/pkg/sentry/platform/safecopy/safecopy.go +++ b/pkg/sentry/platform/safecopy/safecopy.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy_test.go b/pkg/sentry/platform/safecopy/safecopy_test.go index 67df36121..1a682d28a 100644 --- a/pkg/sentry/platform/safecopy/safecopy_test.go +++ b/pkg/sentry/platform/safecopy/safecopy_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/safecopy_unsafe.go b/pkg/sentry/platform/safecopy/safecopy_unsafe.go index 72f243f8d..df1c35b66 100644 --- a/pkg/sentry/platform/safecopy/safecopy_unsafe.go +++ b/pkg/sentry/platform/safecopy/safecopy_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/platform/safecopy/sighandler_amd64.s b/pkg/sentry/platform/safecopy/sighandler_amd64.s index a65cb0c26..06614f1b4 100644 --- a/pkg/sentry/platform/safecopy/sighandler_amd64.s +++ b/pkg/sentry/platform/safecopy/sighandler_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/block_unsafe.go b/pkg/sentry/safemem/block_unsafe.go index 0b58f6497..e91ff66ae 100644 --- a/pkg/sentry/safemem/block_unsafe.go +++ b/pkg/sentry/safemem/block_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/io.go b/pkg/sentry/safemem/io.go index fd917648b..6cb52439f 100644 --- a/pkg/sentry/safemem/io.go +++ b/pkg/sentry/safemem/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/io_test.go b/pkg/sentry/safemem/io_test.go index edac4c1d7..2eda8c3bb 100644 --- a/pkg/sentry/safemem/io_test.go +++ b/pkg/sentry/safemem/io_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/safemem.go b/pkg/sentry/safemem/safemem.go index 2f8002004..090932d3e 100644 --- a/pkg/sentry/safemem/safemem.go +++ b/pkg/sentry/safemem/safemem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/seq_test.go b/pkg/sentry/safemem/seq_test.go index 3e83b3851..fddcaf714 100644 --- a/pkg/sentry/safemem/seq_test.go +++ b/pkg/sentry/safemem/seq_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/safemem/seq_unsafe.go b/pkg/sentry/safemem/seq_unsafe.go index e0d29a0b3..83a6b7183 100644 --- a/pkg/sentry/safemem/seq_unsafe.go +++ b/pkg/sentry/safemem/seq_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/sighandling/sighandling.go b/pkg/sentry/sighandling/sighandling.go index 29bcf55ab..6b5d5f993 100644 --- a/pkg/sentry/sighandling/sighandling.go +++ b/pkg/sentry/sighandling/sighandling.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/sighandling/sighandling_unsafe.go b/pkg/sentry/sighandling/sighandling_unsafe.go index a455b919f..5913d47a8 100644 --- a/pkg/sentry/sighandling/sighandling_unsafe.go +++ b/pkg/sentry/sighandling/sighandling_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/control/control.go b/pkg/sentry/socket/control/control.go index db97e95f2..d44f5e88a 100644 --- a/pkg/sentry/socket/control/control.go +++ b/pkg/sentry/socket/control/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/device.go b/pkg/sentry/socket/epsocket/device.go index 17f2c9559..3cc138eb0 100644 --- a/pkg/sentry/socket/epsocket/device.go +++ b/pkg/sentry/socket/epsocket/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 47c575e7b..e90ef4835 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/provider.go b/pkg/sentry/socket/epsocket/provider.go index dbc232d26..686554437 100644 --- a/pkg/sentry/socket/epsocket/provider.go +++ b/pkg/sentry/socket/epsocket/provider.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/save_restore.go b/pkg/sentry/socket/epsocket/save_restore.go index 2613f90de..34d9a7cf0 100644 --- a/pkg/sentry/socket/epsocket/save_restore.go +++ b/pkg/sentry/socket/epsocket/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/epsocket/stack.go b/pkg/sentry/socket/epsocket/stack.go index e4ed52fc8..c0081c819 100644 --- a/pkg/sentry/socket/epsocket/stack.go +++ b/pkg/sentry/socket/epsocket/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/device.go b/pkg/sentry/socket/hostinet/device.go index a9a673316..c5133f3bb 100644 --- a/pkg/sentry/socket/hostinet/device.go +++ b/pkg/sentry/socket/hostinet/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/hostinet.go b/pkg/sentry/socket/hostinet/hostinet.go index 67c6c8066..7858892ab 100644 --- a/pkg/sentry/socket/hostinet/hostinet.go +++ b/pkg/sentry/socket/hostinet/hostinet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/save_restore.go b/pkg/sentry/socket/hostinet/save_restore.go index 0821a794a..3827f082a 100644 --- a/pkg/sentry/socket/hostinet/save_restore.go +++ b/pkg/sentry/socket/hostinet/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index e82624b44..e4e950fbb 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/socket_unsafe.go b/pkg/sentry/socket/hostinet/socket_unsafe.go index f8bb75636..59c8910ca 100644 --- a/pkg/sentry/socket/hostinet/socket_unsafe.go +++ b/pkg/sentry/socket/hostinet/socket_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/hostinet/stack.go b/pkg/sentry/socket/hostinet/stack.go index f64809d39..4ce73c1f1 100644 --- a/pkg/sentry/socket/hostinet/stack.go +++ b/pkg/sentry/socket/hostinet/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/message.go b/pkg/sentry/socket/netlink/message.go index b902d7ec9..a95172cba 100644 --- a/pkg/sentry/socket/netlink/message.go +++ b/pkg/sentry/socket/netlink/message.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/port/port.go b/pkg/sentry/socket/netlink/port/port.go index 1c5d4c3a5..20b9a6e37 100644 --- a/pkg/sentry/socket/netlink/port/port.go +++ b/pkg/sentry/socket/netlink/port/port.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/port/port_test.go b/pkg/sentry/socket/netlink/port/port_test.go index 34565e2f9..49b3b48ab 100644 --- a/pkg/sentry/socket/netlink/port/port_test.go +++ b/pkg/sentry/socket/netlink/port/port_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/provider.go b/pkg/sentry/socket/netlink/provider.go index 5d0a04a07..06786bd50 100644 --- a/pkg/sentry/socket/netlink/provider.go +++ b/pkg/sentry/socket/netlink/provider.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/route/protocol.go b/pkg/sentry/socket/netlink/route/protocol.go index 70322b9ed..7e70b09b2 100644 --- a/pkg/sentry/socket/netlink/route/protocol.go +++ b/pkg/sentry/socket/netlink/route/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index 0c03997f2..4d4130a4c 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/conn/conn.go b/pkg/sentry/socket/rpcinet/conn/conn.go index f4c8489b1..9c749b888 100644 --- a/pkg/sentry/socket/rpcinet/conn/conn.go +++ b/pkg/sentry/socket/rpcinet/conn/conn.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/device.go b/pkg/sentry/socket/rpcinet/device.go index f7b63436e..d2b9f9222 100644 --- a/pkg/sentry/socket/rpcinet/device.go +++ b/pkg/sentry/socket/rpcinet/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier.go b/pkg/sentry/socket/rpcinet/notifier/notifier.go index f88a908ed..73c255c33 100644 --- a/pkg/sentry/socket/rpcinet/notifier/notifier.go +++ b/pkg/sentry/socket/rpcinet/notifier/notifier.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/rpcinet.go b/pkg/sentry/socket/rpcinet/rpcinet.go index 10b0dedc2..6c98e6acb 100644 --- a/pkg/sentry/socket/rpcinet/rpcinet.go +++ b/pkg/sentry/socket/rpcinet/rpcinet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index c7e761d54..44fa5c620 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/stack.go b/pkg/sentry/socket/rpcinet/stack.go index bcb89fb34..cb8344ec6 100644 --- a/pkg/sentry/socket/rpcinet/stack.go +++ b/pkg/sentry/socket/rpcinet/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/rpcinet/stack_unsafe.go b/pkg/sentry/socket/rpcinet/stack_unsafe.go index 9a896c623..d04fb2069 100644 --- a/pkg/sentry/socket/rpcinet/stack_unsafe.go +++ b/pkg/sentry/socket/rpcinet/stack_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index 31f8d42d7..a235c5249 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/device.go b/pkg/sentry/socket/unix/device.go index e8bcc7a9f..41820dbb3 100644 --- a/pkg/sentry/socket/unix/device.go +++ b/pkg/sentry/socket/unix/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/io.go b/pkg/sentry/socket/unix/io.go index 06333e14b..7d6434696 100644 --- a/pkg/sentry/socket/unix/io.go +++ b/pkg/sentry/socket/unix/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectioned.go b/pkg/sentry/socket/unix/transport/connectioned.go index 566e3d57b..4c913effc 100644 --- a/pkg/sentry/socket/unix/transport/connectioned.go +++ b/pkg/sentry/socket/unix/transport/connectioned.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectioned_state.go b/pkg/sentry/socket/unix/transport/connectioned_state.go index 7e6c73dcc..608a6a97a 100644 --- a/pkg/sentry/socket/unix/transport/connectioned_state.go +++ b/pkg/sentry/socket/unix/transport/connectioned_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/connectionless.go b/pkg/sentry/socket/unix/transport/connectionless.go index 86cd05199..cd4633106 100644 --- a/pkg/sentry/socket/unix/transport/connectionless.go +++ b/pkg/sentry/socket/unix/transport/connectionless.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/queue.go b/pkg/sentry/socket/unix/transport/queue.go index c4d7d863c..5b4dfab68 100644 --- a/pkg/sentry/socket/unix/transport/queue.go +++ b/pkg/sentry/socket/unix/transport/queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/transport/unix.go b/pkg/sentry/socket/unix/transport/unix.go index 2934101a2..157133b65 100644 --- a/pkg/sentry/socket/unix/transport/unix.go +++ b/pkg/sentry/socket/unix/transport/unix.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index 668363864..3543dd81f 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state.go b/pkg/sentry/state/state.go index 43e88a713..70b33f190 100644 --- a/pkg/sentry/state/state.go +++ b/pkg/sentry/state/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state_metadata.go b/pkg/sentry/state/state_metadata.go index afa21672a..7f047b808 100644 --- a/pkg/sentry/state/state_metadata.go +++ b/pkg/sentry/state/state_metadata.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/state/state_unsafe.go b/pkg/sentry/state/state_unsafe.go index 3ff7d24c8..f02e12b2a 100644 --- a/pkg/sentry/state/state_unsafe.go +++ b/pkg/sentry/state/state_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/clone.go b/pkg/sentry/strace/clone.go index b82ca1ad1..e18ce84dc 100644 --- a/pkg/sentry/strace/clone.go +++ b/pkg/sentry/strace/clone.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/futex.go b/pkg/sentry/strace/futex.go index 3da108cb7..ceb3dc21d 100644 --- a/pkg/sentry/strace/futex.go +++ b/pkg/sentry/strace/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/linux64.go b/pkg/sentry/strace/linux64.go index 1df148e7d..99714f12c 100644 --- a/pkg/sentry/strace/linux64.go +++ b/pkg/sentry/strace/linux64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/open.go b/pkg/sentry/strace/open.go index 839d5eda7..5a72a940c 100644 --- a/pkg/sentry/strace/open.go +++ b/pkg/sentry/strace/open.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/ptrace.go b/pkg/sentry/strace/ptrace.go index fcdb7e9f4..c572aafb4 100644 --- a/pkg/sentry/strace/ptrace.go +++ b/pkg/sentry/strace/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/socket.go b/pkg/sentry/strace/socket.go index 26831edd6..375418dc1 100644 --- a/pkg/sentry/strace/socket.go +++ b/pkg/sentry/strace/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/strace.go b/pkg/sentry/strace/strace.go index f7bfa3a1f..4286f0df7 100644 --- a/pkg/sentry/strace/strace.go +++ b/pkg/sentry/strace/strace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/strace.proto b/pkg/sentry/strace/strace.proto index 914e8c7b0..f1fc539d6 100644 --- a/pkg/sentry/strace/strace.proto +++ b/pkg/sentry/strace/strace.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/syscalls.go b/pkg/sentry/strace/syscalls.go index 8be4fa318..9eeb18a03 100644 --- a/pkg/sentry/strace/syscalls.go +++ b/pkg/sentry/strace/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/epoll.go b/pkg/sentry/syscalls/epoll.go index 01dd6fa71..b90d191b7 100644 --- a/pkg/sentry/syscalls/epoll.go +++ b/pkg/sentry/syscalls/epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/error.go b/pkg/sentry/syscalls/linux/error.go index 013b385bc..9fd002955 100644 --- a/pkg/sentry/syscalls/linux/error.go +++ b/pkg/sentry/syscalls/linux/error.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/flags.go b/pkg/sentry/syscalls/linux/flags.go index f01483cd3..d1e0833fc 100644 --- a/pkg/sentry/syscalls/linux/flags.go +++ b/pkg/sentry/syscalls/linux/flags.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go index 4465549ad..75e87f5ec 100644 --- a/pkg/sentry/syscalls/linux/linux64.go +++ b/pkg/sentry/syscalls/linux/linux64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sigset.go b/pkg/sentry/syscalls/linux/sigset.go index bfb541634..a033b7c70 100644 --- a/pkg/sentry/syscalls/linux/sigset.go +++ b/pkg/sentry/syscalls/linux/sigset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_aio.go b/pkg/sentry/syscalls/linux/sys_aio.go index 54e4afa9e..355071131 100644 --- a/pkg/sentry/syscalls/linux/sys_aio.go +++ b/pkg/sentry/syscalls/linux/sys_aio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_capability.go b/pkg/sentry/syscalls/linux/sys_capability.go index 89c81ac90..cf972dc28 100644 --- a/pkg/sentry/syscalls/linux/sys_capability.go +++ b/pkg/sentry/syscalls/linux/sys_capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_epoll.go b/pkg/sentry/syscalls/linux/sys_epoll.go index e69dfc77a..62272efcd 100644 --- a/pkg/sentry/syscalls/linux/sys_epoll.go +++ b/pkg/sentry/syscalls/linux/sys_epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_eventfd.go b/pkg/sentry/syscalls/linux/sys_eventfd.go index 60fe5a133..903172890 100644 --- a/pkg/sentry/syscalls/linux/sys_eventfd.go +++ b/pkg/sentry/syscalls/linux/sys_eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_file.go b/pkg/sentry/syscalls/linux/sys_file.go index 64704bb88..a70f35be0 100644 --- a/pkg/sentry/syscalls/linux/sys_file.go +++ b/pkg/sentry/syscalls/linux/sys_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_futex.go b/pkg/sentry/syscalls/linux/sys_futex.go index d35dcecbe..cf04428bc 100644 --- a/pkg/sentry/syscalls/linux/sys_futex.go +++ b/pkg/sentry/syscalls/linux/sys_futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_getdents.go b/pkg/sentry/syscalls/linux/sys_getdents.go index 29c0d7a39..4b441b31b 100644 --- a/pkg/sentry/syscalls/linux/sys_getdents.go +++ b/pkg/sentry/syscalls/linux/sys_getdents.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_identity.go b/pkg/sentry/syscalls/linux/sys_identity.go index 4fd0ed794..8d594aa83 100644 --- a/pkg/sentry/syscalls/linux/sys_identity.go +++ b/pkg/sentry/syscalls/linux/sys_identity.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_inotify.go b/pkg/sentry/syscalls/linux/sys_inotify.go index 725204dff..26a505782 100644 --- a/pkg/sentry/syscalls/linux/sys_inotify.go +++ b/pkg/sentry/syscalls/linux/sys_inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_lseek.go b/pkg/sentry/syscalls/linux/sys_lseek.go index 97b51ba7c..ad3bfd761 100644 --- a/pkg/sentry/syscalls/linux/sys_lseek.go +++ b/pkg/sentry/syscalls/linux/sys_lseek.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_mmap.go b/pkg/sentry/syscalls/linux/sys_mmap.go index 1a98328dc..f8d9c43fd 100644 --- a/pkg/sentry/syscalls/linux/sys_mmap.go +++ b/pkg/sentry/syscalls/linux/sys_mmap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_mount.go b/pkg/sentry/syscalls/linux/sys_mount.go index 57cedccc1..bf0df7302 100644 --- a/pkg/sentry/syscalls/linux/sys_mount.go +++ b/pkg/sentry/syscalls/linux/sys_mount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_pipe.go b/pkg/sentry/syscalls/linux/sys_pipe.go index 2b544f145..3652c429e 100644 --- a/pkg/sentry/syscalls/linux/sys_pipe.go +++ b/pkg/sentry/syscalls/linux/sys_pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_poll.go b/pkg/sentry/syscalls/linux/sys_poll.go index b9bdefadb..bf0958435 100644 --- a/pkg/sentry/syscalls/linux/sys_poll.go +++ b/pkg/sentry/syscalls/linux/sys_poll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_prctl.go b/pkg/sentry/syscalls/linux/sys_prctl.go index a1242acd3..c7b39ede8 100644 --- a/pkg/sentry/syscalls/linux/sys_prctl.go +++ b/pkg/sentry/syscalls/linux/sys_prctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_random.go b/pkg/sentry/syscalls/linux/sys_random.go index be31e6b17..452dff058 100644 --- a/pkg/sentry/syscalls/linux/sys_random.go +++ b/pkg/sentry/syscalls/linux/sys_random.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_read.go b/pkg/sentry/syscalls/linux/sys_read.go index 0be2d195a..b2e5a5449 100644 --- a/pkg/sentry/syscalls/linux/sys_read.go +++ b/pkg/sentry/syscalls/linux/sys_read.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_rlimit.go b/pkg/sentry/syscalls/linux/sys_rlimit.go index d806b58ab..2f16e1791 100644 --- a/pkg/sentry/syscalls/linux/sys_rlimit.go +++ b/pkg/sentry/syscalls/linux/sys_rlimit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_rusage.go b/pkg/sentry/syscalls/linux/sys_rusage.go index 82e42b589..ab07c77f9 100644 --- a/pkg/sentry/syscalls/linux/sys_rusage.go +++ b/pkg/sentry/syscalls/linux/sys_rusage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sched.go b/pkg/sentry/syscalls/linux/sys_sched.go index ff9e46077..e679a6694 100644 --- a/pkg/sentry/syscalls/linux/sys_sched.go +++ b/pkg/sentry/syscalls/linux/sys_sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_seccomp.go b/pkg/sentry/syscalls/linux/sys_seccomp.go index 4323a4df4..969acaa36 100644 --- a/pkg/sentry/syscalls/linux/sys_seccomp.go +++ b/pkg/sentry/syscalls/linux/sys_seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sem.go b/pkg/sentry/syscalls/linux/sys_sem.go index a8983705b..4ed52c4a7 100644 --- a/pkg/sentry/syscalls/linux/sys_sem.go +++ b/pkg/sentry/syscalls/linux/sys_sem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_shm.go b/pkg/sentry/syscalls/linux/sys_shm.go index 48ff1d5f0..b13d48b98 100644 --- a/pkg/sentry/syscalls/linux/sys_shm.go +++ b/pkg/sentry/syscalls/linux/sys_shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_signal.go b/pkg/sentry/syscalls/linux/sys_signal.go index ecdec5d3a..a539354c5 100644 --- a/pkg/sentry/syscalls/linux/sys_signal.go +++ b/pkg/sentry/syscalls/linux/sys_signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index 5fa5ddce6..0a7551742 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_stat.go b/pkg/sentry/syscalls/linux/sys_stat.go index 619a14d7c..9c433c45d 100644 --- a/pkg/sentry/syscalls/linux/sys_stat.go +++ b/pkg/sentry/syscalls/linux/sys_stat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sync.go b/pkg/sentry/syscalls/linux/sys_sync.go index 902d210db..826c6869d 100644 --- a/pkg/sentry/syscalls/linux/sys_sync.go +++ b/pkg/sentry/syscalls/linux/sys_sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_sysinfo.go b/pkg/sentry/syscalls/linux/sys_sysinfo.go index 6560bac57..5eeb3ba58 100644 --- a/pkg/sentry/syscalls/linux/sys_sysinfo.go +++ b/pkg/sentry/syscalls/linux/sys_sysinfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_syslog.go b/pkg/sentry/syscalls/linux/sys_syslog.go index 792040c81..7193b7aed 100644 --- a/pkg/sentry/syscalls/linux/sys_syslog.go +++ b/pkg/sentry/syscalls/linux/sys_syslog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_thread.go b/pkg/sentry/syscalls/linux/sys_thread.go index 550f63a43..820ca680e 100644 --- a/pkg/sentry/syscalls/linux/sys_thread.go +++ b/pkg/sentry/syscalls/linux/sys_thread.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_time.go b/pkg/sentry/syscalls/linux/sys_time.go index 8e6683444..063fbb106 100644 --- a/pkg/sentry/syscalls/linux/sys_time.go +++ b/pkg/sentry/syscalls/linux/sys_time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_timer.go b/pkg/sentry/syscalls/linux/sys_timer.go index c41074d54..6baf4599b 100644 --- a/pkg/sentry/syscalls/linux/sys_timer.go +++ b/pkg/sentry/syscalls/linux/sys_timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_timerfd.go b/pkg/sentry/syscalls/linux/sys_timerfd.go index 92c6a3d60..f70d13682 100644 --- a/pkg/sentry/syscalls/linux/sys_timerfd.go +++ b/pkg/sentry/syscalls/linux/sys_timerfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_tls.go b/pkg/sentry/syscalls/linux/sys_tls.go index b95d62320..27ddb3808 100644 --- a/pkg/sentry/syscalls/linux/sys_tls.go +++ b/pkg/sentry/syscalls/linux/sys_tls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_utsname.go b/pkg/sentry/syscalls/linux/sys_utsname.go index 899116374..689f2f838 100644 --- a/pkg/sentry/syscalls/linux/sys_utsname.go +++ b/pkg/sentry/syscalls/linux/sys_utsname.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/sys_write.go b/pkg/sentry/syscalls/linux/sys_write.go index caa7b01ea..08e263112 100644 --- a/pkg/sentry/syscalls/linux/sys_write.go +++ b/pkg/sentry/syscalls/linux/sys_write.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/linux/timespec.go b/pkg/sentry/syscalls/linux/timespec.go index e865c6fc0..752ec326d 100644 --- a/pkg/sentry/syscalls/linux/timespec.go +++ b/pkg/sentry/syscalls/linux/timespec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/polling.go b/pkg/sentry/syscalls/polling.go index fd90184ef..2b33d6c19 100644 --- a/pkg/sentry/syscalls/polling.go +++ b/pkg/sentry/syscalls/polling.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/syscalls.go b/pkg/sentry/syscalls/syscalls.go index 1176f858d..bae32d727 100644 --- a/pkg/sentry/syscalls/syscalls.go +++ b/pkg/sentry/syscalls/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/syscalls/unimplemented_syscall.proto b/pkg/sentry/syscalls/unimplemented_syscall.proto index d6febf5b1..41579b016 100644 --- a/pkg/sentry/syscalls/unimplemented_syscall.proto +++ b/pkg/sentry/syscalls/unimplemented_syscall.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/calibrated_clock.go b/pkg/sentry/time/calibrated_clock.go index cbb95e2d7..c8cf4eca4 100644 --- a/pkg/sentry/time/calibrated_clock.go +++ b/pkg/sentry/time/calibrated_clock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/calibrated_clock_test.go b/pkg/sentry/time/calibrated_clock_test.go index 8b6dd5592..a9237630e 100644 --- a/pkg/sentry/time/calibrated_clock_test.go +++ b/pkg/sentry/time/calibrated_clock_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/clock_id.go b/pkg/sentry/time/clock_id.go index 500102e58..1317a5dad 100644 --- a/pkg/sentry/time/clock_id.go +++ b/pkg/sentry/time/clock_id.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/clocks.go b/pkg/sentry/time/clocks.go index 9925b407d..e26386520 100644 --- a/pkg/sentry/time/clocks.go +++ b/pkg/sentry/time/clocks.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/muldiv_amd64.s b/pkg/sentry/time/muldiv_amd64.s index 291940b1d..bfcb8c724 100644 --- a/pkg/sentry/time/muldiv_amd64.s +++ b/pkg/sentry/time/muldiv_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/parameters.go b/pkg/sentry/time/parameters.go index 594b4874b..f3ad58454 100644 --- a/pkg/sentry/time/parameters.go +++ b/pkg/sentry/time/parameters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/parameters_test.go b/pkg/sentry/time/parameters_test.go index 7394fc5ee..4a0c4e880 100644 --- a/pkg/sentry/time/parameters_test.go +++ b/pkg/sentry/time/parameters_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler.go b/pkg/sentry/time/sampler.go index cf581b5fa..445690d49 100644 --- a/pkg/sentry/time/sampler.go +++ b/pkg/sentry/time/sampler.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler_test.go b/pkg/sentry/time/sampler_test.go index caf7e5c53..ec0e442b6 100644 --- a/pkg/sentry/time/sampler_test.go +++ b/pkg/sentry/time/sampler_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/sampler_unsafe.go b/pkg/sentry/time/sampler_unsafe.go index 7ea19d387..0f8eb4fc8 100644 --- a/pkg/sentry/time/sampler_unsafe.go +++ b/pkg/sentry/time/sampler_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/time/tsc_amd64.s b/pkg/sentry/time/tsc_amd64.s index 4cc604392..e53d477f7 100644 --- a/pkg/sentry/time/tsc_amd64.s +++ b/pkg/sentry/time/tsc_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/uniqueid/context.go b/pkg/sentry/uniqueid/context.go index e48fabc2d..399d98c29 100644 --- a/pkg/sentry/uniqueid/context.go +++ b/pkg/sentry/uniqueid/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/cpu.go b/pkg/sentry/usage/cpu.go index ed7b04b9e..cbd7cfe19 100644 --- a/pkg/sentry/usage/cpu.go +++ b/pkg/sentry/usage/cpu.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/io.go b/pkg/sentry/usage/io.go index 49faa507d..8e27a0a88 100644 --- a/pkg/sentry/usage/io.go +++ b/pkg/sentry/usage/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/memory.go b/pkg/sentry/usage/memory.go index 92a478d85..7e065cb76 100644 --- a/pkg/sentry/usage/memory.go +++ b/pkg/sentry/usage/memory.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/memory_unsafe.go b/pkg/sentry/usage/memory_unsafe.go index f990a7750..a3ae668a5 100644 --- a/pkg/sentry/usage/memory_unsafe.go +++ b/pkg/sentry/usage/memory_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usage/usage.go b/pkg/sentry/usage/usage.go index 3b3118659..ab327f8e2 100644 --- a/pkg/sentry/usage/usage.go +++ b/pkg/sentry/usage/usage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/access_type.go b/pkg/sentry/usermem/access_type.go index 75346d854..c71d05afe 100644 --- a/pkg/sentry/usermem/access_type.go +++ b/pkg/sentry/usermem/access_type.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr.go b/pkg/sentry/usermem/addr.go index fc94bee80..2a75aa60c 100644 --- a/pkg/sentry/usermem/addr.go +++ b/pkg/sentry/usermem/addr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr_range_seq_test.go b/pkg/sentry/usermem/addr_range_seq_test.go index cf9d785ed..bd6a1ec8a 100644 --- a/pkg/sentry/usermem/addr_range_seq_test.go +++ b/pkg/sentry/usermem/addr_range_seq_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/addr_range_seq_unsafe.go b/pkg/sentry/usermem/addr_range_seq_unsafe.go index 13b2998b3..f5fd446fa 100644 --- a/pkg/sentry/usermem/addr_range_seq_unsafe.go +++ b/pkg/sentry/usermem/addr_range_seq_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/bytes_io.go b/pkg/sentry/usermem/bytes_io.go index 01a746404..274f568d0 100644 --- a/pkg/sentry/usermem/bytes_io.go +++ b/pkg/sentry/usermem/bytes_io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/bytes_io_unsafe.go b/pkg/sentry/usermem/bytes_io_unsafe.go index efd71fcbc..8bdf3a508 100644 --- a/pkg/sentry/usermem/bytes_io_unsafe.go +++ b/pkg/sentry/usermem/bytes_io_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem.go b/pkg/sentry/usermem/usermem.go index 5d8a1c558..1d6c0b4d6 100644 --- a/pkg/sentry/usermem/usermem.go +++ b/pkg/sentry/usermem/usermem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_test.go b/pkg/sentry/usermem/usermem_test.go index 563560da8..1991a9641 100644 --- a/pkg/sentry/usermem/usermem_test.go +++ b/pkg/sentry/usermem/usermem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_x86.go b/pkg/sentry/usermem/usermem_x86.go index 2484b0d82..9ec90f9ff 100644 --- a/pkg/sentry/usermem/usermem_x86.go +++ b/pkg/sentry/usermem/usermem_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/watchdog/watchdog.go b/pkg/sentry/watchdog/watchdog.go index 5b620693d..75b11237f 100644 --- a/pkg/sentry/watchdog/watchdog.go +++ b/pkg/sentry/watchdog/watchdog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_amd64.s b/pkg/sleep/commit_amd64.s index d525e5b79..d08df7f37 100644 --- a/pkg/sleep/commit_amd64.s +++ b/pkg/sleep/commit_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_asm.go b/pkg/sleep/commit_asm.go index 39a55df7e..90eef4cbc 100644 --- a/pkg/sleep/commit_asm.go +++ b/pkg/sleep/commit_asm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/commit_noasm.go b/pkg/sleep/commit_noasm.go index 584866cd8..967d22e24 100644 --- a/pkg/sleep/commit_noasm.go +++ b/pkg/sleep/commit_noasm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/empty.s b/pkg/sleep/empty.s index 8aca31bee..85d52cd9c 100644 --- a/pkg/sleep/empty.s +++ b/pkg/sleep/empty.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/sleep_test.go b/pkg/sleep/sleep_test.go index bc1738371..8feb9ffc2 100644 --- a/pkg/sleep/sleep_test.go +++ b/pkg/sleep/sleep_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sleep/sleep_unsafe.go b/pkg/sleep/sleep_unsafe.go index b12cce681..45fb6f0ea 100644 --- a/pkg/sleep/sleep_unsafe.go +++ b/pkg/sleep/sleep_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/decode.go b/pkg/state/decode.go index 3ef59610b..54b5ad8b8 100644 --- a/pkg/state/decode.go +++ b/pkg/state/decode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/encode.go b/pkg/state/encode.go index fd052db12..577aaf051 100644 --- a/pkg/state/encode.go +++ b/pkg/state/encode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/encode_unsafe.go b/pkg/state/encode_unsafe.go index d96ba56d4..be94742a8 100644 --- a/pkg/state/encode_unsafe.go +++ b/pkg/state/encode_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/map.go b/pkg/state/map.go index c3d165501..0035d7250 100644 --- a/pkg/state/map.go +++ b/pkg/state/map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/object.proto b/pkg/state/object.proto index c78efed2a..d3b46ea97 100644 --- a/pkg/state/object.proto +++ b/pkg/state/object.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/printer.go b/pkg/state/printer.go index 2c8ce60a5..aee4b69fb 100644 --- a/pkg/state/printer.go +++ b/pkg/state/printer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/state.go b/pkg/state/state.go index 23a0b5922..4b141777e 100644 --- a/pkg/state/state.go +++ b/pkg/state/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/state_test.go b/pkg/state/state_test.go index 38ad9da9c..22bcad9e1 100644 --- a/pkg/state/state_test.go +++ b/pkg/state/state_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go index 9c86c1934..99158fd02 100644 --- a/pkg/state/statefile/statefile.go +++ b/pkg/state/statefile/statefile.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/statefile/statefile_test.go b/pkg/state/statefile/statefile_test.go index fa3fb9f2c..b4f400e01 100644 --- a/pkg/state/statefile/statefile_test.go +++ b/pkg/state/statefile/statefile_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/state/stats.go b/pkg/state/stats.go index ddcc49f78..17ca258fc 100644 --- a/pkg/state/stats.go +++ b/pkg/state/stats.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/atomicptr_unsafe.go b/pkg/sync/atomicptr_unsafe.go index f12e9cb67..d943b7ff4 100644 --- a/pkg/sync/atomicptr_unsafe.go +++ b/pkg/sync/atomicptr_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/atomicptrtest/atomicptr_test.go b/pkg/sync/atomicptrtest/atomicptr_test.go index b458382b1..3262785ce 100644 --- a/pkg/sync/atomicptrtest/atomicptr_test.go +++ b/pkg/sync/atomicptrtest/atomicptr_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/memmove_unsafe.go b/pkg/sync/memmove_unsafe.go index 0c992d5a4..cd7a02dca 100644 --- a/pkg/sync/memmove_unsafe.go +++ b/pkg/sync/memmove_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/norace_unsafe.go b/pkg/sync/norace_unsafe.go index 968665078..1593b9e5d 100644 --- a/pkg/sync/norace_unsafe.go +++ b/pkg/sync/norace_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/race_unsafe.go b/pkg/sync/race_unsafe.go index d143a21c7..473eaddc6 100644 --- a/pkg/sync/race_unsafe.go +++ b/pkg/sync/race_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/seqatomic_unsafe.go b/pkg/sync/seqatomic_unsafe.go index a18e1229a..bea31adc5 100644 --- a/pkg/sync/seqatomic_unsafe.go +++ b/pkg/sync/seqatomic_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/seqatomictest/seqatomic_test.go b/pkg/sync/seqatomictest/seqatomic_test.go index b785d2344..f5e1fbfff 100644 --- a/pkg/sync/seqatomictest/seqatomic_test.go +++ b/pkg/sync/seqatomictest/seqatomic_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/seqcount.go b/pkg/sync/seqcount.go index 8e3304d69..732e856a4 100644 --- a/pkg/sync/seqcount.go +++ b/pkg/sync/seqcount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/seqcount_test.go b/pkg/sync/seqcount_test.go index fa4abed1d..b14a8878e 100644 --- a/pkg/sync/seqcount_test.go +++ b/pkg/sync/seqcount_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sync/sync.go b/pkg/sync/sync.go index 36d4c4dee..22c5348d7 100644 --- a/pkg/sync/sync.go +++ b/pkg/sync/sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/host_linux.go b/pkg/syserr/host_linux.go index 22009a799..74bbe9f5b 100644 --- a/pkg/syserr/host_linux.go +++ b/pkg/syserr/host_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/netstack.go b/pkg/syserr/netstack.go index b9786b48f..20e756edb 100644 --- a/pkg/syserr/netstack.go +++ b/pkg/syserr/netstack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserr/syserr.go b/pkg/syserr/syserr.go index dba6cb7de..6a66e23a2 100644 --- a/pkg/syserr/syserr.go +++ b/pkg/syserr/syserr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserror/syserror.go b/pkg/syserror/syserror.go index 5bc74e65e..4228707f4 100644 --- a/pkg/syserror/syserror.go +++ b/pkg/syserror/syserror.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/syserror/syserror_test.go b/pkg/syserror/syserror_test.go index fb7d8d5ee..0f0da5781 100644 --- a/pkg/syserror/syserror_test.go +++ b/pkg/syserror/syserror_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/adapters/gonet/gonet.go b/pkg/tcpip/adapters/gonet/gonet.go index b64dce720..81428770b 100644 --- a/pkg/tcpip/adapters/gonet/gonet.go +++ b/pkg/tcpip/adapters/gonet/gonet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/adapters/gonet/gonet_test.go b/pkg/tcpip/adapters/gonet/gonet_test.go index 79b7c77ee..05a730a05 100644 --- a/pkg/tcpip/adapters/gonet/gonet_test.go +++ b/pkg/tcpip/adapters/gonet/gonet_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/prependable.go b/pkg/tcpip/buffer/prependable.go index c5dd2819f..d3a9a0f88 100644 --- a/pkg/tcpip/buffer/prependable.go +++ b/pkg/tcpip/buffer/prependable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/view.go b/pkg/tcpip/buffer/view.go index cea4e3657..24479ea40 100644 --- a/pkg/tcpip/buffer/view.go +++ b/pkg/tcpip/buffer/view.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/buffer/view_test.go b/pkg/tcpip/buffer/view_test.go index 02c264593..74a0a96fc 100644 --- a/pkg/tcpip/buffer/view_test.go +++ b/pkg/tcpip/buffer/view_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/checker/checker.go b/pkg/tcpip/checker/checker.go index 206531f20..5dfb3ca1d 100644 --- a/pkg/tcpip/checker/checker.go +++ b/pkg/tcpip/checker/checker.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/arp.go b/pkg/tcpip/header/arp.go index ae373f112..22b259ccb 100644 --- a/pkg/tcpip/header/arp.go +++ b/pkg/tcpip/header/arp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/checksum.go b/pkg/tcpip/header/checksum.go index e67c50f50..12f208fde 100644 --- a/pkg/tcpip/header/checksum.go +++ b/pkg/tcpip/header/checksum.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/eth.go b/pkg/tcpip/header/eth.go index 99c29b750..77365bc41 100644 --- a/pkg/tcpip/header/eth.go +++ b/pkg/tcpip/header/eth.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/gue.go b/pkg/tcpip/header/gue.go index aac4593c5..2ad13955a 100644 --- a/pkg/tcpip/header/gue.go +++ b/pkg/tcpip/header/gue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/icmpv4.go b/pkg/tcpip/header/icmpv4.go index af1e94b7f..3ac89cdae 100644 --- a/pkg/tcpip/header/icmpv4.go +++ b/pkg/tcpip/header/icmpv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/icmpv6.go b/pkg/tcpip/header/icmpv6.go index 7d35caff7..e317975e8 100644 --- a/pkg/tcpip/header/icmpv6.go +++ b/pkg/tcpip/header/icmpv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/interfaces.go b/pkg/tcpip/header/interfaces.go index 042006983..ac327d8a5 100644 --- a/pkg/tcpip/header/interfaces.go +++ b/pkg/tcpip/header/interfaces.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv4.go b/pkg/tcpip/header/ipv4.go index 29570cc34..1b882d3d8 100644 --- a/pkg/tcpip/header/ipv4.go +++ b/pkg/tcpip/header/ipv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv6.go b/pkg/tcpip/header/ipv6.go index 66c778fe1..d985b745d 100644 --- a/pkg/tcpip/header/ipv6.go +++ b/pkg/tcpip/header/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipv6_fragment.go b/pkg/tcpip/header/ipv6_fragment.go index 44b28b326..e36d5177b 100644 --- a/pkg/tcpip/header/ipv6_fragment.go +++ b/pkg/tcpip/header/ipv6_fragment.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/ipversion_test.go b/pkg/tcpip/header/ipversion_test.go index 3ae9b7e4a..8301ba5cf 100644 --- a/pkg/tcpip/header/ipversion_test.go +++ b/pkg/tcpip/header/ipversion_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/tcp.go b/pkg/tcpip/header/tcp.go index 6689a6dc5..567a21167 100644 --- a/pkg/tcpip/header/tcp.go +++ b/pkg/tcpip/header/tcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/tcp_test.go b/pkg/tcpip/header/tcp_test.go index 7854d3523..7cd98df3b 100644 --- a/pkg/tcpip/header/tcp_test.go +++ b/pkg/tcpip/header/tcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/header/udp.go b/pkg/tcpip/header/udp.go index cf2602e50..31c8ef456 100644 --- a/pkg/tcpip/header/udp.go +++ b/pkg/tcpip/header/udp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/channel/channel.go b/pkg/tcpip/link/channel/channel.go index 113cbbf5e..da34032cc 100644 --- a/pkg/tcpip/link/channel/channel.go +++ b/pkg/tcpip/link/channel/channel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/endpoint.go b/pkg/tcpip/link/fdbased/endpoint.go index ee99ada07..24af428dd 100644 --- a/pkg/tcpip/link/fdbased/endpoint.go +++ b/pkg/tcpip/link/fdbased/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/endpoint_test.go b/pkg/tcpip/link/fdbased/endpoint_test.go index 52e532ebb..19b007a9e 100644 --- a/pkg/tcpip/link/fdbased/endpoint_test.go +++ b/pkg/tcpip/link/fdbased/endpoint_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/loopback/loopback.go b/pkg/tcpip/link/loopback/loopback.go index fc3f80c01..e6585be66 100644 --- a/pkg/tcpip/link/loopback/loopback.go +++ b/pkg/tcpip/link/loopback/loopback.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_amd64.s b/pkg/tcpip/link/rawfile/blockingpoll_amd64.s index fc5231831..63b8c4451 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_amd64.s +++ b/pkg/tcpip/link/rawfile/blockingpoll_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go b/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go index a0a9d4acd..6a3e956ad 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go +++ b/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go b/pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go index 1f143c0db..89a8a9954 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go +++ b/pkg/tcpip/link/rawfile/blockingpoll_unsafe_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/errors.go b/pkg/tcpip/link/rawfile/errors.go index de7593d9c..f42ff98db 100644 --- a/pkg/tcpip/link/rawfile/errors.go +++ b/pkg/tcpip/link/rawfile/errors.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/rawfile_unsafe.go b/pkg/tcpip/link/rawfile/rawfile_unsafe.go index cea3cd6a1..be4a4fa9c 100644 --- a/pkg/tcpip/link/rawfile/rawfile_unsafe.go +++ b/pkg/tcpip/link/rawfile/rawfile_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe.go b/pkg/tcpip/link/sharedmem/pipe/pipe.go index 1a0edbaba..e014324cc 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe_test.go b/pkg/tcpip/link/sharedmem/pipe/pipe_test.go index db0737c98..30742ccb1 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe_test.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go b/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go index 480dc4a23..f491d74a2 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/rx.go b/pkg/tcpip/link/sharedmem/pipe/rx.go index ff778cecd..8d641c76f 100644 --- a/pkg/tcpip/link/sharedmem/pipe/rx.go +++ b/pkg/tcpip/link/sharedmem/pipe/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/pipe/tx.go b/pkg/tcpip/link/sharedmem/pipe/tx.go index 717f5a4b1..e75175d98 100644 --- a/pkg/tcpip/link/sharedmem/pipe/tx.go +++ b/pkg/tcpip/link/sharedmem/pipe/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/queue_test.go b/pkg/tcpip/link/sharedmem/queue/queue_test.go index 3d5909cef..391165bc3 100644 --- a/pkg/tcpip/link/sharedmem/queue/queue_test.go +++ b/pkg/tcpip/link/sharedmem/queue/queue_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/rx.go b/pkg/tcpip/link/sharedmem/queue/rx.go index c40d62c33..d3a5da08a 100644 --- a/pkg/tcpip/link/sharedmem/queue/rx.go +++ b/pkg/tcpip/link/sharedmem/queue/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/queue/tx.go b/pkg/tcpip/link/sharedmem/queue/tx.go index 39b595e56..845108db1 100644 --- a/pkg/tcpip/link/sharedmem/queue/tx.go +++ b/pkg/tcpip/link/sharedmem/queue/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/rx.go b/pkg/tcpip/link/sharedmem/rx.go index b8e39eca1..3eeab769e 100644 --- a/pkg/tcpip/link/sharedmem/rx.go +++ b/pkg/tcpip/link/sharedmem/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem.go b/pkg/tcpip/link/sharedmem/sharedmem.go index ce6e86767..27d7eb3b9 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem.go +++ b/pkg/tcpip/link/sharedmem/sharedmem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem_test.go b/pkg/tcpip/link/sharedmem/sharedmem_test.go index ad987d382..4b8061b13 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem_test.go +++ b/pkg/tcpip/link/sharedmem/sharedmem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go b/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go index f0be2dc73..b91adbaf7 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go +++ b/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sharedmem/tx.go b/pkg/tcpip/link/sharedmem/tx.go index 42a21cb43..37da34831 100644 --- a/pkg/tcpip/link/sharedmem/tx.go +++ b/pkg/tcpip/link/sharedmem/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sniffer/pcap.go b/pkg/tcpip/link/sniffer/pcap.go index 04f3d494e..3d0d8d852 100644 --- a/pkg/tcpip/link/sniffer/pcap.go +++ b/pkg/tcpip/link/sniffer/pcap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/sniffer/sniffer.go b/pkg/tcpip/link/sniffer/sniffer.go index a30e57a32..1bd174bc3 100644 --- a/pkg/tcpip/link/sniffer/sniffer.go +++ b/pkg/tcpip/link/sniffer/sniffer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/tun/tun_unsafe.go b/pkg/tcpip/link/tun/tun_unsafe.go index 1dec41982..e4c589dda 100644 --- a/pkg/tcpip/link/tun/tun_unsafe.go +++ b/pkg/tcpip/link/tun/tun_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/waitable/waitable.go b/pkg/tcpip/link/waitable/waitable.go index ef8c88561..9ffb7b7e9 100644 --- a/pkg/tcpip/link/waitable/waitable.go +++ b/pkg/tcpip/link/waitable/waitable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/waitable/waitable_test.go b/pkg/tcpip/link/waitable/waitable_test.go index 0a15c40de..5ebe09664 100644 --- a/pkg/tcpip/link/waitable/waitable_test.go +++ b/pkg/tcpip/link/waitable/waitable_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/arp/arp.go b/pkg/tcpip/network/arp/arp.go index 9d0881e11..2e0024925 100644 --- a/pkg/tcpip/network/arp/arp.go +++ b/pkg/tcpip/network/arp/arp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/arp/arp_test.go b/pkg/tcpip/network/arp/arp_test.go index 50628e4a2..5894f9114 100644 --- a/pkg/tcpip/network/arp/arp_test.go +++ b/pkg/tcpip/network/arp/arp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/frag_heap.go b/pkg/tcpip/network/fragmentation/frag_heap.go index 6c7faafe4..55615c8e6 100644 --- a/pkg/tcpip/network/fragmentation/frag_heap.go +++ b/pkg/tcpip/network/fragmentation/frag_heap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/frag_heap_test.go b/pkg/tcpip/network/fragmentation/frag_heap_test.go index a15540634..1b1b72e88 100644 --- a/pkg/tcpip/network/fragmentation/frag_heap_test.go +++ b/pkg/tcpip/network/fragmentation/frag_heap_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/fragmentation.go b/pkg/tcpip/network/fragmentation/fragmentation.go index 885e3cca2..a5dda0398 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation.go +++ b/pkg/tcpip/network/fragmentation/fragmentation.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/fragmentation_test.go b/pkg/tcpip/network/fragmentation/fragmentation_test.go index fc62a15dd..5bf3463a9 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation_test.go +++ b/pkg/tcpip/network/fragmentation/fragmentation_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/reassembler.go b/pkg/tcpip/network/fragmentation/reassembler.go index b57fe82ec..c9ad2bef6 100644 --- a/pkg/tcpip/network/fragmentation/reassembler.go +++ b/pkg/tcpip/network/fragmentation/reassembler.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/fragmentation/reassembler_test.go b/pkg/tcpip/network/fragmentation/reassembler_test.go index 4c137828f..a2bc9707a 100644 --- a/pkg/tcpip/network/fragmentation/reassembler_test.go +++ b/pkg/tcpip/network/fragmentation/reassembler_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/hash/hash.go b/pkg/tcpip/network/hash/hash.go index eddf7ca4d..07960ddf0 100644 --- a/pkg/tcpip/network/hash/hash.go +++ b/pkg/tcpip/network/hash/hash.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ip_test.go b/pkg/tcpip/network/ip_test.go index e3c7af1f9..5c1e88e56 100644 --- a/pkg/tcpip/network/ip_test.go +++ b/pkg/tcpip/network/ip_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/icmp.go b/pkg/tcpip/network/ipv4/icmp.go index ee8172ac8..f82dc098f 100644 --- a/pkg/tcpip/network/ipv4/icmp.go +++ b/pkg/tcpip/network/ipv4/icmp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go index d4eeeb5d9..d7801ec19 100644 --- a/pkg/tcpip/network/ipv4/ipv4.go +++ b/pkg/tcpip/network/ipv4/ipv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv4/ipv4_test.go b/pkg/tcpip/network/ipv4/ipv4_test.go index 2b7067a50..190d548eb 100644 --- a/pkg/tcpip/network/ipv4/ipv4_test.go +++ b/pkg/tcpip/network/ipv4/ipv4_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go index 81aba0923..14107443b 100644 --- a/pkg/tcpip/network/ipv6/icmp.go +++ b/pkg/tcpip/network/ipv6/icmp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/icmp_test.go b/pkg/tcpip/network/ipv6/icmp_test.go index fabbdc8c7..12c818b48 100644 --- a/pkg/tcpip/network/ipv6/icmp_test.go +++ b/pkg/tcpip/network/ipv6/icmp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go index 25bd998e5..4d0b6ee9c 100644 --- a/pkg/tcpip/network/ipv6/ipv6.go +++ b/pkg/tcpip/network/ipv6/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/ports/ports.go b/pkg/tcpip/ports/ports.go index 4e24efddb..41ef32921 100644 --- a/pkg/tcpip/ports/ports.go +++ b/pkg/tcpip/ports/ports.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/ports/ports_test.go b/pkg/tcpip/ports/ports_test.go index 4ab6a1fa2..72577dfcb 100644 --- a/pkg/tcpip/ports/ports_test.go +++ b/pkg/tcpip/ports/ports_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/sample/tun_tcp_connect/main.go b/pkg/tcpip/sample/tun_tcp_connect/main.go index c4707736e..67e8f0b9e 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/main.go +++ b/pkg/tcpip/sample/tun_tcp_connect/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/sample/tun_tcp_echo/main.go b/pkg/tcpip/sample/tun_tcp_echo/main.go index 910d1257f..ab40e9e0b 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/main.go +++ b/pkg/tcpip/sample/tun_tcp_echo/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/seqnum/seqnum.go b/pkg/tcpip/seqnum/seqnum.go index e507d02f7..f2b988839 100644 --- a/pkg/tcpip/seqnum/seqnum.go +++ b/pkg/tcpip/seqnum/seqnum.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/linkaddrcache.go b/pkg/tcpip/stack/linkaddrcache.go index 3a147a75f..cb7b7116b 100644 --- a/pkg/tcpip/stack/linkaddrcache.go +++ b/pkg/tcpip/stack/linkaddrcache.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/linkaddrcache_test.go b/pkg/tcpip/stack/linkaddrcache_test.go index e46267f12..651fa17ac 100644 --- a/pkg/tcpip/stack/linkaddrcache_test.go +++ b/pkg/tcpip/stack/linkaddrcache_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index dba95369c..3da99ac67 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/registration.go b/pkg/tcpip/stack/registration.go index 0acec2984..b6266eb55 100644 --- a/pkg/tcpip/stack/registration.go +++ b/pkg/tcpip/stack/registration.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/route.go b/pkg/tcpip/stack/route.go index 6c6400c33..2b4185014 100644 --- a/pkg/tcpip/stack/route.go +++ b/pkg/tcpip/stack/route.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index d1ec6a660..d4da980a9 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack_global_state.go b/pkg/tcpip/stack/stack_global_state.go index b6c095efb..f2c6c9a8d 100644 --- a/pkg/tcpip/stack/stack_global_state.go +++ b/pkg/tcpip/stack/stack_global_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/stack_test.go b/pkg/tcpip/stack/stack_test.go index a0b3399a8..74bf2c99e 100644 --- a/pkg/tcpip/stack/stack_test.go +++ b/pkg/tcpip/stack/stack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/transport_demuxer.go b/pkg/tcpip/stack/transport_demuxer.go index a7470d606..c8522ad9e 100644 --- a/pkg/tcpip/stack/transport_demuxer.go +++ b/pkg/tcpip/stack/transport_demuxer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/stack/transport_test.go b/pkg/tcpip/stack/transport_test.go index 98cc3b120..f09760180 100644 --- a/pkg/tcpip/stack/transport_test.go +++ b/pkg/tcpip/stack/transport_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go index bf11c2175..413aee6c6 100644 --- a/pkg/tcpip/tcpip.go +++ b/pkg/tcpip/tcpip.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/tcpip_test.go b/pkg/tcpip/tcpip_test.go index d283f71c7..361e359d4 100644 --- a/pkg/tcpip/tcpip_test.go +++ b/pkg/tcpip/tcpip_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/time.s b/pkg/tcpip/time.s index 8aca31bee..85d52cd9c 100644 --- a/pkg/tcpip/time.s +++ b/pkg/tcpip/time.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/time_unsafe.go b/pkg/tcpip/time_unsafe.go index 2102e9633..231151bf3 100644 --- a/pkg/tcpip/time_unsafe.go +++ b/pkg/tcpip/time_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/ping/endpoint.go b/pkg/tcpip/transport/ping/endpoint.go index 055daa918..b3f54cfe0 100644 --- a/pkg/tcpip/transport/ping/endpoint.go +++ b/pkg/tcpip/transport/ping/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/ping/endpoint_state.go b/pkg/tcpip/transport/ping/endpoint_state.go index a16087304..80721d227 100644 --- a/pkg/tcpip/transport/ping/endpoint_state.go +++ b/pkg/tcpip/transport/ping/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/ping/protocol.go b/pkg/tcpip/transport/ping/protocol.go index 549b1b2d3..1d504773b 100644 --- a/pkg/tcpip/transport/ping/protocol.go +++ b/pkg/tcpip/transport/ping/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go index c22ed5ea7..5a88d25d0 100644 --- a/pkg/tcpip/transport/tcp/accept.go +++ b/pkg/tcpip/transport/tcp/accept.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go index 27dbcace2..800d2409e 100644 --- a/pkg/tcpip/transport/tcp/connect.go +++ b/pkg/tcpip/transport/tcp/connect.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/cubic.go b/pkg/tcpip/transport/tcp/cubic.go index 8cea416d2..003525d86 100644 --- a/pkg/tcpip/transport/tcp/cubic.go +++ b/pkg/tcpip/transport/tcp/cubic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/dual_stack_test.go b/pkg/tcpip/transport/tcp/dual_stack_test.go index c88e98977..d3120c1d8 100644 --- a/pkg/tcpip/transport/tcp/dual_stack_test.go +++ b/pkg/tcpip/transport/tcp/dual_stack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go index 707d6be96..673a65c31 100644 --- a/pkg/tcpip/transport/tcp/endpoint.go +++ b/pkg/tcpip/transport/tcp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/endpoint_state.go b/pkg/tcpip/transport/tcp/endpoint_state.go index bed7ec6a6..e32c73aae 100644 --- a/pkg/tcpip/transport/tcp/endpoint_state.go +++ b/pkg/tcpip/transport/tcp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/forwarder.go b/pkg/tcpip/transport/tcp/forwarder.go index c80f3c7d6..2f90839e9 100644 --- a/pkg/tcpip/transport/tcp/forwarder.go +++ b/pkg/tcpip/transport/tcp/forwarder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go index abdc825cd..753e1419e 100644 --- a/pkg/tcpip/transport/tcp/protocol.go +++ b/pkg/tcpip/transport/tcp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/rcv.go b/pkg/tcpip/transport/tcp/rcv.go index 92ef9c6f7..05ff9e0d7 100644 --- a/pkg/tcpip/transport/tcp/rcv.go +++ b/pkg/tcpip/transport/tcp/rcv.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/reno.go b/pkg/tcpip/transport/tcp/reno.go index feb593234..e4f8b7d5a 100644 --- a/pkg/tcpip/transport/tcp/reno.go +++ b/pkg/tcpip/transport/tcp/reno.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/sack.go b/pkg/tcpip/transport/tcp/sack.go index 05bac08cb..24e48fe7b 100644 --- a/pkg/tcpip/transport/tcp/sack.go +++ b/pkg/tcpip/transport/tcp/sack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment.go b/pkg/tcpip/transport/tcp/segment.go index 51a3d6aba..fc87a05fd 100644 --- a/pkg/tcpip/transport/tcp/segment.go +++ b/pkg/tcpip/transport/tcp/segment.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_heap.go b/pkg/tcpip/transport/tcp/segment_heap.go index e3a3405ef..98422fadf 100644 --- a/pkg/tcpip/transport/tcp/segment_heap.go +++ b/pkg/tcpip/transport/tcp/segment_heap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_queue.go b/pkg/tcpip/transport/tcp/segment_queue.go index 6a2d7bc0b..0c637d7ad 100644 --- a/pkg/tcpip/transport/tcp/segment_queue.go +++ b/pkg/tcpip/transport/tcp/segment_queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/segment_state.go b/pkg/tcpip/transport/tcp/segment_state.go index 22f0bbf18..46b6d85a6 100644 --- a/pkg/tcpip/transport/tcp/segment_state.go +++ b/pkg/tcpip/transport/tcp/segment_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go index 0bd421ff4..eefe93d48 100644 --- a/pkg/tcpip/transport/tcp/snd.go +++ b/pkg/tcpip/transport/tcp/snd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/snd_state.go b/pkg/tcpip/transport/tcp/snd_state.go index d536839af..86bbd643f 100644 --- a/pkg/tcpip/transport/tcp/snd_state.go +++ b/pkg/tcpip/transport/tcp/snd_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_sack_test.go b/pkg/tcpip/transport/tcp/tcp_sack_test.go index a61d0ca64..06b0702c5 100644 --- a/pkg/tcpip/transport/tcp/tcp_sack_test.go +++ b/pkg/tcpip/transport/tcp/tcp_sack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go index 48852ea47..04e046257 100644 --- a/pkg/tcpip/transport/tcp/tcp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go index ca16fc8fa..b08df0fec 100644 --- a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/testing/context/context.go b/pkg/tcpip/transport/tcp/testing/context/context.go index 5b25534f4..0695e8150 100644 --- a/pkg/tcpip/transport/tcp/testing/context/context.go +++ b/pkg/tcpip/transport/tcp/testing/context/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcp/timer.go b/pkg/tcpip/transport/tcp/timer.go index 938c0bcef..38240d2d5 100644 --- a/pkg/tcpip/transport/tcp/timer.go +++ b/pkg/tcpip/transport/tcp/timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go index 5f8f1a64d..f7b2900de 100644 --- a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go +++ b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go index 514722ab7..aaeae9b18 100644 --- a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go +++ b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go index 840e95302..d777a80d0 100644 --- a/pkg/tcpip/transport/udp/endpoint.go +++ b/pkg/tcpip/transport/udp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/endpoint_state.go b/pkg/tcpip/transport/udp/endpoint_state.go index 70a37c7f2..db1e281ad 100644 --- a/pkg/tcpip/transport/udp/endpoint_state.go +++ b/pkg/tcpip/transport/udp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/protocol.go b/pkg/tcpip/transport/udp/protocol.go index 1334fec8a..b3fbed6e4 100644 --- a/pkg/tcpip/transport/udp/protocol.go +++ b/pkg/tcpip/transport/udp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go index c3f592bd4..58a346cd9 100644 --- a/pkg/tcpip/transport/udp/udp_test.go +++ b/pkg/tcpip/transport/udp/udp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tmutex/tmutex.go b/pkg/tmutex/tmutex.go index bd5c681dd..df61d89f5 100644 --- a/pkg/tmutex/tmutex.go +++ b/pkg/tmutex/tmutex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tmutex/tmutex_test.go b/pkg/tmutex/tmutex_test.go index a9dc9972f..a4537cb3b 100644 --- a/pkg/tmutex/tmutex_test.go +++ b/pkg/tmutex/tmutex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet.go b/pkg/unet/unet.go index f4800e0d9..deeea078d 100644 --- a/pkg/unet/unet.go +++ b/pkg/unet/unet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet_test.go b/pkg/unet/unet_test.go index 6c546825f..ecc670925 100644 --- a/pkg/unet/unet_test.go +++ b/pkg/unet/unet_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/unet/unet_unsafe.go b/pkg/unet/unet_unsafe.go index fa15cf744..1d69de542 100644 --- a/pkg/unet/unet_unsafe.go +++ b/pkg/unet/unet_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/urpc/urpc.go b/pkg/urpc/urpc.go index 1ec06dd4c..753366be2 100644 --- a/pkg/urpc/urpc.go +++ b/pkg/urpc/urpc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/urpc/urpc_test.go b/pkg/urpc/urpc_test.go index d9cfc512e..f1b9a85ca 100644 --- a/pkg/urpc/urpc_test.go +++ b/pkg/urpc/urpc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/fdnotifier/fdnotifier.go b/pkg/waiter/fdnotifier/fdnotifier.go index 8bb93e39b..624b1a0c5 100644 --- a/pkg/waiter/fdnotifier/fdnotifier.go +++ b/pkg/waiter/fdnotifier/fdnotifier.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/fdnotifier/poll_unsafe.go b/pkg/waiter/fdnotifier/poll_unsafe.go index 26bca2b53..8459d4c74 100644 --- a/pkg/waiter/fdnotifier/poll_unsafe.go +++ b/pkg/waiter/fdnotifier/poll_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/waiter.go b/pkg/waiter/waiter.go index 832b6a5a9..93390b299 100644 --- a/pkg/waiter/waiter.go +++ b/pkg/waiter/waiter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/waiter/waiter_test.go b/pkg/waiter/waiter_test.go index c45f22889..60853f9c1 100644 --- a/pkg/waiter/waiter_test.go +++ b/pkg/waiter/waiter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/compat.go b/runsc/boot/compat.go index 3250cdcdc..6766953b3 100644 --- a/runsc/boot/compat.go +++ b/runsc/boot/compat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/config.go b/runsc/boot/config.go index 51d20d06d..9ebbde424 100644 --- a/runsc/boot/config.go +++ b/runsc/boot/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go index bee82f344..6dd7fadd9 100644 --- a/runsc/boot/controller.go +++ b/runsc/boot/controller.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/debug.go b/runsc/boot/debug.go index 971962c91..d224d08b7 100644 --- a/runsc/boot/debug.go +++ b/runsc/boot/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/events.go b/runsc/boot/events.go index 595846b10..f954b8c0b 100644 --- a/runsc/boot/events.go +++ b/runsc/boot/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/fds.go b/runsc/boot/fds.go index 9416e3a5c..a3d21d963 100644 --- a/runsc/boot/fds.go +++ b/runsc/boot/fds.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/config.go b/runsc/boot/filter/config.go index 92a73db9a..378396b9b 100644 --- a/runsc/boot/filter/config.go +++ b/runsc/boot/filter/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters.go b/runsc/boot/filter/extra_filters.go index 82cf00dfb..67f3101fe 100644 --- a/runsc/boot/filter/extra_filters.go +++ b/runsc/boot/filter/extra_filters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters_msan.go b/runsc/boot/filter/extra_filters_msan.go index 76f3f6865..fb95283ab 100644 --- a/runsc/boot/filter/extra_filters_msan.go +++ b/runsc/boot/filter/extra_filters_msan.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/extra_filters_race.go b/runsc/boot/filter/extra_filters_race.go index ebd56c553..02a122c95 100644 --- a/runsc/boot/filter/extra_filters_race.go +++ b/runsc/boot/filter/extra_filters_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/filter/filter.go b/runsc/boot/filter/filter.go index b656883ad..dc7294b1d 100644 --- a/runsc/boot/filter/filter.go +++ b/runsc/boot/filter/filter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go index ea825e571..e52c89fe4 100644 --- a/runsc/boot/fs.go +++ b/runsc/boot/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/limits.go b/runsc/boot/limits.go index 510497eba..8ecda6d0e 100644 --- a/runsc/boot/limits.go +++ b/runsc/boot/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go index c79b95bde..fa3de0133 100644 --- a/runsc/boot/loader.go +++ b/runsc/boot/loader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/loader_test.go b/runsc/boot/loader_test.go index 41ff3681b..c342ee005 100644 --- a/runsc/boot/loader_test.go +++ b/runsc/boot/loader_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/network.go b/runsc/boot/network.go index 6a2678ac9..89f186139 100644 --- a/runsc/boot/network.go +++ b/runsc/boot/network.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/boot/strace.go b/runsc/boot/strace.go index 1e898672b..028bcc1f4 100644 --- a/runsc/boot/strace.go +++ b/runsc/boot/strace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cgroup/cgroup.go b/runsc/cgroup/cgroup.go index 7a75a189a..d6058a8a2 100644 --- a/runsc/cgroup/cgroup.go +++ b/runsc/cgroup/cgroup.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cgroup/cgroup_test.go b/runsc/cgroup/cgroup_test.go index cde915329..4a4713d4f 100644 --- a/runsc/cgroup/cgroup_test.go +++ b/runsc/cgroup/cgroup_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/boot.go b/runsc/cmd/boot.go index 023b63dc0..7c14857ba 100644 --- a/runsc/cmd/boot.go +++ b/runsc/cmd/boot.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/capability.go b/runsc/cmd/capability.go index 0b18c5481..e5da021e5 100644 --- a/runsc/cmd/capability.go +++ b/runsc/cmd/capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/capability_test.go b/runsc/cmd/capability_test.go index 3329b308d..dd278b32d 100644 --- a/runsc/cmd/capability_test.go +++ b/runsc/cmd/capability_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/checkpoint.go b/runsc/cmd/checkpoint.go index 023ab2455..d49d0169b 100644 --- a/runsc/cmd/checkpoint.go +++ b/runsc/cmd/checkpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/cmd.go b/runsc/cmd/cmd.go index 2937ae1c4..a1c3491a3 100644 --- a/runsc/cmd/cmd.go +++ b/runsc/cmd/cmd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/create.go b/runsc/cmd/create.go index 275a96f57..b84185b43 100644 --- a/runsc/cmd/create.go +++ b/runsc/cmd/create.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/debug.go b/runsc/cmd/debug.go index cb7d81057..288cbe435 100644 --- a/runsc/cmd/debug.go +++ b/runsc/cmd/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/delete.go b/runsc/cmd/delete.go index 92b609c3c..ea1ca1278 100644 --- a/runsc/cmd/delete.go +++ b/runsc/cmd/delete.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/delete_test.go b/runsc/cmd/delete_test.go index f6d164394..4a5b4774a 100644 --- a/runsc/cmd/delete_test.go +++ b/runsc/cmd/delete_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/events.go b/runsc/cmd/events.go index df65ea31d..df03415ec 100644 --- a/runsc/cmd/events.go +++ b/runsc/cmd/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/exec.go b/runsc/cmd/exec.go index 336edf3f6..9a395e6f1 100644 --- a/runsc/cmd/exec.go +++ b/runsc/cmd/exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/exec_test.go b/runsc/cmd/exec_test.go index 623461e78..686c5e150 100644 --- a/runsc/cmd/exec_test.go +++ b/runsc/cmd/exec_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/gofer.go b/runsc/cmd/gofer.go index fd4eee546..3842fdf64 100644 --- a/runsc/cmd/gofer.go +++ b/runsc/cmd/gofer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/kill.go b/runsc/cmd/kill.go index 7a98d10a2..1f1086250 100644 --- a/runsc/cmd/kill.go +++ b/runsc/cmd/kill.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/list.go b/runsc/cmd/list.go index 4d4a5cb0b..fd59b73e6 100644 --- a/runsc/cmd/list.go +++ b/runsc/cmd/list.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/path.go b/runsc/cmd/path.go index c207b80da..baba937a8 100644 --- a/runsc/cmd/path.go +++ b/runsc/cmd/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/pause.go b/runsc/cmd/pause.go index ac393b48e..5ff6f059c 100644 --- a/runsc/cmd/pause.go +++ b/runsc/cmd/pause.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/ps.go b/runsc/cmd/ps.go index 5d219bfdc..fd76cf975 100644 --- a/runsc/cmd/ps.go +++ b/runsc/cmd/ps.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/restore.go b/runsc/cmd/restore.go index 6dc044672..cc99b3503 100644 --- a/runsc/cmd/restore.go +++ b/runsc/cmd/restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/resume.go b/runsc/cmd/resume.go index a12adf1a3..274b5d084 100644 --- a/runsc/cmd/resume.go +++ b/runsc/cmd/resume.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/run.go b/runsc/cmd/run.go index 9a87cf240..b6a12f5d6 100644 --- a/runsc/cmd/run.go +++ b/runsc/cmd/run.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/spec.go b/runsc/cmd/spec.go index 6281fc49d..57ee37c86 100644 --- a/runsc/cmd/spec.go +++ b/runsc/cmd/spec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/start.go b/runsc/cmd/start.go index 97ea91fff..48bd4c401 100644 --- a/runsc/cmd/start.go +++ b/runsc/cmd/start.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/state.go b/runsc/cmd/state.go index 265014e1b..f8ce8c3d8 100644 --- a/runsc/cmd/state.go +++ b/runsc/cmd/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/wait.go b/runsc/cmd/wait.go index 956349140..121c54554 100644 --- a/runsc/cmd/wait.go +++ b/runsc/cmd/wait.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/console/console.go b/runsc/console/console.go index 3df184742..9f4f9214d 100644 --- a/runsc/console/console.go +++ b/runsc/console/console.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/console_test.go b/runsc/container/console_test.go index 8f019b54a..0b0dfb4cb 100644 --- a/runsc/container/console_test.go +++ b/runsc/container/console_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/container.go b/runsc/container/container.go index f76bad1aa..cb4c9b5c1 100644 --- a/runsc/container/container.go +++ b/runsc/container/container.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/container_test.go b/runsc/container/container_test.go index 662591b3b..243528d35 100644 --- a/runsc/container/container_test.go +++ b/runsc/container/container_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/fs.go b/runsc/container/fs.go index 2ed42fd93..41022686b 100644 --- a/runsc/container/fs.go +++ b/runsc/container/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/fs_test.go b/runsc/container/fs_test.go index 84bde18fb..87cdb078e 100644 --- a/runsc/container/fs_test.go +++ b/runsc/container/fs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/hook.go b/runsc/container/hook.go index 3d93ca0be..6b9e5550a 100644 --- a/runsc/container/hook.go +++ b/runsc/container/hook.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/multi_container_test.go b/runsc/container/multi_container_test.go index 1781a4602..4548eb106 100644 --- a/runsc/container/multi_container_test.go +++ b/runsc/container/multi_container_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/status.go b/runsc/container/status.go index bf177e78a..234ffb0dd 100644 --- a/runsc/container/status.go +++ b/runsc/container/status.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/test_app.go b/runsc/container/test_app.go index cc3b087e1..b5071ada6 100644 --- a/runsc/container/test_app.go +++ b/runsc/container/test_app.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/config.go b/runsc/fsgofer/filter/config.go index 35698f21f..75a087848 100644 --- a/runsc/fsgofer/filter/config.go +++ b/runsc/fsgofer/filter/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters.go b/runsc/fsgofer/filter/extra_filters.go index 82cf00dfb..67f3101fe 100644 --- a/runsc/fsgofer/filter/extra_filters.go +++ b/runsc/fsgofer/filter/extra_filters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters_msan.go b/runsc/fsgofer/filter/extra_filters_msan.go index 169a79ed8..7e142b790 100644 --- a/runsc/fsgofer/filter/extra_filters_msan.go +++ b/runsc/fsgofer/filter/extra_filters_msan.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/extra_filters_race.go b/runsc/fsgofer/filter/extra_filters_race.go index 9e6512d8c..3cd29472a 100644 --- a/runsc/fsgofer/filter/extra_filters_race.go +++ b/runsc/fsgofer/filter/extra_filters_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/filter/filter.go b/runsc/fsgofer/filter/filter.go index 6f341f688..f50b6bc87 100644 --- a/runsc/fsgofer/filter/filter.go +++ b/runsc/fsgofer/filter/filter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer.go b/runsc/fsgofer/fsgofer.go index 9c4864cf1..e03bb7752 100644 --- a/runsc/fsgofer/fsgofer.go +++ b/runsc/fsgofer/fsgofer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer_test.go b/runsc/fsgofer/fsgofer_test.go index a500a2976..48860f952 100644 --- a/runsc/fsgofer/fsgofer_test.go +++ b/runsc/fsgofer/fsgofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/fsgofer/fsgofer_unsafe.go b/runsc/fsgofer/fsgofer_unsafe.go index e676809ac..99bc25ec1 100644 --- a/runsc/fsgofer/fsgofer_unsafe.go +++ b/runsc/fsgofer/fsgofer_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/main.go b/runsc/main.go index 62b1f01b3..4a92db7c0 100644 --- a/runsc/main.go +++ b/runsc/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/chroot.go b/runsc/sandbox/chroot.go index 35b19a0b1..354049871 100644 --- a/runsc/sandbox/chroot.go +++ b/runsc/sandbox/chroot.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/network.go b/runsc/sandbox/network.go index 86a52c6ae..52fe8fc0f 100644 --- a/runsc/sandbox/network.go +++ b/runsc/sandbox/network.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/sandbox.go b/runsc/sandbox/sandbox.go index 923a52f7f..0fe85cfe1 100644 --- a/runsc/sandbox/sandbox.go +++ b/runsc/sandbox/sandbox.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/namespace.go b/runsc/specutils/namespace.go index 00293d45b..73fab13e1 100644 --- a/runsc/specutils/namespace.go +++ b/runsc/specutils/namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/specutils.go b/runsc/specutils/specutils.go index b29802fde..ab14ed1fc 100644 --- a/runsc/specutils/specutils.go +++ b/runsc/specutils/specutils.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/specutils/specutils_test.go b/runsc/specutils/specutils_test.go index 64e2172c8..b61f1ca62 100644 --- a/runsc/specutils/specutils_test.go +++ b/runsc/specutils/specutils_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/image.go b/runsc/test/image/image.go index 069d08013..bcb6f876f 100644 --- a/runsc/test/image/image.go +++ b/runsc/test/image/image.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/image_test.go b/runsc/test/image/image_test.go index d89d80a86..763152b47 100644 --- a/runsc/test/image/image_test.go +++ b/runsc/test/image/image_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/image/mysql.sql b/runsc/test/image/mysql.sql index dd5bfaa4e..c1271e719 100644 --- a/runsc/test/image/mysql.sql +++ b/runsc/test/image/mysql.sql @@ -1,4 +1,4 @@ -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/image/ruby.rb b/runsc/test/image/ruby.rb index ae5de3419..25d1ac129 100644 --- a/runsc/test/image/ruby.rb +++ b/runsc/test/image/ruby.rb @@ -1,4 +1,4 @@ -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/image/ruby.sh b/runsc/test/image/ruby.sh index 54be2c931..d3a9b5656 100644 --- a/runsc/test/image/ruby.sh +++ b/runsc/test/image/ruby.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/install.sh b/runsc/test/install.sh index c239588d4..32e1e884e 100755 --- a/runsc/test/install.sh +++ b/runsc/test/install.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/exec_test.go b/runsc/test/integration/exec_test.go index 3cac674d0..fac8337f4 100644 --- a/runsc/test/integration/exec_test.go +++ b/runsc/test/integration/exec_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/integration.go b/runsc/test/integration/integration.go index 49c3c893a..e15321c87 100644 --- a/runsc/test/integration/integration.go +++ b/runsc/test/integration/integration.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/integration/integration_test.go b/runsc/test/integration/integration_test.go index 536bb17e0..526b3a7a1 100644 --- a/runsc/test/integration/integration_test.go +++ b/runsc/test/integration/integration_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/cgroup_test.go b/runsc/test/root/cgroup_test.go index 5cb4b794f..fdb94ff64 100644 --- a/runsc/test/root/cgroup_test.go +++ b/runsc/test/root/cgroup_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/chroot_test.go b/runsc/test/root/chroot_test.go index 8831e6a78..0ffaaf87b 100644 --- a/runsc/test/root/chroot_test.go +++ b/runsc/test/root/chroot_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/root/root.go b/runsc/test/root/root.go index 790f62c29..586ea0fe3 100644 --- a/runsc/test/root/root.go +++ b/runsc/test/root/root.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/docker.go b/runsc/test/testutil/docker.go index 7d6a72e5f..3f74e0770 100644 --- a/runsc/test/testutil/docker.go +++ b/runsc/test/testutil/docker.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/testutil.go b/runsc/test/testutil/testutil.go index 4d7ac3bc9..1b5a02c0f 100644 --- a/runsc/test/testutil/testutil.go +++ b/runsc/test/testutil/testutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/test/testutil/testutil_race.go b/runsc/test/testutil/testutil_race.go index 59cfdaa7b..9267af150 100644 --- a/runsc/test/testutil/testutil_race.go +++ b/runsc/test/testutil/testutil_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/tools/dockercfg/dockercfg.go b/runsc/tools/dockercfg/dockercfg.go index 0bd6cad93..110a581ff 100644 --- a/runsc/tools/dockercfg/dockercfg.go +++ b/runsc/tools/dockercfg/dockercfg.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics.go b/tools/go_generics/generics.go index cc61a7537..eaf5c4970 100644 --- a/tools/go_generics/generics.go +++ b/tools/go_generics/generics.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_stmts/input.go b/tools/go_generics/generics_tests/all_stmts/input.go index 870af3b6c..19184a3fe 100644 --- a/tools/go_generics/generics_tests/all_stmts/input.go +++ b/tools/go_generics/generics_tests/all_stmts/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_stmts/output/output.go b/tools/go_generics/generics_tests/all_stmts/output/output.go index e4e670bf1..51582346c 100644 --- a/tools/go_generics/generics_tests/all_stmts/output/output.go +++ b/tools/go_generics/generics_tests/all_stmts/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/input.go b/tools/go_generics/generics_tests/all_types/input.go index 3a8643e3d..ed6e97c29 100644 --- a/tools/go_generics/generics_tests/all_types/input.go +++ b/tools/go_generics/generics_tests/all_types/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/lib/lib.go b/tools/go_generics/generics_tests/all_types/lib/lib.go index d3911d12d..7e73e678e 100644 --- a/tools/go_generics/generics_tests/all_types/lib/lib.go +++ b/tools/go_generics/generics_tests/all_types/lib/lib.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/all_types/output/output.go b/tools/go_generics/generics_tests/all_types/output/output.go index b89840936..ec09a6be4 100644 --- a/tools/go_generics/generics_tests/all_types/output/output.go +++ b/tools/go_generics/generics_tests/all_types/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/consts/input.go b/tools/go_generics/generics_tests/consts/input.go index dabf76e1e..394bcc262 100644 --- a/tools/go_generics/generics_tests/consts/input.go +++ b/tools/go_generics/generics_tests/consts/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/consts/output/output.go b/tools/go_generics/generics_tests/consts/output/output.go index 72865607e..91a07fdc2 100644 --- a/tools/go_generics/generics_tests/consts/output/output.go +++ b/tools/go_generics/generics_tests/consts/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/imports/input.go b/tools/go_generics/generics_tests/imports/input.go index 66b43fee5..22e6641a6 100644 --- a/tools/go_generics/generics_tests/imports/input.go +++ b/tools/go_generics/generics_tests/imports/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/imports/output/output.go b/tools/go_generics/generics_tests/imports/output/output.go index 5f20d43ce..2555c0004 100644 --- a/tools/go_generics/generics_tests/imports/output/output.go +++ b/tools/go_generics/generics_tests/imports/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/remove_typedef/input.go b/tools/go_generics/generics_tests/remove_typedef/input.go index c02307d32..d9c9b8530 100644 --- a/tools/go_generics/generics_tests/remove_typedef/input.go +++ b/tools/go_generics/generics_tests/remove_typedef/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/remove_typedef/output/output.go b/tools/go_generics/generics_tests/remove_typedef/output/output.go index d20a89abd..f111a9426 100644 --- a/tools/go_generics/generics_tests/remove_typedef/output/output.go +++ b/tools/go_generics/generics_tests/remove_typedef/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/simple/input.go b/tools/go_generics/generics_tests/simple/input.go index 670161d6e..711687cf5 100644 --- a/tools/go_generics/generics_tests/simple/input.go +++ b/tools/go_generics/generics_tests/simple/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/generics_tests/simple/output/output.go b/tools/go_generics/generics_tests/simple/output/output.go index 75b5467cd..139c9bf9d 100644 --- a/tools/go_generics/generics_tests/simple/output/output.go +++ b/tools/go_generics/generics_tests/simple/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/globals/globals_visitor.go b/tools/go_generics/globals/globals_visitor.go index fc0de4381..daaa17b1d 100644 --- a/tools/go_generics/globals/globals_visitor.go +++ b/tools/go_generics/globals/globals_visitor.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/globals/scope.go b/tools/go_generics/globals/scope.go index 18743bdee..b75a91689 100644 --- a/tools/go_generics/globals/scope.go +++ b/tools/go_generics/globals/scope.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/go_generics_unittest.sh b/tools/go_generics/go_generics_unittest.sh index 699e1f631..e7553a071 100755 --- a/tools/go_generics/go_generics_unittest.sh +++ b/tools/go_generics/go_generics_unittest.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/tools/go_generics/imports.go b/tools/go_generics/imports.go index 97267098b..57f7c3dce 100644 --- a/tools/go_generics/imports.go +++ b/tools/go_generics/imports.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/merge.go b/tools/go_generics/merge.go index ebe7cf4e4..2f83facf8 100644 --- a/tools/go_generics/merge.go +++ b/tools/go_generics/merge.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/remove.go b/tools/go_generics/remove.go index 2a66de762..139d03955 100644 --- a/tools/go_generics/remove.go +++ b/tools/go_generics/remove.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/rules_tests/template.go b/tools/go_generics/rules_tests/template.go index 73c024f0e..f3f31ae8e 100644 --- a/tools/go_generics/rules_tests/template.go +++ b/tools/go_generics/rules_tests/template.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_generics/rules_tests/template_test.go b/tools/go_generics/rules_tests/template_test.go index 76c4cdb64..3a38c8629 100644 --- a/tools/go_generics/rules_tests/template_test.go +++ b/tools/go_generics/rules_tests/template_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/go_stateify/main.go b/tools/go_stateify/main.go index 5646b879a..9e2c8e106 100644 --- a/tools/go_stateify/main.go +++ b/tools/go_stateify/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/tools/workspace_status.sh b/tools/workspace_status.sh index d89db1f99..7d44dad37 100755 --- a/tools/workspace_status.sh +++ b/tools/workspace_status.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/vdso/barrier.h b/vdso/barrier.h index db8185b2e..7866af414 100644 --- a/vdso/barrier.h +++ b/vdso/barrier.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/check_vdso.py b/vdso/check_vdso.py index 9a3142ab8..6f7d7e7ec 100644 --- a/vdso/check_vdso.py +++ b/vdso/check_vdso.py @@ -1,4 +1,4 @@ -# Copyright 2018 Google Inc. +# Copyright 2018 Google LLC # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/vdso/compiler.h b/vdso/compiler.h index a661516c3..d65f148fb 100644 --- a/vdso/compiler.h +++ b/vdso/compiler.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/cycle_clock.h b/vdso/cycle_clock.h index 93c5f2c0d..dfb5b427d 100644 --- a/vdso/cycle_clock.h +++ b/vdso/cycle_clock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/seqlock.h b/vdso/seqlock.h index b527bdbca..ab2f3fda3 100644 --- a/vdso/seqlock.h +++ b/vdso/seqlock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/syscalls.h b/vdso/syscalls.h index fd79c4642..0be8a7f9b 100644 --- a/vdso/syscalls.h +++ b/vdso/syscalls.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso.cc b/vdso/vdso.cc index db3bdef01..f30dc26a2 100644 --- a/vdso/vdso.cc +++ b/vdso/vdso.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso_time.cc b/vdso/vdso_time.cc index 5d5c8de65..a59771bff 100644 --- a/vdso/vdso_time.cc +++ b/vdso/vdso_time.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/vdso/vdso_time.h b/vdso/vdso_time.h index 71d6e2f64..464dadff2 100644 --- a/vdso/vdso_time.h +++ b/vdso/vdso_time.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google Inc. +// Copyright 2018 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. -- cgit v1.2.3 From b2068cf5a5d43f3898cf389ab2d6151cf61908ac Mon Sep 17 00:00:00 2001 From: Fabricio Voznika Date: Sat, 20 Oct 2018 11:12:26 -0700 Subject: Add more unimplemented syscall events Added events for *ctl syscalls that may have multiple different commands. For runsc, each syscall event is only logged once. For *ctl syscalls, use the cmd as identifier, not only the syscall number. PiperOrigin-RevId: 218015941 Change-Id: Ie3c19131ae36124861e9b492a7dbe1765d9e5e59 --- pkg/abi/linux/ioctl.go | 75 +++++++++++++------ pkg/abi/linux/prctl.go | 99 +++++++++++++++++++++++-- pkg/sentry/fs/host/BUILD | 1 + pkg/sentry/fs/host/tty.go | 30 ++++++++ pkg/sentry/fs/tty/BUILD | 1 + pkg/sentry/fs/tty/master.go | 45 ++++++++++- pkg/sentry/fs/tty/slave.go | 3 +- pkg/sentry/kernel/BUILD | 2 + pkg/sentry/kernel/kernel.go | 17 +++++ pkg/sentry/kernel/pipe/reader_writer.go | 3 +- pkg/sentry/kernel/task.go | 3 + pkg/sentry/socket/epsocket/BUILD | 1 + pkg/sentry/socket/epsocket/epsocket.go | 3 + pkg/sentry/socket/rpcinet/BUILD | 1 + pkg/sentry/socket/rpcinet/socket.go | 5 ++ pkg/sentry/syscalls/BUILD | 18 ----- pkg/sentry/syscalls/linux/linux64.go | 2 +- pkg/sentry/syscalls/linux/sys_prctl.go | 39 +++++++++- pkg/sentry/syscalls/linux/sys_shm.go | 1 + pkg/sentry/syscalls/linux/sys_tls.go | 3 + pkg/sentry/syscalls/syscalls.go | 15 +--- pkg/sentry/syscalls/unimplemented_syscall.proto | 27 ------- pkg/sentry/unimpl/BUILD | 30 ++++++++ pkg/sentry/unimpl/events.go | 45 +++++++++++ pkg/sentry/unimpl/unimplemented_syscall.proto | 27 +++++++ runsc/boot/BUILD | 9 ++- runsc/boot/compat.go | 72 ++++++++++++++++-- runsc/boot/compat_amd64.go | 54 ++++++++++++++ runsc/boot/compat_test.go | 66 +++++++++++++++++ 29 files changed, 596 insertions(+), 101 deletions(-) delete mode 100644 pkg/sentry/syscalls/unimplemented_syscall.proto create mode 100644 pkg/sentry/unimpl/BUILD create mode 100644 pkg/sentry/unimpl/events.go create mode 100644 pkg/sentry/unimpl/unimplemented_syscall.proto create mode 100644 runsc/boot/compat_amd64.go create mode 100644 runsc/boot/compat_test.go (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/abi/linux/ioctl.go b/pkg/abi/linux/ioctl.go index 9afc3d1ef..191b26e4d 100644 --- a/pkg/abi/linux/ioctl.go +++ b/pkg/abi/linux/ioctl.go @@ -18,28 +18,59 @@ package linux // // These are ordered by request number (low byte). const ( - TCGETS = 0x00005401 - TCSETS = 0x00005402 - TCSETSW = 0x00005403 - TCSETSF = 0x00005404 - TIOCSCTTY = 0x0000540e - TIOCGPGRP = 0x0000540f - TIOCSPGRP = 0x00005410 - TIOCOUTQ = 0x00005411 - TIOCGWINSZ = 0x00005413 - TIOCSWINSZ = 0x00005414 - TIOCINQ = 0x0000541b - FIONREAD = TIOCINQ - FIONBIO = 0x00005421 - TIOCGPTN = 0x80045430 - TIOCSPTLCK = 0x40045431 - FIONCLEX = 0x00005450 - FIOCLEX = 0x00005451 - FIOASYNC = 0x00005452 - FIOSETOWN = 0x00008901 - SIOCSPGRP = 0x00008902 - FIOGETOWN = 0x00008903 - SIOCGPGRP = 0x00008904 + TCGETS = 0x00005401 + TCSETS = 0x00005402 + TCSETSW = 0x00005403 + TCSETSF = 0x00005404 + TCSBRK = 0x00005409 + TIOCEXCL = 0x0000540c + TIOCNXCL = 0x0000540d + TIOCSCTTY = 0x0000540e + TIOCGPGRP = 0x0000540f + TIOCSPGRP = 0x00005410 + TIOCOUTQ = 0x00005411 + TIOCSTI = 0x00005412 + TIOCGWINSZ = 0x00005413 + TIOCSWINSZ = 0x00005414 + TIOCMGET = 0x00005415 + TIOCMBIS = 0x00005416 + TIOCMBIC = 0x00005417 + TIOCMSET = 0x00005418 + TIOCINQ = 0x0000541b + FIONREAD = TIOCINQ + FIONBIO = 0x00005421 + TIOCSETD = 0x00005423 + TIOCNOTTY = 0x00005422 + TIOCGETD = 0x00005424 + TCSBRKP = 0x00005425 + TIOCSBRK = 0x00005427 + TIOCCBRK = 0x00005428 + TIOCGSID = 0x00005429 + TIOCGPTN = 0x80045430 + TIOCSPTLCK = 0x40045431 + TIOCGDEV = 0x80045432 + TIOCVHANGUP = 0x00005437 + TCFLSH = 0x0000540b + TIOCCONS = 0x0000541d + TIOCSSERIAL = 0x0000541f + TIOCGEXCL = 0x80045440 + TIOCGPTPEER = 0x80045441 + TIOCGICOUNT = 0x0000545d + FIONCLEX = 0x00005450 + FIOCLEX = 0x00005451 + FIOASYNC = 0x00005452 + FIOSETOWN = 0x00008901 + SIOCSPGRP = 0x00008902 + FIOGETOWN = 0x00008903 + SIOCGPGRP = 0x00008904 +) + +// ioctl(2) requests provided by uapi/linux/sockios.h +const ( + SIOCGIFMEM = 0x891f + SIOCGIFPFLAGS = 0x8935 + SIOCGMIIPHY = 0x8947 + SIOCGMIIREG = 0x8948 ) // ioctl(2) requests provided by uapi/linux/android/binder.h diff --git a/pkg/abi/linux/prctl.go b/pkg/abi/linux/prctl.go index e152c4c27..db3206f36 100644 --- a/pkg/abi/linux/prctl.go +++ b/pkg/abi/linux/prctl.go @@ -22,26 +22,102 @@ const ( // PR_GET_PDEATHSIG will get the process' death signal. PR_GET_PDEATHSIG = 2 + // PR_GET_DUMPABLE will get the process's dumpable flag. + PR_GET_DUMPABLE = 3 + + // PR_SET_DUMPABLE will set the process's dumpable flag. + PR_SET_DUMPABLE = 4 + // PR_GET_KEEPCAPS will get the value of the keep capabilities flag. PR_GET_KEEPCAPS = 7 // PR_SET_KEEPCAPS will set the value of the keep capabilities flag. PR_SET_KEEPCAPS = 8 + // PR_GET_TIMING will get the process's timing method. + PR_GET_TIMING = 13 + + // PR_SET_TIMING will set the process's timing method. + PR_SET_TIMING = 14 + // PR_SET_NAME will set the process' name. PR_SET_NAME = 15 // PR_GET_NAME will get the process' name. PR_GET_NAME = 16 + // PR_GET_SECCOMP will get a process' seccomp mode. + PR_GET_SECCOMP = 21 + + // PR_SET_SECCOMP will set a process' seccomp mode. + PR_SET_SECCOMP = 22 + + // PR_CAPBSET_READ will get the capability bounding set. + PR_CAPBSET_READ = 23 + + // PR_CAPBSET_DROP will set the capability bounding set. + PR_CAPBSET_DROP = 24 + + // PR_GET_TSC will get the the value of the flag determining whether the + // timestamp counter can be read. + PR_GET_TSC = 25 + + // PR_SET_TSC will set the the value of the flag determining whether the + // timestamp counter can be read. + PR_SET_TSC = 26 + + // PR_SET_TIMERSLACK set the process's time slack. + PR_SET_TIMERSLACK = 29 + + // PR_GET_TIMERSLACK get the process's time slack. + PR_GET_TIMERSLACK = 30 + + // PR_TASK_PERF_EVENTS_DISABLE disable all performance counters attached to + // the calling process. + PR_TASK_PERF_EVENTS_DISABLE = 31 + + // PR_TASK_PERF_EVENTS_ENABLE enable all performance counters attached to + // the calling process. + PR_TASK_PERF_EVENTS_ENABLE = 32 + + // PR_MCE_KILL set the machine check memory corruption kill policy for the + // calling thread. + PR_MCE_KILL = 33 + + // PR_MCE_KILL_GET get the machine check memory corruption kill policy for the + // calling thread. + PR_MCE_KILL_GET = 34 + // PR_SET_MM will modify certain kernel memory map descriptor fields of the // calling process. See prctl(2) for more information. PR_SET_MM = 35 + PR_SET_MM_START_CODE = 1 + PR_SET_MM_END_CODE = 2 + PR_SET_MM_START_DATA = 3 + PR_SET_MM_END_DATA = 4 + PR_SET_MM_START_STACK = 5 + PR_SET_MM_START_BRK = 6 + PR_SET_MM_BRK = 7 + PR_SET_MM_ARG_START = 8 + PR_SET_MM_ARG_END = 9 + PR_SET_MM_ENV_START = 10 + PR_SET_MM_ENV_END = 11 + PR_SET_MM_AUXV = 12 // PR_SET_MM_EXE_FILE will supersede the /proc/pid/exe symbolic link with a // new one pointing to a new executable file identified by the file descriptor // provided in arg3 argument. See prctl(2) for more information. PR_SET_MM_EXE_FILE = 13 + PR_SET_MM_MAP = 14 + PR_SET_MM_MAP_SIZE = 15 + + // PR_SET_CHILD_SUBREAPER set the "child subreaper" attribute of the calling + // process. + PR_SET_CHILD_SUBREAPER = 36 + + // PR_GET_CHILD_SUBREAPER get the "child subreaper" attribute of the calling + // process. + PR_GET_CHILD_SUBREAPER = 37 // PR_SET_NO_NEW_PRIVS will set the calling thread's no_new_privs bit. PR_SET_NO_NEW_PRIVS = 38 @@ -49,17 +125,24 @@ const ( // PR_GET_NO_NEW_PRIVS will get the calling thread's no_new_privs bit. PR_GET_NO_NEW_PRIVS = 39 - // PR_SET_SECCOMP will set a process' seccomp mode. - PR_SET_SECCOMP = 22 + // PR_GET_TID_ADDRESS retrieve the clear_child_tid address. + PR_GET_TID_ADDRESS = 40 - // PR_GET_SECCOMP will get a process' seccomp mode. - PR_GET_SECCOMP = 21 + // PR_SET_THP_DISABLE set the state of the "THP disable" flag for the calling + // thread. + PR_SET_THP_DISABLE = 41 - // PR_CAPBSET_READ will get the capability bounding set. - PR_CAPBSET_READ = 23 + // PR_GET_THP_DISABLE get the state of the "THP disable" flag for the calling + // thread. + PR_GET_THP_DISABLE = 42 - // PR_CAPBSET_DROP will set the capability bounding set. - PR_CAPBSET_DROP = 24 + // PR_MPX_ENABLE_MANAGEMENT enable kernel management of Memory Protection + // eXtensions (MPX) bounds tables. + PR_MPX_ENABLE_MANAGEMENT = 43 + + // PR_MPX_DISABLE_MANAGEMENTdisable kernel management of Memory Protection + // eXtensions (MPX) bounds tables. + PR_MPX_DISABLE_MANAGEMENT = 44 ) // From diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD index 4f264a024..d1eb9bd64 100644 --- a/pkg/sentry/fs/host/BUILD +++ b/pkg/sentry/fs/host/BUILD @@ -43,6 +43,7 @@ go_library( "//pkg/sentry/socket/control", "//pkg/sentry/socket/unix", "//pkg/sentry/socket/unix/transport", + "//pkg/sentry/unimpl", "//pkg/sentry/uniqueid", "//pkg/sentry/usermem", "//pkg/syserr", diff --git a/pkg/sentry/fs/host/tty.go b/pkg/sentry/fs/host/tty.go index cf3639c46..f0bcdc908 100644 --- a/pkg/sentry/fs/host/tty.go +++ b/pkg/sentry/fs/host/tty.go @@ -22,6 +22,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" ) @@ -179,6 +180,35 @@ func (t *TTYFileOperations) Ioctl(ctx context.Context, io usermem.IO, args arch. err := ioctlSetWinsize(fd, &winsize) return 0, err + // Unimplemented commands. + case linux.TIOCSETD, + linux.TIOCSBRK, + linux.TIOCCBRK, + linux.TCSBRK, + linux.TCSBRKP, + linux.TIOCSTI, + linux.TIOCCONS, + linux.FIONBIO, + linux.TIOCEXCL, + linux.TIOCNXCL, + linux.TIOCGEXCL, + linux.TIOCNOTTY, + linux.TIOCSCTTY, + linux.TIOCGSID, + linux.TIOCGETD, + linux.TIOCVHANGUP, + linux.TIOCGDEV, + linux.TIOCMGET, + linux.TIOCMSET, + linux.TIOCMBIC, + linux.TIOCMBIS, + linux.TIOCGICOUNT, + linux.TCFLSH, + linux.TIOCSSERIAL, + linux.TIOCGPTPEER: + + unimpl.EmitUnimplementedEvent(ctx) + fallthrough default: return 0, syserror.ENOTTY } diff --git a/pkg/sentry/fs/tty/BUILD b/pkg/sentry/fs/tty/BUILD index d4dd20e30..2b45069a6 100644 --- a/pkg/sentry/fs/tty/BUILD +++ b/pkg/sentry/fs/tty/BUILD @@ -27,6 +27,7 @@ go_library( "//pkg/sentry/kernel/auth", "//pkg/sentry/kernel/time", "//pkg/sentry/socket/unix/transport", + "//pkg/sentry/unimpl", "//pkg/sentry/usermem", "//pkg/syserror", "//pkg/waiter", diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index dad0cad79..00bec4c2c 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -20,6 +20,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/waiter" @@ -149,7 +150,7 @@ func (mf *masterFileOperations) Write(ctx context.Context, _ *fs.File, src userm // Ioctl implements fs.FileOperations.Ioctl. func (mf *masterFileOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { - switch args[1].Uint() { + switch cmd := args[1].Uint(); cmd { case linux.FIONREAD: // linux.FIONREAD == linux.TIOCINQ // Get the number of bytes in the output queue read buffer. return 0, mf.t.ld.outputQueueReadSize(ctx, io, args) @@ -177,6 +178,48 @@ func (mf *masterFileOperations) Ioctl(ctx context.Context, io usermem.IO, args a case linux.TIOCSWINSZ: return 0, mf.t.ld.setWindowSize(ctx, io, args) default: + maybeEmitUnimplementedEvent(ctx, cmd) return 0, syserror.ENOTTY } } + +// maybeEmitUnimplementedEvent emits unimplemented event if cmd is valid. +func maybeEmitUnimplementedEvent(ctx context.Context, cmd uint32) { + switch cmd { + case linux.TCGETS, + linux.TCSETS, + linux.TCSETSW, + linux.TCSETSF, + linux.TIOCGPGRP, + linux.TIOCSPGRP, + linux.TIOCGWINSZ, + linux.TIOCSWINSZ, + linux.TIOCSETD, + linux.TIOCSBRK, + linux.TIOCCBRK, + linux.TCSBRK, + linux.TCSBRKP, + linux.TIOCSTI, + linux.TIOCCONS, + linux.FIONBIO, + linux.TIOCEXCL, + linux.TIOCNXCL, + linux.TIOCGEXCL, + linux.TIOCNOTTY, + linux.TIOCSCTTY, + linux.TIOCGSID, + linux.TIOCGETD, + linux.TIOCVHANGUP, + linux.TIOCGDEV, + linux.TIOCMGET, + linux.TIOCMSET, + linux.TIOCMBIC, + linux.TIOCMBIS, + linux.TIOCGICOUNT, + linux.TCFLSH, + linux.TIOCSSERIAL, + linux.TIOCGPTPEER: + + unimpl.EmitUnimplementedEvent(ctx) + } +} diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index 9de3168bf..a696fbb51 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -134,7 +134,7 @@ func (sf *slaveFileOperations) Write(ctx context.Context, _ *fs.File, src userme // Ioctl implements fs.FileOperations.Ioctl. func (sf *slaveFileOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { - switch args[1].Uint() { + switch cmd := args[1].Uint(); cmd { case linux.FIONREAD: // linux.FIONREAD == linux.TIOCINQ // Get the number of bytes in the input queue read buffer. return 0, sf.si.t.ld.inputQueueReadSize(ctx, io, args) @@ -161,6 +161,7 @@ func (sf *slaveFileOperations) Ioctl(ctx context.Context, io usermem.IO, args ar // control. return 0, nil default: + maybeEmitUnimplementedEvent(ctx, cmd) return 0, syserror.ENOTTY } } diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD index e2fb61ba6..389824b25 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -157,6 +157,8 @@ go_library( "//pkg/sentry/socket/netlink/port", "//pkg/sentry/socket/unix/transport", "//pkg/sentry/time", + "//pkg/sentry/unimpl", + "//pkg/sentry/unimpl:unimplemented_syscall_go_proto", "//pkg/sentry/uniqueid", "//pkg/sentry/usage", "//pkg/sentry/usermem", diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index bad558d48..17425e656 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -40,6 +40,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/cpuid" + "gvisor.googlesource.com/gvisor/pkg/eventchannel" "gvisor.googlesource.com/gvisor/pkg/log" "gvisor.googlesource.com/gvisor/pkg/sentry/arch" "gvisor.googlesource.com/gvisor/pkg/sentry/context" @@ -58,6 +59,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/platform" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink/port" sentrytime "gvisor.googlesource.com/gvisor/pkg/sentry/time" + "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl" + uspb "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl/unimplemented_syscall_go_proto" "gvisor.googlesource.com/gvisor/pkg/sentry/uniqueid" "gvisor.googlesource.com/gvisor/pkg/state" "gvisor.googlesource.com/gvisor/pkg/tcpip" @@ -595,6 +598,8 @@ func (ctx *createProcessContext) Value(key interface{}) interface{} { return ctx.k case uniqueid.CtxInotifyCookie: return ctx.k.GenerateInotifyCookie() + case unimpl.CtxEvents: + return ctx.k default: return nil } @@ -1033,6 +1038,16 @@ func (k *Kernel) SupervisorContext() context.Context { } } +// EmitUnimplementedEvent emits an UnimplementedSyscall event via the event +// channel. +func (k *Kernel) EmitUnimplementedEvent(ctx context.Context) { + t := TaskFromContext(ctx) + eventchannel.Emit(&uspb.UnimplementedSyscall{ + Tid: int32(t.ThreadID()), + Registers: t.Arch().StateData().Proto(), + }) +} + type supervisorContext struct { context.NoopSleeper log.Logger @@ -1073,6 +1088,8 @@ func (ctx supervisorContext) Value(key interface{}) interface{} { return ctx.k case uniqueid.CtxInotifyCookie: return ctx.k.GenerateInotifyCookie() + case unimpl.CtxEvents: + return ctx.k default: return nil } diff --git a/pkg/sentry/kernel/pipe/reader_writer.go b/pkg/sentry/kernel/pipe/reader_writer.go index 63efc5bbe..36be1efc3 100644 --- a/pkg/sentry/kernel/pipe/reader_writer.go +++ b/pkg/sentry/kernel/pipe/reader_writer.go @@ -19,6 +19,7 @@ import ( "math" "syscall" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/arch" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" @@ -77,7 +78,7 @@ func (rw *ReaderWriter) Readiness(mask waiter.EventMask) waiter.EventMask { func (rw *ReaderWriter) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { // Switch on ioctl request. switch int(args[1].Int()) { - case syscall.TIOCINQ: + case linux.FIONREAD: v := rw.queuedSize() if v > math.MaxInt32 { panic(fmt.Sprintf("Impossibly large pipe queued size: %d", v)) diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go index e22ec768d..73ba8bee9 100644 --- a/pkg/sentry/kernel/task.go +++ b/pkg/sentry/kernel/task.go @@ -30,6 +30,7 @@ import ( ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/limits" "gvisor.googlesource.com/gvisor/pkg/sentry/platform" + "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl" "gvisor.googlesource.com/gvisor/pkg/sentry/uniqueid" "gvisor.googlesource.com/gvisor/pkg/sentry/usage" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" @@ -594,6 +595,8 @@ func (t *Task) Value(key interface{}) interface{} { return t.k case uniqueid.CtxInotifyCookie: return t.k.GenerateInotifyCookie() + case unimpl.CtxEvents: + return t.k default: return nil } diff --git a/pkg/sentry/socket/epsocket/BUILD b/pkg/sentry/socket/epsocket/BUILD index dbabc931c..da4aaf510 100644 --- a/pkg/sentry/socket/epsocket/BUILD +++ b/pkg/sentry/socket/epsocket/BUILD @@ -32,6 +32,7 @@ go_library( "//pkg/sentry/safemem", "//pkg/sentry/socket", "//pkg/sentry/socket/unix/transport", + "//pkg/sentry/unimpl", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index e90ef4835..39a0b9941 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -45,6 +45,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" + "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" @@ -1184,6 +1185,8 @@ func Ioctl(ctx context.Context, ep commonEndpoint, io usermem.IO, args arch.Sysc }) return 0, err + case linux.SIOCGIFMEM, linux.SIOCGIFPFLAGS, linux.SIOCGMIIPHY, linux.SIOCGMIIREG: + unimpl.EmitUnimplementedEvent(ctx) } return 0, syserror.ENOTTY diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index 3ea433360..38fa54283 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -32,6 +32,7 @@ go_library( "//pkg/sentry/socket/rpcinet/conn", "//pkg/sentry/socket/rpcinet/notifier", "//pkg/sentry/socket/unix/transport", + "//pkg/sentry/unimpl", "//pkg/sentry/usermem", "//pkg/syserr", "//pkg/syserror", diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 44fa5c620..788d853c9 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -32,6 +32,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/notifier" pb "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet/syscall_rpc_go_proto" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" + "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserr" "gvisor.googlesource.com/gvisor/pkg/syserror" @@ -555,6 +556,10 @@ func (s *socketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.S }) return 0, err + + case linux.SIOCGIFMEM, linux.SIOCGIFPFLAGS, linux.SIOCGMIIPHY, linux.SIOCGMIIREG: + unimpl.EmitUnimplementedEvent(ctx) + default: return 0, syserror.ENOTTY } diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD index 22a757095..2a9f0915e 100644 --- a/pkg/sentry/syscalls/BUILD +++ b/pkg/sentry/syscalls/BUILD @@ -1,7 +1,6 @@ package(licenses = ["notice"]) # Apache 2.0 load("//tools/go_stateify:defs.bzl", "go_library") -load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") go_library( name = "syscalls", @@ -13,9 +12,7 @@ go_library( importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/syscalls", visibility = ["//:sandbox"], deps = [ - ":unimplemented_syscall_go_proto", "//pkg/abi/linux", - "//pkg/eventchannel", "//pkg/sentry/arch", "//pkg/sentry/fs", "//pkg/sentry/kernel", @@ -26,18 +23,3 @@ go_library( "//pkg/waiter", ], ) - -proto_library( - name = "unimplemented_syscall_proto", - srcs = ["unimplemented_syscall.proto"], - visibility = ["//visibility:public"], - deps = ["//pkg/sentry/arch:registers_proto"], -) - -go_proto_library( - name = "unimplemented_syscall_go_proto", - importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/syscalls/unimplemented_syscall_go_proto", - proto = ":unimplemented_syscall_proto", - visibility = ["//visibility:public"], - deps = ["//pkg/sentry/arch:registers_go_proto"], -) diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go index 75e87f5ec..11bf81f88 100644 --- a/pkg/sentry/syscalls/linux/linux64.go +++ b/pkg/sentry/syscalls/linux/linux64.go @@ -369,7 +369,7 @@ var AMD64 = &kernel.SyscallTable{ 0xffffffffff600800: 309, // vsyscall getcpu(2) }, Missing: func(t *kernel.Task, sysno uintptr, args arch.SyscallArguments) (uintptr, error) { - syscalls.UnimplementedEvent(t) + t.Kernel().EmitUnimplementedEvent(t) return 0, syserror.ENOSYS }, } diff --git a/pkg/sentry/syscalls/linux/sys_prctl.go b/pkg/sentry/syscalls/linux/sys_prctl.go index c7b39ede8..91e852049 100644 --- a/pkg/sentry/syscalls/linux/sys_prctl.go +++ b/pkg/sentry/syscalls/linux/sys_prctl.go @@ -104,6 +104,22 @@ func Prctl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall // Set the underlying executable. t.MemoryManager().SetExecutable(file.Dirent) + + case linux.PR_SET_MM_AUXV, + linux.PR_SET_MM_START_CODE, + linux.PR_SET_MM_END_CODE, + linux.PR_SET_MM_START_DATA, + linux.PR_SET_MM_END_DATA, + linux.PR_SET_MM_START_STACK, + linux.PR_SET_MM_START_BRK, + linux.PR_SET_MM_BRK, + linux.PR_SET_MM_ARG_START, + linux.PR_SET_MM_ARG_END, + linux.PR_SET_MM_ENV_START, + linux.PR_SET_MM_ENV_END: + + t.Kernel().EmitUnimplementedEvent(t) + fallthrough default: return 0, nil, syscall.EINVAL } @@ -151,8 +167,29 @@ func Prctl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall } return 0, nil, t.DropBoundingCapability(cp) + case linux.PR_GET_DUMPABLE, + linux.PR_SET_DUMPABLE, + linux.PR_GET_TIMING, + linux.PR_SET_TIMING, + linux.PR_GET_TSC, + linux.PR_SET_TSC, + linux.PR_TASK_PERF_EVENTS_DISABLE, + linux.PR_TASK_PERF_EVENTS_ENABLE, + linux.PR_GET_TIMERSLACK, + linux.PR_SET_TIMERSLACK, + linux.PR_MCE_KILL, + linux.PR_MCE_KILL_GET, + linux.PR_GET_TID_ADDRESS, + linux.PR_SET_CHILD_SUBREAPER, + linux.PR_GET_CHILD_SUBREAPER, + linux.PR_GET_THP_DISABLE, + linux.PR_SET_THP_DISABLE, + linux.PR_MPX_ENABLE_MANAGEMENT, + linux.PR_MPX_DISABLE_MANAGEMENT: + + t.Kernel().EmitUnimplementedEvent(t) + fallthrough default: - t.Warningf("Unsupported prctl %d", option) return 0, nil, syscall.EINVAL } diff --git a/pkg/sentry/syscalls/linux/sys_shm.go b/pkg/sentry/syscalls/linux/sys_shm.go index b13d48b98..5f887523a 100644 --- a/pkg/sentry/syscalls/linux/sys_shm.go +++ b/pkg/sentry/syscalls/linux/sys_shm.go @@ -147,6 +147,7 @@ func Shmctl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscal // We currently do not support memmory locking anywhere. // mlock(2)/munlock(2) are currently stubbed out as no-ops so do the // same here. + t.Kernel().EmitUnimplementedEvent(t) return 0, nil, nil default: diff --git a/pkg/sentry/syscalls/linux/sys_tls.go b/pkg/sentry/syscalls/linux/sys_tls.go index 27ddb3808..40e84825b 100644 --- a/pkg/sentry/syscalls/linux/sys_tls.go +++ b/pkg/sentry/syscalls/linux/sys_tls.go @@ -45,6 +45,9 @@ func ArchPrctl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sys regs.Fs = 0 regs.Fs_base = fsbase + case linux.ARCH_GET_GS, linux.ARCH_SET_GS: + t.Kernel().EmitUnimplementedEvent(t) + fallthrough default: return 0, nil, syscall.EINVAL } diff --git a/pkg/sentry/syscalls/syscalls.go b/pkg/sentry/syscalls/syscalls.go index bae32d727..425ce900c 100644 --- a/pkg/sentry/syscalls/syscalls.go +++ b/pkg/sentry/syscalls/syscalls.go @@ -26,10 +26,8 @@ package syscalls import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" - "gvisor.googlesource.com/gvisor/pkg/eventchannel" "gvisor.googlesource.com/gvisor/pkg/sentry/arch" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" - uspb "gvisor.googlesource.com/gvisor/pkg/sentry/syscalls/unimplemented_syscall_go_proto" "gvisor.googlesource.com/gvisor/pkg/syserror" ) @@ -44,7 +42,7 @@ func Error(err error) kernel.SyscallFn { // syscall event via the event channel and returns the passed error. func ErrorWithEvent(err error) kernel.SyscallFn { return func(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { - UnimplementedEvent(t) + t.Kernel().EmitUnimplementedEvent(t) return 0, nil, err } } @@ -57,16 +55,7 @@ func CapError(c linux.Capability) kernel.SyscallFn { if !t.HasCapability(c) { return 0, nil, syserror.EPERM } - UnimplementedEvent(t) + t.Kernel().EmitUnimplementedEvent(t) return 0, nil, syserror.ENOSYS } } - -// UnimplementedEvent emits an UnimplementedSyscall event via the event -// channel. -func UnimplementedEvent(t *kernel.Task) { - eventchannel.Emit(&uspb.UnimplementedSyscall{ - Tid: int32(t.ThreadID()), - Registers: t.Arch().StateData().Proto(), - }) -} diff --git a/pkg/sentry/syscalls/unimplemented_syscall.proto b/pkg/sentry/syscalls/unimplemented_syscall.proto deleted file mode 100644 index 41579b016..000000000 --- a/pkg/sentry/syscalls/unimplemented_syscall.proto +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright 2018 Google LLC -// -// 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. - -syntax = "proto3"; - -package gvisor; - -import "pkg/sentry/arch/registers.proto"; - -message UnimplementedSyscall { - // Task ID. - int32 tid = 1; - - // Registers at the time of the call. - Registers registers = 2; -} diff --git a/pkg/sentry/unimpl/BUILD b/pkg/sentry/unimpl/BUILD new file mode 100644 index 000000000..63da5e81f --- /dev/null +++ b/pkg/sentry/unimpl/BUILD @@ -0,0 +1,30 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("//tools/go_stateify:defs.bzl", "go_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + +proto_library( + name = "unimplemented_syscall_proto", + srcs = ["unimplemented_syscall.proto"], + visibility = ["//visibility:public"], + deps = ["//pkg/sentry/arch:registers_proto"], +) + +go_proto_library( + name = "unimplemented_syscall_go_proto", + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl/unimplemented_syscall_go_proto", + proto = ":unimplemented_syscall_proto", + visibility = ["//visibility:public"], + deps = ["//pkg/sentry/arch:registers_go_proto"], +) + +go_library( + name = "unimpl", + srcs = ["events.go"], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl", + visibility = ["//:sandbox"], + deps = [ + "//pkg/log", + "//pkg/sentry/context", + ], +) diff --git a/pkg/sentry/unimpl/events.go b/pkg/sentry/unimpl/events.go new file mode 100644 index 000000000..f78f8c981 --- /dev/null +++ b/pkg/sentry/unimpl/events.go @@ -0,0 +1,45 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package unimpl contains interface to emit events about unimplemented +// features. +package unimpl + +import ( + "gvisor.googlesource.com/gvisor/pkg/log" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" +) + +// contextID is the events package's type for context.Context.Value keys. +type contextID int + +const ( + // CtxEvents is a Context.Value key for a Events. + CtxEvents contextID = iota +) + +// Events interface defines method to emit unsupported events. +type Events interface { + EmitUnimplementedEvent(context.Context) +} + +// EmitUnimplementedEvent emits unsupported syscall event to the context. +func EmitUnimplementedEvent(ctx context.Context) { + e := ctx.Value(CtxEvents) + if e == nil { + log.Warningf("Context.Value(CtxEvents) not present, unimplemented syscall event not reported.") + return + } + e.(Events).EmitUnimplementedEvent(ctx) +} diff --git a/pkg/sentry/unimpl/unimplemented_syscall.proto b/pkg/sentry/unimpl/unimplemented_syscall.proto new file mode 100644 index 000000000..41579b016 --- /dev/null +++ b/pkg/sentry/unimpl/unimplemented_syscall.proto @@ -0,0 +1,27 @@ +// Copyright 2018 Google LLC +// +// 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. + +syntax = "proto3"; + +package gvisor; + +import "pkg/sentry/arch/registers.proto"; + +message UnimplementedSyscall { + // Task ID. + int32 tid = 1; + + // Registers at the time of the call. + Registers registers = 2; +} diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD index f8f848ebf..04cc0e854 100644 --- a/runsc/boot/BUILD +++ b/runsc/boot/BUILD @@ -6,6 +6,7 @@ go_library( name = "boot", srcs = [ "compat.go", + "compat_amd64.go", "config.go", "controller.go", "debug.go", @@ -59,9 +60,9 @@ go_library( "//pkg/sentry/socket/unix", "//pkg/sentry/state", "//pkg/sentry/strace", - "//pkg/sentry/syscalls:unimplemented_syscall_go_proto", "//pkg/sentry/syscalls/linux", "//pkg/sentry/time", + "//pkg/sentry/unimpl:unimplemented_syscall_go_proto", "//pkg/sentry/usage", "//pkg/sentry/watchdog", "//pkg/syserror", @@ -87,12 +88,16 @@ go_library( go_test( name = "boot_test", size = "small", - srcs = ["loader_test.go"], + srcs = [ + "compat_test.go", + "loader_test.go", + ], embed = [":boot"], deps = [ "//pkg/control/server", "//pkg/log", "//pkg/p9", + "//pkg/sentry/arch:registers_go_proto", "//pkg/sentry/context/contexttest", "//pkg/sentry/fs", "//pkg/unet", diff --git a/runsc/boot/compat.go b/runsc/boot/compat.go index 6766953b3..d18c2f802 100644 --- a/runsc/boot/compat.go +++ b/runsc/boot/compat.go @@ -17,6 +17,8 @@ package boot import ( "fmt" "os" + "sync" + "syscall" "github.com/golang/protobuf/proto" "gvisor.googlesource.com/gvisor/pkg/abi" @@ -25,7 +27,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/arch" rpb "gvisor.googlesource.com/gvisor/pkg/sentry/arch/registers_go_proto" "gvisor.googlesource.com/gvisor/pkg/sentry/strace" - spb "gvisor.googlesource.com/gvisor/pkg/sentry/syscalls/unimplemented_syscall_go_proto" + spb "gvisor.googlesource.com/gvisor/pkg/sentry/unimpl/unimplemented_syscall_go_proto" ) func initCompatLogs(fd int) error { @@ -40,15 +42,27 @@ func initCompatLogs(fd int) error { type compatEmitter struct { sink *log.BasicLogger nameMap strace.SyscallMap + + // mu protects the fields below. + mu sync.Mutex + + // trackers map syscall number to the respective tracker instance. + // Protected by 'mu'. + trackers map[uint64]syscallTracker } func newCompatEmitter(logFD int) (*compatEmitter, error) { - // Always logs to default logger. nameMap, ok := strace.Lookup(abi.Linux, arch.AMD64) if !ok { return nil, fmt.Errorf("amd64 Linux syscall table not found") } - c := &compatEmitter{sink: log.Log(), nameMap: nameMap} + + c := &compatEmitter{ + // Always logs to default logger. + sink: log.Log(), + nameMap: nameMap, + trackers: make(map[uint64]syscallTracker), + } if logFD > 0 { f := os.NewFile(uintptr(logFD), "user log file") @@ -61,10 +75,33 @@ func newCompatEmitter(logFD int) (*compatEmitter, error) { // Emit implements eventchannel.Emitter. func (c *compatEmitter) Emit(msg proto.Message) (hangup bool, err error) { // Only interested in UnimplementedSyscall, skip the rest. - if us, ok := msg.(*spb.UnimplementedSyscall); ok { - regs := us.Registers.GetArch().(*rpb.Registers_Amd64).Amd64 - sysnr := regs.OrigRax + us, ok := msg.(*spb.UnimplementedSyscall) + if !ok { + return false, nil + } + regs := us.Registers.GetArch().(*rpb.Registers_Amd64).Amd64 + + c.mu.Lock() + defer c.mu.Unlock() + + sysnr := regs.OrigRax + tr := c.trackers[sysnr] + if tr == nil { + switch sysnr { + case syscall.SYS_PRCTL, syscall.SYS_ARCH_PRCTL: + tr = newCmdTracker(0) + + case syscall.SYS_IOCTL, syscall.SYS_EPOLL_CTL, syscall.SYS_SHMCTL: + tr = newCmdTracker(1) + + default: + tr = &onceTracker{} + } + c.trackers[sysnr] = tr + } + if tr.shouldReport(regs) { c.sink.Infof("Unsupported syscall: %s, regs: %+v", c.nameMap.Name(uintptr(sysnr)), regs) + tr.onReported(regs) } return false, nil } @@ -74,3 +111,26 @@ func (c *compatEmitter) Close() error { c.sink = nil return nil } + +// syscallTracker interface allows filters to apply differently depending on +// the syscall and arguments. +type syscallTracker interface { + // shouldReport returns true is the syscall should be reported. + shouldReport(regs *rpb.AMD64Registers) bool + + // onReported marks the syscall as reported. + onReported(regs *rpb.AMD64Registers) +} + +// onceTracker reports only a single time, used for most syscalls. +type onceTracker struct { + reported bool +} + +func (o *onceTracker) shouldReport(_ *rpb.AMD64Registers) bool { + return !o.reported +} + +func (o *onceTracker) onReported(_ *rpb.AMD64Registers) { + o.reported = true +} diff --git a/runsc/boot/compat_amd64.go b/runsc/boot/compat_amd64.go new file mode 100644 index 000000000..2bb769a49 --- /dev/null +++ b/runsc/boot/compat_amd64.go @@ -0,0 +1,54 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package boot + +import ( + "fmt" + + rpb "gvisor.googlesource.com/gvisor/pkg/sentry/arch/registers_go_proto" +) + +// cmdTracker reports only a single time for each different command argument in +// the syscall. It's used for generic syscalls like ioctl to report once per +// 'cmd' +type cmdTracker struct { + // argIdx is the syscall argument index where the command is located. + argIdx int + cmds map[uint32]struct{} +} + +func newCmdTracker(argIdx int) *cmdTracker { + return &cmdTracker{argIdx: argIdx, cmds: make(map[uint32]struct{})} +} + +// cmd returns the command based on the syscall argument index. +func (c *cmdTracker) cmd(regs *rpb.AMD64Registers) uint32 { + switch c.argIdx { + case 0: + return uint32(regs.Rdi) + case 1: + return uint32(regs.Rsi) + } + panic(fmt.Sprintf("unsupported syscall argument index %d", c.argIdx)) +} + +func (c *cmdTracker) shouldReport(regs *rpb.AMD64Registers) bool { + _, ok := c.cmds[c.cmd(regs)] + return !ok +} + +func (c *cmdTracker) onReported(regs *rpb.AMD64Registers) { + c.cmds[c.cmd(regs)] = struct{}{} +} diff --git a/runsc/boot/compat_test.go b/runsc/boot/compat_test.go new file mode 100644 index 000000000..30b94798a --- /dev/null +++ b/runsc/boot/compat_test.go @@ -0,0 +1,66 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package boot + +import ( + "testing" + + rpb "gvisor.googlesource.com/gvisor/pkg/sentry/arch/registers_go_proto" +) + +func TestOnceTracker(t *testing.T) { + o := onceTracker{} + if !o.shouldReport(nil) { + t.Error("first call to checkAndMark, got: false, want: true") + } + o.onReported(nil) + for i := 0; i < 2; i++ { + if o.shouldReport(nil) { + t.Error("after first call to checkAndMark, got: true, want: false") + } + } +} + +func TestCmdTracker(t *testing.T) { + for _, tc := range []struct { + name string + idx int + rdi1 uint64 + rdi2 uint64 + rsi1 uint64 + rsi2 uint64 + want bool + }{ + {name: "same rdi", idx: 0, rdi1: 123, rdi2: 123, want: false}, + {name: "same rsi", idx: 1, rsi1: 123, rsi2: 123, want: false}, + {name: "diff rdi", idx: 0, rdi1: 123, rdi2: 321, want: true}, + {name: "diff rsi", idx: 1, rsi1: 123, rsi2: 321, want: true}, + {name: "cmd is uint32", idx: 0, rsi1: 0xdead00000123, rsi2: 0xbeef00000123, want: false}, + } { + t.Run(tc.name, func(t *testing.T) { + c := newCmdTracker(tc.idx) + regs := &rpb.AMD64Registers{Rdi: tc.rdi1, Rsi: tc.rsi1} + if !c.shouldReport(regs) { + t.Error("first call to checkAndMark, got: false, want: true") + } + c.onReported(regs) + + regs.Rdi, regs.Rsi = tc.rdi2, tc.rsi2 + if got := c.shouldReport(regs); tc.want != got { + t.Errorf("after first call to checkAndMark, got: %t, want: %t", got, tc.want) + } + }) + } +} -- cgit v1.2.3 From 75cd70ecc9abfd5daaefea04da5070a0e0d620dd Mon Sep 17 00:00:00 2001 From: Adin Scannell Date: Tue, 23 Oct 2018 00:19:11 -0700 Subject: Track paths and provide a rename hook. This change also adds extensive testing to the p9 package via mocks. The sanity checks and type checks are moved from the gofer into the core package, where they can be more easily validated. PiperOrigin-RevId: 218296768 Change-Id: I4fc3c326e7bf1e0e140a454cbacbcc6fd617ab55 --- WORKSPACE | 20 +- pkg/amutex/BUILD | 4 +- pkg/atomicbitops/BUILD | 4 +- pkg/binary/BUILD | 4 +- pkg/bits/BUILD | 3 +- pkg/compressio/BUILD | 4 +- pkg/control/client/BUILD | 4 +- pkg/control/server/BUILD | 4 +- pkg/dhcp/BUILD | 4 +- pkg/eventchannel/BUILD | 4 +- pkg/fd/BUILD | 4 +- pkg/gate/BUILD | 4 +- pkg/ilist/BUILD | 4 +- pkg/linewriter/BUILD | 4 +- pkg/log/BUILD | 4 +- pkg/metric/BUILD | 4 +- pkg/p9/BUILD | 2 + pkg/p9/buffer_test.go | 31 + pkg/p9/client.go | 6 + pkg/p9/client_file.go | 4 +- pkg/p9/file.go | 151 +- pkg/p9/handlers.go | 697 ++++++-- pkg/p9/local_server/BUILD | 4 +- pkg/p9/local_server/local_server.go | 5 + pkg/p9/messages_test.go | 37 + pkg/p9/p9.go | 24 + pkg/p9/p9test/BUILD | 76 +- pkg/p9/p9test/client_test.go | 2263 ++++++++++++++++++++++--- pkg/p9/p9test/mocks.go | 489 ------ pkg/p9/p9test/p9test.go | 329 ++++ pkg/p9/path_tree.go | 109 ++ pkg/p9/server.go | 228 ++- pkg/p9/transport.go | 10 +- pkg/rand/BUILD | 4 +- pkg/seccomp/BUILD | 4 +- pkg/secio/BUILD | 4 +- pkg/sentry/arch/BUILD | 3 +- pkg/sentry/context/BUILD | 4 +- pkg/sentry/control/BUILD | 4 +- pkg/sentry/device/BUILD | 4 +- pkg/sentry/fs/anon/BUILD | 4 +- pkg/sentry/fs/gofer/BUILD | 4 - pkg/sentry/fs/gofer/context_file.go | 7 - pkg/sentry/fs/gofer/gofer_test.go | 894 ++-------- pkg/sentry/fs/gofer/session.go | 9 +- pkg/sentry/fs/gofer/session_state.go | 4 +- pkg/sentry/fs/proc/device/BUILD | 4 +- pkg/sentry/hostcpu/BUILD | 4 +- pkg/sentry/kernel/kdefs/BUILD | 4 +- pkg/sentry/kernel/memevent/BUILD | 4 +- pkg/sentry/kernel/sched/BUILD | 4 +- pkg/sentry/loader/BUILD | 3 +- pkg/sentry/memutil/BUILD | 4 +- pkg/sentry/platform/interrupt/BUILD | 4 +- pkg/sentry/platform/kvm/BUILD | 3 +- pkg/sentry/platform/kvm/testutil/BUILD | 4 +- pkg/sentry/platform/procid/BUILD | 4 +- pkg/sentry/platform/ptrace/BUILD | 4 +- pkg/sentry/platform/ring0/BUILD | 3 +- pkg/sentry/platform/ring0/gen_offsets/BUILD | 3 +- pkg/sentry/platform/ring0/pagetables/BUILD | 3 +- pkg/sentry/platform/safecopy/BUILD | 4 +- pkg/sentry/safemem/BUILD | 4 +- pkg/sentry/sighandling/BUILD | 4 +- pkg/sentry/socket/rpcinet/BUILD | 4 +- pkg/sentry/socket/rpcinet/conn/BUILD | 4 +- pkg/sentry/socket/rpcinet/notifier/BUILD | 4 +- pkg/sentry/state/BUILD | 4 +- pkg/sentry/strace/BUILD | 4 +- pkg/sentry/syscalls/BUILD | 4 +- pkg/sentry/time/BUILD | 3 +- pkg/sentry/unimpl/BUILD | 4 +- pkg/sentry/uniqueid/BUILD | 4 +- pkg/sentry/watchdog/BUILD | 4 +- pkg/sleep/BUILD | 4 +- pkg/state/BUILD | 5 +- pkg/state/statefile/BUILD | 4 +- pkg/sync/atomicptrtest/BUILD | 3 +- pkg/sync/seqatomictest/BUILD | 3 +- pkg/syserr/BUILD | 4 +- pkg/syserror/BUILD | 4 +- pkg/tcpip/adapters/gonet/BUILD | 4 +- pkg/tcpip/checker/BUILD | 4 +- pkg/tcpip/link/channel/BUILD | 4 +- pkg/tcpip/link/fdbased/BUILD | 4 +- pkg/tcpip/link/loopback/BUILD | 4 +- pkg/tcpip/link/rawfile/BUILD | 4 +- pkg/tcpip/link/sharedmem/BUILD | 4 +- pkg/tcpip/link/sharedmem/pipe/BUILD | 4 +- pkg/tcpip/link/sharedmem/queue/BUILD | 4 +- pkg/tcpip/link/sniffer/BUILD | 4 +- pkg/tcpip/link/tun/BUILD | 4 +- pkg/tcpip/link/waitable/BUILD | 4 +- pkg/tcpip/network/BUILD | 4 +- pkg/tcpip/network/arp/BUILD | 4 +- pkg/tcpip/network/hash/BUILD | 4 +- pkg/tcpip/network/ipv4/BUILD | 4 +- pkg/tcpip/network/ipv6/BUILD | 4 +- pkg/tcpip/ports/BUILD | 4 +- pkg/tcpip/sample/tun_tcp_connect/BUILD | 4 +- pkg/tcpip/sample/tun_tcp_echo/BUILD | 4 +- pkg/tcpip/transport/tcp/testing/context/BUILD | 4 +- pkg/tcpip/transport/tcpconntrack/BUILD | 4 +- pkg/tmutex/BUILD | 4 +- pkg/unet/BUILD | 4 +- pkg/urpc/BUILD | 4 +- pkg/waiter/fdnotifier/BUILD | 4 +- runsc/boot/BUILD | 4 +- runsc/boot/filter/BUILD | 4 +- runsc/cgroup/BUILD | 4 +- runsc/cmd/BUILD | 4 +- runsc/console/BUILD | 4 +- runsc/container/BUILD | 4 +- runsc/fsgofer/BUILD | 4 +- runsc/fsgofer/filter/BUILD | 4 +- runsc/fsgofer/fsgofer.go | 98 +- runsc/fsgofer/fsgofer_test.go | 78 +- runsc/sandbox/BUILD | 4 +- runsc/specutils/BUILD | 4 +- runsc/test/image/BUILD | 4 +- runsc/test/integration/BUILD | 4 +- runsc/test/root/BUILD | 4 +- runsc/test/testutil/BUILD | 4 +- runsc/tools/dockercfg/BUILD | 4 +- tools/go_generics/BUILD | 4 +- tools/go_generics/globals/BUILD | 4 +- tools/go_generics/rules_tests/BUILD | 3 +- tools/go_stateify/BUILD | 4 +- 128 files changed, 3834 insertions(+), 2147 deletions(-) create mode 100644 pkg/p9/buffer_test.go delete mode 100644 pkg/p9/p9test/mocks.go create mode 100644 pkg/p9/p9test/p9test.go create mode 100644 pkg/p9/path_tree.go (limited to 'pkg/sentry/socket/rpcinet') diff --git a/WORKSPACE b/WORKSPACE index 48e0d3436..841a23e06 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -15,7 +15,7 @@ go_register_toolchains(go_version="1.11.1") load("@bazel_gazelle//:deps.bzl", "gazelle_dependencies", "go_repository") gazelle_dependencies() -# Add dependencies on external repositories. +# External repositories, in sorted order. go_repository( name = "com_github_cenkalti_backoff", importpath = "github.com/cenkalti/backoff", @@ -28,6 +28,12 @@ go_repository( commit = "886344bea0798d02ff3fae16a922be5f6b26cee0" ) +go_repository( + name = "com_github_golang_mock", + importpath = "github.com/golang/mock", + commit = "600781dde9cca80734169b9e969d9054ccc57937", +) + go_repository( name = "com_github_google_go-cmp", importpath = "github.com/google/go-cmp", @@ -58,6 +64,12 @@ go_repository( commit = "b2d941ef6a780da2d9982c1fb28d77ad97f54fc7", ) +go_repository( + name = "com_github_syndtr_gocapability", + importpath = "github.com/syndtr/gocapability", + commit = "d98352740cb2c55f81556b63d4a1ec64c5a319c2", +) + go_repository( name = "com_github_vishvananda_netlink", importpath = "github.com/vishvananda/netlink", @@ -81,9 +93,3 @@ go_repository( importpath = "golang.org/x/sys", commit = "0dd5e194bbf5eb84a39666eb4c98a4d007e4203a", ) - -go_repository( - name = "com_github_syndtr_gocapability", - importpath = "github.com/syndtr/gocapability", - commit = "d98352740cb2c55f81556b63d4a1ec64c5a319c2", -) diff --git a/pkg/amutex/BUILD b/pkg/amutex/BUILD index 84e6b79a5..815ee3a69 100644 --- a/pkg/amutex/BUILD +++ b/pkg/amutex/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "amutex", srcs = ["amutex.go"], diff --git a/pkg/atomicbitops/BUILD b/pkg/atomicbitops/BUILD index a8dd17825..235188531 100644 --- a/pkg/atomicbitops/BUILD +++ b/pkg/atomicbitops/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "atomicbitops", srcs = [ diff --git a/pkg/binary/BUILD b/pkg/binary/BUILD index 586d05634..571151f72 100644 --- a/pkg/binary/BUILD +++ b/pkg/binary/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "binary", srcs = ["binary.go"], diff --git a/pkg/bits/BUILD b/pkg/bits/BUILD index 8c943b615..46794bdb8 100644 --- a/pkg/bits/BUILD +++ b/pkg/bits/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_library( diff --git a/pkg/compressio/BUILD b/pkg/compressio/BUILD index d70f982c1..72952d735 100644 --- a/pkg/compressio/BUILD +++ b/pkg/compressio/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "compressio", srcs = ["compressio.go"], diff --git a/pkg/control/client/BUILD b/pkg/control/client/BUILD index d58cd1b71..32853875d 100644 --- a/pkg/control/client/BUILD +++ b/pkg/control/client/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "client", srcs = [ diff --git a/pkg/control/server/BUILD b/pkg/control/server/BUILD index c3f74a532..ba2b1be9f 100644 --- a/pkg/control/server/BUILD +++ b/pkg/control/server/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "server", srcs = ["server.go"], diff --git a/pkg/dhcp/BUILD b/pkg/dhcp/BUILD index 711a72c99..c97dfc14b 100644 --- a/pkg/dhcp/BUILD +++ b/pkg/dhcp/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "dhcp", srcs = [ diff --git a/pkg/eventchannel/BUILD b/pkg/eventchannel/BUILD index 9d531ce12..18348ef54 100644 --- a/pkg/eventchannel/BUILD +++ b/pkg/eventchannel/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "eventchannel", srcs = [ diff --git a/pkg/fd/BUILD b/pkg/fd/BUILD index 435b6fa34..06cfd445e 100644 --- a/pkg/fd/BUILD +++ b/pkg/fd/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "fd", srcs = ["fd.go"], diff --git a/pkg/gate/BUILD b/pkg/gate/BUILD index 872eff531..9a87a3a31 100644 --- a/pkg/gate/BUILD +++ b/pkg/gate/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "gate", srcs = [ diff --git a/pkg/ilist/BUILD b/pkg/ilist/BUILD index 1bd71b800..a67aa2cff 100644 --- a/pkg/ilist/BUILD +++ b/pkg/ilist/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ilist", srcs = [ diff --git a/pkg/linewriter/BUILD b/pkg/linewriter/BUILD index 6c3795432..3f28ba867 100644 --- a/pkg/linewriter/BUILD +++ b/pkg/linewriter/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "linewriter", srcs = ["linewriter.go"], diff --git a/pkg/log/BUILD b/pkg/log/BUILD index fc9281079..bf85b4494 100644 --- a/pkg/log/BUILD +++ b/pkg/log/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "log", srcs = [ diff --git a/pkg/metric/BUILD b/pkg/metric/BUILD index c0cd40c7b..d96e5563b 100644 --- a/pkg/metric/BUILD +++ b/pkg/metric/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "metric", srcs = ["metric.go"], diff --git a/pkg/p9/BUILD b/pkg/p9/BUILD index 1cf5c6458..2c224e65b 100644 --- a/pkg/p9/BUILD +++ b/pkg/p9/BUILD @@ -15,6 +15,7 @@ go_library( "handlers.go", "messages.go", "p9.go", + "path_tree.go", "pool.go", "server.go", "transport.go", @@ -32,6 +33,7 @@ go_test( name = "p9_test", size = "small", srcs = [ + "buffer_test.go", "client_test.go", "messages_test.go", "p9_test.go", diff --git a/pkg/p9/buffer_test.go b/pkg/p9/buffer_test.go new file mode 100644 index 000000000..97eceefa7 --- /dev/null +++ b/pkg/p9/buffer_test.go @@ -0,0 +1,31 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package p9 + +import ( + "testing" +) + +func TestBufferOverrun(t *testing.T) { + buf := &buffer{ + // This header indicates that a large string should follow, but + // it is only two bytes. Reading a string should cause an + // overrun. + data: []byte{0x0, 0x16}, + } + if s := buf.ReadString(); s != "" { + t.Errorf("overrun read got %s, want empty", s) + } +} diff --git a/pkg/p9/client.go b/pkg/p9/client.go index 3ebfab82a..67887874a 100644 --- a/pkg/p9/client.go +++ b/pkg/p9/client.go @@ -116,6 +116,7 @@ func NewClient(socket *unet.Socket, messageSize uint32, version string) (*Client msize: largestFixedSize, } } + // Compute a payload size and round to 512 (normal block size) // if it's larger than a single block. payloadSize := messageSize - largestFixedSize @@ -299,3 +300,8 @@ func (c *Client) sendRecv(t message, r message) error { func (c *Client) Version() uint32 { return c.version } + +// Close closes the underlying socket. +func (c *Client) Close() error { + return c.socket.Close() +} diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go index 066639fda..992d1daf7 100644 --- a/pkg/p9/client_file.go +++ b/pkg/p9/client_file.go @@ -172,6 +172,9 @@ func (c *clientFile) SetAttr(valid SetAttrMask, attr SetAttr) error { } // Remove implements File.Remove. +// +// N.B. This method is no longer part of the file interface and should be +// considered deprecated. func (c *clientFile) Remove() error { // Avoid double close. if !atomic.CompareAndSwapUint32(&c.closed, 0, 1) { @@ -181,7 +184,6 @@ func (c *clientFile) Remove() error { // Send the remove message. if err := c.client.sendRecv(&Tremove{FID: c.fid}, &Rremove{}); err != nil { - log.Warningf("Tremove failed, losing FID %v: %v", c.fid, err) return err } diff --git a/pkg/p9/file.go b/pkg/p9/file.go index d2e89e373..55ceb52e1 100644 --- a/pkg/p9/file.go +++ b/pkg/p9/file.go @@ -31,35 +31,63 @@ type Attacher interface { // File is a set of operations corresponding to a single node. // -// Functions below MUST return syscall.Errno values. -// TODO: Enforce that with the type. +// Note that on the server side, the server logic places constraints on +// concurrent operations to make things easier. This may reduce the need for +// complex, error-prone locking and logic in the backend. These are documented +// for each method. // -// These must be implemented in all circumstances. +// There are three different types of guarantees provided: +// +// none: There is no concurrency guarantee. The method may be invoked +// concurrently with any other method on any other file. +// +// read: The method is guaranteed to be exclusive of any write or global +// operation that is mutating the state of the directory tree starting at this +// node. For example, this means creating new files, symlinks, directories or +// renaming a directory entry (or renaming in to this target), but the method +// may be called concurrently with other read methods. +// +// write: The method is guaranteed to be exclusive of any read, write or global +// operation that is mutating the state of the directory tree starting at this +// node, as described in read above. There may however, be other write +// operations executing concurrently on other components in the directory tree. +// +// global: The method is guaranteed to be exclusive of any read, write or +// global operation. type File interface { // Walk walks to the path components given in names. // // Walk returns QIDs in the same order that the names were passed in. // // An empty list of arguments should return a copy of the current file. + // + // On the server, Walk has a read concurrency guarantee. Walk(names []string) ([]QID, File, error) + // WalkGetAttr walks to the next file and returns its maximal set of + // attributes. + // + // Server-side p9.Files may return syscall.ENOSYS to indicate that Walk + // and GetAttr should be used separately to satisfy this request. + // + // On the server, WalkGetAttr has a read concurrency guarantee. + WalkGetAttr([]string) ([]QID, File, AttrMask, Attr, error) + // StatFS returns information about the file system associated with // this file. + // + // On the server, StatFS has no concurrency guarantee. StatFS() (FSStat, error) // GetAttr returns attributes of this node. + // + // On the server, GetAttr has a read concurrency guarantee. GetAttr(req AttrMask) (QID, AttrMask, Attr, error) // SetAttr sets attributes on this node. - SetAttr(valid SetAttrMask, attr SetAttr) error - - // Remove removes the file. // - // This is deprecated in favor of UnlinkAt below. - Remove() error - - // Rename renames the file. - Rename(directory File, name string) error + // On the server, SetAttr has a write concurrency guarantee. + SetAttr(valid SetAttrMask, attr SetAttr) error // Close is called when all references are dropped on the server side, // and Close should be called by the client to drop all references. @@ -67,65 +95,93 @@ type File interface { // For server-side implementations of Close, the error is ignored. // // Close must be called even when Open has not been called. + // + // On the server, Close has no concurrency guarantee. Close() error - // Open is called prior to using read/write. + // Open must be called prior to using Read, Write or Readdir. Once Open + // is called, some operations, such as Walk, will no longer work. // - // The *fd.FD may be nil. If an *fd.FD is provided, ownership now - // belongs to the caller and the FD must be non-blocking. + // On the client, Open should be called only once. The fd return is + // optional, and may be nil. // - // If Open returns a non-nil *fd.FD, it should do so for all possible - // OpenFlags. If Open returns a nil *fd.FD, it should similarly return - // a nil *fd.FD for all possible OpenFlags. + // On the server, Open has a read concurrency guarantee. If an *fd.FD + // is provided, ownership now belongs to the caller. Open is guaranteed + // to be called only once. // - // This can be assumed to be one-shot only. + // N.B. The server must resolve any lazy paths when open is called. + // After this point, read and write may be called on files with no + // deletion check, so resolving in the data path is not viable. Open(mode OpenFlags) (*fd.FD, QID, uint32, error) - // Read reads from this file. + // Read reads from this file. Open must be called first. // // This may return io.EOF in addition to syscall.Errno values. // - // Preconditions: Open has been called and returned success. + // On the server, ReadAt has a read concurrency guarantee. See Open for + // additional requirements regarding lazy path resolution. ReadAt(p []byte, offset uint64) (int, error) - // Write writes to this file. + // Write writes to this file. Open must be called first. // // This may return io.EOF in addition to syscall.Errno values. // - // Preconditions: Open has been called and returned success. + // On the server, WriteAt has a read concurrency guarantee. See Open + // for additional requirements regarding lazy path resolution. WriteAt(p []byte, offset uint64) (int, error) - // FSync syncs this node. + // FSync syncs this node. Open must be called first. // - // Preconditions: Open has been called and returned success. + // On the server, FSync has a read concurrency guarantee. FSync() error // Create creates a new regular file and opens it according to the - // flags given. + // flags given. This file is already Open. + // + // N.B. On the client, the returned file is a reference to the current + // file, which now represents the created file. This is not the case on + // the server. These semantics are very subtle and can easily lead to + // bugs, but are a consequence of the 9P create operation. // // See p9.File.Open for a description of *fd.FD. + // + // On the server, Create has a write concurrency guarantee. Create(name string, flags OpenFlags, permissions FileMode, uid UID, gid GID) (*fd.FD, File, QID, uint32, error) // Mkdir creates a subdirectory. + // + // On the server, Mkdir has a write concurrency guarantee. Mkdir(name string, permissions FileMode, uid UID, gid GID) (QID, error) // Symlink makes a new symbolic link. - Symlink(oldname string, newname string, uid UID, gid GID) (QID, error) + // + // On the server, Symlink has a write concurrency guarantee. + Symlink(oldName string, newName string, uid UID, gid GID) (QID, error) // Link makes a new hard link. - Link(target File, newname string) error + // + // On the server, Link has a write concurrency guarantee. + Link(target File, newName string) error // Mknod makes a new device node. + // + // On the server, Mknod has a write concurrency guarantee. Mknod(name string, permissions FileMode, major uint32, minor uint32, uid UID, gid GID) (QID, error) + // Rename renames the file. + // + // Rename will never be called on the server, and RenameAt will always + // be used instead. + Rename(newDir File, newName string) error + // RenameAt renames a given file to a new name in a potentially new // directory. // - // oldname must be a name relative to this file, which must be a - // directory. newname is a name relative to newdir. + // oldName must be a name relative to this file, which must be a + // directory. newName is a name relative to newDir. // - // This is deprecated in favor of Rename. - RenameAt(oldname string, newdir File, newname string) error + // On the server, RenameAt has a global concurrency guarantee. + RenameAt(oldName string, newDir File, newName string) error // UnlinkAt the given named file. // @@ -133,16 +189,20 @@ type File interface { // // Flags are implementation-specific (e.g. O_DIRECTORY), but are // generally Linux unlinkat(2) flags. + // + // On the server, UnlinkAt has a write concurrency guarantee. UnlinkAt(name string, flags uint32) error // Readdir reads directory entries. // // This may return io.EOF in addition to syscall.Errno values. // - // Preconditions: Open has been called and returned success. + // On the server, Readdir has a read concurrency guarantee. Readdir(offset uint64, count uint32) ([]Dirent, error) // Readlink reads the link target. + // + // On the server, Readlink has a read concurrency guarantee. Readlink() (string, error) // Flush is called prior to Close. @@ -150,16 +210,11 @@ type File interface { // Whereas Close drops all references to the file, Flush cleans up the // file state. Behavior is implementation-specific. // - // Flush is not related to flush(9p). Flush is an extension to 9P2000.L, + // Flush is not related to flush(9p). Flush is an extension to 9P2000.L, // see version.go. - Flush() error - - // WalkGetAttr walks to the next file and returns its maximal set of - // attributes. // - // Server-side p9.Files may return syscall.ENOSYS to indicate that Walk - // and GetAttr should be used separately to satisfy this request. - WalkGetAttr([]string) ([]QID, File, AttrMask, Attr, error) + // On the server, Flush has a read concurrency guarantee. + Flush() error // Connect establishes a new host-socket backed connection with a // socket. A File does not need to be opened before it can be connected @@ -170,8 +225,22 @@ type File interface { // // The returned FD must be non-blocking. // - // flags indicates the requested type of socket. + // Flags indicates the requested type of socket. + // + // On the server, Connect has a read concurrency guarantee. Connect(flags ConnectFlags) (*fd.FD, error) + + // Renamed is called when this node is renamed. + // + // This may not fail. The file will hold a reference to its parent + // within the p9 package, and is therefore safe to use for the lifetime + // of this File (until Close is called). + // + // This method should not be called by clients, who should use the + // relevant Rename methods. (Although the method will be a no-op.) + // + // On the server, Renamed has a global concurrency guarantee. + Renamed(newDir File, newName string) } // DefaultWalkGetAttr implements File.WalkGetAttr to return ENOSYS for server-side Files. diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go index 959dff31d..0d7a6138f 100644 --- a/pkg/p9/handlers.go +++ b/pkg/p9/handlers.go @@ -15,6 +15,7 @@ package p9 import ( + "fmt" "io" "os" "path" @@ -22,22 +23,43 @@ import ( "sync/atomic" "syscall" + "gvisor.googlesource.com/gvisor/pkg/fd" "gvisor.googlesource.com/gvisor/pkg/log" ) -// newErr returns a new error message from an error. -func newErr(err error) *Rlerror { +const maximumNameLength = 255 + +// ExtractErrno extracts a syscall.Errno from a error, best effort. +func ExtractErrno(err error) syscall.Errno { + switch err { + case os.ErrNotExist: + return syscall.ENOENT + case os.ErrExist: + return syscall.EEXIST + case os.ErrPermission: + return syscall.EACCES + case os.ErrInvalid: + return syscall.EINVAL + } + + // Attempt to unwrap. switch e := err.(type) { case syscall.Errno: - return &Rlerror{Error: uint32(e)} + return e case *os.PathError: - return newErr(e.Err) + return ExtractErrno(e.Err) case *os.SyscallError: - return newErr(e.Err) - default: - log.Warningf("unknown error: %v", err) - return &Rlerror{Error: uint32(syscall.EIO)} + return ExtractErrno(e.Err) } + + // Default case. + log.Warningf("unknown error: %v", err) + return syscall.EIO +} + +// newErr returns a new error message from an error. +func newErr(err error) *Rlerror { + return &Rlerror{Error: uint32(ExtractErrno(err))} } // handler is implemented for server-handled messages. @@ -85,13 +107,15 @@ func (t *Tflush) handle(cs *connState) message { return &Rflush{} } -// isSafeName returns true iff the name does not contain directory characters. -// -// We permit walks only on safe names and store the sequence of paths used for -// any given walk in each FID. (This is immutable.) We use this to mark -// relevant FIDs as moved when a successful rename occurs. -func isSafeName(name string) bool { - return name != "" && !strings.Contains(name, "/") && name != "." && name != ".." +// checkSafeName validates the name and returns nil or returns an error. +func checkSafeName(name string) error { + if name == "" || strings.Contains(name, "/") || name == "." || name == ".." { + return syscall.EINVAL + } + if len(name) > maximumNameLength { + return syscall.ENAMETOOLONG + } + return nil } // handle implements handler.handle. @@ -110,22 +134,54 @@ func (t *Tremove) handle(cs *connState) message { } defer ref.DecRef() + // Frustratingly, because we can't be guaranteed that a rename is not + // occurring simultaneously with this removal, we need to acquire the + // global rename lock for this kind of remove operation to ensure that + // ref.parent does not change out from underneath us. + // + // This is why Tremove is a bad idea, and clients should generally use + // Tunlinkat. All p9 clients will use Tunlinkat. + err := ref.safelyGlobal(func() error { + // Is this a root? Can't remove that. + if ref.isRoot() { + return syscall.EINVAL + } + + // N.B. this remove operation is permitted, even if the file is open. + // See also rename below for reasoning. + + // Is this file already deleted? + if ref.isDeleted() { + return syscall.EINVAL + } + + // Retrieve the file's proper name. + name := ref.parent.pathNode.nameFor(ref) + + // Attempt the removal. + if err := ref.parent.file.UnlinkAt(name, 0); err != nil { + return err + } + + // Mark all relevant fids as deleted. We don't need to lock any + // individual nodes because we already hold the global lock. + ref.parent.markChildDeleted(name) + return nil + }) + // "The remove request asks the file server both to remove the file // represented by fid and to clunk the fid, even if the remove fails." // // "It is correct to consider remove to be a clunk with the side effect // of removing the file if permissions allow." // https://swtch.com/plan9port/man/man9/remove.html - err := ref.file.Remove() - - // Clunk the FID regardless of Remove error. if !cs.DeleteFID(t.FID) { return newErr(syscall.EBADF) } - if err != nil { return newErr(err) } + return &Rremove{} } @@ -168,9 +224,12 @@ func (t *Tattach) handle(cs *connState) message { // Build a transient reference. root := &fidRef{ + server: cs.server, + parent: nil, file: sf, refs: 1, - walkable: attr.Mode.IsDir(), + mode: attr.Mode.FileType(), + pathNode: &cs.server.pathTree, } defer root.DecRef() @@ -183,20 +242,24 @@ func (t *Tattach) handle(cs *connState) message { // We want the same traversal checks to apply on attach, so always // attach at the root and use the regular walk paths. names := strings.Split(t.Auth.AttachName, "/") - _, target, _, attr, err := doWalk(cs, root, names) + _, newRef, _, attr, err := doWalk(cs, root, names) if err != nil { return newErr(err) } + defer newRef.DecRef() // Insert the FID. - cs.InsertFID(t.FID, &fidRef{ - file: target, - walkable: attr.Mode.IsDir(), - }) - + cs.InsertFID(t.FID, newRef) return &Rattach{} } +// CanOpen returns whether this file open can be opened, read and written to. +// +// This includes everything except symlinks and sockets. +func CanOpen(mode FileMode) bool { + return mode.IsRegular() || mode.IsDir() || mode.IsNamedPipe() || mode.IsBlockDevice() || mode.IsCharacterDevice() +} + // handle implements handler.handle. func (t *Tlopen) handle(cs *connState) message { // Lookup the FID. @@ -210,13 +273,35 @@ func (t *Tlopen) handle(cs *connState) message { defer ref.openedMu.Unlock() // Has it been opened already? - if ref.opened { + if ref.opened || !CanOpen(ref.mode) { return newErr(syscall.EINVAL) } - // Do the open. - osFile, qid, ioUnit, err := ref.file.Open(t.Flags) - if err != nil { + // Are flags valid? + if t.Flags&^OpenFlagsModeMask != 0 { + return newErr(syscall.EINVAL) + } + + // Is this an attempt to open a directory as writable? Don't accept. + if ref.mode.IsDir() && t.Flags != ReadOnly { + return newErr(syscall.EINVAL) + } + + var ( + qid QID + ioUnit uint32 + osFile *fd.FD + ) + if err := ref.safelyRead(func() (err error) { + // Has it been deleted already? + if ref.isDeleted() { + return syscall.EINVAL + } + + // Do the open. + osFile, qid, ioUnit, err = ref.file.Open(t.Flags) + return err + }); err != nil { return newErr(err) } @@ -229,8 +314,8 @@ func (t *Tlopen) handle(cs *connState) message { func (t *Tlcreate) do(cs *connState, uid UID) (*Rlcreate, error) { // Don't allow complex names. - if !isSafeName(t.Name) { - return nil, syscall.EINVAL + if err := checkSafeName(t.Name); err != nil { + return nil, err } // Lookup the FID. @@ -240,20 +325,48 @@ func (t *Tlcreate) do(cs *connState, uid UID) (*Rlcreate, error) { } defer ref.DecRef() - // Do the create. - osFile, nsf, qid, ioUnit, err := ref.file.Create(t.Name, t.OpenFlags, t.Permissions, uid, t.GID) - if err != nil { + var ( + osFile *fd.FD + nsf File + qid QID + ioUnit uint32 + newRef *fidRef + ) + if err := ref.safelyWrite(func() (err error) { + // Don't allow creation from non-directories or deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the create. + osFile, nsf, qid, ioUnit, err = ref.file.Create(t.Name, t.OpenFlags, t.Permissions, uid, t.GID) + if err != nil { + return err + } + + newRef = &fidRef{ + server: cs.server, + parent: ref, + file: nsf, + opened: true, + openFlags: t.OpenFlags, + mode: ModeRegular, + pathNode: ref.pathNode.pathNodeFor(t.Name), + } + ref.pathNode.addChild(newRef, t.Name) + ref.IncRef() // Acquire parent reference. + return nil + }); err != nil { return nil, err } // Replace the FID reference. - // - // The new file will be opened already. - cs.InsertFID(t.FID, &fidRef{ - file: nsf, - opened: true, - openFlags: t.OpenFlags, - }) + cs.InsertFID(t.FID, newRef) return &Rlcreate{Rlopen: Rlopen{QID: qid, IoUnit: ioUnit, File: osFile}}, nil } @@ -278,8 +391,8 @@ func (t *Tsymlink) handle(cs *connState) message { func (t *Tsymlink) do(cs *connState, uid UID) (*Rsymlink, error) { // Don't allow complex names. - if !isSafeName(t.Name) { - return nil, syscall.EINVAL + if err := checkSafeName(t.Name); err != nil { + return nil, err } // Lookup the FID. @@ -289,9 +402,22 @@ func (t *Tsymlink) do(cs *connState, uid UID) (*Rsymlink, error) { } defer ref.DecRef() - // Do the symlink. - qid, err := ref.file.Symlink(t.Target, t.Name, uid, t.GID) - if err != nil { + var qid QID + if err := ref.safelyWrite(func() (err error) { + // Don't allow symlinks from non-directories or deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the symlink. + qid, err = ref.file.Symlink(t.Target, t.Name, uid, t.GID) + return err + }); err != nil { return nil, err } @@ -301,8 +427,8 @@ func (t *Tsymlink) do(cs *connState, uid UID) (*Rsymlink, error) { // handle implements handler.handle. func (t *Tlink) handle(cs *connState) message { // Don't allow complex names. - if !isSafeName(t.Name) { - return newErr(syscall.EINVAL) + if err := checkSafeName(t.Name); err != nil { + return newErr(err) } // Lookup the FID. @@ -319,8 +445,20 @@ func (t *Tlink) handle(cs *connState) message { } defer refTarget.DecRef() - // Do the link. - if err := ref.file.Link(refTarget.file, t.Name); err != nil { + if err := ref.safelyWrite(func() (err error) { + // Don't allow create links from non-directories or deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the link. + return ref.file.Link(refTarget.file, t.Name) + }); err != nil { return newErr(err) } @@ -330,8 +468,11 @@ func (t *Tlink) handle(cs *connState) message { // handle implements handler.handle. func (t *Trenameat) handle(cs *connState) message { // Don't allow complex names. - if !isSafeName(t.OldName) || !isSafeName(t.NewName) { - return newErr(syscall.EINVAL) + if err := checkSafeName(t.OldName); err != nil { + return newErr(err) + } + if err := checkSafeName(t.NewName); err != nil { + return newErr(err) } // Lookup the FID. @@ -348,8 +489,32 @@ func (t *Trenameat) handle(cs *connState) message { } defer refTarget.DecRef() - // Do the rename. - if err := ref.file.RenameAt(t.OldName, refTarget.file, t.NewName); err != nil { + // Perform the rename holding the global lock. + if err := ref.safelyGlobal(func() (err error) { + // Don't allow renaming across deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() || refTarget.isDeleted() || !refTarget.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Is this the same file? If yes, short-circuit and return success. + if ref.pathNode == refTarget.pathNode && t.OldName == t.NewName { + return nil + } + + // Attempt the actual rename. + if err := ref.file.RenameAt(t.OldName, refTarget.file, t.NewName); err != nil { + return err + } + + // Update the path tree. + ref.renameChildTo(t.OldName, refTarget, t.NewName) + return nil + }); err != nil { return newErr(err) } @@ -359,8 +524,8 @@ func (t *Trenameat) handle(cs *connState) message { // handle implements handler.handle. func (t *Tunlinkat) handle(cs *connState) message { // Don't allow complex names. - if !isSafeName(t.Name) { - return newErr(syscall.EINVAL) + if err := checkSafeName(t.Name); err != nil { + return newErr(err) } // Lookup the FID. @@ -370,8 +535,40 @@ func (t *Tunlinkat) handle(cs *connState) message { } defer ref.DecRef() - // Do the unlink. - if err := ref.file.UnlinkAt(t.Name, t.Flags); err != nil { + if err := ref.safelyWrite(func() (err error) { + // Don't allow deletion from non-directories or deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Before we do the unlink itself, we need to ensure that there + // are no operations in flight on associated path node. The + // child's path node lock must be held to ensure that the + // unlink at marking the child deleted below is atomic with + // respect to any other read or write operations. + // + // This is one case where we have a lock ordering issue, but + // since we always acquire deeper in the hierarchy, we know + // that we are free of lock cycles. + childPathNode := ref.pathNode.pathNodeFor(t.Name) + childPathNode.mu.Lock() + defer childPathNode.mu.Unlock() + + // Do the unlink. + err = ref.file.UnlinkAt(t.Name, t.Flags) + if err != nil { + return err + } + + // Mark the path as deleted. + ref.markChildDeleted(t.Name) + return nil + }); err != nil { return newErr(err) } @@ -381,8 +578,8 @@ func (t *Tunlinkat) handle(cs *connState) message { // handle implements handler.handle. func (t *Trename) handle(cs *connState) message { // Don't allow complex names. - if !isSafeName(t.Name) { - return newErr(syscall.EINVAL) + if err := checkSafeName(t.Name); err != nil { + return newErr(err) } // Lookup the FID. @@ -399,8 +596,43 @@ func (t *Trename) handle(cs *connState) message { } defer refTarget.DecRef() - // Call the rename method. - if err := ref.file.Rename(refTarget.file, t.Name); err != nil { + if err := ref.safelyGlobal(func() (err error) { + // Don't allow a root rename. + if ref.isRoot() { + return syscall.EINVAL + } + + // Don't allow renaming deleting entries, or target non-directories. + if ref.isDeleted() || refTarget.isDeleted() || !refTarget.mode.IsDir() { + return syscall.EINVAL + } + + // If the parent is deleted, but we not, something is seriously wrong. + // It's fail to die at this point with an assertion failure. + if ref.parent.isDeleted() { + panic(fmt.Sprintf("parent %+v deleted, child %+v is not", ref.parent, ref)) + } + + // N.B. The rename operation is allowed to proceed on open files. It + // does impact the state of its parent, but this is merely a sanity + // check in any case, and the operation is safe. There may be other + // files corresponding to the same path that are renamed anyways. + + // Check for the exact same file and short-circuit. + oldName := ref.parent.pathNode.nameFor(ref) + if ref.parent.pathNode == refTarget.pathNode && oldName == t.Name { + return nil + } + + // Call the rename method on the parent. + if err := ref.parent.file.RenameAt(oldName, refTarget.file, t.Name); err != nil { + return err + } + + // Update the path tree. + ref.parent.renameChildTo(oldName, refTarget, t.Name) + return nil + }); err != nil { return newErr(err) } @@ -416,9 +648,19 @@ func (t *Treadlink) handle(cs *connState) message { } defer ref.DecRef() - // Do the read. - target, err := ref.file.Readlink() - if err != nil { + var target string + if err := ref.safelyRead(func() (err error) { + // Don't allow readlink on deleted files. There is no need to + // check if this file is opened because symlinks cannot be + // opened. + if ref.isDeleted() || !ref.mode.IsSymlink() { + return syscall.EINVAL + } + + // Do the read. + target, err = ref.file.Readlink() + return err + }); err != nil { return newErr(err) } @@ -434,26 +676,30 @@ func (t *Tread) handle(cs *connState) message { } defer ref.DecRef() - // Has it been opened already? - openFlags, opened := ref.OpenFlags() - if !opened { - return newErr(syscall.EINVAL) - } - - // Can it be read? Check permissions. - if openFlags&OpenFlagsModeMask == WriteOnly { - return newErr(syscall.EPERM) - } - // Constrain the size of the read buffer. if int(t.Count) > int(maximumLength) { return newErr(syscall.ENOBUFS) } - // Do the read. - data := make([]byte, t.Count) - n, err := ref.file.ReadAt(data, t.Offset) - if err != nil && err != io.EOF { + var ( + data = make([]byte, t.Count) + n int + ) + if err := ref.safelyRead(func() (err error) { + // Has it been opened already? + openFlags, opened := ref.OpenFlags() + if !opened { + return syscall.EINVAL + } + + // Can it be read? Check permissions. + if openFlags&OpenFlagsModeMask == WriteOnly { + return syscall.EPERM + } + + n, err = ref.file.ReadAt(data, t.Offset) + return err + }); err != nil && err != io.EOF { return newErr(err) } @@ -469,20 +715,22 @@ func (t *Twrite) handle(cs *connState) message { } defer ref.DecRef() - // Has it been opened already? - openFlags, opened := ref.OpenFlags() - if !opened { - return newErr(syscall.EINVAL) - } + var n int + if err := ref.safelyRead(func() (err error) { + // Has it been opened already? + openFlags, opened := ref.OpenFlags() + if !opened { + return syscall.EINVAL + } - // Can it be write? Check permissions. - if openFlags&OpenFlagsModeMask == ReadOnly { - return newErr(syscall.EPERM) - } + // Can it be write? Check permissions. + if openFlags&OpenFlagsModeMask == ReadOnly { + return syscall.EPERM + } - // Do the write. - n, err := ref.file.WriteAt(t.Data, t.Offset) - if err != nil { + n, err = ref.file.WriteAt(t.Data, t.Offset) + return err + }); err != nil { return newErr(err) } @@ -500,8 +748,8 @@ func (t *Tmknod) handle(cs *connState) message { func (t *Tmknod) do(cs *connState, uid UID) (*Rmknod, error) { // Don't allow complex names. - if !isSafeName(t.Name) { - return nil, syscall.EINVAL + if err := checkSafeName(t.Name); err != nil { + return nil, err } // Lookup the FID. @@ -511,9 +759,22 @@ func (t *Tmknod) do(cs *connState, uid UID) (*Rmknod, error) { } defer ref.DecRef() - // Do the mknod. - qid, err := ref.file.Mknod(t.Name, t.Permissions, t.Major, t.Minor, uid, t.GID) - if err != nil { + var qid QID + if err := ref.safelyWrite(func() (err error) { + // Don't allow mknod on deleted files. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the mknod. + qid, err = ref.file.Mknod(t.Name, t.Permissions, t.Major, t.Minor, uid, t.GID) + return err + }); err != nil { return nil, err } @@ -531,8 +792,8 @@ func (t *Tmkdir) handle(cs *connState) message { func (t *Tmkdir) do(cs *connState, uid UID) (*Rmkdir, error) { // Don't allow complex names. - if !isSafeName(t.Name) { - return nil, syscall.EINVAL + if err := checkSafeName(t.Name); err != nil { + return nil, err } // Lookup the FID. @@ -542,9 +803,22 @@ func (t *Tmkdir) do(cs *connState, uid UID) (*Rmkdir, error) { } defer ref.DecRef() - // Do the mkdir. - qid, err := ref.file.Mkdir(t.Name, t.Permissions, uid, t.GID) - if err != nil { + var qid QID + if err := ref.safelyWrite(func() (err error) { + // Don't allow mkdir on deleted files. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Not allowed on open directories. + if _, opened := ref.OpenFlags(); opened { + return syscall.EINVAL + } + + // Do the mkdir. + qid, err = ref.file.Mkdir(t.Name, t.Permissions, uid, t.GID) + return err + }); err != nil { return nil, err } @@ -560,9 +834,20 @@ func (t *Tgetattr) handle(cs *connState) message { } defer ref.DecRef() - // Get attributes. - qid, valid, attr, err := ref.file.GetAttr(t.AttrMask) - if err != nil { + // We allow getattr on deleted files. Depending on the backing + // implementation, it's possible that races exist that might allow + // fetching attributes of other files. But we need to generally allow + // refreshing attributes and this is a minor leak, if at all. + + var ( + qid QID + valid AttrMask + attr Attr + ) + if err := ref.safelyRead(func() (err error) { + qid, valid, attr, err = ref.file.GetAttr(t.AttrMask) + return err + }); err != nil { return newErr(err) } @@ -578,8 +863,18 @@ func (t *Tsetattr) handle(cs *connState) message { } defer ref.DecRef() - // Set attributes. - if err := ref.file.SetAttr(t.Valid, t.SetAttr); err != nil { + if err := ref.safelyWrite(func() error { + // We don't allow setattr on files that have been deleted. + // This might be technically incorrect, as it's possible that + // there were multiple links and you can still change the + // corresponding inode information. + if ref.isDeleted() { + return syscall.EINVAL + } + + // Set the attributes. + return ref.file.SetAttr(t.Valid, t.SetAttr) + }); err != nil { return newErr(err) } @@ -621,14 +916,25 @@ func (t *Treaddir) handle(cs *connState) message { } defer ref.DecRef() - // Has it been opened already? - if _, opened := ref.OpenFlags(); !opened { - return newErr(syscall.EINVAL) - } + var entries []Dirent + if err := ref.safelyRead(func() (err error) { + // Don't allow reading deleted directories. + if ref.isDeleted() || !ref.mode.IsDir() { + return syscall.EINVAL + } + + // Has it been opened already? + if _, opened := ref.OpenFlags(); !opened { + return syscall.EINVAL + } - // Read the entries. - entries, err := ref.file.Readdir(t.Offset, t.Count) - if err != nil && err != io.EOF { + // Read the entries. + entries, err = ref.file.Readdir(t.Offset, t.Count) + if err != nil && err != io.EOF { + return err + } + return nil + }); err != nil { return newErr(err) } @@ -644,13 +950,15 @@ func (t *Tfsync) handle(cs *connState) message { } defer ref.DecRef() - // Has it been opened already? - if _, opened := ref.OpenFlags(); !opened { - return newErr(syscall.EINVAL) - } + if err := ref.safelyRead(func() (err error) { + // Has it been opened already? + if _, opened := ref.OpenFlags(); !opened { + return syscall.EINVAL + } - err := ref.file.FSync() - if err != nil { + // Perform the sync. + return ref.file.FSync() + }); err != nil { return newErr(err) } @@ -671,6 +979,11 @@ func (t *Tstatfs) handle(cs *connState) message { return newErr(err) } + // Constrain the name length. + if st.NameLength > maximumNameLength { + st.NameLength = maximumNameLength + } + return &Rstatfs{st} } @@ -682,7 +995,7 @@ func (t *Tflushf) handle(cs *connState) message { } defer ref.DecRef() - if err := ref.file.Flush(); err != nil { + if err := ref.safelyRead(ref.file.Flush); err != nil { return newErr(err) } @@ -726,12 +1039,14 @@ func walkOne(qids []QID, from File, names []string) ([]QID, File, AttrMask, Attr // doWalk walks from a given fidRef. // -// This enforces that all intermediate nodes are walkable (directories). -func doWalk(cs *connState, ref *fidRef, names []string) (qids []QID, sf File, valid AttrMask, attr Attr, err error) { +// This enforces that all intermediate nodes are walkable (directories). The +// fidRef returned (newRef) has a reference associated with it that is now +// owned by the caller and must be handled appropriately. +func doWalk(cs *connState, ref *fidRef, names []string) (qids []QID, newRef *fidRef, valid AttrMask, attr Attr, err error) { // Check the names. for _, name := range names { - if !isSafeName(name) { - err = syscall.EINVAL + err = checkSafeName(name) + if err != nil { return } } @@ -745,44 +1060,88 @@ func doWalk(cs *connState, ref *fidRef, names []string) (qids []QID, sf File, va // Is this an empty list? Handle specially. We don't actually need to // validate anything since this is always permitted. if len(names) == 0 { - return walkOne(nil, ref.file, nil) - } - - // Is it walkable? - if !ref.walkable { - err = syscall.EINVAL - return + var sf File // Temporary. + if err := ref.maybeParent().safelyRead(func() (err error) { + // Clone the single element. + qids, sf, valid, attr, err = walkOne(nil, ref.file, nil) + if err != nil { + return err + } + + newRef = &fidRef{ + server: cs.server, + parent: ref.parent, + file: sf, + mode: ref.mode, + pathNode: ref.pathNode, + + // For the clone case, the cloned fid must + // preserve the deleted property of the + // original FID. + deleted: ref.deleted, + } + if !ref.isRoot() { + if !newRef.isDeleted() { + // Add only if a non-root node; the same node. + ref.parent.pathNode.addChild(newRef, ref.parent.pathNode.nameFor(ref)) + } + ref.parent.IncRef() // Acquire parent reference. + } + // doWalk returns a reference. + newRef.IncRef() + return nil + }); err != nil { + return nil, nil, AttrMask{}, Attr{}, err + } + return qids, newRef, valid, attr, nil } - from := ref.file // Start at the passed ref. - // Do the walk, one element at a time. + walkRef := ref + walkRef.IncRef() for i := 0; i < len(names); i++ { - qids, sf, valid, attr, err = walkOne(qids, from, names[i:i+1]) - - // Close the intermediate file. Note that we don't close the - // first file because in that case we are walking from the - // existing reference. - if i > 0 { - from.Close() - } - from = sf // Use the new file. - - // Was there an error walking? - if err != nil { - return nil, nil, AttrMask{}, Attr{}, err - } - // We won't allow beyond past symlinks; stop here if this isn't // a proper directory and we have additional paths to walk. - if !valid.Mode || (!attr.Mode.IsDir() && i < len(names)-1) { - from.Close() // Not using the file object. + if !walkRef.mode.IsDir() { + walkRef.DecRef() // Drop walk reference; no lock required. return nil, nil, AttrMask{}, Attr{}, syscall.EINVAL } + + var sf File // Temporary. + if err := walkRef.safelyRead(func() (err error) { + qids, sf, valid, attr, err = walkOne(qids, walkRef.file, names[i:i+1]) + if err != nil { + return err + } + + // Note that we don't need to acquire a lock on any of + // these individual instances. That's because they are + // not actually addressable via a FID. They are + // anonymous. They exist in the tree for tracking + // purposes. + newRef := &fidRef{ + server: cs.server, + parent: walkRef, + file: sf, + mode: attr.Mode.FileType(), + pathNode: walkRef.pathNode.pathNodeFor(names[i]), + } + walkRef.pathNode.addChild(newRef, names[i]) + // We allow our walk reference to become the new parent + // reference here and so we don't IncRef. Instead, just + // set walkRef to the newRef above and acquire a new + // walk reference. + walkRef = newRef + walkRef.IncRef() + return nil + }); err != nil { + walkRef.DecRef() // Drop the old walkRef. + return nil, nil, AttrMask{}, Attr{}, err + } } // Success. - return qids, sf, valid, attr, nil + return qids, walkRef, valid, attr, nil } // handle implements handler.handle. @@ -795,17 +1154,14 @@ func (t *Twalk) handle(cs *connState) message { defer ref.DecRef() // Do the walk. - qids, sf, _, attr, err := doWalk(cs, ref, t.Names) + qids, newRef, _, _, err := doWalk(cs, ref, t.Names) if err != nil { return newErr(err) } + defer newRef.DecRef() // Install the new FID. - cs.InsertFID(t.NewFID, &fidRef{ - file: sf, - walkable: attr.Mode.IsDir(), - }) - + cs.InsertFID(t.NewFID, newRef) return &Rwalk{QIDs: qids} } @@ -819,17 +1175,14 @@ func (t *Twalkgetattr) handle(cs *connState) message { defer ref.DecRef() // Do the walk. - qids, sf, valid, attr, err := doWalk(cs, ref, t.Names) + qids, newRef, valid, attr, err := doWalk(cs, ref, t.Names) if err != nil { return newErr(err) } + defer newRef.DecRef() // Install the new FID. - cs.InsertFID(t.NewFID, &fidRef{ - file: sf, - walkable: attr.Mode.IsDir(), - }) - + cs.InsertFID(t.NewFID, newRef) return &Rwalkgetattr{QIDs: qids, Valid: valid, Attr: attr} } @@ -878,9 +1231,17 @@ func (t *Tlconnect) handle(cs *connState) message { } defer ref.DecRef() - // Do the connect. - osFile, err := ref.file.Connect(t.Flags) - if err != nil { + var osFile *fd.FD + if err := ref.safelyRead(func() (err error) { + // Don't allow connecting to deleted files. + if ref.isDeleted() || !ref.mode.IsSocket() { + return syscall.EINVAL + } + + // Do the connect. + osFile, err = ref.file.Connect(t.Flags) + return err + }); err != nil { return newErr(err) } diff --git a/pkg/p9/local_server/BUILD b/pkg/p9/local_server/BUILD index 8229e6308..b17ebb79d 100644 --- a/pkg/p9/local_server/BUILD +++ b/pkg/p9/local_server/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "local_server", srcs = ["local_server.go"], diff --git a/pkg/p9/local_server/local_server.go b/pkg/p9/local_server/local_server.go index 1e6aaa762..69b90c6cd 100644 --- a/pkg/p9/local_server/local_server.go +++ b/pkg/p9/local_server/local_server.go @@ -318,6 +318,11 @@ func (l *local) Connect(p9.ConnectFlags) (*fd.FD, error) { return nil, syscall.ECONNREFUSED } +// Renamed implements p9.File.Renamed. +func (l *local) Renamed(parent p9.File, newName string) { + l.path = path.Join(parent.(*local).path, newName) +} + func main() { log.SetLevel(log.Debug) diff --git a/pkg/p9/messages_test.go b/pkg/p9/messages_test.go index dfb41bb76..c0d65d82c 100644 --- a/pkg/p9/messages_test.go +++ b/pkg/p9/messages_test.go @@ -15,6 +15,7 @@ package p9 import ( + "fmt" "reflect" "testing" ) @@ -186,6 +187,13 @@ func TestEncodeDecode(t *testing.T) { &Rxattrwalk{ Size: 1, }, + &Txattrcreate{ + FID: 1, + Name: "a", + AttrSize: 2, + Flags: 3, + }, + &Rxattrcreate{}, &Treaddir{ Directory: 1, Offset: 2, @@ -389,3 +397,32 @@ func TestEncodeDecode(t *testing.T) { } } } + +func TestMessageStrings(t *testing.T) { + for typ, fn := range messageRegistry { + name := fmt.Sprintf("%+v", typ) + t.Run(name, func(t *testing.T) { + defer func() { // Ensure no panic. + if r := recover(); r != nil { + t.Errorf("printing %s failed: %v", name, r) + } + }() + m := fn() + _ = fmt.Sprintf("%v", m) + err := ErrInvalidMsgType{typ} + _ = err.Error() + }) + } +} + +func TestRegisterDuplicate(t *testing.T) { + defer func() { + if r := recover(); r == nil { + // We expect a panic. + t.FailNow() + } + }() + + // Register a duplicate. + register(&Rlerror{}) +} diff --git a/pkg/p9/p9.go b/pkg/p9/p9.go index 3b0993ecd..be644e7bf 100644 --- a/pkg/p9/p9.go +++ b/pkg/p9/p9.go @@ -984,6 +984,30 @@ func (s *SetAttr) Encode(b *buffer) { b.Write64(s.MTimeNanoSeconds) } +// Apply applies this to the given Attr. +func (a *Attr) Apply(mask SetAttrMask, attr SetAttr) { + if mask.Permissions { + a.Mode = a.Mode&^PermissionsMask | (attr.Permissions & PermissionsMask) + } + if mask.UID { + a.UID = attr.UID + } + if mask.GID { + a.GID = attr.GID + } + if mask.Size { + a.Size = attr.Size + } + if mask.ATime { + a.ATimeSeconds = attr.ATimeSeconds + a.ATimeNanoSeconds = attr.ATimeNanoSeconds + } + if mask.MTime { + a.MTimeSeconds = attr.MTimeSeconds + a.MTimeNanoSeconds = attr.MTimeNanoSeconds + } +} + // Dirent is used for readdir. type Dirent struct { // QID is the entry QID. diff --git a/pkg/p9/p9test/BUILD b/pkg/p9/p9test/BUILD index d6f428e11..7c4b875ce 100644 --- a/pkg/p9/p9test/BUILD +++ b/pkg/p9/p9test/BUILD @@ -1,16 +1,60 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_binary") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +alias( + name = "mockgen", + actual = "@com_github_golang_mock//mockgen:mockgen", +) -go_test( - name = "p9test_test", - size = "small", - srcs = ["client_test.go"], - embed = [":p9test"], +MOCK_SRC_PACKAGE = "gvisor.googlesource.com/gvisor/pkg/p9" + +# mockgen_reflect is a source file that contains mock generation code that +# imports the p9 package and generates a specification via reflection. The +# usual generation path must be split into two distinct parts because the full +# source tree is not available to all build targets. Only declared depencies +# are available (and even then, not the Go source files). +genrule( + name = "mockgen_reflect", + testonly = 1, + outs = ["mockgen_reflect.go"], + cmd = ( + "$(location :mockgen) " + + "-package p9test " + + "-prog_only " + MOCK_SRC_PACKAGE + " " + + "Attacher,File > $@" + ), + tools = [":mockgen"], +) + +# mockgen_exec is the binary that includes the above reflection generator. +# Running this binary will emit an encoded version of the p9 Attacher and File +# structures. This is consumed by the mocks genrule, below. +go_binary( + name = "mockgen_exec", + testonly = 1, + srcs = ["mockgen_reflect.go"], deps = [ - "//pkg/fd", "//pkg/p9", - "//pkg/unet", + "@com_github_golang_mock//mockgen/model:go_default_library", + ], +) + +# mocks consumes the encoded output above, and generates the full source for a +# set of mocks. These are included directly in the p9test library. +genrule( + name = "mocks", + testonly = 1, + outs = ["mocks.go"], + cmd = ( + "$(location :mockgen) " + + "-package p9test " + + "-exec_only $(location :mockgen_exec) " + MOCK_SRC_PACKAGE + " File > $@" + ), + tools = [ + ":mockgen", + ":mockgen_exec", ], ) @@ -18,11 +62,27 @@ go_library( name = "p9test", srcs = [ "mocks.go", + "p9test.go", ], importpath = "gvisor.googlesource.com/gvisor/pkg/p9/p9test", visibility = ["//:sandbox"], + deps = [ + "//pkg/fd", + "//pkg/log", + "//pkg/p9", + "//pkg/unet", + "@com_github_golang_mock//gomock:go_default_library", + ], +) + +go_test( + name = "client_test", + size = "small", + srcs = ["client_test.go"], + embed = [":p9test"], deps = [ "//pkg/fd", "//pkg/p9", + "@com_github_golang_mock//gomock:go_default_library", ], ) diff --git a/pkg/p9/p9test/client_test.go b/pkg/p9/p9test/client_test.go index db562b9ba..242d81b95 100644 --- a/pkg/p9/p9test/client_test.go +++ b/pkg/p9/p9test/client_test.go @@ -15,360 +15,2059 @@ package p9test import ( - "io/ioutil" + "bytes" + "fmt" + "io" + "math/rand" "os" "reflect" + "strings" + "sync" "syscall" "testing" + "time" + "github.com/golang/mock/gomock" "gvisor.googlesource.com/gvisor/pkg/fd" "gvisor.googlesource.com/gvisor/pkg/p9" - "gvisor.googlesource.com/gvisor/pkg/unet" ) -func TestDonateFD(t *testing.T) { - // Temporary file. - osFile, err := ioutil.TempFile("", "p9") +func TestPanic(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + // Create a new root. + d := h.NewDirectory(nil)(nil) + defer d.Close() // Needed manually. + h.Attacher.EXPECT().Attach().Return(d, nil).Do(func() { + // Panic here, and ensure that we get back EFAULT. + panic("handler") + }) + + // Attach to the client. + if _, err := c.Attach("/"); err != syscall.EFAULT { + t.Fatalf("got attach err %v, want EFAULT", err) + } +} + +func TestAttachNoLeak(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + // Create a new root. + d := h.NewDirectory(nil)(nil) + h.Attacher.EXPECT().Attach().Return(d, nil).Times(1) + + // Attach to the client. + f, err := c.Attach("/") + if err != nil { + t.Fatalf("got attach err %v, want nil", err) + } + + // Don't close the file. This should be closed automatically when the + // client disconnects. The mock asserts that everything is closed + // exactly once. This statement just removes the unused variable error. + _ = f +} + +func TestBadAttach(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + // Return an error on attach. + h.Attacher.EXPECT().Attach().Return(nil, syscall.EINVAL).Times(1) + + // Attach to the client. + if _, err := c.Attach("/"); err != syscall.EINVAL { + t.Fatalf("got attach err %v, want syscall.EINVAL", err) + } +} + +func TestWalkAttach(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + // Create a new root. + d := h.NewDirectory(map[string]Generator{ + "a": h.NewDirectory(map[string]Generator{ + "b": h.NewFile(), + }), + })(nil) + h.Attacher.EXPECT().Attach().Return(d, nil).Times(1) + + // Attach to the client as a non-root, and ensure that the walk above + // occurs as expected. We should get back b, and all references should + // be dropped when the file is closed. + f, err := c.Attach("/a/b") + if err != nil { + t.Fatalf("got attach err %v, want nil", err) + } + defer f.Close() + + // Check that's a regular file. + if _, _, attr, err := f.GetAttr(p9.AttrMaskAll()); err != nil { + t.Errorf("got err %v, want nil", err) + } else if !attr.Mode.IsRegular() { + t.Errorf("got mode %v, want regular file", err) + } +} + +// newTypeMap returns a new type map dictionary. +func newTypeMap(h *Harness) map[string]Generator { + return map[string]Generator{ + "directory": h.NewDirectory(map[string]Generator{}), + "file": h.NewFile(), + "symlink": h.NewSymlink(), + "block-device": h.NewBlockDevice(), + "character-device": h.NewCharacterDevice(), + "named-pipe": h.NewNamedPipe(), + "socket": h.NewSocket(), + } +} + +// newRoot returns a new root filesystem. +// +// This is set up in a deterministic way for testing most operations. +// +// The represented file system looks like: +// - file +// - symlink +// - directory +// ... +// + one +// - file +// - symlink +// - directory +// ... +// + two +// - file +// - symlink +// - directory +// ... +// + three +// - file +// - symlink +// - directory +// ... +func newRoot(h *Harness, c *p9.Client) (*Mock, p9.File) { + root := newTypeMap(h) + one := newTypeMap(h) + two := newTypeMap(h) + three := newTypeMap(h) + one["two"] = h.NewDirectory(two) // Will be nested in one. + root["one"] = h.NewDirectory(one) // Top level. + root["three"] = h.NewDirectory(three) // Alternate top-level. + + // Create a new root. + rootBackend := h.NewDirectory(root)(nil) + h.Attacher.EXPECT().Attach().Return(rootBackend, nil) + + // Attach to the client. + r, err := c.Attach("/") + if err != nil { + h.t.Fatalf("got attach err %v, want nil", err) + } + + return rootBackend, r +} + +func allInvalidNames(from string) []string { + return []string{ + from + "/other", + from + "/..", + from + "/.", + from + "/", + "other/" + from, + "/" + from, + "./" + from, + "../" + from, + ".", + "..", + "/", + "", + } +} + +func TestWalkInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Run relevant tests. + for name := range newTypeMap(h) { + // These are all the various ways that one might attempt to + // construct compound paths. They should all be rejected, as + // any compound that contains a / is not allowed, as well as + // the singular paths of '.' and '..'. + if _, _, err := root.Walk([]string{".", name}); err != syscall.EINVAL { + t.Errorf("Walk through . %s wanted EINVAL, got %v", name, err) + } + if _, _, err := root.Walk([]string{"..", name}); err != syscall.EINVAL { + t.Errorf("Walk through . %s wanted EINVAL, got %v", name, err) + } + if _, _, err := root.Walk([]string{name, "."}); err != syscall.EINVAL { + t.Errorf("Walk through %s . wanted EINVAL, got %v", name, err) + } + if _, _, err := root.Walk([]string{name, ".."}); err != syscall.EINVAL { + t.Errorf("Walk through %s .. wanted EINVAL, got %v", name, err) + } + for _, invalidName := range allInvalidNames(name) { + if _, _, err := root.Walk([]string{invalidName}); err != syscall.EINVAL { + t.Errorf("Walk through %s wanted EINVAL, got %v", invalidName, err) + } + } + wantErr := syscall.EINVAL + if name == "directory" { + // We can attempt a walk through a directory. However, + // we should never see a file named "other", so we + // expect this to return ENOENT. + wantErr = syscall.ENOENT + } + if _, _, err := root.Walk([]string{name, "other"}); err != wantErr { + t.Errorf("Walk through %s/other wanted %v, got %v", name, wantErr, err) + } + + // Do a successful walk. + _, f, err := root.Walk([]string{name}) + if err != nil { + t.Errorf("Walk to %s wanted nil, got %v", name, err) + } + defer f.Close() + local := h.Pop(f) + + // Check that the file matches. + _, localMask, localAttr, localErr := local.GetAttr(p9.AttrMaskAll()) + if _, mask, attr, err := f.GetAttr(p9.AttrMaskAll()); mask != localMask || attr != localAttr || err != localErr { + t.Errorf("GetAttr got (%v, %v, %v), wanted (%v, %v, %v)", + mask, attr, err, localMask, localAttr, localErr) + } + + // Ensure we can't walk backwards. + if _, _, err := f.Walk([]string{"."}); err != syscall.EINVAL { + t.Errorf("Walk through %s/. wanted EINVAL, got %v", name, err) + } + if _, _, err := f.Walk([]string{".."}); err != syscall.EINVAL { + t.Errorf("Walk through %s/.. wanted EINVAL, got %v", name, err) + } + } +} + +// fileGenerator is a function to generate files via walk or create. +// +// Examples are: +// - walkHelper +// - walkAndOpenHelper +// - createHelper +type fileGenerator func(*Harness, string, p9.File) (*Mock, *Mock, p9.File) + +// walkHelper walks to the given file. +// +// The backends of the parent and walked file are returned, as well as the +// walked client file. +func walkHelper(h *Harness, name string, dir p9.File) (parentBackend *Mock, walkedBackend *Mock, walked p9.File) { + _, parent, err := dir.Walk(nil) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer parent.Close() + parentBackend = h.Pop(parent) + + _, walked, err = parent.Walk([]string{name}) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + walkedBackend = h.Pop(walked) + + return parentBackend, walkedBackend, walked +} + +// walkAndOpenHelper additionally opens the walked file, if possible. +func walkAndOpenHelper(h *Harness, name string, dir p9.File) (*Mock, *Mock, p9.File) { + parentBackend, walkedBackend, walked := walkHelper(h, name, dir) + if p9.CanOpen(walkedBackend.Attr.Mode) { + // Open for all file types that we can. We stick to a read-only + // open here because directories may not be opened otherwise. + walkedBackend.EXPECT().Open(p9.ReadOnly).Times(1) + if _, _, _, err := walked.Open(p9.ReadOnly); err != nil { + h.t.Errorf("got open err %v, want nil", err) + } + } else { + // ... or assert an error for others. + if _, _, _, err := walked.Open(p9.ReadOnly); err != syscall.EINVAL { + h.t.Errorf("got open err %v, want EINVAL", err) + } + } + return parentBackend, walkedBackend, walked +} + +// createHelper creates the given file and returns the parent directory, +// created file and client file, which must be closed when done. +func createHelper(h *Harness, name string, dir p9.File) (*Mock, *Mock, p9.File) { + // Clone the directory first, since Create replaces the existing file. + // We change the type after calling create. + _, dirThenFile, err := dir.Walk(nil) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + + // Create a new server-side file. On the server-side, the a new file is + // returned from a create call. The client will reuse the same file, + // but we still expect the normal chain of closes. This complicates + // things a bit because the "parent" will always chain to the cloned + // dir above. + dirBackend := h.Pop(dirThenFile) // New backend directory. + newFile := h.NewFile()(dirBackend) // New file with backend parent. + dirBackend.EXPECT().Create(name, gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, newFile, newFile.QID, uint32(0), nil) + + // Create via the client. + _, dirThenFile, _, _, err = dirThenFile.Create(name, p9.ReadOnly, 0, 0, 0) if err != nil { - t.Fatalf("could not create temporary file: %v", err) + h.t.Fatalf("got create err %v, want nil", err) + } + + // Ensure subsequent walks succeed. + dirBackend.AddChild(name, h.NewFile()) + return dirBackend, newFile, dirThenFile +} + +// deprecatedRemover allows us to access the deprecated Remove operation within +// the p9.File client object. +type deprecatedRemover interface { + Remove() error +} + +// checkDeleted asserts that relevant methods fail for an unlinked file. +// +// This function will close the file at the end. +func checkDeleted(h *Harness, file p9.File) { + defer file.Close() // See doc. + + if _, _, _, err := file.Open(p9.ReadOnly); err != syscall.EINVAL { + h.t.Errorf("open while deleted, got %v, want EINVAL", err) + } + if _, _, _, _, err := file.Create("created", p9.ReadOnly, 0, 0, 0); err != syscall.EINVAL { + h.t.Errorf("create while deleted, got %v, want EINVAL", err) + } + if _, err := file.Symlink("old", "new", 0, 0); err != syscall.EINVAL { + h.t.Errorf("symlink while deleted, got %v, want EINVAL", err) + } + // N.B. This link is technically invalid, but if a call to link is + // actually made in the backend then the mock will panic. + if err := file.Link(file, "new"); err != syscall.EINVAL { + h.t.Errorf("link while deleted, got %v, want EINVAL", err) + } + if err := file.RenameAt("src", file, "dst"); err != syscall.EINVAL { + h.t.Errorf("renameAt while deleted, got %v, want EINVAL", err) + } + if err := file.UnlinkAt("file", 0); err != syscall.EINVAL { + h.t.Errorf("unlinkAt while deleted, got %v, want EINVAL", err) + } + if err := file.Rename(file, "dst"); err != syscall.EINVAL { + h.t.Errorf("rename while deleted, got %v, want EINVAL", err) + } + if _, err := file.Readlink(); err != syscall.EINVAL { + h.t.Errorf("readlink while deleted, got %v, want EINVAL", err) + } + if _, err := file.Mkdir("dir", p9.ModeDirectory, 0, 0); err != syscall.EINVAL { + h.t.Errorf("mkdir while deleted, got %v, want EINVAL", err) + } + if _, err := file.Mknod("dir", p9.ModeDirectory, 0, 0, 0, 0); err != syscall.EINVAL { + h.t.Errorf("mknod while deleted, got %v, want EINVAL", err) + } + if _, err := file.Readdir(0, 1); err != syscall.EINVAL { + h.t.Errorf("readdir while deleted, got %v, want EINVAL", err) + } + if _, err := file.Connect(p9.ConnectFlags(0)); err != syscall.EINVAL { + h.t.Errorf("connect while deleted, got %v, want EINVAL", err) + } + + // The remove method is technically deprecated, but we want to ensure + // that it still checks for deleted appropriately. We must first clone + // the file because remove is equivalent to close. + _, newFile, err := file.Walk(nil) + if err == syscall.EBUSY { + // We can't walk from here because this reference is open + // aleady. Okay, we will also have unopened cases through + // TestUnlink, just skip the remove operation for now. + return + } else if err != nil { + h.t.Fatalf("clone failed, got %v, want nil", err) + } + if err := newFile.(deprecatedRemover).Remove(); err != syscall.EINVAL { + h.t.Errorf("remove while deleted, got %v, want EINVAL", err) + } +} + +// deleter is a function to remove a file. +type deleter func(parent p9.File, name string) error + +// unlinkAt is a deleter. +func unlinkAt(parent p9.File, name string) error { + // Call unlink. Note that a filesystem may normally impose additional + // constaints on unlinkat success, such as ensuring that a directory is + // empty, requiring AT_REMOVEDIR in flags to remove a directory, etc. + // None of that is required internally (entire trees can be marked + // deleted when this operation succeeds), so the mock will succeed. + return parent.UnlinkAt(name, 0) +} + +// remove is a deleter. +func remove(parent p9.File, name string) error { + // See notes above re: remove. + _, newFile, err := parent.Walk([]string{name}) + if err != nil { + // Should not be expected. + return err + } + + // Do the actual remove. + if err := newFile.(deprecatedRemover).Remove(); err != nil { + return err + } + + // Ensure that the remove closed the file. + if err := newFile.(deprecatedRemover).Remove(); err != syscall.EBADF { + return syscall.EBADF // Propagate this code. + } + + return nil +} + +// unlinkHelper unlinks the noted path, and ensures that all relevant +// operations on that path, acquired from multiple paths, start failing. +func unlinkHelper(h *Harness, root p9.File, targetNames []string, targetGen fileGenerator, deleteFn deleter) { + // name is the file to be unlinked. + name := targetNames[len(targetNames)-1] + + // Walk to the directory containing the target. + _, parent, err := root.Walk(targetNames[:len(targetNames)-1]) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer parent.Close() + parentBackend := h.Pop(parent) + + // Walk to or generate the target file. + _, _, target := targetGen(h, name, parent) + defer checkDeleted(h, target) + + // Walk to a second reference. + _, second, err := parent.Walk([]string{name}) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer checkDeleted(h, second) + + // Walk to a third reference, from the start. + _, third, err := root.Walk(targetNames) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer checkDeleted(h, third) + + // This will be translated in the backend to an unlinkat. + parentBackend.EXPECT().UnlinkAt(name, uint32(0)).Return(nil) + + // Actually perform the deletion. + if err := deleteFn(parent, name); err != nil { + h.t.Fatalf("got delete err %v, want nil", err) + } +} + +func unlinkTest(t *testing.T, targetNames []string, targetGen fileGenerator) { + t.Run(fmt.Sprintf("unlinkAt(%s)", strings.Join(targetNames, "/")), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + unlinkHelper(h, root, targetNames, targetGen, unlinkAt) + }) + t.Run(fmt.Sprintf("remove(%s)", strings.Join(targetNames, "/")), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + unlinkHelper(h, root, targetNames, targetGen, remove) + }) +} + +func TestUnlink(t *testing.T) { + // Unlink all files. + for name := range newTypeMap(nil) { + unlinkTest(t, []string{name}, walkHelper) + unlinkTest(t, []string{name}, walkAndOpenHelper) + unlinkTest(t, []string{"one", name}, walkHelper) + unlinkTest(t, []string{"one", name}, walkAndOpenHelper) + unlinkTest(t, []string{"one", "two", name}, walkHelper) + unlinkTest(t, []string{"one", "two", name}, walkAndOpenHelper) + } + + // Unlink a directory. + unlinkTest(t, []string{"one"}, walkHelper) + unlinkTest(t, []string{"one"}, walkAndOpenHelper) + unlinkTest(t, []string{"one", "two"}, walkHelper) + unlinkTest(t, []string{"one", "two"}, walkAndOpenHelper) + + // Unlink created files. + unlinkTest(t, []string{"created"}, createHelper) + unlinkTest(t, []string{"one", "created"}, createHelper) + unlinkTest(t, []string{"one", "two", "created"}, createHelper) +} + +func TestUnlinkAtInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if err := root.UnlinkAt(invalidName, 0); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +// expectRenamed asserts an ordered sequence of rename calls, based on all the +// elements in elements being the source, and the first element therein +// changing to dstName, parented at dstParent. +func expectRenamed(file *Mock, elements []string, dstParent *Mock, dstName string) *gomock.Call { + if len(elements) > 0 { + // Recurse to the parent, if necessary. + call := expectRenamed(file.parent, elements[:len(elements)-1], dstParent, dstName) + + // Recursive case: this element is unchanged, but should have + // it's hook called after the parent. + return file.EXPECT().Renamed(file.parent, elements[len(elements)-1]).Do(func(p p9.File, _ string) { + file.parent = p.(*Mock) + }).After(call) + } + + // Base case: this is the changed element. + return file.EXPECT().Renamed(dstParent, dstName).Do(func(p p9.File, name string) { + file.parent = p.(*Mock) + }) +} + +// renamer is a rename function. +type renamer func(h *Harness, srcParent, dstParent p9.File, origName, newName string, selfRename bool) error + +// renameAt is a renamer. +func renameAt(_ *Harness, srcParent, dstParent p9.File, srcName, dstName string, selfRename bool) error { + return srcParent.RenameAt(srcName, dstParent, dstName) +} + +// rename is a renamer. +func rename(h *Harness, srcParent, dstParent p9.File, srcName, dstName string, selfRename bool) error { + _, f, err := srcParent.Walk([]string{srcName}) + if err != nil { + return err + } + defer f.Close() + if !selfRename { + backend := h.Pop(f) + backend.EXPECT().Renamed(gomock.Any(), dstName).Do(func(p p9.File, name string) { + backend.parent = p.(*Mock) // Required for close ordering. + }) + } + return f.Rename(dstParent, dstName) +} + +// renameHelper executes a rename, and asserts that all relevant elements +// receive expected notifications. If overwriting a file, this includes +// ensuring that the target has been appropriately marked as unlinked. +func renameHelper(h *Harness, root p9.File, srcNames []string, dstNames []string, target fileGenerator, renameFn renamer) { + // Walk to the directory containing the target. + srcQID, targetParent, err := root.Walk(srcNames[:len(srcNames)-1]) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer targetParent.Close() + targetParentBackend := h.Pop(targetParent) + + // Walk to or generate the target file. + _, targetBackend, src := target(h, srcNames[len(srcNames)-1], targetParent) + defer src.Close() + + // Walk to a second reference. + _, second, err := targetParent.Walk([]string{srcNames[len(srcNames)-1]}) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer second.Close() + secondBackend := h.Pop(second) + + // Walk to a third reference, from the start. + _, third, err := root.Walk(srcNames) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer third.Close() + thirdBackend := h.Pop(third) + + // Find the common suffix to identify the rename parent. + var ( + renameDestPath []string + renameSrcPath []string + selfRename bool + ) + for i := 1; i <= len(srcNames) && i <= len(dstNames); i++ { + if srcNames[len(srcNames)-i] != dstNames[len(dstNames)-i] { + // Take the full prefix of dstNames up until this + // point, including the first mismatched name. The + // first mismatch must be the renamed entry. + renameDestPath = dstNames[:len(dstNames)-i+1] + renameSrcPath = srcNames[:len(srcNames)-i+1] + + // Does the renameDestPath fully contain the + // renameSrcPath here? If yes, then this is a mismatch. + // We can't rename the src to some subpath of itself. + if len(renameDestPath) > len(renameSrcPath) && + reflect.DeepEqual(renameDestPath[:len(renameSrcPath)], renameSrcPath) { + renameDestPath = nil + renameSrcPath = nil + continue + } + break + } + } + if len(renameSrcPath) == 0 || len(renameDestPath) == 0 { + // This must be a rename to self, or a tricky look-alike. This + // happens iff we fail to find a suitable divergence in the two + // paths. It's a true self move if the path length is the same. + renameDestPath = dstNames + renameSrcPath = srcNames + selfRename = len(srcNames) == len(dstNames) + } + + // Walk to the source parent. + _, srcParent, err := root.Walk(renameSrcPath[:len(renameSrcPath)-1]) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer srcParent.Close() + srcParentBackend := h.Pop(srcParent) + + // Walk to the destination parent. + _, dstParent, err := root.Walk(renameDestPath[:len(renameDestPath)-1]) + if err != nil { + h.t.Fatalf("got walk err %v, want nil", err) + } + defer dstParent.Close() + dstParentBackend := h.Pop(dstParent) + + // expectedErr is the result of the rename operation. + var expectedErr error + + // Walk to the target file, if one exists. + dstQID, dst, err := root.Walk(renameDestPath) + if err == nil { + if !selfRename && srcQID[0].Type == dstQID[0].Type { + // If there is a destination file, and is it of the + // same type as the source file, then we expect the + // rename to succeed. We expect the destination file to + // be deleted, so we run a deletion test on it in this + // case. + defer checkDeleted(h, dst) + } else { + if !selfRename { + // If the type is different than the + // destination, then we expect the rename to + // fail. We expect ensure that this is + // returned. + expectedErr = syscall.EINVAL + } else { + // This is the file being renamed to itself. + // This is technically allowed and a no-op, but + // all the triggers will fire. + } + dst.Close() + } + } + dstName := renameDestPath[len(renameDestPath)-1] // Renamed element. + srcName := renameSrcPath[len(renameSrcPath)-1] // Renamed element. + if expectedErr == nil && !selfRename { + // Expect all to be renamed appropriately. Note that if this is + // a final file being renamed, then we expect the file to be + // called with the new parent. If not, then we expect the + // rename hook to be called, but the parent will remain + // unchanged. + elements := srcNames[len(renameSrcPath):] + expectRenamed(targetBackend, elements, dstParentBackend, dstName) + expectRenamed(secondBackend, elements, dstParentBackend, dstName) + expectRenamed(thirdBackend, elements, dstParentBackend, dstName) + + // The target parent has also been opened, and may be moved + // directly or indirectly. + if len(elements) > 1 { + expectRenamed(targetParentBackend, elements[:len(elements)-1], dstParentBackend, dstName) + } + } + + // Expect the rename if it's not the same file. Note that like unlink, + // renames are always translated to the at variant in the backend. + if !selfRename { + srcParentBackend.EXPECT().RenameAt(srcName, dstParentBackend, dstName).Return(expectedErr) + } + + // Perform the actual rename; everything has been lined up. + if err := renameFn(h, srcParent, dstParent, srcName, dstName, selfRename); err != expectedErr { + h.t.Fatalf("got rename err %v, want %v", err, expectedErr) + } +} + +func renameTest(t *testing.T, srcNames []string, dstNames []string, target fileGenerator) { + t.Run(fmt.Sprintf("renameAt(%s->%s)", strings.Join(srcNames, "/"), strings.Join(dstNames, "/")), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + renameHelper(h, root, srcNames, dstNames, target, renameAt) + }) + t.Run(fmt.Sprintf("rename(%s->%s)", strings.Join(srcNames, "/"), strings.Join(dstNames, "/")), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + renameHelper(h, root, srcNames, dstNames, target, rename) + }) +} + +func TestRename(t *testing.T) { + // In-directory rename, simple case. + for name := range newTypeMap(nil) { + // Within the root. + renameTest(t, []string{name}, []string{"renamed"}, walkHelper) + renameTest(t, []string{name}, []string{"renamed"}, walkAndOpenHelper) + + // Within a subdirectory. + renameTest(t, []string{"one", name}, []string{"one", "renamed"}, walkHelper) + renameTest(t, []string{"one", name}, []string{"one", "renamed"}, walkAndOpenHelper) + } + + // ... with created files. + renameTest(t, []string{"created"}, []string{"renamed"}, createHelper) + renameTest(t, []string{"one", "created"}, []string{"one", "renamed"}, createHelper) + + // Across directories. + for name := range newTypeMap(nil) { + // Down one level. + renameTest(t, []string{"one", name}, []string{"one", "two", "renamed"}, walkHelper) + renameTest(t, []string{"one", name}, []string{"one", "two", "renamed"}, walkAndOpenHelper) + + // Up one level. + renameTest(t, []string{"one", "two", name}, []string{"one", "renamed"}, walkHelper) + renameTest(t, []string{"one", "two", name}, []string{"one", "renamed"}, walkAndOpenHelper) + + // Across at the same level. + renameTest(t, []string{"one", name}, []string{"three", "renamed"}, walkHelper) + renameTest(t, []string{"one", name}, []string{"three", "renamed"}, walkAndOpenHelper) + } + + // ... with created files. + renameTest(t, []string{"one", "created"}, []string{"one", "two", "renamed"}, createHelper) + renameTest(t, []string{"one", "two", "created"}, []string{"one", "renamed"}, createHelper) + renameTest(t, []string{"one", "created"}, []string{"three", "renamed"}, createHelper) + + // Renaming parents. + for name := range newTypeMap(nil) { + // Rename a parent. + renameTest(t, []string{"one", name}, []string{"renamed", name}, walkHelper) + renameTest(t, []string{"one", name}, []string{"renamed", name}, walkAndOpenHelper) + + // Rename a super parent. + renameTest(t, []string{"one", "two", name}, []string{"renamed", name}, walkHelper) + renameTest(t, []string{"one", "two", name}, []string{"renamed", name}, walkAndOpenHelper) + } + + // ... with created files. + renameTest(t, []string{"one", "created"}, []string{"renamed", "created"}, createHelper) + renameTest(t, []string{"one", "two", "created"}, []string{"renamed", "created"}, createHelper) + + // Over existing files, including itself. + for name := range newTypeMap(nil) { + for other := range newTypeMap(nil) { + // Overwrite the noted file (may be itself). + renameTest(t, []string{"one", name}, []string{"one", other}, walkHelper) + renameTest(t, []string{"one", name}, []string{"one", other}, walkAndOpenHelper) + + // Overwrite other files in another directory. + renameTest(t, []string{"one", name}, []string{"one", "two", other}, walkHelper) + renameTest(t, []string{"one", name}, []string{"one", "two", other}, walkAndOpenHelper) + } + + // Overwrite by moving the parent. + renameTest(t, []string{"three", name}, []string{"one", name}, walkHelper) + renameTest(t, []string{"three", name}, []string{"one", name}, walkAndOpenHelper) + + // Create over the types. + renameTest(t, []string{"one", "created"}, []string{"one", name}, createHelper) + renameTest(t, []string{"one", "created"}, []string{"one", "two", name}, createHelper) + renameTest(t, []string{"three", "created"}, []string{"one", name}, createHelper) + } +} + +func TestRenameInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if err := root.Rename(root, invalidName); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestRenameAtInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if err := root.RenameAt(invalidName, root, "okay"); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + if err := root.RenameAt("okay", root, invalidName); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestReadlink(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, f, err := root.Walk([]string{name}) + if err != nil { + t.Fatalf("walk failed: got %v, wanted nil", err) + } + defer f.Close() + backend := h.Pop(f) + + const symlinkTarget = "symlink-target" + + if backend.Attr.Mode.IsSymlink() { + // This should only go through on symlinks. + backend.EXPECT().Readlink().Return(symlinkTarget, nil) + } + + // Attempt a Readlink operation. + target, err := f.Readlink() + if err != nil && err != syscall.EINVAL { + t.Errorf("readlink got %v, wanted EINVAL", err) + } else if err == nil && target != symlinkTarget { + t.Errorf("readlink got %v, wanted %v", target, symlinkTarget) + } + }) + } +} + +// fdTest is a wrapper around operations that may send file descriptors. This +// asserts that the file descriptors are working as intended. +func fdTest(t *testing.T, sendFn func(*fd.FD) *fd.FD) { + // Create a pipe that we can read from. + r, w, err := os.Pipe() + if err != nil { + t.Fatalf("unable to create pipe: %v", err) + } + defer r.Close() + defer w.Close() + + // Attempt to send the write end. + wFD, err := fd.NewFromFile(w) + if err != nil { + t.Fatalf("unable to convert file: %v", err) + } + defer wFD.Close() // This is a copy. + + // Send wFD and receive newFD. + newFD := sendFn(wFD) + defer newFD.Close() + + // Attempt to write. + const message = "hello" + if _, err := newFD.Write([]byte(message)); err != nil { + t.Fatalf("write got %v, wanted nil", err) + } + + // Should see the message on our end. + buffer := []byte(message) + if _, err := io.ReadFull(r, buffer); err != nil { + t.Fatalf("read got %v, wanted nil", err) + } + if string(buffer) != message { + t.Errorf("got message %v, wanted %v", string(buffer), message) + } +} + +func TestConnect(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Catch all the non-socket cases. + if !backend.Attr.Mode.IsSocket() { + // This has been set up to fail if Connect is called. + if _, err := f.Connect(p9.ConnectFlags(0)); err != syscall.EINVAL { + t.Errorf("connect got %v, wanted EINVAL", err) + } + return + } + + // Ensure the fd exchange works. + fdTest(t, func(send *fd.FD) *fd.FD { + backend.EXPECT().Connect(p9.ConnectFlags(0)).Return(send, nil) + recv, err := backend.Connect(p9.ConnectFlags(0)) + if err != nil { + t.Fatalf("connect got %v, wanted nil", err) + } + return recv + }) + }) + } +} + +func TestReaddir(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Catch all the non-directory cases. + if !backend.Attr.Mode.IsDir() { + // This has also been set up to fail if Readdir is called. + if _, err := f.Readdir(0, 1); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + return + } + + // Ensure that readdir works for directories. + if _, err := f.Readdir(0, 1); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + if _, _, _, err := f.Open(p9.ReadWrite); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + if _, _, _, err := f.Open(p9.WriteOnly); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + backend.EXPECT().Open(p9.ReadOnly).Times(1) + if _, _, _, err := f.Open(p9.ReadOnly); err != nil { + t.Errorf("readdir got %v, wanted nil", err) + } + backend.EXPECT().Readdir(uint64(0), uint32(1)).Times(1) + if _, err := f.Readdir(0, 1); err != nil { + t.Errorf("readdir got %v, wanted nil", err) + } + }) + } +} + +func TestOpen(t *testing.T) { + type openTest struct { + name string + mode p9.OpenFlags + err error + match func(p9.FileMode) bool + } + + cases := []openTest{ + { + name: "invalid", + mode: ^p9.OpenFlagsModeMask, + err: syscall.EINVAL, + match: func(p9.FileMode) bool { return true }, + }, + { + name: "not-openable-read-only", + mode: p9.ReadOnly, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return !p9.CanOpen(mode) }, + }, + { + name: "not-openable-write-only", + mode: p9.WriteOnly, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return !p9.CanOpen(mode) }, + }, + { + name: "not-openable-read-write", + mode: p9.ReadWrite, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return !p9.CanOpen(mode) }, + }, + { + name: "directory-read-only", + mode: p9.ReadOnly, + err: nil, + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + }, + { + name: "directory-read-write", + mode: p9.ReadWrite, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + }, + { + name: "directory-write-only", + mode: p9.WriteOnly, + err: syscall.EINVAL, + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + }, + { + name: "read-only", + mode: p9.ReadOnly, + err: nil, + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) }, + }, + { + name: "write-only", + mode: p9.WriteOnly, + err: nil, + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) && !mode.IsDir() }, + }, + { + name: "read-write", + mode: p9.ReadWrite, + err: nil, + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) && !mode.IsDir() }, + }, + } + + // Open(mode OpenFlags) (*fd.FD, QID, uint32, error) + // - only works on Regular, NamedPipe, BLockDevice, CharacterDevice + // - returning a file works as expected + for name := range newTypeMap(nil) { + for _, tc := range cases { + t.Run(fmt.Sprintf("%s-%s", tc.name, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Does this match the case? + if !tc.match(backend.Attr.Mode) { + t.SkipNow() + } + + // Ensure open-required operations fail. + if _, err := f.ReadAt([]byte("hello"), 0); err != syscall.EINVAL { + t.Errorf("readAt got %v, wanted EINVAL", err) + } + if _, err := f.WriteAt(make([]byte, 6), 0); err != syscall.EINVAL { + t.Errorf("writeAt got %v, wanted EINVAL", err) + } + if err := f.FSync(); err != syscall.EINVAL { + t.Errorf("fsync got %v, wanted EINVAL", err) + } + if _, err := f.Readdir(0, 1); err != syscall.EINVAL { + t.Errorf("readdir got %v, wanted EINVAL", err) + } + + // Attempt the given open. + if tc.err != nil { + // We expect an error, just test and return. + if _, _, _, err := f.Open(tc.mode); err != tc.err { + t.Fatalf("open with mode %v got %v, want %v", tc.mode, err, tc.err) + } + return + } + + // Run an FD test, since we expect success. + fdTest(t, func(send *fd.FD) *fd.FD { + backend.EXPECT().Open(tc.mode).Return(send, p9.QID{}, uint32(0), nil).Times(1) + recv, _, _, err := f.Open(tc.mode) + if err != tc.err { + t.Fatalf("open with mode %v got %v, want %v", tc.mode, err, tc.err) + } + return recv + }) + + // If the open was successful, attempt another one. + if _, _, _, err := f.Open(tc.mode); err != syscall.EINVAL { + t.Errorf("second open with mode %v got %v, want EINVAL", tc.mode, err) + } + + // Ensure that all illegal operations fail. + if _, _, err := f.Walk(nil); err != syscall.EINVAL && err != syscall.EBUSY { + t.Errorf("walk got %v, wanted EINVAL or EBUSY", err) + } + if _, _, _, _, err := f.WalkGetAttr(nil); err != syscall.EINVAL && err != syscall.EBUSY { + t.Errorf("walkgetattr got %v, wanted EINVAL or EBUSY", err) + } + }) + } + } +} + +func TestClose(t *testing.T) { + type closeTest struct { + name string + closeFn func(backend *Mock, f p9.File) + } + + cases := []closeTest{ + { + name: "close", + closeFn: func(_ *Mock, f p9.File) { + f.Close() + }, + }, + { + name: "remove", + closeFn: func(backend *Mock, f p9.File) { + // Allow the rename call in the parent, automatically translated. + backend.parent.EXPECT().UnlinkAt(gomock.Any(), gomock.Any()).Times(1) + f.(deprecatedRemover).Remove() + }, + }, } - os.Remove(osFile.Name()) - hfi, err := osFile.Stat() - if err != nil { - osFile.Close() - t.Fatalf("stat failed: %v", err) + for name := range newTypeMap(nil) { + for _, tc := range cases { + t.Run(fmt.Sprintf("%s(%s)", tc.name, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + + // Close via the prescribed method. + tc.closeFn(backend, f) + + // Everything should fail with EBADF. + if _, _, err := f.Walk(nil); err != syscall.EBADF { + t.Errorf("walk got %v, wanted EBADF", err) + } + if _, err := f.StatFS(); err != syscall.EBADF { + t.Errorf("statfs got %v, wanted EBADF", err) + } + if _, _, _, err := f.GetAttr(p9.AttrMaskAll()); err != syscall.EBADF { + t.Errorf("getattr got %v, wanted EBADF", err) + } + if err := f.SetAttr(p9.SetAttrMask{}, p9.SetAttr{}); err != syscall.EBADF { + t.Errorf("setattrk got %v, wanted EBADF", err) + } + if err := f.Rename(root, "new-name"); err != syscall.EBADF { + t.Errorf("rename got %v, wanted EBADF", err) + } + if err := f.Close(); err != syscall.EBADF { + t.Errorf("close got %v, wanted EBADF", err) + } + if _, _, _, err := f.Open(p9.ReadOnly); err != syscall.EBADF { + t.Errorf("open got %v, wanted EBADF", err) + } + if _, err := f.ReadAt([]byte("hello"), 0); err != syscall.EBADF { + t.Errorf("readAt got %v, wanted EBADF", err) + } + if _, err := f.WriteAt(make([]byte, 6), 0); err != syscall.EBADF { + t.Errorf("writeAt got %v, wanted EBADF", err) + } + if err := f.FSync(); err != syscall.EBADF { + t.Errorf("fsync got %v, wanted EBADF", err) + } + if _, _, _, _, err := f.Create("new-file", p9.ReadWrite, 0, 0, 0); err != syscall.EBADF { + t.Errorf("create got %v, wanted EBADF", err) + } + if _, err := f.Mkdir("new-directory", 0, 0, 0); err != syscall.EBADF { + t.Errorf("mkdir got %v, wanted EBADF", err) + } + if _, err := f.Symlink("old-name", "new-name", 0, 0); err != syscall.EBADF { + t.Errorf("symlink got %v, wanted EBADF", err) + } + if err := f.Link(root, "new-name"); err != syscall.EBADF { + t.Errorf("link got %v, wanted EBADF", err) + } + if _, err := f.Mknod("new-block-device", 0, 0, 0, 0, 0); err != syscall.EBADF { + t.Errorf("mknod got %v, wanted EBADF", err) + } + if err := f.RenameAt("old-name", root, "new-name"); err != syscall.EBADF { + t.Errorf("renameAt got %v, wanted EBADF", err) + } + if err := f.UnlinkAt("name", 0); err != syscall.EBADF { + t.Errorf("unlinkAt got %v, wanted EBADF", err) + } + if _, err := f.Readdir(0, 1); err != syscall.EBADF { + t.Errorf("readdir got %v, wanted EBADF", err) + } + if _, err := f.Readlink(); err != syscall.EBADF { + t.Errorf("readlink got %v, wanted EBADF", err) + } + if err := f.Flush(); err != syscall.EBADF { + t.Errorf("flush got %v, wanted EBADF", err) + } + if _, _, _, _, err := f.WalkGetAttr(nil); err != syscall.EBADF { + t.Errorf("walkgetattr got %v, wanted EBADF", err) + } + if _, err := f.Connect(p9.ConnectFlags(0)); err != syscall.EBADF { + t.Errorf("connect got %v, wanted EBADF", err) + } + }) + } + } +} + +// onlyWorksOnOpenThings is a helper test method for operations that should +// only work on files that have been explicitly opened. +func onlyWorksOnOpenThings(h *Harness, t *testing.T, name string, root p9.File, mode p9.OpenFlags, expectedErr error, fn func(backend *Mock, f p9.File, shouldSucceed bool) error) { + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Does it work before opening? + if err := fn(backend, f, false); err != syscall.EINVAL { + t.Errorf("operation got %v, wanted EINVAL", err) } - osFileStat := hfi.Sys().(*syscall.Stat_t) - f, err := fd.NewFromFile(osFile) - // osFile should always be closed. - osFile.Close() - if err != nil { - t.Fatalf("unable to create file: %v", err) + // Is this openable? + if !p9.CanOpen(backend.Attr.Mode) { + return // Nothing to do. + } + + // If this is a directory, we can't handle writing. + if backend.Attr.Mode.IsDir() && (mode == p9.ReadWrite || mode == p9.WriteOnly) { + return // Skip. + } + + // Open the file. + backend.EXPECT().Open(mode) + if _, _, _, err := f.Open(mode); err != nil { + t.Fatalf("open got %v, wanted nil", err) + } + + // Attempt the operation. + if err := fn(backend, f, expectedErr == nil); err != expectedErr { + t.Fatalf("operation got %v, wanted %v", err, expectedErr) + } +} + +func TestRead(t *testing.T) { + type readTest struct { + name string + mode p9.OpenFlags + err error + } + + cases := []readTest{ + { + name: "read-only", + mode: p9.ReadOnly, + err: nil, + }, + { + name: "read-write", + mode: p9.ReadWrite, + err: nil, + }, + { + name: "write-only", + mode: p9.WriteOnly, + err: syscall.EPERM, + }, + } + + for name := range newTypeMap(nil) { + for _, tc := range cases { + t.Run(fmt.Sprintf("%s-%s", tc.name, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + const message = "hello" + + onlyWorksOnOpenThings(h, t, name, root, tc.mode, tc.err, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if !shouldSucceed { + _, err := f.ReadAt([]byte(message), 0) + return err + } + + // Prepare for the call to readAt in the backend. + backend.EXPECT().ReadAt(gomock.Any(), uint64(0)).Do(func(p []byte, offset uint64) { + copy(p, message) + }).Return(len(message), nil) + + // Make the client call. + p := make([]byte, 2*len(message)) // Double size. + n, err := f.ReadAt(p, 0) + + // Sanity check result. + if err != nil { + return err + } + if n != len(message) { + t.Fatalf("message length incorrect, got %d, want %d", n, len(message)) + } + if !bytes.Equal(p[:n], []byte(message)) { + t.Fatalf("message incorrect, got %v, want %v", p, []byte(message)) + } + return nil // Success. + }) + }) + } + } +} + +func TestWrite(t *testing.T) { + type writeTest struct { + name string + mode p9.OpenFlags + err error } - // Craft attacher to attach to the mocked file which will return our - // temporary file. - fileMock := &FileMock{ - OpenMock: OpenMock{File: f}, - GetAttrMock: GetAttrMock{ - // The mode must be valid always. - Valid: p9.AttrMask{Mode: true}, + cases := []writeTest{ + { + name: "read-only", + mode: p9.ReadOnly, + err: syscall.EPERM, + }, + { + name: "read-write", + mode: p9.ReadWrite, + err: nil, + }, + { + name: "write-only", + mode: p9.WriteOnly, + err: nil, }, } - attacher := &AttachMock{ - File: fileMock, + + for name := range newTypeMap(nil) { + for _, tc := range cases { + t.Run(fmt.Sprintf("%s-%s", tc.name, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + const message = "hello" + + onlyWorksOnOpenThings(h, t, name, root, tc.mode, tc.err, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if !shouldSucceed { + _, err := f.WriteAt([]byte(message), 0) + return err + } + + // Prepare for the call to readAt in the backend. + var output []byte // Saved by Do below. + backend.EXPECT().WriteAt(gomock.Any(), uint64(0)).Do(func(p []byte, offset uint64) { + output = p + }).Return(len(message), nil) + + // Make the client call. + n, err := f.WriteAt([]byte(message), 0) + + // Sanity check result. + if err != nil { + return err + } + if n != len(message) { + t.Fatalf("message length incorrect, got %d, want %d", n, len(message)) + } + if !bytes.Equal(output, []byte(message)) { + t.Fatalf("message incorrect, got %v, want %v", output, []byte(message)) + } + return nil // Success. + }) + }) + } } +} - // Make socket pair. - serverSocket, clientSocket, err := unet.SocketPair(false) - if err != nil { - t.Fatalf("socketpair got err %v wanted nil", err) +func TestFSync(t *testing.T) { + for name := range newTypeMap(nil) { + for _, mode := range []p9.OpenFlags{p9.ReadOnly, p9.WriteOnly, p9.ReadWrite} { + t.Run(fmt.Sprintf("%s-%s", mode, name), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnOpenThings(h, t, name, root, mode, nil, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().FSync().Times(1) + } + return f.FSync() + }) + }) + } } - defer clientSocket.Close() - server := p9.NewServer(attacher) - go server.Handle(serverSocket) - client, err := p9.NewClient(clientSocket, 1024*1024 /* 1M message size */, p9.HighestVersionString()) - if err != nil { - t.Fatalf("new client got %v, expected nil", err) +} + +func TestFlush(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + backend.EXPECT().Flush() + f.Flush() + }) } +} - // Attach to the mocked file. - cFile, err := client.Attach("") - if err != nil { - t.Fatalf("attach failed: %v", err) +// onlyWorksOnDirectories is a helper test method for operations that should +// only work on unopened directories, such as create, mkdir and symlink. +func onlyWorksOnDirectories(h *Harness, t *testing.T, name string, root p9.File, fn func(backend *Mock, f p9.File, shouldSucceed bool) error) { + // Walk to the file normally. + _, backend, f := walkHelper(h, name, root) + defer f.Close() + + // Only directories support mknod. + if !backend.Attr.Mode.IsDir() { + if err := fn(backend, f, false); err != syscall.EINVAL { + t.Errorf("operation got %v, wanted EINVAL", err) + } + return // Nothing else to do. } - // Try to open the mocked file. - clientHostFile, _, _, err := cFile.Open(0) - if err != nil { - t.Fatalf("open failed: %v", err) + // Should succeed. + if err := fn(backend, f, true); err != nil { + t.Fatalf("operation got %v, wanted nil", err) } - var clientStat syscall.Stat_t - if err := syscall.Fstat(clientHostFile.FD(), &clientStat); err != nil { - t.Fatalf("stat failed: %v", err) + + // Open the directory. + backend.EXPECT().Open(p9.ReadOnly).Times(1) + if _, _, _, err := f.Open(p9.ReadOnly); err != nil { + t.Fatalf("open got %v, wanted nil", err) } - // Compare inode nums to make sure it's the same file. - if clientStat.Ino != osFileStat.Ino { - t.Errorf("fd donation failed") + // Should not work again. + if err := fn(backend, f, false); err != syscall.EINVAL { + t.Fatalf("operation got %v, wanted EINVAL", err) } } -// TestClient is a megatest. -// -// This allows us to probe various edge cases, while changing the state of the -// underlying server in expected ways. The test slowly builds server state and -// is documented inline. -// -// We wind up with the following, after probing edge cases: -// -// FID 1: ServerFile (sf). -// FID 2: Directory (d). -// FID 3: File (f). -// FID 4: Symlink (s). -// -// Although you should use the FID method on the individual files. -func TestClient(t *testing.T) { +func TestCreate(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if !shouldSucceed { + _, _, _, _, err := f.Create("new-file", p9.ReadWrite, 0, 1, 2) + return err + } + + // If the create is going to succeed, then we + // need to create a new backend file, and we + // clone to ensure that we don't close the + // original. + _, newF, err := f.Walk(nil) + if err != nil { + t.Fatalf("clone got %v, wanted nil", err) + } + defer newF.Close() + newBackend := h.Pop(newF) + + // Run a regular FD test to validate that path. + fdTest(t, func(send *fd.FD) *fd.FD { + // Return the send FD on success. + newFile := h.NewFile()(backend) // New file with the parent backend. + newBackend.EXPECT().Create("new-file", p9.ReadWrite, p9.FileMode(0), p9.UID(1), p9.GID(2)).Return(send, newFile, p9.QID{}, uint32(0), nil) + + // Receive the fd back. + recv, _, _, _, err := newF.Create("new-file", p9.ReadWrite, 0, 1, 2) + if err != nil { + t.Fatalf("create got %v, wanted nil", err) + } + return recv + }) + + // The above will fail via normal test flow, so + // we can assume that it passed. + return nil + }) + }) + } +} + +func TestCreateInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if _, _, _, _, err := root.Create(invalidName, p9.ReadWrite, 0, 0, 0); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestMkdir(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().Mkdir("new-directory", p9.FileMode(0), p9.UID(1), p9.GID(2)) + } + _, err := f.Mkdir("new-directory", 0, 1, 2) + return err + }) + }) + } +} + +func TestMkdirInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if _, err := root.Mkdir(invalidName, 0, 0, 0); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestSymlink(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().Symlink("old-name", "new-name", p9.UID(1), p9.GID(2)) + } + _, err := f.Symlink("old-name", "new-name", 1, 2) + return err + }) + }) + } +} + +func TestSyminkInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + // We need only test for invalid names in the new name, + // the target can be an arbitrary string and we don't + // need to sanity check it. + if _, err := root.Symlink("old-name", invalidName, 0, 0); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestLink(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().Link(gomock.Any(), "new-link") + } + return f.Link(f, "new-link") + }) + }) + } +} + +func TestLinkInvalid(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + for name := range newTypeMap(nil) { + for _, invalidName := range allInvalidNames(name) { + if err := root.Link(root, invalidName); err != syscall.EINVAL { + t.Errorf("got %v for name %q, want EINVAL", err, invalidName) + } + } + } +} + +func TestMknod(t *testing.T) { + for name := range newTypeMap(nil) { + t.Run(name, func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + onlyWorksOnDirectories(h, t, name, root, func(backend *Mock, f p9.File, shouldSucceed bool) error { + if shouldSucceed { + backend.EXPECT().Mknod("new-block-device", p9.FileMode(0), uint32(1), uint32(2), p9.UID(3), p9.GID(4)).Times(1) + } + _, err := f.Mknod("new-block-device", 0, 1, 2, 3, 4) + return err + }) + }) + } +} + +// concurrentFn is a specification of a concurrent operation. This is used to +// drive the concurrency tests below. +type concurrentFn struct { + name string + match func(p9.FileMode) bool + op func(h *Harness, backend *Mock, f p9.File, callback func()) +} + +func concurrentTest(t *testing.T, name string, fn1, fn2 concurrentFn, sameDir, expectedOkay bool) { var ( - // Sentinel error. - sentinelErr = syscall.Errno(4383) - - // Backend mocks. - a = &AttachMock{} - sf = &FileMock{} - d = &FileMock{} - f = &FileMock{} - s = &FileMock{} - - // Client Files for the above. - sfFile p9.File + names1 []string + names2 []string ) + if sameDir { + // Use the same file one directory up. + names1, names2 = []string{"one", name}, []string{"one", name} + } else { + // For different directories, just use siblings. + names1, names2 = []string{"one", name}, []string{"three", name} + } - testSteps := []struct { - name string - fn func(*p9.Client) error - want error - }{ - { - name: "bad-attach", - want: sentinelErr, - fn: func(c *p9.Client) error { - a.File = nil - a.Err = sentinelErr - _, err := c.Attach("") - return err + t.Run(fmt.Sprintf("%s(%v)+%s(%v)", fn1.name, names1, fn2.name, names2), func(t *testing.T) { + h, c := NewHarness(t) + defer h.Finish() + + _, root := newRoot(h, c) + defer root.Close() + + // Walk to both files as given. + _, f1, err := root.Walk(names1) + if err != nil { + t.Fatalf("error walking, got %v, want nil", err) + } + defer f1.Close() + b1 := h.Pop(f1) + _, f2, err := root.Walk(names2) + if err != nil { + t.Fatalf("error walking, got %v, want nil", err) + } + defer f2.Close() + b2 := h.Pop(f2) + + // Are these a good match for the current test case? + if !fn1.match(b1.Attr.Mode) { + t.SkipNow() + } + if !fn2.match(b2.Attr.Mode) { + t.SkipNow() + } + + // Construct our "concurrency creator". + in1 := make(chan struct{}, 1) + in2 := make(chan struct{}, 1) + var top sync.WaitGroup + var fns sync.WaitGroup + defer top.Wait() + top.Add(2) // Accounting for below. + defer fns.Done() + fns.Add(1) // See line above; released before top.Wait. + go func() { + defer top.Done() + fn1.op(h, b1, f1, func() { + in1 <- struct{}{} + fns.Wait() + }) + }() + go func() { + defer top.Done() + fn2.op(h, b2, f2, func() { + in2 <- struct{}{} + fns.Wait() + }) + }() + + // Compute a reasonable timeout. If we expect the operation to hang, + // give it 10 milliseconds before we assert that it's fine. After all, + // there will be a lot of these tests. If we don't expect it to hang, + // give it a full minute, since the machine could be slow. + timeout := 10 * time.Millisecond + if expectedOkay { + timeout = 1 * time.Minute + } + + // Read the first channel. + var second chan struct{} + select { + case <-in1: + second = in2 + case <-in2: + second = in1 + } + + // Catch concurrency. + select { + case <-second: + // We finished successful. Is this good? Depends on the + // expected result. + if !expectedOkay { + t.Errorf("%q and %q proceeded concurrently!", fn1.name, fn2.name) + } + case <-time.After(timeout): + // Great, things did not proceed concurrently. Is that what we + // expected? + if expectedOkay { + t.Errorf("%q and %q hung concurrently!", fn1.name, fn2.name) + } + } + }) +} + +func randomFileName() string { + return fmt.Sprintf("%x", rand.Int63()) +} + +func TestConcurrency(t *testing.T) { + readExclusive := []concurrentFn{ + { + // N.B. We can't explicitly check WalkGetAttr behavior, + // but we rely on the fact that the internal code paths + // are the same. + name: "walk", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + // See the documentation of WalkCallback. + // Because walk is actually implemented by the + // mock, we need a special place for this + // callback. + // + // Note that a clone actually locks the parent + // node. So we walk from this node to test + // concurrent operations appropriately. + backend.WalkCallback = func() error { + callback() + return nil + } + f.Walk([]string{randomFileName()}) // Won't exist. }, }, { - name: "attach", - fn: func(c *p9.Client) error { - a.Called = false - a.File = sf - a.Err = nil - // The attached root must have a valid mode. - sf.GetAttrMock.Attr = p9.Attr{Mode: p9.ModeDirectory} - sf.GetAttrMock.Valid = p9.AttrMask{Mode: true} - var err error - sfFile, err = c.Attach("") - if !a.Called { - t.Errorf("Attach never Called?") - } - return err + name: "fsync", + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Open(gomock.Any()) + backend.EXPECT().FSync().Do(func() { + callback() + }) + f.Open(p9.ReadOnly) // Required. + f.FSync() }, }, { - name: "bad-walk", - want: sentinelErr, - fn: func(c *p9.Client) error { - // Walk only called when WalkGetAttr not available. - sf.WalkGetAttrMock.Err = syscall.ENOSYS - sf.WalkMock.File = d - sf.WalkMock.Err = sentinelErr - _, _, err := sfFile.Walk([]string{"foo", "bar"}) - return err + name: "readdir", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Open(gomock.Any()) + backend.EXPECT().Readdir(gomock.Any(), gomock.Any()).Do(func(uint64, uint32) { + callback() + }) + f.Open(p9.ReadOnly) // Required. + f.Readdir(0, 1) }, }, { - name: "walk-to-dir", - fn: func(c *p9.Client) error { - // Walk only called when WalkGetAttr not available. - sf.WalkGetAttrMock.Err = syscall.ENOSYS - sf.WalkMock.Called = false - sf.WalkMock.Names = nil - sf.WalkMock.File = d - sf.WalkMock.Err = nil - sf.WalkMock.QIDs = []p9.QID{{Type: 1}} - // All intermediate values must be directories. - d.WalkGetAttrMock.Err = syscall.ENOSYS - d.WalkMock.Called = false - d.WalkMock.Names = nil - d.WalkMock.File = d // Walk to self. - d.WalkMock.Err = nil - d.WalkMock.QIDs = []p9.QID{{Type: 1}} - d.GetAttrMock.Attr = p9.Attr{Mode: p9.ModeDirectory} - d.GetAttrMock.Valid = p9.AttrMask{Mode: true} - var qids []p9.QID - var err error - qids, _, err = sfFile.Walk([]string{"foo", "bar"}) - if !sf.WalkMock.Called { - t.Errorf("Walk never Called?") - } - if !d.GetAttrMock.Called { - t.Errorf("GetAttr never Called?") - } - if !reflect.DeepEqual(sf.WalkMock.Names, []string{"foo"}) { - t.Errorf("got names %v wanted []{foo}", sf.WalkMock.Names) - } - if !reflect.DeepEqual(d.WalkMock.Names, []string{"bar"}) { - t.Errorf("got names %v wanted []{bar}", d.WalkMock.Names) - } - if len(qids) != 2 || qids[len(qids)-1].Type != 1 { - t.Errorf("got qids %v wanted []{..., {Type: 1}}", qids) - } - return err + name: "readlink", + match: func(mode p9.FileMode) bool { return mode.IsSymlink() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Readlink().Do(func() { + callback() + }) + f.Readlink() }, }, { - name: "walkgetattr-to-dir", - fn: func(c *p9.Client) error { - sf.WalkGetAttrMock.Called = false - sf.WalkGetAttrMock.Names = nil - sf.WalkGetAttrMock.File = d - sf.WalkGetAttrMock.Err = nil - sf.WalkGetAttrMock.QIDs = []p9.QID{{Type: 1}} - sf.WalkGetAttrMock.Attr = p9.Attr{Mode: p9.ModeDirectory, UID: 1} - sf.WalkGetAttrMock.Valid = p9.AttrMask{Mode: true} - // See above. - d.WalkGetAttrMock.Called = false - d.WalkGetAttrMock.Names = nil - d.WalkGetAttrMock.File = d // Walk to self. - d.WalkGetAttrMock.Err = nil - d.WalkGetAttrMock.QIDs = []p9.QID{{Type: 1}} - d.WalkGetAttrMock.Attr = p9.Attr{Mode: p9.ModeDirectory, UID: 1} - d.WalkGetAttrMock.Valid = p9.AttrMask{Mode: true} - var qids []p9.QID - var err error - var mask p9.AttrMask - var attr p9.Attr - qids, _, mask, attr, err = sfFile.WalkGetAttr([]string{"foo", "bar"}) - if !sf.WalkGetAttrMock.Called { - t.Errorf("Walk never Called?") - } - if !reflect.DeepEqual(sf.WalkGetAttrMock.Names, []string{"foo"}) { - t.Errorf("got names %v wanted []{foo}", sf.WalkGetAttrMock.Names) - } - if !reflect.DeepEqual(d.WalkGetAttrMock.Names, []string{"bar"}) { - t.Errorf("got names %v wanted []{bar}", d.WalkGetAttrMock.Names) - } - if len(qids) != 2 || qids[len(qids)-1].Type != 1 { - t.Errorf("got qids %v wanted []{..., {Type: 1}}", qids) - } - if !reflect.DeepEqual(attr, sf.WalkGetAttrMock.Attr) { - t.Errorf("got attrs %s wanted %s", attr, sf.WalkGetAttrMock.Attr) - } - if !reflect.DeepEqual(mask, sf.WalkGetAttrMock.Valid) { - t.Errorf("got mask %s wanted %s", mask, sf.WalkGetAttrMock.Valid) - } - return err + name: "connect", + match: func(mode p9.FileMode) bool { return mode.IsSocket() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Connect(gomock.Any()).Do(func(p9.ConnectFlags) { + callback() + }) + f.Connect(0) }, }, { - name: "walk-to-file", - fn: func(c *p9.Client) error { - // Basic sanity check is done in walk-to-dir. - // - // Here we just create basic file FIDs to use. - sf.WalkMock.File = f - sf.WalkMock.Err = nil - var err error - _, _, err = sfFile.Walk(nil) - return err + name: "open", + match: func(mode p9.FileMode) bool { return p9.CanOpen(mode) }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Open(gomock.Any()).Do(func(p9.OpenFlags) { + callback() + }) + f.Open(p9.ReadOnly) }, }, { - name: "walk-to-symlink", - fn: func(c *p9.Client) error { - // See note in walk-to-file. - sf.WalkMock.File = s - sf.WalkMock.Err = nil - var err error - _, _, err = sfFile.Walk(nil) - return err + name: "flush", + match: func(mode p9.FileMode) bool { return true }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Flush().Do(func() { + callback() + }) + f.Flush() + }, + }, + } + writeExclusive := []concurrentFn{ + { + // N.B. We can't really check getattr. But this is an + // extremely low-risk function, it seems likely that + // this check is paranoid anyways. + name: "setattr", + match: func(mode p9.FileMode) bool { return true }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().SetAttr(gomock.Any(), gomock.Any()).Do(func(p9.SetAttrMask, p9.SetAttr) { + callback() + }) + f.SetAttr(p9.SetAttrMask{}, p9.SetAttr{}) }, }, { - name: "bad-statfs", - want: sentinelErr, - fn: func(c *p9.Client) error { - sf.StatFSMock.Err = sentinelErr - _, err := sfFile.StatFS() - return err + name: "unlinkAt", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().UnlinkAt(gomock.Any(), gomock.Any()).Do(func(string, uint32) { + callback() + }) + f.UnlinkAt(randomFileName(), 0) }, }, { - name: "statfs", - fn: func(c *p9.Client) error { - sf.StatFSMock.Called = false - sf.StatFSMock.Stat = p9.FSStat{Type: 1} - sf.StatFSMock.Err = nil - stat, err := sfFile.StatFS() - if !sf.StatFSMock.Called { - t.Errorf("StatfS never Called?") - } - if stat.Type != 1 { - t.Errorf("got stat %v wanted {Type: 1}", stat) - } - return err + name: "mknod", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Mknod(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, p9.FileMode, uint32, uint32, p9.UID, p9.GID) { + callback() + }) + f.Mknod(randomFileName(), 0, 0, 0, 0, 0) + }, + }, + { + name: "link", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Link(gomock.Any(), gomock.Any()).Do(func(p9.File, string) { + callback() + }) + f.Link(f, randomFileName()) + }, + }, + { + name: "symlink", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Symlink(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, string, p9.UID, p9.GID) { + callback() + }) + f.Symlink(randomFileName(), randomFileName(), 0, 0) + }, + }, + { + name: "mkdir", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().Mkdir(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, p9.FileMode, p9.UID, p9.GID) { + callback() + }) + f.Mkdir(randomFileName(), 0, 0, 0) + }, + }, + { + name: "create", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + // Return an error for the creation operation, as this is the simplest. + backend.EXPECT().Create(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil, p9.QID{}, uint32(0), syscall.EINVAL).Do(func(string, p9.OpenFlags, p9.FileMode, p9.UID, p9.GID) { + callback() + }) + f.Create(randomFileName(), p9.ReadOnly, 0, 0, 0) }, }, } + globalExclusive := []concurrentFn{ + { + name: "remove", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + // Remove operates on a locked parent. So we + // add a child, walk to it and call remove. + // Note that because this operation can operate + // concurrently with itself, we need to + // generate a random file name. + randomFile := randomFileName() + backend.AddChild(randomFile, h.NewFile()) + defer backend.RemoveChild(randomFile) + _, file, err := f.Walk([]string{randomFile}) + if err != nil { + h.t.Fatalf("walk got %v, want nil", err) + } - // First, create a new server and connection. - serverSocket, clientSocket, err := unet.SocketPair(false) - if err != nil { - t.Fatalf("socketpair got err %v wanted nil", err) - } - defer clientSocket.Close() - server := p9.NewServer(a) - go server.Handle(serverSocket) - client, err := p9.NewClient(clientSocket, 1024*1024 /* 1M message size */, p9.HighestVersionString()) - if err != nil { - t.Fatalf("new client got err %v, wanted nil", err) - } + // Remove is automatically translated to the parent. + backend.EXPECT().UnlinkAt(gomock.Any(), gomock.Any()).Do(func(string, uint32) { + callback() + }) - // Now, run through each of the test steps. - for _, step := range testSteps { - err := step.fn(client) - if err != step.want { - // Don't fail, just note this one step failed. - t.Errorf("step %q got %v wanted %v", step.name, err, step.want) - } - } -} + // Remove is also a close. + file.(deprecatedRemover).Remove() + }, + }, + { + name: "rename", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + // Similarly to remove, because we need to + // operate on a child, we allow a walk. + randomFile := randomFileName() + backend.AddChild(randomFile, h.NewFile()) + defer backend.RemoveChild(randomFile) + _, file, err := f.Walk([]string{randomFile}) + if err != nil { + h.t.Fatalf("walk got %v, want nil", err) + } + defer file.Close() + fileBackend := h.Pop(file) -func BenchmarkClient(b *testing.B) { - // Backend mock. - a := &AttachMock{ - File: &FileMock{ - ReadAtMock: ReadAtMock{N: 1}, + // Rename is automatically translated to the parent. + backend.EXPECT().RenameAt(gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, p9.File, string) { + callback() + }) + + // Attempt the rename. + fileBackend.EXPECT().Renamed(gomock.Any(), gomock.Any()) + file.Rename(f, randomFileName()) + }, }, - } + { + name: "renameAt", + match: func(mode p9.FileMode) bool { return mode.IsDir() }, + op: func(h *Harness, backend *Mock, f p9.File, callback func()) { + backend.EXPECT().RenameAt(gomock.Any(), gomock.Any(), gomock.Any()).Do(func(string, p9.File, string) { + callback() + }) - // First, create a new server and connection. - serverSocket, clientSocket, err := unet.SocketPair(false) - if err != nil { - b.Fatalf("socketpair got err %v wanted nil", err) - } - defer clientSocket.Close() - server := p9.NewServer(a) - go server.Handle(serverSocket) - client, err := p9.NewClient(clientSocket, 1024*1024 /* 1M message size */, p9.HighestVersionString()) - if err != nil { - b.Fatalf("new client got %v, expected nil", err) + // Attempt the rename. There are no active fids + // with this name, so we don't need to expect + // Renamed hooks on anything. + f.RenameAt(randomFileName(), f, randomFileName()) + }, + }, } - // Attach to the server. - f, err := client.Attach("") - if err != nil { - b.Fatalf("error during attach, got %v wanted nil", err) + for _, fn1 := range readExclusive { + for _, fn2 := range readExclusive { + for name := range newTypeMap(nil) { + // Everything should be able to proceed in parallel. + concurrentTest(t, name, fn1, fn2, true, true) + concurrentTest(t, name, fn1, fn2, false, true) + } + } } - // Open the file. - if _, _, _, err := f.Open(p9.ReadOnly); err != nil { - b.Fatalf("error during open, got %v wanted nil", err) + for _, fn1 := range append(readExclusive, writeExclusive...) { + for _, fn2 := range writeExclusive { + for name := range newTypeMap(nil) { + // Only cross-directory functions should proceed in parallel. + concurrentTest(t, name, fn1, fn2, true, false) + concurrentTest(t, name, fn1, fn2, false, true) + } + } } - // Reset the clock. - b.ResetTimer() - - // Do N reads. - var buf [1]byte - for i := 0; i < b.N; i++ { - _, err := f.ReadAt(buf[:], 0) - if err != nil { - b.Fatalf("error during read %d, got %v wanted nil", i, err) + for _, fn1 := range append(append(readExclusive, writeExclusive...), globalExclusive...) { + for _, fn2 := range globalExclusive { + for name := range newTypeMap(nil) { + // Nothing should be able to run in parallel. + concurrentTest(t, name, fn1, fn2, true, false) + concurrentTest(t, name, fn1, fn2, false, false) + } } } } diff --git a/pkg/p9/p9test/mocks.go b/pkg/p9/p9test/mocks.go deleted file mode 100644 index 9a8c14975..000000000 --- a/pkg/p9/p9test/mocks.go +++ /dev/null @@ -1,489 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package p9test - -import ( - "gvisor.googlesource.com/gvisor/pkg/fd" - "gvisor.googlesource.com/gvisor/pkg/p9" -) - -// StatFSMock mocks p9.File.StatFS. -type StatFSMock struct { - Called bool - - // Return. - Stat p9.FSStat - Err error -} - -// StatFS implements p9.File.StatFS. -func (f *StatFSMock) StatFS() (p9.FSStat, error) { - f.Called = true - return f.Stat, f.Err -} - -// GetAttrMock mocks p9.File.GetAttr. -type GetAttrMock struct { - Called bool - - // Args. - Req p9.AttrMask - - // Return. - QID p9.QID - Valid p9.AttrMask - Attr p9.Attr - Err error -} - -// GetAttr implements p9.File.GetAttr. -func (g *GetAttrMock) GetAttr(req p9.AttrMask) (p9.QID, p9.AttrMask, p9.Attr, error) { - g.Called, g.Req = true, req - return g.QID, g.Valid, g.Attr, g.Err -} - -// WalkGetAttrMock mocks p9.File.WalkGetAttr. -type WalkGetAttrMock struct { - Called bool - - // Args. - Names []string - - // Return. - QIDs []p9.QID - File p9.File - Valid p9.AttrMask - Attr p9.Attr - Err error -} - -// WalkGetAttr implements p9.File.WalkGetAttr. -func (w *WalkGetAttrMock) WalkGetAttr(names []string) ([]p9.QID, p9.File, p9.AttrMask, p9.Attr, error) { - w.Called = true - w.Names = append(w.Names, names...) - return w.QIDs, w.File, w.Valid, w.Attr, w.Err -} - -// SetAttrMock mocks p9.File.SetAttr. -type SetAttrMock struct { - Called bool - - // Args. - Valid p9.SetAttrMask - Attr p9.SetAttr - - // Return. - Err error -} - -// SetAttr implements p9.File.SetAttr. -func (s *SetAttrMock) SetAttr(valid p9.SetAttrMask, attr p9.SetAttr) error { - s.Called, s.Valid, s.Attr = true, valid, attr - return s.Err -} - -// RemoveMock mocks p9.File.Remove. -type RemoveMock struct { - Called bool - - // Return. - Err error -} - -// Remove implements p9.File.Remove. -func (r *RemoveMock) Remove() error { - r.Called = true - return r.Err -} - -// OpenMock mocks p9.File.Open. -type OpenMock struct { - Called bool - - // Args. - Flags p9.OpenFlags - - // Return. - File *fd.FD - QID p9.QID - IOUnit uint32 - Err error -} - -// Open implements p9.File.Open. -func (o *OpenMock) Open(flags p9.OpenFlags) (*fd.FD, p9.QID, uint32, error) { - o.Called, o.Flags = true, flags - return o.File, o.QID, o.IOUnit, o.Err -} - -// ReadAtMock mocks p9.File.ReadAt. -type ReadAtMock struct { - Called bool - - // Args. - P []byte - Offset uint64 - - // Return. - N int - Err error -} - -// ReadAt implements p9.File.ReadAt. -func (r *ReadAtMock) ReadAt(p []byte, offset uint64) (int, error) { - r.Called, r.P, r.Offset = true, p, offset - return r.N, r.Err -} - -// WriteAtMock mocks p9.File.WriteAt. -type WriteAtMock struct { - Called bool - - // Args. - P []byte - Offset uint64 - - // Return. - N int - Err error -} - -// WriteAt implements p9.File.WriteAt. -func (w *WriteAtMock) WriteAt(p []byte, offset uint64) (int, error) { - w.Called, w.P, w.Offset = true, p, offset - return w.N, w.Err -} - -// FSyncMock mocks p9.File.FSync. -type FSyncMock struct { - Called bool - - // Return. - Err error -} - -// FSync implements p9.File.FSync. -func (f *FSyncMock) FSync() error { - f.Called = true - return f.Err -} - -// MkdirMock mocks p9.File.Mkdir. -type MkdirMock struct { - Called bool - - // Args. - Name string - Permissions p9.FileMode - UID p9.UID - GID p9.GID - - // Return. - QID p9.QID - Err error -} - -// Mkdir implements p9.File.Mkdir. -func (s *MkdirMock) Mkdir(name string, permissions p9.FileMode, uid p9.UID, gid p9.GID) (p9.QID, error) { - s.Called, s.Name, s.Permissions, s.UID, s.GID = true, name, permissions, uid, gid - return s.QID, s.Err -} - -// SymlinkMock mocks p9.File.Symlink. -type SymlinkMock struct { - Called bool - - // Args. - Oldname string - Newname string - UID p9.UID - GID p9.GID - - // Return. - QID p9.QID - Err error -} - -// Symlink implements p9.File.Symlink. -func (s *SymlinkMock) Symlink(oldname string, newname string, uid p9.UID, gid p9.GID) (p9.QID, error) { - s.Called, s.Oldname, s.Newname, s.UID, s.GID = true, oldname, newname, uid, gid - return s.QID, s.Err -} - -// MknodMock mocks p9.File.Mknod. -type MknodMock struct { - Called bool - - // Args. - Name string - Permissions p9.FileMode - Major uint32 - Minor uint32 - UID p9.UID - GID p9.GID - - // Return. - QID p9.QID - Err error -} - -// Mknod implements p9.File.Mknod. -func (m *MknodMock) Mknod(name string, permissions p9.FileMode, major uint32, minor uint32, uid p9.UID, gid p9.GID) (p9.QID, error) { - m.Called, m.Name, m.Permissions, m.Major, m.Minor, m.UID, m.GID = true, name, permissions, major, minor, uid, gid - return m.QID, m.Err -} - -// UnlinkAtMock mocks p9.File.UnlinkAt. -type UnlinkAtMock struct { - Called bool - - // Args. - Name string - Flags uint32 - - // Return. - Err error -} - -// UnlinkAt implements p9.File.UnlinkAt. -func (u *UnlinkAtMock) UnlinkAt(name string, flags uint32) error { - u.Called, u.Name, u.Flags = true, name, flags - return u.Err -} - -// ReaddirMock mocks p9.File.Readdir. -type ReaddirMock struct { - Called bool - - // Args. - Offset uint64 - Count uint32 - - // Return. - Dirents []p9.Dirent - Err error -} - -// Readdir implements p9.File.Readdir. -func (r *ReaddirMock) Readdir(offset uint64, count uint32) ([]p9.Dirent, error) { - r.Called, r.Offset, r.Count = true, offset, count - return r.Dirents, r.Err -} - -// ReadlinkMock mocks p9.File.Readlink. -type ReadlinkMock struct { - Called bool - - // Return. - Target string - Err error -} - -// Readlink implements p9.File.Readlink. -func (r *ReadlinkMock) Readlink() (string, error) { - r.Called = true - return r.Target, r.Err -} - -// AttachMock mocks p9.Attacher.Attach. -type AttachMock struct { - Called bool - - // Return. - File p9.File - Err error -} - -// Attach implements p9.Attacher.Attach. -func (a *AttachMock) Attach() (p9.File, error) { - a.Called = true - return a.File, a.Err -} - -// WalkMock mocks p9.File.Walk. -type WalkMock struct { - Called bool - - // Args. - Names []string - - // Return. - QIDs []p9.QID - File p9.File - Err error -} - -// Walk implements p9.File.Walk. -func (w *WalkMock) Walk(names []string) ([]p9.QID, p9.File, error) { - w.Called = true - w.Names = append(w.Names, names...) - return w.QIDs, w.File, w.Err -} - -// RenameMock mocks p9.File.Rename. -type RenameMock struct { - Called bool - - // Args. - Directory p9.File - Name string - - // Return. - Err error -} - -// Rename implements p9.File.Rename. -func (r *RenameMock) Rename(directory p9.File, name string) error { - r.Called, r.Directory, r.Name = true, directory, name - return r.Err -} - -// CloseMock mocks p9.File.Close. -type CloseMock struct { - Called bool - - // Return. - Err error -} - -// Close implements p9.File.Close. -func (d *CloseMock) Close() error { - d.Called = true - return d.Err -} - -// CreateMock mocks p9.File.Create. -type CreateMock struct { - Called bool - - // Args. - Name string - Flags p9.OpenFlags - Permissions p9.FileMode - UID p9.UID - GID p9.GID - - // Return. - HostFile *fd.FD - File p9.File - QID p9.QID - IOUnit uint32 - Err error -} - -// Create implements p9.File.Create. -func (c *CreateMock) Create(name string, flags p9.OpenFlags, permissions p9.FileMode, uid p9.UID, gid p9.GID) (*fd.FD, p9.File, p9.QID, uint32, error) { - c.Called, c.Name, c.Flags, c.Permissions, c.UID, c.GID = true, name, flags, permissions, uid, gid - return c.HostFile, c.File, c.QID, c.IOUnit, c.Err -} - -// LinkMock mocks p9.File.Link. -type LinkMock struct { - Called bool - - // Args. - Target p9.File - Newname string - - // Return. - Err error -} - -// Link implements p9.File.Link. -func (l *LinkMock) Link(target p9.File, newname string) error { - l.Called, l.Target, l.Newname = true, target, newname - return l.Err -} - -// RenameAtMock mocks p9.File.RenameAt. -type RenameAtMock struct { - Called bool - - // Args. - Oldname string - Newdir p9.File - Newname string - - // Return. - Err error -} - -// RenameAt implements p9.File.RenameAt. -func (r *RenameAtMock) RenameAt(oldname string, newdir p9.File, newname string) error { - r.Called, r.Oldname, r.Newdir, r.Newname = true, oldname, newdir, newname - return r.Err -} - -// FlushMock mocks p9.File.Flush. -type FlushMock struct { - Called bool - - // Return. - Err error -} - -// Flush implements p9.File.Flush. -func (f *FlushMock) Flush() error { - return f.Err -} - -// ConnectMock mocks p9.File.Connect. -type ConnectMock struct { - Called bool - - // Args. - Flags p9.ConnectFlags - - // Return. - File *fd.FD - Err error -} - -// Connect implements p9.File.Connect. -func (o *ConnectMock) Connect(flags p9.ConnectFlags) (*fd.FD, error) { - o.Called, o.Flags = true, flags - return o.File, o.Err -} - -// FileMock mocks p9.File. -type FileMock struct { - WalkMock - WalkGetAttrMock - StatFSMock - GetAttrMock - SetAttrMock - RemoveMock - RenameMock - CloseMock - OpenMock - ReadAtMock - WriteAtMock - FSyncMock - CreateMock - MkdirMock - SymlinkMock - LinkMock - MknodMock - RenameAtMock - UnlinkAtMock - ReaddirMock - ReadlinkMock - FlushMock - ConnectMock -} - -var ( - _ p9.File = &FileMock{} -) diff --git a/pkg/p9/p9test/p9test.go b/pkg/p9/p9test/p9test.go new file mode 100644 index 000000000..417b55950 --- /dev/null +++ b/pkg/p9/p9test/p9test.go @@ -0,0 +1,329 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package p9test provides standard mocks for p9. +package p9test + +import ( + "fmt" + "sync" + "sync/atomic" + "syscall" + "testing" + + "github.com/golang/mock/gomock" + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/unet" +) + +// Harness is an attacher mock. +type Harness struct { + t *testing.T + mockCtrl *gomock.Controller + Attacher *MockAttacher + wg sync.WaitGroup + clientSocket *unet.Socket + mu sync.Mutex + created []*Mock +} + +// globalPath is a QID.Path Generator. +var globalPath uint64 + +// MakePath returns a globally unique path. +func MakePath() uint64 { + return atomic.AddUint64(&globalPath, 1) +} + +// Generator is a function that generates a new file. +type Generator func(parent *Mock) *Mock + +// Mock is a common mock element. +type Mock struct { + p9.DefaultWalkGetAttr + *MockFile + parent *Mock + closed bool + harness *Harness + QID p9.QID + Attr p9.Attr + children map[string]Generator + + // WalkCallback is a special function that will be called from within + // the walk context. This is needed for the concurrent tests within + // this package. + WalkCallback func() error +} + +// globalMu protects the children maps in all mocks. Note that this is not a +// particularly elegant solution, but because the test has walks from the root +// through to final nodes, we must share maps below, and it's easiest to simply +// protect against concurrent access globally. +var globalMu sync.RWMutex + +// AddChild adds a new child to the Mock. +func (m *Mock) AddChild(name string, generator Generator) { + globalMu.Lock() + defer globalMu.Unlock() + m.children[name] = generator +} + +// RemoveChild removes the child with the given name. +func (m *Mock) RemoveChild(name string) { + globalMu.Lock() + defer globalMu.Unlock() + delete(m.children, name) +} + +// Matches implements gomock.Matcher.Matches. +func (m *Mock) Matches(x interface{}) bool { + if om, ok := x.(*Mock); ok { + return m.QID.Path == om.QID.Path + } + return false +} + +// String implements gomock.Matcher.String. +func (m *Mock) String() string { + return fmt.Sprintf("Mock{Mode: 0x%x, QID.Path: %d}", m.Attr.Mode, m.QID.Path) +} + +// GetAttr returns the current attributes. +func (m *Mock) GetAttr(mask p9.AttrMask) (p9.QID, p9.AttrMask, p9.Attr, error) { + return m.QID, p9.AttrMaskAll(), m.Attr, nil +} + +// Walk supports clone and walking in directories. +func (m *Mock) Walk(names []string) ([]p9.QID, p9.File, error) { + if m.WalkCallback != nil { + if err := m.WalkCallback(); err != nil { + return nil, nil, err + } + } + if len(names) == 0 { + // Clone the file appropriately. + nm := m.harness.NewMock(m.parent, m.QID.Path, m.Attr) + nm.children = m.children // Inherit children. + return []p9.QID{nm.QID}, nm, nil + } else if len(names) != 1 { + m.harness.t.Fail() // Should not happen. + return nil, nil, syscall.EINVAL + } + + if m.Attr.Mode.IsDir() { + globalMu.RLock() + defer globalMu.RUnlock() + if fn, ok := m.children[names[0]]; ok { + // Generate the child. + nm := fn(m) + return []p9.QID{nm.QID}, nm, nil + } + // No child found. + return nil, nil, syscall.ENOENT + } + + // Call the underlying mock. + return m.MockFile.Walk(names) +} + +// WalkGetAttr calls the default implementation; this is a client-side optimization. +func (m *Mock) WalkGetAttr(names []string) ([]p9.QID, p9.File, p9.AttrMask, p9.Attr, error) { + return m.DefaultWalkGetAttr.WalkGetAttr(names) +} + +// Pop pops off the most recently created Mock and assert that this mock +// represents the same file passed in. If nil is passed in, no check is +// performed. +// +// Precondition: there must be at least one Mock or this will panic. +func (h *Harness) Pop(clientFile p9.File) *Mock { + h.mu.Lock() + defer h.mu.Unlock() + + if clientFile == nil { + // If no clientFile is provided, then we always return the last + // created file. The caller can safely use this as long as + // there is no concurrency. + m := h.created[len(h.created)-1] + h.created = h.created[:len(h.created)-1] + return m + } + + qid, _, _, err := clientFile.GetAttr(p9.AttrMaskAll()) + if err != nil { + // We do not expect this to happen. + panic(fmt.Sprintf("err during Pop: %v", err)) + } + + // Find the relevant file in our created list. We must scan the last + // from back to front to ensure that we favor the most recently + // generated file. + for i := len(h.created) - 1; i >= 0; i-- { + m := h.created[i] + if qid.Path == m.QID.Path { + // Copy and truncate. + copy(h.created[i:], h.created[i+1:]) + h.created = h.created[:len(h.created)-1] + return m + } + } + + // Unable to find relevant file. + panic(fmt.Sprintf("unable to locate file with QID %+v", qid.Path)) +} + +// NewMock returns a new base file. +func (h *Harness) NewMock(parent *Mock, path uint64, attr p9.Attr) *Mock { + m := &Mock{ + MockFile: NewMockFile(h.mockCtrl), + parent: parent, + harness: h, + QID: p9.QID{ + Type: p9.QIDType((attr.Mode & p9.FileModeMask) >> 12), + Path: path, + }, + Attr: attr, + } + + // Always ensure Close is after the parent's close. Note that this + // can't be done via a straight-forward After call, because the parent + // might change after initial creation. We ensure that this is true at + // close time. + m.EXPECT().Close().Return(nil).Times(1).Do(func() { + if m.parent != nil && m.parent.closed { + h.t.FailNow() + } + // Note that this should not be racy, as this operation should + // be protected by the Times(1) above first. + m.closed = true + }) + + // Remember what was created. + h.mu.Lock() + defer h.mu.Unlock() + h.created = append(h.created, m) + + return m +} + +// NewFile returns a new file mock. +// +// Note that ReadAt and WriteAt must be mocked separately. +func (h *Harness) NewFile() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeRegular}) + } +} + +// NewDirectory returns a new mock directory. +// +// Note that Mkdir, Link, Mknod, RenameAt, UnlinkAt and Readdir must be mocked +// separately. Walk is provided and children may be manipulated via AddChild +// and RemoveChild. After calling Walk remotely, one can use Pop to find the +// corresponding backend mock on the server side. +func (h *Harness) NewDirectory(contents map[string]Generator) Generator { + return func(parent *Mock) *Mock { + m := h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeDirectory}) + m.children = contents // Save contents. + return m + } +} + +// NewSymlink returns a new mock directory. +// +// Note that Readlink must be mocked separately. +func (h *Harness) NewSymlink() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeSymlink}) + } +} + +// NewBlockDevice returns a new mock block device. +func (h *Harness) NewBlockDevice() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeBlockDevice}) + } +} + +// NewCharacterDevice returns a new mock character device. +func (h *Harness) NewCharacterDevice() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeCharacterDevice}) + } +} + +// NewNamedPipe returns a new mock named pipe. +func (h *Harness) NewNamedPipe() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeNamedPipe}) + } +} + +// NewSocket returns a new mock socket. +func (h *Harness) NewSocket() Generator { + return func(parent *Mock) *Mock { + return h.NewMock(parent, MakePath(), p9.Attr{Mode: p9.ModeSocket}) + } +} + +// Finish completes all checks and shuts down the server. +func (h *Harness) Finish() { + h.clientSocket.Close() + h.wg.Wait() + h.mockCtrl.Finish() +} + +// NewHarness creates and returns a new test server. +// +// It should always be used as: +// +// h, c := NewHarness(t) +// defer h.Finish() +// +func NewHarness(t *testing.T) (*Harness, *p9.Client) { + // Create the mock. + mockCtrl := gomock.NewController(t) + h := &Harness{ + t: t, + mockCtrl: mockCtrl, + Attacher: NewMockAttacher(mockCtrl), + } + + // Make socket pair. + serverSocket, clientSocket, err := unet.SocketPair(false) + if err != nil { + t.Fatalf("socketpair got err %v wanted nil", err) + } + + // Start the server, synchronized on exit. + server := p9.NewServer(h.Attacher) + h.wg.Add(1) + go func() { + defer h.wg.Done() + server.Handle(serverSocket) + }() + + // Create the client. + client, err := p9.NewClient(clientSocket, 1024, p9.HighestVersionString()) + if err != nil { + serverSocket.Close() + clientSocket.Close() + t.Fatalf("new client got %v, expected nil", err) + return nil, nil // Never hit. + } + + // Capture the client socket. + h.clientSocket = clientSocket + return h, client +} diff --git a/pkg/p9/path_tree.go b/pkg/p9/path_tree.go new file mode 100644 index 000000000..97f90bcd5 --- /dev/null +++ b/pkg/p9/path_tree.go @@ -0,0 +1,109 @@ +// Copyright 2018 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package p9 + +import ( + "fmt" + "sync" +) + +// pathNode is a single node in a path traversal. +// +// These are shared by all fidRefs that point to the same path. +// +// These are not synchronized because we allow certain operations (file walk) +// to proceed without having to acquire a write lock. The lock in this +// structure exists to synchronize high-level, semantic operations, such as the +// simultaneous creation and deletion of a file. +// +// (+) below is the path component string. +type pathNode struct { + mu sync.RWMutex // See above. + fidRefs sync.Map // => map[*fidRef]string(+) + children sync.Map // => map[string(+)]*pathNode + count int64 +} + +// pathNodeFor returns the path node for the given name, or a new one. +// +// Precondition: mu must be held in a readable fashion. +func (p *pathNode) pathNodeFor(name string) *pathNode { + // Load the existing path node. + if pn, ok := p.children.Load(name); ok { + return pn.(*pathNode) + } + + // Create a new pathNode for shared use. + pn, _ := p.children.LoadOrStore(name, new(pathNode)) + return pn.(*pathNode) +} + +// nameFor returns the name for the given fidRef. +// +// Precondition: mu must be held in a readable fashion. +func (p *pathNode) nameFor(ref *fidRef) string { + if s, ok := p.fidRefs.Load(ref); ok { + return s.(string) + } + + // This should not happen, don't proceed. + panic(fmt.Sprintf("expected name for %+v, none found", ref)) +} + +// addChild adds a child to the given pathNode. +// +// This applies only to an individual fidRef. +// +// Precondition: mu must be held in a writable fashion. +func (p *pathNode) addChild(ref *fidRef, name string) { + if s, ok := p.fidRefs.Load(ref); ok { + // This should not happen, don't proceed. + panic(fmt.Sprintf("unexpected fidRef %+v with path %q, wanted %q", ref, s, name)) + } + + p.fidRefs.Store(ref, name) +} + +// removeChild removes the given child. +// +// This applies only to an individual fidRef. +// +// Precondition: mu must be held in a writable fashion. +func (p *pathNode) removeChild(ref *fidRef) { + p.fidRefs.Delete(ref) +} + +// removeWithName removes all references with the given name. +// +// The original pathNode is returned by this function, and removed from this +// pathNode. Any operations on the removed tree must use this value. +// +// The provided function is executed after removal. +// +// Precondition: mu must be held in a writable fashion. +func (p *pathNode) removeWithName(name string, fn func(ref *fidRef)) *pathNode { + p.fidRefs.Range(func(key, value interface{}) bool { + if value.(string) == name { + p.fidRefs.Delete(key) + fn(key.(*fidRef)) + } + return true + }) + + // Return the original path node. + origPathNode := p.pathNodeFor(name) + p.children.Delete(name) + return origPathNode +} diff --git a/pkg/p9/server.go b/pkg/p9/server.go index 5c7cb18c8..3ef151595 100644 --- a/pkg/p9/server.go +++ b/pkg/p9/server.go @@ -15,6 +15,8 @@ package p9 import ( + "io" + "runtime/debug" "sync" "sync/atomic" "syscall" @@ -27,6 +29,19 @@ import ( type Server struct { // attacher provides the attach function. attacher Attacher + + // pathTree is the full set of paths opened on this server. + // + // These may be across different connections, but rename operations + // must be serialized globally for safely. There is a single pathTree + // for the entire server, and not per connection. + pathTree pathNode + + // renameMu is a global lock protecting rename operations. With this + // lock, we can be certain that any given rename operation can safely + // acquire two path nodes in any order, as all other concurrent + // operations acquire at most a single node. + renameMu sync.RWMutex } // NewServer returns a new server. @@ -81,6 +96,9 @@ type connState struct { // fidRef wraps a node and tracks references. type fidRef struct { + // server is the associated server. + server *Server + // file is the associated File. file File @@ -97,13 +115,39 @@ type fidRef struct { // This is updated in handlers.go. opened bool - // walkable indicates this fidRef may be walked. - walkable bool + // mode is the fidRef's mode from the walk. Only the type bits are + // valid, the permissions may change. This is used to sanity check + // operations on this element, and prevent walks across + // non-directories. + mode FileMode // openFlags is the mode used in the open. // // This is updated in handlers.go. openFlags OpenFlags + + // pathNode is the current pathNode for this FID. + pathNode *pathNode + + // parent is the parent fidRef. We hold on to a parent reference to + // ensure that hooks, such as Renamed, can be executed safely by the + // server code. + // + // Note that parent cannot be changed without holding both the global + // rename lock and a writable lock on the associated pathNode for this + // fidRef. Holding either of these locks is sufficient to examine + // parent safely. + // + // The parent will be nil for root fidRefs, and non-nil otherwise. The + // method maybeParent can be used to return a cyclical reference, and + // isRoot should be used to check for root over looking at parent + // directly. + parent *fidRef + + // deleted indicates that the backing file has been deleted. We stop + // many operations at the API level if they are incompatible with a + // file that has already been unlinked. + deleted uint32 } // OpenFlags returns the flags the file was opened with and true iff the fid was opened previously. @@ -113,13 +157,146 @@ func (f *fidRef) OpenFlags() (OpenFlags, bool) { return f.openFlags, f.opened } +// IncRef increases the references on a fid. +func (f *fidRef) IncRef() { + atomic.AddInt64(&f.refs, 1) +} + // DecRef should be called when you're finished with a fid. func (f *fidRef) DecRef() { if atomic.AddInt64(&f.refs, -1) == 0 { f.file.Close() + + // Drop the parent reference. + // + // Since this fidRef is guaranteed to be non-discoverable when + // the references reach zero, we don't need to worry about + // clearing the parent. + if f.parent != nil { + // If we've been previously deleted, this removing this + // ref is a no-op. That's expected. + f.parent.pathNode.removeChild(f) + f.parent.DecRef() + } } } +// isDeleted returns true if this fidRef has been deleted. +func (f *fidRef) isDeleted() bool { + return atomic.LoadUint32(&f.deleted) != 0 +} + +// isRoot indicates whether this is a root fid. +func (f *fidRef) isRoot() bool { + return f.parent == nil +} + +// maybeParent returns a cyclic reference for roots, and the parent otherwise. +func (f *fidRef) maybeParent() *fidRef { + if f.parent != nil { + return f.parent + } + return f // Root has itself. +} + +// notifyDelete marks all fidRefs as deleted. +// +// Precondition: the write lock must be held on the given pathNode. +func notifyDelete(pn *pathNode) { + // Call on all local references. + pn.fidRefs.Range(func(key, _ interface{}) bool { + ref := key.(*fidRef) + atomic.StoreUint32(&ref.deleted, 1) + return true + }) + + // Call on all subtrees. + pn.children.Range(func(_, value interface{}) bool { + notifyDelete(value.(*pathNode)) + return true + }) +} + +// markChildDeleted marks all children below the given name as deleted. +// +// Precondition: this must be called via safelyWrite or safelyGlobal. +func (f *fidRef) markChildDeleted(name string) { + origPathNode := f.pathNode.removeWithName(name, func(ref *fidRef) { + atomic.StoreUint32(&ref.deleted, 1) + }) + + // Mark everything below as deleted. + notifyDelete(origPathNode) +} + +// notifyNameChange calls the relevant Renamed method on all nodes in the path, +// recursively. Note that this applies only for subtrees, as these +// notifications do not apply to the actual file whose name has changed. +// +// Precondition: the write lock must be held on the given pathNode. +func notifyNameChange(pn *pathNode) { + // Call on all local references. + pn.fidRefs.Range(func(key, value interface{}) bool { + ref := key.(*fidRef) + name := value.(string) + ref.file.Renamed(ref.parent.file, name) + return true + }) + + // Call on all subtrees. + pn.children.Range(func(_, value interface{}) bool { + notifyNameChange(value.(*pathNode)) + return true + }) +} + +// renameChildTo renames the given child to the target. +// +// Precondition: this must be called via safelyGlobal. +func (f *fidRef) renameChildTo(oldName string, target *fidRef, newName string) { + target.markChildDeleted(newName) + origPathNode := f.pathNode.removeWithName(oldName, func(ref *fidRef) { + ref.parent.DecRef() // Drop original reference. + ref.parent = target // Change parent. + ref.parent.IncRef() // Acquire new one. + target.pathNode.addChild(ref, newName) + ref.file.Renamed(target.file, newName) + }) + + // Replace the previous (now deleted) path node. + f.pathNode.children.Store(newName, origPathNode) + + // Call Renamed on everything above. + notifyNameChange(origPathNode) +} + +// safelyRead executes the given operation with the local path node locked. +// This implies that paths will not change during the operation. +func (f *fidRef) safelyRead(fn func() error) (err error) { + f.server.renameMu.RLock() + defer f.server.renameMu.RUnlock() + f.pathNode.mu.RLock() + defer f.pathNode.mu.RUnlock() + return fn() +} + +// safelyWrite executes the given operation with the local path node locked in +// a writable fashion. This implies some paths may change. +func (f *fidRef) safelyWrite(fn func() error) (err error) { + f.server.renameMu.RLock() + defer f.server.renameMu.RUnlock() + f.pathNode.mu.Lock() + defer f.pathNode.mu.Unlock() + return fn() +} + +// safelyGlobal executes the given operation with the global path lock held. +func (f *fidRef) safelyGlobal(fn func() error) (err error) { + f.server.renameMu.Lock() + defer f.server.renameMu.Unlock() + return fn() +} + // LookupFID finds the given FID. // // You should call fid.DecRef when you are finished using the fid. @@ -128,7 +305,7 @@ func (cs *connState) LookupFID(fid FID) (*fidRef, bool) { defer cs.fidMu.Unlock() fidRef, ok := cs.fids[fid] if ok { - atomic.AddInt64(&fidRef.refs, 1) + fidRef.IncRef() return fidRef, true } return nil, false @@ -145,7 +322,7 @@ func (cs *connState) InsertFID(fid FID, newRef *fidRef) { if ok { defer origRef.DecRef() } - atomic.AddInt64(&newRef.refs, 1) + newRef.IncRef() cs.fids[fid] = newRef } @@ -229,10 +406,9 @@ func (cs *connState) handleRequest() { cs.recvDone <- nil // Deal with other errors. - if err != nil { + if err != nil && err != io.EOF { // If it's not a connection error, but some other protocol error, // we can send a response immediately. - log.Debugf("err [%05d] %v", tag, err) cs.sendMu.Lock() err := send(cs.conn, tag, newErr(err)) cs.sendMu.Unlock() @@ -243,12 +419,38 @@ func (cs *connState) handleRequest() { // Try to start the tag. if !cs.StartTag(tag) { // Nothing we can do at this point; client is bogus. + log.Debugf("no valid tag [%05d]", tag) cs.sendDone <- ErrNoValidMessage return } // Handle the message. - var r message + var r message // r is the response. + defer func() { + if r == nil { + // Don't allow a panic to propagate. + recover() + + // Include a useful log message. + log.Warningf("panic in handler: %s", debug.Stack()) + + // 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(syscall.EFAULT) + } + + // Clear the tag before sending. That's because as soon as this + // hits the wire, the client can legally send another message + // with the same tag. + cs.ClearTag(tag) + + // Send back the result. + cs.sendMu.Lock() + err = send(cs.conn, tag, r) + cs.sendMu.Unlock() + cs.sendDone <- err + }() if handler, ok := m.(handler); ok { // Call the message handler. r = handler.handle(cs) @@ -256,18 +458,6 @@ func (cs *connState) handleRequest() { // Produce an ENOSYS error. r = newErr(syscall.ENOSYS) } - - // Clear the tag before sending. That's because as soon - // as this hits the wire, the client can legally send - // another message with the same tag. - cs.ClearTag(tag) - - // Send back the result. - cs.sendMu.Lock() - err = send(cs.conn, tag, r) - cs.sendMu.Unlock() - cs.sendDone <- err - return } func (cs *connState) handleRequests() { diff --git a/pkg/p9/transport.go b/pkg/p9/transport.go index 97396806c..bafb377de 100644 --- a/pkg/p9/transport.go +++ b/pkg/p9/transport.go @@ -167,7 +167,7 @@ func recv(s *unet.Socket, msize uint32, lookup lookupTagAndType) (Tag, message, r.EnableFDs(1) n, err := r.ReadVec([][]byte{hdr[:]}) - if err != nil { + if err != nil && (n == 0 || err != io.EOF) { r.CloseFDs() return NoTag, nil, ErrSocket{err} } @@ -189,10 +189,8 @@ func recv(s *unet.Socket, msize uint32, lookup lookupTagAndType) (Tag, message, // Continuing reading for a short header. for n < int(headerLength) { cur, err := r.ReadVec([][]byte{hdr[n:]}) - if err != nil { + if err != nil && (cur == 0 || err != io.EOF) { return NoTag, nil, ErrSocket{err} - } else if cur == 0 { - return NoTag, nil, ErrSocket{io.EOF} } n += cur } @@ -296,10 +294,8 @@ func recv(s *unet.Socket, msize uint32, lookup lookupTagAndType) (Tag, message, r := s.Reader(true) for n := 0; n < int(remaining); { cur, err := r.ReadVec(vecs) - if err != nil { + if err != nil && (cur == 0 || err != io.EOF) { return NoTag, nil, ErrSocket{err} - } else if cur == 0 { - return NoTag, nil, ErrSocket{io.EOF} } n += cur diff --git a/pkg/rand/BUILD b/pkg/rand/BUILD index 97b9ba3ff..0c9efc709 100644 --- a/pkg/rand/BUILD +++ b/pkg/rand/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "rand", srcs = [ diff --git a/pkg/seccomp/BUILD b/pkg/seccomp/BUILD index 1975d17a6..657f923ed 100644 --- a/pkg/seccomp/BUILD +++ b/pkg/seccomp/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_embed_data") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "victim", testonly = 1, diff --git a/pkg/secio/BUILD b/pkg/secio/BUILD index 0ed38c64a..29f751725 100644 --- a/pkg/secio/BUILD +++ b/pkg/secio/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "secio", srcs = [ diff --git a/pkg/sentry/arch/BUILD b/pkg/sentry/arch/BUILD index 314b3e962..9bf04360a 100644 --- a/pkg/sentry/arch/BUILD +++ b/pkg/sentry/arch/BUILD @@ -1,6 +1,7 @@ +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("//tools/go_stateify:defs.bzl", "go_library") go_library( diff --git a/pkg/sentry/context/BUILD b/pkg/sentry/context/BUILD index 2a7a6df23..02d24defd 100644 --- a/pkg/sentry/context/BUILD +++ b/pkg/sentry/context/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "context", srcs = ["context.go"], diff --git a/pkg/sentry/control/BUILD b/pkg/sentry/control/BUILD index fbdde0721..c3b682d6f 100644 --- a/pkg/sentry/control/BUILD +++ b/pkg/sentry/control/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "control", srcs = [ diff --git a/pkg/sentry/device/BUILD b/pkg/sentry/device/BUILD index 69c99b0b3..bebdb2939 100644 --- a/pkg/sentry/device/BUILD +++ b/pkg/sentry/device/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "device", srcs = ["device.go"], diff --git a/pkg/sentry/fs/anon/BUILD b/pkg/sentry/fs/anon/BUILD index ff4ab850a..4bd912e95 100644 --- a/pkg/sentry/fs/anon/BUILD +++ b/pkg/sentry/fs/anon/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "anon", srcs = [ diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD index cef01829a..c9e531e40 100644 --- a/pkg/sentry/fs/gofer/BUILD +++ b/pkg/sentry/fs/gofer/BUILD @@ -56,14 +56,10 @@ go_test( srcs = ["gofer_test.go"], embed = [":gofer"], deps = [ - "//pkg/log", "//pkg/p9", "//pkg/p9/p9test", "//pkg/sentry/context", "//pkg/sentry/context/contexttest", "//pkg/sentry/fs", - "//pkg/sentry/kernel/time", - "//pkg/sentry/usermem", - "//pkg/unet", ], ) diff --git a/pkg/sentry/fs/gofer/context_file.go b/pkg/sentry/fs/gofer/context_file.go index a0265c2aa..455953237 100644 --- a/pkg/sentry/fs/gofer/context_file.go +++ b/pkg/sentry/fs/gofer/context_file.go @@ -58,13 +58,6 @@ func (c *contextFile) setAttr(ctx context.Context, valid p9.SetAttrMask, attr p9 return c.file.SetAttr(valid, attr) } -func (c *contextFile) remove(ctx context.Context) error { - ctx.UninterruptibleSleepStart(false) - defer ctx.UninterruptibleSleepFinish(false) - - return c.file.Remove() -} - func (c *contextFile) rename(ctx context.Context, directory contextFile, name string) error { ctx.UninterruptibleSleepStart(false) defer ctx.UninterruptibleSleepFinish(false) diff --git a/pkg/sentry/fs/gofer/gofer_test.go b/pkg/sentry/fs/gofer/gofer_test.go index 3190d1e18..b450778ca 100644 --- a/pkg/sentry/fs/gofer/gofer_test.go +++ b/pkg/sentry/fs/gofer/gofer_test.go @@ -16,110 +16,102 @@ package gofer import ( "fmt" - "io" "syscall" "testing" "time" - "gvisor.googlesource.com/gvisor/pkg/log" "gvisor.googlesource.com/gvisor/pkg/p9" "gvisor.googlesource.com/gvisor/pkg/p9/p9test" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/context/contexttest" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/unet" ) -// goodMockFile returns a file that can be Walk'ed to and created. -func goodMockFile(mode p9.FileMode, size uint64) *p9test.FileMock { - return &p9test.FileMock{ - GetAttrMock: p9test.GetAttrMock{ - Attr: p9.Attr{Mode: mode, Size: size, RDev: 0}, - Valid: p9.AttrMaskAll(), - }, - } -} - -func newClosedSocket() (*unet.Socket, error) { - fd, err := syscall.Socket(syscall.AF_UNIX, syscall.SOCK_STREAM, 0) - if err != nil { - return nil, err - } - - s, err := unet.NewSocket(fd) - if err != nil { - syscall.Close(fd) - return nil, err - } - - return s, s.Close() -} - -// root returns a p9 file mock and an fs.InodeOperations created from that file. Any -// functions performed on fs.InodeOperations will use the p9 file mock. -func root(ctx context.Context, cp cachePolicy, mode p9.FileMode, size uint64) (*p9test.FileMock, *fs.Inode, error) { - sock, err := newClosedSocket() - if err != nil { - return nil, nil, err - } - - // Construct a dummy session that we can destruct. - s := &session{ - conn: sock, - mounter: fs.RootOwner, - cachePolicy: cp, - client: &p9.Client{}, - } - - rootFile := goodMockFile(mode, size) - sattr, rootInodeOperations := newInodeOperations(ctx, s, contextFile{file: rootFile}, p9.QID{}, rootFile.GetAttrMock.Valid, rootFile.GetAttrMock.Attr, false /* socket */) - m := fs.NewMountSource(s, &filesystem{}, fs.MountSourceFlags{}) - return rootFile, fs.NewInode(rootInodeOperations, m, sattr), nil +// rootTest runs a test with a p9 mock and an fs.InodeOperations created from +// the attached root directory. The root file will be closed and client +// disconnected, but additional files must be closed manually. +func rootTest(t *testing.T, name string, cp cachePolicy, fn func(context.Context, *p9test.Harness, *p9test.Mock, *fs.Inode)) { + t.Run(name, func(t *testing.T) { + h, c := p9test.NewHarness(t) + defer h.Finish() + + // Create a new root. Note that we pass an empty, but non-nil + // map here. This allows tests to extend the root children + // dynamically. + root := h.NewDirectory(map[string]p9test.Generator{})(nil) + + // Return this as the root. + h.Attacher.EXPECT().Attach().Return(root, nil).Times(1) + + // ... and open via the client. + rootFile, err := c.Attach("/") + if err != nil { + t.Fatalf("unable to attach: %v", err) + } + defer rootFile.Close() + + // Wrap an a session. + s := &session{ + mounter: fs.RootOwner, + cachePolicy: cp, + client: c, + } + + // ... and an INode, with only the mode being explicitly valid for now. + ctx := contexttest.Context(t) + sattr, rootInodeOperations := newInodeOperations(ctx, s, contextFile{ + file: rootFile, + }, root.QID, p9.AttrMaskAll(), root.Attr, false /* socket */) + m := fs.NewMountSource(s, &filesystem{}, fs.MountSourceFlags{}) + rootInode := fs.NewInode(rootInodeOperations, m, sattr) + + // Ensure that the cache is fully invalidated, so that any + // close actions actually take place before the full harness is + // torn down. + defer m.FlushDirentRefs() + + // Execute the test. + fn(ctx, h, root, rootInode) + }) } func TestLookup(t *testing.T) { - // Test parameters. type lookupTest struct { // Name of the test. name string - // Function input parameters. - fileName string - // Expected return value. want error } tests := []lookupTest{ { - name: "mock Walk passes (function succeeds)", - fileName: "ppp", - want: nil, + name: "mock Walk passes (function succeeds)", + want: nil, }, { - name: "mock Walk fails (function fails)", - fileName: "ppp", - want: syscall.ENOENT, + name: "mock Walk fails (function fails)", + want: syscall.ENOENT, }, } - ctx := contexttest.Context(t) + const file = "file" // The walked target file. + for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) + rootTest(t, test.name, cacheNone, func(ctx context.Context, h *p9test.Harness, rootFile *p9test.Mock, rootInode *fs.Inode) { + // Setup the appropriate result. + rootFile.WalkCallback = func() error { + return test.want + } + if test.want == nil { + // Set the contents of the root. We expect a + // normal file generator for ppp above. This is + // overriden by setting WalkErr in the mock. + rootFile.AddChild(file, h.NewFile()) } - - rootFile.WalkGetAttrMock.QIDs = []p9.QID{{}} - rootFile.WalkGetAttrMock.Err = test.want - rootFile.WalkGetAttrMock.File = goodMockFile(p9.PermissionsMask, 0) // Call function. - dirent, err := rootInode.Lookup(ctx, test.fileName) + dirent, err := rootInode.Lookup(ctx, file) // Unwrap the InodeOperations. var newInodeOperations fs.InodeOperations @@ -138,19 +130,12 @@ func TestLookup(t *testing.T) { if err == nil && newInodeOperations == nil { t.Errorf("Lookup got non-nil err and non-nil node, wanted at least one non-nil") } - - // Check mock parameters. - if !rootFile.WalkGetAttrMock.Called { - t.Errorf("GetAttr not called; error: %v", err) - } else if rootFile.WalkGetAttrMock.Names[0] != test.fileName { - t.Errorf("file name not set") - } }) } } func TestRevalidation(t *testing.T) { - tests := []struct { + type revalidationTest struct { cachePolicy cachePolicy // Whether dirent should be reloaded before any modifications. @@ -167,7 +152,9 @@ func TestRevalidation(t *testing.T) { // Whether dirent should be reloaded after the remote has // removed the file. postRemovalWantReload bool - }{ + } + + tests := []revalidationTest{ { // Policy cacheNone causes Revalidate to always return // true. @@ -208,67 +195,83 @@ func TestRevalidation(t *testing.T) { }, } - ctx := contexttest.Context(t) + const file = "file" // The file walked below. + for _, test := range tests { name := fmt.Sprintf("cachepolicy=%s", test.cachePolicy) - t.Run(name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, test.cachePolicy, p9.ModeDirectory|p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - + rootTest(t, name, test.cachePolicy, func(ctx context.Context, h *p9test.Harness, rootFile *p9test.Mock, rootInode *fs.Inode) { + // Wrap in a dirent object. rootDir := fs.NewDirent(rootInode, "root") - // Create a mock file that we will walk to from the root. - const ( - name = "foo" - mode = p9.PermissionsMask - ) - file := goodMockFile(mode, 0) - file.GetAttrMock.Valid = p9.AttrMaskAll() - - // Tell the root mock how to walk to this file. - rootFile.WalkGetAttrMock.QIDs = []p9.QID{{}} - rootFile.WalkGetAttrMock.File = file - rootFile.WalkGetAttrMock.Attr = file.GetAttrMock.Attr - rootFile.WalkGetAttrMock.Valid = file.GetAttrMock.Valid + // Create a mock file a child of the root. We save when + // this is generated, so that when the time changed, we + // can update the original entry. + var origMocks []*p9test.Mock + rootFile.AddChild(file, func(parent *p9test.Mock) *p9test.Mock { + // Regular a regular file that has a consistent + // path number. This might be used by + // validation so we don't change it. + m := h.NewMock(parent, 0, p9.Attr{ + Mode: p9.ModeRegular, + }) + origMocks = append(origMocks, m) + return m + }) // Do the walk. - dirent, err := rootDir.Walk(ctx, rootDir, name) + dirent, err := rootDir.Walk(ctx, rootDir, file) if err != nil { - t.Fatalf("Lookup(%q) failed: %v", name, err) + t.Fatalf("Lookup failed: %v", err) } - // Walk again. Depending on the cache policy, we may get a new - // dirent. - newDirent, err := rootDir.Walk(ctx, rootDir, name) + // We must release the dirent, of the test will fail + // with a reference leak. This is tracked by p9test. + defer dirent.DecRef() + + // Walk again. Depending on the cache policy, we may + // get a new dirent. + newDirent, err := rootDir.Walk(ctx, rootDir, file) if err != nil { - t.Fatalf("Lookup(%q) failed: %v", name, err) + t.Fatalf("Lookup failed: %v", err) } if test.preModificationWantReload && dirent == newDirent { - t.Errorf("Lookup(%q) with cachePolicy=%s got old dirent %v, wanted a new dirent", name, test.cachePolicy, dirent) + t.Errorf("Lookup with cachePolicy=%s got old dirent %+v, wanted a new dirent", test.cachePolicy, dirent) } if !test.preModificationWantReload && dirent != newDirent { - t.Errorf("Lookup(%q) with cachePolicy=%s got new dirent %v, wanted old dirent %v", name, test.cachePolicy, newDirent, dirent) + t.Errorf("Lookup with cachePolicy=%s got new dirent %+v, wanted old dirent %+v", test.cachePolicy, newDirent, dirent) } + newDirent.DecRef() // See above. - // Modify the underlying mocked file's modification time. + // Modify the underlying mocked file's modification + // time for the next walk that occurs. nowSeconds := time.Now().Unix() - rootFile.WalkGetAttrMock.Attr.MTimeSeconds = uint64(nowSeconds) - file.GetAttrMock.Attr.MTimeSeconds = uint64(nowSeconds) + rootFile.AddChild(file, func(parent *p9test.Mock) *p9test.Mock { + // Ensure that the path is the same as above, + // but we change only the modification time of + // the file. + return h.NewMock(parent, 0, p9.Attr{ + Mode: p9.ModeRegular, + MTimeSeconds: uint64(nowSeconds), + }) + }) + + // We also modify the original time, so that GetAttr + // behaves as expected for the caching case. + for _, m := range origMocks { + m.Attr.MTimeSeconds = uint64(nowSeconds) + } - // Walk again. Depending on the cache policy, we may get a new - // dirent. - newDirent, err = rootDir.Walk(ctx, rootDir, name) + // Walk again. Depending on the cache policy, we may + // get a new dirent. + newDirent, err = rootDir.Walk(ctx, rootDir, file) if err != nil { - t.Fatalf("Lookup(%q) failed: %v", name, err) + t.Fatalf("Lookup failed: %v", err) } if test.postModificationWantReload && dirent == newDirent { - t.Errorf("Lookup(%q) with cachePolicy=%s got old dirent %v, wanted a new dirent", name, test.cachePolicy, dirent) + t.Errorf("Lookup with cachePolicy=%s got old dirent, wanted a new dirent", test.cachePolicy) } if !test.postModificationWantReload && dirent != newDirent { - t.Errorf("Lookup(%q) with cachePolicy=%s got new dirent %v, wanted old dirent %v", name, test.cachePolicy, newDirent, dirent) + t.Errorf("Lookup with cachePolicy=%s got new dirent, wanted old dirent", test.cachePolicy) } uattrs, err := newDirent.Inode.UnstableAttr(ctx) if err != nil { @@ -276,660 +279,25 @@ func TestRevalidation(t *testing.T) { } gotModTimeSeconds := uattrs.ModificationTime.Seconds() if test.postModificationWantUpdatedAttrs && gotModTimeSeconds != nowSeconds { - t.Fatalf("Lookup(%q) with cachePolicy=%s got new modification time %v, wanted %v", name, test.cachePolicy, gotModTimeSeconds, nowSeconds) + t.Fatalf("Lookup with cachePolicy=%s got new modification time %v, wanted %v", test.cachePolicy, gotModTimeSeconds, nowSeconds) } + newDirent.DecRef() // See above. - // Make WalkGetAttr return ENOENT. This simulates - // removing the file from the remote fs. - rootFile.WalkGetAttrMock = p9test.WalkGetAttrMock{ - Err: syscall.ENOENT, - } + // Remove the file from the remote fs, subsequent walks + // should now fail to find anything. + rootFile.RemoveChild(file) // Walk again. Depending on the cache policy, we may // get ENOENT. - newDirent, err = rootDir.Walk(ctx, rootDir, name) + newDirent, err = rootDir.Walk(ctx, rootDir, file) if test.postRemovalWantReload && err == nil { - t.Errorf("Lookup(%q) with cachePolicy=%s got nil error, wanted ENOENT", name, test.cachePolicy) + t.Errorf("Lookup with cachePolicy=%s got nil error, wanted ENOENT", test.cachePolicy) } if !test.postRemovalWantReload && (err != nil || dirent != newDirent) { - t.Errorf("Lookup(%q) with cachePolicy=%s got new dirent %v and error %v, wanted old dirent %v and nil error", name, test.cachePolicy, newDirent, err, dirent) - } - }) - } -} - -func TestSetTimestamps(t *testing.T) { - // Test parameters. - type setTimestampsTest struct { - // Name of the test. - name string - - // Function input parameters. - ts fs.TimeSpec - } - - ctx := contexttest.Context(t) - now := ktime.NowFromContext(ctx) - tests := []setTimestampsTest{ - { - name: "mock SetAttr passes (function succeeds)", - ts: fs.TimeSpec{ - ATime: now, - MTime: now, - }, - }, - { - name: "mock SetAttr passes, times are 0 (function succeeds)", - ts: fs.TimeSpec{}, - }, - { - name: "mock SetAttr passes, times are 0 and not system time (function succeeds)", - ts: fs.TimeSpec{ - ATimeSetSystemTime: false, - MTimeSetSystemTime: false, - }, - }, - { - name: "mock SetAttr passes, times are set to system time (function succeeds)", - ts: fs.TimeSpec{ - ATimeSetSystemTime: true, - MTimeSetSystemTime: true, - }, - }, - { - name: "mock SetAttr passes, times are omitted (function succeeds)", - ts: fs.TimeSpec{ - ATimeOmit: true, - MTimeOmit: true, - }, - }, - } - - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - // Call function. - err = rootInode.SetTimestamps(ctx, nil /* Dirent */, test.ts) - - // Check return values. - if err != nil { - t.Errorf("SetTimestamps failed: got error %v, want nil", err) - } - - // Check mock parameters. - if !(test.ts.ATimeOmit && test.ts.MTimeOmit) && !rootFile.SetAttrMock.Called { - t.Errorf("TestSetTimestamps failed: SetAttr not called") - return - } - - // Check what was passed to the mock function. - attr := rootFile.SetAttrMock.Attr - atimeGiven := ktime.FromUnix(int64(attr.ATimeSeconds), int64(attr.ATimeNanoSeconds)) - if test.ts.ATimeOmit { - if rootFile.SetAttrMock.Valid.ATime { - t.Errorf("ATime got set true in mask, wanted false") - } - } else { - if got, want := rootFile.SetAttrMock.Valid.ATimeNotSystemTime, !test.ts.ATimeSetSystemTime; got != want { - t.Errorf("got ATimeNotSystemTime %v, want %v", got, want) - } - if !test.ts.ATimeSetSystemTime && !test.ts.ATime.Equal(atimeGiven) { - t.Errorf("ATime got %v, want %v", atimeGiven, test.ts.ATime) - } - } - - mtimeGiven := ktime.FromUnix(int64(attr.MTimeSeconds), int64(attr.MTimeNanoSeconds)) - if test.ts.MTimeOmit { - if rootFile.SetAttrMock.Valid.MTime { - t.Errorf("MTime got set true in mask, wanted false") - } - } else { - if got, want := rootFile.SetAttrMock.Valid.MTimeNotSystemTime, !test.ts.MTimeSetSystemTime; got != want { - t.Errorf("got MTimeNotSystemTime %v, want %v", got, want) - } - if !test.ts.MTimeSetSystemTime && !test.ts.MTime.Equal(mtimeGiven) { - t.Errorf("MTime got %v, want %v", mtimeGiven, test.ts.MTime) - } - } - }) - } -} - -func TestSetPermissions(t *testing.T) { - // Test parameters. - type setPermissionsTest struct { - // Name of the test. - name string - - // SetPermissions input parameters. - perms fs.FilePermissions - - // Error that SetAttr mock should return. - setAttrErr error - - // Expected return value. - want bool - } - - tests := []setPermissionsTest{ - { - name: "SetAttr mock succeeds (function succeeds)", - perms: fs.FilePermissions{User: fs.PermMask{Read: true, Write: true, Execute: true}}, - want: true, - setAttrErr: nil, - }, - { - name: "SetAttr mock fails (function fails)", - perms: fs.FilePermissions{User: fs.PermMask{Read: true, Write: true}}, - want: false, - setAttrErr: syscall.ENOENT, - }, - } - - ctx := contexttest.Context(t) - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, 0, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - rootFile.SetAttrMock.Err = test.setAttrErr - - ok := rootInode.SetPermissions(ctx, nil /* Dirent */, test.perms) - - // Check return value. - if ok != test.want { - t.Errorf("SetPermissions got %v, want %v", ok, test.want) - } - - // Check mock parameters. - pattr := rootFile.SetAttrMock.Attr - if !rootFile.SetAttrMock.Called { - t.Errorf("SetAttr not called") - return - } - if !rootFile.SetAttrMock.Valid.Permissions { - t.Errorf("SetAttr did not get right request (got false, expected SetAttrMask.Permissions true)") - } - if got := fs.FilePermsFromP9(pattr.Permissions); got != test.perms { - t.Errorf("SetAttr did not get right permissions -- got %v, want %v", got, test.perms) - } - }) - } -} - -func TestClose(t *testing.T) { - ctx := contexttest.Context(t) - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - // Call function. - rootInode.InodeOperations.Release(ctx) - - // Check mock parameters. - if !rootFile.CloseMock.Called { - t.Errorf("TestClose failed: Close not called") - } -} - -func TestRename(t *testing.T) { - // Test parameters. - type renameTest struct { - // Name of the test. - name string - - // Input parameters. - newParent *fs.Inode - newName string - - // Rename mock parameters. - renameErr error - renameCalled bool - - // Error want to return given the parameters. (Same as what - // we expect and tell rename to return.) - want error - } - ctx := contexttest.Context(t) - rootFile, rootInode, err := root(ctx, cacheNone, p9.PermissionsMask, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - tests := []renameTest{ - { - name: "mock Rename succeeds (function succeeds)", - newParent: rootInode, - newName: "foo2", - want: nil, - renameErr: nil, - renameCalled: true, - }, - { - name: "mock Rename fails (function fails)", - newParent: rootInode, - newName: "foo2", - want: syscall.ENOENT, - renameErr: syscall.ENOENT, - renameCalled: true, - }, - { - name: "newParent is not inodeOperations but should be (function fails)", - newParent: fs.NewMockInode(ctx, fs.NewMockMountSource(nil), fs.StableAttr{Type: fs.Directory}), - newName: "foo2", - want: syscall.EXDEV, - renameErr: nil, - renameCalled: false, - }, - } - - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - mockFile := goodMockFile(p9.PermissionsMask, 0) - rootFile.WalkGetAttrMock.QIDs = []p9.QID{{}} - rootFile.WalkGetAttrMock.File = mockFile - - dirent, err := rootInode.Lookup(ctx, "foo") - if err != nil { - t.Fatalf("root.Walk failed: %v", err) - } - mockFile.RenameMock.Err = test.renameErr - mockFile.RenameMock.Called = false - - // Use a dummy oldParent to acquire write access to that directory. - oldParent := &inodeOperations{ - readdirCache: fs.NewSortedDentryMap(nil), - } - oldInode := fs.NewInode(oldParent, fs.NewMockMountSource(nil), fs.StableAttr{Type: fs.Directory}) - - // Call function. - err = dirent.Inode.InodeOperations.Rename(ctx, oldInode, "", test.newParent, test.newName) - - // Check return value. - if err != test.want { - t.Errorf("Rename got %v, want %v", err, test.want) - } - - // Check mock parameters. - if got, want := mockFile.RenameMock.Called, test.renameCalled; got != want { - t.Errorf("renameCalled got %v want %v", got, want) - } - }) - } -} - -// This file is read from in TestPreadv. -type readAtFileFake struct { - p9test.FileMock - - // Parameters for faking ReadAt. - FileLength int - Err error - ChunkSize int - Called bool - LengthRead int -} - -func (r *readAtFileFake) ReadAt(p []byte, offset uint64) (int, error) { - r.Called = true - log.Warningf("ReadAt fake: length read so far = %d, len(p) = %d, offset = %d", r.LengthRead, len(p), offset) - if int(offset) != r.LengthRead { - return 0, fmt.Errorf("offset got %d; expected %d", offset, r.LengthRead) - } - - if r.Err != nil { - return 0, r.Err - } - - if r.LengthRead >= r.FileLength { - return 0, io.EOF - } - - // Read at most ChunkSize and read at most what's left in the file. - toBeRead := len(p) - if r.LengthRead+toBeRead >= r.FileLength { - toBeRead = r.FileLength - int(offset) - } - if toBeRead > r.ChunkSize { - toBeRead = r.ChunkSize - } - - r.LengthRead += toBeRead - if r.LengthRead == r.FileLength { - return toBeRead, io.EOF - } - return toBeRead, nil -} - -func TestPreadv(t *testing.T) { - // Test parameters. - type preadvTest struct { - // Name of the test. - name string - - // Mock parameters - mode p9.FileMode - - // Buffer to read into. - buffer [512]byte - sliceSize int - - // How much readAt returns at a time. - chunkSize int - - // Whether or not we expect ReadAt to be called. - readAtCalled bool - readAtErr error - - // Expected return values. - want error - } - - tests := []preadvTest{ - { - name: "fake ReadAt succeeds, 512 bytes requested, 512 byte chunks (function succeeds)", - want: nil, - readAtErr: nil, - mode: p9.PermissionsMask, - readAtCalled: true, - sliceSize: 512, - chunkSize: 512, - }, - { - name: "fake ReadAt succeeds, 512 bytes requested, 200 byte chunks (function succeeds)", - want: nil, - readAtErr: nil, - mode: p9.PermissionsMask, - readAtCalled: true, - sliceSize: 512, - chunkSize: 200, - }, - { - name: "fake ReadAt succeeds, 0 bytes requested (function succeeds)", - want: nil, - readAtErr: nil, - mode: p9.PermissionsMask, - readAtCalled: false, - sliceSize: 0, - chunkSize: 100, - }, - { - name: "fake ReadAt returns 0 bytes and EOF (function fails)", - want: io.EOF, - readAtErr: io.EOF, - mode: p9.PermissionsMask, - readAtCalled: true, - sliceSize: 512, - chunkSize: 512, - }, - } - - ctx := contexttest.Context(t) - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, test.mode, 1024) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - // Set up the read buffer. - dst := usermem.BytesIOSequence(test.buffer[:test.sliceSize]) - - // This file will be read from. - openFile := &readAtFileFake{ - Err: test.readAtErr, - FileLength: test.sliceSize, - ChunkSize: test.chunkSize, - } - rootFile.WalkGetAttrMock.File = openFile - rootFile.WalkGetAttrMock.Attr.Mode = test.mode - rootFile.WalkGetAttrMock.Valid.Mode = true - - f := NewFile( - ctx, - fs.NewDirent(rootInode, ""), - "", - fs.FileFlags{Read: true}, - rootInode.InodeOperations.(*inodeOperations), - &handles{File: contextFile{file: openFile}}, - ) - - // Call function. - _, err = f.Preadv(ctx, dst, 0) - - // Check return value. - if err != test.want { - t.Errorf("Preadv got %v, want %v", err, test.want) - } - - // Check mock parameters. - if test.readAtCalled != openFile.Called { - t.Errorf("ReadAt called: %v, but expected opposite", openFile.Called) - } - }) - } -} - -func TestReadlink(t *testing.T) { - // Test parameters. - type readlinkTest struct { - // Name of the test. - name string - - // Mock parameters - mode p9.FileMode - - // Whether or not we expect ReadAt to be called and what error - // it shall return. - readlinkCalled bool - readlinkErr error - - // Expected return values. - want error - } - - tests := []readlinkTest{ - { - name: "file is not symlink (function fails)", - want: syscall.ENOLINK, - mode: p9.PermissionsMask, - readlinkCalled: false, - readlinkErr: nil, - }, - { - name: "mock Readlink succeeds (function succeeds)", - want: nil, - mode: p9.PermissionsMask | p9.ModeSymlink, - readlinkCalled: true, - readlinkErr: nil, - }, - { - name: "mock Readlink fails (function fails)", - want: syscall.ENOENT, - mode: p9.PermissionsMask | p9.ModeSymlink, - readlinkCalled: true, - readlinkErr: syscall.ENOENT, - }, - } - - ctx := contexttest.Context(t) - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - rootFile, rootInode, err := root(ctx, cacheNone, test.mode, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - openFile := goodMockFile(test.mode, 0) - rootFile.WalkMock.File = openFile - rootFile.ReadlinkMock.Err = test.readlinkErr - - // Call function. - _, err = rootInode.Readlink(ctx) - - // Check return value. - if err != test.want { - t.Errorf("Readlink got %v, want %v", err, test.want) - } - - // Check mock parameters. - if test.readlinkCalled && !rootFile.ReadlinkMock.Called { - t.Errorf("Readlink not called") - } - }) - } -} - -// This file is write from in TestPwritev. -type writeAtFileFake struct { - p9test.FileMock - - // Parameters for faking WriteAt. - Err error - ChunkSize int - Called bool - LengthWritten int -} - -func (r *writeAtFileFake) WriteAt(p []byte, offset uint64) (int, error) { - r.Called = true - log.Warningf("WriteAt fake: length written so far = %d, len(p) = %d, offset = %d", r.LengthWritten, len(p), offset) - if int(offset) != r.LengthWritten { - return 0, fmt.Errorf("offset got %d; want %d", offset, r.LengthWritten) - } - - if r.Err != nil { - return 0, r.Err - } - - // Write at most ChunkSize. - toBeWritten := len(p) - if toBeWritten > r.ChunkSize { - toBeWritten = r.ChunkSize - } - r.LengthWritten += toBeWritten - return toBeWritten, nil -} - -func TestPwritev(t *testing.T) { - // Test parameters. - type pwritevTest struct { - // Name of the test. - name string - - // Mock parameters - mode p9.FileMode - - allowWrite bool - - // Buffer to write into. - buffer [512]byte - sliceSize int - chunkSize int - - // Whether or not we expect writeAt to be called. - writeAtCalled bool - writeAtErr error - - // Expected return values. - want error - } - - tests := []pwritevTest{ - { - name: "fake writeAt succeeds, one chunk (function succeeds)", - want: nil, - writeAtErr: nil, - mode: p9.PermissionsMask, - allowWrite: true, - writeAtCalled: true, - sliceSize: 512, - chunkSize: 512, - }, - { - name: "fake writeAt fails, short write (function fails)", - want: io.ErrShortWrite, - writeAtErr: nil, - mode: p9.PermissionsMask, - allowWrite: true, - writeAtCalled: true, - sliceSize: 512, - chunkSize: 200, - }, - { - name: "fake writeAt succeeds, len 0 (function succeeds)", - want: nil, - writeAtErr: nil, - mode: p9.PermissionsMask, - allowWrite: true, - writeAtCalled: false, - sliceSize: 0, - chunkSize: 0, - }, - { - name: "writeAt can still write despite file permissions read only (function succeeds)", - want: nil, - writeAtErr: nil, - mode: p9.PermissionsMask, - allowWrite: false, - writeAtCalled: true, - sliceSize: 512, - chunkSize: 512, - }, - } - - ctx := contexttest.Context(t) - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - // Set up mock. - _, rootInode, err := root(ctx, cacheNone, test.mode, 0) - if err != nil { - t.Fatalf("error creating root: %v", err) - } - - src := usermem.BytesIOSequence(test.buffer[:test.sliceSize]) - - // This is the file that will be used for writing. - openFile := &writeAtFileFake{ - Err: test.writeAtErr, - ChunkSize: test.chunkSize, - } - - f := NewFile( - ctx, - fs.NewDirent(rootInode, ""), - "", - fs.FileFlags{Write: true}, - rootInode.InodeOperations.(*inodeOperations), - &handles{File: contextFile{file: openFile}}, - ) - - // Call function. - _, err = f.Pwritev(ctx, src, 0) - - // Check return value. - if err != test.want { - t.Errorf("Pwritev got %v, want %v", err, test.want) - } - - // Check mock parameters. - if test.writeAtCalled != openFile.Called { - t.Errorf("WriteAt called: %v, but expected opposite", openFile.Called) - return + t.Errorf("Lookup with cachePolicy=%s got new dirent and error %v, wanted old dirent and nil error", test.cachePolicy, err) } - if openFile.Called && test.writeAtErr != nil && openFile.LengthWritten != test.sliceSize { - t.Errorf("wrote %d bytes, expected %d bytes written", openFile.LengthWritten, test.sliceSize) + if err == nil { + newDirent.DecRef() // See above. } }) } diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go index 7552216f3..f76a83cd9 100644 --- a/pkg/sentry/fs/gofer/session.go +++ b/pkg/sentry/fs/gofer/session.go @@ -91,10 +91,6 @@ func (e *endpointMaps) get(key device.MultiDeviceKey) transport.BoundEndpoint { type session struct { refs.AtomicRefCount - // conn is a unet.Socket that wraps the readFD/writeFD mount option, - // see fs/gofer/fs.go. - conn *unet.Socket `state:"nosave"` - // msize is the value of the msize mount option, see fs/gofer/fs.go. msize uint32 `state:"wait"` @@ -142,7 +138,7 @@ type session struct { // Destroy tears down the session. func (s *session) Destroy() { - s.conn.Close() + s.client.Close() } // Revalidate implements MountSource.Revalidate. @@ -235,7 +231,6 @@ func Root(ctx context.Context, dev string, filesystem fs.Filesystem, superBlockF // Construct the session. s := &session{ connID: dev, - conn: conn, msize: o.msize, version: o.version, cachePolicy: o.policy, @@ -252,7 +247,7 @@ func Root(ctx context.Context, dev string, filesystem fs.Filesystem, superBlockF m := fs.NewMountSource(s, filesystem, superBlockFlags) // Send the Tversion request. - s.client, err = p9.NewClient(s.conn, s.msize, s.version) + s.client, err = p9.NewClient(conn, s.msize, s.version) if err != nil { // Drop our reference on the session, it needs to be torn down. s.DecRef() diff --git a/pkg/sentry/fs/gofer/session_state.go b/pkg/sentry/fs/gofer/session_state.go index f657135fc..d9fd7a221 100644 --- a/pkg/sentry/fs/gofer/session_state.go +++ b/pkg/sentry/fs/gofer/session_state.go @@ -84,13 +84,13 @@ func (s *session) afterLoad() { } // Manually restore the connection. - s.conn, err = unet.NewSocket(opts.fd) + conn, err := unet.NewSocket(opts.fd) if err != nil { panic(fmt.Sprintf("failed to create Socket for FD %d: %v", opts.fd, err)) } // Manually restore the client. - s.client, err = p9.NewClient(s.conn, s.msize, s.version) + s.client, err = p9.NewClient(conn, s.msize, s.version) if err != nil { panic(fmt.Sprintf("failed to connect client to server: %v", err)) } diff --git a/pkg/sentry/fs/proc/device/BUILD b/pkg/sentry/fs/proc/device/BUILD index 34582f275..ff7dacf07 100644 --- a/pkg/sentry/fs/proc/device/BUILD +++ b/pkg/sentry/fs/proc/device/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "device", srcs = ["device.go"], diff --git a/pkg/sentry/hostcpu/BUILD b/pkg/sentry/hostcpu/BUILD index f362d15c8..33197cf14 100644 --- a/pkg/sentry/hostcpu/BUILD +++ b/pkg/sentry/hostcpu/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "hostcpu", srcs = [ diff --git a/pkg/sentry/kernel/kdefs/BUILD b/pkg/sentry/kernel/kdefs/BUILD index fe6fa2260..3f8fa206c 100644 --- a/pkg/sentry/kernel/kdefs/BUILD +++ b/pkg/sentry/kernel/kdefs/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "kdefs", srcs = ["kdefs.go"], diff --git a/pkg/sentry/kernel/memevent/BUILD b/pkg/sentry/kernel/memevent/BUILD index 66899910c..e903badd3 100644 --- a/pkg/sentry/kernel/memevent/BUILD +++ b/pkg/sentry/kernel/memevent/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "memevent", srcs = ["memory_events.go"], diff --git a/pkg/sentry/kernel/sched/BUILD b/pkg/sentry/kernel/sched/BUILD index 125792f39..52e226a39 100644 --- a/pkg/sentry/kernel/sched/BUILD +++ b/pkg/sentry/kernel/sched/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sched", srcs = [ diff --git a/pkg/sentry/loader/BUILD b/pkg/sentry/loader/BUILD index 0beb4561b..83cad186a 100644 --- a/pkg/sentry/loader/BUILD +++ b/pkg/sentry/loader/BUILD @@ -1,6 +1,7 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_embed_data") + package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_embed_data") load("//tools/go_stateify:defs.bzl", "go_library") go_embed_data( diff --git a/pkg/sentry/memutil/BUILD b/pkg/sentry/memutil/BUILD index 341b30b98..88738d65d 100644 --- a/pkg/sentry/memutil/BUILD +++ b/pkg/sentry/memutil/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "memutil", srcs = [ diff --git a/pkg/sentry/platform/interrupt/BUILD b/pkg/sentry/platform/interrupt/BUILD index 35121321a..dbafa3204 100644 --- a/pkg/sentry/platform/interrupt/BUILD +++ b/pkg/sentry/platform/interrupt/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "interrupt", srcs = [ diff --git a/pkg/sentry/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD index 4ef9e20d7..1b71e629f 100644 --- a/pkg/sentry/platform/kvm/BUILD +++ b/pkg/sentry/platform/kvm/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/platform/kvm/testutil/BUILD b/pkg/sentry/platform/kvm/testutil/BUILD index e779e3893..1dffe94a4 100644 --- a/pkg/sentry/platform/kvm/testutil/BUILD +++ b/pkg/sentry/platform/kvm/testutil/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "testutil", testonly = 1, diff --git a/pkg/sentry/platform/procid/BUILD b/pkg/sentry/platform/procid/BUILD index ba68d48f4..d3398d1e8 100644 --- a/pkg/sentry/platform/procid/BUILD +++ b/pkg/sentry/platform/procid/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "procid", srcs = [ diff --git a/pkg/sentry/platform/ptrace/BUILD b/pkg/sentry/platform/ptrace/BUILD index debae058b..2eb354ad4 100644 --- a/pkg/sentry/platform/ptrace/BUILD +++ b/pkg/sentry/platform/ptrace/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ptrace", srcs = [ diff --git a/pkg/sentry/platform/ring0/BUILD b/pkg/sentry/platform/ring0/BUILD index 2485eb2eb..c35d49f2d 100644 --- a/pkg/sentry/platform/ring0/BUILD +++ b/pkg/sentry/platform/ring0/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template( diff --git a/pkg/sentry/platform/ring0/gen_offsets/BUILD b/pkg/sentry/platform/ring0/gen_offsets/BUILD index 3bce56985..b76d7974e 100644 --- a/pkg/sentry/platform/ring0/gen_offsets/BUILD +++ b/pkg/sentry/platform/ring0/gen_offsets/BUILD @@ -1,6 +1,7 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_binary") + package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_binary") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/platform/ring0/pagetables/BUILD b/pkg/sentry/platform/ring0/pagetables/BUILD index 7a86e2234..de1b920af 100644 --- a/pkg/sentry/platform/ring0/pagetables/BUILD +++ b/pkg/sentry/platform/ring0/pagetables/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template( diff --git a/pkg/sentry/platform/safecopy/BUILD b/pkg/sentry/platform/safecopy/BUILD index 7dcf6e561..614d9e21e 100644 --- a/pkg/sentry/platform/safecopy/BUILD +++ b/pkg/sentry/platform/safecopy/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "safecopy", srcs = [ diff --git a/pkg/sentry/safemem/BUILD b/pkg/sentry/safemem/BUILD index e96509ce1..87a9bff12 100644 --- a/pkg/sentry/safemem/BUILD +++ b/pkg/sentry/safemem/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "safemem", srcs = [ diff --git a/pkg/sentry/sighandling/BUILD b/pkg/sentry/sighandling/BUILD index 751176747..41313d334 100644 --- a/pkg/sentry/sighandling/BUILD +++ b/pkg/sentry/sighandling/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sighandling", srcs = [ diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index 38fa54283..06e121946 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "rpcinet", srcs = [ diff --git a/pkg/sentry/socket/rpcinet/conn/BUILD b/pkg/sentry/socket/rpcinet/conn/BUILD index c51ca14b1..a16977f29 100644 --- a/pkg/sentry/socket/rpcinet/conn/BUILD +++ b/pkg/sentry/socket/rpcinet/conn/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # BSD - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # BSD + go_library( name = "conn", srcs = ["conn.go"], diff --git a/pkg/sentry/socket/rpcinet/notifier/BUILD b/pkg/sentry/socket/rpcinet/notifier/BUILD index 2ae902b3f..2bab01774 100644 --- a/pkg/sentry/socket/rpcinet/notifier/BUILD +++ b/pkg/sentry/socket/rpcinet/notifier/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # BSD - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # BSD + go_library( name = "notifier", srcs = ["notifier.go"], diff --git a/pkg/sentry/state/BUILD b/pkg/sentry/state/BUILD index a57a8298e..f1f6fdb7d 100644 --- a/pkg/sentry/state/BUILD +++ b/pkg/sentry/state/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "state", srcs = [ diff --git a/pkg/sentry/strace/BUILD b/pkg/sentry/strace/BUILD index 674554081..52c7f325c 100644 --- a/pkg/sentry/strace/BUILD +++ b/pkg/sentry/strace/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "strace", srcs = [ diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD index 2a9f0915e..35192ff49 100644 --- a/pkg/sentry/syscalls/BUILD +++ b/pkg/sentry/syscalls/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "syscalls", srcs = [ diff --git a/pkg/sentry/time/BUILD b/pkg/sentry/time/BUILD index 9452787fb..5dadb8a2d 100644 --- a/pkg/sentry/time/BUILD +++ b/pkg/sentry/time/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sentry/unimpl/BUILD b/pkg/sentry/unimpl/BUILD index 63da5e81f..42e24ace5 100644 --- a/pkg/sentry/unimpl/BUILD +++ b/pkg/sentry/unimpl/BUILD @@ -1,8 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +package(licenses = ["notice"]) # Apache 2.0 + proto_library( name = "unimplemented_syscall_proto", srcs = ["unimplemented_syscall.proto"], diff --git a/pkg/sentry/uniqueid/BUILD b/pkg/sentry/uniqueid/BUILD index 68b82af47..0929497c3 100644 --- a/pkg/sentry/uniqueid/BUILD +++ b/pkg/sentry/uniqueid/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "uniqueid", srcs = ["context.go"], diff --git a/pkg/sentry/watchdog/BUILD b/pkg/sentry/watchdog/BUILD index 13bc33eb1..b2c687b20 100644 --- a/pkg/sentry/watchdog/BUILD +++ b/pkg/sentry/watchdog/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "watchdog", srcs = ["watchdog.go"], diff --git a/pkg/sleep/BUILD b/pkg/sleep/BUILD index 05e4ca540..338fd9336 100644 --- a/pkg/sleep/BUILD +++ b/pkg/sleep/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sleep", srcs = [ diff --git a/pkg/state/BUILD b/pkg/state/BUILD index 6a5b2d4ff..dd0f250fa 100644 --- a/pkg/state/BUILD +++ b/pkg/state/BUILD @@ -1,7 +1,8 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + +package(licenses = ["notice"]) # Apache 2.0 + load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/state/statefile/BUILD b/pkg/state/statefile/BUILD index 6be78dc9b..66c8f3807 100644 --- a/pkg/state/statefile/BUILD +++ b/pkg/state/statefile/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "statefile", srcs = ["statefile.go"], diff --git a/pkg/sync/atomicptrtest/BUILD b/pkg/sync/atomicptrtest/BUILD index 4fa959df0..9cb7f66fe 100644 --- a/pkg/sync/atomicptrtest/BUILD +++ b/pkg/sync/atomicptrtest/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/sync/seqatomictest/BUILD b/pkg/sync/seqatomictest/BUILD index 07b4f85ab..54f8e59b1 100644 --- a/pkg/sync/seqatomictest/BUILD +++ b/pkg/sync/seqatomictest/BUILD @@ -1,6 +1,7 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") go_template_instance( diff --git a/pkg/syserr/BUILD b/pkg/syserr/BUILD index 5dd2e90bb..30ae20772 100644 --- a/pkg/syserr/BUILD +++ b/pkg/syserr/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "syserr", srcs = [ diff --git a/pkg/syserror/BUILD b/pkg/syserror/BUILD index e050c2043..d4c6da97a 100644 --- a/pkg/syserror/BUILD +++ b/pkg/syserror/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "syserror", srcs = ["syserror.go"], diff --git a/pkg/tcpip/adapters/gonet/BUILD b/pkg/tcpip/adapters/gonet/BUILD index bf618831a..723ad668f 100644 --- a/pkg/tcpip/adapters/gonet/BUILD +++ b/pkg/tcpip/adapters/gonet/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "gonet", srcs = ["gonet.go"], diff --git a/pkg/tcpip/checker/BUILD b/pkg/tcpip/checker/BUILD index e8a524918..a1de808b9 100644 --- a/pkg/tcpip/checker/BUILD +++ b/pkg/tcpip/checker/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "checker", testonly = 1, diff --git a/pkg/tcpip/link/channel/BUILD b/pkg/tcpip/link/channel/BUILD index 9a6f49c45..25f6c1457 100644 --- a/pkg/tcpip/link/channel/BUILD +++ b/pkg/tcpip/link/channel/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "channel", srcs = ["channel.go"], diff --git a/pkg/tcpip/link/fdbased/BUILD b/pkg/tcpip/link/fdbased/BUILD index 6e75e9f47..94391433c 100644 --- a/pkg/tcpip/link/fdbased/BUILD +++ b/pkg/tcpip/link/fdbased/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "fdbased", srcs = ["endpoint.go"], diff --git a/pkg/tcpip/link/loopback/BUILD b/pkg/tcpip/link/loopback/BUILD index cc4247ffd..a46ba7f11 100644 --- a/pkg/tcpip/link/loopback/BUILD +++ b/pkg/tcpip/link/loopback/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "loopback", srcs = ["loopback.go"], diff --git a/pkg/tcpip/link/rawfile/BUILD b/pkg/tcpip/link/rawfile/BUILD index 10b35a37e..829ea7c42 100644 --- a/pkg/tcpip/link/rawfile/BUILD +++ b/pkg/tcpip/link/rawfile/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "rawfile", srcs = [ diff --git a/pkg/tcpip/link/sharedmem/BUILD b/pkg/tcpip/link/sharedmem/BUILD index 5390257c5..d7f1e66ef 100644 --- a/pkg/tcpip/link/sharedmem/BUILD +++ b/pkg/tcpip/link/sharedmem/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sharedmem", srcs = [ diff --git a/pkg/tcpip/link/sharedmem/pipe/BUILD b/pkg/tcpip/link/sharedmem/pipe/BUILD index ff798ae6f..12e813509 100644 --- a/pkg/tcpip/link/sharedmem/pipe/BUILD +++ b/pkg/tcpip/link/sharedmem/pipe/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "pipe", srcs = [ diff --git a/pkg/tcpip/link/sharedmem/queue/BUILD b/pkg/tcpip/link/sharedmem/queue/BUILD index c4a7879c4..661037bb2 100644 --- a/pkg/tcpip/link/sharedmem/queue/BUILD +++ b/pkg/tcpip/link/sharedmem/queue/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "queue", srcs = [ diff --git a/pkg/tcpip/link/sniffer/BUILD b/pkg/tcpip/link/sniffer/BUILD index 7155aea66..52e237c25 100644 --- a/pkg/tcpip/link/sniffer/BUILD +++ b/pkg/tcpip/link/sniffer/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sniffer", srcs = [ diff --git a/pkg/tcpip/link/tun/BUILD b/pkg/tcpip/link/tun/BUILD index a8bb03661..5ec01cec9 100644 --- a/pkg/tcpip/link/tun/BUILD +++ b/pkg/tcpip/link/tun/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "tun", srcs = ["tun_unsafe.go"], diff --git a/pkg/tcpip/link/waitable/BUILD b/pkg/tcpip/link/waitable/BUILD index 7582df32e..ba495c437 100644 --- a/pkg/tcpip/link/waitable/BUILD +++ b/pkg/tcpip/link/waitable/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "waitable", srcs = [ diff --git a/pkg/tcpip/network/BUILD b/pkg/tcpip/network/BUILD index 25a3c98b6..a2a07f533 100644 --- a/pkg/tcpip/network/BUILD +++ b/pkg/tcpip/network/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_test( name = "ip_test", size = "small", diff --git a/pkg/tcpip/network/arp/BUILD b/pkg/tcpip/network/arp/BUILD index 44f2b66e5..f6fb7daf7 100644 --- a/pkg/tcpip/network/arp/BUILD +++ b/pkg/tcpip/network/arp/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "arp", srcs = ["arp.go"], diff --git a/pkg/tcpip/network/hash/BUILD b/pkg/tcpip/network/hash/BUILD index 1c22c52fc..401dce646 100644 --- a/pkg/tcpip/network/hash/BUILD +++ b/pkg/tcpip/network/hash/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "hash", srcs = ["hash.go"], diff --git a/pkg/tcpip/network/ipv4/BUILD b/pkg/tcpip/network/ipv4/BUILD index 90d65d531..e72317e9f 100644 --- a/pkg/tcpip/network/ipv4/BUILD +++ b/pkg/tcpip/network/ipv4/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ipv4", srcs = [ diff --git a/pkg/tcpip/network/ipv6/BUILD b/pkg/tcpip/network/ipv6/BUILD index 2f19a659e..808c37df3 100644 --- a/pkg/tcpip/network/ipv6/BUILD +++ b/pkg/tcpip/network/ipv6/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ipv6", srcs = [ diff --git a/pkg/tcpip/ports/BUILD b/pkg/tcpip/ports/BUILD index 3c3374275..c69fc0744 100644 --- a/pkg/tcpip/ports/BUILD +++ b/pkg/tcpip/ports/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "ports", srcs = ["ports.go"], diff --git a/pkg/tcpip/sample/tun_tcp_connect/BUILD b/pkg/tcpip/sample/tun_tcp_connect/BUILD index 21d32245d..32baf2115 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/BUILD +++ b/pkg/tcpip/sample/tun_tcp_connect/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "tun_tcp_connect", srcs = ["main.go"], diff --git a/pkg/tcpip/sample/tun_tcp_echo/BUILD b/pkg/tcpip/sample/tun_tcp_echo/BUILD index d7402aaa2..760445843 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/BUILD +++ b/pkg/tcpip/sample/tun_tcp_echo/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "tun_tcp_echo", srcs = ["main.go"], diff --git a/pkg/tcpip/transport/tcp/testing/context/BUILD b/pkg/tcpip/transport/tcp/testing/context/BUILD index 7a95594ef..814e5c1ea 100644 --- a/pkg/tcpip/transport/tcp/testing/context/BUILD +++ b/pkg/tcpip/transport/tcp/testing/context/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "context", testonly = 1, diff --git a/pkg/tcpip/transport/tcpconntrack/BUILD b/pkg/tcpip/transport/tcpconntrack/BUILD index 46da3e6f1..ac1a94d4d 100644 --- a/pkg/tcpip/transport/tcpconntrack/BUILD +++ b/pkg/tcpip/transport/tcpconntrack/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "tcpconntrack", srcs = ["tcp_conntrack.go"], diff --git a/pkg/tmutex/BUILD b/pkg/tmutex/BUILD index d18338fff..c20df7005 100644 --- a/pkg/tmutex/BUILD +++ b/pkg/tmutex/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "tmutex", srcs = ["tmutex.go"], diff --git a/pkg/unet/BUILD b/pkg/unet/BUILD index acdfd7cb6..f90e43c89 100644 --- a/pkg/unet/BUILD +++ b/pkg/unet/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "unet", srcs = [ diff --git a/pkg/urpc/BUILD b/pkg/urpc/BUILD index d32c57d1a..21008cf6c 100644 --- a/pkg/urpc/BUILD +++ b/pkg/urpc/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "urpc", srcs = ["urpc.go"], diff --git a/pkg/waiter/fdnotifier/BUILD b/pkg/waiter/fdnotifier/BUILD index 4e582755d..af6baa303 100644 --- a/pkg/waiter/fdnotifier/BUILD +++ b/pkg/waiter/fdnotifier/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("//tools/go_stateify:defs.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "fdnotifier", srcs = [ diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD index 04cc0e854..07afce807 100644 --- a/runsc/boot/BUILD +++ b/runsc/boot/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "boot", srcs = [ diff --git a/runsc/boot/filter/BUILD b/runsc/boot/filter/BUILD index 48f2c8024..004222242 100644 --- a/runsc/boot/filter/BUILD +++ b/runsc/boot/filter/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "filter", srcs = [ diff --git a/runsc/cgroup/BUILD b/runsc/cgroup/BUILD index 10a8e5feb..bf2f373a9 100644 --- a/runsc/cgroup/BUILD +++ b/runsc/cgroup/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "cgroup", srcs = ["cgroup.go"], diff --git a/runsc/cmd/BUILD b/runsc/cmd/BUILD index 7040eb4ec..394bb0e1f 100644 --- a/runsc/cmd/BUILD +++ b/runsc/cmd/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "cmd", srcs = [ diff --git a/runsc/console/BUILD b/runsc/console/BUILD index fa1a7d430..ff4ccff69 100644 --- a/runsc/console/BUILD +++ b/runsc/console/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "console", srcs = ["console.go"], diff --git a/runsc/container/BUILD b/runsc/container/BUILD index f4c6f1525..bdd93aaba 100644 --- a/runsc/container/BUILD +++ b/runsc/container/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "container", srcs = [ diff --git a/runsc/fsgofer/BUILD b/runsc/fsgofer/BUILD index 24e172f48..f28e4fa77 100644 --- a/runsc/fsgofer/BUILD +++ b/runsc/fsgofer/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "fsgofer", srcs = [ diff --git a/runsc/fsgofer/filter/BUILD b/runsc/fsgofer/filter/BUILD index 40f4f2205..c7848d10c 100644 --- a/runsc/fsgofer/filter/BUILD +++ b/runsc/fsgofer/filter/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "filter", srcs = [ diff --git a/runsc/fsgofer/fsgofer.go b/runsc/fsgofer/fsgofer.go index e03bb7752..fd913831a 100644 --- a/runsc/fsgofer/fsgofer.go +++ b/runsc/fsgofer/fsgofer.go @@ -26,7 +26,6 @@ import ( "math" "os" "path" - "strings" "sync" "syscall" @@ -181,18 +180,6 @@ func (a *attachPoint) makeQID(stat syscall.Stat_t) p9.QID { } } -func isNameValid(name string) bool { - if name == "" || name == "." || name == ".." { - log.Warningf("Invalid name: %s", name) - return false - } - if strings.IndexByte(name, '/') >= 0 { - log.Warningf("Invalid name: %s", name) - return false - } - return true -} - // localFile implements p9.File wrapping a local file. The underlying file // is opened during Walk() and stored in 'controlFile' to be used with other // operations. The mode in which the file is opened varies depending on the @@ -228,11 +215,7 @@ type localFile struct { // attachPoint is the attachPoint that serves this localFile. attachPoint *attachPoint - // mu protects 'hostPath' when file is renamed. - mu sync.Mutex - - // TODO: hostPath is not safe to use as path needs to be walked - // everytime (and can change underneath us). Remove all usages. + // hostPath will be safely updated by the Renamed hook. hostPath string // controlFile is opened when localFile is created and it's never nil. @@ -246,6 +229,7 @@ type localFile struct { // if localFile isn't opened. mode p9.OpenFlags + // ft is the fileType for this file. ft fileType // readDirMu protects against concurrent Readdir calls. @@ -296,10 +280,7 @@ func openAnyFile(parent *localFile, name string) (*os.File, string, error) { return nil, "", extractErrno(err) } - parent.mu.Lock() - defer parent.mu.Unlock() newPath := path.Join(parent.hostPath, name) - return os.NewFile(uintptr(fd), newPath), newPath, nil } @@ -382,13 +363,10 @@ func (l *localFile) Open(mode p9.OpenFlags) (*fd.FD, p9.QID, uint32, error) { log.Debugf("Open reopening file, mode: %v, %q", mode, l.controlFile.Name()) var err error - l.mu.Lock() newFile, err = os.OpenFile(l.hostPath, openFlags|mode.OSFlags(), 0) if err != nil { - l.mu.Unlock() return nil, p9.QID{}, 0, extractErrno(err) } - l.mu.Unlock() } stat, err := stat(int(newFile.Fd())) @@ -418,9 +396,6 @@ func (l *localFile) Create(name string, mode p9.OpenFlags, perm p9.FileMode, uid } return nil, nil, p9.QID{}, 0, syscall.EBADF } - if !isNameValid(name) { - return nil, nil, p9.QID{}, 0, syscall.EINVAL - } // Use a single file for both 'controlFile' and 'openedFile'. Mode must include read for control // and whichever else was requested by caller. Note that resulting file might have a wider mode @@ -452,9 +427,6 @@ func (l *localFile) Create(name string, mode p9.OpenFlags, perm p9.FileMode, uid return nil, nil, p9.QID{}, 0, extractErrno(err) } - l.mu.Lock() - defer l.mu.Unlock() - cPath := path.Join(l.hostPath, name) f := os.NewFile(uintptr(fd), cPath) c := &localFile{ @@ -477,10 +449,6 @@ func (l *localFile) Mkdir(name string, perm p9.FileMode, uid p9.UID, gid p9.GID) return p9.QID{}, syscall.EBADF } - if !isNameValid(name) { - return p9.QID{}, syscall.EINVAL - } - if err := syscall.Mkdirat(l.controlFD(), name, uint32(perm.Permissions())); err != nil { return p9.QID{}, extractErrno(err) } @@ -517,9 +485,6 @@ func (l *localFile) Walk(names []string) ([]p9.QID, p9.File, error) { return nil, nil, extractErrno(err) } - l.mu.Lock() - defer l.mu.Unlock() - c := &localFile{ attachPoint: l.attachPoint, hostPath: l.hostPath, @@ -532,10 +497,6 @@ func (l *localFile) Walk(names []string) ([]p9.QID, p9.File, error) { var qids []p9.QID last := l for _, name := range names { - if !isNameValid(name) { - return nil, nil, syscall.EINVAL - } - f, path, err := openAnyFile(last, name) if err != nil { return nil, nil, extractErrno(err) @@ -761,15 +722,15 @@ func (l *localFile) SetAttr(valid p9.SetAttrMask, attr p9.SetAttr) error { return err } -// Remove implements p9.File. -// -// This is deprecated in favor of UnlinkAt. -func (*localFile) Remove() error { - return syscall.ENOSYS +// Rename implements p9.File; this should never be called. +func (l *localFile) Rename(p9.File, string) error { + panic("rename called directly") } -// Rename implements p9.File. -func (l *localFile) Rename(directory p9.File, name string) error { +// RenameAt implements p9.File.RenameAt. +// +// TODO: change to renameat(2). +func (l *localFile) RenameAt(oldName string, directory p9.File, newName string) error { conf := l.attachPoint.conf if conf.ROMount { if conf.PanicOnWrite { @@ -777,34 +738,16 @@ func (l *localFile) Rename(directory p9.File, name string) error { } return syscall.EBADF } - if !isNameValid(name) { - return syscall.EINVAL - } - - l.mu.Lock() - defer l.mu.Unlock() - // TODO: change to renameat(2) - parent := directory.(*localFile) - newPath := path.Join(parent.hostPath, name) - if err := syscall.Rename(l.hostPath, newPath); err != nil { + newParent := directory.(*localFile) + oldPath := path.Join(l.hostPath, oldName) + newPath := path.Join(newParent.hostPath, newName) + if err := syscall.Rename(oldPath, newPath); err != nil { return extractErrno(err) } - - // Update path on success. - // TODO: this doesn't cover cases where any of the - // parents have been renamed. - l.hostPath = newPath return nil } -// RenameAt implements p9.File.RenameAt. -// -// Code still uses [deprecated] Rename(). -func (*localFile) RenameAt(_ string, _ p9.File, _ string) error { - return syscall.ENOSYS -} - // ReadAt implements p9.File. func (l *localFile) ReadAt(p []byte, offset uint64) (int, error) { if l.mode != p9.ReadOnly && l.mode != p9.ReadWrite { @@ -848,9 +791,6 @@ func (l *localFile) Symlink(target, newName string, uid p9.UID, gid p9.GID) (p9. } return p9.QID{}, syscall.EBADF } - if !isNameValid(newName) { - return p9.QID{}, syscall.EINVAL - } if err := unix.Symlinkat(target, l.controlFD(), newName); err != nil { return p9.QID{}, extractErrno(err) @@ -882,9 +822,6 @@ func (l *localFile) Link(target p9.File, newName string) error { } return syscall.EBADF } - if !isNameValid(newName) { - return syscall.EINVAL - } targetFile := target.(*localFile) if err := unix.Linkat(targetFile.controlFD(), "", l.controlFD(), newName, linux.AT_EMPTY_PATH); err != nil { @@ -909,9 +846,7 @@ func (l *localFile) UnlinkAt(name string, flags uint32) error { } return syscall.EBADF } - if !isNameValid(name) { - return syscall.EINVAL - } + if err := unix.Unlinkat(l.controlFD(), name, int(flags)); err != nil { return extractErrno(err) } @@ -1000,6 +935,11 @@ func (l *localFile) Close() error { return err } +// Renamed implements p9.Renamed. +func (l *localFile) Renamed(newDir p9.File, newName string) { + l.hostPath = path.Join(newDir.(*localFile).hostPath, newName) +} + // extractErrno tries to determine the errno. func extractErrno(err error) syscall.Errno { if err == nil { diff --git a/runsc/fsgofer/fsgofer_test.go b/runsc/fsgofer/fsgofer_test.go index 48860f952..34033245b 100644 --- a/runsc/fsgofer/fsgofer_test.go +++ b/runsc/fsgofer/fsgofer_test.go @@ -415,22 +415,22 @@ func TestLink(t *testing.T) { func TestROMountChecks(t *testing.T) { runCustom(t, allTypes, roConfs, func(t *testing.T, s state) { - if _, _, _, _, err := s.file.Create("..", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { + if _, _, _, _, err := s.file.Create("some_file", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { t.Errorf("%v: Create() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if _, err := s.file.Mkdir("..", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { + if _, err := s.file.Mkdir("some_dir", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { t.Errorf("%v: MkDir() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if err := s.file.Rename(s.file, ".."); err != syscall.EBADF { + if err := s.file.RenameAt("some_file", s.file, "other_file"); err != syscall.EBADF { t.Errorf("%v: Rename() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if _, err := s.file.Symlink("some_place", "..", p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { + if _, err := s.file.Symlink("some_place", "some_symlink", p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EBADF { t.Errorf("%v: Symlink() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if err := s.file.UnlinkAt("..", 0); err != syscall.EBADF { + if err := s.file.UnlinkAt("some_file", 0); err != syscall.EBADF { t.Errorf("%v: UnlinkAt() should have failed, got: %v, expected: syscall.EBADF", s, err) } - if err := s.file.Link(s.file, ".."); err != syscall.EBADF { + if err := s.file.Link(s.file, "some_link"); err != syscall.EBADF { t.Errorf("%v: Link() should have failed, got: %v, expected: syscall.EBADF", s, err) } @@ -445,12 +445,12 @@ func TestROMountChecks(t *testing.T) { func TestROMountPanics(t *testing.T) { conf := Config{ROMount: true, PanicOnWrite: true} runCustom(t, allTypes, []Config{conf}, func(t *testing.T, s state) { - assertPanic(t, func() { s.file.Create("..", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) - assertPanic(t, func() { s.file.Mkdir("..", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) - assertPanic(t, func() { s.file.Rename(s.file, "..") }) - assertPanic(t, func() { s.file.Symlink("some_place", "..", p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) - assertPanic(t, func() { s.file.UnlinkAt("..", 0) }) - assertPanic(t, func() { s.file.Link(s.file, "..") }) + assertPanic(t, func() { s.file.Create("some_file", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) + assertPanic(t, func() { s.file.Mkdir("some_dir", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) + assertPanic(t, func() { s.file.RenameAt("some_file", s.file, "other_file") }) + assertPanic(t, func() { s.file.Symlink("some_place", "some_symlink", p9.UID(os.Getuid()), p9.GID(os.Getgid())) }) + assertPanic(t, func() { s.file.UnlinkAt("some_file", 0) }) + assertPanic(t, func() { s.file.Link(s.file, "some_link") }) valid := p9.SetAttrMask{Size: true} attr := p9.SetAttr{Size: 0} @@ -458,60 +458,6 @@ func TestROMountPanics(t *testing.T) { }) } -func TestInvalidName(t *testing.T) { - runCustom(t, []fileType{regular}, rwConfs, func(t *testing.T, s state) { - if _, _, _, _, err := s.file.Create("..", p9.ReadWrite, 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EINVAL { - t.Errorf("%v: Create() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if _, _, err := s.file.Walk([]string{".."}); err != syscall.EINVAL { - t.Errorf("%v: Walk() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if _, err := s.file.Mkdir("..", 0777, p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EINVAL { - t.Errorf("%v: MkDir() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if err := s.file.Rename(s.file, ".."); err != syscall.EINVAL { - t.Errorf("%v: Rename() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if _, err := s.file.Symlink("some_place", "..", p9.UID(os.Getuid()), p9.GID(os.Getgid())); err != syscall.EINVAL { - t.Errorf("%v: Symlink() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if err := s.file.UnlinkAt("..", 0); err != syscall.EINVAL { - t.Errorf("%v: UnlinkAt() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - if err := s.file.Link(s.file, ".."); err != syscall.EINVAL { - t.Errorf("%v: Link() should have failed, got: %v, expected: syscall.EINVAL", s, err) - } - }) -} - -func TestIsNameValid(t *testing.T) { - valid := []string{ - "name", - "123", - "!@#$%^&*()", - ".name", - "..name", - "...", - } - for _, s := range valid { - if got := isNameValid(s); !got { - t.Errorf("isNameValid(%s) failed, got: %v, expected: true", s, got) - } - } - invalid := []string{ - ".", - "..", - "name/name", - "/name", - "name/", - } - for _, s := range invalid { - if got := isNameValid(s); got { - t.Errorf("isNameValid(%s) failed, got: %v, expected: false", s, got) - } - } -} - func TestWalkNotFound(t *testing.T) { runCustom(t, []fileType{directory}, allConfs, func(t *testing.T, s state) { if _, _, err := s.file.Walk([]string{"nobody-here"}); err != syscall.ENOENT { diff --git a/runsc/sandbox/BUILD b/runsc/sandbox/BUILD index eb9c4cd76..d6043bcf7 100644 --- a/runsc/sandbox/BUILD +++ b/runsc/sandbox/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "sandbox", srcs = [ diff --git a/runsc/specutils/BUILD b/runsc/specutils/BUILD index e73b2293f..a1e5da3f5 100644 --- a/runsc/specutils/BUILD +++ b/runsc/specutils/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "specutils", srcs = [ diff --git a/runsc/test/image/BUILD b/runsc/test/image/BUILD index c41161d50..22b3ebd2a 100644 --- a/runsc/test/image/BUILD +++ b/runsc/test/image/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_test( name = "image_test", size = "large", diff --git a/runsc/test/integration/BUILD b/runsc/test/integration/BUILD index 726ebf49e..e7204dc66 100644 --- a/runsc/test/integration/BUILD +++ b/runsc/test/integration/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_test( name = "integration_test", size = "large", diff --git a/runsc/test/root/BUILD b/runsc/test/root/BUILD index c69249b52..c2567ef23 100644 --- a/runsc/test/root/BUILD +++ b/runsc/test/root/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "root", srcs = ["root.go"], diff --git a/runsc/test/testutil/BUILD b/runsc/test/testutil/BUILD index da2535bfa..128bd80fb 100644 --- a/runsc/test/testutil/BUILD +++ b/runsc/test/testutil/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "testutil", srcs = [ diff --git a/runsc/tools/dockercfg/BUILD b/runsc/tools/dockercfg/BUILD index 5abb0c90a..a80b3abab 100644 --- a/runsc/tools/dockercfg/BUILD +++ b/runsc/tools/dockercfg/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "dockercfg", srcs = ["dockercfg.go"], diff --git a/tools/go_generics/BUILD b/tools/go_generics/BUILD index 1afc58625..22c2e62c3 100644 --- a/tools/go_generics/BUILD +++ b/tools/go_generics/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "go_generics", srcs = [ diff --git a/tools/go_generics/globals/BUILD b/tools/go_generics/globals/BUILD index a238becab..c26ac56d2 100644 --- a/tools/go_generics/globals/BUILD +++ b/tools/go_generics/globals/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_library") +package(licenses = ["notice"]) # Apache 2.0 + go_library( name = "globals", srcs = [ diff --git a/tools/go_generics/rules_tests/BUILD b/tools/go_generics/rules_tests/BUILD index 2d9a6fa9d..23b2d656d 100644 --- a/tools/go_generics/rules_tests/BUILD +++ b/tools/go_generics/rules_tests/BUILD @@ -1,6 +1,7 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + package(licenses = ["notice"]) # Apache 2.0 -load("@io_bazel_rules_go//go:def.bzl", "go_test") load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") go_template_instance( diff --git a/tools/go_stateify/BUILD b/tools/go_stateify/BUILD index edbeb4e2d..68d37f5d7 100644 --- a/tools/go_stateify/BUILD +++ b/tools/go_stateify/BUILD @@ -1,7 +1,7 @@ -package(licenses = ["notice"]) # Apache 2.0 - load("@io_bazel_rules_go//go:def.bzl", "go_binary") +package(licenses = ["notice"]) # Apache 2.0 + go_binary( name = "stateify", srcs = ["main.go"], -- cgit v1.2.3 From 5bd02b224fd0eb81fc028644137a24d0bbf7dab5 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Tue, 27 Nov 2018 17:47:16 -0800 Subject: Save shutdown flags first. With rpcinet if shutdown flags are not saved before making the rpc a race is possible where blocked threads are woken up before the flags have been persisted. This would mean that threads can block indefinitely in a recvmsg after a shutdown(SHUT_RD) has happened. PiperOrigin-RevId: 223089783 Change-Id: If595e7add12aece54bcdf668ab64c570910d061a --- pkg/sentry/socket/rpcinet/socket.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 788d853c9..7328661ab 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -354,6 +354,13 @@ func (s *socketOperations) Listen(t *kernel.Task, backlog int) *syserr.Error { // Shutdown implements socket.Socket.Shutdown. func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { + // We save the shutdown state because of strange differences on linux + // related to recvs on blocking vs. non-blocking sockets after a SHUT_RD. + // We need to emulate that behavior on the blocking side. + // TODO: There is a possible race that can exist with loopback, + // where data could possibly be lost. + s.setShutdownFlags(how) + stack := t.NetworkContext().(*Stack) id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Shutdown{&pb.ShutdownRequest{Fd: s.fd, How: int64(how)}}}, false /* ignoreResult */) <-c @@ -362,10 +369,6 @@ func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { return syserr.FromHost(syscall.Errno(e)) } - // We save the shutdown state because of strange differences on linux - // related to recvs on blocking vs. non-blocking sockets after a SHUT_RD. - // We need to emulate that behavior on the blocking side. - s.setShutdownFlags(how) return nil } -- cgit v1.2.3 From ffcbda0c8bd772c9019977775daf1d86891c3f28 Mon Sep 17 00:00:00 2001 From: Brian Geffon Date: Tue, 4 Dec 2018 18:14:17 -0800 Subject: Partial writes should loop in rpcinet. FileOperations.Write should return ErrWouldBlock to allow the upper layer to loop and sendmsg should continue writing where it left off on a partial write. PiperOrigin-RevId: 224081631 Change-Id: Ic61f6943ea6b7abbd82e4279decea215347eac48 --- pkg/sentry/socket/rpcinet/socket.go | 39 +++++++++++++++++++++++++++++++------ 1 file changed, 33 insertions(+), 6 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 7328661ab..90844f10f 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -212,6 +212,11 @@ func (s *socketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IO } n, err := rpcWrite(t, &pb.SyscallRequest_Write{&pb.WriteRequest{Fd: s.fd, Data: v}}) + if n > 0 && n < uint32(src.NumBytes()) { + // The FileOperations.Write interface expects us to return ErrWouldBlock in + // the event of a partial write. + return int64(n), syserror.ErrWouldBlock + } return int64(n), err.ToError() } @@ -735,19 +740,24 @@ func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] // TODO: this needs to change to map directly to a SendMsg syscall // in the RPC. - req := &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ + totalWritten := 0 + n, err := rpcSendMsg(t, &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ Fd: uint32(s.fd), Data: v, Address: to, More: flags&linux.MSG_MORE != 0, EndOfRecord: flags&linux.MSG_EOR != 0, - }} + }}) - n, err := rpcSendMsg(t, req) if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain || flags&linux.MSG_DONTWAIT != 0 { return int(n), err } + if n > 0 { + totalWritten += int(n) + v.TrimFront(int(n)) + } + // We'll have to block. Register for notification and keep trying to // send all the data. e, ch := waiter.NewChannelEntry(nil) @@ -755,13 +765,30 @@ func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] defer s.EventUnregister(&e) for { - n, err := rpcSendMsg(t, req) + n, err := rpcSendMsg(t, &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ + Fd: uint32(s.fd), + Data: v, + Address: to, + More: flags&linux.MSG_MORE != 0, + EndOfRecord: flags&linux.MSG_EOR != 0, + }}) + + if n > 0 { + totalWritten += int(n) + v.TrimFront(int(n)) + + if err == nil && totalWritten < int(src.NumBytes()) { + continue + } + } + if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain { - return int(n), err + // We eat the error in this situation. + return int(totalWritten), nil } if err := t.Block(ch); err != nil { - return 0, syserr.FromError(err) + return int(totalWritten), syserr.FromError(err) } } } -- cgit v1.2.3 From e1dcf92ec5cf7d9bf58fb322f46f6ae2d98699d2 Mon Sep 17 00:00:00 2001 From: Ian Gudger Date: Fri, 14 Dec 2018 16:12:51 -0800 Subject: Implement SO_SNDTIMEO PiperOrigin-RevId: 225620490 Change-Id: Ia726107b3f58093a5f881634f90b071b32d2c269 --- pkg/sentry/fs/host/BUILD | 1 + pkg/sentry/fs/host/socket_test.go | 3 +- pkg/sentry/socket/epsocket/epsocket.go | 35 +++++++- pkg/sentry/socket/hostinet/socket.go | 9 ++- pkg/sentry/socket/netlink/socket.go | 4 +- pkg/sentry/socket/rpcinet/socket.go | 41 ++++++++-- pkg/sentry/socket/socket.go | 49 +++++++++--- pkg/sentry/socket/unix/unix.go | 16 ++-- pkg/sentry/syscalls/linux/sys_socket.go | 36 +++++++-- test/syscalls/linux/socket_generic.cc | 110 ++++++++++++++++++++++++++ test/syscalls/linux/socket_stream_blocking.cc | 22 ++++++ test/syscalls/linux/socket_unix_non_stream.cc | 22 ++++++ 12 files changed, 307 insertions(+), 41 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD index 89d7b2fe7..73d9cc71a 100644 --- a/pkg/sentry/fs/host/BUILD +++ b/pkg/sentry/fs/host/BUILD @@ -71,6 +71,7 @@ go_test( "//pkg/sentry/context", "//pkg/sentry/context/contexttest", "//pkg/sentry/fs", + "//pkg/sentry/kernel/time", "//pkg/sentry/socket", "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", diff --git a/pkg/sentry/fs/host/socket_test.go b/pkg/sentry/fs/host/socket_test.go index 17bf397ef..6ddf63a6a 100644 --- a/pkg/sentry/fs/host/socket_test.go +++ b/pkg/sentry/fs/host/socket_test.go @@ -21,6 +21,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/fd" "gvisor.googlesource.com/gvisor/pkg/sentry/context/contexttest" + ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/socket" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" @@ -142,7 +143,7 @@ func TestSocketSendMsgLen0(t *testing.T) { defer sfile.DecRef() s := sfile.FileOperations.(socket.Socket) - n, terr := s.SendMsg(nil, usermem.BytesIOSequence(nil), []byte{}, 0, socket.ControlMessages{}) + n, terr := s.SendMsg(nil, usermem.BytesIOSequence(nil), []byte{}, 0, false, ktime.Time{}, socket.ControlMessages{}) if n != 0 { t.Fatalf("socket sendmsg() failed: %v wrote: %d", terr, n) } diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 19af7bc45..ab5d82183 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -30,6 +30,7 @@ import ( "strings" "sync" "syscall" + "time" "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/binary" @@ -137,12 +138,12 @@ type commonEndpoint interface { // // +stateify savable type SocketOperations struct { - socket.ReceiveTimeout fsutil.PipeSeek `state:"nosave"` fsutil.NotDirReaddir `state:"nosave"` fsutil.NoFsync `state:"nosave"` fsutil.NoopFlush `state:"nosave"` fsutil.NoMMap `state:"nosave"` + socket.SendReceiveTimeout *waiter.Queue family int @@ -643,7 +644,16 @@ func getSockOptSocket(t *kernel.Task, s socket.Socket, ep commonEndpoint, family } return syscall.Linger{}, nil + case linux.SO_SNDTIMEO: + // TODO: Linux allows shorter lengths for partial results. + if outLen < linux.SizeOfTimeval { + return nil, syserr.ErrInvalidArgument + } + + return linux.NsecToTimeval(s.SendTimeout()), nil + case linux.SO_RCVTIMEO: + // TODO: Linux allows shorter lengths for partial results. if outLen < linux.SizeOfTimeval { return nil, syserr.ErrInvalidArgument } @@ -833,6 +843,19 @@ func setSockOptSocket(t *kernel.Task, s socket.Socket, ep commonEndpoint, name i v := usermem.ByteOrder.Uint32(optVal) return syserr.TranslateNetstackError(ep.SetSockOpt(tcpip.PasscredOption(v))) + case linux.SO_SNDTIMEO: + if len(optVal) < linux.SizeOfTimeval { + return syserr.ErrInvalidArgument + } + + var v linux.Timeval + binary.Unmarshal(optVal[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + if v.Usec < 0 || v.Usec >= int64(time.Second/time.Microsecond) { + return syserr.ErrDomain + } + s.SetSendTimeout(v.ToNsecCapped()) + return nil + case linux.SO_RCVTIMEO: if len(optVal) < linux.SizeOfTimeval { return syserr.ErrInvalidArgument @@ -840,6 +863,9 @@ func setSockOptSocket(t *kernel.Task, s socket.Socket, ep commonEndpoint, name i var v linux.Timeval binary.Unmarshal(optVal[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + if v.Usec < 0 || v.Usec >= int64(time.Second/time.Microsecond) { + return syserr.ErrDomain + } s.SetRecvTimeout(v.ToNsecCapped()) return nil @@ -1365,7 +1391,7 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags // SendMsg implements the linux syscall sendmsg(2) for sockets backed by // tcpip.Endpoint. -func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { +func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) { // Reject Unix control messages. if !controlMessages.Unix.Empty() { return 0, syserr.ErrInvalidArgument @@ -1431,7 +1457,10 @@ func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] return int(total), nil } - if err := t.Block(ch); err != nil { + if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { + if err == syserror.ETIMEDOUT { + return int(total), syserr.ErrTryAgain + } // handleIOError will consume errors from t.Block if needed. return int(total), syserr.FromError(err) } diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index e4e950fbb..34281cac0 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -46,12 +46,12 @@ const ( // socketOperations implements fs.FileOperations and socket.Socket for a socket // implemented using a host socket. type socketOperations struct { - socket.ReceiveTimeout fsutil.PipeSeek `state:"nosave"` fsutil.NotDirReaddir `state:"nosave"` fsutil.NoFsync `state:"nosave"` fsutil.NoopFlush `state:"nosave"` fsutil.NoMMap `state:"nosave"` + socket.SendReceiveTimeout fd int // must be O_NONBLOCK queue waiter.Queue @@ -418,7 +418,7 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags } // SendMsg implements socket.Socket.SendMsg. -func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { +func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) { // Whitelist flags. if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 { return 0, syserr.ErrInvalidArgument @@ -468,7 +468,10 @@ func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] panic(fmt.Sprintf("CopyInTo: got (%d, %v), wanted (0, %v)", n, err, err)) } if ch != nil { - if err = t.Block(ch); err != nil { + if err = t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { + if err == syserror.ETIMEDOUT { + err = syserror.ErrWouldBlock + } break } } else { diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index c4798839e..0a7d4772c 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -65,12 +65,12 @@ var netlinkSocketDevice = device.NewAnonDevice() // // +stateify savable type Socket struct { - socket.ReceiveTimeout fsutil.PipeSeek `state:"nosave"` fsutil.NotDirReaddir `state:"nosave"` fsutil.NoFsync `state:"nosave"` fsutil.NoopFlush `state:"nosave"` fsutil.NoMMap `state:"nosave"` + socket.SendReceiveTimeout // ports provides netlink port allocation. ports *port.Manager @@ -593,7 +593,7 @@ func (s *Socket) sendMsg(ctx context.Context, src usermem.IOSequence, to []byte, } // SendMsg implements socket.Socket.SendMsg. -func (s *Socket) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { +func (s *Socket) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) { return s.sendMsg(t, src, to, flags, controlMessages) } diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 90844f10f..257bc2d71 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -17,6 +17,7 @@ package rpcinet import ( "sync/atomic" "syscall" + "time" "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/binary" @@ -44,12 +45,12 @@ import ( // socketOperations implements fs.FileOperations and socket.Socket for a socket // implemented using a host socket. type socketOperations struct { - socket.ReceiveTimeout fsutil.PipeSeek `state:"nosave"` fsutil.NotDirReaddir `state:"nosave"` fsutil.NoFsync `state:"nosave"` fsutil.NoopFlush `state:"nosave"` fsutil.NoMMap `state:"nosave"` + socket.SendReceiveTimeout fd uint32 // must be O_NONBLOCK wq *waiter.Queue @@ -379,7 +380,8 @@ func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { // GetSockOpt implements socket.Socket.GetSockOpt. func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outLen int) (interface{}, *syserr.Error) { - // SO_RCVTIMEO is special because blocking is performed within the sentry. + // SO_RCVTIMEO and SO_SNDTIMEO are special because blocking is performed + // within the sentry. if level == linux.SOL_SOCKET && name == linux.SO_RCVTIMEO { if outLen < linux.SizeOfTimeval { return nil, syserr.ErrInvalidArgument @@ -387,6 +389,13 @@ func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outLe return linux.NsecToTimeval(s.RecvTimeout()), nil } + if level == linux.SOL_SOCKET && name == linux.SO_SNDTIMEO { + if outLen < linux.SizeOfTimeval { + return nil, syserr.ErrInvalidArgument + } + + return linux.NsecToTimeval(s.SendTimeout()), nil + } stack := t.NetworkContext().(*Stack) id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_GetSockOpt{&pb.GetSockOptRequest{Fd: s.fd, Level: int64(level), Name: int64(name), Length: uint32(outLen)}}}, false /* ignoreResult */) @@ -403,8 +412,9 @@ func (s *socketOperations) GetSockOpt(t *kernel.Task, level int, name int, outLe // SetSockOpt implements socket.Socket.SetSockOpt. func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error { // Because blocking actually happens within the sentry we need to inspect - // this socket option to determine if it's a SO_RCVTIMEO, and if so, we will - // save it and use it as the deadline for recv(2) related syscalls. + // this socket option to determine if it's a SO_RCVTIMEO or SO_SNDTIMEO, + // and if so, we will save it and use it as the deadline for recv(2) + // or send(2) related syscalls. if level == linux.SOL_SOCKET && name == linux.SO_RCVTIMEO { if len(opt) < linux.SizeOfTimeval { return syserr.ErrInvalidArgument @@ -412,9 +422,25 @@ func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt [ var v linux.Timeval binary.Unmarshal(opt[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + if v.Usec < 0 || v.Usec >= int64(time.Second/time.Microsecond) { + return syserr.ErrDomain + } s.SetRecvTimeout(v.ToNsecCapped()) return nil } + if level == linux.SOL_SOCKET && name == linux.SO_SNDTIMEO { + if len(opt) < linux.SizeOfTimeval { + return syserr.ErrInvalidArgument + } + + var v linux.Timeval + binary.Unmarshal(opt[:linux.SizeOfTimeval], usermem.ByteOrder, &v) + if v.Usec < 0 || v.Usec >= int64(time.Second/time.Microsecond) { + return syserr.ErrDomain + } + s.SetSendTimeout(v.ToNsecCapped()) + return nil + } stack := t.NetworkContext().(*Stack) id, c := stack.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_SetSockOpt{&pb.SetSockOptRequest{Fd: s.fd, Level: int64(level), Name: int64(name), Opt: opt}}}, false /* ignoreResult */) @@ -720,7 +746,7 @@ func rpcSendMsg(t *kernel.Task, req *pb.SyscallRequest_Sendmsg) (uint32, *syserr } // SendMsg implements socket.Socket.SendMsg. -func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { +func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) { // Whitelist flags. if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_EOR|syscall.MSG_FASTOPEN|syscall.MSG_MORE|syscall.MSG_NOSIGNAL) != 0 { return 0, syserr.ErrInvalidArgument @@ -787,7 +813,10 @@ func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] return int(totalWritten), nil } - if err := t.Block(ch); err != nil { + if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { + if err == syserror.ETIMEDOUT { + return int(totalWritten), syserr.ErrTryAgain + } return int(totalWritten), syserr.FromError(err) } } diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index f73127ea6..9d4aaeb9d 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -94,15 +94,23 @@ type Socket interface { // ownership of the ControlMessage on error. // // If n > 0, err will either be nil or an error from t.Block. - SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages ControlMessages) (n int, err *syserr.Error) + SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages ControlMessages) (n int, err *syserr.Error) // SetRecvTimeout sets the timeout (in ns) for recv operations. Zero means - // no timeout. + // no timeout, and negative means DONTWAIT. SetRecvTimeout(nanoseconds int64) // RecvTimeout gets the current timeout (in ns) for recv operations. Zero - // means no timeout. + // means no timeout, and negative means DONTWAIT. RecvTimeout() int64 + + // SetSendTimeout sets the timeout (in ns) for send operations. Zero means + // no timeout, and negative means DONTWAIT. + SetSendTimeout(nanoseconds int64) + + // SendTimeout gets the current timeout (in ns) for send operations. Zero + // means no timeout, and negative means DONTWAIT. + SendTimeout() int64 } // Provider is the interface implemented by providers of sockets for specific @@ -192,30 +200,45 @@ func NewDirent(ctx context.Context, d *device.Device) *fs.Dirent { return fs.NewDirent(inode, fmt.Sprintf("socket:[%d]", ino)) } -// ReceiveTimeout stores a timeout for receive calls. +// SendReceiveTimeout stores timeouts for send and receive calls. // // It is meant to be embedded into Socket implementations to help satisfy the // interface. // -// Care must be taken when copying ReceiveTimeout as it contains atomic +// Care must be taken when copying SendReceiveTimeout as it contains atomic // variables. // // +stateify savable -type ReceiveTimeout struct { - // ns is length of the timeout in nanoseconds. +type SendReceiveTimeout struct { + // send is length of the send timeout in nanoseconds. + // + // send must be accessed atomically. + send int64 + + // recv is length of the receive timeout in nanoseconds. // - // ns must be accessed atomically. - ns int64 + // recv must be accessed atomically. + recv int64 } // SetRecvTimeout implements Socket.SetRecvTimeout. -func (rt *ReceiveTimeout) SetRecvTimeout(nanoseconds int64) { - atomic.StoreInt64(&rt.ns, nanoseconds) +func (to *SendReceiveTimeout) SetRecvTimeout(nanoseconds int64) { + atomic.StoreInt64(&to.recv, nanoseconds) } // RecvTimeout implements Socket.RecvTimeout. -func (rt *ReceiveTimeout) RecvTimeout() int64 { - return atomic.LoadInt64(&rt.ns) +func (to *SendReceiveTimeout) RecvTimeout() int64 { + return atomic.LoadInt64(&to.recv) +} + +// SetSendTimeout implements Socket.SetSendTimeout. +func (to *SendReceiveTimeout) SetSendTimeout(nanoseconds int64) { + atomic.StoreInt64(&to.send, nanoseconds) +} + +// SendTimeout implements Socket.SendTimeout. +func (to *SendReceiveTimeout) SendTimeout() int64 { + return atomic.LoadInt64(&to.send) } // GetSockOptEmitUnimplementedEvent emits unimplemented event if name is valid. diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index 4c9dcbd61..da225eabb 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -45,15 +45,16 @@ import ( // // +stateify savable type SocketOperations struct { - refs.AtomicRefCount - socket.ReceiveTimeout fsutil.PipeSeek `state:"nosave"` fsutil.NotDirReaddir `state:"nosave"` fsutil.NoFsync `state:"nosave"` fsutil.NoopFlush `state:"nosave"` fsutil.NoMMap `state:"nosave"` - ep transport.Endpoint - isPacket bool + refs.AtomicRefCount + socket.SendReceiveTimeout + + ep transport.Endpoint + isPacket bool } // New creates a new unix socket. @@ -367,7 +368,7 @@ func (s *SocketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IO // SendMsg implements the linux syscall sendmsg(2) for unix sockets backed by // a transport.Endpoint. -func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages socket.ControlMessages) (int, *syserr.Error) { +func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, haveDeadline bool, deadline ktime.Time, controlMessages socket.ControlMessages) (int, *syserr.Error) { w := EndpointWriter{ Endpoint: s.ep, Control: controlMessages.Unix, @@ -404,7 +405,10 @@ func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] break } - if err := t.Block(ch); err != nil { + if err = t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { + if err == syserror.ETIMEDOUT { + err = syserror.ErrWouldBlock + } break } } diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index 1165d4566..3049fe6e5 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -612,9 +612,11 @@ func RecvMsg(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sysca var haveDeadline bool var deadline ktime.Time - if dl := s.RecvTimeout(); dl != 0 { + if dl := s.RecvTimeout(); dl > 0 { deadline = t.Kernel().MonotonicClock().Now().Add(time.Duration(dl) * time.Nanosecond) haveDeadline = true + } else if dl < 0 { + flags |= linux.MSG_DONTWAIT } n, err := recvSingleMsg(t, s, msgPtr, flags, haveDeadline, deadline) @@ -671,10 +673,11 @@ func RecvMMsg(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sysc } if !haveDeadline { - dl := s.RecvTimeout() - if dl != 0 { + if dl := s.RecvTimeout(); dl > 0 { deadline = t.Kernel().MonotonicClock().Now().Add(time.Duration(dl) * time.Nanosecond) haveDeadline = true + } else if dl < 0 { + flags |= linux.MSG_DONTWAIT } } @@ -821,10 +824,11 @@ func recvFrom(t *kernel.Task, fd kdefs.FD, bufPtr usermem.Addr, bufLen uint64, f var haveDeadline bool var deadline ktime.Time - - if dl := s.RecvTimeout(); dl != 0 { + if dl := s.RecvTimeout(); dl > 0 { deadline = t.Kernel().MonotonicClock().Now().Add(time.Duration(dl) * time.Nanosecond) haveDeadline = true + } else if dl < 0 { + flags |= linux.MSG_DONTWAIT } n, sender, senderLen, cm, e := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, nameLenPtr != 0, 0) @@ -1001,8 +1005,17 @@ func sendSingleMsg(t *kernel.Task, s socket.Socket, file *fs.File, msgPtr userme return 0, err } + var haveDeadline bool + var deadline ktime.Time + if dl := s.SendTimeout(); dl > 0 { + deadline = t.Kernel().MonotonicClock().Now().Add(time.Duration(dl) * time.Nanosecond) + haveDeadline = true + } else if dl < 0 { + flags |= linux.MSG_DONTWAIT + } + // Call the syscall implementation. - n, e := s.SendMsg(t, src, to, int(flags), socket.ControlMessages{Unix: controlMessages}) + n, e := s.SendMsg(t, src, to, int(flags), haveDeadline, deadline, socket.ControlMessages{Unix: controlMessages}) err = handleIOError(t, n != 0, e.ToError(), kernel.ERESTARTSYS, "sendmsg", file) if err != nil { controlMessages.Release() @@ -1052,8 +1065,17 @@ func sendTo(t *kernel.Task, fd kdefs.FD, bufPtr usermem.Addr, bufLen uint64, fla return 0, err } + var haveDeadline bool + var deadline ktime.Time + if dl := s.SendTimeout(); dl > 0 { + deadline = t.Kernel().MonotonicClock().Now().Add(time.Duration(dl) * time.Nanosecond) + haveDeadline = true + } else if dl < 0 { + flags |= linux.MSG_DONTWAIT + } + // Call the syscall implementation. - n, e := s.SendMsg(t, src, to, int(flags), socket.ControlMessages{Unix: control.New(t, s, nil)}) + n, e := s.SendMsg(t, src, to, int(flags), haveDeadline, deadline, socket.ControlMessages{Unix: control.New(t, s, nil)}) return uintptr(n), handleIOError(t, n != 0, e.ToError(), kernel.ERESTARTSYS, "sendto", file) } diff --git a/test/syscalls/linux/socket_generic.cc b/test/syscalls/linux/socket_generic.cc index a9edbb950..c65b29112 100644 --- a/test/syscalls/linux/socket_generic.cc +++ b/test/syscalls/linux/socket_generic.cc @@ -332,6 +332,35 @@ TEST_P(AllSocketPairTest, RecvmsgTimeoutSucceeds) { SyscallFailsWithErrno(EAGAIN)); } +TEST_P(AllSocketPairTest, SendTimeoutAllowsSend) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = 0, .tv_usec = 10 + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_SNDTIMEO, &tv, sizeof(tv)), + SyscallSucceeds()); + + char buf[20] = {}; + ASSERT_THAT(RetryEINTR(send)(sockets->first_fd(), buf, sizeof(buf), 0), + SyscallSucceedsWithValue(sizeof(buf))); +} + +TEST_P(AllSocketPairTest, SendmsgTimeoutAllowsSend) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = 0, .tv_usec = 10 + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_SNDTIMEO, &tv, sizeof(tv)), + SyscallSucceeds()); + + char buf[20] = {}; + ASSERT_NO_FATAL_FAILURE(SendNullCmsg(sockets->first_fd(), buf, sizeof(buf))); +} + TEST_P(AllSocketPairTest, SoRcvTimeoIsSet) { auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); @@ -382,6 +411,87 @@ TEST_P(AllSocketPairTest, RecvmsgTimeoutOneSecondSucceeds) { SyscallFailsWithErrno(EAGAIN)); } +TEST_P(AllSocketPairTest, RecvTimeoutUsecTooLarge) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = 0, .tv_usec = 2000000 // 2 seconds. + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_RCVTIMEO, &tv, sizeof(tv)), + SyscallFailsWithErrno(EDOM)); +} + +TEST_P(AllSocketPairTest, SendTimeoutUsecTooLarge) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = 0, .tv_usec = 2000000 // 2 seconds. + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_SNDTIMEO, &tv, sizeof(tv)), + SyscallFailsWithErrno(EDOM)); +} + +TEST_P(AllSocketPairTest, RecvTimeoutUsecNeg) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = 0, .tv_usec = -1 + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_RCVTIMEO, &tv, sizeof(tv)), + SyscallFailsWithErrno(EDOM)); +} + +TEST_P(AllSocketPairTest, SendTimeoutUsecNeg) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = 0, .tv_usec = -1 + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_SNDTIMEO, &tv, sizeof(tv)), + SyscallFailsWithErrno(EDOM)); +} + +TEST_P(AllSocketPairTest, RecvTimeoutNegSec) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = -1, .tv_usec = 0 + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_RCVTIMEO, &tv, sizeof(tv)), + SyscallSucceeds()); + + char buf[20] = {}; + EXPECT_THAT(RetryEINTR(recv)(sockets->first_fd(), buf, sizeof(buf), 0), + SyscallFailsWithErrno(EAGAIN)); +} + +TEST_P(AllSocketPairTest, RecvmsgTimeoutNegSec) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = -1, .tv_usec = 0 + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_RCVTIMEO, &tv, sizeof(tv)), + SyscallSucceeds()); + + struct msghdr msg = {}; + char buf[20] = {}; + struct iovec iov; + iov.iov_base = buf; + iov.iov_len = sizeof(buf); + msg.msg_iov = &iov; + msg.msg_iovlen = 1; + + EXPECT_THAT(RetryEINTR(recvmsg)(sockets->first_fd(), &msg, 0), + SyscallFailsWithErrno(EAGAIN)); +} + TEST_P(AllSocketPairTest, RecvWaitAll) { auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); diff --git a/test/syscalls/linux/socket_stream_blocking.cc b/test/syscalls/linux/socket_stream_blocking.cc index 3fbbe54d8..6cfadc9da 100644 --- a/test/syscalls/linux/socket_stream_blocking.cc +++ b/test/syscalls/linux/socket_stream_blocking.cc @@ -125,5 +125,27 @@ TEST_P(BlockingStreamSocketPairTest, RecvLessThanBufferWaitAll) { EXPECT_GE(after - before, kDuration); } +TEST_P(BlockingStreamSocketPairTest, SendTimeout) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = 0, .tv_usec = 10 + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_SNDTIMEO, &tv, sizeof(tv)), + SyscallSucceeds()); + + char buf[100] = {}; + for (;;) { + int ret; + ASSERT_THAT( + ret = RetryEINTR(send)(sockets->first_fd(), buf, sizeof(buf), 0), + ::testing::AnyOf(SyscallSucceeds(), SyscallFailsWithErrno(EAGAIN))); + if (ret == -1) { + break; + } + } +} + } // namespace testing } // namespace gvisor diff --git a/test/syscalls/linux/socket_unix_non_stream.cc b/test/syscalls/linux/socket_unix_non_stream.cc index 620397746..264b7fe6a 100644 --- a/test/syscalls/linux/socket_unix_non_stream.cc +++ b/test/syscalls/linux/socket_unix_non_stream.cc @@ -225,5 +225,27 @@ TEST_P(UnixNonStreamSocketPairTest, FragmentedRecvMsg) { EXPECT_EQ(0, memcmp(write_buf.data(), ptr, buffer_size)); } +TEST_P(UnixNonStreamSocketPairTest, SendTimeout) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + struct timeval tv { + .tv_sec = 0, .tv_usec = 10 + }; + EXPECT_THAT( + setsockopt(sockets->first_fd(), SOL_SOCKET, SO_SNDTIMEO, &tv, sizeof(tv)), + SyscallSucceeds()); + + char buf[100] = {}; + for (;;) { + int ret; + ASSERT_THAT( + ret = RetryEINTR(send)(sockets->first_fd(), buf, sizeof(buf), 0), + ::testing::AnyOf(SyscallSucceeds(), SyscallFailsWithErrno(EAGAIN))); + if (ret == -1) { + break; + } + } +} + } // namespace testing } // namespace gvisor -- cgit v1.2.3 From 652d068119052b0b3bc4a0808a4400a22380a30b Mon Sep 17 00:00:00 2001 From: Andrei Vagin Date: Fri, 28 Dec 2018 11:26:01 -0800 Subject: Implement SO_REUSEPORT for TCP and UDP sockets This option allows multiple sockets to be bound to the same port. Incoming packets are distributed to sockets using a hash based on source and destination addresses. This means that all packets from one sender will be received by the same server socket. PiperOrigin-RevId: 227153413 Change-Id: I59b6edda9c2209d5b8968671e9129adb675920cf --- pkg/sentry/socket/epsocket/epsocket.go | 20 ++ pkg/sentry/socket/rpcinet/socket.go | 5 +- pkg/tcpip/hash/jenkins/BUILD | 21 ++ pkg/tcpip/hash/jenkins/jenkins.go | 80 ++++++++ pkg/tcpip/hash/jenkins/jenkins_test.go | 176 +++++++++++++++++ pkg/tcpip/ports/BUILD | 4 +- pkg/tcpip/ports/ports.go | 74 +++++-- pkg/tcpip/ports/ports_test.go | 134 ++++++++----- pkg/tcpip/stack/BUILD | 1 + pkg/tcpip/stack/stack.go | 12 +- pkg/tcpip/stack/transport_demuxer.go | 144 +++++++++++++- pkg/tcpip/stack/transport_test.go | 2 +- pkg/tcpip/tcpip.go | 4 + pkg/tcpip/transport/ping/endpoint.go | 8 +- pkg/tcpip/transport/tcp/accept.go | 2 +- pkg/tcpip/transport/tcp/endpoint.go | 34 +++- pkg/tcpip/transport/udp/endpoint.go | 30 ++- pkg/tcpip/transport/udp/udp_test.go | 85 ++++++++ test/syscalls/linux/BUILD | 4 + test/syscalls/linux/socket_inet_loopback.cc | 289 ++++++++++++++++++++++++++++ test/syscalls/syscall_test_runner.go | 1 + 21 files changed, 1025 insertions(+), 105 deletions(-) create mode 100644 pkg/tcpip/hash/jenkins/BUILD create mode 100644 pkg/tcpip/hash/jenkins/jenkins.go create mode 100644 pkg/tcpip/hash/jenkins/jenkins_test.go (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 1b9c75949..d65b5f49e 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -634,6 +634,18 @@ func getSockOptSocket(t *kernel.Task, s socket.Socket, ep commonEndpoint, family return int32(v), nil + case linux.SO_REUSEPORT: + if outLen < sizeOfInt32 { + return nil, syserr.ErrInvalidArgument + } + + var v tcpip.ReusePortOption + if err := ep.GetSockOpt(&v); err != nil { + return nil, syserr.TranslateNetstackError(err) + } + + return int32(v), nil + case linux.SO_KEEPALIVE: if outLen < sizeOfInt32 { return nil, syserr.ErrInvalidArgument @@ -900,6 +912,14 @@ func setSockOptSocket(t *kernel.Task, s socket.Socket, ep commonEndpoint, name i v := usermem.ByteOrder.Uint32(optVal) return syserr.TranslateNetstackError(ep.SetSockOpt(tcpip.ReuseAddressOption(v))) + case linux.SO_REUSEPORT: + if len(optVal) < sizeOfInt32 { + return syserr.ErrInvalidArgument + } + + v := usermem.ByteOrder.Uint32(optVal) + return syserr.TranslateNetstackError(ep.SetSockOpt(tcpip.ReusePortOption(v))) + case linux.SO_PASSCRED: if len(optVal) < sizeOfInt32 { return syserr.ErrInvalidArgument diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 257bc2d71..8c8ebadb7 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -285,7 +285,10 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, if blocking && se == syserr.ErrTryAgain { // Register for notifications. e, ch := waiter.NewChannelEntry(nil) - s.EventRegister(&e, waiter.EventIn) + // FIXME: This waiter.EventHUp is a partial + // measure, need to figure out how to translate linux events to + // internal events. + s.EventRegister(&e, waiter.EventIn|waiter.EventHUp) defer s.EventUnregister(&e) // Try to accept the connection again; if it fails, then wait until we diff --git a/pkg/tcpip/hash/jenkins/BUILD b/pkg/tcpip/hash/jenkins/BUILD new file mode 100644 index 000000000..bbb764db8 --- /dev/null +++ b/pkg/tcpip/hash/jenkins/BUILD @@ -0,0 +1,21 @@ +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + +package(licenses = ["notice"]) # Apache 2.0 + +go_library( + name = "jenkins", + srcs = ["jenkins.go"], + importpath = "gvisor.googlesource.com/gvisor/pkg/tcpip/hash/jenkins", + visibility = [ + "//visibility:public", + ], +) + +go_test( + name = "jenkins_test", + size = "small", + srcs = [ + "jenkins_test.go", + ], + embed = [":jenkins"], +) diff --git a/pkg/tcpip/hash/jenkins/jenkins.go b/pkg/tcpip/hash/jenkins/jenkins.go new file mode 100644 index 000000000..e66d5f12b --- /dev/null +++ b/pkg/tcpip/hash/jenkins/jenkins.go @@ -0,0 +1,80 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package jenkins implements Jenkins's one_at_a_time, non-cryptographic hash +// functions created by by Bob Jenkins. +// +// See https://en.wikipedia.org/wiki/Jenkins_hash_function#cite_note-dobbsx-1 +// +package jenkins + +import ( + "hash" +) + +// Sum32 represents Jenkins's one_at_a_time hash. +// +// Use the Sum32 type directly (as opposed to New32 below) +// to avoid allocations. +type Sum32 uint32 + +// New32 returns a new 32-bit Jenkins's one_at_a_time hash.Hash. +// +// Its Sum method will lay the value out in big-endian byte order. +func New32() hash.Hash32 { + var s Sum32 + return &s +} + +// Reset resets the hash to its initial state. +func (s *Sum32) Reset() { *s = 0 } + +// Sum32 returns the hash value +func (s *Sum32) Sum32() uint32 { + hash := *s + + hash += (hash << 3) + hash ^= hash >> 11 + hash += hash << 15 + + return uint32(hash) +} + +// Write adds more data to the running hash. +// +// It never returns an error. +func (s *Sum32) Write(data []byte) (int, error) { + hash := *s + for _, b := range data { + hash += Sum32(b) + hash += hash << 10 + hash ^= hash >> 6 + } + *s = hash + return len(data), nil +} + +// Size returns the number of bytes Sum will return. +func (s *Sum32) Size() int { return 4 } + +// BlockSize returns the hash's underlying block size. +func (s *Sum32) BlockSize() int { return 1 } + +// Sum appends the current hash to in and returns the resulting slice. +// +// It does not change the underlying hash state. +func (s *Sum32) Sum(in []byte) []byte { + v := s.Sum32() + return append(in, byte(v>>24), byte(v>>16), byte(v>>8), byte(v)) +} diff --git a/pkg/tcpip/hash/jenkins/jenkins_test.go b/pkg/tcpip/hash/jenkins/jenkins_test.go new file mode 100644 index 000000000..9d86174aa --- /dev/null +++ b/pkg/tcpip/hash/jenkins/jenkins_test.go @@ -0,0 +1,176 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package jenkins + +import ( + "bytes" + "encoding/binary" + "hash" + "hash/fnv" + "math" + "testing" +) + +func TestGolden32(t *testing.T) { + var golden32 = []struct { + out []byte + in string + }{ + {[]byte{0x00, 0x00, 0x00, 0x00}, ""}, + {[]byte{0xca, 0x2e, 0x94, 0x42}, "a"}, + {[]byte{0x45, 0xe6, 0x1e, 0x58}, "ab"}, + {[]byte{0xed, 0x13, 0x1f, 0x5b}, "abc"}, + } + + hash := New32() + + for _, g := range golden32 { + hash.Reset() + done, error := hash.Write([]byte(g.in)) + if error != nil { + t.Fatalf("write error: %s", error) + } + if done != len(g.in) { + t.Fatalf("wrote only %d out of %d bytes", done, len(g.in)) + } + if actual := hash.Sum(nil); !bytes.Equal(g.out, actual) { + t.Errorf("hash(%q) = 0x%x want 0x%x", g.in, actual, g.out) + } + } +} + +func TestIntegrity32(t *testing.T) { + data := []byte{'1', '2', 3, 4, 5} + + h := New32() + h.Write(data) + sum := h.Sum(nil) + + if size := h.Size(); size != len(sum) { + t.Fatalf("Size()=%d but len(Sum())=%d", size, len(sum)) + } + + if a := h.Sum(nil); !bytes.Equal(sum, a) { + t.Fatalf("first Sum()=0x%x, second Sum()=0x%x", sum, a) + } + + h.Reset() + h.Write(data) + if a := h.Sum(nil); !bytes.Equal(sum, a) { + t.Fatalf("Sum()=0x%x, but after Reset() Sum()=0x%x", sum, a) + } + + h.Reset() + h.Write(data[:2]) + h.Write(data[2:]) + if a := h.Sum(nil); !bytes.Equal(sum, a) { + t.Fatalf("Sum()=0x%x, but with partial writes, Sum()=0x%x", sum, a) + } + + sum32 := h.(hash.Hash32).Sum32() + if sum32 != binary.BigEndian.Uint32(sum) { + t.Fatalf("Sum()=0x%x, but Sum32()=0x%x", sum, sum32) + } +} + +func BenchmarkJenkins32KB(b *testing.B) { + h := New32() + + b.SetBytes(1024) + data := make([]byte, 1024) + for i := range data { + data[i] = byte(i) + } + in := make([]byte, 0, h.Size()) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + h.Reset() + h.Write(data) + h.Sum(in) + } +} + +func BenchmarkFnv32(b *testing.B) { + arr := make([]int64, 1000) + for i := 0; i < b.N; i++ { + var payload [8]byte + binary.BigEndian.PutUint32(payload[:4], uint32(i)) + binary.BigEndian.PutUint32(payload[4:], uint32(i)) + + h := fnv.New32() + h.Write(payload[:]) + idx := int(h.Sum32()) % len(arr) + arr[idx]++ + } + b.StopTimer() + c := 0 + if b.N > 1000000 { + for i := 0; i < len(arr)-1; i++ { + if math.Abs(float64(arr[i]-arr[i+1]))/float64(arr[i]) > float64(0.1) { + if c == 0 { + b.Logf("i %d val[i] %d val[i+1] %d b.N %b\n", i, arr[i], arr[i+1], b.N) + } + c++ + } + } + if c > 0 { + b.Logf("Unbalanced buckets: %d", c) + } + } +} + +func BenchmarkSum32(b *testing.B) { + arr := make([]int64, 1000) + for i := 0; i < b.N; i++ { + var payload [8]byte + binary.BigEndian.PutUint32(payload[:4], uint32(i)) + binary.BigEndian.PutUint32(payload[4:], uint32(i)) + h := Sum32(0) + h.Write(payload[:]) + idx := int(h.Sum32()) % len(arr) + arr[idx]++ + } + b.StopTimer() + if b.N > 1000000 { + for i := 0; i < len(arr)-1; i++ { + if math.Abs(float64(arr[i]-arr[i+1]))/float64(arr[i]) > float64(0.1) { + b.Logf("val[%3d]=%8d\tval[%3d]=%8d\tb.N=%b\n", i, arr[i], i+1, arr[i+1], b.N) + break + } + } + } +} + +func BenchmarkNew32(b *testing.B) { + arr := make([]int64, 1000) + for i := 0; i < b.N; i++ { + var payload [8]byte + binary.BigEndian.PutUint32(payload[:4], uint32(i)) + binary.BigEndian.PutUint32(payload[4:], uint32(i)) + h := New32() + h.Write(payload[:]) + idx := int(h.Sum32()) % len(arr) + arr[idx]++ + } + b.StopTimer() + if b.N > 1000000 { + for i := 0; i < len(arr)-1; i++ { + if math.Abs(float64(arr[i]-arr[i+1]))/float64(arr[i]) > float64(0.1) { + b.Logf("val[%3d]=%8d\tval[%3d]=%8d\tb.N=%b\n", i, arr[i], i+1, arr[i+1], b.N) + break + } + } + } +} diff --git a/pkg/tcpip/ports/BUILD b/pkg/tcpip/ports/BUILD index c69fc0744..a2fa9b84a 100644 --- a/pkg/tcpip/ports/BUILD +++ b/pkg/tcpip/ports/BUILD @@ -7,7 +7,9 @@ go_library( srcs = ["ports.go"], importpath = "gvisor.googlesource.com/gvisor/pkg/tcpip/ports", visibility = ["//:sandbox"], - deps = ["//pkg/tcpip"], + deps = [ + "//pkg/tcpip", + ], ) go_test( diff --git a/pkg/tcpip/ports/ports.go b/pkg/tcpip/ports/ports.go index 41ef32921..d212a5792 100644 --- a/pkg/tcpip/ports/ports.go +++ b/pkg/tcpip/ports/ports.go @@ -42,23 +42,47 @@ type PortManager struct { allocatedPorts map[portDescriptor]bindAddresses } +type portNode struct { + reuse bool + refs int +} + // bindAddresses is a set of IP addresses. -type bindAddresses map[tcpip.Address]struct{} +type bindAddresses map[tcpip.Address]portNode // isAvailable checks whether an IP address is available to bind to. -func (b bindAddresses) isAvailable(addr tcpip.Address) bool { +func (b bindAddresses) isAvailable(addr tcpip.Address, reuse bool) bool { if addr == anyIPAddress { - return len(b) == 0 + if len(b) == 0 { + return true + } + if !reuse { + return false + } + for _, n := range b { + if !n.reuse { + return false + } + } + return true } // If all addresses for this portDescriptor are already bound, no // address is available. - if _, ok := b[anyIPAddress]; ok { - return false + if n, ok := b[anyIPAddress]; ok { + if !reuse { + return false + } + if !n.reuse { + return false + } } - if _, ok := b[addr]; ok { - return false + if n, ok := b[addr]; ok { + if !reuse { + return false + } + return n.reuse } return true } @@ -92,17 +116,17 @@ func (s *PortManager) PickEphemeralPort(testPort func(p uint16) (bool, *tcpip.Er } // IsPortAvailable tests if the given port is available on all given protocols. -func (s *PortManager) IsPortAvailable(networks []tcpip.NetworkProtocolNumber, transport tcpip.TransportProtocolNumber, addr tcpip.Address, port uint16) bool { +func (s *PortManager) IsPortAvailable(networks []tcpip.NetworkProtocolNumber, transport tcpip.TransportProtocolNumber, addr tcpip.Address, port uint16, reuse bool) bool { s.mu.Lock() defer s.mu.Unlock() - return s.isPortAvailableLocked(networks, transport, addr, port) + return s.isPortAvailableLocked(networks, transport, addr, port, reuse) } -func (s *PortManager) isPortAvailableLocked(networks []tcpip.NetworkProtocolNumber, transport tcpip.TransportProtocolNumber, addr tcpip.Address, port uint16) bool { +func (s *PortManager) isPortAvailableLocked(networks []tcpip.NetworkProtocolNumber, transport tcpip.TransportProtocolNumber, addr tcpip.Address, port uint16, reuse bool) bool { for _, network := range networks { desc := portDescriptor{network, transport, port} if addrs, ok := s.allocatedPorts[desc]; ok { - if !addrs.isAvailable(addr) { + if !addrs.isAvailable(addr, reuse) { return false } } @@ -114,14 +138,14 @@ func (s *PortManager) isPortAvailableLocked(networks []tcpip.NetworkProtocolNumb // reserved by another endpoint. If port is zero, ReservePort will search for // an unreserved ephemeral port and reserve it, returning its value in the // "port" return value. -func (s *PortManager) ReservePort(networks []tcpip.NetworkProtocolNumber, transport tcpip.TransportProtocolNumber, addr tcpip.Address, port uint16) (reservedPort uint16, err *tcpip.Error) { +func (s *PortManager) ReservePort(networks []tcpip.NetworkProtocolNumber, transport tcpip.TransportProtocolNumber, addr tcpip.Address, port uint16, reuse bool) (reservedPort uint16, err *tcpip.Error) { s.mu.Lock() defer s.mu.Unlock() // If a port is specified, just try to reserve it for all network // protocols. if port != 0 { - if !s.reserveSpecificPort(networks, transport, addr, port) { + if !s.reserveSpecificPort(networks, transport, addr, port, reuse) { return 0, tcpip.ErrPortInUse } return port, nil @@ -129,13 +153,13 @@ func (s *PortManager) ReservePort(networks []tcpip.NetworkProtocolNumber, transp // A port wasn't specified, so try to find one. return s.PickEphemeralPort(func(p uint16) (bool, *tcpip.Error) { - return s.reserveSpecificPort(networks, transport, addr, p), nil + return s.reserveSpecificPort(networks, transport, addr, p, reuse), nil }) } // reserveSpecificPort tries to reserve the given port on all given protocols. -func (s *PortManager) reserveSpecificPort(networks []tcpip.NetworkProtocolNumber, transport tcpip.TransportProtocolNumber, addr tcpip.Address, port uint16) bool { - if !s.isPortAvailableLocked(networks, transport, addr, port) { +func (s *PortManager) reserveSpecificPort(networks []tcpip.NetworkProtocolNumber, transport tcpip.TransportProtocolNumber, addr tcpip.Address, port uint16, reuse bool) bool { + if !s.isPortAvailableLocked(networks, transport, addr, port, reuse) { return false } @@ -147,7 +171,12 @@ func (s *PortManager) reserveSpecificPort(networks []tcpip.NetworkProtocolNumber m = make(bindAddresses) s.allocatedPorts[desc] = m } - m[addr] = struct{}{} + if n, ok := m[addr]; ok { + n.refs++ + m[addr] = n + } else { + m[addr] = portNode{reuse: reuse, refs: 1} + } } return true @@ -162,7 +191,16 @@ func (s *PortManager) ReleasePort(networks []tcpip.NetworkProtocolNumber, transp for _, network := range networks { desc := portDescriptor{network, transport, port} if m, ok := s.allocatedPorts[desc]; ok { - delete(m, addr) + n, ok := m[addr] + if !ok { + continue + } + n.refs-- + if n.refs == 0 { + delete(m, addr) + } else { + m[addr] = n + } if len(m) == 0 { delete(s.allocatedPorts, desc) } diff --git a/pkg/tcpip/ports/ports_test.go b/pkg/tcpip/ports/ports_test.go index 72577dfcb..01e7320b4 100644 --- a/pkg/tcpip/ports/ports_test.go +++ b/pkg/tcpip/ports/ports_test.go @@ -28,67 +28,99 @@ const ( fakeIPAddress1 = tcpip.Address("\x08\x08\x08\x09") ) -func TestPortReservation(t *testing.T) { - pm := NewPortManager() - net := []tcpip.NetworkProtocolNumber{fakeNetworkNumber} +type portReserveTestAction struct { + port uint16 + ip tcpip.Address + want *tcpip.Error + reuse bool + release bool +} +func TestPortReservation(t *testing.T) { for _, test := range []struct { - port uint16 - ip tcpip.Address - want *tcpip.Error + tname string + actions []portReserveTestAction }{ { - port: 80, - ip: fakeIPAddress, - want: nil, - }, - { - port: 80, - ip: fakeIPAddress1, - want: nil, - }, - { - /* N.B. Order of tests matters! */ - port: 80, - ip: anyIPAddress, - want: tcpip.ErrPortInUse, - }, - { - port: 22, - ip: anyIPAddress, - want: nil, - }, - { - port: 22, - ip: fakeIPAddress, - want: tcpip.ErrPortInUse, - }, - { - port: 0, - ip: fakeIPAddress, - want: nil, + tname: "bind to ip", + actions: []portReserveTestAction{ + {port: 80, ip: fakeIPAddress, want: nil}, + {port: 80, ip: fakeIPAddress1, want: nil}, + /* N.B. Order of tests matters! */ + {port: 80, ip: anyIPAddress, want: tcpip.ErrPortInUse}, + {port: 80, ip: fakeIPAddress, want: tcpip.ErrPortInUse, reuse: true}, + }, }, { - port: 0, - ip: fakeIPAddress, - want: nil, + tname: "bind to inaddr any", + actions: []portReserveTestAction{ + {port: 22, ip: anyIPAddress, want: nil}, + {port: 22, ip: fakeIPAddress, want: tcpip.ErrPortInUse}, + /* release fakeIPAddress, but anyIPAddress is still inuse */ + {port: 22, ip: fakeIPAddress, release: true}, + {port: 22, ip: fakeIPAddress, want: tcpip.ErrPortInUse}, + {port: 22, ip: fakeIPAddress, want: tcpip.ErrPortInUse, reuse: true}, + /* Release port 22 from any IP address, then try to reserve fake IP address on 22 */ + {port: 22, ip: anyIPAddress, want: nil, release: true}, + {port: 22, ip: fakeIPAddress, want: nil}, + }, + }, { + tname: "bind to zero port", + actions: []portReserveTestAction{ + {port: 00, ip: fakeIPAddress, want: nil}, + {port: 00, ip: fakeIPAddress, want: nil}, + {port: 00, ip: fakeIPAddress, reuse: true, want: nil}, + }, + }, { + tname: "bind to ip with reuseport", + actions: []portReserveTestAction{ + {port: 25, ip: fakeIPAddress, reuse: true, want: nil}, + {port: 25, ip: fakeIPAddress, reuse: true, want: nil}, + + {port: 25, ip: fakeIPAddress, reuse: false, want: tcpip.ErrPortInUse}, + {port: 25, ip: anyIPAddress, reuse: false, want: tcpip.ErrPortInUse}, + + {port: 25, ip: anyIPAddress, reuse: true, want: nil}, + }, + }, { + tname: "bind to inaddr any with reuseport", + actions: []portReserveTestAction{ + {port: 24, ip: anyIPAddress, reuse: true, want: nil}, + {port: 24, ip: anyIPAddress, reuse: true, want: nil}, + + {port: 24, ip: anyIPAddress, reuse: false, want: tcpip.ErrPortInUse}, + {port: 24, ip: fakeIPAddress, reuse: false, want: tcpip.ErrPortInUse}, + + {port: 24, ip: fakeIPAddress, reuse: true, want: nil}, + {port: 24, ip: fakeIPAddress, release: true, want: nil}, + + {port: 24, ip: anyIPAddress, release: true}, + {port: 24, ip: anyIPAddress, reuse: false, want: tcpip.ErrPortInUse}, + + {port: 24, ip: anyIPAddress, release: true}, + {port: 24, ip: anyIPAddress, reuse: false, want: nil}, + }, }, } { - gotPort, err := pm.ReservePort(net, fakeTransNumber, test.ip, test.port) - if err != test.want { - t.Fatalf("ReservePort(.., .., %s, %d) = %v, want %v", test.ip, test.port, err, test.want) - } - if test.port == 0 && (gotPort == 0 || gotPort < FirstEphemeral) { - t.Fatalf("ReservePort(.., .., .., 0) = %d, want port number >= %d to be picked", gotPort, FirstEphemeral) - } - } + t.Run(test.tname, func(t *testing.T) { + pm := NewPortManager() + net := []tcpip.NetworkProtocolNumber{fakeNetworkNumber} - // Release port 22 from any IP address, then try to reserve fake IP - // address on 22. - pm.ReleasePort(net, fakeTransNumber, anyIPAddress, 22) + for _, test := range test.actions { + if test.release { + pm.ReleasePort(net, fakeTransNumber, test.ip, test.port) + continue + } + gotPort, err := pm.ReservePort(net, fakeTransNumber, test.ip, test.port, test.reuse) + if err != test.want { + t.Fatalf("ReservePort(.., .., %s, %d, %t) = %v, want %v", test.ip, test.port, test.release, err, test.want) + } + if test.port == 0 && (gotPort == 0 || gotPort < FirstEphemeral) { + t.Fatalf("ReservePort(.., .., .., 0) = %d, want port number >= %d to be picked", gotPort, FirstEphemeral) + } + } + }) - if port, err := pm.ReservePort(net, fakeTransNumber, fakeIPAddress, 22); port != 22 || err != nil { - t.Fatalf("ReservePort(.., .., .., %d) = (port %d, err %v), want (22, nil); failed to reserve port after it should have been released", 22, port, err) } } diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD index 90cc05cda..9ff1c8731 100644 --- a/pkg/tcpip/stack/BUILD +++ b/pkg/tcpip/stack/BUILD @@ -22,6 +22,7 @@ go_library( "//pkg/sleep", "//pkg/tcpip", "//pkg/tcpip/buffer", + "//pkg/tcpip/hash/jenkins", "//pkg/tcpip/header", "//pkg/tcpip/ports", "//pkg/tcpip/seqnum", diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index 0ac116675..7aa9dbd46 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -883,9 +883,9 @@ func (s *Stack) RemoveWaker(nicid tcpip.NICID, addr tcpip.Address, waker *sleep. // transport dispatcher. Received packets that match the provided id will be // delivered to the given endpoint; specifying a nic is optional, but // nic-specific IDs have precedence over global ones. -func (s *Stack) RegisterTransportEndpoint(nicID tcpip.NICID, netProtos []tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, ep TransportEndpoint) *tcpip.Error { +func (s *Stack) RegisterTransportEndpoint(nicID tcpip.NICID, netProtos []tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, ep TransportEndpoint, reusePort bool) *tcpip.Error { if nicID == 0 { - return s.demux.registerEndpoint(netProtos, protocol, id, ep) + return s.demux.registerEndpoint(netProtos, protocol, id, ep, reusePort) } s.mu.RLock() @@ -896,14 +896,14 @@ func (s *Stack) RegisterTransportEndpoint(nicID tcpip.NICID, netProtos []tcpip.N return tcpip.ErrUnknownNICID } - return nic.demux.registerEndpoint(netProtos, protocol, id, ep) + return nic.demux.registerEndpoint(netProtos, protocol, id, ep, reusePort) } // UnregisterTransportEndpoint removes the endpoint with the given id from the // stack transport dispatcher. -func (s *Stack) UnregisterTransportEndpoint(nicID tcpip.NICID, netProtos []tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID) { +func (s *Stack) UnregisterTransportEndpoint(nicID tcpip.NICID, netProtos []tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, ep TransportEndpoint) { if nicID == 0 { - s.demux.unregisterEndpoint(netProtos, protocol, id) + s.demux.unregisterEndpoint(netProtos, protocol, id, ep) return } @@ -912,7 +912,7 @@ func (s *Stack) UnregisterTransportEndpoint(nicID tcpip.NICID, netProtos []tcpip nic := s.nics[nicID] if nic != nil { - nic.demux.unregisterEndpoint(netProtos, protocol, id) + nic.demux.unregisterEndpoint(netProtos, protocol, id, ep) } } diff --git a/pkg/tcpip/stack/transport_demuxer.go b/pkg/tcpip/stack/transport_demuxer.go index c8522ad9e..a5ff2159a 100644 --- a/pkg/tcpip/stack/transport_demuxer.go +++ b/pkg/tcpip/stack/transport_demuxer.go @@ -15,10 +15,12 @@ package stack import ( + "math/rand" "sync" "gvisor.googlesource.com/gvisor/pkg/tcpip" "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer" + "gvisor.googlesource.com/gvisor/pkg/tcpip/hash/jenkins" "gvisor.googlesource.com/gvisor/pkg/tcpip/header" ) @@ -34,6 +36,23 @@ type transportEndpoints struct { endpoints map[TransportEndpointID]TransportEndpoint } +// unregisterEndpoint unregisters the endpoint with the given id such that it +// won't receive any more packets. +func (eps *transportEndpoints) unregisterEndpoint(id TransportEndpointID, ep TransportEndpoint) { + eps.mu.Lock() + defer eps.mu.Unlock() + e, ok := eps.endpoints[id] + if !ok { + return + } + if multiPortEp, ok := e.(*multiPortEndpoint); ok { + if !multiPortEp.unregisterEndpoint(ep) { + return + } + } + delete(eps.endpoints, id) +} + // transportDemuxer demultiplexes packets targeted at a transport endpoint // (i.e., after they've been parsed by the network layer). It does two levels // of demultiplexing: first based on the network and transport protocols, then @@ -57,10 +76,10 @@ func newTransportDemuxer(stack *Stack) *transportDemuxer { // registerEndpoint registers the given endpoint with the dispatcher such that // packets that match the endpoint ID are delivered to it. -func (d *transportDemuxer) registerEndpoint(netProtos []tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, ep TransportEndpoint) *tcpip.Error { +func (d *transportDemuxer) registerEndpoint(netProtos []tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, ep TransportEndpoint, reusePort bool) *tcpip.Error { for i, n := range netProtos { - if err := d.singleRegisterEndpoint(n, protocol, id, ep); err != nil { - d.unregisterEndpoint(netProtos[:i], protocol, id) + if err := d.singleRegisterEndpoint(n, protocol, id, ep, reusePort); err != nil { + d.unregisterEndpoint(netProtos[:i], protocol, id, ep) return err } } @@ -68,7 +87,97 @@ func (d *transportDemuxer) registerEndpoint(netProtos []tcpip.NetworkProtocolNum return nil } -func (d *transportDemuxer) singleRegisterEndpoint(netProto tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, ep TransportEndpoint) *tcpip.Error { +// multiPortEndpoint is a container for TransportEndpoints which are bound to +// the same pair of address and port. +type multiPortEndpoint struct { + mu sync.RWMutex + endpointsArr []TransportEndpoint + endpointsMap map[TransportEndpoint]int + // seed is a random secret for a jenkins hash. + seed uint32 +} + +// reciprocalScale scales a value into range [0, n). +// +// This is similar to val % n, but faster. +// See http://lemire.me/blog/2016/06/27/a-fast-alternative-to-the-modulo-reduction/ +func reciprocalScale(val, n uint32) uint32 { + return uint32((uint64(val) * uint64(n)) >> 32) +} + +// selectEndpoint calculates a hash of destination and source addresses and +// ports then uses it to select a socket. In this case, all packets from one +// address will be sent to same endpoint. +func (ep *multiPortEndpoint) selectEndpoint(id TransportEndpointID) TransportEndpoint { + ep.mu.RLock() + defer ep.mu.RUnlock() + + payload := []byte{ + byte(id.LocalPort), + byte(id.LocalPort >> 8), + byte(id.RemotePort), + byte(id.RemotePort >> 8), + } + + h := jenkins.Sum32(ep.seed) + h.Write(payload) + h.Write([]byte(id.LocalAddress)) + h.Write([]byte(id.RemoteAddress)) + hash := h.Sum32() + + idx := reciprocalScale(hash, uint32(len(ep.endpointsArr))) + return ep.endpointsArr[idx] +} + +// HandlePacket is called by the stack when new packets arrive to this transport +// endpoint. +func (ep *multiPortEndpoint) HandlePacket(r *Route, id TransportEndpointID, vv buffer.VectorisedView) { + ep.selectEndpoint(id).HandlePacket(r, id, vv) +} + +// HandleControlPacket implements stack.TransportEndpoint.HandleControlPacket. +func (ep *multiPortEndpoint) HandleControlPacket(id TransportEndpointID, typ ControlType, extra uint32, vv buffer.VectorisedView) { + ep.selectEndpoint(id).HandleControlPacket(id, typ, extra, vv) +} + +func (ep *multiPortEndpoint) singleRegisterEndpoint(t TransportEndpoint) { + ep.mu.Lock() + defer ep.mu.Unlock() + + // A new endpoint is added into endpointsArr and its index there is + // saved in endpointsMap. This will allows to remove endpoint from + // the array fast. + ep.endpointsMap[ep] = len(ep.endpointsArr) + ep.endpointsArr = append(ep.endpointsArr, t) +} + +// unregisterEndpoint returns true if multiPortEndpoint has to be unregistered. +func (ep *multiPortEndpoint) unregisterEndpoint(t TransportEndpoint) bool { + ep.mu.Lock() + defer ep.mu.Unlock() + + idx, ok := ep.endpointsMap[t] + if !ok { + return false + } + delete(ep.endpointsMap, t) + l := len(ep.endpointsArr) + if l > 1 { + // The last endpoint in endpointsArr is moved instead of the deleted one. + lastEp := ep.endpointsArr[l-1] + ep.endpointsArr[idx] = lastEp + ep.endpointsMap[lastEp] = idx + ep.endpointsArr = ep.endpointsArr[0 : l-1] + return false + } + return true +} + +func (d *transportDemuxer) singleRegisterEndpoint(netProto tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, ep TransportEndpoint, reusePort bool) *tcpip.Error { + if id.RemotePort != 0 { + reusePort = false + } + eps, ok := d.protocol[protocolIDs{netProto, protocol}] if !ok { return nil @@ -77,10 +186,29 @@ func (d *transportDemuxer) singleRegisterEndpoint(netProto tcpip.NetworkProtocol eps.mu.Lock() defer eps.mu.Unlock() + var multiPortEp *multiPortEndpoint if _, ok := eps.endpoints[id]; ok { - return tcpip.ErrPortInUse + if !reusePort { + return tcpip.ErrPortInUse + } + multiPortEp, ok = eps.endpoints[id].(*multiPortEndpoint) + if !ok { + return tcpip.ErrPortInUse + } } + if reusePort { + if multiPortEp == nil { + multiPortEp = &multiPortEndpoint{} + multiPortEp.endpointsMap = make(map[TransportEndpoint]int) + multiPortEp.seed = rand.Uint32() + eps.endpoints[id] = multiPortEp + } + + multiPortEp.singleRegisterEndpoint(ep) + + return nil + } eps.endpoints[id] = ep return nil @@ -88,12 +216,10 @@ func (d *transportDemuxer) singleRegisterEndpoint(netProto tcpip.NetworkProtocol // unregisterEndpoint unregisters the endpoint with the given id such that it // won't receive any more packets. -func (d *transportDemuxer) unregisterEndpoint(netProtos []tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID) { +func (d *transportDemuxer) unregisterEndpoint(netProtos []tcpip.NetworkProtocolNumber, protocol tcpip.TransportProtocolNumber, id TransportEndpointID, ep TransportEndpoint) { for _, n := range netProtos { if eps, ok := d.protocol[protocolIDs{n, protocol}]; ok { - eps.mu.Lock() - delete(eps.endpoints, id) - eps.mu.Unlock() + eps.unregisterEndpoint(id, ep) } } } diff --git a/pkg/tcpip/stack/transport_test.go b/pkg/tcpip/stack/transport_test.go index f09760180..022207081 100644 --- a/pkg/tcpip/stack/transport_test.go +++ b/pkg/tcpip/stack/transport_test.go @@ -107,7 +107,7 @@ func (f *fakeTransportEndpoint) Connect(addr tcpip.FullAddress) *tcpip.Error { // Try to register so that we can start receiving packets. f.id.RemoteAddress = addr.Addr - err = f.stack.RegisterTransportEndpoint(0, []tcpip.NetworkProtocolNumber{fakeNetNumber}, fakeTransNumber, f.id, f) + err = f.stack.RegisterTransportEndpoint(0, []tcpip.NetworkProtocolNumber{fakeNetNumber}, fakeTransNumber, f.id, f, false) if err != nil { return err } diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go index 627786808..7d4fbe075 100644 --- a/pkg/tcpip/tcpip.go +++ b/pkg/tcpip/tcpip.go @@ -436,6 +436,10 @@ type CorkOption int // should allow reuse of local address. type ReuseAddressOption int +// ReusePortOption is used by SetSockOpt/GetSockOpt to permit multiple sockets +// to be bound to an identical socket address. +type ReusePortOption int + // QuickAckOption is stubbed out in SetSockOpt/GetSockOpt. type QuickAckOption int diff --git a/pkg/tcpip/transport/ping/endpoint.go b/pkg/tcpip/transport/ping/endpoint.go index d1b9b136c..29f6c543d 100644 --- a/pkg/tcpip/transport/ping/endpoint.go +++ b/pkg/tcpip/transport/ping/endpoint.go @@ -100,7 +100,7 @@ func (e *endpoint) Close() { e.shutdownFlags = tcpip.ShutdownRead | tcpip.ShutdownWrite switch e.state { case stateBound, stateConnected: - e.stack.UnregisterTransportEndpoint(e.regNICID, []tcpip.NetworkProtocolNumber{e.netProto}, e.transProto, e.id) + e.stack.UnregisterTransportEndpoint(e.regNICID, []tcpip.NetworkProtocolNumber{e.netProto}, e.transProto, e.id, e) } // Close the receive list and drain it. @@ -541,14 +541,14 @@ func (e *endpoint) registerWithStack(nicid tcpip.NICID, netProtos []tcpip.Networ if id.LocalPort != 0 { // The endpoint already has a local port, just attempt to // register it. - err := e.stack.RegisterTransportEndpoint(nicid, netProtos, e.transProto, id, e) + err := e.stack.RegisterTransportEndpoint(nicid, netProtos, e.transProto, id, e, false) return id, err } // We need to find a port for the endpoint. _, err := e.stack.PickEphemeralPort(func(p uint16) (bool, *tcpip.Error) { id.LocalPort = p - err := e.stack.RegisterTransportEndpoint(nicid, netProtos, e.transProto, id, e) + err := e.stack.RegisterTransportEndpoint(nicid, netProtos, e.transProto, id, e, false) switch err { case nil: return true, nil @@ -597,7 +597,7 @@ func (e *endpoint) bindLocked(addr tcpip.FullAddress, commit func() *tcpip.Error if commit != nil { if err := commit(); err != nil { // Unregister, the commit failed. - e.stack.UnregisterTransportEndpoint(addr.NIC, netProtos, e.transProto, id) + e.stack.UnregisterTransportEndpoint(addr.NIC, netProtos, e.transProto, id, e) return err } } diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go index d0e1d6782..78d2c76e0 100644 --- a/pkg/tcpip/transport/tcp/accept.go +++ b/pkg/tcpip/transport/tcp/accept.go @@ -215,7 +215,7 @@ func (l *listenContext) createConnectedEndpoint(s *segment, iss seqnum.Value, ir n.maybeEnableSACKPermitted(rcvdSynOpts) // Register new endpoint so that packets are routed to it. - if err := n.stack.RegisterTransportEndpoint(n.boundNICID, n.effectiveNetProtos, ProtocolNumber, n.id, n); err != nil { + if err := n.stack.RegisterTransportEndpoint(n.boundNICID, n.effectiveNetProtos, ProtocolNumber, n.id, n, n.reusePort); err != nil { n.Close() return nil, err } diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go index d4eda50ec..5281f8be2 100644 --- a/pkg/tcpip/transport/tcp/endpoint.go +++ b/pkg/tcpip/transport/tcp/endpoint.go @@ -162,6 +162,9 @@ type endpoint struct { // sack holds TCP SACK related information for this endpoint. sack SACKInfo + // reusePort is set to true if SO_REUSEPORT is enabled. + reusePort bool + // delay enables Nagle's algorithm. // // delay is a boolean (0 is false) and must be accessed atomically. @@ -416,7 +419,7 @@ func (e *endpoint) Close() { e.isPortReserved = false if e.isRegistered { - e.stack.UnregisterTransportEndpoint(e.boundNICID, e.effectiveNetProtos, ProtocolNumber, e.id) + e.stack.UnregisterTransportEndpoint(e.boundNICID, e.effectiveNetProtos, ProtocolNumber, e.id, e) e.isRegistered = false } } @@ -453,7 +456,7 @@ func (e *endpoint) cleanupLocked() { e.workerCleanup = false if e.isRegistered { - e.stack.UnregisterTransportEndpoint(e.boundNICID, e.effectiveNetProtos, ProtocolNumber, e.id) + e.stack.UnregisterTransportEndpoint(e.boundNICID, e.effectiveNetProtos, ProtocolNumber, e.id, e) } e.route.Release() @@ -681,6 +684,12 @@ func (e *endpoint) SetSockOpt(opt interface{}) *tcpip.Error { e.mu.Unlock() return nil + case tcpip.ReusePortOption: + e.mu.Lock() + e.reusePort = v != 0 + e.mu.Unlock() + return nil + case tcpip.QuickAckOption: if v == 0 { atomic.StoreUint32(&e.slowAck, 1) @@ -875,6 +884,17 @@ func (e *endpoint) GetSockOpt(opt interface{}) *tcpip.Error { } return nil + case *tcpip.ReusePortOption: + e.mu.RLock() + v := e.reusePort + e.mu.RUnlock() + + *o = 0 + if v { + *o = 1 + } + return nil + case *tcpip.QuickAckOption: *o = 1 if v := atomic.LoadUint32(&e.slowAck); v != 0 { @@ -1057,7 +1077,7 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) (er if e.id.LocalPort != 0 { // The endpoint is bound to a port, attempt to register it. - err := e.stack.RegisterTransportEndpoint(nicid, netProtos, ProtocolNumber, e.id, e) + err := e.stack.RegisterTransportEndpoint(nicid, netProtos, ProtocolNumber, e.id, e, e.reusePort) if err != nil { return err } @@ -1071,13 +1091,13 @@ func (e *endpoint) connect(addr tcpip.FullAddress, handshake bool, run bool) (er if sameAddr && p == e.id.RemotePort { return false, nil } - if !e.stack.IsPortAvailable(netProtos, ProtocolNumber, e.id.LocalAddress, p) { + if !e.stack.IsPortAvailable(netProtos, ProtocolNumber, e.id.LocalAddress, p, false) { return false, nil } id := e.id id.LocalPort = p - switch e.stack.RegisterTransportEndpoint(nicid, netProtos, ProtocolNumber, id, e) { + switch e.stack.RegisterTransportEndpoint(nicid, netProtos, ProtocolNumber, id, e, e.reusePort) { case nil: e.id = id return true, nil @@ -1234,7 +1254,7 @@ func (e *endpoint) Listen(backlog int) (err *tcpip.Error) { } // Register the endpoint. - if err := e.stack.RegisterTransportEndpoint(e.boundNICID, e.effectiveNetProtos, ProtocolNumber, e.id, e); err != nil { + if err := e.stack.RegisterTransportEndpoint(e.boundNICID, e.effectiveNetProtos, ProtocolNumber, e.id, e, e.reusePort); err != nil { return err } @@ -1315,7 +1335,7 @@ func (e *endpoint) Bind(addr tcpip.FullAddress, commit func() *tcpip.Error) (err } } - port, err := e.stack.ReservePort(netProtos, ProtocolNumber, addr.Addr, addr.Port) + port, err := e.stack.ReservePort(netProtos, ProtocolNumber, addr.Addr, addr.Port, e.reusePort) if err != nil { return err } diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go index 67e9ca0ac..b2a27a7cb 100644 --- a/pkg/tcpip/transport/udp/endpoint.go +++ b/pkg/tcpip/transport/udp/endpoint.go @@ -81,6 +81,7 @@ type endpoint struct { dstPort uint16 v6only bool multicastTTL uint8 + reusePort bool // shutdownFlags represent the current shutdown state of the endpoint. shutdownFlags tcpip.ShutdownFlags @@ -132,7 +133,7 @@ func NewConnectedEndpoint(stack *stack.Stack, r *stack.Route, id stack.Transport ep := newEndpoint(stack, r.NetProto, waiterQueue) // Register new endpoint so that packets are routed to it. - if err := stack.RegisterTransportEndpoint(r.NICID(), []tcpip.NetworkProtocolNumber{r.NetProto}, ProtocolNumber, id, ep); err != nil { + if err := stack.RegisterTransportEndpoint(r.NICID(), []tcpip.NetworkProtocolNumber{r.NetProto}, ProtocolNumber, id, ep, ep.reusePort); err != nil { ep.Close() return nil, err } @@ -155,7 +156,7 @@ func (e *endpoint) Close() { switch e.state { case stateBound, stateConnected: - e.stack.UnregisterTransportEndpoint(e.regNICID, e.effectiveNetProtos, ProtocolNumber, e.id) + e.stack.UnregisterTransportEndpoint(e.regNICID, e.effectiveNetProtos, ProtocolNumber, e.id, e) e.stack.ReleasePort(e.effectiveNetProtos, ProtocolNumber, e.id.LocalAddress, e.id.LocalPort) } @@ -449,6 +450,12 @@ func (e *endpoint) SetSockOpt(opt interface{}) *tcpip.Error { break } } + + case tcpip.ReusePortOption: + e.mu.Lock() + e.reusePort = v != 0 + e.mu.Unlock() + return nil } return nil } @@ -513,6 +520,17 @@ func (e *endpoint) GetSockOpt(opt interface{}) *tcpip.Error { e.mu.Unlock() return nil + case *tcpip.ReusePortOption: + e.mu.RLock() + v := e.reusePort + e.mu.RUnlock() + + *o = 0 + if v { + *o = 1 + } + return nil + case *tcpip.KeepaliveEnabledOption: *o = 0 return nil @@ -648,7 +666,7 @@ func (e *endpoint) Connect(addr tcpip.FullAddress) *tcpip.Error { // Remove the old registration. if e.id.LocalPort != 0 { - e.stack.UnregisterTransportEndpoint(e.regNICID, e.effectiveNetProtos, ProtocolNumber, e.id) + e.stack.UnregisterTransportEndpoint(e.regNICID, e.effectiveNetProtos, ProtocolNumber, e.id, e) } e.id = id @@ -711,14 +729,14 @@ func (*endpoint) Accept() (tcpip.Endpoint, *waiter.Queue, *tcpip.Error) { func (e *endpoint) registerWithStack(nicid tcpip.NICID, netProtos []tcpip.NetworkProtocolNumber, id stack.TransportEndpointID) (stack.TransportEndpointID, *tcpip.Error) { if e.id.LocalPort == 0 { - port, err := e.stack.ReservePort(netProtos, ProtocolNumber, id.LocalAddress, id.LocalPort) + port, err := e.stack.ReservePort(netProtos, ProtocolNumber, id.LocalAddress, id.LocalPort, e.reusePort) if err != nil { return id, err } id.LocalPort = port } - err := e.stack.RegisterTransportEndpoint(nicid, netProtos, ProtocolNumber, id, e) + err := e.stack.RegisterTransportEndpoint(nicid, netProtos, ProtocolNumber, id, e, e.reusePort) if err != nil { e.stack.ReleasePort(netProtos, ProtocolNumber, id.LocalAddress, id.LocalPort) } @@ -766,7 +784,7 @@ func (e *endpoint) bindLocked(addr tcpip.FullAddress, commit func() *tcpip.Error if commit != nil { if err := commit(); err != nil { // Unregister, the commit failed. - e.stack.UnregisterTransportEndpoint(addr.NIC, netProtos, ProtocolNumber, id) + e.stack.UnregisterTransportEndpoint(addr.NIC, netProtos, ProtocolNumber, id, e) e.stack.ReleasePort(netProtos, ProtocolNumber, id.LocalAddress, id.LocalPort) return err } diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go index 58a346cd9..2a9cf4b57 100644 --- a/pkg/tcpip/transport/udp/udp_test.go +++ b/pkg/tcpip/transport/udp/udp_test.go @@ -16,6 +16,7 @@ package udp_test import ( "bytes" + "math" "math/rand" "testing" "time" @@ -254,6 +255,90 @@ func newPayload() []byte { return b } +func TestBindPortReuse(t *testing.T) { + c := newDualTestContext(t, defaultMTU) + defer c.cleanup() + + c.createV6Endpoint(false) + + var eps [5]tcpip.Endpoint + reusePortOpt := tcpip.ReusePortOption(1) + + pollChannel := make(chan tcpip.Endpoint) + for i := 0; i < len(eps); i++ { + // Try to receive the data. + wq := waiter.Queue{} + we, ch := waiter.NewChannelEntry(nil) + wq.EventRegister(&we, waiter.EventIn) + defer wq.EventUnregister(&we) + defer close(ch) + + var err *tcpip.Error + eps[i], err = c.s.NewEndpoint(udp.ProtocolNumber, ipv6.ProtocolNumber, &wq) + if err != nil { + c.t.Fatalf("NewEndpoint failed: %v", err) + } + + go func(ep tcpip.Endpoint) { + for range ch { + pollChannel <- ep + } + }(eps[i]) + + defer eps[i].Close() + if err := eps[i].SetSockOpt(reusePortOpt); err != nil { + c.t.Fatalf("SetSockOpt failed failed: %v", err) + } + if err := eps[i].Bind(tcpip.FullAddress{Addr: stackV6Addr, Port: stackPort}, nil); err != nil { + t.Fatalf("ep.Bind(...) failed: %v", err) + } + } + + npackets := 100000 + nports := 10000 + ports := make(map[uint16]tcpip.Endpoint) + stats := make(map[tcpip.Endpoint]int) + for i := 0; i < npackets; i++ { + // Send a packet. + port := uint16(i % nports) + payload := newPayload() + c.sendV6Packet(payload, &headers{ + srcPort: testPort + port, + dstPort: stackPort, + }) + + var addr tcpip.FullAddress + ep := <-pollChannel + _, _, err := ep.Read(&addr) + if err != nil { + c.t.Fatalf("Read failed: %v", err) + } + stats[ep]++ + if i < nports { + ports[uint16(i)] = ep + } else { + // Check that all packets from one client are handled + // by the same socket. + if ports[port] != ep { + t.Fatalf("Port mismatch") + } + } + } + + if len(stats) != len(eps) { + t.Fatalf("Only %d(expected %d) sockets received packets", len(stats), len(eps)) + } + + // Check that a packet distribution is fair between sockets. + for _, c := range stats { + n := float64(npackets) / float64(len(eps)) + // The deviation is less than 10%. + if math.Abs(float64(c)-n) > n/10 { + t.Fatal(c, n) + } + } +} + func testV4Read(c *testContext) { // Send a packet. payload := newPayload() diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD index ae33d14da..f0e61e083 100644 --- a/test/syscalls/linux/BUILD +++ b/test/syscalls/linux/BUILD @@ -2163,9 +2163,13 @@ cc_binary( ":socket_test_util", "//test/util:file_descriptor", "//test/util:posix_error", + "//test/util:save_util", "//test/util:test_main", "//test/util:test_util", + "//test/util:thread_util", + "@com_google_absl//absl/memory", "@com_google_absl//absl/strings", + "@com_google_absl//absl/time", "@com_google_googletest//:gtest", ], ) diff --git a/test/syscalls/linux/socket_inet_loopback.cc b/test/syscalls/linux/socket_inet_loopback.cc index 17a46e149..0893be5a7 100644 --- a/test/syscalls/linux/socket_inet_loopback.cc +++ b/test/syscalls/linux/socket_inet_loopback.cc @@ -17,17 +17,24 @@ #include #include +#include +#include #include #include #include #include +#include "gmock/gmock.h" #include "gtest/gtest.h" +#include "absl/memory/memory.h" #include "absl/strings/str_cat.h" +#include "absl/time/time.h" #include "test/syscalls/linux/socket_test_util.h" #include "test/util/file_descriptor.h" #include "test/util/posix_error.h" +#include "test/util/save_util.h" #include "test/util/test_util.h" +#include "test/util/thread_util.h" namespace gvisor { namespace testing { @@ -227,6 +234,238 @@ INSTANTIATE_TEST_CASE_P( TestParam{V6Loopback(), V6Loopback()}), DescribeTestParam); +using SocketInetReusePortTest = ::testing::TestWithParam; + +TEST_P(SocketInetReusePortTest, TcpPortReuseMultiThread) { + auto const& param = GetParam(); + + TestAddress const& listener = param.listener; + TestAddress const& connector = param.connector; + sockaddr_storage listen_addr = listener.addr; + sockaddr_storage conn_addr = connector.addr; + constexpr int kThreadCount = 3; + + // Create the listening socket. + FileDescriptor listener_fds[kThreadCount]; + for (int i = 0; i < kThreadCount; i++) { + listener_fds[i] = ASSERT_NO_ERRNO_AND_VALUE( + Socket(listener.family(), SOCK_STREAM, IPPROTO_TCP)); + int fd = listener_fds[i].get(); + + ASSERT_THAT(setsockopt(fd, SOL_SOCKET, SO_REUSEPORT, &kSockOptOn, + sizeof(kSockOptOn)), + SyscallSucceeds()); + ASSERT_THAT( + bind(fd, reinterpret_cast(&listen_addr), listener.addr_len), + SyscallSucceeds()); + ASSERT_THAT(listen(fd, 40), SyscallSucceeds()); + + // On the first bind we need to determine which port was bound. + if (i != 0) continue; + + // Get the port bound by the listening socket. + socklen_t addrlen = listener.addr_len; + ASSERT_THAT( + getsockname(listener_fds[0].get(), + reinterpret_cast(&listen_addr), &addrlen), + SyscallSucceeds()); + uint16_t const port = + ASSERT_NO_ERRNO_AND_VALUE(AddrPort(listener.family(), listen_addr)); + ASSERT_NO_ERRNO(SetAddrPort(listener.family(), &listen_addr, port)); + ASSERT_NO_ERRNO(SetAddrPort(connector.family(), &conn_addr, port)); + } + + constexpr int kConnectAttempts = 10000; + std::atomic connects_received = ATOMIC_VAR_INIT(0); + std::unique_ptr listen_thread[kThreadCount]; + int accept_counts[kThreadCount] = {}; + // TODO: figure how to not disable S/R for the whole test. + // We need to take into account that this test executes a lot of system + // calls from many threads. + DisableSave ds; + + for (int i = 0; i < kThreadCount; i++) { + listen_thread[i] = absl::make_unique( + [&listener_fds, &accept_counts, i, &connects_received]() { + do { + auto fd = Accept(listener_fds[i].get(), nullptr, nullptr); + if (!fd.ok()) { + if (connects_received >= kConnectAttempts) { + // Another thread have shutdown our read side causing the + // accept to fail. + break; + } + ASSERT_NO_ERRNO(fd); + break; + } + // Receive some data from a socket to be sure that the connect() + // system call has been completed on another side. + int data; + EXPECT_THAT( + RetryEINTR(recv)(fd.ValueOrDie().get(), &data, sizeof(data), 0), + SyscallSucceedsWithValue(sizeof(data))); + accept_counts[i]++; + } while (++connects_received < kConnectAttempts); + + // Shutdown all sockets to wake up other threads. + for (int j = 0; j < kThreadCount; j++) { + shutdown(listener_fds[j].get(), SHUT_RDWR); + } + }); + } + + ScopedThread connecting_thread([&connector, &conn_addr]() { + for (int i = 0; i < kConnectAttempts; i++) { + const FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE( + Socket(connector.family(), SOCK_STREAM, IPPROTO_TCP)); + ASSERT_THAT( + RetryEINTR(connect)(fd.get(), reinterpret_cast(&conn_addr), + connector.addr_len), + SyscallSucceeds()); + + EXPECT_THAT(RetryEINTR(send)(fd.get(), &i, sizeof(i), 0), + SyscallSucceedsWithValue(sizeof(i))); + } + }); + + // Join threads to be sure that all connections have been counted + connecting_thread.Join(); + for (int i = 0; i < kThreadCount; i++) { + listen_thread[i]->Join(); + } + // Check that connections are distributed fairly between listening sockets + for (int i = 0; i < kThreadCount; i++) + EXPECT_THAT(accept_counts[i], + EquivalentWithin((kConnectAttempts / kThreadCount), 0.10)); +} + +TEST_P(SocketInetReusePortTest, UdpPortReuseMultiThread) { + auto const& param = GetParam(); + + TestAddress const& listener = param.listener; + TestAddress const& connector = param.connector; + sockaddr_storage listen_addr = listener.addr; + sockaddr_storage conn_addr = connector.addr; + constexpr int kThreadCount = 3; + + // Create the listening socket. + FileDescriptor listener_fds[kThreadCount]; + for (int i = 0; i < kThreadCount; i++) { + listener_fds[i] = + ASSERT_NO_ERRNO_AND_VALUE(Socket(listener.family(), SOCK_DGRAM, 0)); + int fd = listener_fds[i].get(); + + ASSERT_THAT(setsockopt(fd, SOL_SOCKET, SO_REUSEPORT, &kSockOptOn, + sizeof(kSockOptOn)), + SyscallSucceeds()); + ASSERT_THAT( + bind(fd, reinterpret_cast(&listen_addr), listener.addr_len), + SyscallSucceeds()); + + // On the first bind we need to determine which port was bound. + if (i != 0) continue; + + // Get the port bound by the listening socket. + socklen_t addrlen = listener.addr_len; + ASSERT_THAT( + getsockname(listener_fds[0].get(), + reinterpret_cast(&listen_addr), &addrlen), + SyscallSucceeds()); + uint16_t const port = + ASSERT_NO_ERRNO_AND_VALUE(AddrPort(listener.family(), listen_addr)); + ASSERT_NO_ERRNO(SetAddrPort(listener.family(), &listen_addr, port)); + ASSERT_NO_ERRNO(SetAddrPort(connector.family(), &conn_addr, port)); + } + + constexpr int kConnectAttempts = 10000; + std::atomic packets_received = ATOMIC_VAR_INIT(0); + std::unique_ptr receiver_thread[kThreadCount]; + int packets_per_socket[kThreadCount] = {}; + // TODO: figure how to not disable S/R for the whole test. + DisableSave ds; // Too expensive. + + for (int i = 0; i < kThreadCount; i++) { + receiver_thread[i] = absl::make_unique( + [&listener_fds, &packets_per_socket, i, &packets_received]() { + do { + struct sockaddr_storage addr = {}; + socklen_t addrlen = sizeof(addr); + int data; + + auto ret = RetryEINTR(recvfrom)( + listener_fds[i].get(), &data, sizeof(data), 0, + reinterpret_cast(&addr), &addrlen); + + if (packets_received < kConnectAttempts) { + ASSERT_THAT(ret, SyscallSucceedsWithValue(sizeof(data))); + } + + if (ret != sizeof(data)) { + // Another thread may have shutdown our read side causing the + // recvfrom to fail. + break; + } + + packets_received++; + packets_per_socket[i]++; + + // A response is required to synchronize with the main thread, + // otherwise the main thread can send more than can fit into receive + // queues. + EXPECT_THAT(RetryEINTR(sendto)( + listener_fds[i].get(), &data, sizeof(data), 0, + reinterpret_cast(&addr), addrlen), + SyscallSucceedsWithValue(sizeof(data))); + } while (packets_received < kConnectAttempts); + + // Shutdown all sockets to wake up other threads. + for (int j = 0; j < kThreadCount; j++) + shutdown(listener_fds[j].get(), SHUT_RDWR); + }); + } + + ScopedThread main_thread([&connector, &conn_addr]() { + for (int i = 0; i < kConnectAttempts; i++) { + const FileDescriptor fd = + ASSERT_NO_ERRNO_AND_VALUE(Socket(connector.family(), SOCK_DGRAM, 0)); + EXPECT_THAT(RetryEINTR(sendto)(fd.get(), &i, sizeof(i), 0, + reinterpret_cast(&conn_addr), + connector.addr_len), + SyscallSucceedsWithValue(sizeof(i))); + int data; + EXPECT_THAT(RetryEINTR(recv)(fd.get(), &data, sizeof(data), 0), + SyscallSucceedsWithValue(sizeof(data))); + } + }); + + main_thread.Join(); + + // Join threads to be sure that all connections have been counted + for (int i = 0; i < kThreadCount; i++) { + receiver_thread[i]->Join(); + } + // Check that packets are distributed fairly between listening sockets. + for (int i = 0; i < kThreadCount; i++) + EXPECT_THAT(packets_per_socket[i], + EquivalentWithin((kConnectAttempts / kThreadCount), 0.10)); +} + +INSTANTIATE_TEST_CASE_P( + All, SocketInetReusePortTest, + ::testing::Values( + // Listeners bound to IPv4 addresses refuse connections using IPv6 + // addresses. + TestParam{V4Any(), V4Loopback()}, + TestParam{V4Loopback(), V4MappedLoopback()}, + + // Listeners bound to IN6ADDR_ANY accept all connections. + TestParam{V6Any(), V4Loopback()}, TestParam{V6Any(), V6Loopback()}, + + // Listeners bound to IN6ADDR_LOOPBACK refuse connections using IPv4 + // addresses. + TestParam{V6Loopback(), V6Loopback()}), + DescribeTestParam); + struct ProtocolTestParam { std::string description; int type; @@ -806,6 +1045,56 @@ TEST_P(SocketMultiProtocolInetLoopbackTest, V4EphemeralPortReserved) { } } +TEST_P(SocketMultiProtocolInetLoopbackTest, PortReuseTwoSockets) { + auto const& param = GetParam(); + TestAddress const& test_addr = V4Loopback(); + sockaddr_storage addr = test_addr.addr; + + for (int i = 0; i < 2; i++) { + const int portreuse1 = i % 2; + auto s1 = + ASSERT_NO_ERRNO_AND_VALUE(Socket(test_addr.family(), param.type, 0)); + int fd1 = s1.get(); + socklen_t addrlen = test_addr.addr_len; + + EXPECT_THAT( + setsockopt(fd1, SOL_SOCKET, SO_REUSEPORT, &portreuse1, sizeof(int)), + SyscallSucceeds()); + + ASSERT_THAT(bind(fd1, reinterpret_cast(&addr), addrlen), + SyscallSucceeds()); + + ASSERT_THAT(getsockname(fd1, reinterpret_cast(&addr), &addrlen), + SyscallSucceeds()); + if (param.type == SOCK_STREAM) { + ASSERT_THAT(listen(fd1, 1), SyscallSucceeds()); + } + + // j is less than 4 to check that the port reuse logic works correctly after + // closing bound sockets. + for (int j = 0; j < 4; j++) { + const int portreuse2 = j % 2; + auto s2 = + ASSERT_NO_ERRNO_AND_VALUE(Socket(test_addr.family(), param.type, 0)); + int fd2 = s2.get(); + + EXPECT_THAT( + setsockopt(fd2, SOL_SOCKET, SO_REUSEPORT, &portreuse2, sizeof(int)), + SyscallSucceeds()); + + LOG(INFO) << portreuse1 << " " << portreuse2; + int ret = bind(fd2, reinterpret_cast(&addr), addrlen); + + // Verify that two sockets can be bound to the same port only if + // SO_REUSEPORT is set for both of them. + if (!portreuse1 || !portreuse2) + ASSERT_THAT(ret, SyscallFailsWithErrno(EADDRINUSE)); + else + ASSERT_THAT(ret, SyscallSucceeds()); + } + } +} + INSTANTIATE_TEST_CASE_P(AllFamlies, SocketMultiProtocolInetLoopbackTest, ::testing::Values(ProtocolTestParam{"TCP", SOCK_STREAM}, ProtocolTestParam{"UDP", SOCK_DGRAM}), diff --git a/test/syscalls/syscall_test_runner.go b/test/syscalls/syscall_test_runner.go index 9ee0361ee..ec048f10f 100644 --- a/test/syscalls/syscall_test_runner.go +++ b/test/syscalls/syscall_test_runner.go @@ -118,6 +118,7 @@ func runTestCaseRunsc(testBin string, tc gtest.TestCase, t *testing.T) { // Mark the root as writeable, as some tests attempt to // write to the rootfs, and expect EACCES, not EROFS. spec.Root.Readonly = false + spec.Mounts = nil // Set environment variable that indicates we are // running in gVisor and with the given platform. -- cgit v1.2.3 From dc8450b5676d4c4ac9bcfa23cabd862e0060527d Mon Sep 17 00:00:00 2001 From: Nicolas Lacasse Date: Mon, 14 Jan 2019 20:33:29 -0800 Subject: Remove fs.Handle, ramfs.Entry, and all the DeprecatedFileOperations. More helper structs have been added to the fsutil package to make it easier to implement fs.InodeOperations and fs.FileOperations. PiperOrigin-RevId: 229305982 Change-Id: Ib6f8d3862f4216745116857913dbfa351530223b --- pkg/abi/linux/fs.go | 2 + pkg/sentry/fs/BUILD | 3 +- pkg/sentry/fs/anon/anon.go | 16 +- pkg/sentry/fs/ashmem/BUILD | 1 + pkg/sentry/fs/ashmem/area.go | 16 +- pkg/sentry/fs/ashmem/device.go | 133 +-------- pkg/sentry/fs/attr.go | 47 +++ pkg/sentry/fs/binder/BUILD | 2 +- pkg/sentry/fs/binder/binder.go | 135 +-------- pkg/sentry/fs/dev/BUILD | 1 + pkg/sentry/fs/dev/dev.go | 29 +- pkg/sentry/fs/dev/fs.go | 2 + pkg/sentry/fs/dev/full.go | 55 ++-- pkg/sentry/fs/dev/null.go | 88 ++++-- pkg/sentry/fs/dev/random.go | 55 ++-- pkg/sentry/fs/dirent.go | 8 +- pkg/sentry/fs/fdpipe/pipe.go | 14 +- pkg/sentry/fs/file_operations.go | 2 +- pkg/sentry/fs/file_overlay_test.go | 7 +- pkg/sentry/fs/filetest/filetest.go | 16 +- pkg/sentry/fs/fsutil/BUILD | 15 - pkg/sentry/fs/fsutil/file.go | 226 ++++++++++----- pkg/sentry/fs/fsutil/fsutil.go | 2 - pkg/sentry/fs/fsutil/handle.go | 128 --------- pkg/sentry/fs/fsutil/handle_test.go | 227 --------------- pkg/sentry/fs/fsutil/inode.go | 409 ++++++++++++++------------ pkg/sentry/fs/fsutil/inode_cached_test.go | 14 +- pkg/sentry/fs/gofer/file.go | 2 +- pkg/sentry/fs/gofer/fs.go | 14 +- pkg/sentry/fs/gofer/inode.go | 1 - pkg/sentry/fs/host/file.go | 4 +- pkg/sentry/fs/host/fs.go | 2 + pkg/sentry/fs/host/inode.go | 1 - pkg/sentry/fs/inode.go | 29 +- pkg/sentry/fs/inode_operations.go | 80 ------ pkg/sentry/fs/inode_overlay.go | 13 - pkg/sentry/fs/inode_overlay_test.go | 62 +++- pkg/sentry/fs/mock.go | 11 - pkg/sentry/fs/mount.go | 14 +- pkg/sentry/fs/mounts_test.go | 15 +- pkg/sentry/fs/proc/BUILD | 4 +- pkg/sentry/fs/proc/cpuinfo.go | 41 +-- pkg/sentry/fs/proc/exec_args.go | 57 +++- pkg/sentry/fs/proc/fds.go | 138 +++++---- pkg/sentry/fs/proc/file.go | 58 ---- pkg/sentry/fs/proc/inode.go | 96 +++++++ pkg/sentry/fs/proc/net.go | 55 ++-- pkg/sentry/fs/proc/proc.go | 152 +++++----- pkg/sentry/fs/proc/rpcinet_proc.go | 246 ++++++++-------- pkg/sentry/fs/proc/seqfile/BUILD | 7 +- pkg/sentry/fs/proc/seqfile/seqfile.go | 133 ++++++--- pkg/sentry/fs/proc/seqfile/seqfile_test.go | 45 +-- pkg/sentry/fs/proc/sys.go | 116 +++++--- pkg/sentry/fs/proc/sys_net.go | 325 +++++++++++++-------- pkg/sentry/fs/proc/sys_net_state.go | 17 +- pkg/sentry/fs/proc/sys_net_test.go | 28 +- pkg/sentry/fs/proc/task.go | 264 +++++++++++------ pkg/sentry/fs/proc/uid_gid_map.go | 17 +- pkg/sentry/fs/proc/uptime.go | 40 ++- pkg/sentry/fs/ramfs/BUILD | 6 +- pkg/sentry/fs/ramfs/dir.go | 223 +++++++++++---- pkg/sentry/fs/ramfs/file.go | 150 ---------- pkg/sentry/fs/ramfs/ramfs.go | 441 ----------------------------- pkg/sentry/fs/ramfs/socket.go | 48 +++- pkg/sentry/fs/ramfs/symlink.go | 67 +++-- pkg/sentry/fs/ramfs/test/BUILD | 16 -- pkg/sentry/fs/ramfs/test/test.go | 46 --- pkg/sentry/fs/ramfs/tree.go | 3 +- pkg/sentry/fs/ramfs/tree_test.go | 2 +- pkg/sentry/fs/sys/BUILD | 3 +- pkg/sentry/fs/sys/devices.go | 51 ++-- pkg/sentry/fs/sys/fs.go | 2 + pkg/sentry/fs/sys/sys.go | 10 +- pkg/sentry/fs/timerfd/timerfd.go | 12 +- pkg/sentry/fs/tmpfs/BUILD | 2 + pkg/sentry/fs/tmpfs/file_regular.go | 14 +- pkg/sentry/fs/tmpfs/file_test.go | 6 +- pkg/sentry/fs/tmpfs/fs.go | 2 + pkg/sentry/fs/tmpfs/inode_file.go | 112 ++++---- pkg/sentry/fs/tmpfs/tmpfs.go | 164 +++++++++-- pkg/sentry/fs/tty/BUILD | 2 - pkg/sentry/fs/tty/dir.go | 108 ++----- pkg/sentry/fs/tty/inode.go | 145 ---------- pkg/sentry/fs/tty/master.go | 23 +- pkg/sentry/fs/tty/slave.go | 25 +- pkg/sentry/kernel/epoll/epoll.go | 12 +- pkg/sentry/kernel/eventfd/eventfd.go | 14 +- pkg/sentry/kernel/pipe/node.go | 40 +-- pkg/sentry/kernel/pipe/node_test.go | 36 +-- pkg/sentry/kernel/pipe/pipe.go | 31 +- pkg/sentry/kernel/pipe/reader_writer.go | 10 +- pkg/sentry/loader/vdso.go | 48 ++-- pkg/sentry/socket/epsocket/epsocket.go | 10 +- pkg/sentry/socket/hostinet/socket.go | 10 +- pkg/sentry/socket/netlink/socket.go | 10 +- pkg/sentry/socket/rpcinet/socket.go | 10 +- pkg/sentry/socket/socket.go | 18 +- pkg/sentry/socket/unix/unix.go | 10 +- runsc/boot/fs.go | 4 +- test/syscalls/linux/proc.cc | 5 + 100 files changed, 2547 insertions(+), 3144 deletions(-) delete mode 100644 pkg/sentry/fs/fsutil/handle.go delete mode 100644 pkg/sentry/fs/fsutil/handle_test.go delete mode 100644 pkg/sentry/fs/proc/file.go create mode 100644 pkg/sentry/fs/proc/inode.go delete mode 100644 pkg/sentry/fs/ramfs/file.go delete mode 100644 pkg/sentry/fs/ramfs/ramfs.go delete mode 100644 pkg/sentry/fs/ramfs/test/BUILD delete mode 100644 pkg/sentry/fs/ramfs/test/test.go delete mode 100644 pkg/sentry/fs/tty/inode.go (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/abi/linux/fs.go b/pkg/abi/linux/fs.go index 0b1c9f3db..a9f2ba132 100644 --- a/pkg/abi/linux/fs.go +++ b/pkg/abi/linux/fs.go @@ -22,8 +22,10 @@ const ( DEVPTS_SUPER_MAGIC = 0x00001cd1 OVERLAYFS_SUPER_MAGIC = 0x794c7630 PIPEFS_MAGIC = 0x50495045 + PROC_SUPER_MAGIC = 0x9fa0 RAMFS_MAGIC = 0x09041934 SOCKFS_MAGIC = 0x534F434B + SYSFS_MAGIC = 0x62656572 TMPFS_MAGIC = 0x01021994 V9FS_MAGIC = 0x01021997 ) diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD index 0fe2b14bf..6f368b0da 100644 --- a/pkg/sentry/fs/BUILD +++ b/pkg/sentry/fs/BUILD @@ -94,7 +94,8 @@ go_test( deps = [ ":fs", "//pkg/sentry/context", - "//pkg/sentry/fs/ramfs/test", + "//pkg/sentry/fs/fsutil", + "//pkg/sentry/fs/ramfs", "//pkg/sentry/fs/tmpfs", "//pkg/sentry/kernel/contexttest", "//pkg/sentry/usermem", diff --git a/pkg/sentry/fs/anon/anon.go b/pkg/sentry/fs/anon/anon.go index 743cf511f..a5e8c4f0d 100644 --- a/pkg/sentry/fs/anon/anon.go +++ b/pkg/sentry/fs/anon/anon.go @@ -28,16 +28,12 @@ import ( // with any real filesystem. Some types depend on completely pseudo // "anon" inodes (eventfds, epollfds, etc). func NewInode(ctx context.Context) *fs.Inode { - return fs.NewInode(fsutil.NewSimpleInodeOperations(fsutil.InodeSimpleAttributes{ - FSType: linux.ANON_INODE_FS_MAGIC, - UAttr: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: fs.FileOwnerFromContext(ctx), - Perms: fs.FilePermissions{ - User: fs.PermMask{Read: true, Write: true}, - }, - Links: 1, - }), - }), fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}), fs.StableAttr{ + iops := &fsutil.SimpleFileInode{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, fs.RootOwner, fs.FilePermissions{ + User: fs.PermMask{Read: true, Write: true}, + }, linux.ANON_INODE_FS_MAGIC), + } + return fs.NewInode(iops, fs.NewPseudoMountSource(), fs.StableAttr{ Type: fs.Anonymous, DeviceID: PseudoDevice.DeviceID(), InodeID: PseudoDevice.NextIno(), diff --git a/pkg/sentry/fs/ashmem/BUILD b/pkg/sentry/fs/ashmem/BUILD index 44ef82e64..2463111a8 100644 --- a/pkg/sentry/fs/ashmem/BUILD +++ b/pkg/sentry/fs/ashmem/BUILD @@ -28,6 +28,7 @@ go_library( "//pkg/sentry/usage", "//pkg/sentry/usermem", "//pkg/syserror", + "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go index d7dd2c084..7c1b11464 100644 --- a/pkg/sentry/fs/ashmem/area.go +++ b/pkg/sentry/fs/ashmem/area.go @@ -28,6 +28,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/usage" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) const ( @@ -42,9 +43,10 @@ const ( // // +stateify savable type Area struct { - fsutil.NoFsync `state:"nosave"` - fsutil.DeprecatedFileOperations `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` ad *Device @@ -98,11 +100,6 @@ func (a *Area) Write(ctx context.Context, file *fs.File, src usermem.IOSequence, return 0, syserror.ENOSYS } -// Flush implements fs.FileOperations.Flush. -func (a *Area) Flush(ctx context.Context, file *fs.File) error { - return nil -} - // ConfigureMMap implements fs.FileOperations.ConfigureMMap. func (a *Area) ConfigureMMap(ctx context.Context, file *fs.File, opts *memmap.MMapOpts) error { a.mu.Lock() @@ -122,8 +119,7 @@ func (a *Area) ConfigureMMap(ctx context.Context, file *fs.File, opts *memmap.MM return syserror.ENOMEM } tmpfsInodeOps := tmpfs.NewInMemoryFile(ctx, usage.Tmpfs, fs.UnstableAttr{}, k) - // This is not backed by a real filesystem, so we pass in nil. - tmpfsInode := fs.NewInode(tmpfsInodeOps, fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}), fs.StableAttr{}) + tmpfsInode := fs.NewInode(tmpfsInodeOps, fs.NewPseudoMountSource(), fs.StableAttr{}) dirent := fs.NewDirent(tmpfsInode, namePrefix+"/"+a.name) tmpfsFile, err := tmpfsInode.GetFile(ctx, dirent, fs.FileFlags{Read: true, Write: true}) // Drop the extra reference on the Dirent. diff --git a/pkg/sentry/fs/ashmem/device.go b/pkg/sentry/fs/ashmem/device.go index 962da141b..5369d1b0d 100644 --- a/pkg/sentry/fs/ashmem/device.go +++ b/pkg/sentry/fs/ashmem/device.go @@ -16,49 +16,40 @@ package ashmem import ( - "sync" - + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" - "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/syserror" ) // Device implements fs.InodeOperations. // // +stateify savable type Device struct { - fsutil.DeprecatedFileOperations `state:"nosave"` + fsutil.InodeGenericChecker `state:"nosave"` fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopTruncate `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` fsutil.InodeNotDirectory `state:"nosave"` - fsutil.InodeNotRenameable `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` fsutil.InodeNotSocket `state:"nosave"` fsutil.InodeNotSymlink `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoMappable `state:"nosave"` - fsutil.NoopWriteOut `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` - mu sync.Mutex `state:"nosave"` - unstable fs.UnstableAttr + fsutil.InodeSimpleAttributes } +var _ fs.InodeOperations = (*Device)(nil) + // NewDevice creates and intializes a Device structure. func NewDevice(ctx context.Context, owner fs.FileOwner, fp fs.FilePermissions) *Device { return &Device{ - unstable: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: owner, - Perms: fp, - Links: 1, - }), + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, fp, linux.ANON_INODE_FS_MAGIC), } } -// Release implements fs.InodeOperations.Release. -func (ad *Device) Release(context.Context) {} - // GetFile implements fs.InodeOperations.GetFile. func (ad *Device) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { return fs.NewFile(ctx, d, flags, &Area{ @@ -67,105 +58,3 @@ func (ad *Device) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) perms: usermem.AnyAccess, }), nil } - -// UnstableAttr implements fs.InodeOperations.UnstableAttr. -func (ad *Device) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - ad.mu.Lock() - defer ad.mu.Unlock() - return ad.unstable, nil -} - -// Check implements fs.InodeOperations.Check. -func (ad *Device) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { - return fs.ContextCanAccessFile(ctx, inode, p) -} - -// SetPermissions implements fs.InodeOperations.SetPermissions. -func (ad *Device) SetPermissions(ctx context.Context, inode *fs.Inode, fp fs.FilePermissions) bool { - ad.mu.Lock() - defer ad.mu.Unlock() - ad.unstable.Perms = fp - ad.unstable.StatusChangeTime = time.NowFromContext(ctx) - return true -} - -// SetOwner implements fs.InodeOperations.SetOwner. -func (ad *Device) SetOwner(ctx context.Context, inode *fs.Inode, owner fs.FileOwner) error { - ad.mu.Lock() - defer ad.mu.Unlock() - if owner.UID.Ok() { - ad.unstable.Owner.UID = owner.UID - } - if owner.GID.Ok() { - ad.unstable.Owner.GID = owner.GID - } - return nil -} - -// SetTimestamps implements fs.InodeOperations.SetTimestamps. -func (ad *Device) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error { - if ts.ATimeOmit && ts.MTimeOmit { - return nil - } - - ad.mu.Lock() - defer ad.mu.Unlock() - - now := time.NowFromContext(ctx) - if !ts.ATimeOmit { - if ts.ATimeSetSystemTime { - ad.unstable.AccessTime = now - } else { - ad.unstable.AccessTime = ts.ATime - } - } - if !ts.MTimeOmit { - if ts.MTimeSetSystemTime { - ad.unstable.ModificationTime = now - } else { - ad.unstable.ModificationTime = ts.MTime - } - } - ad.unstable.StatusChangeTime = now - return nil -} - -// Truncate implements fs.InodeOperations.WriteOut. -// -// Ignored by ashmem. -func (ad *Device) Truncate(ctx context.Context, inode *fs.Inode, size int64) error { - return nil -} - -// AddLink implements fs.InodeOperations.AddLink. -// -// Ashmem doesn't support links, no-op. -func (ad *Device) AddLink() {} - -// DropLink implements fs.InodeOperations.DropLink. -// -// Ashmem doesn't support links, no-op. -func (ad *Device) DropLink() {} - -// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. -func (ad *Device) NotifyStatusChange(ctx context.Context) { - ad.mu.Lock() - defer ad.mu.Unlock() - now := time.NowFromContext(ctx) - ad.unstable.ModificationTime = now - ad.unstable.StatusChangeTime = now -} - -// IsVirtual implements fs.InodeOperations.IsVirtual. -// -// Ashmem is virtual. -func (ad *Device) IsVirtual() bool { - return true -} - -// StatFS implements fs.InodeOperations.StatFS. -// -// Ashmem doesn't support querying for filesystem info. -func (ad *Device) StatFS(context.Context) (fs.Info, error) { - return fs.Info{}, syserror.ENOSYS -} diff --git a/pkg/sentry/fs/attr.go b/pkg/sentry/fs/attr.go index 59e060e3c..3523b068a 100644 --- a/pkg/sentry/fs/attr.go +++ b/pkg/sentry/fs/attr.go @@ -180,6 +180,53 @@ type UnstableAttr struct { Links uint64 } +// SetOwner sets the owner and group if they are valid. +// +// This method is NOT thread-safe. Callers must prevent concurrent calls. +func (ua *UnstableAttr) SetOwner(ctx context.Context, owner FileOwner) { + if owner.UID.Ok() { + ua.Owner.UID = owner.UID + } + if owner.GID.Ok() { + ua.Owner.GID = owner.GID + } + ua.StatusChangeTime = ktime.NowFromContext(ctx) +} + +// SetPermissions sets the permissions. +// +// This method is NOT thread-safe. Callers must prevent concurrent calls. +func (ua *UnstableAttr) SetPermissions(ctx context.Context, p FilePermissions) { + ua.Perms = p + ua.StatusChangeTime = ktime.NowFromContext(ctx) +} + +// SetTimestamps sets the timestamps according to the TimeSpec. +// +// This method is NOT thread-safe. Callers must prevent concurrent calls. +func (ua *UnstableAttr) SetTimestamps(ctx context.Context, ts TimeSpec) { + if ts.ATimeOmit && ts.MTimeOmit { + return + } + + now := ktime.NowFromContext(ctx) + if !ts.ATimeOmit { + if ts.ATimeSetSystemTime { + ua.AccessTime = now + } else { + ua.AccessTime = ts.ATime + } + } + if !ts.MTimeOmit { + if ts.MTimeSetSystemTime { + ua.ModificationTime = now + } else { + ua.ModificationTime = ts.MTime + } + } + ua.StatusChangeTime = now +} + // WithCurrentTime returns u with AccessTime == ModificationTime == current time. func WithCurrentTime(ctx context.Context, u UnstableAttr) UnstableAttr { t := ktime.NowFromContext(ctx) diff --git a/pkg/sentry/fs/binder/BUILD b/pkg/sentry/fs/binder/BUILD index a077b91d2..27155819e 100644 --- a/pkg/sentry/fs/binder/BUILD +++ b/pkg/sentry/fs/binder/BUILD @@ -16,11 +16,11 @@ go_library( "//pkg/sentry/fs", "//pkg/sentry/fs/fsutil", "//pkg/sentry/kernel", - "//pkg/sentry/kernel/time", "//pkg/sentry/memmap", "//pkg/sentry/platform", "//pkg/sentry/usage", "//pkg/sentry/usermem", "//pkg/syserror", + "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go index e642c7f22..19cd55e65 100644 --- a/pkg/sentry/fs/binder/binder.go +++ b/pkg/sentry/fs/binder/binder.go @@ -24,12 +24,12 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" - "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" "gvisor.googlesource.com/gvisor/pkg/sentry/platform" "gvisor.googlesource.com/gvisor/pkg/sentry/usage" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) const ( @@ -43,34 +43,29 @@ const ( // // +stateify savable type Device struct { + fsutil.InodeGenericChecker `state:"nosave"` fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopTruncate `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` fsutil.InodeNotDirectory `state:"nosave"` - fsutil.InodeNotRenameable `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` fsutil.InodeNotSocket `state:"nosave"` fsutil.InodeNotSymlink `state:"nosave"` - fsutil.NoMappable `state:"nosave"` - fsutil.NoopWriteOut `state:"nosave"` - fsutil.DeprecatedFileOperations `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` - // mu protects unstable. - mu sync.Mutex `state:"nosave"` - unstable fs.UnstableAttr + fsutil.InodeSimpleAttributes } +var _ fs.InodeOperations = (*Device)(nil) + // NewDevice creates and intializes a Device structure. func NewDevice(ctx context.Context, owner fs.FileOwner, fp fs.FilePermissions) *Device { return &Device{ - unstable: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: owner, - Perms: fp, - Links: 1, - }), + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, fp, 0), } } -// Release implements fs.InodeOperations.Release. -func (bd *Device) Release(context.Context) {} - // GetFile implements fs.InodeOperations.GetFile. // // TODO: Add functionality to GetFile: Additional fields will be @@ -85,115 +80,13 @@ func (bd *Device) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) }), nil } -// UnstableAttr implements fs.InodeOperations.UnstableAttr. -func (bd *Device) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - bd.mu.Lock() - defer bd.mu.Unlock() - return bd.unstable, nil -} - -// Check implements fs.InodeOperations.Check. -func (bd *Device) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { - return fs.ContextCanAccessFile(ctx, inode, p) -} - -// SetPermissions implements fs.InodeOperations.SetPermissions. -func (bd *Device) SetPermissions(ctx context.Context, inode *fs.Inode, fp fs.FilePermissions) bool { - bd.mu.Lock() - defer bd.mu.Unlock() - bd.unstable.Perms = fp - bd.unstable.StatusChangeTime = time.NowFromContext(ctx) - return true -} - -// SetOwner implements fs.InodeOperations.SetOwner. -func (bd *Device) SetOwner(ctx context.Context, inode *fs.Inode, owner fs.FileOwner) error { - bd.mu.Lock() - defer bd.mu.Unlock() - if owner.UID.Ok() { - bd.unstable.Owner.UID = owner.UID - } - if owner.GID.Ok() { - bd.unstable.Owner.GID = owner.GID - } - return nil -} - -// SetTimestamps implements fs.InodeOperations.SetTimestamps. -func (bd *Device) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error { - if ts.ATimeOmit && ts.MTimeOmit { - return nil - } - - bd.mu.Lock() - defer bd.mu.Unlock() - - now := time.NowFromContext(ctx) - if !ts.ATimeOmit { - if ts.ATimeSetSystemTime { - bd.unstable.AccessTime = now - } else { - bd.unstable.AccessTime = ts.ATime - } - } - if !ts.MTimeOmit { - if ts.MTimeSetSystemTime { - bd.unstable.ModificationTime = now - } else { - bd.unstable.ModificationTime = ts.MTime - } - } - bd.unstable.StatusChangeTime = now - return nil -} - -// Truncate implements fs.InodeOperations.WriteOut. -// -// Ignored for a character device, such as Binder. -func (bd *Device) Truncate(ctx context.Context, inode *fs.Inode, size int64) error { - return nil -} - -// AddLink implements fs.InodeOperations.AddLink. -// -// Binder doesn't support links, no-op. -func (bd *Device) AddLink() {} - -// DropLink implements fs.InodeOperations.DropLink. -// -// Binder doesn't support links, no-op. -func (bd *Device) DropLink() {} - -// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. -func (bd *Device) NotifyStatusChange(ctx context.Context) { - bd.mu.Lock() - defer bd.mu.Unlock() - now := time.NowFromContext(ctx) - bd.unstable.ModificationTime = now - bd.unstable.StatusChangeTime = now -} - -// IsVirtual implements fs.InodeOperations.IsVirtual. -// -// Binder is virtual. -func (bd *Device) IsVirtual() bool { - return true -} - -// StatFS implements fs.InodeOperations.StatFS. -// -// Binder doesn't support querying for filesystem info. -func (bd *Device) StatFS(context.Context) (fs.Info, error) { - return fs.Info{}, syserror.ENOSYS -} - // Proc implements fs.FileOperations and fs.IoctlGetter. // // +stateify savable type Proc struct { - fsutil.NoFsync `state:"nosave"` - fsutil.DeprecatedFileOperations `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` bd *Device task *kernel.Task diff --git a/pkg/sentry/fs/dev/BUILD b/pkg/sentry/fs/dev/BUILD index b17b5202c..b9cfae05f 100644 --- a/pkg/sentry/fs/dev/BUILD +++ b/pkg/sentry/fs/dev/BUILD @@ -32,5 +32,6 @@ go_library( "//pkg/sentry/safemem", "//pkg/sentry/usermem", "//pkg/syserror", + "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/dev/dev.go b/pkg/sentry/fs/dev/dev.go index 3e127bf04..f8e8099f7 100644 --- a/pkg/sentry/fs/dev/dev.go +++ b/pkg/sentry/fs/dev/dev.go @@ -16,6 +16,8 @@ package dev import ( + "math" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ashmem" @@ -26,13 +28,6 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" ) -// Dev is the root node. -// -// +stateify savable -type Dev struct { - ramfs.Dir -} - func newCharacterDevice(iops fs.InodeOperations, msrc *fs.MountSource) *fs.Inode { return fs.NewInode(iops, msrc, fs.StableAttr{ DeviceID: devDevice.DeviceID(), @@ -43,8 +38,7 @@ func newCharacterDevice(iops fs.InodeOperations, msrc *fs.MountSource) *fs.Inode } func newDirectory(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - iops := &ramfs.Dir{} - iops.InitDir(ctx, map[string]*fs.Inode{}, fs.RootOwner, fs.FilePermsFromMode(0555)) + iops := ramfs.NewDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) return fs.NewInode(iops, msrc, fs.StableAttr{ DeviceID: devDevice.DeviceID(), InodeID: devDevice.NextIno(), @@ -54,8 +48,7 @@ func newDirectory(ctx context.Context, msrc *fs.MountSource) *fs.Inode { } func newSymlink(ctx context.Context, target string, msrc *fs.MountSource) *fs.Inode { - iops := &ramfs.Symlink{} - iops.InitSymlink(ctx, fs.RootOwner, target) + iops := ramfs.NewSymlink(ctx, fs.RootOwner, target) return fs.NewInode(iops, msrc, fs.StableAttr{ DeviceID: devDevice.DeviceID(), InodeID: devDevice.NextIno(), @@ -66,8 +59,6 @@ func newSymlink(ctx context.Context, target string, msrc *fs.MountSource) *fs.In // New returns the root node of a device filesystem. func New(ctx context.Context, msrc *fs.MountSource, binderEnabled bool, ashmemEnabled bool) *fs.Inode { - d := &Dev{} - contents := map[string]*fs.Inode{ "fd": newSymlink(ctx, "/proc/self/fd", msrc), "stdin": newSymlink(ctx, "/proc/self/fd/0", msrc), @@ -114,11 +105,19 @@ func New(ctx context.Context, msrc *fs.MountSource, binderEnabled bool, ashmemEn contents["ashmem"] = newCharacterDevice(ashmem, msrc) } - d.InitDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) - return fs.NewInode(d, msrc, fs.StableAttr{ + iops := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return fs.NewInode(iops, msrc, fs.StableAttr{ DeviceID: devDevice.DeviceID(), InodeID: devDevice.NextIno(), BlockSize: usermem.PageSize, Type: fs.Directory, }) } + +// readZeros implements fs.FileOperations.Read with infinite null bytes. +type readZeros struct{} + +// Read implements fs.FileOperations.Read. +func (readZeros) Read(ctx context.Context, file *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { + return dst.ZeroOut(ctx, math.MaxInt64) +} diff --git a/pkg/sentry/fs/dev/fs.go b/pkg/sentry/fs/dev/fs.go index d96f4f423..abfe689f0 100644 --- a/pkg/sentry/fs/dev/fs.go +++ b/pkg/sentry/fs/dev/fs.go @@ -33,6 +33,8 @@ const ashmemEnabledKey = "ashmem_enabled" // +stateify savable type filesystem struct{} +var _ fs.Filesystem = (*filesystem)(nil) + func init() { fs.RegisterFilesystem(&filesystem{}) } diff --git a/pkg/sentry/fs/dev/full.go b/pkg/sentry/fs/dev/full.go index eeda646ab..cbdd40161 100644 --- a/pkg/sentry/fs/dev/full.go +++ b/pkg/sentry/fs/dev/full.go @@ -15,41 +15,64 @@ package dev import ( - "math" - "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // fullDevice is used to implement /dev/full. // // +stateify savable type fullDevice struct { - ramfs.Entry + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopTruncate `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes } +var _ fs.InodeOperations = (*fullDevice)(nil) + func newFullDevice(ctx context.Context, owner fs.FileOwner, mode linux.FileMode) *fullDevice { - f := &fullDevice{} - f.InitEntry(ctx, owner, fs.FilePermsFromMode(mode)) + f := &fullDevice{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, fs.FilePermsFromMode(mode), linux.TMPFS_MAGIC), + } return f } -// DeprecatedPwritev implements fs.InodeOperations.DeprecatedPwritev by -// returining ENOSPC. -func (f *fullDevice) DeprecatedPwritev(_ context.Context, _ usermem.IOSequence, _ int64) (int64, error) { - return 0, syserror.ENOSPC +// GetFile implements fs.InodeOperations.GetFile. +func (f *fullDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + flags.Pread = true + return fs.NewFile(ctx, dirent, flags, &fullFileOperations{}), nil } -// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. -func (f *fullDevice) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, _ int64) (int64, error) { - return dst.ZeroOut(ctx, math.MaxInt64) +// +stateify savable +type fullFileOperations struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + readZeros `state:"nosave"` } -// Truncate should be simply ignored for character devices on linux. -func (f *fullDevice) Truncate(context.Context, *fs.Inode, int64) error { - return nil +var _ fs.FileOperations = (*fullFileOperations)(nil) + +// Write implements FileOperations.Write. +func (fullFileOperations) Write(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { + return 0, syserror.ENOSPC } diff --git a/pkg/sentry/fs/dev/null.go b/pkg/sentry/fs/dev/null.go index 68090f353..73fd09058 100644 --- a/pkg/sentry/fs/dev/null.go +++ b/pkg/sentry/fs/dev/null.go @@ -15,78 +15,104 @@ package dev import ( - "io" - "math" - "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" "gvisor.googlesource.com/gvisor/pkg/sentry/mm" "gvisor.googlesource.com/gvisor/pkg/sentry/platform" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // +stateify savable type nullDevice struct { - ramfs.Entry + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopTruncate `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes } +var _ fs.InodeOperations = (*nullDevice)(nil) + func newNullDevice(ctx context.Context, owner fs.FileOwner, mode linux.FileMode) *nullDevice { - n := &nullDevice{} - n.InitEntry(ctx, owner, fs.FilePermsFromMode(mode)) + n := &nullDevice{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, fs.FilePermsFromMode(mode), linux.TMPFS_MAGIC), + } return n } -// DeprecatedPreadv reads data from the device. -func (n *nullDevice) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - return 0, io.EOF -} +// GetFile implements fs.FileOperations.GetFile. +func (n *nullDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + flags.Pread = true + flags.Pwrite = true -// DeprecatedPwritev discards writes. -func (n *nullDevice) DeprecatedPwritev(_ context.Context, src usermem.IOSequence, offset int64) (int64, error) { - return src.NumBytes(), nil + return fs.NewFile(ctx, dirent, flags, &nullFileOperations{}), nil } -// Truncate should be simply ignored for character devices on linux. -func (n *nullDevice) Truncate(context.Context, *fs.Inode, int64) error { - return nil +// +stateify savable +type nullFileOperations struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRead `state:"nosave"` + fsutil.FileNoopWrite `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` } +var _ fs.FileOperations = (*nullFileOperations)(nil) + // +stateify savable type zeroDevice struct { nullDevice } +var _ fs.InodeOperations = (*zeroDevice)(nil) + func newZeroDevice(ctx context.Context, owner fs.FileOwner, mode linux.FileMode) *zeroDevice { - zd := &zeroDevice{} - zd.InitEntry(ctx, owner, fs.FilePermsFromMode(mode)) + zd := &zeroDevice{ + nullDevice: nullDevice{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, fs.FilePermsFromMode(mode), linux.TMPFS_MAGIC), + }, + } return zd } -// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. -func (zd *zeroDevice) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - return dst.ZeroOut(ctx, math.MaxInt64) -} - -// GetFile overrides ramfs.Entry.GetFile and returns a zeroFile instead. +// GetFile implements fs.FileOperations.GetFile. func (zd *zeroDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { - // Allow pread(2) and pwrite(2) on this file. flags.Pread = true flags.Pwrite = true - return fs.NewFile(ctx, dirent, flags, &zeroFileOperations{ - FileOperations: &fsutil.Handle{HandleOperations: dirent.Inode.HandleOps()}, - }), nil + return fs.NewFile(ctx, dirent, flags, &zeroFileOperations{}), nil } // +stateify savable type zeroFileOperations struct { - fs.FileOperations + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopWrite `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + readZeros `state:"nosave"` } +var _ fs.FileOperations = (*zeroFileOperations)(nil) + // ConfigureMMap implements fs.FileOperations.ConfigureMMap. func (*zeroFileOperations) ConfigureMMap(ctx context.Context, file *fs.File, opts *memmap.MMapOpts) error { m, err := mm.NewSharedAnonMappable(opts.Length, platform.FromContext(ctx)) diff --git a/pkg/sentry/fs/dev/random.go b/pkg/sentry/fs/dev/random.go index 33e4913e4..837b7793a 100644 --- a/pkg/sentry/fs/dev/random.go +++ b/pkg/sentry/fs/dev/random.go @@ -19,37 +19,58 @@ import ( "gvisor.googlesource.com/gvisor/pkg/rand" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // +stateify savable type randomDevice struct { - ramfs.Entry + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopTruncate `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes } +var _ fs.InodeOperations = (*randomDevice)(nil) + func newRandomDevice(ctx context.Context, owner fs.FileOwner, mode linux.FileMode) *randomDevice { - r := &randomDevice{} - r.InitEntry(ctx, owner, fs.FilePermsFromMode(mode)) + r := &randomDevice{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, fs.FilePermsFromMode(mode), linux.TMPFS_MAGIC), + } return r } -// DeprecatedPreadv reads random data. -func (*randomDevice) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - return dst.CopyOutFrom(ctx, safemem.FromIOReader{rand.Reader}) +// GetFile implements fs.InodeOperations.GetFile. +func (randomDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &randomFileOperations{}), nil } -// DeprecatedPwritev implements fs.HandleOperations.DeprecatedPwritev. -func (*randomDevice) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { - // On Linux, "Writing to /dev/random or /dev/urandom will update the - // entropy pool with the data written, but this will not result in a higher - // entropy count" - random(4). We don't need to support this, but we do - // need to support the write, so just make it a no-op a la /dev/null. - return src.NumBytes(), nil +// +stateify savable +type randomFileOperations struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopWrite `state:"nosave"` } -// Truncate should be simply ignored for character devices on linux. -func (r *randomDevice) Truncate(context.Context, *fs.Inode, int64) error { - return nil +var _ fs.FileOperations = (*randomFileOperations)(nil) + +// Read implements fs.FileOperations.Read. +func (randomFileOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, _ int64) (int64, error) { + return dst.CopyOutFrom(ctx, safemem.FromIOReader{rand.Reader}) } diff --git a/pkg/sentry/fs/dirent.go b/pkg/sentry/fs/dirent.go index c4918a11b..d6a19dc81 100644 --- a/pkg/sentry/fs/dirent.go +++ b/pkg/sentry/fs/dirent.go @@ -837,8 +837,8 @@ func (d *Dirent) CreateFifo(ctx context.Context, root *Dirent, name string, perm }) } -// getDotAttrs returns the DentAttrs corresponding to "." and ".." directories. -func (d *Dirent) getDotAttrs(root *Dirent) (DentAttr, DentAttr) { +// GetDotAttrs returns the DentAttrs corresponding to "." and ".." directories. +func (d *Dirent) GetDotAttrs(root *Dirent) (DentAttr, DentAttr) { // Get '.'. sattr := d.Inode.StableAttr dot := DentAttr{ @@ -870,7 +870,7 @@ func (d *Dirent) readdirFrozen(root *Dirent, offset int64, dirCtx *DirCtx) (int6 // Collect attrs for "." and "..". attrs := make(map[string]DentAttr) names := []string{".", ".."} - attrs["."], attrs[".."] = d.getDotAttrs(root) + attrs["."], attrs[".."] = d.GetDotAttrs(root) // Get info from all children. d.mu.Lock() @@ -965,7 +965,7 @@ func direntReaddir(ctx context.Context, d *Dirent, it DirIterator, root *Dirent, } // Collect attrs for "." and "..". - dot, dotdot := d.getDotAttrs(root) + dot, dotdot := d.GetDotAttrs(root) // Emit "." and ".." if the offset is low enough. if offset == 0 { diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go index e3b830747..b4d11cb45 100644 --- a/pkg/sentry/fs/fdpipe/pipe.go +++ b/pkg/sentry/fs/fdpipe/pipe.go @@ -37,13 +37,13 @@ import ( // // +stateify savable type pipeOperations struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` - fsutil.NoIoctl `state:"nosave"` - waiter.Queue `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + waiter.Queue `state:"nosave"` // flags are the flags used to open the pipe. flags fs.FileFlags `state:".(fs.FileFlags)"` diff --git a/pkg/sentry/fs/file_operations.go b/pkg/sentry/fs/file_operations.go index 28e8e233d..81c6e2b5d 100644 --- a/pkg/sentry/fs/file_operations.go +++ b/pkg/sentry/fs/file_operations.go @@ -91,7 +91,7 @@ type FileOperations interface { Flush(ctx context.Context, file *File) error // ConfigureMMap mutates opts to implement mmap(2) for the file. Most - // implementations can either embed fsutil.NoMMap (if they don't support + // implementations can either embed fsutil.FileNoMMap (if they don't support // memory mapping) or call fsutil.GenericConfigureMMap with the appropriate // memmap.Mappable. ConfigureMMap(ctx context.Context, file *File, opts *memmap.MMapOpts) error diff --git a/pkg/sentry/fs/file_overlay_test.go b/pkg/sentry/fs/file_overlay_test.go index f121cbdda..a4ac58763 100644 --- a/pkg/sentry/fs/file_overlay_test.go +++ b/pkg/sentry/fs/file_overlay_test.go @@ -20,7 +20,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - ramfstest "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs/test" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/contexttest" ) @@ -135,7 +136,7 @@ func TestReaddirRevalidation(t *testing.T) { // Get a handle to the dirent in the upper filesystem so that we can // modify it without going through the dirent. - upperDir := upper.InodeOperations.(*dir).InodeOperations.(*ramfstest.Dir) + upperDir := upper.InodeOperations.(*dir).InodeOperations.(*ramfs.Dir) // Check that overlay returns the files from both upper and lower. openDir, err := overlay.GetFile(ctx, fs.NewDirent(overlay, "stub"), fs.FileFlags{Read: true}) @@ -155,7 +156,7 @@ func TestReaddirRevalidation(t *testing.T) { if err := upperDir.Remove(ctx, upper, "a"); err != nil { t.Fatalf("error removing child: %v", err) } - upperDir.AddChild(ctx, "c", fs.NewInode(ramfstest.NewFile(ctx, fs.FilePermissions{}), + upperDir.AddChild(ctx, "c", fs.NewInode(fsutil.NewSimpleFileInode(ctx, fs.RootOwner, fs.FilePermissions{}, 0), upper.MountSource, fs.StableAttr{Type: fs.RegularFile})) // Seek to beginning of the directory and do the readdir again. diff --git a/pkg/sentry/fs/filetest/filetest.go b/pkg/sentry/fs/filetest/filetest.go index 65ca196d9..40d84d9f2 100644 --- a/pkg/sentry/fs/filetest/filetest.go +++ b/pkg/sentry/fs/filetest/filetest.go @@ -31,14 +31,14 @@ import ( // TestFileOperations is an implementation of the File interface. It provides all // required methods. type TestFileOperations struct { - fsutil.NoopRelease `state:"nosave"` - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` - fsutil.NoIoctl `state:"nosave"` - waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` } // NewTestFile creates and initializes a new test file. diff --git a/pkg/sentry/fs/fsutil/BUILD b/pkg/sentry/fs/fsutil/BUILD index 6834e1272..4965e1a5f 100644 --- a/pkg/sentry/fs/fsutil/BUILD +++ b/pkg/sentry/fs/fsutil/BUILD @@ -67,7 +67,6 @@ go_library( "frame_ref_set.go", "frame_ref_set_impl.go", "fsutil.go", - "handle.go", "host_file_mapper.go", "host_file_mapper_state.go", "host_file_mapper_unsafe.go", @@ -96,20 +95,6 @@ go_library( ], ) -go_test( - name = "fsutil_x_test", - size = "small", - srcs = ["handle_test.go"], - deps = [ - ":fsutil", - "//pkg/sentry/context", - "//pkg/sentry/context/contexttest", - "//pkg/sentry/fs", - "//pkg/sentry/fs/ramfs/test", - "//pkg/sentry/usermem", - ], -) - go_test( name = "fsutil_test", size = "small", diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index 46db2e51c..0970f782b 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -24,12 +24,12 @@ import ( "gvisor.googlesource.com/gvisor/pkg/waiter" ) -// NoopRelease implements FileOperations.Release for files that have no +// FileNoopRelease implements fs.FileOperations.Release for files that have no // resources to release. -type NoopRelease struct{} +type FileNoopRelease struct{} // Release is a no-op. -func (NoopRelease) Release() {} +func (FileNoopRelease) Release() {} // SeekWithDirCursor is used to implement fs.FileOperations.Seek. If dirCursor // is not nil and the seek was on a directory, the cursor will be updated. @@ -127,71 +127,81 @@ func SeekWithDirCursor(ctx context.Context, file *fs.File, whence fs.SeekWhence, return current, syserror.EINVAL } -// GenericSeek implements FileOperations.Seek for files that use a generic -// seek implementation. -type GenericSeek struct{} +// FileGenericSeek implements fs.FileOperations.Seek for files that use a +// generic seek implementation. +type FileGenericSeek struct{} // Seek implements fs.FileOperations.Seek. -func (GenericSeek) Seek(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64) (int64, error) { +func (FileGenericSeek) Seek(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64) (int64, error) { return SeekWithDirCursor(ctx, file, whence, offset, nil) } -// ZeroSeek implements FileOperations.Seek for files that maintain a constant -// zero-value offset and require a no-op Seek. -type ZeroSeek struct{} +// FileZeroSeek implements fs.FileOperations.Seek for files that maintain a +// constant zero-value offset and require a no-op Seek. +type FileZeroSeek struct{} -// Seek implements FileOperations.Seek. -func (ZeroSeek) Seek(context.Context, *fs.File, fs.SeekWhence, int64) (int64, error) { +// Seek implements fs.FileOperations.Seek. +func (FileZeroSeek) Seek(context.Context, *fs.File, fs.SeekWhence, int64) (int64, error) { return 0, nil } -// PipeSeek implements FileOperations.Seek and can be used for files that behave -// like pipes (seeking is not supported). -type PipeSeek struct{} +// FileNoSeek implements fs.FileOperations.Seek to return EINVAL. +type FileNoSeek struct{} + +// Seek implements fs.FileOperations.Seek. +func (FileNoSeek) Seek(context.Context, *fs.File, fs.SeekWhence, int64) (int64, error) { + return 0, syserror.EINVAL +} -// Seek implements FileOperations.Seek. -func (PipeSeek) Seek(context.Context, *fs.File, fs.SeekWhence, int64) (int64, error) { +// FilePipeSeek implements fs.FileOperations.Seek and can be used for files +// that behave like pipes (seeking is not supported). +type FilePipeSeek struct{} + +// Seek implements fs.FileOperations.Seek. +func (FilePipeSeek) Seek(context.Context, *fs.File, fs.SeekWhence, int64) (int64, error) { return 0, syserror.ESPIPE } -// NotDirReaddir implements FileOperations.Readdir for non-directories. -type NotDirReaddir struct{} +// FileNotDirReaddir implements fs.FileOperations.Readdir for non-directories. +type FileNotDirReaddir struct{} -// Readdir implements FileOperations.NotDirReaddir. -func (NotDirReaddir) Readdir(context.Context, *fs.File, fs.DentrySerializer) (int64, error) { +// Readdir implements fs.FileOperations.FileNotDirReaddir. +func (FileNotDirReaddir) Readdir(context.Context, *fs.File, fs.DentrySerializer) (int64, error) { return 0, syserror.ENOTDIR } -// NoFsync implements FileOperations.Fsync for files that don't support syncing. -type NoFsync struct{} +// FileNoFsync implements fs.FileOperations.Fsync for files that don't support +// syncing. +type FileNoFsync struct{} -// Fsync implements FileOperations.Fsync. -func (NoFsync) Fsync(context.Context, *fs.File, int64, int64, fs.SyncType) error { +// Fsync implements fs.FileOperations.Fsync. +func (FileNoFsync) Fsync(context.Context, *fs.File, int64, int64, fs.SyncType) error { return syserror.EINVAL } -// NoopFsync implements FileOperations.Fsync for files that don't need to synced. -type NoopFsync struct{} +// FileNoopFsync implements fs.FileOperations.Fsync for files that don't need +// to synced. +type FileNoopFsync struct{} -// Fsync implements FileOperations.Fsync. -func (NoopFsync) Fsync(context.Context, *fs.File, int64, int64, fs.SyncType) error { +// Fsync implements fs.FileOperations.Fsync. +func (FileNoopFsync) Fsync(context.Context, *fs.File, int64, int64, fs.SyncType) error { return nil } -// NoopFlush implements FileOperations.Flush as a no-op. -type NoopFlush struct{} +// FileNoopFlush implements fs.FileOperations.Flush as a no-op. +type FileNoopFlush struct{} -// Flush implements FileOperations.Flush. -func (NoopFlush) Flush(context.Context, *fs.File) error { +// Flush implements fs.FileOperations.Flush. +func (FileNoopFlush) Flush(context.Context, *fs.File) error { return nil } -// NoMMap implements fs.FileOperations.Mappable for files that cannot +// FileNoMMap implements fs.FileOperations.Mappable for files that cannot // be memory mapped. -type NoMMap struct{} +type FileNoMMap struct{} // ConfigureMMap implements fs.FileOperations.ConfigureMMap. -func (NoMMap) ConfigureMMap(context.Context, *fs.File, *memmap.MMapOpts) error { +func (FileNoMMap) ConfigureMMap(context.Context, *fs.File, *memmap.MMapOpts) error { return syserror.ENODEV } @@ -204,26 +214,43 @@ func GenericConfigureMMap(file *fs.File, m memmap.Mappable, opts *memmap.MMapOpt return nil } -// NoIoctl implements fs.FileOperations.Ioctl for files that don't implement -// the ioctl syscall. -type NoIoctl struct{} +// FileNoIoctl implements fs.FileOperations.Ioctl for files that don't +// implement the ioctl syscall. +type FileNoIoctl struct{} // Ioctl implements fs.FileOperations.Ioctl. -func (NoIoctl) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { +func (FileNoIoctl) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { return 0, syserror.ENOTTY } -// DirFileOperations implements FileOperations for directories. +// DirFileOperations implements most of fs.FileOperations for directories, +// except for Readdir which the embedding type must implement. +type DirFileOperations struct { + waiter.AlwaysReady + FileGenericSeek + FileNoFsync + FileNoIoctl + FileNoMMap + FileNoopFlush + FileNoopRelease +} + +// Read implements fs.FileOperations.Read +func (*DirFileOperations) Read(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { + return 0, syserror.EISDIR +} + +// Write implements fs.FileOperations.Write. +func (*DirFileOperations) Write(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { + return 0, syserror.EISDIR +} + +// StaticDirFileOperations implements fs.FileOperations for directories with +// static children. // // +stateify savable -type DirFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - NoopRelease `state:"nosave"` - GenericSeek `state:"nosave"` - NoFsync `state:"nosave"` - NoopFlush `state:"nosave"` - NoMMap `state:"nosave"` - NoIoctl `state:"nosave"` +type StaticDirFileOperations struct { + DirFileOperations // dentryMap is a SortedDentryMap used to implement Readdir. dentryMap *fs.SortedDentryMap @@ -233,37 +260,106 @@ type DirFileOperations struct { dirCursor string } -// NewDirFileOperations returns a new DirFileOperations that will iterate the -// given denty map. -func NewDirFileOperations(dentries *fs.SortedDentryMap) *DirFileOperations { - return &DirFileOperations{ +// NewStaticDirFileOperations returns a new StaticDirFileOperations that will +// iterate the given denty map. +func NewStaticDirFileOperations(dentries *fs.SortedDentryMap) *StaticDirFileOperations { + return &StaticDirFileOperations{ dentryMap: dentries, } } // IterateDir implements DirIterator.IterateDir. -func (dfo *DirFileOperations) IterateDir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { - n, err := fs.GenericReaddir(dirCtx, dfo.dentryMap) +func (sdfo *StaticDirFileOperations) IterateDir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { + n, err := fs.GenericReaddir(dirCtx, sdfo.dentryMap) return offset + n, err } -// Readdir implements FileOperations.Readdir. -func (dfo *DirFileOperations) Readdir(ctx context.Context, file *fs.File, serializer fs.DentrySerializer) (int64, error) { +// Readdir implements fs.FileOperations.Readdir. +func (sdfo *StaticDirFileOperations) Readdir(ctx context.Context, file *fs.File, serializer fs.DentrySerializer) (int64, error) { root := fs.RootFromContext(ctx) defer root.DecRef() dirCtx := &fs.DirCtx{ Serializer: serializer, - DirCursor: &dfo.dirCursor, + DirCursor: &sdfo.dirCursor, } - return fs.DirentReaddir(ctx, file.Dirent, dfo, root, dirCtx, file.Offset()) + return fs.DirentReaddir(ctx, file.Dirent, sdfo, root, dirCtx, file.Offset()) } -// Read implements FileOperations.Read -func (*DirFileOperations) Read(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { - return 0, syserror.EISDIR +// NoReadWriteFile is a file that does not support reading or writing. +// +// +stateify savable +type NoReadWriteFile struct { + waiter.AlwaysReady `state:"nosave"` + FileGenericSeek `state:"nosave"` + FileNoIoctl `state:"nosave"` + FileNoMMap `state:"nosave"` + FileNoopFsync `state:"nosave"` + FileNoopFlush `state:"nosave"` + FileNoopRelease `state:"nosave"` + FileNoRead `state:"nosave"` + FileNoWrite `state:"nosave"` + FileNotDirReaddir `state:"nosave"` } -// Write implements FileOperations.Write. -func (*DirFileOperations) Write(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { - return 0, syserror.EISDIR +var _ fs.FileOperations = (*NoReadWriteFile)(nil) + +// FileStaticContentReader is a helper to implement fs.FileOperations.Read with +// static content. +// +// +stateify savable +type FileStaticContentReader struct { + // content is immutable. + content []byte +} + +// NewFileStaticContentReader initializes a FileStaticContentReader with the +// given content. +func NewFileStaticContentReader(b []byte) FileStaticContentReader { + return FileStaticContentReader{ + content: b, + } +} + +// Read implements fs.FileOperations.Read. +func (scr *FileStaticContentReader) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { + if offset < 0 { + return 0, syserror.EINVAL + } + if offset >= int64(len(scr.content)) { + return 0, nil + } + n, err := dst.CopyOut(ctx, scr.content[offset:]) + return int64(n), err +} + +// FileNoopWrite implements fs.FileOperations.Write as a noop. +type FileNoopWrite struct{} + +// Write implements fs.FileOperations.Write. +func (FileNoopWrite) Write(_ context.Context, _ *fs.File, src usermem.IOSequence, _ int64) (int64, error) { + return src.NumBytes(), nil +} + +// FileNoRead implements fs.FileOperations.Read to return EINVAL. +type FileNoRead struct{} + +// Read implements fs.FileOperations.Read. +func (FileNoRead) Read(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { + return 0, syserror.EINVAL +} + +// FileNoWrite implements fs.FileOperations.Write to return EINVAL. +type FileNoWrite struct{} + +// Write implements fs.FileOperations.Write. +func (FileNoWrite) Write(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { + return 0, syserror.EINVAL +} + +// FileNoopRead implement fs.FileOperations.Read as a noop. +type FileNoopRead struct{} + +// Read implements fs.FileOperations.Read. +func (FileNoopRead) Read(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { + return 0, nil } diff --git a/pkg/sentry/fs/fsutil/fsutil.go b/pkg/sentry/fs/fsutil/fsutil.go index 3d7f3732d..319c4841b 100644 --- a/pkg/sentry/fs/fsutil/fsutil.go +++ b/pkg/sentry/fs/fsutil/fsutil.go @@ -20,7 +20,5 @@ // - For fs.Inodes that require a page cache to be memory mapped, see // inode_cache.go. // -// - For fs.Files that implement fs.HandleOps, see handle.go. -// // - For anon fs.Inodes, see anon.go. package fsutil diff --git a/pkg/sentry/fs/fsutil/handle.go b/pkg/sentry/fs/fsutil/handle.go deleted file mode 100644 index 8920b72ee..000000000 --- a/pkg/sentry/fs/fsutil/handle.go +++ /dev/null @@ -1,128 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package fsutil - -import ( - "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/waiter" -) - -// Handle implements FileOperations. -// -// FIXME: Remove Handle entirely in favor of individual fs.File -// implementations using simple generic utilities. -// -// +stateify savable -type Handle struct { - NoopRelease `state:"nosave"` - NoIoctl `state:"nosave"` - HandleOperations fs.HandleOperations - - // dirCursor is the directory cursor. - dirCursor string -} - -// NewHandle returns a File backed by the Dirent and FileFlags. -func NewHandle(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags, hops fs.HandleOperations) *fs.File { - if !fs.IsPipe(dirent.Inode.StableAttr) && !fs.IsSocket(dirent.Inode.StableAttr) { - // Allow reading/writing at an arbitrary offset for non-pipes - // and non-sockets. - flags.Pread = true - flags.Pwrite = true - } - - return fs.NewFile(ctx, dirent, flags, &Handle{HandleOperations: hops}) -} - -// Readiness implements waiter.Waitable.Readiness. -func (h *Handle) Readiness(mask waiter.EventMask) waiter.EventMask { - return h.HandleOperations.Readiness(mask) -} - -// EventRegister implements waiter.Waitable.EventRegister. -func (h *Handle) EventRegister(e *waiter.Entry, mask waiter.EventMask) { - h.HandleOperations.EventRegister(e, mask) -} - -// EventUnregister implements waiter.Waitable.EventUnregister. -func (h *Handle) EventUnregister(e *waiter.Entry) { - h.HandleOperations.EventUnregister(e) -} - -// Readdir implements FileOperations.Readdir. -func (h *Handle) Readdir(ctx context.Context, file *fs.File, serializer fs.DentrySerializer) (int64, error) { - root := fs.RootFromContext(ctx) - defer root.DecRef() - dirCtx := &fs.DirCtx{ - Serializer: serializer, - DirCursor: &h.dirCursor, - } - n, err := fs.DirentReaddir(ctx, file.Dirent, h, root, dirCtx, file.Offset()) - return n, err -} - -// Seek implements FileOperations.Seek. -func (h *Handle) Seek(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64) (int64, error) { - return SeekWithDirCursor(ctx, file, whence, offset, &h.dirCursor) -} - -// IterateDir implements DirIterator.IterateDir. -func (h *Handle) IterateDir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { - return h.HandleOperations.DeprecatedReaddir(ctx, dirCtx, offset) -} - -// Read implements FileOperations.Read. -func (h *Handle) Read(ctx context.Context, file *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { - return h.HandleOperations.DeprecatedPreadv(ctx, dst, offset) -} - -// Write implements FileOperations.Write. -func (h *Handle) Write(ctx context.Context, file *fs.File, src usermem.IOSequence, offset int64) (int64, error) { - return h.HandleOperations.DeprecatedPwritev(ctx, src, offset) -} - -// Fsync implements FileOperations.Fsync. -func (h *Handle) Fsync(ctx context.Context, file *fs.File, start int64, end int64, syncType fs.SyncType) error { - switch syncType { - case fs.SyncAll, fs.SyncData: - // Write out metadata. - if err := file.Dirent.Inode.WriteOut(ctx); err != nil { - return err - } - fallthrough - case fs.SyncBackingStorage: - // Use DeprecatedFsync to sync disks. - return h.HandleOperations.DeprecatedFsync() - } - panic("invalid sync type") -} - -// Flush implements FileOperations.Flush. -func (h *Handle) Flush(context.Context, *fs.File) error { - return h.HandleOperations.DeprecatedFlush() -} - -// ConfigureMMap implements FileOperations.ConfigureMMap. -func (h *Handle) ConfigureMMap(ctx context.Context, file *fs.File, opts *memmap.MMapOpts) error { - mappable := file.Dirent.Inode.Mappable() - if mappable == nil { - return syserror.ENODEV - } - return GenericConfigureMMap(file, mappable, opts) -} diff --git a/pkg/sentry/fs/fsutil/handle_test.go b/pkg/sentry/fs/fsutil/handle_test.go deleted file mode 100644 index 43e1a3bdf..000000000 --- a/pkg/sentry/fs/fsutil/handle_test.go +++ /dev/null @@ -1,227 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package fsutil_test - -import ( - "io" - "syscall" - "testing" - - "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/sentry/context/contexttest" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" - ramfstest "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs/test" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" -) - -type testInodeOperations struct { - fs.InodeOperations - fs.InodeType - FileSize int64 - writes uint - reads uint -} - -func (t *testInodeOperations) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - return fs.UnstableAttr{Size: t.FileSize}, nil -} - -// Check implements InodeOperations.Check. -func (t *testInodeOperations) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { - return fs.ContextCanAccessFile(ctx, inode, p) -} - -func (t *testInodeOperations) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - t.reads++ - return t.InodeOperations.DeprecatedPreadv(ctx, dst, offset) -} - -func (t *testInodeOperations) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { - t.writes++ - return t.InodeOperations.DeprecatedPwritev(ctx, src, offset) -} - -// testHandle returns a handle for a test node. -// -// The size of the node is fixed at 20 bytes. -func testHandle(t *testing.T, flags fs.FileFlags, nt fs.InodeType) (*fs.File, *testInodeOperations) { - ctx := contexttest.Context(t) - m := fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}) - n := &testInodeOperations{ - InodeOperations: ramfstest.NewFile(ctx, fs.FilePermissions{User: fs.PermMask{Read: true, Write: true}}), - FileSize: 20, - } - d := fs.NewDirent(fs.NewInode(n, m, fs.StableAttr{Type: nt}), "test") - return fsutil.NewHandle(ctx, d, flags, d.Inode.HandleOps()), n -} - -func TestHandleOps(t *testing.T) { - h, n := testHandle(t, fs.FileFlags{Read: true, Write: true}, fs.RegularFile) - defer h.DecRef() - - // Make sure a write request works. - if n, err := h.Writev(contexttest.Context(t), usermem.BytesIOSequence([]byte("a"))); n != 1 || err != nil { - t.Fatalf("Writev: got (%d, %v), wanted (1, nil)", n, err) - } - if n.writes != 1 { - t.Errorf("found %d writes, expected 1", n.writes) - } - - // Make sure a read request works. - dst := make([]byte, 1) - if n, err := h.Preadv(contexttest.Context(t), usermem.BytesIOSequence(dst), 0); n != 1 || (err != nil && err != io.EOF) { - t.Errorf("Preadv: got (%d, %v), wanted (1, nil or EOF)", n, err) - } - if dst[0] != 'a' { - t.Errorf("Preadv: read %q, wanted 'a'", dst[0]) - } - if n.reads != 1 { - t.Errorf("found %d reads, expected 1", n.reads) - } -} - -type seekTest struct { - whence fs.SeekWhence - offset int64 - result int64 - err error -} - -type seekSuite struct { - nodeType fs.InodeType - cases []seekTest -} - -// FIXME: This is currently missing fs.SeekEnd tests due to the -// fact that NullInodeOperations returns an error on stat. -func TestHandleSeek(t *testing.T) { - ts := []seekSuite{ - { - nodeType: fs.RegularFile, - cases: []seekTest{ - {fs.SeekSet, 0, 0, nil}, - {fs.SeekSet, 10, 10, nil}, - {fs.SeekSet, -5, 10, syscall.EINVAL}, - {fs.SeekCurrent, -1, 9, nil}, - {fs.SeekCurrent, 2, 11, nil}, - {fs.SeekCurrent, -12, 11, syscall.EINVAL}, - {fs.SeekEnd, -1, 19, nil}, - {fs.SeekEnd, 0, 20, nil}, - {fs.SeekEnd, 2, 22, nil}, - }, - }, - { - nodeType: fs.Directory, - cases: []seekTest{ - {fs.SeekSet, 0, 0, nil}, - {fs.SeekSet, 10, 0, syscall.EINVAL}, - {fs.SeekSet, -5, 0, syscall.EINVAL}, - {fs.SeekCurrent, 0, 0, nil}, - {fs.SeekCurrent, 11, 0, syscall.EINVAL}, - {fs.SeekCurrent, -6, 0, syscall.EINVAL}, - {fs.SeekEnd, 0, 0, syscall.EINVAL}, - {fs.SeekEnd, -1, 0, syscall.EINVAL}, - {fs.SeekEnd, 2, 0, syscall.EINVAL}, - }, - }, - { - nodeType: fs.Symlink, - cases: []seekTest{ - {fs.SeekSet, 5, 0, syscall.EINVAL}, - {fs.SeekSet, -5, 0, syscall.EINVAL}, - {fs.SeekSet, 0, 0, syscall.EINVAL}, - {fs.SeekCurrent, 5, 0, syscall.EINVAL}, - {fs.SeekCurrent, -5, 0, syscall.EINVAL}, - {fs.SeekCurrent, 0, 0, syscall.EINVAL}, - {fs.SeekEnd, 5, 0, syscall.EINVAL}, - {fs.SeekEnd, -5, 0, syscall.EINVAL}, - {fs.SeekEnd, 0, 0, syscall.EINVAL}, - }, - }, - { - nodeType: fs.Pipe, - cases: []seekTest{ - {fs.SeekSet, 5, 0, syscall.ESPIPE}, - {fs.SeekSet, -5, 0, syscall.ESPIPE}, - {fs.SeekSet, 0, 0, syscall.ESPIPE}, - {fs.SeekCurrent, 5, 0, syscall.ESPIPE}, - {fs.SeekCurrent, -5, 0, syscall.ESPIPE}, - {fs.SeekCurrent, 0, 0, syscall.ESPIPE}, - {fs.SeekEnd, 5, 0, syscall.ESPIPE}, - {fs.SeekEnd, -5, 0, syscall.ESPIPE}, - {fs.SeekEnd, 0, 0, syscall.ESPIPE}, - }, - }, - { - nodeType: fs.Socket, - cases: []seekTest{ - {fs.SeekSet, 5, 0, syscall.ESPIPE}, - {fs.SeekSet, -5, 0, syscall.ESPIPE}, - {fs.SeekSet, 0, 0, syscall.ESPIPE}, - {fs.SeekCurrent, 5, 0, syscall.ESPIPE}, - {fs.SeekCurrent, -5, 0, syscall.ESPIPE}, - {fs.SeekCurrent, 0, 0, syscall.ESPIPE}, - {fs.SeekEnd, 5, 0, syscall.ESPIPE}, - {fs.SeekEnd, -5, 0, syscall.ESPIPE}, - {fs.SeekEnd, 0, 0, syscall.ESPIPE}, - }, - }, - { - nodeType: fs.CharacterDevice, - cases: []seekTest{ - {fs.SeekSet, 5, 0, nil}, - {fs.SeekSet, -5, 0, nil}, - {fs.SeekSet, 0, 0, nil}, - {fs.SeekCurrent, 5, 0, nil}, - {fs.SeekCurrent, -5, 0, nil}, - {fs.SeekCurrent, 0, 0, nil}, - {fs.SeekEnd, 5, 0, nil}, - {fs.SeekEnd, -5, 0, nil}, - {fs.SeekEnd, 0, 0, nil}, - }, - }, - { - nodeType: fs.BlockDevice, - cases: []seekTest{ - {fs.SeekSet, 0, 0, nil}, - {fs.SeekSet, 10, 10, nil}, - {fs.SeekSet, -5, 10, syscall.EINVAL}, - {fs.SeekCurrent, -1, 9, nil}, - {fs.SeekCurrent, 2, 11, nil}, - {fs.SeekCurrent, -12, 11, syscall.EINVAL}, - {fs.SeekEnd, -1, 19, nil}, - {fs.SeekEnd, 0, 20, nil}, - {fs.SeekEnd, 2, 22, nil}, - }, - }, - } - - for _, s := range ts { - h, _ := testHandle(t, fs.FileFlags{Read: true, Write: true}, s.nodeType) - defer h.DecRef() - - for _, c := range s.cases { - // Try the given seek. - offset, err := h.Seek(contexttest.Context(t), c.whence, c.offset) - if err != c.err { - t.Errorf("seek(%s, %d) on %s had unexpected error: expected %v, got %v", c.whence, c.offset, s.nodeType, c.err, err) - } - if err == nil && offset != c.result { - t.Errorf("seek(%s, %d) on %s had bad result: expected %v, got %v", c.whence, c.offset, s.nodeType, c.result, offset) - } - } - } -} diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go index d4db1c2de..f1f5ec1de 100644 --- a/pkg/sentry/fs/fsutil/inode.go +++ b/pkg/sentry/fs/fsutil/inode.go @@ -15,213 +15,270 @@ package fsutil import ( + "sync" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/waiter" ) -// NewSimpleInodeOperations constructs fs.InodeOperations from InodeSimpleAttributes. -func NewSimpleInodeOperations(i InodeSimpleAttributes) fs.InodeOperations { - return &simpleInodeOperations{InodeSimpleAttributes: i} +// SimpleFileInode is a simple implementation of InodeOperations. +// +// +stateify savable +type SimpleFileInode struct { + InodeGenericChecker `state:"nosave"` + InodeNoExtendedAttributes `state:"nosave"` + InodeNoopRelease `state:"nosave"` + InodeNoopWriteOut `state:"nosave"` + InodeNotDirectory `state:"nosave"` + InodeNotMappable `state:"nosave"` + InodeNotOpenable `state:"nosave"` + InodeNotSocket `state:"nosave"` + InodeNotSymlink `state:"nosave"` + InodeNotTruncatable `state:"nosave"` + InodeNotVirtual `state:"nosave"` + + InodeSimpleAttributes +} + +// NewSimpleFileInode returns a new SimpleFileInode. +func NewSimpleFileInode(ctx context.Context, owner fs.FileOwner, perms fs.FilePermissions, typ uint64) *SimpleFileInode { + return &SimpleFileInode{ + InodeSimpleAttributes: NewInodeSimpleAttributes(ctx, owner, perms, typ), + } } -// simpleInodeOperations is a simple implementation of Inode. +// NoReadWriteFileInode is an implementation of InodeOperations that supports +// opening files that are not readable or writeable. // // +stateify savable -type simpleInodeOperations struct { - DeprecatedFileOperations `state:"nosave"` +type NoReadWriteFileInode struct { + InodeGenericChecker `state:"nosave"` + InodeNoExtendedAttributes `state:"nosave"` + InodeNoopRelease `state:"nosave"` + InodeNoopWriteOut `state:"nosave"` InodeNotDirectory `state:"nosave"` + InodeNotMappable `state:"nosave"` InodeNotSocket `state:"nosave"` - InodeNotRenameable `state:"nosave"` - InodeNotOpenable `state:"nosave"` - InodeNotVirtual `state:"nosave"` InodeNotSymlink `state:"nosave"` - InodeNoExtendedAttributes `state:"nosave"` - NoMappable `state:"nosave"` - NoopWriteOut `state:"nosave"` + InodeNotTruncatable `state:"nosave"` + InodeNotVirtual `state:"nosave"` InodeSimpleAttributes } -// InodeSimpleAttributes implements a subset of the Inode interface. It provides -// read-only access to attributes. +// NewNoReadWriteFileInode returns a new NoReadWriteFileInode. +func NewNoReadWriteFileInode(ctx context.Context, owner fs.FileOwner, perms fs.FilePermissions, typ uint64) *NoReadWriteFileInode { + return &NoReadWriteFileInode{ + InodeSimpleAttributes: NewInodeSimpleAttributes(ctx, owner, perms, typ), + } +} + +// GetFile implements fs.InodeOperations.GetFile. +func (*NoReadWriteFileInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &NoReadWriteFile{}), nil +} + +// InodeSimpleAttributes implements methods for updating in-memory unstable +// attributes. // // +stateify savable type InodeSimpleAttributes struct { - // FSType is the filesystem type reported by StatFS. + // FSType is the immutable filesystem type that will be returned by + // StatFS. FSType uint64 - // UAttr are the unstable attributes of the Inode. - UAttr fs.UnstableAttr + // mu protects unstable. + mu sync.RWMutex `state:"nosave"` + Unstable fs.UnstableAttr } -// Release implements fs.InodeOperations.Release. -func (i *InodeSimpleAttributes) Release(context.Context) {} - -// StatFS implements fs.InodeOperations.StatFS. -func (i *InodeSimpleAttributes) StatFS(context.Context) (fs.Info, error) { - return fs.Info{Type: i.FSType}, nil +// NewInodeSimpleAttributes returns a new InodeSimpleAttributes. +func NewInodeSimpleAttributes(ctx context.Context, owner fs.FileOwner, perms fs.FilePermissions, typ uint64) InodeSimpleAttributes { + return InodeSimpleAttributes{ + FSType: typ, + Unstable: fs.WithCurrentTime(ctx, fs.UnstableAttr{ + Owner: owner, + Perms: perms, + }), + } } // UnstableAttr implements fs.InodeOperations.UnstableAttr. -func (i *InodeSimpleAttributes) UnstableAttr(context.Context, *fs.Inode) (fs.UnstableAttr, error) { - return i.UAttr, nil +func (i *InodeSimpleAttributes) UnstableAttr(ctx context.Context, _ *fs.Inode) (fs.UnstableAttr, error) { + i.mu.RLock() + u := i.Unstable + i.mu.RUnlock() + return u, nil } -// Check implements fs.InodeOperations.Check. -func (i *InodeSimpleAttributes) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { - return fs.ContextCanAccessFile(ctx, inode, p) +// SetPermissions implements fs.InodeOperations.SetPermissions. +func (i *InodeSimpleAttributes) SetPermissions(ctx context.Context, _ *fs.Inode, p fs.FilePermissions) bool { + i.mu.Lock() + i.Unstable.SetPermissions(ctx, p) + i.mu.Unlock() + return true +} + +// SetOwner implements fs.InodeOperations.SetOwner. +func (i *InodeSimpleAttributes) SetOwner(ctx context.Context, _ *fs.Inode, owner fs.FileOwner) error { + i.mu.Lock() + i.Unstable.SetOwner(ctx, owner) + i.mu.Unlock() + return nil +} + +// SetTimestamps implements fs.InodeOperations.SetTimestamps. +func (i *InodeSimpleAttributes) SetTimestamps(ctx context.Context, _ *fs.Inode, ts fs.TimeSpec) error { + i.mu.Lock() + i.Unstable.SetTimestamps(ctx, ts) + i.mu.Unlock() + return nil } // AddLink implements fs.InodeOperations.AddLink. -func (*InodeSimpleAttributes) AddLink() {} +func (i *InodeSimpleAttributes) AddLink() { + i.mu.Lock() + i.Unstable.Links++ + i.mu.Unlock() +} // DropLink implements fs.InodeOperations.DropLink. -func (*InodeSimpleAttributes) DropLink() {} - -// NotifyStatusChange implements fs.fs.InodeOperations. -func (i *InodeSimpleAttributes) NotifyStatusChange(ctx context.Context) { - i.UAttr.StatusChangeTime = ktime.NowFromContext(ctx) +func (i *InodeSimpleAttributes) DropLink() { + i.mu.Lock() + i.Unstable.Links-- + i.mu.Unlock() } -// SetPermissions implements fs.InodeOperations.SetPermissions. -func (*InodeSimpleAttributes) SetPermissions(context.Context, *fs.Inode, fs.FilePermissions) bool { - return false +// StatFS implements fs.InodeOperations.StatFS. +func (i *InodeSimpleAttributes) StatFS(context.Context) (fs.Info, error) { + if i.FSType == 0 { + return fs.Info{}, syserror.ENOSYS + } + return fs.Info{Type: i.FSType}, nil } -// SetOwner implements fs.InodeOperations.SetOwner. -func (*InodeSimpleAttributes) SetOwner(context.Context, *fs.Inode, fs.FileOwner) error { - return syserror.EINVAL +// NotifyAccess updates the access time. +func (i *InodeSimpleAttributes) NotifyAccess(ctx context.Context) { + i.mu.Lock() + i.Unstable.AccessTime = ktime.NowFromContext(ctx) + i.mu.Unlock() } -// SetTimestamps implements fs.InodeOperations.SetTimestamps. -func (*InodeSimpleAttributes) SetTimestamps(context.Context, *fs.Inode, fs.TimeSpec) error { - return syserror.EINVAL +// NotifyModification updates the modification time. +func (i *InodeSimpleAttributes) NotifyModification(ctx context.Context) { + i.mu.Lock() + i.Unstable.ModificationTime = ktime.NowFromContext(ctx) + i.mu.Unlock() } -// Truncate implements fs.InodeOperations.Truncate. -func (*InodeSimpleAttributes) Truncate(context.Context, *fs.Inode, int64) error { - return syserror.EINVAL +// NotifyStatusChange updates the status change time. +func (i *InodeSimpleAttributes) NotifyStatusChange(ctx context.Context) { + i.mu.Lock() + i.Unstable.StatusChangeTime = ktime.NowFromContext(ctx) + i.mu.Unlock() } -// InMemoryAttributes implements utilities for updating in-memory unstable -// attributes and extended attributes. It is not thread-safe. -// -// Users need not initialize Xattrs to non-nil (it will be initialized -// when the first extended attribute is set. +// InodeSimpleExtendedAttributes implements +// fs.InodeOperations.{Get,Set,List}xattr. // // +stateify savable -type InMemoryAttributes struct { - Unstable fs.UnstableAttr - Xattrs map[string][]byte +type InodeSimpleExtendedAttributes struct { + // mu protects xattrs. + mu sync.RWMutex `state:"nosave"` + xattrs map[string][]byte } -// SetPermissions updates the permissions to p. -func (i *InMemoryAttributes) SetPermissions(ctx context.Context, p fs.FilePermissions) bool { - i.Unstable.Perms = p - i.Unstable.StatusChangeTime = ktime.NowFromContext(ctx) - return true +// Getxattr implements fs.InodeOperations.Getxattr. +func (i *InodeSimpleExtendedAttributes) Getxattr(_ *fs.Inode, name string) ([]byte, error) { + i.mu.RLock() + value, ok := i.xattrs[name] + i.mu.RUnlock() + if !ok { + return nil, syserror.ENOATTR + } + return value, nil } -// SetOwner updates the file owner to owner. -func (i *InMemoryAttributes) SetOwner(ctx context.Context, owner fs.FileOwner) error { - if owner.UID.Ok() { - i.Unstable.Owner.UID = owner.UID - } - if owner.GID.Ok() { - i.Unstable.Owner.GID = owner.GID +// Setxattr implements fs.InodeOperations.Setxattr. +func (i *InodeSimpleExtendedAttributes) Setxattr(_ *fs.Inode, name string, value []byte) error { + i.mu.Lock() + if i.xattrs == nil { + i.xattrs = make(map[string][]byte) } + i.xattrs[name] = value + i.mu.Unlock() return nil } -// SetTimestamps sets the timestamps to ts. -func (i *InMemoryAttributes) SetTimestamps(ctx context.Context, ts fs.TimeSpec) error { - if ts.ATimeOmit && ts.MTimeOmit { - return nil - } - - now := ktime.NowFromContext(ctx) - if !ts.ATimeOmit { - if ts.ATimeSetSystemTime { - i.Unstable.AccessTime = now - } else { - i.Unstable.AccessTime = ts.ATime - } - } - if !ts.MTimeOmit { - if ts.MTimeSetSystemTime { - i.Unstable.ModificationTime = now - } else { - i.Unstable.ModificationTime = ts.MTime - } +// Listxattr implements fs.InodeOperations.Listxattr. +func (i *InodeSimpleExtendedAttributes) Listxattr(_ *fs.Inode) (map[string]struct{}, error) { + i.mu.RLock() + names := make(map[string]struct{}, len(i.xattrs)) + for name := range i.xattrs { + names[name] = struct{}{} } - i.Unstable.StatusChangeTime = now - return nil + i.mu.RUnlock() + return names, nil } -// TouchAccessTime updates access time to the current time. -func (i *InMemoryAttributes) TouchAccessTime(ctx context.Context) { - i.Unstable.AccessTime = ktime.NowFromContext(ctx) -} +// staticFile is a file with static contents. It is returned by +// InodeStaticFileGetter.GetFile. +// +// +stateify savable +type staticFile struct { + waiter.AlwaysReady `state:"nosave"` + FileGenericSeek `state:"nosave"` + FileNoIoctl `state:"nosave"` + FileNoMMap `state:"nosave"` + FileNoopFsync `state:"nosave"` + FileNoopFlush `state:"nosave"` + FileNoopRelease `state:"nosave"` + FileNoopWrite `state:"nosave"` + FileNotDirReaddir `state:"nosave"` -// TouchModificationTime updates modification and status change -// time to the current time. -func (i *InMemoryAttributes) TouchModificationTime(ctx context.Context) { - now := ktime.NowFromContext(ctx) - i.Unstable.ModificationTime = now - i.Unstable.StatusChangeTime = now + FileStaticContentReader } -// TouchStatusChangeTime updates status change time to the current time. -func (i *InMemoryAttributes) TouchStatusChangeTime(ctx context.Context) { - i.Unstable.StatusChangeTime = ktime.NowFromContext(ctx) -} +// InodeNoStatFS implement StatFS by retuning ENOSYS. +type InodeNoStatFS struct{} -// Getxattr returns the extended attribute at name or ENOATTR if -// it isn't set. -func (i *InMemoryAttributes) Getxattr(name string) ([]byte, error) { - if value, ok := i.Xattrs[name]; ok { - return value, nil - } - return nil, syserror.ENOATTR +// StatFS implements fs.InodeOperations.StatFS. +func (InodeNoStatFS) StatFS(context.Context) (fs.Info, error) { + return fs.Info{}, syserror.ENOSYS } -// Setxattr sets the extended attribute at name to value. -func (i *InMemoryAttributes) Setxattr(name string, value []byte) error { - if i.Xattrs == nil { - i.Xattrs = make(map[string][]byte) - } - i.Xattrs[name] = value - return nil +// InodeStaticFileGetter implements GetFile for a file with static contents. +// +// +stateify savable +type InodeStaticFileGetter struct { + Contents []byte } -// Listxattr returns the set of all currently set extended attributes. -func (i *InMemoryAttributes) Listxattr() (map[string]struct{}, error) { - names := make(map[string]struct{}, len(i.Xattrs)) - for name := range i.Xattrs { - names[name] = struct{}{} - } - return names, nil +// GetFile implements fs.InodeOperations.GetFile. +func (i *InodeStaticFileGetter) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &staticFile{ + FileStaticContentReader: NewFileStaticContentReader(i.Contents), + }), nil } -// NoMappable returns a nil memmap.Mappable. -type NoMappable struct{} +// InodeNotMappable returns a nil memmap.Mappable. +type InodeNotMappable struct{} // Mappable implements fs.InodeOperations.Mappable. -func (NoMappable) Mappable(*fs.Inode) memmap.Mappable { +func (InodeNotMappable) Mappable(*fs.Inode) memmap.Mappable { return nil } -// NoopWriteOut is a no-op implementation of Inode.WriteOut. -type NoopWriteOut struct{} +// InodeNoopWriteOut is a no-op implementation of fs.InodeOperations.WriteOut. +type InodeNoopWriteOut struct{} // WriteOut is a no-op. -func (NoopWriteOut) WriteOut(context.Context, *fs.Inode) error { +func (InodeNoopWriteOut) WriteOut(context.Context, *fs.Inode) error { return nil } @@ -273,6 +330,11 @@ func (InodeNotDirectory) RemoveDirectory(context.Context, *fs.Inode, string) err return syserror.ENOTDIR } +// Rename implements fs.FileOperations.Rename. +func (InodeNotDirectory) Rename(context.Context, *fs.Inode, string, *fs.Inode, string) error { + return syserror.EINVAL +} + // InodeNotSocket can be used by Inodes that are not sockets. type InodeNotSocket struct{} @@ -281,7 +343,31 @@ func (InodeNotSocket) BoundEndpoint(*fs.Inode, string) transport.BoundEndpoint { return nil } -// InodeNotRenameable can be used by Inodes that cannot be renamed. +// InodeNotTruncatable can be used by Inodes that cannot be truncated. +type InodeNotTruncatable struct{} + +// Truncate implements fs.InodeOperations.Truncate. +func (InodeNotTruncatable) Truncate(context.Context, *fs.Inode, int64) error { + return syserror.EINVAL +} + +// InodeIsDirTruncate implements fs.InodeOperations.Truncate for directories. +type InodeIsDirTruncate struct{} + +// Truncate implements fs.InodeOperations.Truncate. +func (InodeIsDirTruncate) Truncate(context.Context, *fs.Inode, int64) error { + return syserror.EISDIR +} + +// InodeNoopTruncate implements fs.InodeOperations.Truncate as a noop. +type InodeNoopTruncate struct{} + +// Truncate implements fs.InodeOperations.Truncate. +func (InodeNoopTruncate) Truncate(context.Context, *fs.Inode, int64) error { + return nil +} + +// InodeNotRenameable can be used by Inodes that cannot be truncated. type InodeNotRenameable struct{} // Rename implements fs.InodeOperations.Rename. @@ -305,6 +391,14 @@ func (InodeNotVirtual) IsVirtual() bool { return false } +// InodeVirtual can be used by Inodes that are virtual. +type InodeVirtual struct{} + +// IsVirtual implements fs.InodeOperations.IsVirtual. +func (InodeVirtual) IsVirtual() bool { + return true +} + // InodeNotSymlink can be used by Inodes that are not symlinks. type InodeNotSymlink struct{} @@ -337,50 +431,17 @@ func (InodeNoExtendedAttributes) Listxattr(*fs.Inode) (map[string]struct{}, erro return nil, syserror.EOPNOTSUPP } -// DeprecatedFileOperations panics if any deprecated Inode method is called. -type DeprecatedFileOperations struct{} +// InodeNoopRelease implements fs.InodeOperations.Release as a noop. +type InodeNoopRelease struct{} -// Readiness implements fs.InodeOperations.Waitable.Readiness. -func (DeprecatedFileOperations) Readiness(waiter.EventMask) waiter.EventMask { - panic("not implemented") -} - -// EventRegister implements fs.InodeOperations.Waitable.EventRegister. -func (DeprecatedFileOperations) EventRegister(*waiter.Entry, waiter.EventMask) { - panic("not implemented") -} - -// EventUnregister implements fs.InodeOperations.Waitable.EventUnregister. -func (DeprecatedFileOperations) EventUnregister(*waiter.Entry) { - panic("not implemented") -} - -// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. -func (DeprecatedFileOperations) DeprecatedPreadv(context.Context, usermem.IOSequence, int64) (int64, error) { - panic("not implemented") -} - -// DeprecatedPwritev implements fs.InodeOperations.DeprecatedPwritev. -func (DeprecatedFileOperations) DeprecatedPwritev(context.Context, usermem.IOSequence, int64) (int64, error) { - panic("not implemented") -} - -// DeprecatedReaddir implements fs.InodeOperations.DeprecatedReaddir. -func (DeprecatedFileOperations) DeprecatedReaddir(context.Context, *fs.DirCtx, int) (int, error) { - panic("not implemented") -} - -// DeprecatedFsync implements fs.InodeOperations.DeprecatedFsync. -func (DeprecatedFileOperations) DeprecatedFsync() error { - panic("not implemented") -} +// Release implements fs.InodeOperations.Release. +func (InodeNoopRelease) Release(context.Context) {} -// DeprecatedFlush implements fs.InodeOperations.DeprecatedFlush. -func (DeprecatedFileOperations) DeprecatedFlush() error { - panic("not implemented") -} +// InodeGenericChecker implements fs.InodeOperations.Check with a generic +// implementation. +type InodeGenericChecker struct{} -// DeprecatedMappable implements fs.InodeOperations.DeprecatedMappable. -func (DeprecatedFileOperations) DeprecatedMappable(context.Context, *fs.Inode) (memmap.Mappable, bool) { - panic("not implemented") +// Check implements fs.InodeOperations.Check. +func (InodeGenericChecker) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { + return fs.ContextCanAccessFile(ctx, inode, p) } diff --git a/pkg/sentry/fs/fsutil/inode_cached_test.go b/pkg/sentry/fs/fsutil/inode_cached_test.go index ce5201a40..9c9391511 100644 --- a/pkg/sentry/fs/fsutil/inode_cached_test.go +++ b/pkg/sentry/fs/fsutil/inode_cached_test.go @@ -261,15 +261,11 @@ func (noopMappingSpace) Invalidate(ar usermem.AddrRange, opts memmap.InvalidateO } func anonInode(ctx context.Context) *fs.Inode { - return fs.NewInode(NewSimpleInodeOperations(InodeSimpleAttributes{ - UAttr: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: fs.FileOwnerFromContext(ctx), - Perms: fs.FilePermissions{ - User: fs.PermMask{Read: true, Write: true}, - }, - Links: 1, - }), - }), fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}), fs.StableAttr{ + return fs.NewInode(&SimpleFileInode{ + InodeSimpleAttributes: NewInodeSimpleAttributes(ctx, fs.FileOwnerFromContext(ctx), fs.FilePermissions{ + User: fs.PermMask{Read: true, Write: true}, + }, 0), + }, fs.NewPseudoMountSource(), fs.StableAttr{ Type: fs.Anonymous, BlockSize: usermem.PageSize, }) diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go index 6d961813d..3578b07a0 100644 --- a/pkg/sentry/fs/gofer/file.go +++ b/pkg/sentry/fs/gofer/file.go @@ -37,7 +37,7 @@ var openedWX = metric.MustCreateNewUint64Metric("/gofer/opened_write_execute_fil // // +stateify savable type fileOperations struct { - fsutil.NoIoctl `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` waiter.AlwaysReady `state:"nosave"` // inodeOperations is the inodeOperations backing the file. It is protected diff --git a/pkg/sentry/fs/gofer/fs.go b/pkg/sentry/fs/gofer/fs.go index ed30cb1f1..2dc000c6f 100644 --- a/pkg/sentry/fs/gofer/fs.go +++ b/pkg/sentry/fs/gofer/fs.go @@ -75,11 +75,11 @@ var ( // ErrNoTransport is returned when there is no 'trans' option. ErrNoTransport = errors.New("missing required option: 'trans='") - // ErrNoReadFD is returned when there is no 'rfdno' option. - ErrNoReadFD = errors.New("missing required option: 'rfdno='") + // ErrFileNoReadFD is returned when there is no 'rfdno' option. + ErrFileNoReadFD = errors.New("missing required option: 'rfdno='") - // ErrNoWriteFD is returned when there is no 'wfdno' option. - ErrNoWriteFD = errors.New("missing required option: 'wfdno='") + // ErrFileNoWriteFD is returned when there is no 'wfdno' option. + ErrFileNoWriteFD = errors.New("missing required option: 'wfdno='") ) // filesystem is a 9p client. @@ -87,6 +87,8 @@ var ( // +stateify savable type filesystem struct{} +var _ fs.Filesystem = (*filesystem)(nil) + func init() { fs.RegisterFilesystem(&filesystem{}) } @@ -160,14 +162,14 @@ func options(data string) (opts, error) { // Check for the required 'rfdno=' option. srfd, ok := options[readFDKey] if !ok { - return o, ErrNoReadFD + return o, ErrFileNoReadFD } delete(options, readFDKey) // Check for the required 'wfdno=' option. swfd, ok := options[writeFDKey] if !ok { - return o, ErrNoWriteFD + return o, ErrFileNoWriteFD } delete(options, writeFDKey) diff --git a/pkg/sentry/fs/gofer/inode.go b/pkg/sentry/fs/gofer/inode.go index 7c6e5b025..f0dc99fd0 100644 --- a/pkg/sentry/fs/gofer/inode.go +++ b/pkg/sentry/fs/gofer/inode.go @@ -40,7 +40,6 @@ import ( type inodeOperations struct { fsutil.InodeNotVirtual `state:"nosave"` fsutil.InodeNoExtendedAttributes `state:"nosave"` - fsutil.DeprecatedFileOperations `state:"nosave"` // fileState implements fs.CachedFileObject. It exists // to break a circular load dependency between inodeOperations diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go index bc6ee7aa4..4e84d1d6c 100644 --- a/pkg/sentry/fs/host/file.go +++ b/pkg/sentry/fs/host/file.go @@ -36,8 +36,8 @@ import ( // // +stateify savable type fileOperations struct { - fsutil.NoIoctl `state:"nosave"` - fsutil.NoopRelease `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` // iops are the Inode operations for this file. iops *inodeOperations `state:"wait"` diff --git a/pkg/sentry/fs/host/fs.go b/pkg/sentry/fs/host/fs.go index 54cbb94f9..d2ba38449 100644 --- a/pkg/sentry/fs/host/fs.go +++ b/pkg/sentry/fs/host/fs.go @@ -58,6 +58,8 @@ type Filesystem struct { paths []string } +var _ fs.Filesystem = (*Filesystem)(nil) + // Name is the identifier of this file system. func (*Filesystem) Name() string { return FilesystemName diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go index 08754bd6b..6ff6c3254 100644 --- a/pkg/sentry/fs/host/inode.go +++ b/pkg/sentry/fs/host/inode.go @@ -39,7 +39,6 @@ import ( type inodeOperations struct { fsutil.InodeNotVirtual `state:"nosave"` fsutil.InodeNoExtendedAttributes `state:"nosave"` - fsutil.DeprecatedFileOperations `state:"nosave"` // fileState implements fs.CachedFileObject. It exists // to break a circular load dependency between inodeOperations diff --git a/pkg/sentry/fs/inode.go b/pkg/sentry/fs/inode.go index fa3beb111..d32f52d55 100644 --- a/pkg/sentry/fs/inode.go +++ b/pkg/sentry/fs/inode.go @@ -356,11 +356,10 @@ func (i *Inode) AddLink() { if i.overlay != nil { // FIXME: Remove this from InodeOperations altogether. // - // This interface (including DropLink and NotifyStatusChange) - // is only used by ramfs to update metadata of children. These - // filesystems should _never_ have overlay Inodes cached as - // children. So explicitly disallow this scenario and avoid plumbing - // Dirents through to do copy up. + // This interface is only used by ramfs to update metadata of + // children. These filesystems should _never_ have overlay + // Inodes cached as children. So explicitly disallow this + // scenario and avoid plumbing Dirents through to do copy up. panic("overlay Inodes cached in ramfs directories are not supported") } i.InodeOperations.AddLink() @@ -375,15 +374,6 @@ func (i *Inode) DropLink() { i.InodeOperations.DropLink() } -// NotifyStatusChange calls i.InodeOperations.NotifyStatusChange. -func (i *Inode) NotifyStatusChange(ctx context.Context) { - if i.overlay != nil { - // Same as AddLink. - panic("overlay Inodes cached in ramfs directories are not supported") - } - i.InodeOperations.NotifyStatusChange(ctx) -} - // IsVirtual calls i.InodeOperations.IsVirtual. func (i *Inode) IsVirtual() bool { if i.overlay != nil { @@ -401,17 +391,6 @@ func (i *Inode) StatFS(ctx context.Context) (Info, error) { return i.InodeOperations.StatFS(ctx) } -// HandleOps extracts HandleOperations from i. -func (i *Inode) HandleOps() HandleOperations { - if i.overlay != nil { - return overlayHandleOps(i.overlay) - } - if h, ok := i.InodeOperations.(HandleOperations); ok { - return h - } - return nil -} - // CheckOwnership checks whether `ctx` owns this Inode or may act as its owner. // Compare Linux's fs/inode.c:inode_owner_or_capable(). func (i *Inode) CheckOwnership(ctx context.Context) bool { diff --git a/pkg/sentry/fs/inode_operations.go b/pkg/sentry/fs/inode_operations.go index 77973ce79..db40b5256 100644 --- a/pkg/sentry/fs/inode_operations.go +++ b/pkg/sentry/fs/inode_operations.go @@ -21,8 +21,6 @@ import ( ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/waiter" ) var ( @@ -303,83 +301,5 @@ type InodeOperations interface { // StatFS returns a filesystem Info implementation or an error. If // the filesystem does not support this operation (maybe in the future // it will), then ENOSYS should be returned. - // - // Move to MountSourceOperations. StatFS(context.Context) (Info, error) - - HandleOperations -} - -// HandleOperations are extended InodeOperations that are only implemented -// for file systems that use fs/handle.go:Handle to generate open Files. -// -// Handle is deprecated; these methods are deprecated as well. -// -// Filesystems are encouraged to implement the File interface directly -// instead of using Handle. To indicate that the below methods should never -// be called, embed DeprecatedFileOperations to satisfy this interface. -type HandleOperations interface { - waiter.Waitable - - // DeprecatedPreadv is deprecated in favor of filesystems - // implementing File.Preadv directly. - // - // DeprecatedPreadv reads up to dst.NumBytes() bytes into dst, starting at - // the given offset, and returns the number of bytes read. - // - // Preadv may return a partial read result before EOF is reached. - // - // If a symlink, Preadv reads the target value of the symlink. - // - // Preadv should not check for readable permissions. - DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) - - // DeprecatedPwritev is deprecated in favor of filesystems - // implementing File.Pwritev directly. - // - // DeprecatedPwritev writes up to src.NumBytes() bytes from src to the - // Inode, starting at the given offset and returns the number of bytes - // written. - // - // Pwritev should not check that the Inode has writable permissions. - DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) - - // DeprecatedReaddir is deprecated in favor of filesystems - // implementing File.Readdir directly. - // - // DeprecatedReaddir emits directory entries by calling dirCtx.EmitDir, - // beginning with the entry at offset. - // - // Entries for "." and ".." must *not* be included. - // - // If the offset returned is the same as the argument offset, then - // nothing has been serialized. This is equivalent to reaching EOF. - // In this case serializer.Written() should return 0. - // - // The order of entries to emit must be consistent between Readdir - // calls, and must start with the given offset. - // - // The caller must ensure that this operation is permitted. - DeprecatedReaddir(ctx context.Context, dirCtx *DirCtx, offset int) (int, error) - - // DeprecatedFsync is deprecated in favor of filesystems implementing - // File.Fsync directly. - // - // DeprecatedFsync syncs a file. - DeprecatedFsync() error - - // DeprecatedMappable is deprecated in favor of filesystems implementing - // File.Mappable directly. - // - // DeprecatedMappable returns a Mappable if the Inode can be mapped. - DeprecatedMappable(ctx context.Context, inode *Inode) (memmap.Mappable, bool) - - // DeprecatedFlush is deprecated in favor of filesystems implementing - // File.Flush directly. - // - // DeprecatedFlush flushes a file. - // - // Implementations may choose to free up memory or complete pending I/O - // but also may implement Flush as a no-op. - DeprecatedFlush() error } diff --git a/pkg/sentry/fs/inode_overlay.go b/pkg/sentry/fs/inode_overlay.go index 512a0da28..77a2623ef 100644 --- a/pkg/sentry/fs/inode_overlay.go +++ b/pkg/sentry/fs/inode_overlay.go @@ -596,19 +596,6 @@ func overlayStatFS(ctx context.Context, o *overlayEntry) (Info, error) { return i, nil } -func overlayHandleOps(o *overlayEntry) HandleOperations { - // Hot path. Avoid defers. - var hops HandleOperations - o.copyMu.RLock() - if o.upper != nil { - hops = o.upper.HandleOps() - } else { - hops = o.lower.HandleOps() - } - o.copyMu.RUnlock() - return hops -} - // NewTestOverlayDir returns an overlay Inode for tests. // // If `revalidate` is true, then the upper filesystem will require diff --git a/pkg/sentry/fs/inode_overlay_test.go b/pkg/sentry/fs/inode_overlay_test.go index 9e922d008..bc91be226 100644 --- a/pkg/sentry/fs/inode_overlay_test.go +++ b/pkg/sentry/fs/inode_overlay_test.go @@ -19,7 +19,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - ramfstest "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs/test" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/contexttest" "gvisor.googlesource.com/gvisor/pkg/syserror" ) @@ -376,7 +377,8 @@ type dir struct { // List of negative child names. negative []string - // Whether DeprecatedReaddir has been called on this dir. + // ReaddirCalled records whether Readdir was called on a file + // corresponding to this inode. ReaddirCalled bool } @@ -390,10 +392,19 @@ func (d *dir) Getxattr(inode *fs.Inode, name string) ([]byte, error) { return nil, syserror.ENOATTR } -// DeprecatedReaddir implements InodeOperations.DeprecatedReaddir. -func (d *dir) DeprecatedReaddir(ctx context.Context, dirctx *fs.DirCtx, offset int) (int, error) { - d.ReaddirCalled = true - return d.InodeOperations.DeprecatedReaddir(ctx, dirctx, offset) +// GetFile implements InodeOperations.GetFile. +func (d *dir) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + file, err := d.InodeOperations.GetFile(ctx, dirent, flags) + if err != nil { + return nil, err + } + defer file.DecRef() + // Wrap the file's FileOperations in a dirFile. + fops := &dirFile{ + FileOperations: file.FileOperations, + inode: d, + } + return fs.NewFile(ctx, dirent, flags, fops), nil } type dirContent struct { @@ -401,12 +412,45 @@ type dirContent struct { dir bool } +type dirFile struct { + fs.FileOperations + inode *dir +} + +type inode struct { + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeNotTruncatable `state:"nosave"` + fsutil.InodeNotVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes + fsutil.InodeStaticFileGetter +} + +// Readdir implements fs.FileOperations.Readdir. It sets the ReaddirCalled +// field on the inode. +func (f *dirFile) Readdir(ctx context.Context, file *fs.File, ser fs.DentrySerializer) (int64, error) { + f.inode.ReaddirCalled = true + return f.FileOperations.Readdir(ctx, file, ser) +} + func newTestRamfsInode(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - return fs.NewInode(ramfstest.NewFile(ctx, fs.FilePermissions{}), msrc, fs.StableAttr{Type: fs.RegularFile}) + inode := fs.NewInode(&inode{ + InodeStaticFileGetter: fsutil.InodeStaticFileGetter{ + Contents: []byte("foobar"), + }, + }, msrc, fs.StableAttr{Type: fs.RegularFile}) + return inode } func newTestRamfsDir(ctx context.Context, contains []dirContent, negative []string) *fs.Inode { - msrc := fs.NewCachingMountSource(nil, fs.MountSourceFlags{}) + msrc := fs.NewPseudoMountSource() contents := make(map[string]*fs.Inode) for _, c := range contains { if c.dir { @@ -415,7 +459,7 @@ func newTestRamfsDir(ctx context.Context, contains []dirContent, negative []stri contents[c.name] = newTestRamfsInode(ctx, msrc) } } - dops := ramfstest.NewDir(ctx, contents, fs.FilePermissions{ + dops := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermissions{ User: fs.PermMask{Read: true, Execute: true}, }) return fs.NewInode(&dir{ diff --git a/pkg/sentry/fs/mock.go b/pkg/sentry/fs/mock.go index 6bfcda6bb..abfdc6a25 100644 --- a/pkg/sentry/fs/mock.go +++ b/pkg/sentry/fs/mock.go @@ -16,7 +16,6 @@ package fs import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" ) @@ -151,16 +150,6 @@ func (n *MockInodeOperations) Truncate(ctx context.Context, inode *Inode, size i return nil } -// DeprecatedPwritev implements fs.InodeOperations.DeprecatedPwritev. -func (n *MockInodeOperations) DeprecatedPwritev(context.Context, usermem.IOSequence, int64) (int64, error) { - return 0, nil -} - -// DeprecatedReaddir implements fs.InodeOperations.DeprecatedReaddir. -func (n *MockInodeOperations) DeprecatedReaddir(context.Context, *DirCtx, int) (int, error) { - return 0, nil -} - // Remove implements fs.InodeOperations.Remove. func (n *MockInodeOperations) Remove(context.Context, *Inode, string) error { return nil diff --git a/pkg/sentry/fs/mount.go b/pkg/sentry/fs/mount.go index 24e28ddb2..dd6e64b4c 100644 --- a/pkg/sentry/fs/mount.go +++ b/pkg/sentry/fs/mount.go @@ -43,8 +43,6 @@ type DirentOperations interface { // MountSourceOperations contains filesystem specific operations. type MountSourceOperations interface { // TODO: Add: - // - // StatFS() (Info, error) // BlockSize() int64 // FS() Filesystem @@ -249,7 +247,7 @@ func (msrc *MountSource) FlushDirentRefs() { } // NewCachingMountSource returns a generic mount that will cache dirents -// aggressively. Filesystem may be nil if there is no backing filesystem. +// aggressively. func NewCachingMountSource(filesystem Filesystem, flags MountSourceFlags) *MountSource { return NewMountSource(&SimpleMountSourceOperations{ keep: true, @@ -258,7 +256,6 @@ func NewCachingMountSource(filesystem Filesystem, flags MountSourceFlags) *Mount } // NewNonCachingMountSource returns a generic mount that will never cache dirents. -// Filesystem may be nil if there is no backing filesystem. func NewNonCachingMountSource(filesystem Filesystem, flags MountSourceFlags) *MountSource { return NewMountSource(&SimpleMountSourceOperations{ keep: false, @@ -275,6 +272,15 @@ func NewRevalidatingMountSource(filesystem Filesystem, flags MountSourceFlags) * }, filesystem, flags) } +// NewPseudoMountSource returns a "pseudo" mount source that is not backed by +// an actual filesystem. It is always non-caching. +func NewPseudoMountSource() *MountSource { + return NewMountSource(&SimpleMountSourceOperations{ + keep: false, + revalidate: false, + }, nil, MountSourceFlags{}) +} + // SimpleMountSourceOperations implements MountSourceOperations. // // +stateify savable diff --git a/pkg/sentry/fs/mounts_test.go b/pkg/sentry/fs/mounts_test.go index 7d682d99b..54000614f 100644 --- a/pkg/sentry/fs/mounts_test.go +++ b/pkg/sentry/fs/mounts_test.go @@ -19,7 +19,8 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - ramfstest "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs/test" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/contexttest" ) @@ -29,15 +30,15 @@ import ( // |-bar (file) func createMountNamespace(ctx context.Context) (*fs.MountNamespace, error) { perms := fs.FilePermsFromMode(0777) - m := fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}) + m := fs.NewPseudoMountSource() - barFile := ramfstest.NewFile(ctx, perms) - fooDir := ramfstest.NewDir(ctx, map[string]*fs.Inode{ + barFile := fsutil.NewSimpleFileInode(ctx, fs.RootOwner, perms, 0) + fooDir := ramfs.NewDir(ctx, map[string]*fs.Inode{ "bar": fs.NewInode(barFile, m, fs.StableAttr{Type: fs.RegularFile}), - }, perms) - rootDir := ramfstest.NewDir(ctx, map[string]*fs.Inode{ + }, fs.RootOwner, perms) + rootDir := ramfs.NewDir(ctx, map[string]*fs.Inode{ "foo": fs.NewInode(fooDir, m, fs.StableAttr{Type: fs.Directory}), - }, perms) + }, fs.RootOwner, perms) return fs.NewMountNamespace(ctx, fs.NewInode(rootDir, m, fs.StableAttr{Type: fs.Directory})) } diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD index aff3c3c01..74954f213 100644 --- a/pkg/sentry/fs/proc/BUILD +++ b/pkg/sentry/fs/proc/BUILD @@ -8,9 +8,9 @@ go_library( "cpuinfo.go", "exec_args.go", "fds.go", - "file.go", "filesystems.go", "fs.go", + "inode.go", "loadavg.go", "meminfo.go", "mounts.go", @@ -32,6 +32,7 @@ go_library( "//pkg/abi/linux", "//pkg/sentry/context", "//pkg/sentry/fs", + "//pkg/sentry/fs/fsutil", "//pkg/sentry/fs/proc/device", "//pkg/sentry/fs/proc/seqfile", "//pkg/sentry/fs/ramfs", @@ -45,6 +46,7 @@ go_library( "//pkg/sentry/usage", "//pkg/sentry/usermem", "//pkg/syserror", + "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/proc/cpuinfo.go b/pkg/sentry/fs/proc/cpuinfo.go index f8be06dc3..f756c45bf 100644 --- a/pkg/sentry/fs/proc/cpuinfo.go +++ b/pkg/sentry/fs/proc/cpuinfo.go @@ -15,52 +15,21 @@ package proc import ( - "io" - "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" ) -// cpuinfo is a file describing the CPU capabilities. -// -// Presently cpuinfo never changes, so it doesn't need to be a SeqFile. -// -// +stateify savable -type cpuinfo struct { - ramfs.Entry - - // k is the system kernel. - k *kernel.Kernel -} - -// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. -func (c *cpuinfo) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - features := c.k.FeatureSet() +func newCPUInfo(ctx context.Context, msrc *fs.MountSource) *fs.Inode { + k := kernel.KernelFromContext(ctx) + features := k.FeatureSet() if features == nil { // Kernel is always initialized with a FeatureSet. panic("cpuinfo read with nil FeatureSet") } - contents := make([]byte, 0, 1024) - for i, max := uint(0), c.k.ApplicationCores(); i < max; i++ { + for i, max := uint(0), k.ApplicationCores(); i < max; i++ { contents = append(contents, []byte(features.CPUInfo(i))...) } - if offset >= int64(len(contents)) { - return 0, io.EOF - } - - n, err := dst.CopyOut(ctx, contents[offset:]) - return int64(n), err -} - -func (p *proc) newCPUInfo(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - f := &cpuinfo{ - k: p.k, - } - f.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0444)) - - return newFile(f, msrc, fs.SpecialFile, nil) + return newStaticProcInode(ctx, msrc, contents) } diff --git a/pkg/sentry/fs/proc/exec_args.go b/pkg/sentry/fs/proc/exec_args.go index b4896053f..ddda67f54 100644 --- a/pkg/sentry/fs/proc/exec_args.go +++ b/pkg/sentry/fs/proc/exec_args.go @@ -18,12 +18,14 @@ import ( "fmt" "io" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // execArgType enumerates the types of exec arguments that are exposed through @@ -35,12 +37,12 @@ const ( environExecArg ) -// execArgFile is a file containing the exec args (either cmdline or environ) +// execArgInode is a inode containing the exec args (either cmdline or environ) // for a given task. // // +stateify savable -type execArgFile struct { - ramfs.Entry +type execArgInode struct { + fsutil.SimpleFileInode // arg is the type of exec argument this file contains. arg execArgType @@ -49,21 +51,52 @@ type execArgFile struct { t *kernel.Task } +var _ fs.InodeOperations = (*execArgInode)(nil) + // newExecArgFile creates a file containing the exec args of the given type. -func newExecArgFile(t *kernel.Task, msrc *fs.MountSource, arg execArgType) *fs.Inode { +func newExecArgInode(t *kernel.Task, msrc *fs.MountSource, arg execArgType) *fs.Inode { if arg != cmdlineExecArg && arg != environExecArg { panic(fmt.Sprintf("unknown exec arg type %v", arg)) } - f := &execArgFile{ - arg: arg, - t: t, + f := &execArgInode{ + SimpleFileInode: *fsutil.NewSimpleFileInode(t, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + arg: arg, + t: t, } - f.InitEntry(t, fs.RootOwner, fs.FilePermsFromMode(0444)) - return newFile(f, msrc, fs.SpecialFile, t) + return newProcInode(f, msrc, fs.SpecialFile, t) +} + +// GetFile implements fs.InodeOperations.GetFile. +func (i *execArgInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &execArgFile{ + arg: i.arg, + t: i.t, + }), nil } -// DeprecatedPreadv reads the exec arg from the process's address space.. -func (f *execArgFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { +// +stateify savable +type execArgFile struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopWrite `state:"nosave"` + + // arg is the type of exec argument this file contains. + arg execArgType + + // t is the Task to read the exec arg line from. + t *kernel.Task +} + +var _ fs.FileOperations = (*execArgFile)(nil) + +// Read reads the exec arg from the process's address space.. +func (f *execArgFile) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { if offset < 0 { return 0, syserror.EINVAL } diff --git a/pkg/sentry/fs/proc/fds.go b/pkg/sentry/fs/proc/fds.go index 5acbce75e..b8a0a5eff 100644 --- a/pkg/sentry/fs/proc/fds.go +++ b/pkg/sentry/fs/proc/fds.go @@ -21,11 +21,11 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/device" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" ) @@ -55,7 +55,7 @@ func walkDescriptors(t *kernel.Task, p string, toInode func(*fs.File, kernel.FDF // readDescriptors reads fds in the task starting at offset, and calls the // toDentAttr callback for each to get a DentAttr, which it then emits. This is // a helper for implementing fs.InodeOperations.Readdir. -func readDescriptors(t *kernel.Task, c *fs.DirCtx, offset int, toDentAttr func(int) fs.DentAttr) (int, error) { +func readDescriptors(t *kernel.Task, c *fs.DirCtx, offset int64, toDentAttr func(int) fs.DentAttr) (int64, error) { var fds kernel.FDs t.WithMuLocked(func(t *kernel.Task) { if fdm := t.FDMap(); fdm != nil { @@ -69,7 +69,7 @@ func readDescriptors(t *kernel.Task, c *fs.DirCtx, offset int, toDentAttr func(i } // Find the fd to start at. - idx := sort.SearchInts(fdInts, offset) + idx := sort.SearchInts(fdInts, int(offset)) if idx == len(fdInts) { return offset, nil } @@ -80,28 +80,32 @@ func readDescriptors(t *kernel.Task, c *fs.DirCtx, offset int, toDentAttr func(i name := strconv.FormatUint(uint64(fd), 10) if err := c.DirEmit(name, toDentAttr(fd)); err != nil { // Returned offset is the next fd to serialize. - return fd, err + return int64(fd), err } } // We serialized them all. Next offset should be higher than last // serialized fd. - return fd + 1, nil + return int64(fd + 1), nil } -// fd is a single file in /proc/TID/fd/. +// fd implements fs.InodeOperations for a file in /proc/TID/fd/. type fd struct { ramfs.Symlink *fs.File } +var _ fs.InodeOperations = (*fd)(nil) + // newFd returns a new fd based on an existing file. // // This inherits one reference to the file. func newFd(t *kernel.Task, f *fs.File, msrc *fs.MountSource) *fs.Inode { - fd := &fd{File: f} - // RootOwner by default, is overridden in UnstableAttr() - fd.InitSymlink(t, fs.RootOwner, "") - return newFile(fd, msrc, fs.Symlink, t) + fd := &fd{ + // RootOwner overridden by taskOwnedInodeOps.UnstableAttrs(). + Symlink: *ramfs.NewSymlink(t, fs.RootOwner, ""), + File: f, + } + return newProcInode(fd, msrc, fs.Symlink, t) } // GetFile returns the fs.File backing this fd. The dirent and flags @@ -142,7 +146,7 @@ func (f *fd) Close() error { return nil } -// fdDir implements /proc/TID/fd. +// fdDir is an InodeOperations for /proc/TID/fd. // // +stateify savable type fdDir struct { @@ -154,11 +158,15 @@ type fdDir struct { t *kernel.Task } +var _ fs.InodeOperations = (*fdDir)(nil) + // newFdDir creates a new fdDir. func newFdDir(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - f := &fdDir{t: t} - f.InitDir(t, nil, fs.RootOwner, fs.FilePermissions{User: fs.PermMask{Read: true, Execute: true}}) - return newFile(f, msrc, fs.SpecialDirectory, t) + f := &fdDir{ + Dir: *ramfs.NewDir(t, nil, fs.RootOwner, fs.FilePermissions{User: fs.PermMask{Read: true, Execute: true}}), + t: t, + } + return newProcInode(f, msrc, fs.SpecialDirectory, t) } // Check implements InodeOperations.Check. @@ -191,49 +199,55 @@ func (f *fdDir) Lookup(ctx context.Context, dir *fs.Inode, p string) (*fs.Dirent return fs.NewDirent(n, p), nil } -// DeprecatedReaddir lists fds in /proc/TID/fd. -func (f *fdDir) DeprecatedReaddir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { - return readDescriptors(f.t, dirCtx, offset, func(fd int) fs.DentAttr { - return fs.GenericDentAttr(fs.Symlink, device.ProcDevice) +// GetFile implements fs.FileOperations.GetFile. +func (f *fdDir) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + fops := &fdDirFile{ + isInfoFile: false, + t: f.t, + } + return fs.NewFile(ctx, dirent, flags, fops), nil +} + +// +stateify savable +type fdDirFile struct { + fsutil.DirFileOperations `state:"nosave"` + + isInfoFile bool + + t *kernel.Task +} + +var _ fs.FileOperations = (*fdDirFile)(nil) + +// Readdir implements fs.FileOperations.Readdir. +func (f *fdDirFile) Readdir(ctx context.Context, file *fs.File, ser fs.DentrySerializer) (int64, error) { + dirCtx := &fs.DirCtx{ + Serializer: ser, + } + typ := fs.RegularFile + if f.isInfoFile { + typ = fs.Symlink + } + return readDescriptors(f.t, dirCtx, file.Offset(), func(fd int) fs.DentAttr { + return fs.GenericDentAttr(typ, device.ProcDevice) }) } -// fdInfo is a single file in /proc/TID/fdinfo/. +// fdInfoInode is a single file in /proc/TID/fdinfo/. // // +stateify savable -type fdInfo struct { - ramfs.File +type fdInfoInode struct { + staticFileInodeOps file *fs.File flags fs.FileFlags fdFlags kernel.FDFlags } -// newFdInfo returns a new fdInfo based on an existing file. -func newFdInfo(t *kernel.Task, file *fs.File, fdFlags kernel.FDFlags, msrc *fs.MountSource) *fs.Inode { - fdi := &fdInfo{file: file, flags: file.Flags(), fdFlags: fdFlags} - fdi.InitFile(t, fs.RootOwner, fs.FilePermissions{User: fs.PermMask{Read: true}}) - // TODO: Get pos, locks, and other data. For now we only - // have flags. - // See https://www.kernel.org/doc/Documentation/filesystems/proc.txt - - flags := file.Flags().ToLinux() | fdFlags.ToLinuxFileFlags() - fdi.Append([]byte(fmt.Sprintf("flags:\t0%o\n", flags))) - return newFile(fdi, msrc, fs.SpecialFile, t) -} - -// DeprecatedPwritev implements fs.HandleOperations.DeprecatedPwritev. -func (*fdInfo) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { - return 0, ramfs.ErrInvalidOp -} - -// Truncate implements fs.InodeOperations.Truncate. -func (*fdInfo) Truncate(ctx context.Context, inode *fs.Inode, size int64) error { - return ramfs.ErrInvalidOp -} +var _ fs.InodeOperations = (*fdInfoInode)(nil) -func (f *fdInfo) Release(ctx context.Context) { - f.File.Release(ctx) +// Release implements fs.InodeOperations.Release. +func (f *fdInfoInode) Release(ctx context.Context) { f.file.DecRef() } @@ -249,25 +263,37 @@ type fdInfoDir struct { // newFdInfoDir creates a new fdInfoDir. func newFdInfoDir(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - fdid := &fdInfoDir{t: t} - fdid.InitDir(t, nil, fs.RootOwner, fs.FilePermsFromMode(0500)) - return newFile(fdid, msrc, fs.SpecialDirectory, t) + fdid := &fdInfoDir{ + Dir: *ramfs.NewDir(t, nil, fs.RootOwner, fs.FilePermsFromMode(0500)), + t: t, + } + return newProcInode(fdid, msrc, fs.SpecialDirectory, t) } // Lookup loads an fd in /proc/TID/fdinfo into a Dirent. func (fdid *fdInfoDir) Lookup(ctx context.Context, dir *fs.Inode, p string) (*fs.Dirent, error) { - n, err := walkDescriptors(fdid.t, p, func(file *fs.File, fdFlags kernel.FDFlags) *fs.Inode { - return newFdInfo(fdid.t, file, fdFlags, dir.MountSource) + inode, err := walkDescriptors(fdid.t, p, func(file *fs.File, fdFlags kernel.FDFlags) *fs.Inode { + // TODO: Using a static inode here means that the + // data can be out-of-date if, for instance, the flags on the + // FD change before we read this file. We should switch to + // generating the data on Read(). Also, we should include pos, + // locks, and other data. For now we only have flags. + // See https://www.kernel.org/doc/Documentation/filesystems/proc.txt + flags := file.Flags().ToLinux() | fdFlags.ToLinuxFileFlags() + contents := []byte(fmt.Sprintf("flags:\t0%o\n", flags)) + return newStaticProcInode(ctx, dir.MountSource, contents) }) if err != nil { return nil, err } - return fs.NewDirent(n, p), nil + return fs.NewDirent(inode, p), nil } -// DeprecatedReaddir lists fds in /proc/TID/fdinfo. -func (fdid *fdInfoDir) DeprecatedReaddir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { - return readDescriptors(fdid.t, dirCtx, offset, func(fd int) fs.DentAttr { - return fs.GenericDentAttr(fs.RegularFile, device.ProcDevice) - }) +// GetFile implements fs.FileOperations.GetFile. +func (fdid *fdInfoDir) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + fops := &fdDirFile{ + isInfoFile: true, + t: fdid.t, + } + return fs.NewFile(ctx, dirent, flags, fops), nil } diff --git a/pkg/sentry/fs/proc/file.go b/pkg/sentry/fs/proc/file.go deleted file mode 100644 index f659e590a..000000000 --- a/pkg/sentry/fs/proc/file.go +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package proc - -import ( - "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/device" - "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" -) - -// +stateify savable -type file struct { - fs.InodeOperations - - // nodeType is the file type of this file. - nodeType fs.InodeType - - // t is the associated kernel task that owns this file. - t *kernel.Task -} - -func newFile(node fs.InodeOperations, msrc *fs.MountSource, nodeType fs.InodeType, t *kernel.Task) *fs.Inode { - iops := &file{node, nodeType, t} - sattr := fs.StableAttr{ - DeviceID: device.ProcDevice.DeviceID(), - InodeID: device.ProcDevice.NextIno(), - BlockSize: usermem.PageSize, - Type: nodeType, - } - return fs.NewInode(iops, msrc, sattr) -} - -// UnstableAttr returns all attributes of this file. -func (f *file) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - uattr, err := f.InodeOperations.UnstableAttr(ctx, inode) - if err != nil { - return fs.UnstableAttr{}, err - } - if f.t != nil { - creds := f.t.Credentials() - uattr.Owner = fs.FileOwner{creds.EffectiveKUID, creds.EffectiveKGID} - } - return uattr, nil -} diff --git a/pkg/sentry/fs/proc/inode.go b/pkg/sentry/fs/proc/inode.go new file mode 100644 index 000000000..3c36af5ea --- /dev/null +++ b/pkg/sentry/fs/proc/inode.go @@ -0,0 +1,96 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package proc + +import ( + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/device" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +// taskOwnedInodeOps wraps an fs.InodeOperations and overrides the UnstableAttr +// method to return the task as the owner. +// +// +stateify savable +type taskOwnedInodeOps struct { + fs.InodeOperations + + // t is the task that owns this file. + t *kernel.Task +} + +// UnstableAttr implement fs.InodeOperations.UnstableAttr. +func (i *taskOwnedInodeOps) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { + uattr, err := i.InodeOperations.UnstableAttr(ctx, inode) + if err != nil { + return fs.UnstableAttr{}, err + } + // Set the task owner as the file owner. + creds := i.t.Credentials() + uattr.Owner = fs.FileOwner{creds.EffectiveKUID, creds.EffectiveKGID} + return uattr, nil +} + +// staticFileInodeOps is an InodeOperations implementation that can be used to +// return file contents which are constant. This file is not writable and will +// always have mode 0444. +// +// +stateify savable +type staticFileInodeOps struct { + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopTruncate `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes + fsutil.InodeStaticFileGetter +} + +var _ fs.InodeOperations = (*staticFileInodeOps)(nil) + +// newStaticFileInode returns a procfs InodeOperations with static contents. +func newStaticProcInode(ctx context.Context, msrc *fs.MountSource, contents []byte) *fs.Inode { + iops := &staticFileInodeOps{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + InodeStaticFileGetter: fsutil.InodeStaticFileGetter{ + Contents: contents, + }, + } + return newProcInode(iops, msrc, fs.SpecialFile, nil) +} + +// newProcInode creates a new inode from the given inode operations. +func newProcInode(iops fs.InodeOperations, msrc *fs.MountSource, typ fs.InodeType, t *kernel.Task) *fs.Inode { + sattr := fs.StableAttr{ + DeviceID: device.ProcDevice.DeviceID(), + InodeID: device.ProcDevice.NextIno(), + BlockSize: usermem.PageSize, + Type: typ, + } + if t != nil { + iops = &taskOwnedInodeOps{iops, t} + } + return fs.NewInode(iops, msrc, sattr) +} diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go index 2806d6035..3ed85a538 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -28,35 +28,36 @@ import ( // newNet creates a new proc net entry. func (p *proc) newNetDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) + var contents map[string]*fs.Inode if s := p.k.NetworkStack(); s != nil && s.SupportsIPv6() { - d.AddChild(ctx, "dev", seqfile.NewSeqFileInode(ctx, &netDev{s: s}, msrc)) - d.AddChild(ctx, "if_inet6", seqfile.NewSeqFileInode(ctx, &ifinet6{s: s}, msrc)) - - // The following files are simple stubs until they are implemented in - // netstack, if the file contains a header the stub is just the header - // otherwise it is an empty file. - d.AddChild(ctx, "arp", p.newStubProcFSFile(ctx, msrc, []byte("IP address HW type Flags HW address Mask Device"))) - d.AddChild(ctx, "ipv6_route", p.newStubProcFSFile(ctx, msrc, []byte(""))) - d.AddChild(ctx, "netlink", p.newStubProcFSFile(ctx, msrc, []byte("sk Eth Pid Groups Rmem Wmem Dump Locks Drops Inode"))) - d.AddChild(ctx, "netstat", p.newStubProcFSFile(ctx, msrc, []byte("TcpExt: SyncookiesSent SyncookiesRecv SyncookiesFailed EmbryonicRsts PruneCalled RcvPruned OfoPruned OutOfWindowIcmps LockDroppedIcmps ArpFilter TW TWRecycled TWKilled PAWSPassive PAWSActive PAWSEstab DelayedACKs DelayedACKLocked DelayedACKLost ListenOverflows ListenDrops TCPPrequeued TCPDirectCopyFromBacklog TCPDirectCopyFromPrequeue TCPPrequeueDropped TCPHPHits TCPHPHitsToUser TCPPureAcks TCPHPAcks TCPRenoRecovery TCPSackRecovery TCPSACKReneging TCPFACKReorder TCPSACKReorder TCPRenoReorder TCPTSReorder TCPFullUndo TCPPartialUndo TCPDSACKUndo TCPLossUndo TCPLostRetransmit TCPRenoFailures TCPSackFailures TCPLossFailures TCPFastRetrans TCPForwardRetrans TCPSlowStartRetrans TCPTimeouts TCPLossProbes TCPLossProbeRecovery TCPRenoRecoveryFail TCPSackRecoveryFail TCPSchedulerFailed TCPRcvCollapsed TCPDSACKOldSent TCPDSACKOfoSent TCPDSACKRecv TCPDSACKOfoRecv TCPAbortOnData TCPAbortOnClose TCPAbortOnMemory TCPAbortOnTimeout TCPAbortOnLinger TCPAbortFailed TCPMemoryPressures TCPSACKDiscard TCPDSACKIgnoredOld TCPDSACKIgnoredNoUndo TCPSpuriousRTOs TCPMD5NotFound TCPMD5Unexpected TCPMD5Failure TCPSackShifted TCPSackMerged TCPSackShiftFallback TCPBacklogDrop TCPMinTTLDrop TCPDeferAcceptDrop IPReversePathFilter TCPTimeWaitOverflow TCPReqQFullDoCookies TCPReqQFullDrop TCPRetransFail TCPRcvCoalesce TCPOFOQueue TCPOFODrop TCPOFOMerge TCPChallengeACK TCPSYNChallenge TCPFastOpenActive TCPFastOpenActiveFail TCPFastOpenPassive TCPFastOpenPassiveFail TCPFastOpenListenOverflow TCPFastOpenCookieReqd TCPSpuriousRtxHostQueues BusyPollRxPackets TCPAutoCorking TCPFromZeroWindowAdv TCPToZeroWindowAdv TCPWantZeroWindowAdv TCPSynRetrans TCPOrigDataSent TCPHystartTrainDetect TCPHystartTrainCwnd TCPHystartDelayDetect TCPHystartDelayCwnd TCPACKSkippedSynRecv TCPACKSkippedPAWS TCPACKSkippedSeq TCPACKSkippedFinWait2 TCPACKSkippedTimeWait TCPACKSkippedChallenge TCPWinProbe TCPKeepAlive TCPMTUPFail TCPMTUPSuccess"))) - d.AddChild(ctx, "packet", p.newStubProcFSFile(ctx, msrc, []byte("sk RefCnt Type Proto Iface R Rmem User Inode"))) - d.AddChild(ctx, "protocols", p.newStubProcFSFile(ctx, msrc, []byte("protocol size sockets memory press maxhdr slab module cl co di ac io in de sh ss gs se re sp bi br ha uh gp em"))) - - // Linux sets these values to: nsec per usec, psched tick in ns, 1000000, - // high res timer ticks per sec (ClockGetres returns 1ns resolution). - psched := fmt.Sprintf("%08x %08x %08x %08x\n", uint64(time.Microsecond/time.Nanosecond), 64, 1000000, uint64(time.Second/time.Nanosecond)) - d.AddChild(ctx, "psched", p.newStubProcFSFile(ctx, msrc, []byte(psched))) - - d.AddChild(ctx, "ptype", p.newStubProcFSFile(ctx, msrc, []byte("Type Device Function"))) - d.AddChild(ctx, "route", p.newStubProcFSFile(ctx, msrc, []byte("Iface Destination Gateway Flags RefCnt Use Metric Mask MTU Window IRTT"))) - d.AddChild(ctx, "tcp", p.newStubProcFSFile(ctx, msrc, []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode"))) - d.AddChild(ctx, "tcp6", p.newStubProcFSFile(ctx, msrc, []byte(" sl local_address remote_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode"))) - d.AddChild(ctx, "udp", p.newStubProcFSFile(ctx, msrc, []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode ref pointer drops"))) - d.AddChild(ctx, "udp6", p.newStubProcFSFile(ctx, msrc, []byte(" sl local_address remote_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode"))) + contents = map[string]*fs.Inode{ + "dev": seqfile.NewSeqFileInode(ctx, &netDev{s: s}, msrc), + "if_inet6": seqfile.NewSeqFileInode(ctx, &ifinet6{s: s}, msrc), + + // The following files are simple stubs until they are + // implemented in netstack, if the file contains a + // header the stub is just the header otherwise it is + // an empty file. + "arp": newStaticProcInode(ctx, msrc, []byte("IP address HW type Flags HW address Mask Device")), + "ipv6_route": newStaticProcInode(ctx, msrc, []byte("")), + "netlink": newStaticProcInode(ctx, msrc, []byte("sk Eth Pid Groups Rmem Wmem Dump Locks Drops Inode")), + "netstat": newStaticProcInode(ctx, msrc, []byte("TcpExt: SyncookiesSent SyncookiesRecv SyncookiesFailed EmbryonicRsts PruneCalled RcvPruned OfoPruned OutOfWindowIcmps LockDroppedIcmps ArpFilter TW TWRecycled TWKilled PAWSPassive PAWSActive PAWSEstab DelayedACKs DelayedACKLocked DelayedACKLost ListenOverflows ListenDrops TCPPrequeued TCPDirectCopyFromBacklog TCPDirectCopyFromPrequeue TCPPrequeueDropped TCPHPHits TCPHPHitsToUser TCPPureAcks TCPHPAcks TCPRenoRecovery TCPSackRecovery TCPSACKReneging TCPFACKReorder TCPSACKReorder TCPRenoReorder TCPTSReorder TCPFullUndo TCPPartialUndo TCPDSACKUndo TCPLossUndo TCPLostRetransmit TCPRenoFailures TCPSackFailures TCPLossFailures TCPFastRetrans TCPForwardRetrans TCPSlowStartRetrans TCPTimeouts TCPLossProbes TCPLossProbeRecovery TCPRenoRecoveryFail TCPSackRecoveryFail TCPSchedulerFailed TCPRcvCollapsed TCPDSACKOldSent TCPDSACKOfoSent TCPDSACKRecv TCPDSACKOfoRecv TCPAbortOnData TCPAbortOnClose TCPAbortOnMemory TCPAbortOnTimeout TCPAbortOnLinger TCPAbortFailed TCPMemoryPressures TCPSACKDiscard TCPDSACKIgnoredOld TCPDSACKIgnoredNoUndo TCPSpuriousRTOs TCPMD5NotFound TCPMD5Unexpected TCPMD5Failure TCPSackShifted TCPSackMerged TCPSackShiftFallback TCPBacklogDrop TCPMinTTLDrop TCPDeferAcceptDrop IPReversePathFilter TCPTimeWaitOverflow TCPReqQFullDoCookies TCPReqQFullDrop TCPRetransFail TCPRcvCoalesce TCPOFOQueue TCPOFODrop TCPOFOMerge TCPChallengeACK TCPSYNChallenge TCPFastOpenActive TCPFastOpenActiveFail TCPFastOpenPassive TCPFastOpenPassiveFail TCPFastOpenListenOverflow TCPFastOpenCookieReqd TCPSpuriousRtxHostQueues BusyPollRxPackets TCPAutoCorking TCPFromZeroWindowAdv TCPToZeroWindowAdv TCPWantZeroWindowAdv TCPSynRetrans TCPOrigDataSent TCPHystartTrainDetect TCPHystartTrainCwnd TCPHystartDelayDetect TCPHystartDelayCwnd TCPACKSkippedSynRecv TCPACKSkippedPAWS TCPACKSkippedSeq TCPACKSkippedFinWait2 TCPACKSkippedTimeWait TCPACKSkippedChallenge TCPWinProbe TCPKeepAlive TCPMTUPFail TCPMTUPSuccess")), + "packet": newStaticProcInode(ctx, msrc, []byte("sk RefCnt Type Proto Iface R Rmem User Inode")), + "protocols": newStaticProcInode(ctx, msrc, []byte("protocol size sockets memory press maxhdr slab module cl co di ac io in de sh ss gs se re sp bi br ha uh gp em")), + // Linux sets psched values to: nsec per usec, psched + // tick in ns, 1000000, high res timer ticks per sec + // (ClockGetres returns 1ns resolution). + "psched": newStaticProcInode(ctx, msrc, []byte(fmt.Sprintf("%08x %08x %08x %08x\n", uint64(time.Microsecond/time.Nanosecond), 64, 1000000, uint64(time.Second/time.Nanosecond)))), + "ptype": newStaticProcInode(ctx, msrc, []byte("Type Device Function")), + "route": newStaticProcInode(ctx, msrc, []byte("Iface Destination Gateway Flags RefCnt Use Metric Mask MTU Window IRTT")), + "tcp": newStaticProcInode(ctx, msrc, []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode")), + "tcp6": newStaticProcInode(ctx, msrc, []byte(" sl local_address remote_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode")), + "udp": newStaticProcInode(ctx, msrc, []byte(" sl local_address rem_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode ref pointer drops")), + "udp6": newStaticProcInode(ctx, msrc, []byte(" sl local_address remote_address st tx_queue rx_queue tr tm->when retrnsmt uid timeout inode")), + } } - return newFile(d, msrc, fs.SpecialDirectory, nil) + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } // ifinet6 implements seqfile.SeqSource for /proc/net/if_inet6. diff --git a/pkg/sentry/fs/proc/proc.go b/pkg/sentry/fs/proc/proc.go index 70e549c31..d1c699418 100644 --- a/pkg/sentry/fs/proc/proc.go +++ b/pkg/sentry/fs/proc/proc.go @@ -17,18 +17,17 @@ package proc import ( "fmt" - "io" "sort" "strconv" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/device" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/seqfile" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" ) @@ -46,32 +45,6 @@ type proc struct { pidns *kernel.PIDNamespace } -// stubProcFSFile is a file type that can be used to return file contents -// which are constant. This file is not writable and will always have mode -// 0444. -// -// +stateify savable -type stubProcFSFile struct { - ramfs.Entry - - // contents are the immutable file contents that will always be returned. - contents []byte -} - -// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. -func (s *stubProcFSFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - if offset < 0 { - return 0, syserror.EINVAL - } - - if offset >= int64(len(s.contents)) { - return 0, io.EOF - } - - n, err := dst.CopyOut(ctx, s.contents[offset:]) - return int64(n), err -} - // New returns the root node of a partial simple procfs. func New(ctx context.Context, msrc *fs.MountSource) (*fs.Inode, error) { k := kernel.KernelFromContext(ctx) @@ -83,29 +56,39 @@ func New(ctx context.Context, msrc *fs.MountSource) (*fs.Inode, error) { return nil, fmt.Errorf("procfs requires a PID namespace") } - p := &proc{k: k, pidns: pidns} - p.InitDir(ctx, map[string]*fs.Inode{ + // Note that these are just the static members. There are dynamic + // members populated in Readdir and Lookup below. + contents := map[string]*fs.Inode{ + "cpuinfo": newCPUInfo(ctx, msrc), "filesystems": seqfile.NewSeqFileInode(ctx, &filesystemsData{}, msrc), "loadavg": seqfile.NewSeqFileInode(ctx, &loadavgData{}, msrc), "meminfo": seqfile.NewSeqFileInode(ctx, &meminfoData{k}, msrc), - "mounts": newMountsSymlink(ctx, msrc), + "mounts": newProcInode(ramfs.NewSymlink(ctx, fs.RootOwner, "self/mounts"), msrc, fs.Symlink, nil), + "self": newSelf(ctx, pidns, msrc), "stat": seqfile.NewSeqFileInode(ctx, &statData{k}, msrc), + "thread-self": newThreadSelf(ctx, pidns, msrc), + "uptime": newUptime(ctx, msrc), "version": seqfile.NewSeqFileInode(ctx, &versionData{k}, msrc), - }, fs.RootOwner, fs.FilePermsFromMode(0555)) + } + + // Construct the proc InodeOperations. + p := &proc{ + Dir: *ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)), + k: k, + pidns: pidns, + } + + // Add more contents that need proc to be initialized. + p.AddChild(ctx, "sys", p.newSysDir(ctx, msrc)) - p.AddChild(ctx, "cpuinfo", p.newCPUInfo(ctx, msrc)) // If we're using rpcinet we will let it manage /proc/net. if _, ok := p.k.NetworkStack().(*rpcinet.Stack); ok { - p.AddChild(ctx, "net", newRPCInetProcNet(ctx, msrc)) + contents["net"] = newRPCInetProcNet(ctx, msrc) } else { - p.AddChild(ctx, "net", p.newNetDir(ctx, msrc)) + contents["net"] = p.newNetDir(ctx, msrc) } - p.AddChild(ctx, "self", p.newSelf(ctx, msrc)) - p.AddChild(ctx, "sys", p.newSysDir(ctx, msrc)) - p.AddChild(ctx, "thread-self", p.newThreadSelf(ctx, msrc)) - p.AddChild(ctx, "uptime", p.newUptime(ctx, msrc)) - return newFile(p, msrc, fs.SpecialDirectory, nil), nil + return newProcInode(p, msrc, fs.SpecialDirectory, nil), nil } // self is a magical link. @@ -118,26 +101,21 @@ type self struct { } // newSelf returns a new "self" node. -func (p *proc) newSelf(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - s := &self{pidns: p.pidns} - s.InitSymlink(ctx, fs.RootOwner, "") - return newFile(s, msrc, fs.Symlink, nil) +func newSelf(ctx context.Context, pidns *kernel.PIDNamespace, msrc *fs.MountSource) *fs.Inode { + s := &self{ + Symlink: *ramfs.NewSymlink(ctx, fs.RootOwner, ""), + pidns: pidns, + } + return newProcInode(s, msrc, fs.Symlink, nil) } // newThreadSelf returns a new "threadSelf" node. -func (p *proc) newThreadSelf(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - s := &threadSelf{pidns: p.pidns} - s.InitSymlink(ctx, fs.RootOwner, "") - return newFile(s, msrc, fs.Symlink, nil) -} - -// newStubProcFsFile returns a procfs file with constant contents. -func (p *proc) newStubProcFSFile(ctx context.Context, msrc *fs.MountSource, c []byte) *fs.Inode { - u := &stubProcFSFile{ - contents: c, +func newThreadSelf(ctx context.Context, pidns *kernel.PIDNamespace, msrc *fs.MountSource) *fs.Inode { + s := &threadSelf{ + Symlink: *ramfs.NewSymlink(ctx, fs.RootOwner, ""), + pidns: pidns, } - u.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0444)) - return newFile(u, msrc, fs.SpecialFile, nil) + return newProcInode(s, msrc, fs.Symlink, nil) } // Readlink implements fs.InodeOperations.Readlink. @@ -145,13 +123,13 @@ func (s *self) Readlink(ctx context.Context, inode *fs.Inode) (string, error) { if t := kernel.TaskFromContext(ctx); t != nil { tgid := s.pidns.IDOfThreadGroup(t.ThreadGroup()) if tgid == 0 { - return "", ramfs.ErrNotFound + return "", syserror.ENOENT } return strconv.FormatUint(uint64(tgid), 10), nil } // Who is reading this link? - return "", ramfs.ErrInvalidOp + return "", syserror.EINVAL } // threadSelf is more magical than "self" link. @@ -169,13 +147,13 @@ func (s *threadSelf) Readlink(ctx context.Context, inode *fs.Inode) (string, err tgid := s.pidns.IDOfThreadGroup(t.ThreadGroup()) tid := s.pidns.IDOfTask(t) if tid == 0 || tgid == 0 { - return "", ramfs.ErrNotFound + return "", syserror.ENOENT } return fmt.Sprintf("%d/task/%d", tgid, tid), nil } // Who is reading this link? - return "", ramfs.ErrInvalidOp + return "", syserror.EINVAL } // Lookup loads an Inode at name into a Dirent. @@ -204,25 +182,44 @@ func (p *proc) Lookup(ctx context.Context, dir *fs.Inode, name string) (*fs.Dire return fs.NewDirent(td, name), nil } -// Readdir synthesizes proc contents. -func (p *proc) DeprecatedReaddir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { - // Serialize normal contents. - _, err := p.Dir.DeprecatedReaddir(ctx, dirCtx, offset) - if err != nil { - return offset, err +// GetFile implements fs.InodeOperations. +func (p *proc) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &rootProcFile{iops: p}), nil +} + +// rootProcFile implements fs.FileOperations for the proc directory. +// +// +stateify savable +type rootProcFile struct { + fsutil.DirFileOperations `state:"nosave"` + + iops *proc +} + +var _ fs.FileOperations = (*rootProcFile)(nil) + +// Readdir implements fs.FileOperations.Readdir. +func (rpf *rootProcFile) Readdir(ctx context.Context, file *fs.File, ser fs.DentrySerializer) (int64, error) { + offset := file.Offset() + dirCtx := &fs.DirCtx{ + Serializer: ser, } - m := make(map[string]fs.DentAttr) - var names []string + // Get normal directory contents from ramfs dir. + names, m := rpf.iops.Dir.Children() - // Add special files. - m["sys"] = fs.GenericDentAttr(fs.SpecialFile, device.ProcDevice) - names = append(names, "sys") + // Add dot and dotdot. + root := fs.RootFromContext(ctx) + defer root.DecRef() + dot, dotdot := file.Dirent.GetDotAttrs(root) + names = append(names, ".", "..") + m["."] = dot + m[".."] = dotdot // Collect tasks. // Per linux we only include it in directory listings if it's the leader. // But for whatever crazy reason, you can still walk to the given node. - for _, tg := range p.pidns.ThreadGroups() { + for _, tg := range rpf.iops.pidns.ThreadGroups() { if leader := tg.Leader(); leader != nil { name := strconv.FormatUint(uint64(tg.ID()), 10) m[name] = fs.GenericDentAttr(fs.SpecialDirectory, device.ProcDevice) @@ -230,7 +227,7 @@ func (p *proc) DeprecatedReaddir(ctx context.Context, dirCtx *fs.DirCtx, offset } } - if offset >= len(m) { + if offset >= int64(len(m)) { return offset, nil } sort.Strings(names) @@ -241,12 +238,5 @@ func (p *proc) DeprecatedReaddir(ctx context.Context, dirCtx *fs.DirCtx, offset } offset++ } - return offset, err -} - -// newMountsSymlink returns a symlink to "self/mounts". -func newMountsSymlink(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - s := &ramfs.Symlink{} - s.InitSymlink(ctx, fs.RootOwner, "self/mounts") - return newFile(s, msrc, fs.Symlink, nil) + return offset, nil } diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go index d025069df..65faa21f2 100644 --- a/pkg/sentry/fs/proc/rpcinet_proc.go +++ b/pkg/sentry/fs/proc/rpcinet_proc.go @@ -20,32 +20,72 @@ import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) -// rpcinetFile implments fs.InodeOperations as RPCs. -type rpcinetFile struct { - ramfs.Entry +// rpcInetInode implments fs.InodeOperations. +type rpcInetInode struct { + fsutil.SimpleFileInode - // filepath is the full path of this rpcinetFile. + // filepath is the full path of this rpcInetInode. filepath string k *kernel.Kernel } -// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. -// This method can panic if an rpcinetFile was created without an rpcinet +func newRPCInetInode(ctx context.Context, msrc *fs.MountSource, filepath string, mode linux.FileMode) *fs.Inode { + f := &rpcInetInode{ + SimpleFileInode: *fsutil.NewSimpleFileInode(ctx, fs.RootOwner, fs.FilePermsFromMode(mode), linux.PROC_SUPER_MAGIC), + filepath: filepath, + k: kernel.KernelFromContext(ctx), + } + return newProcInode(f, msrc, fs.SpecialFile, nil) +} + +// GetFile implements fs.InodeOperations.GetFile. +func (i *rpcInetInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + flags.Pread = true + flags.Pwrite = true + fops := &rpcInetFile{ + inode: i, + } + return fs.NewFile(ctx, dirent, flags, fops), nil +} + +// rpcInetFile implements fs.FileOperations as RPCs. +type rpcInetFile struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + + inode *rpcInetInode +} + +// Read implements fs.FileOperations.Read. +// +// This method can panic if an rpcInetInode was created without an rpcinet // stack. -func (r rpcinetFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - s, ok := r.k.NetworkStack().(*rpcinet.Stack) +func (f *rpcInetFile) Read(ctx context.Context, file *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { + if offset < 0 { + return 0, syserror.EINVAL + } + s, ok := f.inode.k.NetworkStack().(*rpcinet.Stack) if !ok { panic("Network stack is not a rpcinet.") } - contents, se := s.RPCReadFile(r.filepath) + contents, se := s.RPCReadFile(f.inode.filepath) if se != nil || offset >= int64(len(contents)) { return 0, io.EOF } @@ -54,16 +94,12 @@ func (r rpcinetFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequenc return int64(n), err } -// Truncate implements fs.InodeOperations.Truncate. -func (r rpcinetFile) Truncate(context.Context, *fs.Inode, int64) error { - return nil -} - -// DeprecatedPwritev implements fs.InodeOperations.DeprecatedPwritev. -// This method can panic if an rpcinetFile was created without an rpcinet +// Write implements fs.FileOperations.Write. +// +// This method can panic if an rpcInetInode was created without an rpcInet // stack. -func (r rpcinetFile) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { - s, ok := r.k.NetworkStack().(*rpcinet.Stack) +func (f *rpcInetFile) Write(ctx context.Context, file *fs.File, src usermem.IOSequence, offset int64) (int64, error) { + s, ok := f.inode.k.NetworkStack().(*rpcinet.Stack) if !ok { panic("Network stack is not a rpcinet.") } @@ -78,116 +114,102 @@ func (r rpcinetFile) DeprecatedPwritev(ctx context.Context, src usermem.IOSequen return int64(n), err } - written, se := s.RPCWriteFile(r.filepath, b) + written, se := s.RPCWriteFile(f.inode.filepath, b) return int64(written), se.ToError() } -func newRPCProcFSFile(ctx context.Context, msrc *fs.MountSource, filepath string, mode linux.FileMode) *fs.Inode { - f := &rpcinetFile{ - filepath: filepath, - k: kernel.KernelFromContext(ctx), - } - f.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(mode)) - - fi := newFile(f, msrc, fs.SpecialFile, nil) - return fi -} - // newRPCInetProcNet will build an inode for /proc/net. func newRPCInetProcNet(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - - // Add all the files we want to forward for /proc/net. - d.AddChild(ctx, "arp", newRPCProcFSFile(ctx, msrc, "/proc/net/arp", 0444)) - d.AddChild(ctx, "dev", newRPCProcFSFile(ctx, msrc, "/proc/net/dev", 0444)) - d.AddChild(ctx, "if_inet6", newRPCProcFSFile(ctx, msrc, "/proc/net/if_inet6", 0444)) - d.AddChild(ctx, "ipv6_route", newRPCProcFSFile(ctx, msrc, "/proc/net/ipv6_route", 0444)) - d.AddChild(ctx, "netlink", newRPCProcFSFile(ctx, msrc, "/proc/net/netlink", 0444)) - d.AddChild(ctx, "netstat", newRPCProcFSFile(ctx, msrc, "/proc/net/netstat", 0444)) - d.AddChild(ctx, "packet", newRPCProcFSFile(ctx, msrc, "/proc/net/packet", 0444)) - d.AddChild(ctx, "protocols", newRPCProcFSFile(ctx, msrc, "/proc/net/protocols", 0444)) - d.AddChild(ctx, "psched", newRPCProcFSFile(ctx, msrc, "/proc/net/psched", 0444)) - d.AddChild(ctx, "ptype", newRPCProcFSFile(ctx, msrc, "/proc/net/ptype", 0444)) - d.AddChild(ctx, "route", newRPCProcFSFile(ctx, msrc, "/proc/net/route", 0444)) - d.AddChild(ctx, "tcp", newRPCProcFSFile(ctx, msrc, "/proc/net/tcp", 0444)) - d.AddChild(ctx, "tcp6", newRPCProcFSFile(ctx, msrc, "/proc/net/tcp6", 0444)) - d.AddChild(ctx, "udp", newRPCProcFSFile(ctx, msrc, "/proc/net/udp", 0444)) - d.AddChild(ctx, "udp6", newRPCProcFSFile(ctx, msrc, "/proc/net/udp6", 0444)) - - return newFile(d, msrc, fs.SpecialDirectory, nil) + contents := map[string]*fs.Inode{ + "arp": newRPCInetInode(ctx, msrc, "/proc/net/arp", 0444), + "dev": newRPCInetInode(ctx, msrc, "/proc/net/dev", 0444), + "if_inet6": newRPCInetInode(ctx, msrc, "/proc/net/if_inet6", 0444), + "ipv6_route": newRPCInetInode(ctx, msrc, "/proc/net/ipv6_route", 0444), + "netlink": newRPCInetInode(ctx, msrc, "/proc/net/netlink", 0444), + "netstat": newRPCInetInode(ctx, msrc, "/proc/net/netstat", 0444), + "packet": newRPCInetInode(ctx, msrc, "/proc/net/packet", 0444), + "protocols": newRPCInetInode(ctx, msrc, "/proc/net/protocols", 0444), + "psched": newRPCInetInode(ctx, msrc, "/proc/net/psched", 0444), + "ptype": newRPCInetInode(ctx, msrc, "/proc/net/ptype", 0444), + "route": newRPCInetInode(ctx, msrc, "/proc/net/route", 0444), + "tcp": newRPCInetInode(ctx, msrc, "/proc/net/tcp", 0444), + "tcp6": newRPCInetInode(ctx, msrc, "/proc/net/tcp6", 0444), + "udp": newRPCInetInode(ctx, msrc, "/proc/net/udp", 0444), + "udp6": newRPCInetInode(ctx, msrc, "/proc/net/udp6", 0444), + } + + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } // newRPCInetProcSysNet will build an inode for /proc/sys/net. func newRPCInetProcSysNet(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - d.AddChild(ctx, "ipv4", newRPCInetSysNetIPv4Dir(ctx, msrc)) - d.AddChild(ctx, "core", newRPCInetSysNetCore(ctx, msrc)) + contents := map[string]*fs.Inode{ + "ipv4": newRPCInetSysNetIPv4Dir(ctx, msrc), + "core": newRPCInetSysNetCore(ctx, msrc), + } - return newFile(d, msrc, fs.SpecialDirectory, nil) + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } // newRPCInetSysNetCore builds the /proc/sys/net/core directory. func newRPCInetSysNetCore(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - - // Add all the files we want to forward over RPC for /proc/sys/net/core - d.AddChild(ctx, "default_qdisc", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/default_qdisc", 0444)) - d.AddChild(ctx, "message_burst", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/message_burst", 0444)) - d.AddChild(ctx, "message_cost", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/message_cost", 0444)) - d.AddChild(ctx, "optmem_max", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/optmem_max", 0444)) - d.AddChild(ctx, "rmem_default", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/rmem_default", 0444)) - d.AddChild(ctx, "rmem_max", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/rmem_max", 0444)) - d.AddChild(ctx, "somaxconn", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/somaxconn", 0444)) - d.AddChild(ctx, "wmem_default", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/wmem_default", 0444)) - d.AddChild(ctx, "wmem_max", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/core/wmem_max", 0444)) - - return newFile(d, msrc, fs.SpecialDirectory, nil) + contents := map[string]*fs.Inode{ + "default_qdisc": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/default_qdisc", 0444), + "message_burst": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/message_burst", 0444), + "message_cost": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/message_cost", 0444), + "optmem_max": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/optmem_max", 0444), + "rmem_default": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/rmem_default", 0444), + "rmem_max": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/rmem_max", 0444), + "somaxconn": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/somaxconn", 0444), + "wmem_default": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/wmem_default", 0444), + "wmem_max": newRPCInetInode(ctx, msrc, "/proc/sys/net/core/wmem_max", 0444), + } + + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } // newRPCInetSysNetIPv4Dir builds the /proc/sys/net/ipv4 directory. func newRPCInetSysNetIPv4Dir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - - // Add all the files we want to forward over RPC for /proc/sys/net/ipv4. - d.AddChild(ctx, "ip_local_port_range", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ip_local_port_range", 0444)) - d.AddChild(ctx, "ip_local_reserved_ports", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ip_local_reserved_ports", 0444)) - d.AddChild(ctx, "ipfrag_time", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ipfrag_time", 0444)) - d.AddChild(ctx, "ip_nonlocal_bind", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ip_nonlocal_bind", 0444)) - d.AddChild(ctx, "ip_no_pmtu_disc", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/ip_no_pmtu_disc", 0444)) - - d.AddChild(ctx, "tcp_allowed_congestion_control", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_allowed_congestion_control", 0444)) - d.AddChild(ctx, "tcp_available_congestion_control", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_available_congestion_control", 0444)) - d.AddChild(ctx, "tcp_base_mss", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_base_mss", 0444)) - d.AddChild(ctx, "tcp_congestion_control", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_congestion_control", 0644)) - d.AddChild(ctx, "tcp_dsack", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_dsack", 0644)) - d.AddChild(ctx, "tcp_early_retrans", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_early_retrans", 0644)) - d.AddChild(ctx, "tcp_fack", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_fack", 0644)) - d.AddChild(ctx, "tcp_fastopen", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_fastopen", 0644)) - d.AddChild(ctx, "tcp_fastopen_key", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_fastopen_key", 0444)) - d.AddChild(ctx, "tcp_fin_timeout", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_fin_timeout", 0644)) - d.AddChild(ctx, "tcp_invalid_ratelimit", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_invalid_ratelimit", 0444)) - d.AddChild(ctx, "tcp_keepalive_intvl", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_intvl", 0644)) - d.AddChild(ctx, "tcp_keepalive_probes", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_probes", 0644)) - d.AddChild(ctx, "tcp_keepalive_time", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_time", 0644)) - d.AddChild(ctx, "tcp_mem", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_mem", 0444)) - d.AddChild(ctx, "tcp_mtu_probing", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_mtu_probing", 0644)) - d.AddChild(ctx, "tcp_no_metrics_save", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_no_metrics_save", 0444)) - d.AddChild(ctx, "tcp_probe_interval", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_probe_interval", 0444)) - d.AddChild(ctx, "tcp_probe_threshold", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_probe_threshold", 0444)) - d.AddChild(ctx, "tcp_retries1", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_retries1", 0644)) - d.AddChild(ctx, "tcp_retries2", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_retries2", 0644)) - d.AddChild(ctx, "tcp_rfc1337", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_rfc1337", 0444)) - d.AddChild(ctx, "tcp_rmem", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_rmem", 0444)) - d.AddChild(ctx, "tcp_sack", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_sack", 0644)) - d.AddChild(ctx, "tcp_slow_start_after_idle", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_slow_start_after_idle", 0644)) - d.AddChild(ctx, "tcp_synack_retries", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_synack_retries", 0644)) - d.AddChild(ctx, "tcp_syn_retries", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_syn_retries", 0644)) - d.AddChild(ctx, "tcp_timestamps", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_timestamps", 0644)) - d.AddChild(ctx, "tcp_wmem", newRPCProcFSFile(ctx, msrc, "/proc/sys/net/ipv4/tcp_wmem", 0444)) - - return newFile(d, msrc, fs.SpecialDirectory, nil) + contents := map[string]*fs.Inode{ + "ip_local_port_range": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/ip_local_port_range", 0444), + "ip_local_reserved_ports": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/ip_local_reserved_ports", 0444), + "ipfrag_time": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/ipfrag_time", 0444), + "ip_nonlocal_bind": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/ip_nonlocal_bind", 0444), + "ip_no_pmtu_disc": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/ip_no_pmtu_disc", 0444), + "tcp_allowed_congestion_control": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_allowed_congestion_control", 0444), + "tcp_available_congestion_control": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_available_congestion_control", 0444), + "tcp_base_mss": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_base_mss", 0444), + "tcp_congestion_control": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_congestion_control", 0644), + "tcp_dsack": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_dsack", 0644), + "tcp_early_retrans": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_early_retrans", 0644), + "tcp_fack": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_fack", 0644), + "tcp_fastopen": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_fastopen", 0644), + "tcp_fastopen_key": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_fastopen_key", 0444), + "tcp_fin_timeout": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_fin_timeout", 0644), + "tcp_invalid_ratelimit": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_invalid_ratelimit", 0444), + "tcp_keepalive_intvl": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_intvl", 0644), + "tcp_keepalive_probes": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_probes", 0644), + "tcp_keepalive_time": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_keepalive_time", 0644), + "tcp_mem": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_mem", 0444), + "tcp_mtu_probing": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_mtu_probing", 0644), + "tcp_no_metrics_save": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_no_metrics_save", 0444), + "tcp_probe_interval": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_probe_interval", 0444), + "tcp_probe_threshold": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_probe_threshold", 0444), + "tcp_retries1": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_retries1", 0644), + "tcp_retries2": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_retries2", 0644), + "tcp_rfc1337": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_rfc1337", 0444), + "tcp_rmem": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_rmem", 0444), + "tcp_sack": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_sack", 0644), + "tcp_slow_start_after_idle": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_slow_start_after_idle", 0644), + "tcp_synack_retries": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_synack_retries", 0644), + "tcp_syn_retries": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_syn_retries", 0644), + "tcp_timestamps": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_timestamps", 0644), + "tcp_wmem": newRPCInetInode(ctx, msrc, "/proc/sys/net/ipv4/tcp_wmem", 0444), + } + + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } diff --git a/pkg/sentry/fs/proc/seqfile/BUILD b/pkg/sentry/fs/proc/seqfile/BUILD index 53c475652..b4ba64e10 100644 --- a/pkg/sentry/fs/proc/seqfile/BUILD +++ b/pkg/sentry/fs/proc/seqfile/BUILD @@ -8,12 +8,15 @@ go_library( importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/seqfile", visibility = ["//pkg/sentry:internal"], deps = [ + "//pkg/abi/linux", "//pkg/sentry/context", "//pkg/sentry/fs", + "//pkg/sentry/fs/fsutil", "//pkg/sentry/fs/proc/device", - "//pkg/sentry/fs/ramfs", "//pkg/sentry/kernel/time", "//pkg/sentry/usermem", + "//pkg/syserror", + "//pkg/waiter", ], ) @@ -26,7 +29,7 @@ go_test( "//pkg/sentry/context", "//pkg/sentry/context/contexttest", "//pkg/sentry/fs", - "//pkg/sentry/fs/ramfs/test", + "//pkg/sentry/fs/ramfs", "//pkg/sentry/usermem", ], ) diff --git a/pkg/sentry/fs/proc/seqfile/seqfile.go b/pkg/sentry/fs/proc/seqfile/seqfile.go index 0499ba65b..16fc6789e 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile.go @@ -18,12 +18,15 @@ import ( "io" "sync" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/device" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // SeqHandle is a helper handle to seek in the file. @@ -87,7 +90,18 @@ func (s *SeqGenerationCounter) IsCurrent(generation int64) bool { // // +stateify savable type SeqFile struct { - ramfs.Entry + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeNotTruncatable `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + fsutil.InodeSimpleExtendedAttributes + fsutil.InodeSimpleAttributes // mu protects the fields below. mu sync.Mutex `state:"nosave"` @@ -99,11 +113,14 @@ type SeqFile struct { lastRead int64 } +var _ fs.InodeOperations = (*SeqFile)(nil) + // NewSeqFile returns a seqfile suitable for use by external consumers. func NewSeqFile(ctx context.Context, source SeqSource) *SeqFile { - s := &SeqFile{SeqSource: source} - s.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0444)) - return s + return &SeqFile{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + SeqSource: source, + } } // NewSeqFileInode returns an Inode with SeqFile InodeOperations. @@ -120,11 +137,19 @@ func NewSeqFileInode(ctx context.Context, source SeqSource, msrc *fs.MountSource // UnstableAttr returns unstable attributes of the SeqFile. func (s *SeqFile) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - uattr, _ := s.Entry.UnstableAttr(ctx, inode) + uattr, err := s.InodeSimpleAttributes.UnstableAttr(ctx, inode) + if err != nil { + return fs.UnstableAttr{}, err + } uattr.ModificationTime = ktime.NowFromContext(ctx) return uattr, nil } +// GetFile implements fs.InodeOperations.GetFile. +func (s *SeqFile) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &seqFileOperations{seqFile: s}), nil +} + // findIndexAndOffset finds the unit that corresponds to a certain offset. // Returns the unit and the offset within the unit. If there are not enough // units len(data) and leftover offset is returned. @@ -139,36 +164,74 @@ func findIndexAndOffset(data []SeqData, offset int64) (int, int64) { return len(data), offset } -// DeprecatedPreadv reads from the file at the given offset. -func (s *SeqFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - s.mu.Lock() - defer s.mu.Unlock() +// updateSourceLocked requires that s.mu is held. +func (s *SeqFile) updateSourceLocked(ctx context.Context, record int) { + var h SeqHandle + if record == 0 { + h = nil + } else { + h = s.source[record-1].Handle + } + // Save what we have previously read. + s.source = s.source[:record] + var newSource []SeqData + newSource, s.generation = s.SeqSource.ReadSeqFileData(ctx, h) + s.source = append(s.source, newSource...) +} + +// seqFileOperations implements fs.FileOperations. +// +// +stateify savable +type seqFileOperations struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + + seqFile *SeqFile +} + +var _ fs.FileOperations = (*seqFileOperations)(nil) + +// Write implements fs.FileOperations.Write. +func (*seqFileOperations) Write(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { + return 0, syserror.EACCES +} + +// Read implements fs.FileOperations.Read. +func (sfo *seqFileOperations) Read(ctx context.Context, file *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { + sfo.seqFile.mu.Lock() + defer sfo.seqFile.mu.Unlock() - s.Entry.NotifyAccess(ctx) - defer func() { s.lastRead = offset }() + sfo.seqFile.NotifyAccess(ctx) + defer func() { sfo.seqFile.lastRead = offset }() updated := false // Try to find where we should start reading this file. - i, recordOffset := findIndexAndOffset(s.source, offset) - if i == len(s.source) { + i, recordOffset := findIndexAndOffset(sfo.seqFile.source, offset) + if i == len(sfo.seqFile.source) { // Ok, we're at EOF. Let's first check to see if there might be // more data available to us. If there is more data, add it to // the end and try reading again. - if !s.SeqSource.NeedsUpdate(s.generation) { + if !sfo.seqFile.SeqSource.NeedsUpdate(sfo.seqFile.generation) { return 0, io.EOF } - oldLen := len(s.source) - s.updateSourceLocked(ctx, len(s.source)) + oldLen := len(sfo.seqFile.source) + sfo.seqFile.updateSourceLocked(ctx, len(sfo.seqFile.source)) updated = true // We know that we had consumed everything up until this point // so we search in the new slice instead of starting over. - i, recordOffset = findIndexAndOffset(s.source[oldLen:], recordOffset) + i, recordOffset = findIndexAndOffset(sfo.seqFile.source[oldLen:], recordOffset) i += oldLen // i is at most the length of the slice which is - // len(s.source) - oldLen. So at most i will be equal to - // len(s.source). - if i == len(s.source) { + // len(sfo.seqFile.source) - oldLen. So at most i will be equal to + // len(sfo.seqFile.source). + if i == len(sfo.seqFile.source) { return 0, io.EOF } } @@ -178,7 +241,7 @@ func (s *SeqFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, // before continuing on to the next. We don't refresh our data source // before this record is completed. if recordOffset != 0 { - n, err := dst.CopyOut(ctx, s.source[i].Buf[recordOffset:]) + n, err := dst.CopyOut(ctx, sfo.seqFile.source[i].Buf[recordOffset:]) done += int64(n) dst = dst.DropFirst(n) if dst.NumBytes() == 0 || err != nil { @@ -190,15 +253,15 @@ func (s *SeqFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, // Next/New unit, update the source file if necessary. Make an extra // check to see if we've seeked backwards and if so always update our // data source. - if !updated && (s.SeqSource.NeedsUpdate(s.generation) || s.lastRead > offset) { - s.updateSourceLocked(ctx, i) + if !updated && (sfo.seqFile.SeqSource.NeedsUpdate(sfo.seqFile.generation) || sfo.seqFile.lastRead > offset) { + sfo.seqFile.updateSourceLocked(ctx, i) // recordOffset is 0 here and we won't update records behind the // current one so recordOffset is still 0 even though source // just got updated. Just read the next record. } // Finish by reading all the available data. - for _, buf := range s.source[i:] { + for _, buf := range sfo.seqFile.source[i:] { n, err := dst.CopyOut(ctx, buf.Buf) done += int64(n) dst = dst.DropFirst(n) @@ -214,23 +277,3 @@ func (s *SeqFile) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, } return done, nil } - -// updateSourceLocked requires that s.mu is held. -func (s *SeqFile) updateSourceLocked(ctx context.Context, record int) { - var h SeqHandle - if record == 0 { - h = nil - } else { - h = s.source[record-1].Handle - } - // Save what we have previously read. - s.source = s.source[:record] - var newSource []SeqData - newSource, s.generation = s.SeqSource.ReadSeqFileData(ctx, h) - s.source = append(s.source, newSource...) -} - -// DeprecatedPwritev is always denied. -func (*SeqFile) DeprecatedPwritev(context.Context, usermem.IOSequence, int64) (int64, error) { - return 0, ramfs.ErrDenied -} diff --git a/pkg/sentry/fs/proc/seqfile/seqfile_test.go b/pkg/sentry/fs/proc/seqfile/seqfile_test.go index f9a2ca38e..35403ab7f 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile_test.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile_test.go @@ -23,7 +23,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/context/contexttest" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - ramfstest "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs/test" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" ) @@ -91,10 +91,15 @@ type testTable struct { expectedError error } -func runTableTests(ctx context.Context, table []testTable, n fs.InodeOperations) error { +func runTableTests(ctx context.Context, table []testTable, dirent *fs.Dirent) error { for _, tt := range table { + file, err := dirent.Inode.InodeOperations.GetFile(ctx, dirent, fs.FileFlags{Read: true}) + if err != nil { + return fmt.Errorf("GetFile returned error: %v", err) + } + data := make([]byte, tt.readBufferSize) - resultLen, err := n.DeprecatedPreadv(ctx, usermem.BytesIOSequence(data), tt.offset) + resultLen, err := file.Preadv(ctx, usermem.BytesIOSequence(data), tt.offset) if err != tt.expectedError { return fmt.Errorf("t.Preadv(len: %v, offset: %v) (error) => %v expected %v", tt.readBufferSize, tt.offset, err, tt.expectedError) } @@ -115,12 +120,12 @@ func TestSeqFile(t *testing.T) { testSource.Init() // Create a file that can be R/W. - m := fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}) + m := fs.NewPseudoMountSource() ctx := contexttest.Context(t) contents := map[string]*fs.Inode{ "foo": NewSeqFileInode(ctx, testSource, m), } - root := ramfstest.NewDir(ctx, contents, fs.FilePermsFromMode(0777)) + root := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0777)) // How about opening it? inode := fs.NewInode(root, m, fs.StableAttr{Type: fs.Directory}) @@ -129,9 +134,13 @@ func TestSeqFile(t *testing.T) { t.Fatalf("failed to walk to foo for n2: %v", err) } n2 := dirent2.Inode.InodeOperations + file2, err := n2.GetFile(ctx, dirent2, fs.FileFlags{Read: true, Write: true}) + if err != nil { + t.Fatalf("GetFile returned error: %v", err) + } // Writing? - if _, err := n2.DeprecatedPwritev(nil, usermem.BytesIOSequence([]byte("test")), 0); err == nil { + if _, err := file2.Writev(ctx, usermem.BytesIOSequence([]byte("test"))); err == nil { t.Fatalf("managed to write to n2: %v", err) } @@ -141,7 +150,6 @@ func TestSeqFile(t *testing.T) { t.Fatalf("failed to walk to foo: %v", err) } n3 := dirent3.Inode.InodeOperations - if n2 != n3 { t.Error("got n2 != n3, want same") } @@ -170,13 +178,13 @@ func TestSeqFile(t *testing.T) { // Read the last 3 bytes. {97, 10, testSource.actual[9].Buf[7:], nil}, } - if err := runTableTests(ctx, table, n2); err != nil { + if err := runTableTests(ctx, table, dirent2); err != nil { t.Errorf("runTableTest failed with testSource.update = %v : %v", testSource.update, err) } // Disable updates and do it again. testSource.update = false - if err := runTableTests(ctx, table, n2); err != nil { + if err := runTableTests(ctx, table, dirent2); err != nil { t.Errorf("runTableTest failed with testSource.update = %v: %v", testSource.update, err) } } @@ -188,25 +196,24 @@ func TestSeqFileFileUpdated(t *testing.T) { testSource.update = true // Create a file that can be R/W. - m := fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}) + m := fs.NewPseudoMountSource() ctx := contexttest.Context(t) contents := map[string]*fs.Inode{ "foo": NewSeqFileInode(ctx, testSource, m), } - root := ramfstest.NewDir(ctx, contents, fs.FilePermsFromMode(0777)) + root := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0777)) // How about opening it? inode := fs.NewInode(root, m, fs.StableAttr{Type: fs.Directory}) dirent2, err := root.Lookup(ctx, inode, "foo") if err != nil { - t.Fatalf("failed to walk to foo for n2: %v", err) + t.Fatalf("failed to walk to foo for dirent2: %v", err) } - n2 := dirent2.Inode.InodeOperations table := []testTable{ {0, 16, flatten(testSource.actual[0].Buf, testSource.actual[1].Buf[:6]), nil}, } - if err := runTableTests(ctx, table, n2); err != nil { + if err := runTableTests(ctx, table, dirent2); err != nil { t.Errorf("runTableTest failed: %v", err) } // Delete the first entry. @@ -224,7 +231,7 @@ func TestSeqFileFileUpdated(t *testing.T) { // Read the following two lines. {30, 20, flatten(testSource.actual[3].Buf, testSource.actual[4].Buf), nil}, } - if err := runTableTests(ctx, table, n2); err != nil { + if err := runTableTests(ctx, table, dirent2); err != nil { t.Errorf("runTableTest failed after removing first entry: %v", err) } @@ -238,7 +245,7 @@ func TestSeqFileFileUpdated(t *testing.T) { table = []testTable{ {50, 20, flatten(testSource.actual[4].Buf, testSource.actual[5].Buf), nil}, } - if err := runTableTests(ctx, table, n2); err != nil { + if err := runTableTests(ctx, table, dirent2); err != nil { t.Errorf("runTableTest failed after adding middle entry: %v", err) } // This will be used in a later test. @@ -249,7 +256,7 @@ func TestSeqFileFileUpdated(t *testing.T) { table = []testTable{ {20, 20, []byte{}, io.EOF}, } - if err := runTableTests(ctx, table, n2); err != nil { + if err := runTableTests(ctx, table, dirent2); err != nil { t.Errorf("runTableTest failed after removing all entries: %v", err) } // Restore some of the data. @@ -257,7 +264,7 @@ func TestSeqFileFileUpdated(t *testing.T) { table = []testTable{ {6, 20, testSource.actual[0].Buf[6:], nil}, } - if err := runTableTests(ctx, table, n2); err != nil { + if err := runTableTests(ctx, table, dirent2); err != nil { t.Errorf("runTableTest failed after adding first entry back: %v", err) } @@ -266,7 +273,7 @@ func TestSeqFileFileUpdated(t *testing.T) { table = []testTable{ {30, 20, flatten(testSource.actual[3].Buf, testSource.actual[4].Buf), nil}, } - if err := runTableTests(ctx, table, n2); err != nil { + if err := runTableTests(ctx, table, dirent2); err != nil { t.Errorf("runTableTest failed after extending testSource: %v", err) } } diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go index 54562508d..ee6b9f262 100644 --- a/pkg/sentry/fs/proc/sys.go +++ b/pkg/sentry/fs/proc/sys.go @@ -22,39 +22,15 @@ import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/seqfile" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/rpcinet" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) -// hostname is a file containing the system hostname. -// -// +stateify savable -type hostname struct { - ramfs.Entry -} - -// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. -func (hostname) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - utsns := kernel.UTSNamespaceFromContext(ctx) - contents := []byte(utsns.HostName() + "\n") - - if offset >= int64(len(contents)) { - return 0, io.EOF - } - - n, err := dst.CopyOut(ctx, contents[offset:]) - return int64(n), err -} - -func (p *proc) newHostname(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - h := &hostname{} - h.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0444)) - return newFile(h, msrc, fs.SpecialFile, nil) -} - // mmapMinAddrData backs /proc/sys/vm/mmap_min_addr. // // +stateify savable @@ -101,36 +77,84 @@ func (*overcommitMemory) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandl } func (p *proc) newKernelDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - d.AddChild(ctx, "hostname", p.newHostname(ctx, msrc)) - - d.AddChild(ctx, "shmmax", p.newStubProcFSFile(ctx, msrc, []byte(strconv.FormatUint(linux.SHMMAX, 10)))) - d.AddChild(ctx, "shmall", p.newStubProcFSFile(ctx, msrc, []byte(strconv.FormatUint(linux.SHMALL, 10)))) - d.AddChild(ctx, "shmmni", p.newStubProcFSFile(ctx, msrc, []byte(strconv.FormatUint(linux.SHMMNI, 10)))) - return newFile(d, msrc, fs.SpecialDirectory, nil) + h := hostname{ + SimpleFileInode: *fsutil.NewSimpleFileInode(ctx, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + } + + children := map[string]*fs.Inode{ + "hostname": newProcInode(&h, msrc, fs.SpecialFile, nil), + "shmall": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.SHMALL, 10))), + "shmmax": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.SHMMAX, 10))), + "shmmni": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.SHMMNI, 10))), + } + + d := ramfs.NewDir(ctx, children, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } func (p *proc) newVMDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - d.AddChild(ctx, "mmap_min_addr", seqfile.NewSeqFileInode(ctx, &mmapMinAddrData{p.k}, msrc)) - d.AddChild(ctx, "overcommit_memory", seqfile.NewSeqFileInode(ctx, &overcommitMemory{}, msrc)) - return newFile(d, msrc, fs.SpecialDirectory, nil) + children := map[string]*fs.Inode{ + "mmap_min_addr": seqfile.NewSeqFileInode(ctx, &mmapMinAddrData{p.k}, msrc), + "overcommit_memory": seqfile.NewSeqFileInode(ctx, &overcommitMemory{}, msrc), + } + d := ramfs.NewDir(ctx, children, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } func (p *proc) newSysDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - d.AddChild(ctx, "kernel", p.newKernelDir(ctx, msrc)) - d.AddChild(ctx, "vm", p.newVMDir(ctx, msrc)) + children := map[string]*fs.Inode{ + "kernel": p.newKernelDir(ctx, msrc), + "vm": p.newVMDir(ctx, msrc), + } // If we're using rpcinet we will let it manage /proc/sys/net. if _, ok := p.k.NetworkStack().(*rpcinet.Stack); ok { - d.AddChild(ctx, "net", newRPCInetProcSysNet(ctx, msrc)) + children["net"] = newRPCInetProcSysNet(ctx, msrc) } else { - d.AddChild(ctx, "net", p.newSysNetDir(ctx, msrc)) + children["net"] = p.newSysNetDir(ctx, msrc) } - return newFile(d, msrc, fs.SpecialDirectory, nil) + d := ramfs.NewDir(ctx, children, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) +} + +// hostname is the inode for a file containing the system hostname. +// +// +stateify savable +type hostname struct { + fsutil.SimpleFileInode +} + +// GetFile implements fs.InodeOperations.GetFile. +func (h *hostname) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, d, flags, &hostnameFile{}), nil +} + +var _ fs.InodeOperations = (*hostname)(nil) + +// +stateify savable +type hostnameFile struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSeek `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` } + +// Read implements fs.FileOperations.Read. +func (hf *hostnameFile) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { + utsns := kernel.UTSNamespaceFromContext(ctx) + contents := []byte(utsns.HostName() + "\n") + if offset >= int64(len(contents)) { + return 0, io.EOF + } + n, err := dst.CopyOut(ctx, contents[offset:]) + return int64(n), err + +} + +var _ fs.FileOperations = (*hostnameFile)(nil) diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index b50d43d70..42e9bc47f 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -17,13 +17,17 @@ package proc import ( "fmt" "io" + "sync" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/device" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/inet" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) type tcpMemDir int @@ -33,21 +37,37 @@ const ( tcpWMem ) +// tcpMemInode is used to read/write the size of netstack tcp buffers. +// +// TODO: If we have multiple proc mounts, concurrent writes can +// leave netstack and the proc files in an inconsistent state. Since we set the +// buffer size from these proc files on restore, we may also race and end up in +// an inconsistent state on restore. +// // +stateify savable -type tcpMem struct { - ramfs.Entry - s inet.Stack `state:"wait"` +type tcpMemInode struct { + fsutil.SimpleFileInode + dir tcpMemDir + s inet.Stack `state:"wait"` + + // size stores the tcp buffer size during save, and sets the buffer + // size in netstack in restore. We must save/restore this here, since + // netstack itself is stateless. size inet.TCPBufferSize - dir tcpMemDir -} -func newTCPMem(s inet.Stack, size inet.TCPBufferSize, dir tcpMemDir) *tcpMem { - return &tcpMem{s: s, size: size, dir: dir} + // mu protects against concurrent reads/writes to files based on this + // inode. + mu sync.Mutex `state:"nosave"` } -func newTCPMemInode(ctx context.Context, msrc *fs.MountSource, s inet.Stack, size inet.TCPBufferSize, dir tcpMemDir) *fs.Inode { - tm := newTCPMem(s, size, dir) - tm.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0644)) +var _ fs.InodeOperations = (*tcpMemInode)(nil) + +func newTCPMemInode(ctx context.Context, msrc *fs.MountSource, s inet.Stack, dir tcpMemDir) *fs.Inode { + tm := &tcpMemInode{ + SimpleFileInode: *fsutil.NewSimpleFileInode(ctx, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + s: s, + dir: dir, + } sattr := fs.StableAttr{ DeviceID: device.ProcDevice.DeviceID(), InodeID: device.ProcDevice.NextIno(), @@ -57,62 +77,105 @@ func newTCPMemInode(ctx context.Context, msrc *fs.MountSource, s inet.Stack, siz return fs.NewInode(tm, msrc, sattr) } -// DeprecatedPreadv implements fs.InodeOperations.DeprecatedPreadv. -func (m *tcpMem) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { +// GetFile implements fs.InodeOperations.GetFile. +func (m *tcpMemInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + flags.Pread = true + return fs.NewFile(ctx, dirent, flags, &tcpMemFile{tcpMemInode: m}), nil +} + +// +stateify savable +type tcpMemFile struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + + tcpMemInode *tcpMemInode +} + +var _ fs.FileOperations = (*tcpMemFile)(nil) + +// Read implements fs.FileOperations.Read. +func (f *tcpMemFile) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { if offset != 0 { return 0, io.EOF } - s := fmt.Sprintf("%d\t%d\t%d\n", m.size.Min, m.size.Default, m.size.Max) + f.tcpMemInode.mu.Lock() + defer f.tcpMemInode.mu.Unlock() + + size, err := readSize(f.tcpMemInode.dir, f.tcpMemInode.s) + if err != nil { + return 0, err + } + s := fmt.Sprintf("%d\t%d\t%d\n", size.Min, size.Default, size.Max) n, err := dst.CopyOut(ctx, []byte(s)) return int64(n), err } -// Truncate implements fs.InodeOperations.Truncate. -func (*tcpMem) Truncate(context.Context, *fs.Inode, int64) error { - return nil -} - -// DeprecatedPwritev implements fs.InodeOperations.DeprecatedPwritev. -func (m *tcpMem) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { +// Write implements fs.FileOperations.Write. +func (f *tcpMemFile) Write(ctx context.Context, _ *fs.File, src usermem.IOSequence, offset int64) (int64, error) { if src.NumBytes() == 0 { return 0, nil } - src = src.TakeFirst(usermem.PageSize - 1) + f.tcpMemInode.mu.Lock() + defer f.tcpMemInode.mu.Unlock() - buf := []int32{int32(m.size.Min), int32(m.size.Default), int32(m.size.Max)} + src = src.TakeFirst(usermem.PageSize - 1) + size, err := readSize(f.tcpMemInode.dir, f.tcpMemInode.s) + if err != nil { + return 0, err + } + buf := []int32{int32(size.Min), int32(size.Default), int32(size.Max)} n, cperr := usermem.CopyInt32StringsInVec(ctx, src.IO, src.Addrs, buf, src.Opts) - m.size = inet.TCPBufferSize{ + newSize := inet.TCPBufferSize{ Min: int(buf[0]), Default: int(buf[1]), Max: int(buf[2]), } - if err := m.writeSize(); err != nil { + if err := writeSize(f.tcpMemInode.dir, f.tcpMemInode.s, newSize); err != nil { return n, err } return n, cperr } -func (m *tcpMem) writeSize() error { - switch m.dir { +func readSize(dirType tcpMemDir, s inet.Stack) (inet.TCPBufferSize, error) { + switch dirType { + case tcpRMem: + return s.TCPReceiveBufferSize() + case tcpWMem: + return s.TCPSendBufferSize() + default: + panic(fmt.Sprintf("unknown tcpMemFile type: %v", dirType)) + } +} + +func writeSize(dirType tcpMemDir, s inet.Stack, size inet.TCPBufferSize) error { + switch dirType { case tcpRMem: - return m.s.SetTCPReceiveBufferSize(m.size) + return s.SetTCPReceiveBufferSize(size) case tcpWMem: - return m.s.SetTCPSendBufferSize(m.size) + return s.SetTCPSendBufferSize(size) default: - panic(fmt.Sprintf("unknown tcpMem.dir: %v", m.dir)) + panic(fmt.Sprintf("unknown tcpMemFile type: %v", dirType)) } } // +stateify savable type tcpSack struct { - ramfs.Entry - s inet.Stack `state:"wait"` + stack inet.Stack `state:"wait"` enabled *bool + fsutil.SimpleFileInode } func newTCPSackInode(ctx context.Context, msrc *fs.MountSource, s inet.Stack) *fs.Inode { - ts := &tcpSack{s: s} - ts.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0644)) + ts := &tcpSack{ + SimpleFileInode: *fsutil.NewSimpleFileInode(ctx, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + stack: s, + } sattr := fs.StableAttr{ DeviceID: device.ProcDevice.DeviceID(), InodeID: device.ProcDevice.NextIno(), @@ -122,21 +185,48 @@ func newTCPSackInode(ctx context.Context, msrc *fs.MountSource, s inet.Stack) *f return fs.NewInode(ts, msrc, sattr) } -func (s *tcpSack) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { +// GetFile implements fs.InodeOperations.GetFile. +func (s *tcpSack) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + flags.Pread = true + flags.Pwrite = true + return fs.NewFile(ctx, dirent, flags, &tcpSackFile{ + tcpSack: s, + stack: s.stack, + }), nil +} + +// +stateify savable +type tcpSackFile struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + + tcpSack *tcpSack + + stack inet.Stack `state:"wait"` +} + +// Read implements fs.FileOperations.Read. +func (f *tcpSackFile) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { if offset != 0 { return 0, io.EOF } - if s.enabled == nil { - sack, err := s.s.TCPSACKEnabled() + if f.tcpSack.enabled == nil { + sack, err := f.stack.TCPSACKEnabled() if err != nil { return 0, err } - s.enabled = &sack + f.tcpSack.enabled = &sack } val := "0\n" - if *s.enabled { + if *f.tcpSack.enabled { // Technically, this is not quite compatible with Linux. Linux // stores these as an integer, so if you write "2" into // tcp_sack, you should get 2 back. Tough luck. @@ -146,13 +236,8 @@ func (s *tcpSack) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, return int64(n), err } -// Truncate implements fs.InodeOperations.Truncate. -func (*tcpSack) Truncate(context.Context, *fs.Inode, int64) error { - return nil -} - -// DeprecatedPwritev implements fs.InodeOperations.DeprecatedPwritev. -func (s *tcpSack) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { +// Write implements fs.FileOperations.Write. +func (f *tcpSackFile) Write(ctx context.Context, _ *fs.File, src usermem.IOSequence, offset int64) (int64, error) { if src.NumBytes() == 0 { return 0, nil } @@ -163,100 +248,104 @@ func (s *tcpSack) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, if err != nil { return n, err } - if s.enabled == nil { - s.enabled = new(bool) + if f.tcpSack.enabled == nil { + f.tcpSack.enabled = new(bool) } - *s.enabled = v != 0 - return n, s.s.SetTCPSACKEnabled(*s.enabled) + *f.tcpSack.enabled = v != 0 + return n, f.tcpSack.stack.SetTCPSACKEnabled(*f.tcpSack.enabled) } func (p *proc) newSysNetCore(ctx context.Context, msrc *fs.MountSource, s inet.Stack) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - // The following files are simple stubs until they are implemented in // netstack, most of these files are configuration related. We use the // value closest to the actual netstack behavior or any empty file, // all of these files will have mode 0444 (read-only for all users). - d.AddChild(ctx, "default_qdisc", p.newStubProcFSFile(ctx, msrc, []byte("pfifo_fast"))) - d.AddChild(ctx, "message_burst", p.newStubProcFSFile(ctx, msrc, []byte("10"))) - d.AddChild(ctx, "message_cost", p.newStubProcFSFile(ctx, msrc, []byte("5"))) - d.AddChild(ctx, "optmem_max", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "rmem_default", p.newStubProcFSFile(ctx, msrc, []byte("212992"))) - d.AddChild(ctx, "rmem_max", p.newStubProcFSFile(ctx, msrc, []byte("212992"))) - d.AddChild(ctx, "somaxconn", p.newStubProcFSFile(ctx, msrc, []byte("128"))) - d.AddChild(ctx, "wmem_default", p.newStubProcFSFile(ctx, msrc, []byte("212992"))) - d.AddChild(ctx, "wmem_max", p.newStubProcFSFile(ctx, msrc, []byte("212992"))) - - return newFile(d, msrc, fs.SpecialDirectory, nil) + contents := map[string]*fs.Inode{ + "default_qdisc": newStaticProcInode(ctx, msrc, []byte("pfifo_fast")), + "message_burst": newStaticProcInode(ctx, msrc, []byte("10")), + "message_cost": newStaticProcInode(ctx, msrc, []byte("5")), + "optmem_max": newStaticProcInode(ctx, msrc, []byte("0")), + "rmem_default": newStaticProcInode(ctx, msrc, []byte("212992")), + "rmem_max": newStaticProcInode(ctx, msrc, []byte("212992")), + "somaxconn": newStaticProcInode(ctx, msrc, []byte("128")), + "wmem_default": newStaticProcInode(ctx, msrc, []byte("212992")), + "wmem_max": newStaticProcInode(ctx, msrc, []byte("212992")), + } + + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } func (p *proc) newSysNetIPv4Dir(ctx context.Context, msrc *fs.MountSource, s inet.Stack) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) + contents := map[string]*fs.Inode{ + // Add tcp_sack. + "tcp_sack": newTCPSackInode(ctx, msrc, s), + + // The following files are simple stubs until they are + // implemented in netstack, most of these files are + // configuration related. We use the value closest to the + // actual netstack behavior or any empty file, all of these + // files will have mode 0444 (read-only for all users). + "ip_local_port_range": newStaticProcInode(ctx, msrc, []byte("16000 65535")), + "ip_local_reserved_ports": newStaticProcInode(ctx, msrc, []byte("")), + "ipfrag_time": newStaticProcInode(ctx, msrc, []byte("30")), + "ip_nonlocal_bind": newStaticProcInode(ctx, msrc, []byte("0")), + "ip_no_pmtu_disc": newStaticProcInode(ctx, msrc, []byte("1")), + + // tcp_allowed_congestion_control tell the user what they are + // able to do as an unprivledged process so we leave it empty. + "tcp_allowed_congestion_control": newStaticProcInode(ctx, msrc, []byte("")), + "tcp_available_congestion_control": newStaticProcInode(ctx, msrc, []byte("reno")), + "tcp_congestion_control": newStaticProcInode(ctx, msrc, []byte("reno")), + + // Many of the following stub files are features netstack + // doesn't support. The unsupported features return "0" to + // indicate they are disabled. + "tcp_base_mss": newStaticProcInode(ctx, msrc, []byte("1280")), + "tcp_dsack": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_early_retrans": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_fack": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_fastopen": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_fastopen_key": newStaticProcInode(ctx, msrc, []byte("")), + "tcp_invalid_ratelimit": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_keepalive_intvl": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_keepalive_probes": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_keepalive_time": newStaticProcInode(ctx, msrc, []byte("7200")), + "tcp_mtu_probing": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_no_metrics_save": newStaticProcInode(ctx, msrc, []byte("1")), + "tcp_probe_interval": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_probe_threshold": newStaticProcInode(ctx, msrc, []byte("0")), + "tcp_retries1": newStaticProcInode(ctx, msrc, []byte("3")), + "tcp_retries2": newStaticProcInode(ctx, msrc, []byte("15")), + "tcp_rfc1337": newStaticProcInode(ctx, msrc, []byte("1")), + "tcp_slow_start_after_idle": newStaticProcInode(ctx, msrc, []byte("1")), + "tcp_synack_retries": newStaticProcInode(ctx, msrc, []byte("5")), + "tcp_syn_retries": newStaticProcInode(ctx, msrc, []byte("3")), + "tcp_timestamps": newStaticProcInode(ctx, msrc, []byte("1")), + } // Add tcp_rmem. - if rs, err := s.TCPReceiveBufferSize(); err == nil { - d.AddChild(ctx, "tcp_rmem", newTCPMemInode(ctx, msrc, s, rs, tcpRMem)) + if _, err := s.TCPReceiveBufferSize(); err == nil { + contents["tcp_rmem"] = newTCPMemInode(ctx, msrc, s, tcpRMem) } // Add tcp_wmem. - if ss, err := s.TCPSendBufferSize(); err == nil { - d.AddChild(ctx, "tcp_wmem", newTCPMemInode(ctx, msrc, s, ss, tcpWMem)) + if _, err := s.TCPSendBufferSize(); err == nil { + contents["tcp_wmem"] = newTCPMemInode(ctx, msrc, s, tcpWMem) } - // Add tcp_sack. - d.AddChild(ctx, "tcp_sack", newTCPSackInode(ctx, msrc, s)) - - // The following files are simple stubs until they are implemented in - // netstack, most of these files are configuration related. We use the - // value closest to the actual netstack behavior or any empty file, - // all of these files will have mode 0444 (read-only for all users). - d.AddChild(ctx, "ip_local_port_range", p.newStubProcFSFile(ctx, msrc, []byte("16000 65535"))) - d.AddChild(ctx, "ip_local_reserved_ports", p.newStubProcFSFile(ctx, msrc, []byte(""))) - d.AddChild(ctx, "ipfrag_time", p.newStubProcFSFile(ctx, msrc, []byte("30"))) - d.AddChild(ctx, "ip_nonlocal_bind", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "ip_no_pmtu_disc", p.newStubProcFSFile(ctx, msrc, []byte("1"))) - - // tcp_allowed_congestion_control tell the user what they are able to do as an - // unprivledged process so we leave it empty. - d.AddChild(ctx, "tcp_allowed_congestion_control", p.newStubProcFSFile(ctx, msrc, []byte(""))) - d.AddChild(ctx, "tcp_available_congestion_control", p.newStubProcFSFile(ctx, msrc, []byte("reno"))) - d.AddChild(ctx, "tcp_congestion_control", p.newStubProcFSFile(ctx, msrc, []byte("reno"))) - - // Many of the following stub files are features netstack doesn't support - // and are therefore "0" for disabled. - d.AddChild(ctx, "tcp_base_mss", p.newStubProcFSFile(ctx, msrc, []byte("1280"))) - d.AddChild(ctx, "tcp_dsack", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_early_retrans", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_fack", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_fastopen", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_fastopen_key", p.newStubProcFSFile(ctx, msrc, []byte(""))) - d.AddChild(ctx, "tcp_invalid_ratelimit", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_keepalive_intvl", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_keepalive_probes", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_keepalive_time", p.newStubProcFSFile(ctx, msrc, []byte("7200"))) - d.AddChild(ctx, "tcp_mtu_probing", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_no_metrics_save", p.newStubProcFSFile(ctx, msrc, []byte("1"))) - d.AddChild(ctx, "tcp_probe_interval", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_probe_threshold", p.newStubProcFSFile(ctx, msrc, []byte("0"))) - d.AddChild(ctx, "tcp_retries1", p.newStubProcFSFile(ctx, msrc, []byte("3"))) - d.AddChild(ctx, "tcp_retries2", p.newStubProcFSFile(ctx, msrc, []byte("15"))) - d.AddChild(ctx, "tcp_rfc1337", p.newStubProcFSFile(ctx, msrc, []byte("1"))) - d.AddChild(ctx, "tcp_slow_start_after_idle", p.newStubProcFSFile(ctx, msrc, []byte("1"))) - d.AddChild(ctx, "tcp_synack_retries", p.newStubProcFSFile(ctx, msrc, []byte("5"))) - d.AddChild(ctx, "tcp_syn_retries", p.newStubProcFSFile(ctx, msrc, []byte("3"))) - d.AddChild(ctx, "tcp_timestamps", p.newStubProcFSFile(ctx, msrc, []byte("1"))) - - return newFile(d, msrc, fs.SpecialDirectory, nil) + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } func (p *proc) newSysNetDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(ctx, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) + var contents map[string]*fs.Inode if s := p.k.NetworkStack(); s != nil { - d.AddChild(ctx, "ipv4", p.newSysNetIPv4Dir(ctx, msrc, s)) - d.AddChild(ctx, "core", p.newSysNetCore(ctx, msrc, s)) + contents = map[string]*fs.Inode{ + "ipv4": p.newSysNetIPv4Dir(ctx, msrc, s), + "core": p.newSysNetCore(ctx, msrc, s), + } } - return newFile(d, msrc, fs.SpecialDirectory, nil) + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + return newProcInode(d, msrc, fs.SpecialDirectory, nil) } diff --git a/pkg/sentry/fs/proc/sys_net_state.go b/pkg/sentry/fs/proc/sys_net_state.go index 7f46776c0..5f481a1cf 100644 --- a/pkg/sentry/fs/proc/sys_net_state.go +++ b/pkg/sentry/fs/proc/sys_net_state.go @@ -16,17 +16,26 @@ package proc import "fmt" +// beforeSave is invoked by stateify. +func (t *tcpMemInode) beforeSave() { + size, err := readSize(t.dir, t.s) + if err != nil { + panic(fmt.Sprintf("failed to read TCP send / receive buffer sizes: %v", err)) + } + t.size = size +} + // afterLoad is invoked by stateify. -func (m *tcpMem) afterLoad() { - if err := m.writeSize(); err != nil { - panic(fmt.Sprintf("failed to write previous TCP send / receive buffer sizes [%v]: %v", m.size, err)) +func (t *tcpMemInode) afterLoad() { + if err := writeSize(t.dir, t.s, t.size); err != nil { + panic(fmt.Sprintf("failed to write previous TCP send / receive buffer sizes [%v]: %v", t.size, err)) } } // afterLoad is invoked by stateify. func (s *tcpSack) afterLoad() { if s.enabled != nil { - if err := s.s.SetTCPSACKEnabled(*s.enabled); err != nil { + if err := s.stack.SetTCPSACKEnabled(*s.enabled); err != nil { panic(fmt.Sprintf("failed to set previous TCP sack configuration [%v]: %v", *s.enabled, err)) } } diff --git a/pkg/sentry/fs/proc/sys_net_test.go b/pkg/sentry/fs/proc/sys_net_test.go index 0ce9d30f1..ea0d94fce 100644 --- a/pkg/sentry/fs/proc/sys_net_test.go +++ b/pkg/sentry/fs/proc/sys_net_test.go @@ -26,13 +26,14 @@ func TestQuerySendBufferSize(t *testing.T) { ctx := context.Background() s := inet.NewTestStack() s.TCPSendBufSize = inet.TCPBufferSize{100, 200, 300} - tm := newTCPMem(s, s.TCPSendBufSize, tcpWMem) + tmi := &tcpMemInode{s: s, dir: tcpWMem} + tmf := &tcpMemFile{tcpMemInode: tmi} buf := make([]byte, 100) dst := usermem.BytesIOSequence(buf) - n, err := tm.DeprecatedPreadv(ctx, dst, 0) + n, err := tmf.Read(ctx, nil, dst, 0) if err != nil { - t.Fatalf("DeprecatedPreadv failed: %v", err) + t.Fatalf("Read failed: %v", err) } if got, want := string(buf[:n]), "100\t200\t300\n"; got != want { @@ -44,13 +45,14 @@ func TestQueryRecvBufferSize(t *testing.T) { ctx := context.Background() s := inet.NewTestStack() s.TCPRecvBufSize = inet.TCPBufferSize{100, 200, 300} - tm := newTCPMem(s, s.TCPRecvBufSize, tcpRMem) + tmi := &tcpMemInode{s: s, dir: tcpRMem} + tmf := &tcpMemFile{tcpMemInode: tmi} buf := make([]byte, 100) dst := usermem.BytesIOSequence(buf) - n, err := tm.DeprecatedPreadv(ctx, dst, 0) + n, err := tmf.Read(ctx, nil, dst, 0) if err != nil { - t.Fatalf("DeprecatedPreadv failed: %v", err) + t.Fatalf("Read failed: %v", err) } if got, want := string(buf[:n]), "100\t200\t300\n"; got != want { @@ -85,12 +87,13 @@ func TestConfigureSendBufferSize(t *testing.T) { s := inet.NewTestStack() for _, c := range cases { s.TCPSendBufSize = c.initial - tm := newTCPMem(s, c.initial, tcpWMem) + tmi := &tcpMemInode{s: s, dir: tcpWMem} + tmf := &tcpMemFile{tcpMemInode: tmi} // Write the values. src := usermem.BytesIOSequence([]byte(c.str)) - if n, err := tm.DeprecatedPwritev(ctx, src, 0); n != int64(len(c.str)) || err != nil { - t.Errorf("DeprecatedPwritev, case = %q: got (%d, %v), wanted (%d, nil)", c.str, n, err, len(c.str)) + if n, err := tmf.Write(ctx, nil, src, 0); n != int64(len(c.str)) || err != nil { + t.Errorf("Write, case = %q: got (%d, %v), wanted (%d, nil)", c.str, n, err, len(c.str)) } // Read the values from the stack and check them. @@ -105,12 +108,13 @@ func TestConfigureRecvBufferSize(t *testing.T) { s := inet.NewTestStack() for _, c := range cases { s.TCPRecvBufSize = c.initial - tm := newTCPMem(s, c.initial, tcpRMem) + tmi := &tcpMemInode{s: s, dir: tcpRMem} + tmf := &tcpMemFile{tcpMemInode: tmi} // Write the values. src := usermem.BytesIOSequence([]byte(c.str)) - if n, err := tm.DeprecatedPwritev(ctx, src, 0); n != int64(len(c.str)) || err != nil { - t.Errorf("DeprecatedPwritev, case = %q: got (%d, %v), wanted (%d, nil)", c.str, n, err, len(c.str)) + if n, err := tmf.Write(ctx, nil, src, 0); n != int64(len(c.str)) || err != nil { + t.Errorf("Write, case = %q: got (%d, %v), wanted (%d, nil)", c.str, n, err, len(c.str)) } // Read the values from the stack and check them. diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go index 91bda8a95..41981a973 100644 --- a/pkg/sentry/fs/proc/task.go +++ b/pkg/sentry/fs/proc/task.go @@ -24,6 +24,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/device" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc/seqfile" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" @@ -32,6 +33,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/usage" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // getTaskMM returns t's MemoryManager. If getTaskMM succeeds, the MemoryManager's @@ -57,19 +59,19 @@ func getTaskMM(t *kernel.Task) (*mm.MemoryManager, error) { type taskDir struct { ramfs.Dir - // t is the associated kernel task that owns this file. - t *kernel.Task + t *kernel.Task + pidns *kernel.PIDNamespace } +var _ fs.InodeOperations = (*taskDir)(nil) + // newTaskDir creates a new proc task entry. func newTaskDir(t *kernel.Task, msrc *fs.MountSource, pidns *kernel.PIDNamespace, showSubtasks bool) *fs.Inode { - d := &taskDir{t: t} - // TODO: Set EUID/EGID based on dumpability. - d.InitDir(t, map[string]*fs.Inode{ + contents := map[string]*fs.Inode{ "auxv": newAuxvec(t, msrc), - "cmdline": newExecArgFile(t, msrc, cmdlineExecArg), + "cmdline": newExecArgInode(t, msrc, cmdlineExecArg), "comm": newComm(t, msrc), - "environ": newExecArgFile(t, msrc, environExecArg), + "environ": newExecArgInode(t, msrc, environExecArg), "exe": newExe(t, msrc), "fd": newFdDir(t, msrc), "fdinfo": newFdInfoDir(t, msrc), @@ -87,11 +89,18 @@ func newTaskDir(t *kernel.Task, msrc *fs.MountSource, pidns *kernel.PIDNamespace "statm": newStatm(t, msrc), "status": newStatus(t, msrc, pidns), "uid_map": newUIDMap(t, msrc), - }, fs.RootOwner, fs.FilePermsFromMode(0555)) + } if showSubtasks { - d.AddChild(t, "task", newSubtasks(t, msrc, pidns)) + contents["task"] = newSubtasks(t, msrc, pidns) } - return newFile(d, msrc, fs.SpecialDirectory, t) + + // TODO: Set EUID/EGID based on dumpability. + d := &taskDir{ + Dir: *ramfs.NewDir(t, contents, fs.RootOwner, fs.FilePermsFromMode(0555)), + t: t, + pidns: pidns, + } + return newProcInode(d, msrc, fs.SpecialDirectory, t) } // subtasks represents a /proc/TID/task directory. @@ -100,15 +109,19 @@ func newTaskDir(t *kernel.Task, msrc *fs.MountSource, pidns *kernel.PIDNamespace type subtasks struct { ramfs.Dir - t *kernel.Task - + t *kernel.Task pidns *kernel.PIDNamespace } +var _ fs.InodeOperations = (*subtasks)(nil) + func newSubtasks(t *kernel.Task, msrc *fs.MountSource, pidns *kernel.PIDNamespace) *fs.Inode { - s := &subtasks{t: t, pidns: pidns} - s.InitDir(t, nil, fs.RootOwner, fs.FilePermsFromMode(0555)) - return newFile(s, msrc, fs.SpecialDirectory, t) + s := &subtasks{ + Dir: *ramfs.NewDir(t, nil, fs.RootOwner, fs.FilePermsFromMode(0555)), + t: t, + pidns: pidns, + } + return newProcInode(s, msrc, fs.SpecialDirectory, t) } // UnstableAttr returns unstable attributes of the subtasks. @@ -123,35 +136,52 @@ func (s *subtasks) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.Unstab return uattr, nil } -// Lookup loads an Inode in a task's subtask directory into a Dirent. -func (s *subtasks) Lookup(ctx context.Context, dir *fs.Inode, p string) (*fs.Dirent, error) { - tid, err := strconv.ParseUint(p, 10, 32) - if err != nil { - return nil, syserror.ENOENT - } +// GetFile implements fs.InodeOperations.GetFile. +func (s *subtasks) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &subtasksFile{t: s.t, pidns: s.pidns}), nil +} - task := s.pidns.TaskWithID(kernel.ThreadID(tid)) - if task == nil { - return nil, syserror.ENOENT - } - if task.ThreadGroup() != s.t.ThreadGroup() { - return nil, syserror.ENOENT - } +// +stateify savable +type subtasksFile struct { + fsutil.DirFileOperations `state:"nosave"` - td := newTaskDir(task, dir.MountSource, s.pidns, false) - return fs.NewDirent(td, p), nil + t *kernel.Task + pidns *kernel.PIDNamespace } -// DeprecatedReaddir lists a task's subtask directory. -func (s *subtasks) DeprecatedReaddir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { - tasks := s.t.ThreadGroup().MemberIDs(s.pidns) +// Readdir implements fs.FileOperations.Readdir. +func (f *subtasksFile) Readdir(ctx context.Context, file *fs.File, ser fs.DentrySerializer) (int64, error) { + dirCtx := fs.DirCtx{ + Serializer: ser, + } + + // Note that unlike most Readdir implementations, the offset here is + // not an index into the subtasks, but rather the TID of the next + // subtask to emit. + offset := file.Offset() + + if offset == 0 { + // Serialize "." and "..". + root := fs.RootFromContext(ctx) + defer root.DecRef() + dot, dotdot := file.Dirent.GetDotAttrs(root) + if err := dirCtx.DirEmit(".", dot); err != nil { + return offset, err + } + if err := dirCtx.DirEmit("..", dotdot); err != nil { + return offset, err + } + } + + // Serialize tasks. + tasks := f.t.ThreadGroup().MemberIDs(f.pidns) taskInts := make([]int, 0, len(tasks)) for _, tid := range tasks { taskInts = append(taskInts, int(tid)) } // Find the task to start at. - idx := sort.SearchInts(taskInts, offset) + idx := sort.SearchInts(taskInts, int(offset)) if idx == len(taskInts) { return offset, nil } @@ -163,12 +193,33 @@ func (s *subtasks) DeprecatedReaddir(ctx context.Context, dirCtx *fs.DirCtx, off attr := fs.GenericDentAttr(fs.SpecialDirectory, device.ProcDevice) if err := dirCtx.DirEmit(name, attr); err != nil { // Returned offset is next tid to serialize. - return tid, err + return int64(tid), err } } // We serialized them all. Next offset should be higher than last // serialized tid. - return tid + 1, nil + return int64(tid) + 1, nil +} + +var _ fs.FileOperations = (*subtasksFile)(nil) + +// Lookup loads an Inode in a task's subtask directory into a Dirent. +func (s *subtasks) Lookup(ctx context.Context, dir *fs.Inode, p string) (*fs.Dirent, error) { + tid, err := strconv.ParseUint(p, 10, 32) + if err != nil { + return nil, syserror.ENOENT + } + + task := s.pidns.TaskWithID(kernel.ThreadID(tid)) + if task == nil { + return nil, syserror.ENOENT + } + if task.ThreadGroup() != s.t.ThreadGroup() { + return nil, syserror.ENOENT + } + + td := newTaskDir(task, dir.MountSource, s.pidns, false) + return fs.NewDirent(td, p), nil } // exe is an fs.InodeOperations symlink for the /proc/PID/exe file. @@ -181,9 +232,11 @@ type exe struct { } func newExe(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - exeSymlink := &exe{t: t} - exeSymlink.InitSymlink(t, fs.RootOwner, "") - return newFile(exeSymlink, msrc, fs.Symlink, t) + exeSymlink := &exe{ + Symlink: *ramfs.NewSymlink(t, fs.RootOwner, ""), + t: t, + } + return newProcInode(exeSymlink, msrc, fs.Symlink, t) } func (e *exe) executable() (d *fs.Dirent, err error) { @@ -231,55 +284,48 @@ func (e *exe) Readlink(ctx context.Context, inode *fs.Inode) (string, error) { return n, nil } -// namespaceFile represents a file in the namespacefs, such as the files in -// /proc//ns. +// namespaceSymlink represents a symlink in the namespacefs, such as the files +// in /proc//ns. // // +stateify savable -type namespaceFile struct { +type namespaceSymlink struct { ramfs.Symlink t *kernel.Task } -func newNamespaceFile(t *kernel.Task, msrc *fs.MountSource, name string) *fs.Inode { - n := &namespaceFile{t: t} - n.InitSymlink(t, fs.RootOwner, "") - +func newNamespaceSymlink(t *kernel.Task, msrc *fs.MountSource, name string) *fs.Inode { // TODO: Namespace symlinks should contain the namespace name and the // inode number for the namespace instance, so for example user:[123456]. We // currently fake the inode number by sticking the symlink inode in its // place. - n.Target = fmt.Sprintf("%s:[%d]", name, device.ProcDevice.NextIno()) - - return newFile(n, msrc, fs.Symlink, t) + target := fmt.Sprintf("%s:[%d]", name, device.ProcDevice.NextIno()) + n := &namespaceSymlink{ + Symlink: *ramfs.NewSymlink(t, fs.RootOwner, target), + t: t, + } + return newProcInode(n, msrc, fs.Symlink, t) } // Getlink implements fs.InodeOperations.Getlink. -func (n *namespaceFile) Getlink(ctx context.Context, inode *fs.Inode) (*fs.Dirent, error) { +func (n *namespaceSymlink) Getlink(ctx context.Context, inode *fs.Inode) (*fs.Dirent, error) { if !kernel.ContextCanTrace(ctx, n.t, false) { return nil, syserror.EACCES } // Create a new regular file to fake the namespace file. - node := &ramfs.Entry{} - node.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0777)) - sattr := fs.StableAttr{ - DeviceID: device.ProcDevice.DeviceID(), - InodeID: device.ProcDevice.NextIno(), - BlockSize: usermem.PageSize, - Type: fs.RegularFile, - } - return fs.NewDirent(fs.NewInode(node, inode.MountSource, sattr), n.Symlink.Target), nil + iops := fsutil.NewNoReadWriteFileInode(ctx, fs.RootOwner, fs.FilePermsFromMode(0777), linux.PROC_SUPER_MAGIC) + return fs.NewDirent(newProcInode(iops, inode.MountSource, fs.RegularFile, nil), n.Symlink.Target), nil } func newNamespaceDir(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - d := &ramfs.Dir{} - d.InitDir(t, map[string]*fs.Inode{ - "net": newNamespaceFile(t, msrc, "net"), - "pid": newNamespaceFile(t, msrc, "pid"), - "user": newNamespaceFile(t, msrc, "user"), - }, fs.RootOwner, fs.FilePermsFromMode(0511)) - return newFile(d, msrc, fs.SpecialDirectory, t) + contents := map[string]*fs.Inode{ + "net": newNamespaceSymlink(t, msrc, "net"), + "pid": newNamespaceSymlink(t, msrc, "pid"), + "user": newNamespaceSymlink(t, msrc, "user"), + } + d := ramfs.NewDir(t, contents, fs.RootOwner, fs.FilePermsFromMode(0511)) + return newProcInode(d, msrc, fs.SpecialDirectory, t) } // mapsData implements seqfile.SeqSource for /proc/[pid]/maps. @@ -290,7 +336,7 @@ type mapsData struct { } func newMaps(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - return newFile(seqfile.NewSeqFile(t, &mapsData{t}), msrc, fs.SpecialFile, t) + return newProcInode(seqfile.NewSeqFile(t, &mapsData{t}), msrc, fs.SpecialFile, t) } func (md *mapsData) mm() *mm.MemoryManager { @@ -330,7 +376,7 @@ type smapsData struct { } func newSmaps(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - return newFile(seqfile.NewSeqFile(t, &smapsData{t}), msrc, fs.SpecialFile, t) + return newProcInode(seqfile.NewSeqFile(t, &smapsData{t}), msrc, fs.SpecialFile, t) } func (sd *smapsData) mm() *mm.MemoryManager { @@ -376,7 +422,7 @@ type taskStatData struct { } func newTaskStat(t *kernel.Task, msrc *fs.MountSource, showSubtasks bool, pidns *kernel.PIDNamespace) *fs.Inode { - return newFile(seqfile.NewSeqFile(t, &taskStatData{t, showSubtasks /* tgstats */, pidns}), msrc, fs.SpecialFile, t) + return newProcInode(seqfile.NewSeqFile(t, &taskStatData{t, showSubtasks /* tgstats */, pidns}), msrc, fs.SpecialFile, t) } // NeedsUpdate returns whether the generation is old or not. @@ -450,7 +496,7 @@ type statmData struct { } func newStatm(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - return newFile(seqfile.NewSeqFile(t, &statmData{t}), msrc, fs.SpecialFile, t) + return newProcInode(seqfile.NewSeqFile(t, &statmData{t}), msrc, fs.SpecialFile, t) } // NeedsUpdate implements seqfile.SeqSource.NeedsUpdate. @@ -487,7 +533,7 @@ type statusData struct { } func newStatus(t *kernel.Task, msrc *fs.MountSource, pidns *kernel.PIDNamespace) *fs.Inode { - return newFile(seqfile.NewSeqFile(t, &statusData{t, pidns}), msrc, fs.SpecialFile, t) + return newProcInode(seqfile.NewSeqFile(t, &statusData{t, pidns}), msrc, fs.SpecialFile, t) } // NeedsUpdate implements seqfile.SeqSource.NeedsUpdate. @@ -552,7 +598,7 @@ type ioData struct { } func newIO(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - return newFile(seqfile.NewSeqFile(t, &ioData{t.ThreadGroup()}), msrc, fs.SpecialFile, t) + return newProcInode(seqfile.NewSeqFile(t, &ioData{t.ThreadGroup()}), msrc, fs.SpecialFile, t) } // NeedsUpdate returns whether the generation is old or not. @@ -590,25 +636,49 @@ func (i *ioData) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]se // // +stateify savable type comm struct { - ramfs.Entry + fsutil.SimpleFileInode t *kernel.Task } // newComm returns a new comm file. func newComm(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - c := &comm{t: t} - c.InitEntry(t, fs.RootOwner, fs.FilePermsFromMode(0444)) - return newFile(c, msrc, fs.SpecialFile, t) + c := &comm{ + SimpleFileInode: *fsutil.NewSimpleFileInode(t, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + t: t, + } + return newProcInode(c, msrc, fs.SpecialFile, t) +} + +// GetFile implements fs.InodeOperations.GetFile. +func (c *comm) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &commFile{t: c.t}), nil +} + +// +stateify savable +type commFile struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + + t *kernel.Task } -// DeprecatedPreadv reads the current command name. -func (c *comm) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { +var _ fs.FileOperations = (*commFile)(nil) + +// Read implements fs.FileOperations.Read. +func (f *commFile) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { if offset < 0 { return 0, syserror.EINVAL } - buf := []byte(c.t.Name() + "\n") + buf := []byte(f.t.Name() + "\n") if offset >= int64(len(buf)) { return 0, io.EOF } @@ -621,25 +691,47 @@ func (c *comm) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, off // // +stateify savable type auxvec struct { - ramfs.Entry + fsutil.SimpleFileInode t *kernel.Task } // newAuxvec returns a new auxvec file. func newAuxvec(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { - a := &auxvec{t: t} - a.InitEntry(t, fs.RootOwner, fs.FilePermsFromMode(0400)) - return newFile(a, msrc, fs.SpecialFile, t) + a := &auxvec{ + SimpleFileInode: *fsutil.NewSimpleFileInode(t, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + t: t, + } + return newProcInode(a, msrc, fs.SpecialFile, t) +} + +// GetFile implements fs.InodeOperations.GetFile. +func (a *auxvec) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &auxvecFile{t: a.t}), nil +} + +// +stateify savable +type auxvecFile struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + + t *kernel.Task } -// DeprecatedPreadv reads the current auxiliary vector. -func (a *auxvec) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { +// Read implements fs.FileOperations.Read. +func (f *auxvecFile) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { if offset < 0 { return 0, syserror.EINVAL } - m, err := getTaskMM(a.t) + m, err := getTaskMM(f.t) if err != nil { return 0, err } diff --git a/pkg/sentry/fs/proc/uid_gid_map.go b/pkg/sentry/fs/proc/uid_gid_map.go index f70399686..815c40b7f 100644 --- a/pkg/sentry/fs/proc/uid_gid_map.go +++ b/pkg/sentry/fs/proc/uid_gid_map.go @@ -90,12 +90,13 @@ func newGIDMap(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { } func newIDMap(t *kernel.Task, msrc *fs.MountSource, gids bool) *fs.Inode { - imsf := &idMapSeqFile{seqfile.SeqFile{SeqSource: &idMapSeqSource{ - t: t, - gids: gids, - }}} - imsf.InitEntry(t, fs.RootOwner, fs.FilePermsFromMode(0644)) - return newFile(imsf, msrc, fs.SpecialFile, t) + imsf := &idMapSeqFile{ + *seqfile.NewSeqFile(t, &idMapSeqSource{ + t: t, + gids: gids, + }), + } + return newProcInode(imsf, msrc, fs.SpecialFile, t) } func (imsf *idMapSeqFile) source() *idMapSeqSource { @@ -106,8 +107,8 @@ func (imsf *idMapSeqFile) source() *idMapSeqSource { // Linux 3.18, the limit is five lines." - user_namespaces(7) const maxIDMapLines = 5 -// DeprecatedPwritev implements fs.InodeOperations.DeprecatedPwritev. -func (imsf *idMapSeqFile) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { +// Write implements fs.FileOperations.Write. +func (imsf *idMapSeqFile) Write(ctx context.Context, _ *fs.File, src usermem.IOSequence, offset int64) (int64, error) { // "In addition, the number of bytes written to the file must be less than // the system page size, and the write must be performed at the start of // the file ..." - user_namespaces(7) diff --git a/pkg/sentry/fs/proc/uptime.go b/pkg/sentry/fs/proc/uptime.go index 80c7ce0b4..40d0fd1fd 100644 --- a/pkg/sentry/fs/proc/uptime.go +++ b/pkg/sentry/fs/proc/uptime.go @@ -18,42 +18,64 @@ import ( "fmt" "io" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // uptime is a file containing the system uptime. // // +stateify savable type uptime struct { - ramfs.Entry + fsutil.SimpleFileInode // The "start time" of the sandbox. startTime ktime.Time } // newUptime returns a new uptime file. -func (p *proc) newUptime(ctx context.Context, msrc *fs.MountSource) *fs.Inode { +func newUptime(ctx context.Context, msrc *fs.MountSource) *fs.Inode { u := &uptime{ - startTime: ktime.NowFromContext(ctx), + SimpleFileInode: *fsutil.NewSimpleFileInode(ctx, fs.RootOwner, fs.FilePermsFromMode(0444), linux.PROC_SUPER_MAGIC), + startTime: ktime.NowFromContext(ctx), } - u.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0444)) - return newFile(u, msrc, fs.SpecialFile, nil) + return newProcInode(u, msrc, fs.SpecialFile, nil) } -// DeprecatedPreadv reads the current uptime. -func (u *uptime) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { +// GetFile implements fs.InodeOperations.GetFile. +func (u *uptime) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &uptimeFile{startTime: u.startTime}), nil +} + +// +stateify savable +type uptimeFile struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + + startTime ktime.Time +} + +// Read implements fs.FileOperations.Read. +func (f *uptimeFile) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { if offset < 0 { return 0, syserror.EINVAL } now := ktime.NowFromContext(ctx) // Pretend that we've spent zero time sleeping (second number). - s := []byte(fmt.Sprintf("%.2f 0.00\n", now.Sub(u.startTime).Seconds())) + s := []byte(fmt.Sprintf("%.2f 0.00\n", now.Sub(f.startTime).Seconds())) if offset >= int64(len(s)) { return 0, io.EOF } diff --git a/pkg/sentry/fs/ramfs/BUILD b/pkg/sentry/fs/ramfs/BUILD index a93ad6240..a476c9cce 100644 --- a/pkg/sentry/fs/ramfs/BUILD +++ b/pkg/sentry/fs/ramfs/BUILD @@ -6,8 +6,6 @@ go_library( name = "ramfs", srcs = [ "dir.go", - "file.go", - "ramfs.go", "socket.go", "symlink.go", "tree.go", @@ -15,14 +13,12 @@ go_library( importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs", visibility = ["//pkg/sentry:internal"], deps = [ - "//pkg/secio", + "//pkg/abi/linux", "//pkg/sentry/context", "//pkg/sentry/fs", "//pkg/sentry/fs/anon", "//pkg/sentry/fs/fsutil", "//pkg/sentry/kernel/time", - "//pkg/sentry/memmap", - "//pkg/sentry/safemem", "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usermem", "//pkg/syserror", diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index 0a911b155..729f37694 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -18,10 +18,12 @@ import ( "sync" "syscall" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" ) @@ -47,7 +49,17 @@ type CreateOps struct { // // +stateify savable type Dir struct { - Entry + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeIsDirTruncate `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes + fsutil.InodeSimpleExtendedAttributes // CreateOps may be provided. // @@ -64,17 +76,23 @@ type Dir struct { children map[string]*fs.Inode // dentryMap is a sortedDentryMap containing entries for all children. - // Its entries ar kept up-to-date with d.children. + // Its entries are kept up-to-date with d.children. dentryMap *fs.SortedDentryMap } -// InitDir initializes a directory. -func (d *Dir) InitDir(ctx context.Context, contents map[string]*fs.Inode, owner fs.FileOwner, perms fs.FilePermissions) { - d.InitEntry(ctx, owner, perms) +var _ fs.InodeOperations = (*Dir)(nil) + +// NewDir returns a new Dir with the given contents and attributes. +func NewDir(ctx context.Context, contents map[string]*fs.Inode, owner fs.FileOwner, perms fs.FilePermissions) *Dir { + d := &Dir{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, perms, linux.RAMFS_MAGIC), + } + if contents == nil { contents = make(map[string]*fs.Inode) } d.children = contents + // Build the entries map ourselves, rather than calling addChildLocked, // because it will be faster. entries := make(map[string]fs.DentAttr, len(contents)) @@ -88,6 +106,8 @@ func (d *Dir) InitDir(ctx context.Context, contents map[string]*fs.Inode, owner // Directories have an extra link, corresponding to '.'. d.AddLink() + + return d } // addChildLocked add the child inode, inheriting its reference. @@ -124,17 +144,24 @@ func (d *Dir) FindChild(name string) (*fs.Inode, bool) { return child, ok } +// Children returns the names and DentAttrs of all children. It can be used to +// implement Readdir for types that embed ramfs.Dir. +func (d *Dir) Children() ([]string, map[string]fs.DentAttr) { + d.mu.Lock() + defer d.mu.Unlock() + return d.dentryMap.GetAll() +} + // removeChildLocked attempts to remove an entry from this directory. -// This Entry's mutex must be held. It returns the removed Inode. func (d *Dir) removeChildLocked(ctx context.Context, name string) (*fs.Inode, error) { inode, ok := d.children[name] if !ok { - return nil, ErrNotFound + return nil, syserror.EACCES } delete(d.children, name) d.dentryMap.Remove(name) - d.Entry.NotifyModification(ctx) + d.NotifyModification(ctx) // If the child was a subdirectory, then we must decrement this dir's // link count which was the child's ".." directory entry. @@ -143,7 +170,7 @@ func (d *Dir) removeChildLocked(ctx context.Context, name string) (*fs.Inode, er } // Update ctime. - inode.NotifyStatusChange(ctx) + inode.InodeOperations.NotifyStatusChange(ctx) // Given we're now removing this inode to the directory we must also // decrease its link count. Similarly it is increased in addChildLocked. @@ -152,8 +179,8 @@ func (d *Dir) removeChildLocked(ctx context.Context, name string) (*fs.Inode, er return inode, nil } -// RemoveEntry attempts to remove an entry from this directory. -func (d *Dir) RemoveEntry(ctx context.Context, name string) error { +// Remove removes the named non-directory. +func (d *Dir) Remove(ctx context.Context, _ *fs.Inode, name string) error { d.mu.Lock() defer d.mu.Unlock() inode, err := d.removeChildLocked(ctx, name) @@ -166,27 +193,23 @@ func (d *Dir) RemoveEntry(ctx context.Context, name string) error { return nil } -// Remove removes the named non-directory. -func (d *Dir) Remove(ctx context.Context, dir *fs.Inode, name string) error { - return d.RemoveEntry(ctx, name) -} - // RemoveDirectory removes the named directory. -func (d *Dir) RemoveDirectory(ctx context.Context, dir *fs.Inode, name string) error { +func (d *Dir) RemoveDirectory(ctx context.Context, _ *fs.Inode, name string) error { d.mu.Lock() defer d.mu.Unlock() - n, err := d.walkLocked(ctx, name) + // Get the child and make sure it is not empty. + childInode, err := d.walkLocked(ctx, name) if err != nil { return err } - dirCtx := &fs.DirCtx{} - if _, err := n.HandleOps().DeprecatedReaddir(ctx, dirCtx, 0); err != nil { + if ok, err := hasChildren(ctx, childInode); err != nil { return err + } else if ok { + return syserror.ENOTEMPTY } - if len(dirCtx.DentAttrs()) > 0 { - return ErrNotEmpty - } + + // Child was empty. Proceed with removal. inode, err := d.removeChildLocked(ctx, name) if err != nil { return err @@ -195,11 +218,11 @@ func (d *Dir) RemoveDirectory(ctx context.Context, dir *fs.Inode, name string) e // Remove our reference on the inode. inode.DecRef() - return err + return nil } // Lookup loads an inode at p into a Dirent. -func (d *Dir) Lookup(ctx context.Context, dir *fs.Inode, p string) (*fs.Dirent, error) { +func (d *Dir) Lookup(ctx context.Context, _ *fs.Inode, p string) (*fs.Dirent, error) { d.mu.Lock() defer d.mu.Unlock() @@ -214,9 +237,9 @@ func (d *Dir) Lookup(ctx context.Context, dir *fs.Inode, p string) (*fs.Dirent, return fs.NewDirent(inode, p), nil } -// walkLocked must be called with this Entry's mutex held. +// walkLocked must be called with d.mu held. func (d *Dir) walkLocked(ctx context.Context, p string) (*fs.Inode, error) { - d.Entry.NotifyAccess(ctx) + d.NotifyAccess(ctx) // Lookup a child node. if inode, ok := d.children[p]; ok { @@ -244,7 +267,7 @@ func (d *Dir) createInodeOperationsCommon(ctx context.Context, name string, make } d.addChildLocked(name, inode) - d.Entry.NotifyModification(ctx) + d.NotifyModification(ctx) return inode, nil } @@ -252,7 +275,7 @@ func (d *Dir) createInodeOperationsCommon(ctx context.Context, name string, make // Create creates a new Inode with the given name and returns its File. func (d *Dir) Create(ctx context.Context, dir *fs.Inode, name string, flags fs.FileFlags, perms fs.FilePermissions) (*fs.File, error) { if d.CreateOps == nil || d.CreateOps.NewFile == nil { - return nil, ErrDenied + return nil, syserror.EACCES } inode, err := d.createInodeOperationsCommon(ctx, name, func() (*fs.Inode, error) { @@ -274,7 +297,7 @@ func (d *Dir) Create(ctx context.Context, dir *fs.Inode, name string, flags fs.F // CreateLink returns a new link. func (d *Dir) CreateLink(ctx context.Context, dir *fs.Inode, oldname, newname string) error { if d.CreateOps == nil || d.CreateOps.NewSymlink == nil { - return ErrDenied + return syserror.EACCES } _, err := d.createInodeOperationsCommon(ctx, newname, func() (*fs.Inode, error) { return d.NewSymlink(ctx, dir, oldname) @@ -292,10 +315,10 @@ func (d *Dir) CreateHardLink(ctx context.Context, dir *fs.Inode, target *fs.Inod // The link count will be incremented in addChildLocked. d.addChildLocked(name, target) - d.Entry.NotifyModification(ctx) + d.NotifyModification(ctx) // Update ctime. - target.NotifyStatusChange(ctx) + target.InodeOperations.NotifyStatusChange(ctx) return nil } @@ -303,7 +326,7 @@ func (d *Dir) CreateHardLink(ctx context.Context, dir *fs.Inode, target *fs.Inod // CreateDirectory returns a new subdirectory. func (d *Dir) CreateDirectory(ctx context.Context, dir *fs.Inode, name string, perms fs.FilePermissions) error { if d.CreateOps == nil || d.CreateOps.NewDir == nil { - return ErrDenied + return syserror.EACCES } _, err := d.createInodeOperationsCommon(ctx, name, func() (*fs.Inode, error) { return d.NewDir(ctx, dir, perms) @@ -316,7 +339,7 @@ func (d *Dir) CreateDirectory(ctx context.Context, dir *fs.Inode, name string, p // Bind implements fs.InodeOperations.Bind. func (d *Dir) Bind(ctx context.Context, dir *fs.Inode, name string, ep transport.BoundEndpoint, perms fs.FilePermissions) (*fs.Dirent, error) { if d.CreateOps == nil || d.CreateOps.NewBoundEndpoint == nil { - return nil, ErrDenied + return nil, syserror.EACCES } inode, err := d.createInodeOperationsCommon(ctx, name, func() (*fs.Inode, error) { return d.NewBoundEndpoint(ctx, dir, ep, perms) @@ -335,7 +358,7 @@ func (d *Dir) Bind(ctx context.Context, dir *fs.Inode, name string, ep transport // CreateFifo implements fs.InodeOperations.CreateFifo. func (d *Dir) CreateFifo(ctx context.Context, dir *fs.Inode, name string, perms fs.FilePermissions) error { if d.CreateOps == nil || d.CreateOps.NewFifo == nil { - return ErrDenied + return syserror.EACCES } _, err := d.createInodeOperationsCommon(ctx, name, func() (*fs.Inode, error) { return d.NewFifo(ctx, dir, perms) @@ -343,29 +366,125 @@ func (d *Dir) CreateFifo(ctx context.Context, dir *fs.Inode, name string, perms return err } -func (d *Dir) readdirLocked(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { - // Serialize the entries in dentryMap. - n, err := fs.GenericReaddir(dirCtx, d.dentryMap) +// GetFile implements fs.InodeOperations.GetFile. +func (d *Dir) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + flags.Pread = true + return fs.NewFile(ctx, dirent, flags, &dirFileOperations{dir: d}), nil +} - // Touch the access time. - d.Entry.NotifyAccess(ctx) +// Rename implements fs.InodeOperations.Rename. +func (*Dir) Rename(ctx context.Context, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error { + return Rename(ctx, oldParent.InodeOperations, oldName, newParent.InodeOperations, newName) +} +// dirFileOperations implements fs.FileOperations for a ramfs directory. +// +// +stateify savable +type dirFileOperations struct { + fsutil.DirFileOperations `state:"nosave"` + + // dirCursor contains the name of the last directory entry that was + // serialized. + dirCursor string + + // dir is the ramfs dir that this file corresponds to. + dir *Dir +} + +var _ fs.FileOperations = (*dirFileOperations)(nil) + +// Seek implements fs.FileOperations.Seek. +func (dfo *dirFileOperations) Seek(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64) (int64, error) { + return fsutil.SeekWithDirCursor(ctx, file, whence, offset, &dfo.dirCursor) +} + +// IterateDir implements DirIterator.IterateDir. +func (dfo *dirFileOperations) IterateDir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { + dfo.dir.mu.Lock() + defer dfo.dir.mu.Unlock() + + n, err := fs.GenericReaddir(dirCtx, dfo.dir.dentryMap) return offset + n, err } -// DeprecatedReaddir emits the entries contained in this directory. -func (d *Dir) DeprecatedReaddir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { - d.mu.Lock() - defer d.mu.Unlock() - return d.readdirLocked(ctx, dirCtx, offset) +// Readdir implements FileOperations.Readdir. +func (dfo *dirFileOperations) Readdir(ctx context.Context, file *fs.File, serializer fs.DentrySerializer) (int64, error) { + root := fs.RootFromContext(ctx) + defer root.DecRef() + dirCtx := &fs.DirCtx{ + Serializer: serializer, + DirCursor: &dfo.dirCursor, + } + dfo.dir.mu.Lock() + dfo.dir.InodeSimpleAttributes.Unstable.AccessTime = ktime.NowFromContext(ctx) + dfo.dir.mu.Unlock() + return fs.DirentReaddir(ctx, file.Dirent, dfo, root, dirCtx, file.Offset()) } -// DeprecatedPreadv always returns ErrIsDirectory -func (*Dir) DeprecatedPreadv(context.Context, usermem.IOSequence, int64) (int64, error) { - return 0, ErrIsDirectory +// hasChildren is a helper method that determines whether an arbitrary inode +// (not necessarily ramfs) has any children. +func hasChildren(ctx context.Context, inode *fs.Inode) (bool, error) { + // Take an extra ref on inode which will be given to the dirent and + // dropped when that dirent is destroyed. + inode.IncRef() + d := fs.NewTransientDirent(inode) + defer d.DecRef() + + file, err := inode.GetFile(ctx, d, fs.FileFlags{Read: true}) + if err != nil { + return false, err + } + defer file.DecRef() + + ser := &fs.CollectEntriesSerializer{} + if err := file.Readdir(ctx, ser); err != nil { + return false, err + } + // We will always write "." and "..", so ignore those two. + if ser.Written() > 2 { + return true, nil + } + return false, nil } -// DeprecatedPwritev always returns ErrIsDirectory -func (*Dir) DeprecatedPwritev(context.Context, usermem.IOSequence, int64) (int64, error) { - return 0, ErrIsDirectory +// Rename renames from a *ramfs.Dir to another *ramfs.Dir. +func Rename(ctx context.Context, oldParent fs.InodeOperations, oldName string, newParent fs.InodeOperations, newName string) error { + op, ok := oldParent.(*Dir) + if !ok { + return syserror.EXDEV + } + np, ok := newParent.(*Dir) + if !ok { + return syserror.EXDEV + } + + np.mu.Lock() + defer np.mu.Unlock() + + // Check whether the ramfs entry to be replaced is a non-empty directory. + if replaced, ok := np.children[newName]; ok { + if fs.IsDir(replaced.StableAttr) { + if ok, err := hasChildren(ctx, replaced); err != nil { + return err + } else if ok { + return syserror.ENOTEMPTY + } + } + } + + // Be careful, we may have already grabbed this mutex above. + if op != np { + op.mu.Lock() + defer op.mu.Unlock() + } + + // Do the swap. + n := op.children[oldName] + op.removeChildLocked(ctx, oldName) + np.addChildLocked(newName, n) + + // Update ctime. + n.InodeOperations.NotifyStatusChange(ctx) + + return nil } diff --git a/pkg/sentry/fs/ramfs/file.go b/pkg/sentry/fs/ramfs/file.go deleted file mode 100644 index b7fc98ffc..000000000 --- a/pkg/sentry/fs/ramfs/file.go +++ /dev/null @@ -1,150 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package ramfs - -import ( - "io" - "sync" - - "gvisor.googlesource.com/gvisor/pkg/secio" - "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/syserror" -) - -// File represents a unique file. It uses a simple byte slice as storage, and -// thus should only be used for small files. -// -// A File is not mappable. -// -// +stateify savable -type File struct { - Entry - - // mu protects the fields below. - mu sync.Mutex `state:"nosave"` - - // data tracks backing data for the file. - data []byte -} - -// InitFile initializes a file. -func (f *File) InitFile(ctx context.Context, owner fs.FileOwner, perms fs.FilePermissions) { - f.InitEntry(ctx, owner, perms) -} - -// UnstableAttr returns unstable attributes of this ramfs file. -func (f *File) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - f.mu.Lock() - defer f.mu.Unlock() - - uattr, _ := f.Entry.UnstableAttr(ctx, inode) - uattr.Size = int64(len(f.data)) - uattr.Usage = f.usageLocked() - - return uattr, nil -} - -// usageLocked returns the disk usage. Caller must hold f.mu. -func (f *File) usageLocked() int64 { - return int64(len(f.data)) -} - -// Append appends the given data. This is for internal use. -func (f *File) Append(data []byte) { - f.mu.Lock() - defer f.mu.Unlock() - f.data = append(f.data, data...) -} - -// Truncate truncates this node. -func (f *File) Truncate(ctx context.Context, inode *fs.Inode, l int64) error { - f.mu.Lock() - defer f.mu.Unlock() - if l < int64(len(f.data)) { - // Remove excess bytes. - f.data = f.data[:l] - return nil - } else if l > int64(len(f.data)) { - // Create a new slice with size l, and copy f.data into it. - d := make([]byte, l) - copy(d, f.data) - f.data = d - } - f.Entry.NotifyModification(ctx) - return nil -} - -// ReadAt implements io.ReaderAt. -func (f *File) ReadAt(data []byte, offset int64) (int, error) { - if offset < 0 { - return 0, ErrInvalidOp - } - if offset >= int64(len(f.data)) { - return 0, io.EOF - } - n := copy(data, f.data[offset:]) - // Did we read past the end? - if offset+int64(len(data)) >= int64(len(f.data)) { - return n, io.EOF - } - return n, nil -} - -// DeprecatedPreadv reads into a collection of slices from a given offset. -func (f *File) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - f.mu.Lock() - defer f.mu.Unlock() - if offset >= int64(len(f.data)) { - return 0, io.EOF - } - n, err := dst.CopyOut(ctx, f.data[offset:]) - if n > 0 { - f.Entry.NotifyAccess(ctx) - } - return int64(n), err -} - -// WriteAt implements io.WriterAt. -func (f *File) WriteAt(data []byte, offset int64) (int, error) { - if offset < 0 { - return 0, ErrInvalidOp - } - newLen := offset + int64(len(data)) - if newLen < 0 { - // Overflow. - return 0, syserror.EINVAL - } - if newLen > int64(len(f.data)) { - // Copy f.data into new slice with expanded length. - d := make([]byte, newLen) - copy(d, f.data) - f.data = d - } - return copy(f.data[offset:], data), nil -} - -// DeprecatedPwritev writes from a collection of slices at a given offset. -func (f *File) DeprecatedPwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { - f.mu.Lock() - defer f.mu.Unlock() - n, err := src.CopyInTo(ctx, safemem.FromIOWriter{secio.NewOffsetWriter(f, offset)}) - if n > 0 { - f.Entry.NotifyModification(ctx) - } - return n, err -} diff --git a/pkg/sentry/fs/ramfs/ramfs.go b/pkg/sentry/fs/ramfs/ramfs.go deleted file mode 100644 index d77688a34..000000000 --- a/pkg/sentry/fs/ramfs/ramfs.go +++ /dev/null @@ -1,441 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package ramfs implements an in-memory file system that can be associated with -// any device. -package ramfs - -import ( - "errors" - "sync" - "syscall" - - "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" - ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" - "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" - "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/syserror" - "gvisor.googlesource.com/gvisor/pkg/waiter" -) - -var ( - // ErrInvalidOp indicates the operation is not valid. - ErrInvalidOp = errors.New("invalid operation") - - // ErrDenied indicates the operation was denied. - ErrDenied = errors.New("operation denied") - - // ErrNotFound indicates that a node was not found on a walk. - ErrNotFound = errors.New("node not found") - - // ErrCrossDevice indicates a cross-device link or rename. - ErrCrossDevice = errors.New("can't link across filesystems") - - // ErrIsDirectory indicates that the operation failed because - // the node is a directory. - ErrIsDirectory = errors.New("is a directory") - - // ErrNotDirectory indicates that the operation failed because - // the node is a not directory. - ErrNotDirectory = errors.New("not a directory") - - // ErrNotEmpty indicates that the operation failed because the - // directory is not empty. - ErrNotEmpty = errors.New("directory not empty") -) - -// Entry represents common internal state for file and directory nodes. -// This may be used by other packages to easily create ramfs files. -// -// +stateify savable -type Entry struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.NoMappable `state:"nosave"` - fsutil.NoopWriteOut `state:"nosave"` - fsutil.InodeNotSocket `state:"nosave"` - - // mu protects the fields below. - mu sync.Mutex `state:"nosave"` - - // unstable is unstable attributes. - unstable fs.UnstableAttr - - // xattrs are the extended attributes of the Entry. - xattrs map[string][]byte -} - -// InitEntry initializes an entry. -func (e *Entry) InitEntry(ctx context.Context, owner fs.FileOwner, p fs.FilePermissions) { - e.InitEntryWithAttr(ctx, fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: owner, - Perms: p, - // Always start unlinked. - Links: 0, - })) -} - -// InitEntryWithAttr initializes an entry with a complete set of attributes. -func (e *Entry) InitEntryWithAttr(ctx context.Context, uattr fs.UnstableAttr) { - e.unstable = uattr - e.xattrs = make(map[string][]byte) -} - -// UnstableAttr implements fs.InodeOperations.UnstableAttr. -func (e *Entry) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - e.mu.Lock() - attr := e.unstable - e.mu.Unlock() - return attr, nil -} - -// Check implements fs.InodeOperations.Check. -func (*Entry) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { - return fs.ContextCanAccessFile(ctx, inode, p) -} - -// Getxattr implements fs.InodeOperations.Getxattr. -func (e *Entry) Getxattr(inode *fs.Inode, name string) ([]byte, error) { - // Hot path. Avoid defers. - e.mu.Lock() - value, ok := e.xattrs[name] - e.mu.Unlock() - if ok { - return value, nil - } - return nil, syserror.ENOATTR -} - -// Setxattr implements fs.InodeOperations.Setxattr. -func (e *Entry) Setxattr(inode *fs.Inode, name string, value []byte) error { - e.mu.Lock() - e.xattrs[name] = value - e.mu.Unlock() - return nil -} - -// Listxattr implements fs.InodeOperations.Listxattr. -func (e *Entry) Listxattr(inode *fs.Inode) (map[string]struct{}, error) { - e.mu.Lock() - names := make(map[string]struct{}, len(e.xattrs)) - for name := range e.xattrs { - names[name] = struct{}{} - } - e.mu.Unlock() - return names, nil -} - -// GetFile returns a fs.File backed by the dirent argument and flags. -func (*Entry) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { - return fsutil.NewHandle(ctx, d, flags, d.Inode.HandleOps()), nil -} - -// SetPermissions always sets the permissions. -func (e *Entry) SetPermissions(ctx context.Context, inode *fs.Inode, p fs.FilePermissions) bool { - e.mu.Lock() - e.unstable.Perms = p - e.unstable.StatusChangeTime = ktime.NowFromContext(ctx) - e.mu.Unlock() - return true -} - -// SetOwner always sets ownership. -func (e *Entry) SetOwner(ctx context.Context, inode *fs.Inode, owner fs.FileOwner) error { - e.mu.Lock() - if owner.UID.Ok() { - e.unstable.Owner.UID = owner.UID - } - if owner.GID.Ok() { - e.unstable.Owner.GID = owner.GID - } - e.mu.Unlock() - return nil -} - -// SetTimestamps sets the timestamps. -func (e *Entry) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error { - if ts.ATimeOmit && ts.MTimeOmit { - return nil - } - - e.mu.Lock() - now := ktime.NowFromContext(ctx) - if !ts.ATimeOmit { - if ts.ATimeSetSystemTime { - e.unstable.AccessTime = now - } else { - e.unstable.AccessTime = ts.ATime - } - } - if !ts.MTimeOmit { - if ts.MTimeSetSystemTime { - e.unstable.ModificationTime = now - } else { - e.unstable.ModificationTime = ts.MTime - } - } - e.unstable.StatusChangeTime = now - e.mu.Unlock() - return nil -} - -// NotifyStatusChange updates the status change time (ctime). -func (e *Entry) NotifyStatusChange(ctx context.Context) { - e.mu.Lock() - e.unstable.StatusChangeTime = ktime.NowFromContext(ctx) - e.mu.Unlock() -} - -// StatusChangeTime returns the last status change time for this node. -func (e *Entry) StatusChangeTime() ktime.Time { - e.mu.Lock() - t := e.unstable.StatusChangeTime - e.mu.Unlock() - return t -} - -// NotifyModification updates the modification time and the status change time. -func (e *Entry) NotifyModification(ctx context.Context) { - e.mu.Lock() - now := ktime.NowFromContext(ctx) - e.unstable.ModificationTime = now - e.unstable.StatusChangeTime = now - e.mu.Unlock() -} - -// ModificationTime returns the last modification time for this node. -func (e *Entry) ModificationTime() ktime.Time { - e.mu.Lock() - t := e.unstable.ModificationTime - e.mu.Unlock() - return t -} - -// NotifyAccess updates the access time. -func (e *Entry) NotifyAccess(ctx context.Context) { - e.mu.Lock() - now := ktime.NowFromContext(ctx) - e.unstable.AccessTime = now - e.mu.Unlock() -} - -// AccessTime returns the last access time for this node. -func (e *Entry) AccessTime() ktime.Time { - e.mu.Lock() - t := e.unstable.AccessTime - e.mu.Unlock() - return t -} - -// Permissions returns permissions on this entry. -func (e *Entry) Permissions() fs.FilePermissions { - e.mu.Lock() - p := e.unstable.Perms - e.mu.Unlock() - return p -} - -// Lookup is not supported by default. -func (*Entry) Lookup(context.Context, *fs.Inode, string) (*fs.Dirent, error) { - return nil, ErrInvalidOp -} - -// Create is not supported by default. -func (*Entry) Create(context.Context, *fs.Inode, string, fs.FileFlags, fs.FilePermissions) (*fs.File, error) { - return nil, ErrInvalidOp -} - -// CreateLink is not supported by default. -func (*Entry) CreateLink(context.Context, *fs.Inode, string, string) error { - return ErrInvalidOp -} - -// CreateHardLink is not supported by default. -func (*Entry) CreateHardLink(context.Context, *fs.Inode, *fs.Inode, string) error { - return ErrInvalidOp -} - -// IsVirtual returns true. -func (*Entry) IsVirtual() bool { - return true -} - -// CreateDirectory is not supported by default. -func (*Entry) CreateDirectory(context.Context, *fs.Inode, string, fs.FilePermissions) error { - return ErrInvalidOp -} - -// Bind is not supported by default. -func (*Entry) Bind(context.Context, *fs.Inode, string, transport.BoundEndpoint, fs.FilePermissions) (*fs.Dirent, error) { - return nil, ErrInvalidOp -} - -// CreateFifo implements fs.InodeOperations.CreateFifo. CreateFifo is not supported by -// default. -func (*Entry) CreateFifo(context.Context, *fs.Inode, string, fs.FilePermissions) error { - return ErrInvalidOp -} - -// Remove is not supported by default. -func (*Entry) Remove(context.Context, *fs.Inode, string) error { - return ErrInvalidOp -} - -// RemoveDirectory is not supported by default. -func (*Entry) RemoveDirectory(context.Context, *fs.Inode, string) error { - return ErrInvalidOp -} - -// StatFS always returns ENOSYS. -func (*Entry) StatFS(context.Context) (fs.Info, error) { - return fs.Info{}, syscall.ENOSYS -} - -// Rename implements fs.InodeOperations.Rename. -func (e *Entry) Rename(ctx context.Context, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error { - return Rename(ctx, oldParent.InodeOperations, oldName, newParent.InodeOperations, newName) -} - -// Rename renames from a *ramfs.Dir to another *ramfs.Dir. -func Rename(ctx context.Context, oldParent fs.InodeOperations, oldName string, newParent fs.InodeOperations, newName string) error { - op, ok := oldParent.(*Dir) - if !ok { - return ErrCrossDevice - } - np, ok := newParent.(*Dir) - if !ok { - return ErrCrossDevice - } - - np.mu.Lock() - defer np.mu.Unlock() - - // Check whether the ramfs entry to be replaced is a non-empty directory. - if replaced, ok := np.children[newName]; ok { - if fs.IsDir(replaced.StableAttr) { - // FIXME: simplify by pinning children of ramfs-backed directories - // in the Dirent tree: this allows us to generalize ramfs operations without - // relying on an implementation of Readdir (which may do anything, like require - // that the file be open ... which would be reasonable). - dirCtx := &fs.DirCtx{} - _, err := replaced.HandleOps().DeprecatedReaddir(ctx, dirCtx, 0) - if err != nil { - return err - } - attrs := dirCtx.DentAttrs() - - // ramfs-backed directories should not contain "." and "..", but we do this - // just in case. - delete(attrs, ".") - delete(attrs, "..") - - // If the directory to be replaced is not empty, reject the rename. - if len(attrs) != 0 { - return ErrNotEmpty - } - } - } - - // Be careful, we may have already grabbed this mutex above. - if op != np { - op.mu.Lock() - defer op.mu.Unlock() - } - - // Do the swap. - n := op.children[oldName] - op.removeChildLocked(ctx, oldName) - np.addChildLocked(newName, n) - - // Update ctime. - n.NotifyStatusChange(ctx) - - return nil -} - -// Truncate is not supported by default. -func (*Entry) Truncate(context.Context, *fs.Inode, int64) error { - return ErrInvalidOp -} - -// Readlink always returns ENOLINK. -func (*Entry) Readlink(context.Context, *fs.Inode) (string, error) { - return "", syscall.ENOLINK -} - -// Getlink always returns ENOLINK. -func (*Entry) Getlink(context.Context, *fs.Inode) (*fs.Dirent, error) { - return nil, syscall.ENOLINK -} - -// Release is a no-op. -func (e *Entry) Release(context.Context) {} - -// AddLink implements InodeOperationss.AddLink. -func (e *Entry) AddLink() { - e.mu.Lock() - e.unstable.Links++ - e.mu.Unlock() -} - -// DropLink implements InodeOperationss.DropLink. -func (e *Entry) DropLink() { - e.mu.Lock() - e.unstable.Links-- - e.mu.Unlock() -} - -// DeprecatedReaddir is not supported by default. -func (*Entry) DeprecatedReaddir(context.Context, *fs.DirCtx, int) (int, error) { - return 0, ErrNotDirectory -} - -// DeprecatedPreadv always returns ErrInvalidOp. -func (*Entry) DeprecatedPreadv(context.Context, usermem.IOSequence, int64) (int64, error) { - return 0, ErrInvalidOp -} - -// DeprecatedPwritev always returns ErrInvalidOp. -func (*Entry) DeprecatedPwritev(context.Context, usermem.IOSequence, int64) (int64, error) { - return 0, ErrInvalidOp -} - -// DeprecatedFsync is a noop. -func (*Entry) DeprecatedFsync() error { - // Ignore, this is in memory. - return nil -} - -// DeprecatedFlush always returns nil. -func (*Entry) DeprecatedFlush() error { - return nil -} - -// DeprecatedMappable implements fs.InodeOperations.DeprecatedMappable. -func (*Entry) DeprecatedMappable(context.Context, *fs.Inode) (memmap.Mappable, bool) { - return nil, false -} - -func init() { - // Register ramfs errors. - syserror.AddErrorTranslation(ErrInvalidOp, syscall.EINVAL) - syserror.AddErrorTranslation(ErrDenied, syscall.EACCES) - syserror.AddErrorTranslation(ErrNotFound, syscall.ENOENT) - syserror.AddErrorTranslation(ErrCrossDevice, syscall.EXDEV) - syserror.AddErrorTranslation(ErrIsDirectory, syscall.EISDIR) - syserror.AddErrorTranslation(ErrNotDirectory, syscall.ENOTDIR) - syserror.AddErrorTranslation(ErrNotEmpty, syscall.ENOTEMPTY) -} diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go index 8c81478c8..2c1295897 100644 --- a/pkg/sentry/fs/ramfs/socket.go +++ b/pkg/sentry/fs/ramfs/socket.go @@ -15,25 +15,42 @@ package ramfs import ( + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // Socket represents a socket. // // +stateify savable type Socket struct { - Entry + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeNotTruncatable `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes + fsutil.InodeSimpleExtendedAttributes // ep is the bound endpoint. ep transport.BoundEndpoint } -// InitSocket initializes a socket. -func (s *Socket) InitSocket(ctx context.Context, ep transport.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions) { - s.InitEntry(ctx, owner, perms) - s.ep = ep +var _ fs.InodeOperations = (*Socket)(nil) + +// NewSocket returns a new Socket. +func NewSocket(ctx context.Context, ep transport.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions) *Socket { + return &Socket{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, perms, linux.SOCKFS_MAGIC), + ep: ep, + } } // BoundEndpoint returns the socket data. @@ -42,3 +59,24 @@ func (s *Socket) BoundEndpoint(*fs.Inode, string) transport.BoundEndpoint { // care about the path argument. return s.ep } + +// GetFile implements fs.FileOperations.GetFile. +func (s *Socket) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &socketFileOperations{}), nil +} + +// +stateify savable +type socketFileOperations struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoRead `state:"nosave"` + fsutil.FileNoSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` +} + +var _ fs.FileOperations = (*socketFileOperations)(nil) diff --git a/pkg/sentry/fs/ramfs/symlink.go b/pkg/sentry/fs/ramfs/symlink.go index a21fac2c7..47dae380b 100644 --- a/pkg/sentry/fs/ramfs/symlink.go +++ b/pkg/sentry/fs/ramfs/symlink.go @@ -15,44 +15,55 @@ package ramfs import ( - "sync" - + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/waiter" ) // Symlink represents a symlink. // // +stateify savable type Symlink struct { - Entry + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotTruncatable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` - mu sync.Mutex `state:"nosave"` + fsutil.InodeSimpleAttributes + fsutil.InodeSimpleExtendedAttributes // Target is the symlink target. Target string } -// InitSymlink initializes a symlink, pointing to the given target. -// A symlink is assumed to always have permissions 0777. -func (s *Symlink) InitSymlink(ctx context.Context, owner fs.FileOwner, target string) { - s.InitEntry(ctx, owner, fs.FilePermsFromMode(0777)) - s.Target = target +var _ fs.InodeOperations = (*Symlink)(nil) + +// NewSymlink returns a new Symlink. +func NewSymlink(ctx context.Context, owner fs.FileOwner, target string) *Symlink { + // A symlink is assumed to always have permissions 0777. + return &Symlink{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, owner, fs.FilePermsFromMode(0777), linux.RAMFS_MAGIC), + Target: target, + } } // UnstableAttr returns all attributes of this ramfs symlink. func (s *Symlink) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - uattr, _ := s.Entry.UnstableAttr(ctx, inode) + uattr, err := s.InodeSimpleAttributes.UnstableAttr(ctx, inode) + if err != nil { + return fs.UnstableAttr{}, err + } uattr.Size = int64(len(s.Target)) uattr.Usage = uattr.Size return uattr, nil } -// Check implements InodeOperations.Check. -func (s *Symlink) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { - return fs.ContextCanAccessFile(ctx, inode, p) -} - // SetPermissions on a symlink is always rejected. func (s *Symlink) SetPermissions(context.Context, *fs.Inode, fs.FilePermissions) bool { return false @@ -60,10 +71,7 @@ func (s *Symlink) SetPermissions(context.Context, *fs.Inode, fs.FilePermissions) // Readlink reads the symlink value. func (s *Symlink) Readlink(ctx context.Context, _ *fs.Inode) (string, error) { - s.mu.Lock() - defer s.mu.Unlock() - - s.Entry.NotifyAccess(ctx) + s.NotifyAccess(ctx) return s.Target, nil } @@ -72,3 +80,24 @@ func (s *Symlink) Readlink(ctx context.Context, _ *fs.Inode) (string, error) { func (*Symlink) Getlink(context.Context, *fs.Inode) (*fs.Dirent, error) { return nil, fs.ErrResolveViaReadlink } + +// GetFile implements fs.FileOperations.GetFile. +func (s *Symlink) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return fs.NewFile(ctx, dirent, flags, &symlinkFileOperations{}), nil +} + +// +stateify savable +type symlinkFileOperations struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoRead `state:"nosave"` + fsutil.FileNoSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` +} + +var _ fs.FileOperations = (*symlinkFileOperations)(nil) diff --git a/pkg/sentry/fs/ramfs/test/BUILD b/pkg/sentry/fs/ramfs/test/BUILD deleted file mode 100644 index 187eac49d..000000000 --- a/pkg/sentry/fs/ramfs/test/BUILD +++ /dev/null @@ -1,16 +0,0 @@ -package(licenses = ["notice"]) # Apache 2.0 - -load("//tools/go_stateify:defs.bzl", "go_library") - -go_library( - name = "test", - testonly = 1, - srcs = ["test.go"], - importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs/test", - visibility = ["//pkg/sentry:internal"], - deps = [ - "//pkg/sentry/context", - "//pkg/sentry/fs", - "//pkg/sentry/fs/ramfs", - ], -) diff --git a/pkg/sentry/fs/ramfs/test/test.go b/pkg/sentry/fs/ramfs/test/test.go deleted file mode 100644 index 11bff7729..000000000 --- a/pkg/sentry/fs/ramfs/test/test.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// Package test provides a simple ramfs-based filesystem for use in testing. -package test - -import ( - "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" -) - -// Dir is a simple ramfs.Dir that supports save/restore as-is. -type Dir struct { - ramfs.Dir -} - -// NewDir returns a simple ramfs directory with the passed contents. -func NewDir(ctx context.Context, contents map[string]*fs.Inode, perms fs.FilePermissions) *Dir { - d := &Dir{} - d.InitDir(ctx, contents, fs.RootOwner, perms) - return d -} - -// File is a simple ramfs.File that supports save/restore as-is. -type File struct { - ramfs.File -} - -// NewFile returns a simple ramfs File. -func NewFile(ctx context.Context, perms fs.FilePermissions) *File { - f := &File{} - f.InitFile(ctx, fs.RootOwner, perms) - return f -} diff --git a/pkg/sentry/fs/ramfs/tree.go b/pkg/sentry/fs/ramfs/tree.go index 29a70f698..f6d5ffdec 100644 --- a/pkg/sentry/fs/ramfs/tree.go +++ b/pkg/sentry/fs/ramfs/tree.go @@ -60,8 +60,7 @@ func makeSubdir(ctx context.Context, msrc *fs.MountSource, root *Dir, subdir str // emptyDir returns an empty *ramfs.Dir that is traversable but not writable. func emptyDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - dir := &Dir{} - dir.InitDir(ctx, make(map[string]*fs.Inode), fs.RootOwner, fs.FilePermsFromMode(0555)) + dir := NewDir(ctx, make(map[string]*fs.Inode), fs.RootOwner, fs.FilePermsFromMode(0555)) return fs.NewInode(dir, msrc, fs.StableAttr{ DeviceID: anon.PseudoDevice.DeviceID(), InodeID: anon.PseudoDevice.NextIno(), diff --git a/pkg/sentry/fs/ramfs/tree_test.go b/pkg/sentry/fs/ramfs/tree_test.go index 54df2143c..8bee9cfc1 100644 --- a/pkg/sentry/fs/ramfs/tree_test.go +++ b/pkg/sentry/fs/ramfs/tree_test.go @@ -22,7 +22,7 @@ import ( ) func TestMakeDirectoryTree(t *testing.T) { - mount := fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}) + mount := fs.NewPseudoMountSource() for _, test := range []struct { name string diff --git a/pkg/sentry/fs/sys/BUILD b/pkg/sentry/fs/sys/BUILD index 5ba23d5da..7de928e16 100644 --- a/pkg/sentry/fs/sys/BUILD +++ b/pkg/sentry/fs/sys/BUILD @@ -13,12 +13,13 @@ go_library( importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs/sys", visibility = ["//pkg/sentry:internal"], deps = [ + "//pkg/abi/linux", "//pkg/sentry/context", "//pkg/sentry/device", "//pkg/sentry/fs", + "//pkg/sentry/fs/fsutil", "//pkg/sentry/fs/ramfs", "//pkg/sentry/kernel", "//pkg/sentry/usermem", - "//pkg/syserror", ], ) diff --git a/pkg/sentry/fs/sys/devices.go b/pkg/sentry/fs/sys/devices.go index e64aa0edc..8b728a4e4 100644 --- a/pkg/sentry/fs/sys/devices.go +++ b/pkg/sentry/fs/sys/devices.go @@ -16,43 +16,50 @@ package sys import ( "fmt" - "io" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" - "gvisor.googlesource.com/gvisor/pkg/syserror" ) // +stateify savable type cpunum struct { - ramfs.Entry + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeNotVirtual `state:"nosave"` + fsutil.InodeNotTruncatable `state:"nosave"` + + fsutil.InodeSimpleAttributes + fsutil.InodeStaticFileGetter + + // k is the system kernel. + k *kernel.Kernel } -func (c *cpunum) DeprecatedPreadv(ctx context.Context, dst usermem.IOSequence, offset int64) (int64, error) { - if offset < 0 { - return 0, syserror.EINVAL - } +var _ fs.InodeOperations = (*cpunum)(nil) +func newPossible(ctx context.Context, msrc *fs.MountSource) *fs.Inode { + var maxCore uint k := kernel.KernelFromContext(ctx) - if k == nil { - return 0, io.EOF + if k != nil { + maxCore = k.ApplicationCores() - 1 } + contents := []byte(fmt.Sprintf("0-%d\n", maxCore)) - str := []byte(fmt.Sprintf("0-%d\n", k.ApplicationCores()-1)) - if offset >= int64(len(str)) { - return 0, io.EOF + c := &cpunum{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, fs.RootOwner, fs.FilePermsFromMode(0444), linux.SYSFS_MAGIC), + InodeStaticFileGetter: fsutil.InodeStaticFileGetter{ + Contents: contents, + }, } - - n, err := dst.CopyOut(ctx, str[offset:]) - return int64(n), err -} - -func newPossible(ctx context.Context, msrc *fs.MountSource) *fs.Inode { - c := &cpunum{} - c.InitEntry(ctx, fs.RootOwner, fs.FilePermsFromMode(0444)) return newFile(c, msrc) } diff --git a/pkg/sentry/fs/sys/fs.go b/pkg/sentry/fs/sys/fs.go index 5ce33f87f..301fef038 100644 --- a/pkg/sentry/fs/sys/fs.go +++ b/pkg/sentry/fs/sys/fs.go @@ -24,6 +24,8 @@ import ( // +stateify savable type filesystem struct{} +var _ fs.Filesystem = (*filesystem)(nil) + func init() { fs.RegisterFilesystem(&filesystem{}) } diff --git a/pkg/sentry/fs/sys/sys.go b/pkg/sentry/fs/sys/sys.go index 7cc1942c7..c5b56fe69 100644 --- a/pkg/sentry/fs/sys/sys.go +++ b/pkg/sentry/fs/sys/sys.go @@ -22,13 +22,6 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" ) -// sys is a root sys node. -// -// +stateify savable -type sys struct { - ramfs.Dir -} - func newFile(node fs.InodeOperations, msrc *fs.MountSource) *fs.Inode { sattr := fs.StableAttr{ DeviceID: sysfsDevice.DeviceID(), @@ -40,8 +33,7 @@ func newFile(node fs.InodeOperations, msrc *fs.MountSource) *fs.Inode { } func newDir(ctx context.Context, msrc *fs.MountSource, contents map[string]*fs.Inode) *fs.Inode { - d := &sys{} - d.InitDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) + d := ramfs.NewDir(ctx, contents, fs.RootOwner, fs.FilePermsFromMode(0555)) return fs.NewInode(d, msrc, fs.StableAttr{ DeviceID: sysfsDevice.DeviceID(), InodeID: sysfsDevice.NextIno(), diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go index 7423e816c..b26466b9d 100644 --- a/pkg/sentry/fs/timerfd/timerfd.go +++ b/pkg/sentry/fs/timerfd/timerfd.go @@ -33,12 +33,12 @@ import ( // // +stateify savable type TimerOperations struct { - fsutil.ZeroSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` - fsutil.NoIoctl `state:"nosave"` + fsutil.FileZeroSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` events waiter.Queue `state:"zerovalue"` timer *ktime.Timer diff --git a/pkg/sentry/fs/tmpfs/BUILD b/pkg/sentry/fs/tmpfs/BUILD index 9065cdd5d..14c7a9e62 100644 --- a/pkg/sentry/fs/tmpfs/BUILD +++ b/pkg/sentry/fs/tmpfs/BUILD @@ -23,11 +23,13 @@ go_library( "//pkg/sentry/kernel", "//pkg/sentry/kernel/auth", "//pkg/sentry/kernel/pipe", + "//pkg/sentry/kernel/time", "//pkg/sentry/memmap", "//pkg/sentry/safemem", "//pkg/sentry/socket/unix/transport", "//pkg/sentry/usage", "//pkg/sentry/usermem", + "//pkg/syserror", "//pkg/waiter", ], ) diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go index 1f9d69909..2c1eb0fd2 100644 --- a/pkg/sentry/fs/tmpfs/file_regular.go +++ b/pkg/sentry/fs/tmpfs/file_regular.go @@ -28,13 +28,13 @@ import ( // // +stateify savable type regularFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.NoopRelease `state:"nosave"` - fsutil.GenericSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoopFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoIoctl `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` // iops is the InodeOperations of a regular tmpfs file. It is // guaranteed to be the same as file.Dirent.Inode.InodeOperations, diff --git a/pkg/sentry/fs/tmpfs/file_test.go b/pkg/sentry/fs/tmpfs/file_test.go index 02da9af82..e7bbdc404 100644 --- a/pkg/sentry/fs/tmpfs/file_test.go +++ b/pkg/sentry/fs/tmpfs/file_test.go @@ -52,19 +52,19 @@ func TestGrow(t *testing.T) { abuf := bytes.Repeat([]byte{'a'}, 68) n, err := f.Pwritev(ctx, usermem.BytesIOSequence(abuf), 0) if n != int64(len(abuf)) || err != nil { - t.Fatalf("DeprecatedPwritev got (%d, %v) want (%d, nil)", n, err, len(abuf)) + t.Fatalf("Pwritev got (%d, %v) want (%d, nil)", n, err, len(abuf)) } bbuf := bytes.Repeat([]byte{'b'}, 856) n, err = f.Pwritev(ctx, usermem.BytesIOSequence(bbuf), 68) if n != int64(len(bbuf)) || err != nil { - t.Fatalf("DeprecatedPwritev got (%d, %v) want (%d, nil)", n, err, len(bbuf)) + t.Fatalf("Pwritev got (%d, %v) want (%d, nil)", n, err, len(bbuf)) } rbuf := make([]byte, len(abuf)+len(bbuf)) n, err = f.Preadv(ctx, usermem.BytesIOSequence(rbuf), 0) if n != int64(len(rbuf)) || err != nil { - t.Fatalf("DeprecatedPreadv got (%d, %v) want (%d, nil)", n, err, len(rbuf)) + t.Fatalf("Preadv got (%d, %v) want (%d, nil)", n, err, len(rbuf)) } if want := append(abuf, bbuf...); !bytes.Equal(rbuf, want) { diff --git a/pkg/sentry/fs/tmpfs/fs.go b/pkg/sentry/fs/tmpfs/fs.go index 88f85b85a..caa3220ee 100644 --- a/pkg/sentry/fs/tmpfs/fs.go +++ b/pkg/sentry/fs/tmpfs/fs.go @@ -50,6 +50,8 @@ const ( // +stateify savable type Filesystem struct{} +var _ fs.Filesystem = (*Filesystem)(nil) + func init() { fs.RegisterFilesystem(&Filesystem{}) } diff --git a/pkg/sentry/fs/tmpfs/inode_file.go b/pkg/sentry/fs/tmpfs/inode_file.go index ca2b4aabb..42d4bc76f 100644 --- a/pkg/sentry/fs/tmpfs/inode_file.go +++ b/pkg/sentry/fs/tmpfs/inode_file.go @@ -22,6 +22,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" "gvisor.googlesource.com/gvisor/pkg/sentry/usage" @@ -46,11 +47,13 @@ import ( // // +stateify savable type fileInodeOperations struct { - fsutil.DeprecatedFileOperations `state:"nosave"` - fsutil.InodeNotDirectory `state:"nosave"` - fsutil.InodeNotSocket `state:"nosave"` - fsutil.InodeNotSymlink `state:"nosave"` - fsutil.NoopWriteOut `state:"nosave"` + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + + fsutil.InodeSimpleExtendedAttributes // kernel is used to allocate platform memory that stores the file's contents. kernel *kernel.Kernel @@ -62,10 +65,10 @@ type fileInodeOperations struct { // attr contains the unstable metadata for the file. // - // attr is protected by attrMu. attr.Unstable.Size is protected by both - // attrMu and dataMu; reading it requires locking either mutex, while - // mutating it requires locking both. - attr fsutil.InMemoryAttributes + // attr is protected by attrMu. attr.Size is protected by both attrMu + // and dataMu; reading it requires locking either mutex, while mutating + // it requires locking both. + attr fs.UnstableAttr mapsMu sync.Mutex `state:"nosave"` @@ -83,12 +86,12 @@ type fileInodeOperations struct { data fsutil.FileRangeSet } +var _ fs.InodeOperations = (*fileInodeOperations)(nil) + // NewInMemoryFile returns a new file backed by p.Memory(). func NewInMemoryFile(ctx context.Context, usage usage.MemoryKind, uattr fs.UnstableAttr, k *kernel.Kernel) fs.InodeOperations { return &fileInodeOperations{ - attr: fsutil.InMemoryAttributes{ - Unstable: uattr, - }, + attr: uattr, kernel: k, memUsage: usage, } @@ -121,71 +124,56 @@ func (f *fileInodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flags f // UnstableAttr returns unstable attributes of this tmpfs file. func (f *fileInodeOperations) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { f.attrMu.Lock() - defer f.attrMu.Unlock() f.dataMu.RLock() - defer f.dataMu.RUnlock() - attr := f.attr.Unstable + attr := f.attr attr.Usage = int64(f.data.Span()) + f.dataMu.RUnlock() + f.attrMu.Unlock() return attr, nil } -// Getxattr implements fs.InodeOperations.Getxattr. -func (f *fileInodeOperations) Getxattr(inode *fs.Inode, name string) ([]byte, error) { - f.attrMu.Lock() - defer f.attrMu.Unlock() - return f.attr.Getxattr(name) -} - -// Setxattr implements fs.InodeOperations.Setxattr. -func (f *fileInodeOperations) Setxattr(inode *fs.Inode, name string, value []byte) error { - f.attrMu.Lock() - defer f.attrMu.Unlock() - return f.attr.Setxattr(name, value) -} - -// Listxattr implements fs.InodeOperations.Listxattr. -func (f *fileInodeOperations) Listxattr(inode *fs.Inode) (map[string]struct{}, error) { - f.attrMu.Lock() - defer f.attrMu.Unlock() - return f.attr.Listxattr() -} - // Check implements fs.InodeOperations.Check. func (f *fileInodeOperations) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { return fs.ContextCanAccessFile(ctx, inode, p) } // SetPermissions implements fs.InodeOperations.SetPermissions. -func (f *fileInodeOperations) SetPermissions(ctx context.Context, inode *fs.Inode, p fs.FilePermissions) bool { +func (f *fileInodeOperations) SetPermissions(ctx context.Context, _ *fs.Inode, p fs.FilePermissions) bool { f.attrMu.Lock() - defer f.attrMu.Unlock() - return f.attr.SetPermissions(ctx, p) + f.attr.SetPermissions(ctx, p) + f.attrMu.Unlock() + return true } // SetTimestamps implements fs.InodeOperations.SetTimestamps. -func (f *fileInodeOperations) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error { +func (f *fileInodeOperations) SetTimestamps(ctx context.Context, _ *fs.Inode, ts fs.TimeSpec) error { f.attrMu.Lock() - defer f.attrMu.Unlock() - return f.attr.SetTimestamps(ctx, ts) + f.attr.SetTimestamps(ctx, ts) + f.attrMu.Unlock() + return nil } // SetOwner implements fs.InodeOperations.SetOwner. -func (f *fileInodeOperations) SetOwner(ctx context.Context, inode *fs.Inode, owner fs.FileOwner) error { +func (f *fileInodeOperations) SetOwner(ctx context.Context, _ *fs.Inode, owner fs.FileOwner) error { f.attrMu.Lock() - defer f.attrMu.Unlock() - return f.attr.SetOwner(ctx, owner) + f.attr.SetOwner(ctx, owner) + f.attrMu.Unlock() + return nil } // Truncate implements fs.InodeOperations.Truncate. -func (f *fileInodeOperations) Truncate(ctx context.Context, inode *fs.Inode, size int64) error { +func (f *fileInodeOperations) Truncate(ctx context.Context, _ *fs.Inode, size int64) error { f.attrMu.Lock() defer f.attrMu.Unlock() f.dataMu.Lock() - oldSize := f.attr.Unstable.Size + oldSize := f.attr.Size if oldSize != size { - f.attr.Unstable.Size = size - f.attr.TouchModificationTime(ctx) + f.attr.Size = size + // Update mtime and ctime. + now := ktime.NowFromContext(ctx) + f.attr.ModificationTime = now + f.attr.StatusChangeTime = now } f.dataMu.Unlock() @@ -220,21 +208,21 @@ func (f *fileInodeOperations) Truncate(ctx context.Context, inode *fs.Inode, siz // AddLink implements fs.InodeOperations.AddLink. func (f *fileInodeOperations) AddLink() { f.attrMu.Lock() - f.attr.Unstable.Links++ + f.attr.Links++ f.attrMu.Unlock() } // DropLink implements fs.InodeOperations.DropLink. func (f *fileInodeOperations) DropLink() { f.attrMu.Lock() - f.attr.Unstable.Links-- + f.attr.Links-- f.attrMu.Unlock() } // NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. func (f *fileInodeOperations) NotifyStatusChange(ctx context.Context) { f.attrMu.Lock() - f.attr.TouchStatusChangeTime(ctx) + f.attr.StatusChangeTime = ktime.NowFromContext(ctx) f.attrMu.Unlock() } @@ -264,7 +252,7 @@ func (f *fileInodeOperations) read(ctx context.Context, dst usermem.IOSequence, // TODO: Separate out f.attr.Size and use atomics instead of // f.dataMu. f.dataMu.RLock() - size := f.attr.Unstable.Size + size := f.attr.Size f.dataMu.RUnlock() if offset >= size { return 0, io.EOF @@ -273,7 +261,7 @@ func (f *fileInodeOperations) read(ctx context.Context, dst usermem.IOSequence, n, err := dst.CopyOutFrom(ctx, &fileReadWriter{f, offset}) // Compare Linux's mm/filemap.c:do_generic_file_read() => file_accessed(). f.attrMu.Lock() - f.attr.TouchAccessTime(ctx) + f.attr.AccessTime = ktime.NowFromContext(ctx) f.attrMu.Unlock() return n, err } @@ -287,7 +275,9 @@ func (f *fileInodeOperations) write(ctx context.Context, src usermem.IOSequence, f.attrMu.Lock() defer f.attrMu.Unlock() // Compare Linux's mm/filemap.c:__generic_file_write_iter() => file_update_time(). - f.attr.TouchModificationTime(ctx) + now := ktime.NowFromContext(ctx) + f.attr.ModificationTime = now + f.attr.StatusChangeTime = now return src.CopyInTo(ctx, &fileReadWriter{f, offset}) } @@ -302,10 +292,10 @@ func (rw *fileReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error) { defer rw.f.dataMu.RUnlock() // Compute the range to read. - if rw.offset >= rw.f.attr.Unstable.Size { + if rw.offset >= rw.f.attr.Size { return 0, io.EOF } - end := fs.ReadEndOffset(rw.offset, int64(dsts.NumBytes()), rw.f.attr.Unstable.Size) + end := fs.ReadEndOffset(rw.offset, int64(dsts.NumBytes()), rw.f.attr.Size) if end == rw.offset { // dsts.NumBytes() == 0? return 0, nil } @@ -371,8 +361,8 @@ func (rw *fileReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) defer func() { // If the write ends beyond the file's previous size, it causes the // file to grow. - if rw.offset > rw.f.attr.Unstable.Size { - rw.f.attr.Unstable.Size = rw.offset + if rw.offset > rw.f.attr.Size { + rw.f.attr.Size = rw.offset } }() @@ -450,9 +440,9 @@ func (f *fileInodeOperations) Translate(ctx context.Context, required, optional f.dataMu.Lock() defer f.dataMu.Unlock() - // Constrain translations to f.attr.Unstable.Size (rounded up) to prevent + // Constrain translations to f.attr.Size (rounded up) to prevent // translation to pages that may be concurrently truncated. - pgend := fs.OffsetPageEnd(f.attr.Unstable.Size) + pgend := fs.OffsetPageEnd(f.attr.Size) var beyondEOF bool if required.End > pgend { if required.Start >= pgend { diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go index 40a8c4b1e..a0277a132 100644 --- a/pkg/sentry/fs/tmpfs/tmpfs.go +++ b/pkg/sentry/fs/tmpfs/tmpfs.go @@ -19,12 +19,14 @@ import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/pipe" "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport" "gvisor.googlesource.com/gvisor/pkg/sentry/usage" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserror" ) var fsInfo = fs.Info{ @@ -39,32 +41,54 @@ var fsInfo = fs.Info{ func rename(ctx context.Context, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error { op, ok := oldParent.InodeOperations.(*Dir) if !ok { - return ramfs.ErrCrossDevice + return syserror.EXDEV } np, ok := newParent.InodeOperations.(*Dir) if !ok { - return ramfs.ErrCrossDevice + return syserror.EXDEV } - return ramfs.Rename(ctx, &op.Dir, oldName, &np.Dir, newName) + return ramfs.Rename(ctx, op.ramfsDir, oldName, np.ramfsDir, newName) } // Dir is a directory. // // +stateify savable type Dir struct { - ramfs.Dir + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeIsDirTruncate `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + // Ideally this would be embedded, so that we "inherit" all of the + // InodeOperations implemented by ramfs.Dir for free. + // + // However, ramfs.dirFileOperations stores a pointer to a ramfs.Dir, + // and our save/restore package does not allow saving a pointer to an + // embedded field elsewhere. + // + // Thus, we must make the ramfs.Dir is a field, and we delegate all the + // InodeOperation methods to it. + ramfsDir *ramfs.Dir // kernel is used to allocate platform memory as storage for tmpfs Files. kernel *kernel.Kernel } +var _ fs.InodeOperations = (*Dir)(nil) + // NewDir returns a new directory. func NewDir(ctx context.Context, contents map[string]*fs.Inode, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource, kernel *kernel.Kernel) *fs.Inode { - d := &Dir{kernel: kernel} - d.InitDir(ctx, contents, owner, perms) + d := &Dir{ + ramfsDir: ramfs.NewDir(ctx, contents, owner, perms), + kernel: kernel, + } // Manually set the CreateOps. - d.CreateOps = d.newCreateOps() + d.ramfsDir.CreateOps = d.newCreateOps() return fs.NewInode(d, msrc, fs.StableAttr{ DeviceID: tmpfsDevice.DeviceID(), @@ -77,7 +101,107 @@ func NewDir(ctx context.Context, contents map[string]*fs.Inode, owner fs.FileOwn // afterLoad is invoked by stateify. func (d *Dir) afterLoad() { // Per NewDir, manually set the CreateOps. - d.Dir.CreateOps = d.newCreateOps() + d.ramfsDir.CreateOps = d.newCreateOps() +} + +// GetFile implements fs.InodeOperations.GetFile. +func (d *Dir) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + return d.ramfsDir.GetFile(ctx, dirent, flags) +} + +// AddLink implements fs.InodeOperations.AddLink. +func (d *Dir) AddLink() { + d.ramfsDir.AddLink() +} + +// DropLink implements fs.InodeOperations.DropLink. +func (d *Dir) DropLink() { + d.ramfsDir.DropLink() +} + +// Bind implements fs.InodeOperations.Bind. +func (d *Dir) Bind(ctx context.Context, dir *fs.Inode, name string, ep transport.BoundEndpoint, perms fs.FilePermissions) (*fs.Dirent, error) { + return d.ramfsDir.Bind(ctx, dir, name, ep, perms) +} + +// Create implements fs.InodeOperations.Create. +func (d *Dir) Create(ctx context.Context, dir *fs.Inode, name string, flags fs.FileFlags, perms fs.FilePermissions) (*fs.File, error) { + return d.ramfsDir.Create(ctx, dir, name, flags, perms) +} + +// CreateLink implements fs.InodeOperations.CreateLink. +func (d *Dir) CreateLink(ctx context.Context, dir *fs.Inode, oldname, newname string) error { + return d.ramfsDir.CreateLink(ctx, dir, oldname, newname) +} + +// CreateHardLink implements fs.InodeOperations.CreateHardLink. +func (d *Dir) CreateHardLink(ctx context.Context, dir *fs.Inode, target *fs.Inode, name string) error { + return d.ramfsDir.CreateHardLink(ctx, dir, target, name) +} + +// CreateDirectory implements fs.InodeOperations.CreateDirectory. +func (d *Dir) CreateDirectory(ctx context.Context, dir *fs.Inode, name string, perms fs.FilePermissions) error { + return d.ramfsDir.CreateDirectory(ctx, dir, name, perms) +} + +// CreateFifo implements fs.InodeOperations.CreateFifo. +func (d *Dir) CreateFifo(ctx context.Context, dir *fs.Inode, name string, perms fs.FilePermissions) error { + return d.ramfsDir.CreateFifo(ctx, dir, name, perms) +} + +// Getxattr implements fs.InodeOperations.Getxattr. +func (d *Dir) Getxattr(i *fs.Inode, name string) ([]byte, error) { + return d.ramfsDir.Getxattr(i, name) +} + +// Setxattr implements fs.InodeOperations.Setxattr. +func (d *Dir) Setxattr(i *fs.Inode, name string, value []byte) error { + return d.ramfsDir.Setxattr(i, name, value) +} + +// Listxattr implements fs.InodeOperations.Listxattr. +func (d *Dir) Listxattr(i *fs.Inode) (map[string]struct{}, error) { + return d.ramfsDir.Listxattr(i) +} + +// Lookup implements fs.InodeOperations.Lookup. +func (d *Dir) Lookup(ctx context.Context, i *fs.Inode, p string) (*fs.Dirent, error) { + return d.ramfsDir.Lookup(ctx, i, p) +} + +// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. +func (d *Dir) NotifyStatusChange(ctx context.Context) { + d.ramfsDir.NotifyStatusChange(ctx) +} + +// Remove implements fs.InodeOperations.Remove. +func (d *Dir) Remove(ctx context.Context, i *fs.Inode, name string) error { + return d.ramfsDir.Remove(ctx, i, name) +} + +// RemoveDirectory implements fs.InodeOperations.RemoveDirectory. +func (d *Dir) RemoveDirectory(ctx context.Context, i *fs.Inode, name string) error { + return d.ramfsDir.RemoveDirectory(ctx, i, name) +} + +// UnstableAttr implements fs.InodeOperations.UnstableAttr. +func (d *Dir) UnstableAttr(ctx context.Context, i *fs.Inode) (fs.UnstableAttr, error) { + return d.ramfsDir.UnstableAttr(ctx, i) +} + +// SetPermissions implements fs.InodeOperations.SetPermissions. +func (d *Dir) SetPermissions(ctx context.Context, i *fs.Inode, p fs.FilePermissions) bool { + return d.ramfsDir.SetPermissions(ctx, i, p) +} + +// SetOwner implements fs.InodeOperations.SetOwner. +func (d *Dir) SetOwner(ctx context.Context, i *fs.Inode, owner fs.FileOwner) error { + return d.ramfsDir.SetOwner(ctx, i, owner) +} + +// SetTimestamps implements fs.InodeOperations.SetTimestamps. +func (d *Dir) SetTimestamps(ctx context.Context, i *fs.Inode, ts fs.TimeSpec) error { + return d.ramfsDir.SetTimestamps(ctx, i, ts) } // newCreateOps builds the custom CreateOps for this Dir. @@ -132,8 +256,7 @@ type Symlink struct { // NewSymlink returns a new symlink with the provided permissions. func NewSymlink(ctx context.Context, target string, owner fs.FileOwner, msrc *fs.MountSource) *fs.Inode { - s := &Symlink{} - s.InitSymlink(ctx, owner, target) + s := &Symlink{Symlink: *ramfs.NewSymlink(ctx, owner, target)} return fs.NewInode(s, msrc, fs.StableAttr{ DeviceID: tmpfsDevice.DeviceID(), InodeID: tmpfsDevice.NextIno(), @@ -157,12 +280,12 @@ func (s *Symlink) StatFS(context.Context) (fs.Info, error) { // +stateify savable type Socket struct { ramfs.Socket + fsutil.InodeNotTruncatable `state:"nosave"` } // NewSocket returns a new socket with the provided permissions. func NewSocket(ctx context.Context, socket transport.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode { - s := &Socket{} - s.InitSocket(ctx, socket, owner, perms) + s := &Socket{Socket: *ramfs.NewSocket(ctx, socket, owner, perms)} return fs.NewInode(s, msrc, fs.StableAttr{ DeviceID: tmpfsDevice.DeviceID(), InodeID: tmpfsDevice.NextIno(), @@ -185,15 +308,22 @@ func (s *Socket) StatFS(context.Context) (fs.Info, error) { // // +stateify savable type Fifo struct { - ramfs.Entry + fs.InodeOperations } // NewFifo creates a new named pipe. func NewFifo(ctx context.Context, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode { - f := &Fifo{} - f.InitEntry(ctx, owner, perms) - iops := pipe.NewInodeOperations(f, pipe.NewPipe(ctx, true /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize)) - return fs.NewInode(iops, msrc, fs.StableAttr{ + // First create a pipe. + p := pipe.NewPipe(ctx, true /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize) + + // Build pipe InodeOperations. + iops := pipe.NewInodeOperations(ctx, perms, p) + + // Wrap the iops with our Fifo. + fifoIops := &Fifo{iops} + + // Build a new Inode. + return fs.NewInode(fifoIops, msrc, fs.StableAttr{ DeviceID: tmpfsDevice.DeviceID(), InodeID: tmpfsDevice.NextIno(), BlockSize: usermem.PageSize, diff --git a/pkg/sentry/fs/tty/BUILD b/pkg/sentry/fs/tty/BUILD index 2b45069a6..011cb6955 100644 --- a/pkg/sentry/fs/tty/BUILD +++ b/pkg/sentry/fs/tty/BUILD @@ -7,7 +7,6 @@ go_library( srcs = [ "dir.go", "fs.go", - "inode.go", "line_discipline.go", "master.go", "queue.go", @@ -25,7 +24,6 @@ go_library( "//pkg/sentry/fs", "//pkg/sentry/fs/fsutil", "//pkg/sentry/kernel/auth", - "//pkg/sentry/kernel/time", "//pkg/sentry/socket/unix/transport", "//pkg/sentry/unimpl", "//pkg/sentry/usermem", diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index e32b05c1d..485cdb456 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -52,13 +52,17 @@ import ( // // +stateify savable type dirInodeOperations struct { - fsutil.DeprecatedFileOperations `state:"nosave"` - fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` fsutil.InodeNotRenameable `state:"nosave"` fsutil.InodeNotSymlink `state:"nosave"` - fsutil.InodeNoExtendedAttributes `state:"nosave"` - fsutil.NoMappable `state:"nosave"` - fsutil.NoopWriteOut `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotTruncatable `state:"nosave"` + fsutil.InodeVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes // msrc is the super block this directory is on. // @@ -68,9 +72,6 @@ type dirInodeOperations struct { // mu protects the fields below. mu sync.Mutex `state:"nosave"` - // attr contains the UnstableAttrs. - attr fsutil.InMemoryAttributes - // master is the master PTY inode. master *fs.Inode @@ -97,15 +98,10 @@ var _ fs.InodeOperations = (*dirInodeOperations)(nil) // newDir creates a new dir with a ptmx file and no terminals. func newDir(ctx context.Context, m *fs.MountSource) *fs.Inode { d := &dirInodeOperations{ - attr: fsutil.InMemoryAttributes{ - Unstable: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: fs.RootOwner, - Perms: fs.FilePermsFromMode(0555), - }), - }, - msrc: m, - slaves: make(map[uint32]*fs.Inode), - dentryMap: fs.NewSortedDentryMap(nil), + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, fs.RootOwner, fs.FilePermsFromMode(0555), linux.DEVPTS_SUPER_MAGIC), + msrc: m, + slaves: make(map[uint32]*fs.Inode), + dentryMap: fs.NewSortedDentryMap(nil), } // Linux devpts uses a default mode of 0000 for ptmx which can be // changed with the ptmxmode mount option. However, that default is not @@ -224,70 +220,6 @@ func (d *dirInodeOperations) GetFile(ctx context.Context, dirent *fs.Dirent, fla return fs.NewFile(ctx, dirent, flags, &dirFileOperations{di: d}), nil } -// UnstableAttr implements fs.InodeOperations.UnstableAttr. -func (d *dirInodeOperations) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - d.mu.Lock() - defer d.mu.Unlock() - return d.attr.Unstable, nil -} - -// Check implements fs.InodeOperations.Check. -func (d *dirInodeOperations) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { - return fs.ContextCanAccessFile(ctx, inode, p) -} - -// SetPermissions implements fs.InodeOperations.SetPermissions. -func (d *dirInodeOperations) SetPermissions(ctx context.Context, inode *fs.Inode, p fs.FilePermissions) bool { - d.mu.Lock() - defer d.mu.Unlock() - return d.attr.SetPermissions(ctx, p) -} - -// SetOwner implements fs.InodeOperations.SetOwner. -func (d *dirInodeOperations) SetOwner(ctx context.Context, inode *fs.Inode, owner fs.FileOwner) error { - d.mu.Lock() - defer d.mu.Unlock() - return d.attr.SetOwner(ctx, owner) -} - -// SetTimestamps implements fs.InodeOperations.SetTimestamps. -func (d *dirInodeOperations) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error { - d.mu.Lock() - defer d.mu.Unlock() - return d.attr.SetTimestamps(ctx, ts) -} - -// Truncate implements fs.InodeOperations.Truncate. -func (d *dirInodeOperations) Truncate(ctx context.Context, inode *fs.Inode, size int64) error { - return syserror.EINVAL -} - -// AddLink implements fs.InodeOperations.AddLink. -func (d *dirInodeOperations) AddLink() {} - -// DropLink implements fs.InodeOperations.DropLink. -func (d *dirInodeOperations) DropLink() {} - -// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. -func (d *dirInodeOperations) NotifyStatusChange(ctx context.Context) { - d.mu.Lock() - defer d.mu.Unlock() - - d.attr.TouchStatusChangeTime(ctx) -} - -// IsVirtual implements fs.InodeOperations.IsVirtual. -func (d *dirInodeOperations) IsVirtual() bool { - return true -} - -// StatFS implements fs.InodeOperations.StatFS. -func (d *dirInodeOperations) StatFS(ctx context.Context) (fs.Info, error) { - return fs.Info{ - Type: linux.DEVPTS_SUPER_MAGIC, - }, nil -} - // allocateTerminal creates a new Terminal and installs a pts node for it. // // The caller must call DecRef when done with the returned Terminal. @@ -353,13 +285,13 @@ func (d *dirInodeOperations) masterClose(t *Terminal) { // // +stateify savable type dirFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.NoopRelease `state:"nosave"` - fsutil.GenericSeek `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` - fsutil.NoIoctl `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` // di is the inode operations. di *dirInodeOperations diff --git a/pkg/sentry/fs/tty/inode.go b/pkg/sentry/fs/tty/inode.go deleted file mode 100644 index d5d1caafc..000000000 --- a/pkg/sentry/fs/tty/inode.go +++ /dev/null @@ -1,145 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tty - -import ( - "sync" - - "gvisor.googlesource.com/gvisor/pkg/abi/linux" - "gvisor.googlesource.com/gvisor/pkg/sentry/context" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" - ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" - "gvisor.googlesource.com/gvisor/pkg/syserror" -) - -// inodeOperations are the base fs.InodeOperations for master and slave Inodes. -// -// inodeOperations does not implement: -// -// * fs.InodeOperations.Release -// * fs.InodeOperations.GetFile -// -// +stateify savable -type inodeOperations struct { - fsutil.DeprecatedFileOperations `state:"nosave"` - fsutil.InodeNoExtendedAttributes `state:"nosave"` - fsutil.InodeNotDirectory `state:"nosave"` - fsutil.InodeNotRenameable `state:"nosave"` - fsutil.InodeNotSocket `state:"nosave"` - fsutil.InodeNotSymlink `state:"nosave"` - fsutil.NoMappable `state:"nosave"` - fsutil.NoopWriteOut `state:"nosave"` - - // mu protects the fields below. - mu sync.Mutex `state:"nosave"` - - // uattr is the inode's UnstableAttr. - uattr fs.UnstableAttr -} - -// UnstableAttr implements fs.InodeOperations.UnstableAttr. -func (i *inodeOperations) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { - i.mu.Lock() - defer i.mu.Unlock() - return i.uattr, nil -} - -// Check implements fs.InodeOperations.Check. -func (i *inodeOperations) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool { - return fs.ContextCanAccessFile(ctx, inode, p) -} - -// SetPermissions implements fs.InodeOperations.SetPermissions -func (i *inodeOperations) SetPermissions(ctx context.Context, inode *fs.Inode, p fs.FilePermissions) bool { - i.mu.Lock() - defer i.mu.Unlock() - i.uattr.Perms = p - i.uattr.StatusChangeTime = ktime.NowFromContext(ctx) - return true -} - -// SetOwner implements fs.InodeOperations.SetOwner. -func (i *inodeOperations) SetOwner(ctx context.Context, inode *fs.Inode, owner fs.FileOwner) error { - i.mu.Lock() - defer i.mu.Unlock() - if owner.UID.Ok() { - i.uattr.Owner.UID = owner.UID - } - if owner.GID.Ok() { - i.uattr.Owner.GID = owner.GID - } - return nil -} - -// SetTimestamps implements fs.InodeOperations.SetTimestamps. -func (i *inodeOperations) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error { - if ts.ATimeOmit && ts.MTimeOmit { - return nil - } - - i.mu.Lock() - defer i.mu.Unlock() - - now := ktime.NowFromContext(ctx) - if !ts.ATimeOmit { - if ts.ATime.IsZero() { - i.uattr.AccessTime = now - } else { - i.uattr.AccessTime = ts.ATime - } - } - if !ts.MTimeOmit { - if ts.MTime.IsZero() { - i.uattr.ModificationTime = now - } else { - i.uattr.ModificationTime = ts.MTime - } - } - i.uattr.StatusChangeTime = now - return nil -} - -// Truncate implements fs.InodeOperations.Truncate. -func (i *inodeOperations) Truncate(ctx context.Context, inode *fs.Inode, size int64) error { - return syserror.EINVAL -} - -// AddLink implements fs.InodeOperations.AddLink. -func (i *inodeOperations) AddLink() { -} - -// DropLink implements fs.InodeOperations.DropLink. -func (i *inodeOperations) DropLink() { -} - -// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. -func (i *inodeOperations) NotifyStatusChange(ctx context.Context) { - i.mu.Lock() - defer i.mu.Unlock() - i.uattr.StatusChangeTime = ktime.NowFromContext(ctx) -} - -// IsVirtual implements fs.InodeOperations.IsVirtual. -func (i *inodeOperations) IsVirtual() bool { - return true -} - -// StatFS implements fs.InodeOperations.StatFS. -func (i *inodeOperations) StatFS(ctx context.Context) (fs.Info, error) { - return fs.Info{ - Type: linux.DEVPTS_SUPER_MAGIC, - }, nil -} diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index 00bec4c2c..b5e13ab36 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -31,7 +31,7 @@ import ( // // +stateify savable type masterInodeOperations struct { - inodeOperations + fsutil.SimpleFileInode // d is the containing dir. d *dirInodeOperations @@ -42,15 +42,8 @@ var _ fs.InodeOperations = (*masterInodeOperations)(nil) // newMasterInode creates an Inode for the master end of a terminal. func newMasterInode(ctx context.Context, d *dirInodeOperations, owner fs.FileOwner, p fs.FilePermissions) *fs.Inode { iops := &masterInodeOperations{ - inodeOperations: inodeOperations{ - uattr: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: owner, - Perms: p, - Links: 1, - // Size and Blocks are always 0. - }), - }, - d: d, + SimpleFileInode: *fsutil.NewSimpleFileInode(ctx, owner, p, linux.DEVPTS_SUPER_MAGIC), + d: d, } return fs.NewInode(iops, d.msrc, fs.StableAttr{ @@ -102,11 +95,11 @@ func (mi *masterInodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flag // // +stateify savable type masterFileOperations struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` // d is the containing dir. d *dirInodeOperations diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index a696fbb51..6dbce90b4 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -30,7 +30,7 @@ import ( // // +stateify savable type slaveInodeOperations struct { - inodeOperations + fsutil.SimpleFileInode // d is the containing dir. d *dirInodeOperations @@ -46,16 +46,9 @@ var _ fs.InodeOperations = (*slaveInodeOperations)(nil) // newSlaveInode takes ownership of t. func newSlaveInode(ctx context.Context, d *dirInodeOperations, t *Terminal, owner fs.FileOwner, p fs.FilePermissions) *fs.Inode { iops := &slaveInodeOperations{ - inodeOperations: inodeOperations{ - uattr: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: owner, - Perms: p, - Links: 1, - // Size and Blocks are always 0. - }), - }, - d: d, - t: t, + SimpleFileInode: *fsutil.NewSimpleFileInode(ctx, owner, p, linux.DEVPTS_SUPER_MAGIC), + d: d, + t: t, } return fs.NewInode(iops, d.msrc, fs.StableAttr{ @@ -91,11 +84,11 @@ func (si *slaveInodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flags // // +stateify savable type slaveFileOperations struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` // si is the inode operations. si *slaveInodeOperations diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go index 9c13ecfcc..502395f18 100644 --- a/pkg/sentry/kernel/epoll/epoll.go +++ b/pkg/sentry/kernel/epoll/epoll.go @@ -99,12 +99,12 @@ func (p *pollEntry) WeakRefGone() { // // +stateify savable type EventPoll struct { - fsutil.PipeSeek `state:"zerovalue"` - fsutil.NotDirReaddir `state:"zerovalue"` - fsutil.NoFsync `state:"zerovalue"` - fsutil.NoopFlush `state:"zerovalue"` - fsutil.NoMMap `state:"zerovalue"` - fsutil.NoIoctl `state:"zerovalue"` + fsutil.FilePipeSeek `state:"zerovalue"` + fsutil.FileNotDirReaddir `state:"zerovalue"` + fsutil.FileNoFsync `state:"zerovalue"` + fsutil.FileNoopFlush `state:"zerovalue"` + fsutil.FileNoMMap `state:"zerovalue"` + fsutil.FileNoIoctl `state:"zerovalue"` // Wait queue is used to notify interested parties when the event poll // object itself becomes readable or writable. diff --git a/pkg/sentry/kernel/eventfd/eventfd.go b/pkg/sentry/kernel/eventfd/eventfd.go index 063a1d5f5..2d43c986d 100644 --- a/pkg/sentry/kernel/eventfd/eventfd.go +++ b/pkg/sentry/kernel/eventfd/eventfd.go @@ -38,13 +38,13 @@ import ( // // +stateify savable type EventOperations struct { - fsutil.NoopRelease `state:"nosave"` - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` - fsutil.NoIoctl `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` // Mutex that protects accesses to the fields of this event. mu sync.Mutex `state:"nosave"` diff --git a/pkg/sentry/kernel/pipe/node.go b/pkg/sentry/kernel/pipe/node.go index 4b0e00b85..1336b6293 100644 --- a/pkg/sentry/kernel/pipe/node.go +++ b/pkg/sentry/kernel/pipe/node.go @@ -17,17 +17,30 @@ package pipe import ( "sync" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/amutex" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/syserror" ) -// inodeOperations wraps fs.InodeOperations operations with common pipe opening semantics. +// inodeOperations implements fs.InodeOperations for pipes. // // +stateify savable type inodeOperations struct { - fs.InodeOperations + fsutil.InodeGenericChecker `state:"nosave"` + fsutil.InodeNoExtendedAttributes `state:"nosave"` + fsutil.InodeNoopRelease `state:"nosave"` + fsutil.InodeNoopTruncate `state:"nosave"` + fsutil.InodeNoopWriteOut `state:"nosave"` + fsutil.InodeNotDirectory `state:"nosave"` + fsutil.InodeNotMappable `state:"nosave"` + fsutil.InodeNotSocket `state:"nosave"` + fsutil.InodeNotSymlink `state:"nosave"` + fsutil.InodeNotVirtual `state:"nosave"` + + fsutil.InodeSimpleAttributes // mu protects the fields below. mu sync.Mutex `state:"nosave"` @@ -46,12 +59,15 @@ type inodeOperations struct { wWakeup chan struct{} `state:"nosave"` } -// NewInodeOperations creates a new pipe fs.InodeOperations. -func NewInodeOperations(base fs.InodeOperations, p *Pipe) fs.InodeOperations { +var _ fs.InodeOperations = (*inodeOperations)(nil) + +// NewInodeOperations returns a new fs.InodeOperations for a given pipe. +func NewInodeOperations(ctx context.Context, perms fs.FilePermissions, p *Pipe) *inodeOperations { return &inodeOperations{ - InodeOperations: base, - p: p, + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, fs.FileOwnerFromContext(ctx), perms, linux.PIPEFS_MAGIC), + p: p, } + } // GetFile implements fs.InodeOperations.GetFile. Named pipes have special blocking @@ -164,18 +180,6 @@ func (i *inodeOperations) waitFor(wakeupChan *chan struct{}, sleeper amutex.Slee } } -// Truncate implements fs.InodeOperations.Truncate -// -// This method is required to override the default i.InodeOperations.Truncate -// which may return ErrInvalidOperation, this allows open related -// syscalls to set the O_TRUNC flag without returning an error by -// calling Truncate directly during openat. The ftruncate and truncate -// system calls will check that the file is an actual file and return -// EINVAL because it's a PIPE, making this behavior consistent with linux. -func (i *inodeOperations) Truncate(context.Context, *fs.Inode, int64) error { - return nil -} - // newHandleLocked signals a new pipe reader or writer depending on where // 'wakeupChan' points. This unblocks any corresponding reader or writer // waiting for the other end of the channel to be opened, see Fifo.waitFor. diff --git a/pkg/sentry/kernel/pipe/node_test.go b/pkg/sentry/kernel/pipe/node_test.go index eda551594..ad103b195 100644 --- a/pkg/sentry/kernel/pipe/node_test.go +++ b/pkg/sentry/kernel/pipe/node_test.go @@ -53,6 +53,10 @@ type openResult struct { error } +var perms fs.FilePermissions = fs.FilePermissions{ + User: fs.PermMask{Read: true, Write: true}, +} + func testOpenOrDie(ctx context.Context, t *testing.T, n fs.InodeOperations, flags fs.FileFlags, doneChan chan<- struct{}) (*fs.File, error) { file, err := n.GetFile(ctx, nil, flags) if err != nil { @@ -93,8 +97,8 @@ func assertRecvBlocks(t *testing.T, c <-chan struct{}, blockDuration time.Durati } func TestReadOpenBlocksForWriteOpen(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) rDone := make(chan struct{}) go testOpenOrDie(ctx, t, f, fs.FileFlags{Read: true}, rDone) @@ -111,8 +115,8 @@ func TestReadOpenBlocksForWriteOpen(t *testing.T) { } func TestWriteOpenBlocksForReadOpen(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) wDone := make(chan struct{}) go testOpenOrDie(ctx, t, f, fs.FileFlags{Write: true}, wDone) @@ -129,8 +133,8 @@ func TestWriteOpenBlocksForReadOpen(t *testing.T) { } func TestMultipleWriteOpenDoesntCountAsReadOpen(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) rDone1 := make(chan struct{}) rDone2 := make(chan struct{}) @@ -151,8 +155,8 @@ func TestMultipleWriteOpenDoesntCountAsReadOpen(t *testing.T) { } func TestClosedReaderBlocksWriteOpen(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) rFile, _ := testOpenOrDie(ctx, t, f, fs.FileFlags{Read: true, NonBlocking: true}, nil) rFile.DecRef() @@ -172,8 +176,8 @@ func TestClosedReaderBlocksWriteOpen(t *testing.T) { } func TestReadWriteOpenNeverBlocks(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) rwDone := make(chan struct{}) // Open for read-write never wait for a reader or writer, even if the @@ -183,8 +187,8 @@ func TestReadWriteOpenNeverBlocks(t *testing.T) { } func TestReadWriteOpenUnblocksReadOpen(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) rDone := make(chan struct{}) go testOpenOrDie(ctx, t, f, fs.FileFlags{Read: true}, rDone) @@ -197,8 +201,8 @@ func TestReadWriteOpenUnblocksReadOpen(t *testing.T) { } func TestReadWriteOpenUnblocksWriteOpen(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) wDone := make(chan struct{}) go testOpenOrDie(ctx, t, f, fs.FileFlags{Write: true}, wDone) @@ -211,8 +215,8 @@ func TestReadWriteOpenUnblocksWriteOpen(t *testing.T) { } func TestBlockedOpenIsCancellable(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) done := make(chan openResult) go testOpen(ctx, t, f, fs.FileFlags{Read: true}, done) @@ -233,18 +237,18 @@ func TestBlockedOpenIsCancellable(t *testing.T) { } } -func TestNonblockingReadOpenNoWriters(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) +func TestNonblockingReadOpenFileNoWriters(t *testing.T) { ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) if _, err := testOpen(ctx, t, f, fs.FileFlags{Read: true, NonBlocking: true}, nil); err != nil { t.Fatalf("Nonblocking open for read failed with error %v.", err) } } -func TestNonblockingWriteOpenNoReaders(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) +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 { t.Fatalf("Nonblocking open for write failed unexpected error %v.", err) @@ -252,8 +256,8 @@ func TestNonblockingWriteOpenNoReaders(t *testing.T) { } func TestNonBlockingReadOpenWithWriter(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) wDone := make(chan struct{}) go testOpenOrDie(ctx, t, f, fs.FileFlags{Write: true}, wDone) @@ -271,8 +275,8 @@ func TestNonBlockingReadOpenWithWriter(t *testing.T) { } func TestNonBlockingWriteOpenWithReader(t *testing.T) { - f := NewInodeOperations(nil, newNamedPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newNamedPipe(t)) rDone := make(chan struct{}) go testOpenOrDie(ctx, t, f, fs.FileFlags{Read: true}, rDone) @@ -290,8 +294,8 @@ func TestNonBlockingWriteOpenWithReader(t *testing.T) { } func TestAnonReadOpen(t *testing.T) { - f := NewInodeOperations(nil, newAnonPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newAnonPipe(t)) if _, err := testOpen(ctx, t, f, fs.FileFlags{Read: true}, nil); err != nil { t.Fatalf("open anon pipe for read failed: %v", err) @@ -299,8 +303,8 @@ func TestAnonReadOpen(t *testing.T) { } func TestAnonWriteOpen(t *testing.T) { - f := NewInodeOperations(nil, newAnonPipe(t)) ctx := newSleeperContext(t) + f := NewInodeOperations(ctx, perms, newAnonPipe(t)) if _, err := testOpen(ctx, t, f, fs.FileFlags{Write: true}, nil); err != nil { t.Fatalf("open anon pipe for write failed: %v", err) diff --git a/pkg/sentry/kernel/pipe/pipe.go b/pkg/sentry/kernel/pipe/pipe.go index 126054826..fad077d2d 100644 --- a/pkg/sentry/kernel/pipe/pipe.go +++ b/pkg/sentry/kernel/pipe/pipe.go @@ -25,11 +25,9 @@ import ( "sync/atomic" "syscall" - "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/ilist" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/waiter" @@ -50,7 +48,7 @@ type Pipe struct { isNamed bool // The dirent backing this pipe. Shared by all readers and writers. - dirent *fs.Dirent + Dirent *fs.Dirent // The buffered byte queue. data ilist.List @@ -97,28 +95,19 @@ func NewPipe(ctx context.Context, isNamed bool, sizeBytes, atomicIOBytes int) *P // Build the fs.Dirent of this pipe, shared by all fs.Files associated // with this pipe. + perms := fs.FilePermissions{ + User: fs.PermMask{Read: true, Write: true}, + } + iops := NewInodeOperations(ctx, perms, p) ino := pipeDevice.NextIno() - base := fsutil.NewSimpleInodeOperations(fsutil.InodeSimpleAttributes{ - FSType: linux.PIPEFS_MAGIC, - UAttr: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: fs.FileOwnerFromContext(ctx), - Perms: fs.FilePermissions{ - User: fs.PermMask{Read: true, Write: true}, - }, - Links: 1, - }), - }) sattr := fs.StableAttr{ Type: fs.Pipe, DeviceID: pipeDevice.DeviceID(), InodeID: ino, BlockSize: int64(atomicIOBytes), } - // There is no real filesystem backing this pipe, so we pass in a nil - // Filesystem. - sb := fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}) - p.dirent = fs.NewDirent(fs.NewInode(NewInodeOperations(base, p), sb, sattr), fmt.Sprintf("pipe:[%d]", ino)) - + ms := fs.NewPseudoMountSource() + p.Dirent = fs.NewDirent(fs.NewInode(iops, ms, sattr), fmt.Sprintf("pipe:[%d]", ino)) return p } @@ -135,7 +124,7 @@ func NewConnectedPipe(ctx context.Context, sizeBytes int, atomicIOBytes int) (*f // ROpen opens the pipe for reading. func (p *Pipe) ROpen(ctx context.Context) *fs.File { p.rOpen() - return fs.NewFile(ctx, p.dirent, fs.FileFlags{Read: true}, &Reader{ + return fs.NewFile(ctx, p.Dirent, fs.FileFlags{Read: true}, &Reader{ ReaderWriter: ReaderWriter{Pipe: p}, }) } @@ -143,7 +132,7 @@ func (p *Pipe) ROpen(ctx context.Context) *fs.File { // WOpen opens the pipe for writing. func (p *Pipe) WOpen(ctx context.Context) *fs.File { p.wOpen() - return fs.NewFile(ctx, p.dirent, fs.FileFlags{Write: true}, &Writer{ + return fs.NewFile(ctx, p.Dirent, fs.FileFlags{Write: true}, &Writer{ ReaderWriter: ReaderWriter{Pipe: p}, }) } @@ -152,7 +141,7 @@ func (p *Pipe) WOpen(ctx context.Context) *fs.File { func (p *Pipe) RWOpen(ctx context.Context) *fs.File { p.rOpen() p.wOpen() - return fs.NewFile(ctx, p.dirent, fs.FileFlags{Read: true, Write: true}, &ReaderWriter{ + return fs.NewFile(ctx, p.Dirent, fs.FileFlags{Read: true, Write: true}, &ReaderWriter{ Pipe: p, }) } diff --git a/pkg/sentry/kernel/pipe/reader_writer.go b/pkg/sentry/kernel/pipe/reader_writer.go index 36be1efc3..028175530 100644 --- a/pkg/sentry/kernel/pipe/reader_writer.go +++ b/pkg/sentry/kernel/pipe/reader_writer.go @@ -35,11 +35,11 @@ import ( // // +stateify savable type ReaderWriter struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` *Pipe } diff --git a/pkg/sentry/loader/vdso.go b/pkg/sentry/loader/vdso.go index 437cc5da1..c070c7316 100644 --- a/pkg/sentry/loader/vdso.go +++ b/pkg/sentry/loader/vdso.go @@ -20,7 +20,6 @@ import ( "io" "gvisor.googlesource.com/gvisor/pkg/abi" - "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/log" "gvisor.googlesource.com/gvisor/pkg/sentry/arch" "gvisor.googlesource.com/gvisor/pkg/sentry/context" @@ -38,20 +37,6 @@ import ( "gvisor.googlesource.com/gvisor/pkg/waiter" ) -// byteReaderFileOperations implements fs.FileOperations for reading -// from a []byte source. -type byteReader struct { - fsutil.NoopRelease - fsutil.PipeSeek - fsutil.NotDirReaddir - fsutil.NoFsync - fsutil.NoopFlush - fsutil.NoMMap - fsutil.NoIoctl - waiter.AlwaysReady - data []byte -} - type fileContext struct { context.Context } @@ -65,17 +50,34 @@ func (f *fileContext) Value(key interface{}) interface{} { } } +// byteReader implements fs.FileOperations for reading from a []byte source. +type byteReader struct { + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + + data []byte +} + +var _ fs.FileOperations = (*byteReader)(nil) + // newByteReaderFile creates a fake file to read data from. func newByteReaderFile(data []byte) *fs.File { // Create a fake inode. - inode := fs.NewInode(fsutil.NewSimpleInodeOperations(fsutil.InodeSimpleAttributes{ - FSType: linux.ANON_INODE_FS_MAGIC, - }), fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}), fs.StableAttr{ - Type: fs.Anonymous, - DeviceID: anon.PseudoDevice.DeviceID(), - InodeID: anon.PseudoDevice.NextIno(), - BlockSize: usermem.PageSize, - }) + inode := fs.NewInode( + &fsutil.SimpleFileInode{}, + fs.NewPseudoMountSource(), + fs.StableAttr{ + Type: fs.Anonymous, + DeviceID: anon.PseudoDevice.DeviceID(), + InodeID: anon.PseudoDevice.NextIno(), + BlockSize: usermem.PageSize, + }) // Use the fake inode to create a fake dirent. dirent := fs.NewTransientDirent(inode) diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index d65b5f49e..ca865b111 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -138,11 +138,11 @@ type commonEndpoint interface { // // +stateify savable type SocketOperations struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` socket.SendReceiveTimeout *waiter.Queue diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index f3ecb6dc3..2c54e8de2 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -46,11 +46,11 @@ const ( // socketOperations implements fs.FileOperations and socket.Socket for a socket // implemented using a host socket. type socketOperations struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` socket.SendReceiveTimeout fd int // must be O_NONBLOCK diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index 0a7d4772c..5b0c11c84 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -65,11 +65,11 @@ var netlinkSocketDevice = device.NewAnonDevice() // // +stateify savable type Socket struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` socket.SendReceiveTimeout // ports provides netlink port allocation. diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 8c8ebadb7..13681100e 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -45,11 +45,11 @@ import ( // socketOperations implements fs.FileOperations and socket.Socket for a socket // implemented using a host socket. type socketOperations struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` socket.SendReceiveTimeout fd uint32 // must be O_NONBLOCK diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index 9d4aaeb9d..e28d2c4fa 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -178,18 +178,12 @@ func Pair(t *kernel.Task, family int, stype transport.SockType, protocol int) (* // NewDirent returns a sockfs fs.Dirent that resides on device d. func NewDirent(ctx context.Context, d *device.Device) *fs.Dirent { ino := d.NextIno() - // There is no real filesystem backing this pipe, so we pass in a nil - // Filesystem. - inode := fs.NewInode(fsutil.NewSimpleInodeOperations(fsutil.InodeSimpleAttributes{ - FSType: linux.SOCKFS_MAGIC, - UAttr: fs.WithCurrentTime(ctx, fs.UnstableAttr{ - Owner: fs.FileOwnerFromContext(ctx), - Perms: fs.FilePermissions{ - User: fs.PermMask{Read: true, Write: true}, - }, - Links: 1, - }), - }), fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}), fs.StableAttr{ + iops := &fsutil.SimpleFileInode{ + InodeSimpleAttributes: fsutil.NewInodeSimpleAttributes(ctx, fs.FileOwnerFromContext(ctx), fs.FilePermissions{ + User: fs.PermMask{Read: true, Write: true}, + }, linux.SOCKFS_MAGIC), + } + inode := fs.NewInode(iops, fs.NewPseudoMountSource(), fs.StableAttr{ Type: fs.Socket, DeviceID: d.DeviceID(), InodeID: ino, diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index da225eabb..19258e692 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -45,11 +45,11 @@ import ( // // +stateify savable type SocketOperations struct { - fsutil.PipeSeek `state:"nosave"` - fsutil.NotDirReaddir `state:"nosave"` - fsutil.NoFsync `state:"nosave"` - fsutil.NoopFlush `state:"nosave"` - fsutil.NoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` refs.AtomicRefCount socket.SendReceiveTimeout diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go index 1e75b0efc..942315d6e 100644 --- a/runsc/boot/fs.go +++ b/runsc/boot/fs.go @@ -489,9 +489,7 @@ func mustFindFilesystem(name string) fs.Filesystem { // addSubmountOverlay overlays the inode over a ramfs tree containing the given // paths. func addSubmountOverlay(ctx context.Context, inode *fs.Inode, submounts []string) (*fs.Inode, error) { - // There is no real filesystem backing this ramfs tree, so we pass in - // "nil" here. - msrc := fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}) + msrc := fs.NewPseudoMountSource() mountTree, err := ramfs.MakeDirectoryTree(ctx, msrc, submounts) if err != nil { return nil, fmt.Errorf("error creating mount tree: %v", err) diff --git a/test/syscalls/linux/proc.cc b/test/syscalls/linux/proc.cc index e64df97b0..6ffe9aed6 100644 --- a/test/syscalls/linux/proc.cc +++ b/test/syscalls/linux/proc.cc @@ -334,6 +334,11 @@ int ReadlinkWhileExited(std::string const& basename, char* buf, size_t count) { return ret; } +TEST(ProcTest, NotFoundInRoot) { + struct stat s; + EXPECT_THAT(stat("/proc/foobar", &s), SyscallFailsWithErrno(ENOENT)); +} + TEST(ProcSelfTest, IsThreadGroupLeader) { ScopedThread([] { const pid_t tgid = getpid(); -- cgit v1.2.3 From 2a0c69b19f4b55c3f9777f0098a72af123ccff3c Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Thu, 31 Jan 2019 11:11:44 -0800 Subject: Remove license comments Nothing reads them and they can simply get stale. Generated with: $ sed -i "s/licenses(\(.*\)).*/licenses(\1)/" **/BUILD PiperOrigin-RevId: 231818945 Change-Id: Ibc3f9838546b7e94f13f217060d31f4ada9d4bf0 --- pkg/abi/BUILD | 2 +- pkg/abi/linux/BUILD | 2 +- pkg/amutex/BUILD | 2 +- pkg/atomicbitops/BUILD | 2 +- pkg/binary/BUILD | 2 +- pkg/bits/BUILD | 2 +- pkg/bpf/BUILD | 2 +- pkg/compressio/BUILD | 2 +- pkg/control/client/BUILD | 2 +- pkg/control/server/BUILD | 2 +- pkg/cpuid/BUILD | 2 +- pkg/dhcp/BUILD | 2 +- pkg/eventchannel/BUILD | 2 +- pkg/fd/BUILD | 2 +- pkg/fdnotifier/BUILD | 2 +- pkg/gate/BUILD | 2 +- pkg/ilist/BUILD | 2 +- pkg/linewriter/BUILD | 2 +- pkg/log/BUILD | 2 +- pkg/metric/BUILD | 2 +- pkg/p9/BUILD | 2 +- pkg/p9/local_server/BUILD | 2 +- pkg/p9/p9test/BUILD | 2 +- pkg/rand/BUILD | 2 +- pkg/refs/BUILD | 2 +- pkg/seccomp/BUILD | 2 +- pkg/secio/BUILD | 2 +- pkg/segment/BUILD | 2 +- pkg/segment/test/BUILD | 2 +- pkg/sentry/BUILD | 2 +- pkg/sentry/arch/BUILD | 2 +- pkg/sentry/context/BUILD | 2 +- pkg/sentry/context/contexttest/BUILD | 2 +- pkg/sentry/control/BUILD | 2 +- pkg/sentry/device/BUILD | 2 +- pkg/sentry/fs/BUILD | 2 +- pkg/sentry/fs/anon/BUILD | 2 +- pkg/sentry/fs/ashmem/BUILD | 2 +- pkg/sentry/fs/binder/BUILD | 2 +- pkg/sentry/fs/dev/BUILD | 2 +- pkg/sentry/fs/fdpipe/BUILD | 2 +- pkg/sentry/fs/filetest/BUILD | 2 +- pkg/sentry/fs/fsutil/BUILD | 2 +- pkg/sentry/fs/gofer/BUILD | 2 +- pkg/sentry/fs/host/BUILD | 2 +- pkg/sentry/fs/lock/BUILD | 2 +- pkg/sentry/fs/proc/BUILD | 2 +- pkg/sentry/fs/proc/device/BUILD | 2 +- pkg/sentry/fs/proc/seqfile/BUILD | 2 +- pkg/sentry/fs/ramfs/BUILD | 2 +- pkg/sentry/fs/sys/BUILD | 2 +- pkg/sentry/fs/timerfd/BUILD | 2 +- pkg/sentry/fs/tmpfs/BUILD | 2 +- pkg/sentry/fs/tty/BUILD | 2 +- pkg/sentry/hostcpu/BUILD | 2 +- pkg/sentry/inet/BUILD | 2 +- pkg/sentry/kernel/BUILD | 2 +- pkg/sentry/kernel/auth/BUILD | 2 +- pkg/sentry/kernel/contexttest/BUILD | 2 +- pkg/sentry/kernel/epoll/BUILD | 2 +- pkg/sentry/kernel/eventfd/BUILD | 2 +- pkg/sentry/kernel/fasync/BUILD | 2 +- pkg/sentry/kernel/futex/BUILD | 2 +- pkg/sentry/kernel/kdefs/BUILD | 2 +- pkg/sentry/kernel/memevent/BUILD | 2 +- pkg/sentry/kernel/pipe/BUILD | 2 +- pkg/sentry/kernel/sched/BUILD | 2 +- pkg/sentry/kernel/semaphore/BUILD | 2 +- pkg/sentry/kernel/shm/BUILD | 2 +- pkg/sentry/kernel/time/BUILD | 2 +- pkg/sentry/limits/BUILD | 2 +- pkg/sentry/loader/BUILD | 2 +- pkg/sentry/memmap/BUILD | 2 +- pkg/sentry/memutil/BUILD | 2 +- pkg/sentry/mm/BUILD | 2 +- pkg/sentry/platform/BUILD | 2 +- pkg/sentry/platform/filemem/BUILD | 2 +- pkg/sentry/platform/interrupt/BUILD | 2 +- pkg/sentry/platform/kvm/BUILD | 2 +- pkg/sentry/platform/kvm/testutil/BUILD | 2 +- pkg/sentry/platform/procid/BUILD | 2 +- pkg/sentry/platform/ptrace/BUILD | 2 +- pkg/sentry/platform/ring0/BUILD | 2 +- pkg/sentry/platform/ring0/gen_offsets/BUILD | 2 +- pkg/sentry/platform/ring0/pagetables/BUILD | 2 +- pkg/sentry/platform/safecopy/BUILD | 2 +- pkg/sentry/safemem/BUILD | 2 +- pkg/sentry/sighandling/BUILD | 2 +- pkg/sentry/socket/BUILD | 2 +- pkg/sentry/socket/control/BUILD | 2 +- pkg/sentry/socket/epsocket/BUILD | 2 +- pkg/sentry/socket/hostinet/BUILD | 2 +- pkg/sentry/socket/netlink/BUILD | 2 +- pkg/sentry/socket/netlink/port/BUILD | 2 +- pkg/sentry/socket/netlink/route/BUILD | 2 +- pkg/sentry/socket/rpcinet/BUILD | 2 +- pkg/sentry/socket/rpcinet/conn/BUILD | 2 +- pkg/sentry/socket/rpcinet/notifier/BUILD | 2 +- pkg/sentry/socket/unix/BUILD | 2 +- pkg/sentry/socket/unix/transport/BUILD | 2 +- pkg/sentry/state/BUILD | 2 +- pkg/sentry/strace/BUILD | 2 +- pkg/sentry/syscalls/BUILD | 2 +- pkg/sentry/syscalls/linux/BUILD | 2 +- pkg/sentry/time/BUILD | 2 +- pkg/sentry/unimpl/BUILD | 2 +- pkg/sentry/uniqueid/BUILD | 2 +- pkg/sentry/usage/BUILD | 2 +- pkg/sentry/usermem/BUILD | 2 +- pkg/sentry/watchdog/BUILD | 2 +- pkg/sleep/BUILD | 2 +- pkg/state/BUILD | 2 +- pkg/state/statefile/BUILD | 2 +- pkg/sync/BUILD | 2 +- pkg/sync/atomicptrtest/BUILD | 2 +- pkg/sync/seqatomictest/BUILD | 2 +- pkg/syserr/BUILD | 2 +- pkg/syserror/BUILD | 2 +- pkg/tcpip/BUILD | 2 +- pkg/tcpip/adapters/gonet/BUILD | 2 +- pkg/tcpip/buffer/BUILD | 2 +- pkg/tcpip/checker/BUILD | 2 +- pkg/tcpip/hash/jenkins/BUILD | 2 +- pkg/tcpip/header/BUILD | 2 +- pkg/tcpip/link/channel/BUILD | 2 +- pkg/tcpip/link/fdbased/BUILD | 2 +- pkg/tcpip/link/loopback/BUILD | 2 +- pkg/tcpip/link/rawfile/BUILD | 2 +- pkg/tcpip/link/sharedmem/BUILD | 2 +- pkg/tcpip/link/sharedmem/pipe/BUILD | 2 +- pkg/tcpip/link/sharedmem/queue/BUILD | 2 +- pkg/tcpip/link/sniffer/BUILD | 2 +- pkg/tcpip/link/tun/BUILD | 2 +- pkg/tcpip/link/waitable/BUILD | 2 +- pkg/tcpip/network/BUILD | 2 +- pkg/tcpip/network/arp/BUILD | 2 +- pkg/tcpip/network/fragmentation/BUILD | 2 +- pkg/tcpip/network/hash/BUILD | 2 +- pkg/tcpip/network/ipv4/BUILD | 2 +- pkg/tcpip/network/ipv6/BUILD | 2 +- pkg/tcpip/ports/BUILD | 2 +- pkg/tcpip/sample/tun_tcp_connect/BUILD | 2 +- pkg/tcpip/sample/tun_tcp_echo/BUILD | 2 +- pkg/tcpip/seqnum/BUILD | 2 +- pkg/tcpip/stack/BUILD | 2 +- pkg/tcpip/transport/ping/BUILD | 2 +- pkg/tcpip/transport/tcp/BUILD | 2 +- pkg/tcpip/transport/tcp/testing/context/BUILD | 2 +- pkg/tcpip/transport/tcpconntrack/BUILD | 2 +- pkg/tcpip/transport/udp/BUILD | 2 +- pkg/tmutex/BUILD | 2 +- pkg/unet/BUILD | 2 +- pkg/urpc/BUILD | 2 +- pkg/waiter/BUILD | 2 +- runsc/boot/BUILD | 2 +- runsc/boot/filter/BUILD | 2 +- runsc/cgroup/BUILD | 2 +- runsc/cmd/BUILD | 2 +- runsc/console/BUILD | 2 +- runsc/container/BUILD | 2 +- runsc/fsgofer/BUILD | 2 +- runsc/fsgofer/filter/BUILD | 2 +- runsc/sandbox/BUILD | 2 +- runsc/specutils/BUILD | 2 +- runsc/test/image/BUILD | 2 +- runsc/test/integration/BUILD | 2 +- runsc/test/root/BUILD | 2 +- runsc/test/root/testdata/BUILD | 2 +- runsc/test/testutil/BUILD | 2 +- runsc/tools/dockercfg/BUILD | 2 +- test/syscalls/BUILD | 2 +- test/syscalls/gtest/BUILD | 2 +- test/syscalls/linux/BUILD | 2 +- test/util/BUILD | 2 +- tools/go_generics/BUILD | 2 +- tools/go_generics/globals/BUILD | 2 +- tools/go_generics/go_merge/BUILD | 2 +- tools/go_generics/rules_tests/BUILD | 2 +- tools/go_stateify/BUILD | 2 +- vdso/BUILD | 2 +- 180 files changed, 180 insertions(+), 180 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/abi/BUILD b/pkg/abi/BUILD index 1ba4f3a46..323263ebf 100644 --- a/pkg/abi/BUILD +++ b/pkg/abi/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/abi/linux/BUILD b/pkg/abi/linux/BUILD index e6043abf4..7648c9469 100644 --- a/pkg/abi/linux/BUILD +++ b/pkg/abi/linux/BUILD @@ -2,7 +2,7 @@ # Linux kernel. It should be used instead of syscall or golang.org/x/sys/unix # when the host OS may not be Linux. -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/amutex/BUILD b/pkg/amutex/BUILD index 7cda07418..bdb6e8f2c 100644 --- a/pkg/amutex/BUILD +++ b/pkg/amutex/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "amutex", diff --git a/pkg/atomicbitops/BUILD b/pkg/atomicbitops/BUILD index 235188531..9555bf645 100644 --- a/pkg/atomicbitops/BUILD +++ b/pkg/atomicbitops/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "atomicbitops", diff --git a/pkg/binary/BUILD b/pkg/binary/BUILD index 571151f72..bd37376b0 100644 --- a/pkg/binary/BUILD +++ b/pkg/binary/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "binary", diff --git a/pkg/bits/BUILD b/pkg/bits/BUILD index 46794bdb8..5214b2c24 100644 --- a/pkg/bits/BUILD +++ b/pkg/bits/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") diff --git a/pkg/bpf/BUILD b/pkg/bpf/BUILD index 564df3af5..3c7ae3103 100644 --- a/pkg/bpf/BUILD +++ b/pkg/bpf/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/compressio/BUILD b/pkg/compressio/BUILD index 72952d735..3a0ac64e6 100644 --- a/pkg/compressio/BUILD +++ b/pkg/compressio/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "compressio", diff --git a/pkg/control/client/BUILD b/pkg/control/client/BUILD index 32853875d..22a4a4a5a 100644 --- a/pkg/control/client/BUILD +++ b/pkg/control/client/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "client", diff --git a/pkg/control/server/BUILD b/pkg/control/server/BUILD index ba2b1be9f..76b2e9787 100644 --- a/pkg/control/server/BUILD +++ b/pkg/control/server/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "server", diff --git a/pkg/cpuid/BUILD b/pkg/cpuid/BUILD index 46fc4703b..29cc38778 100644 --- a/pkg/cpuid/BUILD +++ b/pkg/cpuid/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/dhcp/BUILD b/pkg/dhcp/BUILD index c97dfc14b..003620b48 100644 --- a/pkg/dhcp/BUILD +++ b/pkg/dhcp/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "dhcp", diff --git a/pkg/eventchannel/BUILD b/pkg/eventchannel/BUILD index 18348ef54..5c2a44aa1 100644 --- a/pkg/eventchannel/BUILD +++ b/pkg/eventchannel/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "eventchannel", diff --git a/pkg/fd/BUILD b/pkg/fd/BUILD index 06cfd445e..ab1109157 100644 --- a/pkg/fd/BUILD +++ b/pkg/fd/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "fd", diff --git a/pkg/fdnotifier/BUILD b/pkg/fdnotifier/BUILD index 27d378d5b..8c8d193cc 100644 --- a/pkg/fdnotifier/BUILD +++ b/pkg/fdnotifier/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "fdnotifier", diff --git a/pkg/gate/BUILD b/pkg/gate/BUILD index 9a87a3a31..83679f2da 100644 --- a/pkg/gate/BUILD +++ b/pkg/gate/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "gate", diff --git a/pkg/ilist/BUILD b/pkg/ilist/BUILD index a67aa2cff..dbd65ab12 100644 --- a/pkg/ilist/BUILD +++ b/pkg/ilist/BUILD @@ -1,7 +1,7 @@ load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ilist", diff --git a/pkg/linewriter/BUILD b/pkg/linewriter/BUILD index 3f28ba867..d1aa2e7d6 100644 --- a/pkg/linewriter/BUILD +++ b/pkg/linewriter/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "linewriter", diff --git a/pkg/log/BUILD b/pkg/log/BUILD index 94ac66db3..b2d18eddb 100644 --- a/pkg/log/BUILD +++ b/pkg/log/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "log", diff --git a/pkg/metric/BUILD b/pkg/metric/BUILD index d96e5563b..4b2c7a00e 100644 --- a/pkg/metric/BUILD +++ b/pkg/metric/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "metric", diff --git a/pkg/p9/BUILD b/pkg/p9/BUILD index 2c224e65b..5d972309d 100644 --- a/pkg/p9/BUILD +++ b/pkg/p9/BUILD @@ -2,7 +2,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//visibility:public"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) go_library( diff --git a/pkg/p9/local_server/BUILD b/pkg/p9/local_server/BUILD index b17ebb79d..aa6db186c 100644 --- a/pkg/p9/local_server/BUILD +++ b/pkg/p9/local_server/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "local_server", diff --git a/pkg/p9/p9test/BUILD b/pkg/p9/p9test/BUILD index 7c4b875ce..cf22edde8 100644 --- a/pkg/p9/p9test/BUILD +++ b/pkg/p9/p9test/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) alias( name = "mockgen", diff --git a/pkg/rand/BUILD b/pkg/rand/BUILD index 0c9efc709..4eec3a4dd 100644 --- a/pkg/rand/BUILD +++ b/pkg/rand/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "rand", diff --git a/pkg/refs/BUILD b/pkg/refs/BUILD index 98150ba8f..fc562f821 100644 --- a/pkg/refs/BUILD +++ b/pkg/refs/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/seccomp/BUILD b/pkg/seccomp/BUILD index 657f923ed..0e9c4692d 100644 --- a/pkg/seccomp/BUILD +++ b/pkg/seccomp/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_embed_data") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "victim", diff --git a/pkg/secio/BUILD b/pkg/secio/BUILD index 29f751725..2b4b87c61 100644 --- a/pkg/secio/BUILD +++ b/pkg/secio/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "secio", diff --git a/pkg/segment/BUILD b/pkg/segment/BUILD index 964d73af8..700385907 100644 --- a/pkg/segment/BUILD +++ b/pkg/segment/BUILD @@ -1,6 +1,6 @@ package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) load("//tools/go_generics:defs.bzl", "go_template") diff --git a/pkg/segment/test/BUILD b/pkg/segment/test/BUILD index bdf53e24e..81e929b8c 100644 --- a/pkg/segment/test/BUILD +++ b/pkg/segment/test/BUILD @@ -2,7 +2,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//visibility:private"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/BUILD b/pkg/sentry/BUILD index d18cf3555..53989301f 100644 --- a/pkg/sentry/BUILD +++ b/pkg/sentry/BUILD @@ -1,7 +1,7 @@ # This BUILD file defines a package_group that allows for interdependencies for # sentry-internal packages. -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) package_group( name = "internal", diff --git a/pkg/sentry/arch/BUILD b/pkg/sentry/arch/BUILD index 9bf04360a..0c044bc33 100644 --- a/pkg/sentry/arch/BUILD +++ b/pkg/sentry/arch/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/context/BUILD b/pkg/sentry/context/BUILD index 02d24defd..a3c8d0177 100644 --- a/pkg/sentry/context/BUILD +++ b/pkg/sentry/context/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "context", diff --git a/pkg/sentry/context/contexttest/BUILD b/pkg/sentry/context/contexttest/BUILD index 01bb40b04..bed156b70 100644 --- a/pkg/sentry/context/contexttest/BUILD +++ b/pkg/sentry/context/contexttest/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/control/BUILD b/pkg/sentry/control/BUILD index c3b682d6f..f54e01ee8 100644 --- a/pkg/sentry/control/BUILD +++ b/pkg/sentry/control/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "control", diff --git a/pkg/sentry/device/BUILD b/pkg/sentry/device/BUILD index bebdb2939..01de708d3 100644 --- a/pkg/sentry/device/BUILD +++ b/pkg/sentry/device/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "device", diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD index 6f368b0da..e58333da3 100644 --- a/pkg/sentry/fs/BUILD +++ b/pkg/sentry/fs/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/anon/BUILD b/pkg/sentry/fs/anon/BUILD index 4bd912e95..2111df2e8 100644 --- a/pkg/sentry/fs/anon/BUILD +++ b/pkg/sentry/fs/anon/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "anon", diff --git a/pkg/sentry/fs/ashmem/BUILD b/pkg/sentry/fs/ashmem/BUILD index e5bb661b5..dcf620dca 100644 --- a/pkg/sentry/fs/ashmem/BUILD +++ b/pkg/sentry/fs/ashmem/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/fs/binder/BUILD b/pkg/sentry/fs/binder/BUILD index 27155819e..8a448175f 100644 --- a/pkg/sentry/fs/binder/BUILD +++ b/pkg/sentry/fs/binder/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/dev/BUILD b/pkg/sentry/fs/dev/BUILD index 85371032a..e5b962c8c 100644 --- a/pkg/sentry/fs/dev/BUILD +++ b/pkg/sentry/fs/dev/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/fdpipe/BUILD b/pkg/sentry/fs/fdpipe/BUILD index 8a0937cda..098463e97 100644 --- a/pkg/sentry/fs/fdpipe/BUILD +++ b/pkg/sentry/fs/fdpipe/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/filetest/BUILD b/pkg/sentry/fs/filetest/BUILD index d137fee4c..05ca72aa0 100644 --- a/pkg/sentry/fs/filetest/BUILD +++ b/pkg/sentry/fs/filetest/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/fsutil/BUILD b/pkg/sentry/fs/fsutil/BUILD index d4767642b..7dff970ea 100644 --- a/pkg/sentry/fs/fsutil/BUILD +++ b/pkg/sentry/fs/fsutil/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD index 35ffadd13..f2c79b475 100644 --- a/pkg/sentry/fs/gofer/BUILD +++ b/pkg/sentry/fs/gofer/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD index 6877eb161..ea2ca11bf 100644 --- a/pkg/sentry/fs/host/BUILD +++ b/pkg/sentry/fs/host/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/lock/BUILD b/pkg/sentry/fs/lock/BUILD index 3159ff1da..7164744b8 100644 --- a/pkg/sentry/fs/lock/BUILD +++ b/pkg/sentry/fs/lock/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/proc/BUILD b/pkg/sentry/fs/proc/BUILD index 74954f213..f6bc90634 100644 --- a/pkg/sentry/fs/proc/BUILD +++ b/pkg/sentry/fs/proc/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/proc/device/BUILD b/pkg/sentry/fs/proc/device/BUILD index ff7dacf07..64b0c5a3a 100644 --- a/pkg/sentry/fs/proc/device/BUILD +++ b/pkg/sentry/fs/proc/device/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "device", diff --git a/pkg/sentry/fs/proc/seqfile/BUILD b/pkg/sentry/fs/proc/seqfile/BUILD index b4ba64e10..6b44c0075 100644 --- a/pkg/sentry/fs/proc/seqfile/BUILD +++ b/pkg/sentry/fs/proc/seqfile/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/ramfs/BUILD b/pkg/sentry/fs/ramfs/BUILD index 4a629e38e..f36e4a5e8 100644 --- a/pkg/sentry/fs/ramfs/BUILD +++ b/pkg/sentry/fs/ramfs/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/sys/BUILD b/pkg/sentry/fs/sys/BUILD index 7de928e16..42e98230e 100644 --- a/pkg/sentry/fs/sys/BUILD +++ b/pkg/sentry/fs/sys/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/timerfd/BUILD b/pkg/sentry/fs/timerfd/BUILD index ffdd7e0dc..0e06a5028 100644 --- a/pkg/sentry/fs/timerfd/BUILD +++ b/pkg/sentry/fs/timerfd/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/fs/tmpfs/BUILD b/pkg/sentry/fs/tmpfs/BUILD index c5ec85460..bf5b68869 100644 --- a/pkg/sentry/fs/tmpfs/BUILD +++ b/pkg/sentry/fs/tmpfs/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/fs/tty/BUILD b/pkg/sentry/fs/tty/BUILD index 011cb6955..bee2db3f3 100644 --- a/pkg/sentry/fs/tty/BUILD +++ b/pkg/sentry/fs/tty/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/hostcpu/BUILD b/pkg/sentry/hostcpu/BUILD index 33197cf14..b5067ae6d 100644 --- a/pkg/sentry/hostcpu/BUILD +++ b/pkg/sentry/hostcpu/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "hostcpu", diff --git a/pkg/sentry/inet/BUILD b/pkg/sentry/inet/BUILD index 159c50efb..e288d34e9 100644 --- a/pkg/sentry/inet/BUILD +++ b/pkg/sentry/inet/BUILD @@ -1,6 +1,6 @@ package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD index 7d41626dc..b230aff98 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/auth/BUILD b/pkg/sentry/kernel/auth/BUILD index a81085372..abd4f2dae 100644 --- a/pkg/sentry/kernel/auth/BUILD +++ b/pkg/sentry/kernel/auth/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/contexttest/BUILD b/pkg/sentry/kernel/contexttest/BUILD index 391986291..5769a3b28 100644 --- a/pkg/sentry/kernel/contexttest/BUILD +++ b/pkg/sentry/kernel/contexttest/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/epoll/BUILD b/pkg/sentry/kernel/epoll/BUILD index 5e8b36ed6..1567d5050 100644 --- a/pkg/sentry/kernel/epoll/BUILD +++ b/pkg/sentry/kernel/epoll/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/eventfd/BUILD b/pkg/sentry/kernel/eventfd/BUILD index d96803fc9..f2f1a1223 100644 --- a/pkg/sentry/kernel/eventfd/BUILD +++ b/pkg/sentry/kernel/eventfd/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/fasync/BUILD b/pkg/sentry/kernel/fasync/BUILD index 17749c0de..5faf95909 100644 --- a/pkg/sentry/kernel/fasync/BUILD +++ b/pkg/sentry/kernel/fasync/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/futex/BUILD b/pkg/sentry/kernel/futex/BUILD index afd35985f..da24c36c1 100644 --- a/pkg/sentry/kernel/futex/BUILD +++ b/pkg/sentry/kernel/futex/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/kdefs/BUILD b/pkg/sentry/kernel/kdefs/BUILD index 3f8fa206c..38aaca134 100644 --- a/pkg/sentry/kernel/kdefs/BUILD +++ b/pkg/sentry/kernel/kdefs/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "kdefs", diff --git a/pkg/sentry/kernel/memevent/BUILD b/pkg/sentry/kernel/memevent/BUILD index dfd8dd062..347a69062 100644 --- a/pkg/sentry/kernel/memevent/BUILD +++ b/pkg/sentry/kernel/memevent/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "memevent", diff --git a/pkg/sentry/kernel/pipe/BUILD b/pkg/sentry/kernel/pipe/BUILD index 19b23c6d2..011a3f349 100644 --- a/pkg/sentry/kernel/pipe/BUILD +++ b/pkg/sentry/kernel/pipe/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/sched/BUILD b/pkg/sentry/kernel/sched/BUILD index 52e226a39..184e8a35b 100644 --- a/pkg/sentry/kernel/sched/BUILD +++ b/pkg/sentry/kernel/sched/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sched", diff --git a/pkg/sentry/kernel/semaphore/BUILD b/pkg/sentry/kernel/semaphore/BUILD index bdcf4ce5c..840943ca8 100644 --- a/pkg/sentry/kernel/semaphore/BUILD +++ b/pkg/sentry/kernel/semaphore/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/kernel/shm/BUILD b/pkg/sentry/kernel/shm/BUILD index 40e641355..f45770eef 100644 --- a/pkg/sentry/kernel/shm/BUILD +++ b/pkg/sentry/kernel/shm/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/kernel/time/BUILD b/pkg/sentry/kernel/time/BUILD index 5d8db2273..584f7c7cc 100644 --- a/pkg/sentry/kernel/time/BUILD +++ b/pkg/sentry/kernel/time/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/limits/BUILD b/pkg/sentry/limits/BUILD index 90f4395d4..800166675 100644 --- a/pkg/sentry/limits/BUILD +++ b/pkg/sentry/limits/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/loader/BUILD b/pkg/sentry/loader/BUILD index 24e734b49..1ea260a4e 100644 --- a/pkg/sentry/loader/BUILD +++ b/pkg/sentry/loader/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_embed_data") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/memmap/BUILD b/pkg/sentry/memmap/BUILD index c9e0b95a0..9c2cbd18b 100644 --- a/pkg/sentry/memmap/BUILD +++ b/pkg/sentry/memmap/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/memutil/BUILD b/pkg/sentry/memutil/BUILD index 88738d65d..68b03d4cc 100644 --- a/pkg/sentry/memutil/BUILD +++ b/pkg/sentry/memutil/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "memutil", diff --git a/pkg/sentry/mm/BUILD b/pkg/sentry/mm/BUILD index 0997ec0a7..f679262d0 100644 --- a/pkg/sentry/mm/BUILD +++ b/pkg/sentry/mm/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/platform/BUILD b/pkg/sentry/platform/BUILD index af9ba5394..ac8a6cb7f 100644 --- a/pkg/sentry/platform/BUILD +++ b/pkg/sentry/platform/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/platform/filemem/BUILD b/pkg/sentry/platform/filemem/BUILD index 2a5982763..1a61cfaa5 100644 --- a/pkg/sentry/platform/filemem/BUILD +++ b/pkg/sentry/platform/filemem/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/platform/interrupt/BUILD b/pkg/sentry/platform/interrupt/BUILD index dbafa3204..eeccd4d0e 100644 --- a/pkg/sentry/platform/interrupt/BUILD +++ b/pkg/sentry/platform/interrupt/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "interrupt", diff --git a/pkg/sentry/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD index 1b71e629f..6e40b3177 100644 --- a/pkg/sentry/platform/kvm/BUILD +++ b/pkg/sentry/platform/kvm/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/platform/kvm/testutil/BUILD b/pkg/sentry/platform/kvm/testutil/BUILD index 1dffe94a4..e10087e8e 100644 --- a/pkg/sentry/platform/kvm/testutil/BUILD +++ b/pkg/sentry/platform/kvm/testutil/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "testutil", diff --git a/pkg/sentry/platform/procid/BUILD b/pkg/sentry/platform/procid/BUILD index 20c8bc02c..277509624 100644 --- a/pkg/sentry/platform/procid/BUILD +++ b/pkg/sentry/platform/procid/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "procid", diff --git a/pkg/sentry/platform/ptrace/BUILD b/pkg/sentry/platform/ptrace/BUILD index 2eb354ad4..f86790942 100644 --- a/pkg/sentry/platform/ptrace/BUILD +++ b/pkg/sentry/platform/ptrace/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ptrace", diff --git a/pkg/sentry/platform/ring0/BUILD b/pkg/sentry/platform/ring0/BUILD index c35d49f2d..ecb3e9a9c 100644 --- a/pkg/sentry/platform/ring0/BUILD +++ b/pkg/sentry/platform/ring0/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") diff --git a/pkg/sentry/platform/ring0/gen_offsets/BUILD b/pkg/sentry/platform/ring0/gen_offsets/BUILD index b76d7974e..d7029d5a9 100644 --- a/pkg/sentry/platform/ring0/gen_offsets/BUILD +++ b/pkg/sentry/platform/ring0/gen_offsets/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/platform/ring0/pagetables/BUILD b/pkg/sentry/platform/ring0/pagetables/BUILD index de1b920af..fe93d3030 100644 --- a/pkg/sentry/platform/ring0/pagetables/BUILD +++ b/pkg/sentry/platform/ring0/pagetables/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") diff --git a/pkg/sentry/platform/safecopy/BUILD b/pkg/sentry/platform/safecopy/BUILD index cb8347dd8..05a6a61ae 100644 --- a/pkg/sentry/platform/safecopy/BUILD +++ b/pkg/sentry/platform/safecopy/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0, portions BSD, MIT +package(licenses = ["notice"]) go_library( name = "safecopy", diff --git a/pkg/sentry/safemem/BUILD b/pkg/sentry/safemem/BUILD index 87a9bff12..3ab453718 100644 --- a/pkg/sentry/safemem/BUILD +++ b/pkg/sentry/safemem/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "safemem", diff --git a/pkg/sentry/sighandling/BUILD b/pkg/sentry/sighandling/BUILD index 41313d334..cec3af92e 100644 --- a/pkg/sentry/sighandling/BUILD +++ b/pkg/sentry/sighandling/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sighandling", diff --git a/pkg/sentry/socket/BUILD b/pkg/sentry/socket/BUILD index 3a8044b5f..076f953e7 100644 --- a/pkg/sentry/socket/BUILD +++ b/pkg/sentry/socket/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/control/BUILD b/pkg/sentry/socket/control/BUILD index d3a63f15f..9f4763906 100644 --- a/pkg/sentry/socket/control/BUILD +++ b/pkg/sentry/socket/control/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/epsocket/BUILD b/pkg/sentry/socket/epsocket/BUILD index da4aaf510..45e418db3 100644 --- a/pkg/sentry/socket/epsocket/BUILD +++ b/pkg/sentry/socket/epsocket/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/hostinet/BUILD b/pkg/sentry/socket/hostinet/BUILD index b8dceb102..a469af7ac 100644 --- a/pkg/sentry/socket/hostinet/BUILD +++ b/pkg/sentry/socket/hostinet/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/netlink/BUILD b/pkg/sentry/socket/netlink/BUILD index cff922cb8..148306329 100644 --- a/pkg/sentry/socket/netlink/BUILD +++ b/pkg/sentry/socket/netlink/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/netlink/port/BUILD b/pkg/sentry/socket/netlink/port/BUILD index 3a7dbc5ed..a7370a4ec 100644 --- a/pkg/sentry/socket/netlink/port/BUILD +++ b/pkg/sentry/socket/netlink/port/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/socket/netlink/route/BUILD b/pkg/sentry/socket/netlink/route/BUILD index e1bcfe252..be0419679 100644 --- a/pkg/sentry/socket/netlink/route/BUILD +++ b/pkg/sentry/socket/netlink/route/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/rpcinet/BUILD b/pkg/sentry/socket/rpcinet/BUILD index 06e121946..4da14a1e0 100644 --- a/pkg/sentry/socket/rpcinet/BUILD +++ b/pkg/sentry/socket/rpcinet/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "rpcinet", diff --git a/pkg/sentry/socket/rpcinet/conn/BUILD b/pkg/sentry/socket/rpcinet/conn/BUILD index a16977f29..4336ae9b4 100644 --- a/pkg/sentry/socket/rpcinet/conn/BUILD +++ b/pkg/sentry/socket/rpcinet/conn/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # BSD +package(licenses = ["notice"]) go_library( name = "conn", diff --git a/pkg/sentry/socket/rpcinet/notifier/BUILD b/pkg/sentry/socket/rpcinet/notifier/BUILD index 2bab01774..b0b107ddb 100644 --- a/pkg/sentry/socket/rpcinet/notifier/BUILD +++ b/pkg/sentry/socket/rpcinet/notifier/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # BSD +package(licenses = ["notice"]) go_library( name = "notifier", diff --git a/pkg/sentry/socket/unix/BUILD b/pkg/sentry/socket/unix/BUILD index a12fa93db..fe6871cc6 100644 --- a/pkg/sentry/socket/unix/BUILD +++ b/pkg/sentry/socket/unix/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/socket/unix/transport/BUILD b/pkg/sentry/socket/unix/transport/BUILD index 5a90837bc..5a2de0c4c 100644 --- a/pkg/sentry/socket/unix/transport/BUILD +++ b/pkg/sentry/socket/unix/transport/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/state/BUILD b/pkg/sentry/state/BUILD index f1f6fdb7d..42c459acc 100644 --- a/pkg/sentry/state/BUILD +++ b/pkg/sentry/state/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "state", diff --git a/pkg/sentry/strace/BUILD b/pkg/sentry/strace/BUILD index 8517db1ac..552e79686 100644 --- a/pkg/sentry/strace/BUILD +++ b/pkg/sentry/strace/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "strace", diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD index 35192ff49..6b5469e45 100644 --- a/pkg/sentry/syscalls/BUILD +++ b/pkg/sentry/syscalls/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "syscalls", diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD index 7621bfdbd..846601881 100644 --- a/pkg/sentry/syscalls/linux/BUILD +++ b/pkg/sentry/syscalls/linux/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/time/BUILD b/pkg/sentry/time/BUILD index 1191010e6..c4b6dcc63 100644 --- a/pkg/sentry/time/BUILD +++ b/pkg/sentry/time/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0, portions BSD +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sentry/unimpl/BUILD b/pkg/sentry/unimpl/BUILD index 42e24ace5..b608867a9 100644 --- a/pkg/sentry/unimpl/BUILD +++ b/pkg/sentry/unimpl/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) proto_library( name = "unimplemented_syscall_proto", diff --git a/pkg/sentry/uniqueid/BUILD b/pkg/sentry/uniqueid/BUILD index 0929497c3..ccc5a28d3 100644 --- a/pkg/sentry/uniqueid/BUILD +++ b/pkg/sentry/uniqueid/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "uniqueid", diff --git a/pkg/sentry/usage/BUILD b/pkg/sentry/usage/BUILD index 868dfd400..09198496b 100644 --- a/pkg/sentry/usage/BUILD +++ b/pkg/sentry/usage/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/sentry/usermem/BUILD b/pkg/sentry/usermem/BUILD index dae41ed0e..1a560b6f3 100644 --- a/pkg/sentry/usermem/BUILD +++ b/pkg/sentry/usermem/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/sentry/watchdog/BUILD b/pkg/sentry/watchdog/BUILD index b2c687b20..0bbf3705c 100644 --- a/pkg/sentry/watchdog/BUILD +++ b/pkg/sentry/watchdog/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "watchdog", diff --git a/pkg/sleep/BUILD b/pkg/sleep/BUILD index 338fd9336..2b005bf66 100644 --- a/pkg/sleep/BUILD +++ b/pkg/sleep/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sleep", diff --git a/pkg/state/BUILD b/pkg/state/BUILD index dd0f250fa..0a975e162 100644 --- a/pkg/state/BUILD +++ b/pkg/state/BUILD @@ -1,7 +1,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/state/statefile/BUILD b/pkg/state/statefile/BUILD index 66c8f3807..5967781e8 100644 --- a/pkg/state/statefile/BUILD +++ b/pkg/state/statefile/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "statefile", diff --git a/pkg/sync/BUILD b/pkg/sync/BUILD index 6ddc6e812..1624e681c 100644 --- a/pkg/sync/BUILD +++ b/pkg/sync/BUILD @@ -2,7 +2,7 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0, portions BSD + licenses = ["notice"], ) load("//tools/go_generics:defs.bzl", "go_template") diff --git a/pkg/sync/atomicptrtest/BUILD b/pkg/sync/atomicptrtest/BUILD index 9cb7f66fe..198fbb895 100644 --- a/pkg/sync/atomicptrtest/BUILD +++ b/pkg/sync/atomicptrtest/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/sync/seqatomictest/BUILD b/pkg/sync/seqatomictest/BUILD index 54f8e59b1..23132650a 100644 --- a/pkg/sync/seqatomictest/BUILD +++ b/pkg/sync/seqatomictest/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") diff --git a/pkg/syserr/BUILD b/pkg/syserr/BUILD index 30ae20772..0d65115ef 100644 --- a/pkg/syserr/BUILD +++ b/pkg/syserr/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "syserr", diff --git a/pkg/syserror/BUILD b/pkg/syserror/BUILD index d4c6da97a..ac478d0ff 100644 --- a/pkg/syserror/BUILD +++ b/pkg/syserror/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "syserror", diff --git a/pkg/tcpip/BUILD b/pkg/tcpip/BUILD index daff9a0a0..83524cc8a 100644 --- a/pkg/tcpip/BUILD +++ b/pkg/tcpip/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/adapters/gonet/BUILD b/pkg/tcpip/adapters/gonet/BUILD index 723ad668f..ee2417238 100644 --- a/pkg/tcpip/adapters/gonet/BUILD +++ b/pkg/tcpip/adapters/gonet/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "gonet", diff --git a/pkg/tcpip/buffer/BUILD b/pkg/tcpip/buffer/BUILD index 11a725423..648d12cdf 100644 --- a/pkg/tcpip/buffer/BUILD +++ b/pkg/tcpip/buffer/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/checker/BUILD b/pkg/tcpip/checker/BUILD index a1de808b9..f597d0b24 100644 --- a/pkg/tcpip/checker/BUILD +++ b/pkg/tcpip/checker/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "checker", diff --git a/pkg/tcpip/hash/jenkins/BUILD b/pkg/tcpip/hash/jenkins/BUILD index bbb764db8..ce2194a4d 100644 --- a/pkg/tcpip/hash/jenkins/BUILD +++ b/pkg/tcpip/hash/jenkins/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "jenkins", diff --git a/pkg/tcpip/header/BUILD b/pkg/tcpip/header/BUILD index 8e455fe1e..a5c7290ee 100644 --- a/pkg/tcpip/header/BUILD +++ b/pkg/tcpip/header/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/link/channel/BUILD b/pkg/tcpip/link/channel/BUILD index 25f6c1457..ae285e495 100644 --- a/pkg/tcpip/link/channel/BUILD +++ b/pkg/tcpip/link/channel/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "channel", diff --git a/pkg/tcpip/link/fdbased/BUILD b/pkg/tcpip/link/fdbased/BUILD index a4aa3feec..0d78c9b15 100644 --- a/pkg/tcpip/link/fdbased/BUILD +++ b/pkg/tcpip/link/fdbased/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "fdbased", diff --git a/pkg/tcpip/link/loopback/BUILD b/pkg/tcpip/link/loopback/BUILD index a46ba7f11..710a05ede 100644 --- a/pkg/tcpip/link/loopback/BUILD +++ b/pkg/tcpip/link/loopback/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "loopback", diff --git a/pkg/tcpip/link/rawfile/BUILD b/pkg/tcpip/link/rawfile/BUILD index 2746d4ced..f01bb2c07 100644 --- a/pkg/tcpip/link/rawfile/BUILD +++ b/pkg/tcpip/link/rawfile/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "rawfile", diff --git a/pkg/tcpip/link/sharedmem/BUILD b/pkg/tcpip/link/sharedmem/BUILD index d7f1e66ef..dc8f1543e 100644 --- a/pkg/tcpip/link/sharedmem/BUILD +++ b/pkg/tcpip/link/sharedmem/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sharedmem", diff --git a/pkg/tcpip/link/sharedmem/pipe/BUILD b/pkg/tcpip/link/sharedmem/pipe/BUILD index 12e813509..85deafa38 100644 --- a/pkg/tcpip/link/sharedmem/pipe/BUILD +++ b/pkg/tcpip/link/sharedmem/pipe/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "pipe", diff --git a/pkg/tcpip/link/sharedmem/queue/BUILD b/pkg/tcpip/link/sharedmem/queue/BUILD index 661037bb2..d7dc631eb 100644 --- a/pkg/tcpip/link/sharedmem/queue/BUILD +++ b/pkg/tcpip/link/sharedmem/queue/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "queue", diff --git a/pkg/tcpip/link/sniffer/BUILD b/pkg/tcpip/link/sniffer/BUILD index 52e237c25..7d0d1781e 100644 --- a/pkg/tcpip/link/sniffer/BUILD +++ b/pkg/tcpip/link/sniffer/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sniffer", diff --git a/pkg/tcpip/link/tun/BUILD b/pkg/tcpip/link/tun/BUILD index 5ec01cec9..e54852d3f 100644 --- a/pkg/tcpip/link/tun/BUILD +++ b/pkg/tcpip/link/tun/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "tun", diff --git a/pkg/tcpip/link/waitable/BUILD b/pkg/tcpip/link/waitable/BUILD index ba495c437..89a9eee23 100644 --- a/pkg/tcpip/link/waitable/BUILD +++ b/pkg/tcpip/link/waitable/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "waitable", diff --git a/pkg/tcpip/network/BUILD b/pkg/tcpip/network/BUILD index a2a07f533..f36f49453 100644 --- a/pkg/tcpip/network/BUILD +++ b/pkg/tcpip/network/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_test( name = "ip_test", diff --git a/pkg/tcpip/network/arp/BUILD b/pkg/tcpip/network/arp/BUILD index f6fb7daf7..ef18bb93d 100644 --- a/pkg/tcpip/network/arp/BUILD +++ b/pkg/tcpip/network/arp/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "arp", diff --git a/pkg/tcpip/network/fragmentation/BUILD b/pkg/tcpip/network/fragmentation/BUILD index aaabfcb9a..bf0a7b99c 100644 --- a/pkg/tcpip/network/fragmentation/BUILD +++ b/pkg/tcpip/network/fragmentation/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/network/hash/BUILD b/pkg/tcpip/network/hash/BUILD index 401dce646..ea520c6ed 100644 --- a/pkg/tcpip/network/hash/BUILD +++ b/pkg/tcpip/network/hash/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "hash", diff --git a/pkg/tcpip/network/ipv4/BUILD b/pkg/tcpip/network/ipv4/BUILD index e72317e9f..7a5341def 100644 --- a/pkg/tcpip/network/ipv4/BUILD +++ b/pkg/tcpip/network/ipv4/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ipv4", diff --git a/pkg/tcpip/network/ipv6/BUILD b/pkg/tcpip/network/ipv6/BUILD index 808c37df3..000e00dba 100644 --- a/pkg/tcpip/network/ipv6/BUILD +++ b/pkg/tcpip/network/ipv6/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ipv6", diff --git a/pkg/tcpip/ports/BUILD b/pkg/tcpip/ports/BUILD index a2fa9b84a..3ee80c62b 100644 --- a/pkg/tcpip/ports/BUILD +++ b/pkg/tcpip/ports/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "ports", diff --git a/pkg/tcpip/sample/tun_tcp_connect/BUILD b/pkg/tcpip/sample/tun_tcp_connect/BUILD index 32baf2115..996939581 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/BUILD +++ b/pkg/tcpip/sample/tun_tcp_connect/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "tun_tcp_connect", diff --git a/pkg/tcpip/sample/tun_tcp_echo/BUILD b/pkg/tcpip/sample/tun_tcp_echo/BUILD index 760445843..dad8ef399 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/BUILD +++ b/pkg/tcpip/sample/tun_tcp_echo/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "tun_tcp_echo", diff --git a/pkg/tcpip/seqnum/BUILD b/pkg/tcpip/seqnum/BUILD index c5c889239..a63665efc 100644 --- a/pkg/tcpip/seqnum/BUILD +++ b/pkg/tcpip/seqnum/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD index 8a598c57d..551c3c73e 100644 --- a/pkg/tcpip/stack/BUILD +++ b/pkg/tcpip/stack/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/transport/ping/BUILD b/pkg/tcpip/transport/ping/BUILD index 982b6795c..4d4241d4b 100644 --- a/pkg/tcpip/transport/ping/BUILD +++ b/pkg/tcpip/transport/ping/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library") diff --git a/pkg/tcpip/transport/tcp/BUILD b/pkg/tcpip/transport/tcp/BUILD index 726107739..e5c05f8c0 100644 --- a/pkg/tcpip/transport/tcp/BUILD +++ b/pkg/tcpip/transport/tcp/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tcpip/transport/tcp/testing/context/BUILD b/pkg/tcpip/transport/tcp/testing/context/BUILD index 814e5c1ea..1584e4095 100644 --- a/pkg/tcpip/transport/tcp/testing/context/BUILD +++ b/pkg/tcpip/transport/tcp/testing/context/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "context", diff --git a/pkg/tcpip/transport/tcpconntrack/BUILD b/pkg/tcpip/transport/tcpconntrack/BUILD index ac1a94d4d..31a845dee 100644 --- a/pkg/tcpip/transport/tcpconntrack/BUILD +++ b/pkg/tcpip/transport/tcpconntrack/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "tcpconntrack", diff --git a/pkg/tcpip/transport/udp/BUILD b/pkg/tcpip/transport/udp/BUILD index 4225e28dc..8ccb79c48 100644 --- a/pkg/tcpip/transport/udp/BUILD +++ b/pkg/tcpip/transport/udp/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template_instance") load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/pkg/tmutex/BUILD b/pkg/tmutex/BUILD index c20df7005..69035044d 100644 --- a/pkg/tmutex/BUILD +++ b/pkg/tmutex/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "tmutex", diff --git a/pkg/unet/BUILD b/pkg/unet/BUILD index f90e43c89..5e177e78e 100644 --- a/pkg/unet/BUILD +++ b/pkg/unet/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "unet", diff --git a/pkg/urpc/BUILD b/pkg/urpc/BUILD index 21008cf6c..36cae67e1 100644 --- a/pkg/urpc/BUILD +++ b/pkg/urpc/BUILD @@ -1,6 +1,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "urpc", diff --git a/pkg/waiter/BUILD b/pkg/waiter/BUILD index 5e611c54f..b748246da 100644 --- a/pkg/waiter/BUILD +++ b/pkg/waiter/BUILD @@ -1,4 +1,4 @@ -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_stateify:defs.bzl", "go_library", "go_test") diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD index 15a7cdae1..540e99151 100644 --- a/runsc/boot/BUILD +++ b/runsc/boot/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "boot", diff --git a/runsc/boot/filter/BUILD b/runsc/boot/filter/BUILD index 004222242..3b6020cf3 100644 --- a/runsc/boot/filter/BUILD +++ b/runsc/boot/filter/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "filter", diff --git a/runsc/cgroup/BUILD b/runsc/cgroup/BUILD index 4f9a25a25..620d33a19 100644 --- a/runsc/cgroup/BUILD +++ b/runsc/cgroup/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "cgroup", diff --git a/runsc/cmd/BUILD b/runsc/cmd/BUILD index a908172af..9e2be0d37 100644 --- a/runsc/cmd/BUILD +++ b/runsc/cmd/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "cmd", diff --git a/runsc/console/BUILD b/runsc/console/BUILD index ff4ccff69..3ff9eba27 100644 --- a/runsc/console/BUILD +++ b/runsc/console/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "console", diff --git a/runsc/container/BUILD b/runsc/container/BUILD index 354ce2661..3b25ff79a 100644 --- a/runsc/container/BUILD +++ b/runsc/container/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "container", diff --git a/runsc/fsgofer/BUILD b/runsc/fsgofer/BUILD index 756c20ad7..4adc9c1bc 100644 --- a/runsc/fsgofer/BUILD +++ b/runsc/fsgofer/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "fsgofer", diff --git a/runsc/fsgofer/filter/BUILD b/runsc/fsgofer/filter/BUILD index c7848d10c..78c5b526c 100644 --- a/runsc/fsgofer/filter/BUILD +++ b/runsc/fsgofer/filter/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "filter", diff --git a/runsc/sandbox/BUILD b/runsc/sandbox/BUILD index 899fd99de..2ed793333 100644 --- a/runsc/sandbox/BUILD +++ b/runsc/sandbox/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "sandbox", diff --git a/runsc/specutils/BUILD b/runsc/specutils/BUILD index 77a10e2b6..372799850 100644 --- a/runsc/specutils/BUILD +++ b/runsc/specutils/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "specutils", diff --git a/runsc/test/image/BUILD b/runsc/test/image/BUILD index 22b3ebd2a..e8b629c6a 100644 --- a/runsc/test/image/BUILD +++ b/runsc/test/image/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_test( name = "image_test", diff --git a/runsc/test/integration/BUILD b/runsc/test/integration/BUILD index e7204dc66..779d30ec9 100644 --- a/runsc/test/integration/BUILD +++ b/runsc/test/integration/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_test( name = "integration_test", diff --git a/runsc/test/root/BUILD b/runsc/test/root/BUILD index 77dcbd79e..75826a521 100644 --- a/runsc/test/root/BUILD +++ b/runsc/test/root/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "root", diff --git a/runsc/test/root/testdata/BUILD b/runsc/test/root/testdata/BUILD index 6c9fe0aea..7f272dcd3 100644 --- a/runsc/test/root/testdata/BUILD +++ b/runsc/test/root/testdata/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "testdata", diff --git a/runsc/test/testutil/BUILD b/runsc/test/testutil/BUILD index 8c3919320..ddec81444 100644 --- a/runsc/test/testutil/BUILD +++ b/runsc/test/testutil/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "testutil", diff --git a/runsc/tools/dockercfg/BUILD b/runsc/tools/dockercfg/BUILD index a80b3abab..fd406ab93 100644 --- a/runsc/tools/dockercfg/BUILD +++ b/runsc/tools/dockercfg/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "dockercfg", diff --git a/test/syscalls/BUILD b/test/syscalls/BUILD index 8c391c8a6..148d9c366 100644 --- a/test/syscalls/BUILD +++ b/test/syscalls/BUILD @@ -1,7 +1,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") load("//test/syscalls:build_defs.bzl", "syscall_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) syscall_test(test = "//test/syscalls/linux:32bit_test") diff --git a/test/syscalls/gtest/BUILD b/test/syscalls/gtest/BUILD index d078fd3d5..22e061652 100644 --- a/test/syscalls/gtest/BUILD +++ b/test/syscalls/gtest/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "gtest", diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD index e70742875..a311ca12c 100644 --- a/test/syscalls/linux/BUILD +++ b/test/syscalls/linux/BUILD @@ -1,6 +1,6 @@ package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) cc_binary( diff --git a/test/util/BUILD b/test/util/BUILD index f2e563507..fac0730b4 100644 --- a/test/util/BUILD +++ b/test/util/BUILD @@ -1,6 +1,6 @@ package( default_visibility = ["//:sandbox"], - licenses = ["notice"], # Apache 2.0 + licenses = ["notice"], ) cc_library( diff --git a/tools/go_generics/BUILD b/tools/go_generics/BUILD index 2d97d99dc..39318b877 100644 --- a/tools/go_generics/BUILD +++ b/tools/go_generics/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "go_generics", diff --git a/tools/go_generics/globals/BUILD b/tools/go_generics/globals/BUILD index c26ac56d2..6628132f5 100644 --- a/tools/go_generics/globals/BUILD +++ b/tools/go_generics/globals/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_library( name = "globals", diff --git a/tools/go_generics/go_merge/BUILD b/tools/go_generics/go_merge/BUILD index a60437962..02b09120e 100644 --- a/tools/go_generics/go_merge/BUILD +++ b/tools/go_generics/go_merge/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "go_merge", diff --git a/tools/go_generics/rules_tests/BUILD b/tools/go_generics/rules_tests/BUILD index 23b2d656d..a6f8cdd3c 100644 --- a/tools/go_generics/rules_tests/BUILD +++ b/tools/go_generics/rules_tests/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) load("//tools/go_generics:defs.bzl", "go_template", "go_template_instance") diff --git a/tools/go_stateify/BUILD b/tools/go_stateify/BUILD index 68d37f5d7..bb53f8ae9 100644 --- a/tools/go_stateify/BUILD +++ b/tools/go_stateify/BUILD @@ -1,6 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_binary") -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) go_binary( name = "stateify", diff --git a/vdso/BUILD b/vdso/BUILD index fd395511c..c43d24070 100644 --- a/vdso/BUILD +++ b/vdso/BUILD @@ -3,7 +3,7 @@ # normal system VDSO (time, gettimeofday, clock_gettimeofday) but which uses # timekeeping parameters managed by the sandbox kernel. -package(licenses = ["notice"]) # Apache 2.0 +package(licenses = ["notice"]) genrule( name = "vdso", -- cgit v1.2.3 From 81f4829d1195276d037f8bd23a2ef69e88f5ae6c Mon Sep 17 00:00:00 2001 From: Rahat Mahmood Date: Wed, 20 Mar 2019 14:30:00 -0700 Subject: Record sockets created during accept(2) for all families. Track new sockets created during accept(2) in the socket table for all families. Previously we were only doing this for unix domain sockets. PiperOrigin-RevId: 239475550 Change-Id: I16f009f24a06245bfd1d72ffd2175200f837c6ac --- pkg/sentry/socket/epsocket/epsocket.go | 2 ++ pkg/sentry/socket/hostinet/socket.go | 14 ++++++++------ pkg/sentry/socket/rpcinet/socket.go | 3 +++ 3 files changed, 13 insertions(+), 6 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 468e65373..e74bd1bdd 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -504,6 +504,8 @@ func (s *SocketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, } fd, e := t.FDMap().NewFDFrom(0, ns, fdFlags, t.ThreadGroup().Limits()) + t.Kernel().RecordSocket(ns, s.family) + return fd, addr, addrLen, syserr.FromError(e) } diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index 2c54e8de2..a0a8a3220 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -53,14 +53,15 @@ type socketOperations struct { fsutil.FileNoMMap `state:"nosave"` socket.SendReceiveTimeout - fd int // must be O_NONBLOCK - queue waiter.Queue + family int // Read-only. + fd int // must be O_NONBLOCK + queue waiter.Queue } var _ = socket.Socket(&socketOperations{}) -func newSocketFile(ctx context.Context, fd int, nonblock bool) (*fs.File, *syserr.Error) { - s := &socketOperations{fd: fd} +func newSocketFile(ctx context.Context, family int, fd int, nonblock bool) (*fs.File, *syserr.Error) { + s := &socketOperations{family: family, fd: fd} if err := fdnotifier.AddFD(int32(fd), &s.queue); err != nil { return nil, syserr.FromError(err) } @@ -218,7 +219,7 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, return 0, peerAddr, peerAddrlen, syserr.FromError(syscallErr) } - f, err := newSocketFile(t, fd, flags&syscall.SOCK_NONBLOCK != 0) + f, err := newSocketFile(t, s.family, fd, flags&syscall.SOCK_NONBLOCK != 0) if err != nil { syscall.Close(fd) return 0, nil, 0, err @@ -229,6 +230,7 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, CloseOnExec: flags&syscall.SOCK_CLOEXEC != 0, } kfd, kerr := t.FDMap().NewFDFrom(0, f, fdFlags, t.ThreadGroup().Limits()) + t.Kernel().RecordSocket(f, s.family) return kfd, peerAddr, peerAddrlen, syserr.FromError(kerr) } @@ -552,7 +554,7 @@ func (p *socketProvider) Socket(t *kernel.Task, stypeflags transport.SockType, p if err != nil { return nil, syserr.FromError(err) } - return newSocketFile(t, fd, stypeflags&syscall.SOCK_NONBLOCK != 0) + return newSocketFile(t, p.family, fd, stypeflags&syscall.SOCK_NONBLOCK != 0) } // Pair implements socket.Provider.Pair. diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 13681100e..548a22f32 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -52,6 +52,7 @@ type socketOperations struct { fsutil.FileNoMMap `state:"nosave"` socket.SendReceiveTimeout + family int // Read-only. fd uint32 // must be O_NONBLOCK wq *waiter.Queue rpcConn *conn.RPCConnection @@ -83,6 +84,7 @@ func newSocketFile(ctx context.Context, stack *Stack, family int, skType int, pr dirent := socket.NewDirent(ctx, socketDevice) defer dirent.DecRef() return fs.NewFile(ctx, dirent, fs.FileFlags{Read: true, Write: true}, &socketOperations{ + family: family, wq: &wq, fd: fd, rpcConn: stack.rpcConn, @@ -329,6 +331,7 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, if err != nil { return 0, nil, 0, syserr.FromError(err) } + t.Kernel().RecordSocket(file, s.family) if peerRequested { return fd, payload.Address.Address, payload.Address.Length, nil -- cgit v1.2.3 From 4209edafb6a9eeff8741a4360100557179b47b35 Mon Sep 17 00:00:00 2001 From: Jamie Liu Date: Thu, 11 Apr 2019 00:41:42 -0700 Subject: Use open fids when fstat()ing gofer files. PiperOrigin-RevId: 243018347 Change-Id: I1e5b80607c1df0747482abea61db7fcf24536d37 --- pkg/sentry/fs/ashmem/area.go | 9 ++++---- pkg/sentry/fs/binder/binder.go | 7 +++--- pkg/sentry/fs/dev/full.go | 19 +++++++-------- pkg/sentry/fs/dev/null.go | 40 +++++++++++++++++--------------- pkg/sentry/fs/dev/random.go | 19 +++++++-------- pkg/sentry/fs/fdpipe/pipe.go | 15 ++++++------ pkg/sentry/fs/file.go | 12 ++++++++++ pkg/sentry/fs/file_operations.go | 6 +++++ pkg/sentry/fs/file_overlay.go | 26 +++++++++++++++++++++ pkg/sentry/fs/filetest/filetest.go | 17 +++++++------- pkg/sentry/fs/fsutil/file.go | 35 ++++++++++++++++++---------- pkg/sentry/fs/fsutil/inode.go | 19 +++++++-------- pkg/sentry/fs/gofer/file.go | 16 +++++++++++++ pkg/sentry/fs/host/file.go | 5 ++-- pkg/sentry/fs/inotify.go | 5 ++++ pkg/sentry/fs/proc/exec_args.go | 19 +++++++-------- pkg/sentry/fs/proc/fds.go | 21 +++++++++-------- pkg/sentry/fs/proc/proc.go | 3 ++- pkg/sentry/fs/proc/rpcinet_proc.go | 17 +++++++------- pkg/sentry/fs/proc/seqfile/seqfile.go | 17 +++++++------- pkg/sentry/fs/proc/sys.go | 19 +++++++-------- pkg/sentry/fs/proc/sys_net.go | 34 ++++++++++++++------------- pkg/sentry/fs/proc/task.go | 41 ++++++++++++++++++--------------- pkg/sentry/fs/proc/uid_gid_map.go | 17 +++++++------- pkg/sentry/fs/proc/uptime.go | 19 +++++++-------- pkg/sentry/fs/ramfs/dir.go | 3 ++- pkg/sentry/fs/ramfs/socket.go | 21 +++++++++-------- pkg/sentry/fs/ramfs/symlink.go | 21 +++++++++-------- pkg/sentry/fs/timerfd/timerfd.go | 13 ++++++----- pkg/sentry/fs/tmpfs/file_regular.go | 15 ++++++------ pkg/sentry/fs/tty/dir.go | 15 ++++++------ pkg/sentry/fs/tty/master.go | 11 +++++---- pkg/sentry/fs/tty/slave.go | 11 +++++---- pkg/sentry/kernel/epoll/epoll.go | 13 ++++++----- pkg/sentry/kernel/eventfd/eventfd.go | 15 ++++++------ pkg/sentry/kernel/pipe/reader_writer.go | 11 +++++---- pkg/sentry/loader/vdso.go | 17 +++++++------- pkg/sentry/socket/epsocket/epsocket.go | 11 +++++---- pkg/sentry/socket/hostinet/socket.go | 11 +++++---- pkg/sentry/socket/netlink/socket.go | 11 +++++---- pkg/sentry/socket/rpcinet/socket.go | 11 +++++---- pkg/sentry/socket/unix/unix.go | 11 +++++---- pkg/sentry/syscalls/linux/sys_stat.go | 37 +++++++++++++++++++---------- 43 files changed, 422 insertions(+), 293 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go index 710b5185f..651cbc164 100644 --- a/pkg/sentry/fs/ashmem/area.go +++ b/pkg/sentry/fs/ashmem/area.go @@ -42,10 +42,11 @@ const ( // // +stateify savable type Area struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` ad *Device diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go index 188353961..a41b5dcae 100644 --- a/pkg/sentry/fs/binder/binder.go +++ b/pkg/sentry/fs/binder/binder.go @@ -85,9 +85,10 @@ func (bd *Device) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) // // +stateify savable type Proc struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` bd *Device task *kernel.Task diff --git a/pkg/sentry/fs/dev/full.go b/pkg/sentry/fs/dev/full.go index 0cb513004..82da9aae9 100644 --- a/pkg/sentry/fs/dev/full.go +++ b/pkg/sentry/fs/dev/full.go @@ -59,15 +59,16 @@ func (f *fullDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.Fi // +stateify savable type fullFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - readZeros `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` + readZeros `state:"nosave"` } var _ fs.FileOperations = (*fullFileOperations)(nil) diff --git a/pkg/sentry/fs/dev/null.go b/pkg/sentry/fs/dev/null.go index 83f43c203..5d306d352 100644 --- a/pkg/sentry/fs/dev/null.go +++ b/pkg/sentry/fs/dev/null.go @@ -60,16 +60,17 @@ func (n *nullDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.Fi // +stateify savable type nullFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRead `state:"nosave"` - fsutil.FileNoopWrite `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRead `state:"nosave"` + fsutil.FileNoopWrite `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` } var _ fs.FileOperations = (*nullFileOperations)(nil) @@ -100,15 +101,16 @@ func (zd *zeroDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.F // +stateify savable type zeroFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoopWrite `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - readZeros `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopWrite `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` + readZeros `state:"nosave"` } var _ fs.FileOperations = (*zeroFileOperations)(nil) diff --git a/pkg/sentry/fs/dev/random.go b/pkg/sentry/fs/dev/random.go index b9b78db7a..ffd5cf6c3 100644 --- a/pkg/sentry/fs/dev/random.go +++ b/pkg/sentry/fs/dev/random.go @@ -57,15 +57,16 @@ func (*randomDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.Fi // +stateify savable type randomFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoopWrite `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopWrite `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` } var _ fs.FileOperations = (*randomFileOperations)(nil) diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go index b4d11cb45..98483ab68 100644 --- a/pkg/sentry/fs/fdpipe/pipe.go +++ b/pkg/sentry/fs/fdpipe/pipe.go @@ -37,13 +37,14 @@ import ( // // +stateify savable type pipeOperations struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - waiter.Queue `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.Queue `state:"nosave"` // flags are the flags used to open the pipe. flags fs.FileFlags `state:".(fs.FileFlags)"` diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go index 01c18647c..2c2126f17 100644 --- a/pkg/sentry/fs/file.go +++ b/pkg/sentry/fs/file.go @@ -408,6 +408,18 @@ func (f *File) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpts) error { return f.FileOperations.ConfigureMMap(ctx, f, opts) } +// UnstableAttr calls f.FileOperations.UnstableAttr with f as the File. +// +// Returns syserror.ErrInterrupted if interrupted. +func (f *File) UnstableAttr(ctx context.Context) (UnstableAttr, error) { + if !f.mu.Lock(ctx) { + return UnstableAttr{}, syserror.ErrInterrupted + } + defer f.mu.Unlock() + + return f.FileOperations.UnstableAttr(ctx, f) +} + // MappedName implements memmap.MappingIdentity.MappedName. func (f *File) MappedName(ctx context.Context) string { root := RootFromContext(ctx) diff --git a/pkg/sentry/fs/file_operations.go b/pkg/sentry/fs/file_operations.go index 81c6e2b5d..e0fa5135f 100644 --- a/pkg/sentry/fs/file_operations.go +++ b/pkg/sentry/fs/file_operations.go @@ -96,6 +96,12 @@ type FileOperations interface { // memmap.Mappable. ConfigureMMap(ctx context.Context, file *File, opts *memmap.MMapOpts) error + // UnstableAttr returns the "unstable" attributes of the inode represented + // by the file. Most implementations can embed + // fsutil.FileUseInodeUnstableAttr, which delegates to + // InodeOperations.UnstableAttr. + UnstableAttr(ctx context.Context, file *File) (UnstableAttr, error) + // Ioctl implements the ioctl(2) linux syscall. // // io provides access to the virtual memory space to which pointers in args diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go index 4efe85832..e1f02f0f4 100644 --- a/pkg/sentry/fs/file_overlay.go +++ b/pkg/sentry/fs/file_overlay.go @@ -339,6 +339,32 @@ func (*overlayFileOperations) ConfigureMMap(ctx context.Context, file *File, opt return nil } +// UnstableAttr implements fs.FileOperations.UnstableAttr. +func (f *overlayFileOperations) UnstableAttr(ctx context.Context, file *File) (UnstableAttr, error) { + // Hot path. Avoid defers. + f.upperMu.Lock() + if f.upper != nil { + attr, err := f.upper.UnstableAttr(ctx) + f.upperMu.Unlock() + return attr, err + } + f.upperMu.Unlock() + + // It's possible that copy-up has occurred, but we haven't opened a upper + // file yet. If this is the case, just use the upper inode's UnstableAttr + // rather than opening a file. + o := file.Dirent.Inode.overlay + o.copyMu.RLock() + if o.upper != nil { + attr, err := o.upper.UnstableAttr(ctx) + o.copyMu.RUnlock() + return attr, err + } + o.copyMu.RUnlock() + + return f.lower.UnstableAttr(ctx) +} + // Ioctl implements fs.FileOperations.Ioctl and always returns ENOTTY. func (*overlayFileOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { return 0, syserror.ENOTTY diff --git a/pkg/sentry/fs/filetest/filetest.go b/pkg/sentry/fs/filetest/filetest.go index 40d84d9f2..388a1ce36 100644 --- a/pkg/sentry/fs/filetest/filetest.go +++ b/pkg/sentry/fs/filetest/filetest.go @@ -31,14 +31,15 @@ import ( // TestFileOperations is an implementation of the File interface. It provides all // required methods. type TestFileOperations struct { - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` } // NewTestFile creates and initializes a new test file. diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index ce329b37a..df34dc788 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -224,7 +224,7 @@ func (FileNoIoctl) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallAr } // DirFileOperations implements most of fs.FileOperations for directories, -// except for Readdir which the embedding type must implement. +// except for Readdir and UnstableAttr which the embedding type must implement. type DirFileOperations struct { waiter.AlwaysReady FileGenericSeek @@ -250,7 +250,8 @@ func (*DirFileOperations) Write(context.Context, *fs.File, usermem.IOSequence, i // // +stateify savable type StaticDirFileOperations struct { - DirFileOperations + DirFileOperations `state:"nosave"` + FileUseInodeUnstableAttr `state:"nosave"` // dentryMap is a SortedDentryMap used to implement Readdir. dentryMap *fs.SortedDentryMap @@ -291,16 +292,17 @@ func (sdfo *StaticDirFileOperations) Readdir(ctx context.Context, file *fs.File, // // +stateify savable type NoReadWriteFile struct { - waiter.AlwaysReady `state:"nosave"` - FileGenericSeek `state:"nosave"` - FileNoIoctl `state:"nosave"` - FileNoMMap `state:"nosave"` - FileNoopFsync `state:"nosave"` - FileNoopFlush `state:"nosave"` - FileNoopRelease `state:"nosave"` - FileNoRead `state:"nosave"` - FileNoWrite `state:"nosave"` - FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + FileGenericSeek `state:"nosave"` + FileNoIoctl `state:"nosave"` + FileNoMMap `state:"nosave"` + FileNoopFsync `state:"nosave"` + FileNoopFlush `state:"nosave"` + FileNoopRelease `state:"nosave"` + FileNoRead `state:"nosave"` + FileNoWrite `state:"nosave"` + FileNotDirReaddir `state:"nosave"` + FileUseInodeUnstableAttr `state:"nosave"` } var _ fs.FileOperations = (*NoReadWriteFile)(nil) @@ -365,3 +367,12 @@ type FileNoopRead struct{} func (FileNoopRead) Read(context.Context, *fs.File, usermem.IOSequence, int64) (int64, error) { return 0, nil } + +// FileUseInodeUnstableAttr implements fs.FileOperations.UnstableAttr by calling +// InodeOperations.UnstableAttr. +type FileUseInodeUnstableAttr struct{} + +// UnstableAttr implements fs.FileOperations.UnstableAttr. +func (FileUseInodeUnstableAttr) UnstableAttr(ctx context.Context, file *fs.File) (fs.UnstableAttr, error) { + return file.Dirent.Inode.UnstableAttr(ctx) +} diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go index 37490e5b2..468171a9b 100644 --- a/pkg/sentry/fs/fsutil/inode.go +++ b/pkg/sentry/fs/fsutil/inode.go @@ -238,15 +238,16 @@ func (i *InodeSimpleExtendedAttributes) Listxattr(_ *fs.Inode) (map[string]struc // // +stateify savable type staticFile struct { - waiter.AlwaysReady `state:"nosave"` - FileGenericSeek `state:"nosave"` - FileNoIoctl `state:"nosave"` - FileNoMMap `state:"nosave"` - FileNoopFsync `state:"nosave"` - FileNoopFlush `state:"nosave"` - FileNoopRelease `state:"nosave"` - FileNoopWrite `state:"nosave"` - FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + FileGenericSeek `state:"nosave"` + FileNoIoctl `state:"nosave"` + FileNoMMap `state:"nosave"` + FileNoopFsync `state:"nosave"` + FileNoopFlush `state:"nosave"` + FileNoopRelease `state:"nosave"` + FileNoopWrite `state:"nosave"` + FileNotDirReaddir `state:"nosave"` + FileUseInodeUnstableAttr `state:"nosave"` FileStaticContentReader } diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go index e49ae2201..80d1e08a6 100644 --- a/pkg/sentry/fs/gofer/file.go +++ b/pkg/sentry/fs/gofer/file.go @@ -310,6 +310,22 @@ func (f *fileOperations) ConfigureMMap(ctx context.Context, file *fs.File, opts return f.inodeOperations.configureMMap(file, opts) } +// UnstableAttr implements fs.FileOperations.UnstableAttr. +func (f *fileOperations) UnstableAttr(ctx context.Context, file *fs.File) (fs.UnstableAttr, error) { + s := f.inodeOperations.session() + if s.cachePolicy.cacheUAttrs(file.Dirent.Inode) { + return f.inodeOperations.cachingInodeOps.UnstableAttr(ctx, file.Dirent.Inode) + } + // Use f.handles.File, which represents 9P fids that have been opened, + // instead of inodeFileState.file, which represents 9P fids that have not. + // This may be significantly more efficient in some implementations. + _, valid, pattr, err := getattr(ctx, f.handles.File) + if err != nil { + return fs.UnstableAttr{}, err + } + return unstable(ctx, valid, pattr, s.mounter, s.client), nil +} + // Seek implements fs.FileOperations.Seek. func (f *fileOperations) Seek(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64) (int64, error) { return fsutil.SeekWithDirCursor(ctx, file, whence, offset, &f.dirCursor) diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go index d67a0795f..2a8f285ff 100644 --- a/pkg/sentry/fs/host/file.go +++ b/pkg/sentry/fs/host/file.go @@ -36,8 +36,9 @@ import ( // // +stateify savable type fileOperations struct { - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // iops are the Inode operations for this file. iops *inodeOperations `state:"wait"` diff --git a/pkg/sentry/fs/inotify.go b/pkg/sentry/fs/inotify.go index 5d6a7074b..59fa662f3 100644 --- a/pkg/sentry/fs/inotify.go +++ b/pkg/sentry/fs/inotify.go @@ -186,6 +186,11 @@ func (*Inotify) ConfigureMMap(context.Context, *File, *memmap.MMapOpts) error { return syserror.ENODEV } +// UnstableAttr implements FileOperations.UnstableAttr. +func (i *Inotify) UnstableAttr(ctx context.Context, file *File) (UnstableAttr, error) { + return file.Dirent.Inode.UnstableAttr(ctx) +} + // Ioctl implements fs.FileOperations.Ioctl. func (i *Inotify) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { switch args[1].Int() { diff --git a/pkg/sentry/fs/proc/exec_args.go b/pkg/sentry/fs/proc/exec_args.go index 9daad5d2b..fc21dfbbd 100644 --- a/pkg/sentry/fs/proc/exec_args.go +++ b/pkg/sentry/fs/proc/exec_args.go @@ -77,15 +77,16 @@ func (i *execArgInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs. // +stateify savable type execArgFile struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopWrite `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopWrite `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // arg is the type of exec argument this file contains. arg execArgType diff --git a/pkg/sentry/fs/proc/fds.go b/pkg/sentry/fs/proc/fds.go index 3c471bad9..939ebaba1 100644 --- a/pkg/sentry/fs/proc/fds.go +++ b/pkg/sentry/fs/proc/fds.go @@ -91,7 +91,7 @@ func readDescriptors(t *kernel.Task, c *fs.DirCtx, offset int64, toDentAttr func // fd implements fs.InodeOperations for a file in /proc/TID/fd/. type fd struct { ramfs.Symlink - *fs.File + file *fs.File } var _ fs.InodeOperations = (*fd)(nil) @@ -103,7 +103,7 @@ func newFd(t *kernel.Task, f *fs.File, msrc *fs.MountSource) *fs.Inode { fd := &fd{ // RootOwner overridden by taskOwnedInodeOps.UnstableAttrs(). Symlink: *ramfs.NewSymlink(t, fs.RootOwner, ""), - File: f, + file: f, } return newProcInode(fd, msrc, fs.Symlink, t) } @@ -112,8 +112,8 @@ func newFd(t *kernel.Task, f *fs.File, msrc *fs.MountSource) *fs.Inode { // arguments are ignored. func (f *fd) GetFile(context.Context, *fs.Dirent, fs.FileFlags) (*fs.File, error) { // Take a reference on the fs.File. - f.File.IncRef() - return f.File, nil + f.file.IncRef() + return f.file, nil } // Readlink returns the current target. @@ -122,14 +122,14 @@ func (f *fd) Readlink(ctx context.Context, _ *fs.Inode) (string, error) { if root != nil { defer root.DecRef() } - n, _ := f.Dirent.FullName(root) + n, _ := f.file.Dirent.FullName(root) return n, nil } // Getlink implements fs.InodeOperations.Getlink. func (f *fd) Getlink(context.Context, *fs.Inode) (*fs.Dirent, error) { - f.Dirent.IncRef() - return f.Dirent, nil + f.file.Dirent.IncRef() + return f.file.Dirent, nil } // Truncate is ignored. @@ -139,12 +139,12 @@ func (f *fd) Truncate(context.Context, *fs.Inode, int64) error { func (f *fd) Release(ctx context.Context) { f.Symlink.Release(ctx) - f.File.DecRef() + f.file.DecRef() } // Close releases the reference on the file. func (f *fd) Close() error { - f.DecRef() + f.file.DecRef() return nil } @@ -212,7 +212,8 @@ func (f *fdDir) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFla // +stateify savable type fdDirFile struct { - fsutil.DirFileOperations `state:"nosave"` + fsutil.DirFileOperations `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` isInfoFile bool diff --git a/pkg/sentry/fs/proc/proc.go b/pkg/sentry/fs/proc/proc.go index c9e659533..64e1e1998 100644 --- a/pkg/sentry/fs/proc/proc.go +++ b/pkg/sentry/fs/proc/proc.go @@ -191,7 +191,8 @@ func (p *proc) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlag // // +stateify savable type rootProcFile struct { - fsutil.DirFileOperations `state:"nosave"` + fsutil.DirFileOperations `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` iops *proc } diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go index 65faa21f2..81f64a28b 100644 --- a/pkg/sentry/fs/proc/rpcinet_proc.go +++ b/pkg/sentry/fs/proc/rpcinet_proc.go @@ -60,14 +60,15 @@ func (i *rpcInetInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs. // rpcInetFile implements fs.FileOperations as RPCs. type rpcInetFile struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` inode *rpcInetInode } diff --git a/pkg/sentry/fs/proc/seqfile/seqfile.go b/pkg/sentry/fs/proc/seqfile/seqfile.go index 16fc6789e..0a0eb45e2 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile.go @@ -183,14 +183,15 @@ func (s *SeqFile) updateSourceLocked(ctx context.Context, record int) { // // +stateify savable type seqFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` seqFile *SeqFile } diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go index ee6b9f262..a7bc9198e 100644 --- a/pkg/sentry/fs/proc/sys.go +++ b/pkg/sentry/fs/proc/sys.go @@ -134,15 +134,16 @@ var _ fs.InodeOperations = (*hostname)(nil) // +stateify savable type hostnameFile struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoSeek `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSeek `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` } // Read implements fs.FileOperations.Read. diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index 42e9bc47f..728a46a74 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -85,14 +85,15 @@ func (m *tcpMemInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.F // +stateify savable type tcpMemFile struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` tcpMemInode *tcpMemInode } @@ -197,14 +198,15 @@ func (s *tcpSack) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileF // +stateify savable type tcpSackFile struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` tcpSack *tcpSack diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go index 4b1f84942..0edcdfce2 100644 --- a/pkg/sentry/fs/proc/task.go +++ b/pkg/sentry/fs/proc/task.go @@ -142,7 +142,8 @@ func (s *subtasks) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.File // +stateify savable type subtasksFile struct { - fsutil.DirFileOperations `state:"nosave"` + fsutil.DirFileOperations `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` t *kernel.Task pidns *kernel.PIDNamespace @@ -669,15 +670,16 @@ func (c *comm) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlag // +stateify savable type commFile struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` t *kernel.Task } @@ -724,15 +726,16 @@ func (a *auxvec) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl // +stateify savable type auxvecFile struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` t *kernel.Task } diff --git a/pkg/sentry/fs/proc/uid_gid_map.go b/pkg/sentry/fs/proc/uid_gid_map.go index 0c68bbfc9..d433632cf 100644 --- a/pkg/sentry/fs/proc/uid_gid_map.go +++ b/pkg/sentry/fs/proc/uid_gid_map.go @@ -81,14 +81,15 @@ func (imio *idMapInodeOperations) GetFile(ctx context.Context, dirent *fs.Dirent // +stateify savable type idMapFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` iops *idMapInodeOperations } diff --git a/pkg/sentry/fs/proc/uptime.go b/pkg/sentry/fs/proc/uptime.go index 40d0fd1fd..d7ae26fcf 100644 --- a/pkg/sentry/fs/proc/uptime.go +++ b/pkg/sentry/fs/proc/uptime.go @@ -54,15 +54,16 @@ func (u *uptime) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl // +stateify savable type uptimeFile struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` startTime ktime.Time } diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index a3b33c0f8..011cf3a16 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -388,7 +388,8 @@ func (*Dir) Rename(ctx context.Context, oldParent *fs.Inode, oldName string, new // // +stateify savable type dirFileOperations struct { - fsutil.DirFileOperations `state:"nosave"` + fsutil.DirFileOperations `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // dirCursor contains the name of the last directory entry that was // serialized. diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go index 2c1295897..5bcb6c364 100644 --- a/pkg/sentry/fs/ramfs/socket.go +++ b/pkg/sentry/fs/ramfs/socket.go @@ -67,16 +67,17 @@ func (s *Socket) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl // +stateify savable type socketFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoRead `state:"nosave"` - fsutil.FileNoSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoRead `state:"nosave"` + fsutil.FileNoSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` } var _ fs.FileOperations = (*socketFileOperations)(nil) diff --git a/pkg/sentry/fs/ramfs/symlink.go b/pkg/sentry/fs/ramfs/symlink.go index 47dae380b..35dabdad2 100644 --- a/pkg/sentry/fs/ramfs/symlink.go +++ b/pkg/sentry/fs/ramfs/symlink.go @@ -88,16 +88,17 @@ func (s *Symlink) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileF // +stateify savable type symlinkFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoRead `state:"nosave"` - fsutil.FileNoSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNoRead `state:"nosave"` + fsutil.FileNoSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` } var _ fs.FileOperations = (*symlinkFileOperations)(nil) diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go index b26466b9d..ef9a08854 100644 --- a/pkg/sentry/fs/timerfd/timerfd.go +++ b/pkg/sentry/fs/timerfd/timerfd.go @@ -33,12 +33,13 @@ import ( // // +stateify savable type TimerOperations struct { - fsutil.FileZeroSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileZeroSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` events waiter.Queue `state:"zerovalue"` timer *ktime.Timer diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go index be6298130..d0c9b8bea 100644 --- a/pkg/sentry/fs/tmpfs/file_regular.go +++ b/pkg/sentry/fs/tmpfs/file_regular.go @@ -28,13 +28,14 @@ import ( // // +stateify savable type regularFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoopFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoopFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // iops is the InodeOperations of a regular tmpfs file. It is // guaranteed to be the same as file.Dirent.Inode.InodeOperations, diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index 11bf736d6..33b4c6438 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -285,13 +285,14 @@ func (d *dirInodeOperations) masterClose(t *Terminal) { // // +stateify savable type dirFileOperations struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileGenericSeek `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileGenericSeek `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // di is the inode operations. di *dirInodeOperations diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index b5e13ab36..7c256abb0 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -95,11 +95,12 @@ func (mi *masterInodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flag // // +stateify savable type masterFileOperations struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // d is the containing dir. d *dirInodeOperations diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index 6dbce90b4..e8368bcdd 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -84,11 +84,12 @@ func (si *slaveInodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flags // // +stateify savable type slaveFileOperations struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // si is the inode operations. si *slaveInodeOperations diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go index 61c0fb7c5..befefb11c 100644 --- a/pkg/sentry/kernel/epoll/epoll.go +++ b/pkg/sentry/kernel/epoll/epoll.go @@ -98,12 +98,13 @@ func (p *pollEntry) WeakRefGone() { // // +stateify savable type EventPoll struct { - fsutil.FilePipeSeek `state:"zerovalue"` - fsutil.FileNotDirReaddir `state:"zerovalue"` - fsutil.FileNoFsync `state:"zerovalue"` - fsutil.FileNoopFlush `state:"zerovalue"` - fsutil.FileNoMMap `state:"zerovalue"` - fsutil.FileNoIoctl `state:"zerovalue"` + fsutil.FilePipeSeek `state:"zerovalue"` + fsutil.FileNotDirReaddir `state:"zerovalue"` + fsutil.FileNoFsync `state:"zerovalue"` + fsutil.FileNoopFlush `state:"zerovalue"` + fsutil.FileNoMMap `state:"zerovalue"` + fsutil.FileNoIoctl `state:"zerovalue"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // Wait queue is used to notify interested parties when the event poll // object itself becomes readable or writable. diff --git a/pkg/sentry/kernel/eventfd/eventfd.go b/pkg/sentry/kernel/eventfd/eventfd.go index 2d43c986d..b448ad813 100644 --- a/pkg/sentry/kernel/eventfd/eventfd.go +++ b/pkg/sentry/kernel/eventfd/eventfd.go @@ -38,13 +38,14 @@ import ( // // +stateify savable type EventOperations struct { - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` // Mutex that protects accesses to the fields of this event. mu sync.Mutex `state:"nosave"` diff --git a/pkg/sentry/kernel/pipe/reader_writer.go b/pkg/sentry/kernel/pipe/reader_writer.go index 028175530..1090432d7 100644 --- a/pkg/sentry/kernel/pipe/reader_writer.go +++ b/pkg/sentry/kernel/pipe/reader_writer.go @@ -35,11 +35,12 @@ import ( // // +stateify savable type ReaderWriter struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` *Pipe } diff --git a/pkg/sentry/loader/vdso.go b/pkg/sentry/loader/vdso.go index fabf0cbe4..18b7e90d8 100644 --- a/pkg/sentry/loader/vdso.go +++ b/pkg/sentry/loader/vdso.go @@ -52,14 +52,15 @@ func (f *fileContext) Value(key interface{}) interface{} { // byteReader implements fs.FileOperations for reading from a []byte source. type byteReader struct { - waiter.AlwaysReady `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoIoctl `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FilePipeSeek `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoopRelease `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` data []byte } diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 5bcafad98..78f43178f 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -210,11 +210,12 @@ type commonEndpoint interface { // // +stateify savable type SocketOperations struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` socket.SendReceiveTimeout *waiter.Queue diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index a0a8a3220..be63823d8 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -46,11 +46,12 @@ const ( // socketOperations implements fs.FileOperations and socket.Socket for a socket // implemented using a host socket. type socketOperations struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` socket.SendReceiveTimeout family int // Read-only. diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index 7223773ad..2503a67c5 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -65,11 +65,12 @@ var netlinkSocketDevice = device.NewAnonDevice() // // +stateify savable type Socket struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` socket.SendReceiveTimeout // ports provides netlink port allocation. diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 548a22f32..896b5b7ce 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -45,11 +45,12 @@ import ( // socketOperations implements fs.FileOperations and socket.Socket for a socket // implemented using a host socket. type socketOperations struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` socket.SendReceiveTimeout family int // Read-only. diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index c857a0f33..92411c901 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -45,11 +45,12 @@ import ( // // +stateify savable type SocketOperations struct { - fsutil.FilePipeSeek `state:"nosave"` - fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` + fsutil.FilePipeSeek `state:"nosave"` + fsutil.FileNotDirReaddir `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileUseInodeUnstableAttr `state:"nosave"` refs.AtomicRefCount socket.SendReceiveTimeout diff --git a/pkg/sentry/syscalls/linux/sys_stat.go b/pkg/sentry/syscalls/linux/sys_stat.go index bdfb9b3ef..02634b2dd 100644 --- a/pkg/sentry/syscalls/linux/sys_stat.go +++ b/pkg/sentry/syscalls/linux/sys_stat.go @@ -60,7 +60,7 @@ func Fstatat(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sysca } defer file.DecRef() - return 0, nil, stat(t, file.Dirent, false, statAddr) + return 0, nil, fstat(t, file, statAddr) } return 0, nil, fileOpOn(t, fd, path, flags&linux.AT_SYMLINK_NOFOLLOW == 0, func(root *fs.Dirent, d *fs.Dirent) error { @@ -98,7 +98,7 @@ func Fstat(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall } defer file.DecRef() - return 0, nil, stat(t, file.Dirent, false /* dirPath */, statAddr) + return 0, nil, fstat(t, file, statAddr) } // stat implements stat from the given *fs.Dirent. @@ -110,9 +110,26 @@ func stat(t *kernel.Task, d *fs.Dirent, dirPath bool, statAddr usermem.Addr) err if err != nil { return err } + return copyOutStat(t, statAddr, d.Inode.StableAttr, uattr) +} + +// fstat implements fstat for the given *fs.File. +func fstat(t *kernel.Task, f *fs.File, statAddr usermem.Addr) error { + uattr, err := f.UnstableAttr(t) + if err != nil { + return err + } + return copyOutStat(t, statAddr, f.Dirent.Inode.StableAttr, uattr) +} +// copyOutStat copies the attributes (sattr, uattr) to the struct stat at +// address dst in t's address space. It encodes the stat struct to bytes +// manually, as stat() is a very common syscall for many applications, and +// t.CopyObjectOut has noticeable performance impact due to its many slice +// allocations and use of reflection. +func copyOutStat(t *kernel.Task, dst usermem.Addr, sattr fs.StableAttr, uattr fs.UnstableAttr) error { var mode uint32 - switch d.Inode.StableAttr.Type { + switch sattr.Type { case fs.RegularFile, fs.SpecialFile: mode |= linux.ModeRegular case fs.Symlink: @@ -129,16 +146,12 @@ func stat(t *kernel.Task, d *fs.Dirent, dirPath bool, statAddr usermem.Addr) err mode |= linux.ModeSocket } - // We encode the stat struct to bytes manually, as stat() is a very - // common syscall for many applications, and t.CopyObjectOut has - // noticeable performance impact due to its many slice allocations and - // use of reflection. b := t.CopyScratchBuffer(int(linux.SizeOfStat))[:0] // Dev (uint64) - b = binary.AppendUint64(b, usermem.ByteOrder, uint64(d.Inode.StableAttr.DeviceID)) + b = binary.AppendUint64(b, usermem.ByteOrder, uint64(sattr.DeviceID)) // Ino (uint64) - b = binary.AppendUint64(b, usermem.ByteOrder, uint64(d.Inode.StableAttr.InodeID)) + b = binary.AppendUint64(b, usermem.ByteOrder, uint64(sattr.InodeID)) // Nlink (uint64) b = binary.AppendUint64(b, usermem.ByteOrder, uattr.Links) // Mode (uint32) @@ -150,11 +163,11 @@ func stat(t *kernel.Task, d *fs.Dirent, dirPath bool, statAddr usermem.Addr) err // Padding (uint32) b = binary.AppendUint32(b, usermem.ByteOrder, 0) // Rdev (uint64) - b = binary.AppendUint64(b, usermem.ByteOrder, uint64(linux.MakeDeviceID(d.Inode.StableAttr.DeviceFileMajor, d.Inode.StableAttr.DeviceFileMinor))) + b = binary.AppendUint64(b, usermem.ByteOrder, uint64(linux.MakeDeviceID(sattr.DeviceFileMajor, sattr.DeviceFileMinor))) // Size (uint64) b = binary.AppendUint64(b, usermem.ByteOrder, uint64(uattr.Size)) // Blksize (uint64) - b = binary.AppendUint64(b, usermem.ByteOrder, uint64(d.Inode.StableAttr.BlockSize)) + b = binary.AppendUint64(b, usermem.ByteOrder, uint64(sattr.BlockSize)) // Blocks (uint64) b = binary.AppendUint64(b, usermem.ByteOrder, uint64(uattr.Usage/512)) @@ -173,7 +186,7 @@ func stat(t *kernel.Task, d *fs.Dirent, dirPath bool, statAddr usermem.Addr) err b = binary.AppendUint64(b, usermem.ByteOrder, uint64(ctime.Sec)) b = binary.AppendUint64(b, usermem.ByteOrder, uint64(ctime.Nsec)) - _, err = t.CopyOutBytes(statAddr, b) + _, err := t.CopyOutBytes(dst, b) return err } -- cgit v1.2.3 From 08d99c5fbea76ecc92038280387d24ecdf7ed814 Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Wed, 17 Apr 2019 12:13:46 -0700 Subject: Convert poll/select to operate more directly on linux.PollFD Current, doPoll copies the user struct pollfd array into a []syscalls.PollFD, which contains internal kdefs.FD and waiter.EventMask types. While these are currently binary-compatible with the Linux versions, we generally discourage copying directly to internal types (someone may inadvertantly change kdefs.FD to uint64). Instead, copy directly to a []linux.PollFD, which will certainly be binary compatible. Most of syscalls/polling.go is included directly into syscalls/linux/sys_poll.go, as it can then operate directly on linux.PollFD. The additional syscalls.PollFD type is providing little value. I've also added explicit conversion functions for waiter.EventMask, which creates the possibility of a different binary format. PiperOrigin-RevId: 244042947 Change-Id: I24e5b642002a32b3afb95a9dcb80d4acd1288abf --- pkg/fdnotifier/fdnotifier.go | 6 +- pkg/fdnotifier/poll_unsafe.go | 6 +- pkg/sentry/socket/rpcinet/notifier/notifier.go | 8 +- pkg/sentry/syscalls/BUILD | 2 - pkg/sentry/syscalls/linux/sys_epoll.go | 2 +- pkg/sentry/syscalls/linux/sys_poll.go | 133 +++++++++++++++++++++--- pkg/sentry/syscalls/polling.go | 137 ------------------------- pkg/waiter/waiter.go | 26 +++-- 8 files changed, 153 insertions(+), 167 deletions(-) delete mode 100644 pkg/sentry/syscalls/polling.go (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/fdnotifier/fdnotifier.go b/pkg/fdnotifier/fdnotifier.go index 624b1a0c5..aa4906ca0 100644 --- a/pkg/fdnotifier/fdnotifier.go +++ b/pkg/fdnotifier/fdnotifier.go @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +// +build linux + // Package fdnotifier contains an adapter that translates IO events (e.g., a // file became readable/writable) from native FDs to the notifications in the // waiter package. It uses epoll in edge-triggered mode to receive notifications @@ -70,7 +72,7 @@ func (n *notifier) waitFD(fd int32, fi *fdInfo, mask waiter.EventMask) error { } e := syscall.EpollEvent{ - Events: uint32(mask) | -syscall.EPOLLET, + Events: mask.ToLinux() | -syscall.EPOLLET, Fd: fd, } @@ -155,7 +157,7 @@ func (n *notifier) waitAndNotify() error { n.mu.Lock() for i := 0; i < v; i++ { if fi, ok := n.fdMap[e[i].Fd]; ok { - fi.queue.Notify(waiter.EventMask(e[i].Events)) + fi.queue.Notify(waiter.EventMaskFromLinux(e[i].Events)) } } n.mu.Unlock() diff --git a/pkg/fdnotifier/poll_unsafe.go b/pkg/fdnotifier/poll_unsafe.go index 8459d4c74..05be9aeb5 100644 --- a/pkg/fdnotifier/poll_unsafe.go +++ b/pkg/fdnotifier/poll_unsafe.go @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +// +build linux + package fdnotifier import ( @@ -30,7 +32,7 @@ func NonBlockingPoll(fd int32, mask waiter.EventMask) waiter.EventMask { revents int16 }{ fd: fd, - events: int16(mask), + events: int16(mask.ToLinux()), } for { @@ -51,7 +53,7 @@ func NonBlockingPoll(fd int32, mask waiter.EventMask) waiter.EventMask { } // Otherwise we got the ready events in the revents field. - return waiter.EventMask(e.revents) + return waiter.EventMaskFromLinux(uint32(e.revents)) } } diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier.go b/pkg/sentry/socket/rpcinet/notifier/notifier.go index 73c255c33..d9bda78b0 100644 --- a/pkg/sentry/socket/rpcinet/notifier/notifier.go +++ b/pkg/sentry/socket/rpcinet/notifier/notifier.go @@ -76,7 +76,7 @@ func (n *Notifier) waitFD(fd uint32, fi *fdInfo, mask waiter.EventMask) error { } e := pb.EpollEvent{ - Events: uint32(mask) | -syscall.EPOLLET, + Events: mask.ToLinux() | -syscall.EPOLLET, Fd: fd, } @@ -178,7 +178,7 @@ func (n *Notifier) waitAndNotify() error { n.mu.Lock() for _, e := range res.(*pb.EpollWaitResponse_Events).Events.Events { if fi, ok := n.fdMap[e.Fd]; ok { - fi.queue.Notify(waiter.EventMask(e.Events)) + fi.queue.Notify(waiter.EventMaskFromLinux(e.Events)) } } n.mu.Unlock() @@ -214,7 +214,7 @@ func (n *Notifier) HasFD(fd uint32) bool { // although the syscall is non-blocking. func (n *Notifier) NonBlockingPoll(fd uint32, mask waiter.EventMask) waiter.EventMask { for { - id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Poll{&pb.PollRequest{Fd: fd, Events: uint32(mask)}}}, false /* ignoreResult */) + id, c := n.rpcConn.NewRequest(pb.SyscallRequest{Args: &pb.SyscallRequest_Poll{&pb.PollRequest{Fd: fd, Events: mask.ToLinux()}}}, false /* ignoreResult */) <-c res := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_Poll).Poll.Result @@ -225,6 +225,6 @@ func (n *Notifier) NonBlockingPoll(fd uint32, mask waiter.EventMask) waiter.Even return mask } - return waiter.EventMask(res.(*pb.PollResponse_Events).Events) + return waiter.EventMaskFromLinux(res.(*pb.PollResponse_Events).Events) } } diff --git a/pkg/sentry/syscalls/BUILD b/pkg/sentry/syscalls/BUILD index 6b5469e45..877318fa9 100644 --- a/pkg/sentry/syscalls/BUILD +++ b/pkg/sentry/syscalls/BUILD @@ -6,7 +6,6 @@ go_library( name = "syscalls", srcs = [ "epoll.go", - "polling.go", "syscalls.go", ], importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/syscalls", @@ -14,7 +13,6 @@ go_library( deps = [ "//pkg/abi/linux", "//pkg/sentry/arch", - "//pkg/sentry/fs", "//pkg/sentry/kernel", "//pkg/sentry/kernel/epoll", "//pkg/sentry/kernel/kdefs", diff --git a/pkg/sentry/syscalls/linux/sys_epoll.go b/pkg/sentry/syscalls/linux/sys_epoll.go index 62272efcd..200c46355 100644 --- a/pkg/sentry/syscalls/linux/sys_epoll.go +++ b/pkg/sentry/syscalls/linux/sys_epoll.go @@ -87,7 +87,7 @@ func EpollCtl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sysc flags |= epoll.EdgeTriggered } - mask = waiter.EventMask(e.Events) + mask = waiter.EventMaskFromLinux(e.Events) data[0] = e.Fd data[1] = e.Pad } diff --git a/pkg/sentry/syscalls/linux/sys_poll.go b/pkg/sentry/syscalls/linux/sys_poll.go index 0cf6aad7f..23fcb907f 100644 --- a/pkg/sentry/syscalls/linux/sys_poll.go +++ b/pkg/sentry/syscalls/linux/sys_poll.go @@ -19,11 +19,11 @@ import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" "gvisor.googlesource.com/gvisor/pkg/sentry/limits" - "gvisor.googlesource.com/gvisor/pkg/sentry/syscalls" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" "gvisor.googlesource.com/gvisor/pkg/syserror" "gvisor.googlesource.com/gvisor/pkg/waiter" @@ -37,23 +37,130 @@ const fileCap = 1024 * 1024 const ( // selectReadEvents is analogous to the Linux kernel's // fs/select.c:POLLIN_SET. - selectReadEvents = waiter.EventIn | waiter.EventHUp | waiter.EventErr + selectReadEvents = linux.POLLIN | linux.POLLHUP | linux.POLLERR // selectWriteEvents is analogous to the Linux kernel's // fs/select.c:POLLOUT_SET. - selectWriteEvents = waiter.EventOut | waiter.EventErr + selectWriteEvents = linux.POLLOUT | linux.POLLERR // selectExceptEvents is analogous to the Linux kernel's // fs/select.c:POLLEX_SET. - selectExceptEvents = waiter.EventPri + selectExceptEvents = linux.POLLPRI ) +// pollState tracks the associated file descriptor and waiter of a PollFD. +type pollState struct { + file *fs.File + waiter waiter.Entry +} + +// initReadiness gets the current ready mask for the file represented by the FD +// stored in pfd.FD. If a channel is passed in, the waiter entry in "state" is +// used to register with the file for event notifications, and a reference to +// the file is stored in "state". +func initReadiness(t *kernel.Task, pfd *linux.PollFD, state *pollState, ch chan struct{}) { + if pfd.FD < 0 { + pfd.REvents = 0 + return + } + + file := t.FDMap().GetFile(kdefs.FD(pfd.FD)) + if file == nil { + pfd.REvents = linux.POLLNVAL + return + } + + if ch == nil { + defer file.DecRef() + } else { + state.file = file + state.waiter, _ = waiter.NewChannelEntry(ch) + file.EventRegister(&state.waiter, waiter.EventMaskFromLinux(uint32(pfd.Events))) + } + + r := file.Readiness(waiter.EventMaskFromLinux(uint32(pfd.Events))) + pfd.REvents = int16(r.ToLinux()) & pfd.Events +} + +// releaseState releases all the pollState in "state". +func releaseState(state []pollState) { + for i := range state { + if state[i].file != nil { + state[i].file.EventUnregister(&state[i].waiter) + state[i].file.DecRef() + } + } +} + +// pollBlock polls the PollFDs in "pfd" with a bounded time specified in "timeout" +// when "timeout" is greater than zero. +// +// pollBlock returns the remaining timeout, which is always 0 on a timeout; and 0 or +// positive if interrupted by a signal. +func pollBlock(t *kernel.Task, pfd []linux.PollFD, timeout time.Duration) (time.Duration, uintptr, error) { + var ch chan struct{} + if timeout != 0 { + ch = make(chan struct{}, 1) + } + + // Register for event notification in the files involved if we may + // block (timeout not zero). Once we find a file that has a non-zero + // result, we stop registering for events but still go through all files + // to get their ready masks. + state := make([]pollState, len(pfd)) + defer releaseState(state) + n := uintptr(0) + for i := range pfd { + initReadiness(t, &pfd[i], &state[i], ch) + if pfd[i].REvents != 0 { + n++ + ch = nil + } + } + + if timeout == 0 { + return timeout, n, nil + } + + forever := timeout < 0 + + for n == 0 { + var err error + // Wait for a notification. + timeout, err = t.BlockWithTimeout(ch, !forever, timeout) + if err != nil { + if err == syserror.ETIMEDOUT { + err = nil + } + return timeout, 0, err + } + + // We got notified, count how many files are ready. If none, + // then this was a spurious notification, and we just go back + // to sleep with the remaining timeout. + for i := range state { + if state[i].file == nil { + continue + } + + r := state[i].file.Readiness(waiter.EventMaskFromLinux(uint32(pfd[i].Events))) + rl := int16(r.ToLinux()) & pfd[i].Events + if rl != 0 { + pfd[i].REvents = rl + n++ + } + } + } + + return timeout, n, nil +} + func doPoll(t *kernel.Task, pfdAddr usermem.Addr, nfds uint, timeout time.Duration) (time.Duration, uintptr, error) { if uint64(nfds) > t.ThreadGroup().Limits().GetCapped(limits.NumberOfFiles, fileCap) { return timeout, 0, syserror.EINVAL } - pfd := make([]syscalls.PollFD, nfds) + pfd := make([]linux.PollFD, nfds) if nfds > 0 { if _, err := t.CopyIn(pfdAddr, &pfd); err != nil { return timeout, 0, err @@ -65,9 +172,9 @@ func doPoll(t *kernel.Task, pfdAddr usermem.Addr, nfds uint, timeout time.Durati // polling, changing event masks here is an application-visible difference. // (Linux also doesn't copy out event masks at all, only revents.) for i := range pfd { - pfd[i].Events |= waiter.EventHUp | waiter.EventErr + pfd[i].Events |= linux.POLLHUP | linux.POLLERR } - remainingTimeout, n, err := syscalls.Poll(t, pfd, timeout) + remainingTimeout, n, err := pollBlock(t, pfd, timeout) err = syserror.ConvertIntr(err, syserror.EINTR) // The poll entries are copied out regardless of whether @@ -136,8 +243,8 @@ func doSelect(t *kernel.Task, nfds int, readFDs, writeFDs, exceptFDs usermem.Add } // Build the PollFD array. - pfd := make([]syscalls.PollFD, 0, fdCount) - fd := kdefs.FD(0) + pfd := make([]linux.PollFD, 0, fdCount) + var fd int32 for i := 0; i < byteCount; i++ { rV, wV, eV := r[i], w[i], e[i] v := rV | wV | eV @@ -148,13 +255,13 @@ func doSelect(t *kernel.Task, nfds int, readFDs, writeFDs, exceptFDs usermem.Add // immediately to ensure we don't leak. Note, another thread // might be about to close fd. This is racy, but that's // OK. Linux is racy in the same way. - file := t.FDMap().GetFile(fd) + file := t.FDMap().GetFile(kdefs.FD(fd)) if file == nil { return 0, syserror.EBADF } file.DecRef() - mask := waiter.EventMask(0) + var mask int16 if (rV & m) != 0 { mask |= selectReadEvents } @@ -167,7 +274,7 @@ func doSelect(t *kernel.Task, nfds int, readFDs, writeFDs, exceptFDs usermem.Add mask |= selectExceptEvents } - pfd = append(pfd, syscalls.PollFD{ + pfd = append(pfd, linux.PollFD{ FD: fd, Events: mask, }) @@ -179,7 +286,7 @@ func doSelect(t *kernel.Task, nfds int, readFDs, writeFDs, exceptFDs usermem.Add } // Do the syscall, then count the number of bits set. - _, _, err := syscalls.Poll(t, pfd, timeout) + _, _, err := pollBlock(t, pfd, timeout) if err != nil { return 0, syserror.ConvertIntr(err, syserror.EINTR) } diff --git a/pkg/sentry/syscalls/polling.go b/pkg/sentry/syscalls/polling.go deleted file mode 100644 index 2b33d6c19..000000000 --- a/pkg/sentry/syscalls/polling.go +++ /dev/null @@ -1,137 +0,0 @@ -// Copyright 2018 Google LLC -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package syscalls - -import ( - "syscall" - "time" - - "gvisor.googlesource.com/gvisor/pkg/sentry/fs" - "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" - "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" - "gvisor.googlesource.com/gvisor/pkg/waiter" -) - -// PollFD describes a pollable FD. -type PollFD struct { - FD kdefs.FD - Events waiter.EventMask - REvents waiter.EventMask -} - -// pollState tracks the associated file descriptor and waiter of a PollFD. -type pollState struct { - file *fs.File - waiter waiter.Entry -} - -// initReadiness gets the current ready mask for the file represented by the FD -// stored in pfd.FD. If a channel is passed in, the waiter entry in "state" is -// used to register with the file for event notifications, and a reference to -// the file is stored in "state". -func (pfd *PollFD) initReadiness(t *kernel.Task, state *pollState, ch chan struct{}) { - if pfd.FD < 0 { - pfd.REvents = 0 - return - } - - file := t.FDMap().GetFile(pfd.FD) - if file == nil { - pfd.REvents = waiter.EventNVal - return - } - - if ch == nil { - defer file.DecRef() - } else { - state.file = file - state.waiter, _ = waiter.NewChannelEntry(ch) - file.EventRegister(&state.waiter, pfd.Events) - } - - pfd.REvents = file.Readiness(pfd.Events) & pfd.Events -} - -// releaseState releases all the pollState in "state". -func releaseState(state []pollState) { - for i := range state { - if state[i].file != nil { - state[i].file.EventUnregister(&state[i].waiter) - state[i].file.DecRef() - } - } -} - -// Poll polls the PollFDs in "pfd" with a bounded time specified in "timeout" -// when "timeout" is greater than zero. -// -// Poll returns the remaining timeout, which is always 0 on a timeout; and 0 or -// positive if interrupted by a signal. -func Poll(t *kernel.Task, pfd []PollFD, timeout time.Duration) (time.Duration, uintptr, error) { - var ch chan struct{} - if timeout != 0 { - ch = make(chan struct{}, 1) - } - - // Register for event notification in the files involved if we may - // block (timeout not zero). Once we find a file that has a non-zero - // result, we stop registering for events but still go through all files - // to get their ready masks. - state := make([]pollState, len(pfd)) - defer releaseState(state) - n := uintptr(0) - for i := range pfd { - pfd[i].initReadiness(t, &state[i], ch) - if pfd[i].REvents != 0 { - n++ - ch = nil - } - } - - if timeout == 0 { - return timeout, n, nil - } - - forever := timeout < 0 - - for n == 0 { - var err error - // Wait for a notification. - timeout, err = t.BlockWithTimeout(ch, !forever, timeout) - if err != nil { - if err == syscall.ETIMEDOUT { - err = nil - } - return timeout, 0, err - } - - // We got notified, count how many files are ready. If none, - // then this was a spurious notification, and we just go back - // to sleep with the remaining timeout. - for i := range state { - if state[i].file == nil { - continue - } - - ready := state[i].file.Readiness(pfd[i].Events) & pfd[i].Events - if ready != 0 { - pfd[i].REvents = ready - n++ - } - } - } - - return timeout, n, nil -} diff --git a/pkg/waiter/waiter.go b/pkg/waiter/waiter.go index fd429f733..a6c9dff3c 100644 --- a/pkg/waiter/waiter.go +++ b/pkg/waiter/waiter.go @@ -67,14 +67,28 @@ type EventMask uint16 // Events that waiters can wait on. The meaning is the same as those in the // poll() syscall. const ( - EventIn EventMask = 0x01 // syscall.EPOLLIN - EventPri EventMask = 0x02 // syscall.EPOLLPRI - EventOut EventMask = 0x04 // syscall.EPOLLOUT - EventErr EventMask = 0x08 // syscall.EPOLLERR - EventHUp EventMask = 0x10 // syscall.EPOLLHUP - EventNVal EventMask = 0x20 // Not defined in syscall. + EventIn EventMask = 0x01 // POLLIN + EventPri EventMask = 0x02 // POLLPRI + EventOut EventMask = 0x04 // POLLOUT + EventErr EventMask = 0x08 // POLLERR + EventHUp EventMask = 0x10 // POLLHUP + + allEvents EventMask = 0x1f ) +// EventMaskFromLinux returns an EventMask representing the supported events +// from the Linux events e, which is in the format used by poll(2). +func EventMaskFromLinux(e uint32) EventMask { + // Our flag definitions are currently identical to Linux. + return EventMask(e) & allEvents +} + +// ToLinux returns e in the format used by Linux poll(2). +func (e EventMask) ToLinux() uint32 { + // Our flag definitions are currently identical to Linux. + return uint32(e) +} + // Waitable contains the methods that need to be implemented by waitable // objects. type Waitable interface { -- cgit v1.2.3 From 358eb52a76ebd41baf52972f901af0ff398e131b Mon Sep 17 00:00:00 2001 From: Ian Gudger Date: Fri, 19 Apr 2019 16:15:37 -0700 Subject: Add support for the MSG_TRUNC msghdr flag. The MSG_TRUNC flag is set in the msghdr when a message is truncated. Fixes google/gvisor#200 PiperOrigin-RevId: 244440486 Change-Id: I03c7d5e7f5935c0c6b8d69b012db1780ac5b8456 --- pkg/sentry/socket/epsocket/epsocket.go | 45 ++++++++-------- pkg/sentry/socket/hostinet/socket.go | 12 +++-- pkg/sentry/socket/netlink/socket.go | 18 +++++-- pkg/sentry/socket/rpcinet/socket.go | 16 +++--- pkg/sentry/socket/socket.go | 2 +- pkg/sentry/socket/unix/unix.go | 31 +++++++---- pkg/sentry/syscalls/linux/sys_socket.go | 16 +++--- test/syscalls/linux/socket_netlink_route.cc | 80 +++++++++++++++++++++++++++++ test/syscalls/linux/socket_non_stream.cc | 55 ++++++++++++++++++++ test/syscalls/linux/socket_stream.cc | 27 ++++++++++ 10 files changed, 245 insertions(+), 57 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index f370b803b..23138d874 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -376,7 +376,7 @@ func (s *SocketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOS if dst.NumBytes() == 0 { return 0, nil } - n, _, _, _, err := s.nonBlockingRead(ctx, dst, false, false, false) + n, _, _, _, _, err := s.nonBlockingRead(ctx, dst, false, false, false) if err == syserr.ErrWouldBlock { return int64(n), syserror.ErrWouldBlock } @@ -1696,7 +1696,7 @@ func (s *SocketOperations) coalescingRead(ctx context.Context, dst usermem.IOSeq // nonBlockingRead issues a non-blocking read. // // TODO: Support timestamps for stream sockets. -func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSequence, peek, trunc, senderRequested bool) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { +func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSequence, peek, trunc, senderRequested bool) (int, int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { isPacket := s.isPacketBased() // Fast path for regular reads from stream (e.g., TCP) endpoints. Note @@ -1712,14 +1712,14 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe s.readMu.Lock() n, err := s.coalescingRead(ctx, dst, trunc) s.readMu.Unlock() - return n, nil, 0, socket.ControlMessages{}, err + return n, 0, nil, 0, socket.ControlMessages{}, err } s.readMu.Lock() defer s.readMu.Unlock() if err := s.fetchReadView(); err != nil { - return 0, nil, 0, socket.ControlMessages{}, err + return 0, 0, nil, 0, socket.ControlMessages{}, err } if !isPacket && peek && trunc { @@ -1727,14 +1727,14 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe // amount that could be read. var rql tcpip.ReceiveQueueSizeOption if err := s.Endpoint.GetSockOpt(&rql); err != nil { - return 0, nil, 0, socket.ControlMessages{}, syserr.TranslateNetstackError(err) + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.TranslateNetstackError(err) } available := len(s.readView) + int(rql) bufLen := int(dst.NumBytes()) if available < bufLen { - return available, nil, 0, socket.ControlMessages{}, nil + return available, 0, nil, 0, socket.ControlMessages{}, nil } - return bufLen, nil, 0, socket.ControlMessages{}, nil + return bufLen, 0, nil, 0, socket.ControlMessages{}, nil } n, err := dst.CopyOut(ctx, s.readView) @@ -1751,11 +1751,11 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe if peek { if l := len(s.readView); trunc && l > n { // isPacket must be true. - return l, addr, addrLen, s.controlMessages(), syserr.FromError(err) + return l, linux.MSG_TRUNC, addr, addrLen, s.controlMessages(), syserr.FromError(err) } if isPacket || err != nil { - return int(n), addr, addrLen, s.controlMessages(), syserr.FromError(err) + return n, 0, addr, addrLen, s.controlMessages(), syserr.FromError(err) } // We need to peek beyond the first message. @@ -1773,7 +1773,7 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe // We got some data, so no need to return an error. err = nil } - return int(n), nil, 0, s.controlMessages(), syserr.FromError(err) + return n, 0, nil, 0, s.controlMessages(), syserr.FromError(err) } var msgLen int @@ -1785,11 +1785,16 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe s.readView.TrimFront(int(n)) } + var flags int + if msgLen > int(n) { + flags |= linux.MSG_TRUNC + } + if trunc { - return msgLen, addr, addrLen, s.controlMessages(), syserr.FromError(err) + n = msgLen } - return int(n), addr, addrLen, s.controlMessages(), syserr.FromError(err) + return n, flags, addr, addrLen, s.controlMessages(), syserr.FromError(err) } func (s *SocketOperations) controlMessages() socket.ControlMessages { @@ -1810,7 +1815,7 @@ func (s *SocketOperations) updateTimestamp() { // RecvMsg implements the linux syscall recvmsg(2) for sockets backed by // tcpip.Endpoint. -func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) { +func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, msgFlags int, senderAddr interface{}, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) { trunc := flags&linux.MSG_TRUNC != 0 peek := flags&linux.MSG_PEEK != 0 dontWait := flags&linux.MSG_DONTWAIT != 0 @@ -1819,16 +1824,16 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags // Stream sockets ignore the sender address. senderRequested = false } - n, senderAddr, senderAddrLen, controlMessages, err = s.nonBlockingRead(t, dst, peek, trunc, senderRequested) + n, msgFlags, senderAddr, senderAddrLen, controlMessages, err = s.nonBlockingRead(t, dst, peek, trunc, senderRequested) if s.isPacketBased() && err == syserr.ErrClosedForReceive && flags&linux.MSG_DONTWAIT != 0 { // In this situation we should return EAGAIN. - return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } if err != nil && (err != syserr.ErrWouldBlock || dontWait) { // Read failed and we should not retry. - return 0, nil, 0, socket.ControlMessages{}, err + return 0, 0, nil, 0, socket.ControlMessages{}, err } if err == nil && (dontWait || !waitAll || s.isPacketBased() || int64(n) >= dst.NumBytes()) { @@ -1847,7 +1852,7 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags for { var rn int - rn, senderAddr, senderAddrLen, controlMessages, err = s.nonBlockingRead(t, dst, peek, trunc, senderRequested) + rn, msgFlags, senderAddr, senderAddrLen, controlMessages, err = s.nonBlockingRead(t, dst, peek, trunc, senderRequested) n += rn if err != nil && err != syserr.ErrWouldBlock { // Always stop on errors other than would block as we generally @@ -1866,12 +1871,12 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { if n > 0 { - return n, senderAddr, senderAddrLen, controlMessages, nil + return n, msgFlags, senderAddr, senderAddrLen, controlMessages, nil } if err == syserror.ETIMEDOUT { - return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } - return 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) } } } diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index be63823d8..c4848b313 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -345,14 +345,14 @@ func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt [ } // RecvMsg implements socket.Socket.RecvMsg. -func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { +func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { // Whitelist flags. // // FIXME: We can't support MSG_ERRQUEUE because it uses ancillary // messages that netstack/tcpip/transport/unix doesn't understand. Kill the // Socket interface's dependence on netstack. if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_PEEK|syscall.MSG_TRUNC) != 0 { - return 0, nil, 0, socket.ControlMessages{}, syserr.ErrInvalidArgument + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.ErrInvalidArgument } var senderAddr []byte @@ -360,6 +360,8 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags senderAddr = make([]byte, sizeofSockaddr) } + var msgFlags int + recvmsgToBlocks := safemem.ReaderFunc(func(dsts safemem.BlockSeq) (uint64, error) { // Refuse to do anything if any part of dst.Addrs was unusable. if uint64(dst.NumBytes()) != dsts.NumBytes() { @@ -391,6 +393,7 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags return 0, err } senderAddr = senderAddr[:msg.Namelen] + msgFlags = int(msg.Flags) return n, nil }) @@ -417,7 +420,10 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags } } - return int(n), senderAddr, uint32(len(senderAddr)), socket.ControlMessages{}, syserr.FromError(err) + // We don't allow control messages. + msgFlags &^= linux.MSG_CTRUNC + + return int(n), msgFlags, senderAddr, uint32(len(senderAddr)), socket.ControlMessages{}, syserr.FromError(err) } // SendMsg implements socket.Socket.SendMsg. diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index 2503a67c5..0fe9b39b6 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -397,7 +397,7 @@ func (s *Socket) GetPeerName(t *kernel.Task) (interface{}, uint32, *syserr.Error } // RecvMsg implements socket.Socket.RecvMsg. -func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { +func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { from := linux.SockAddrNetlink{ Family: linux.AF_NETLINK, PortID: 0, @@ -412,10 +412,14 @@ func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, have } if n, err := dst.CopyOutFrom(t, &r); err != syserror.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { + var mflags int + if n < int64(r.MsgSize) { + mflags |= linux.MSG_TRUNC + } if trunc { n = int64(r.MsgSize) } - return int(n), from, fromLen, socket.ControlMessages{}, syserr.FromError(err) + return int(n), mflags, from, fromLen, socket.ControlMessages{}, syserr.FromError(err) } // We'll have to block. Register for notification and keep trying to @@ -426,17 +430,21 @@ func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, have for { if n, err := dst.CopyOutFrom(t, &r); err != syserror.ErrWouldBlock { + var mflags int + if n < int64(r.MsgSize) { + mflags |= linux.MSG_TRUNC + } if trunc { n = int64(r.MsgSize) } - return int(n), from, fromLen, socket.ControlMessages{}, syserr.FromError(err) + return int(n), mflags, from, fromLen, socket.ControlMessages{}, syserr.FromError(err) } if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { if err == syserror.ETIMEDOUT { - return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } - return 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) } } } diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 896b5b7ce..3418a6d75 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -673,7 +673,7 @@ func (s *socketOperations) extractControlMessages(payload *pb.RecvmsgResponse_Re } // RecvMsg implements socket.Socket.RecvMsg. -func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { +func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { req := &pb.SyscallRequest_Recvmsg{&pb.RecvmsgRequest{ Fd: s.fd, Length: uint32(dst.NumBytes()), @@ -694,10 +694,10 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags } } c := s.extractControlMessages(res) - return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), c, syserr.FromError(e) + return int(res.Length), 0, res.Address.GetAddress(), res.Address.GetLength(), c, syserr.FromError(e) } if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain || flags&linux.MSG_DONTWAIT != 0 { - return 0, nil, 0, socket.ControlMessages{}, err + return 0, 0, nil, 0, socket.ControlMessages{}, err } // We'll have to block. Register for notifications and keep trying to @@ -718,23 +718,23 @@ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags } } c := s.extractControlMessages(res) - return int(res.Length), res.Address.GetAddress(), res.Address.GetLength(), c, syserr.FromError(e) + return int(res.Length), 0, res.Address.GetAddress(), res.Address.GetLength(), c, syserr.FromError(e) } if err != syserr.ErrWouldBlock && err != syserr.ErrTryAgain { - return 0, nil, 0, socket.ControlMessages{}, err + return 0, 0, nil, 0, socket.ControlMessages{}, err } if s.isShutRdSet() { // Blocking would have caused us to block indefinitely so we return 0, // this is the same behavior as Linux. - return 0, nil, 0, socket.ControlMessages{}, nil + return 0, 0, nil, 0, socket.ControlMessages{}, nil } if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { if err == syserror.ETIMEDOUT { - return 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } - return 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) + return 0, 0, nil, 0, socket.ControlMessages{}, syserr.FromError(err) } } } diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index 5ab423f3c..62ba13782 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -88,7 +88,7 @@ type Socket interface { // not necessarily the actual length of the address. // // If err != nil, the recv was not successful. - RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages ControlMessages, err *syserr.Error) + RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, msgFlags int, senderAddr interface{}, senderAddrLen uint32, controlMessages ControlMessages, err *syserr.Error) // SendMsg implements the sendmsg(2) linux syscall. SendMsg does not take // ownership of the ControlMessage on error. diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index 92411c901..01efd24d3 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -477,7 +477,7 @@ func (s *SocketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOS // RecvMsg implements the linux syscall recvmsg(2) for sockets backed by // a transport.Endpoint. -func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, senderAddr interface{}, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) { +func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, msgFlags int, senderAddr interface{}, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) { trunc := flags&linux.MSG_TRUNC != 0 peek := flags&linux.MSG_PEEK != 0 dontWait := flags&linux.MSG_DONTWAIT != 0 @@ -515,11 +515,17 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags if r.From != nil { from, fromLen = epsocket.ConvertAddress(linux.AF_UNIX, *r.From) } - if trunc { - n = int64(r.MsgSize) - } + if err != nil || dontWait || !waitAll || s.isPacket || n >= dst.NumBytes() { - return int(n), from, fromLen, socket.ControlMessages{Unix: r.Control}, syserr.FromError(err) + if s.isPacket && n < int64(r.MsgSize) { + msgFlags |= linux.MSG_TRUNC + } + + if trunc { + n = int64(r.MsgSize) + } + + return int(n), msgFlags, from, fromLen, socket.ControlMessages{Unix: r.Control}, syserr.FromError(err) } // Don't overwrite any data we received. @@ -541,14 +547,19 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags from, fromLen = epsocket.ConvertAddress(linux.AF_UNIX, *r.From) } if trunc { - n = int64(r.MsgSize) + // n and r.MsgSize are the same for streams. + total += int64(r.MsgSize) + } else { + total += n } - total += n if err != nil || !waitAll || s.isPacket || n >= dst.NumBytes() { if total > 0 { err = nil } - return int(total), from, fromLen, socket.ControlMessages{Unix: r.Control}, syserr.FromError(err) + if s.isPacket && n < int64(r.MsgSize) { + msgFlags |= linux.MSG_TRUNC + } + return int(total), msgFlags, from, fromLen, socket.ControlMessages{Unix: r.Control}, syserr.FromError(err) } // Don't overwrite any data we received. @@ -560,9 +571,9 @@ func (s *SocketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags err = nil } if err == syserror.ETIMEDOUT { - return int(total), nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain + return int(total), msgFlags, nil, 0, socket.ControlMessages{}, syserr.ErrTryAgain } - return int(total), nil, 0, socket.ControlMessages{}, syserr.FromError(err) + return int(total), msgFlags, nil, 0, socket.ControlMessages{}, syserr.FromError(err) } } } diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index 49e6f4aeb..30ccc3f66 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -742,17 +742,15 @@ func recvSingleMsg(t *kernel.Task, s socket.Socket, msgPtr usermem.Addr, flags i // Fast path when no control message nor name buffers are provided. if msg.ControlLen == 0 && msg.NameLen == 0 { - n, _, _, cms, err := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, false, 0) + n, mflags, _, _, cms, err := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, false, 0) if err != nil { return 0, syserror.ConvertIntr(err.ToError(), kernel.ERESTARTSYS) } cms.Unix.Release() - if msg.Flags != 0 { + if int(msg.Flags) != mflags { // Copy out the flags to the caller. - // - // TODO: Plumb through actual flags. - if _, err := t.CopyOut(msgPtr+flagsOffset, int32(0)); err != nil { + if _, err := t.CopyOut(msgPtr+flagsOffset, int32(mflags)); err != nil { return 0, err } } @@ -763,7 +761,7 @@ func recvSingleMsg(t *kernel.Task, s socket.Socket, msgPtr usermem.Addr, flags i if msg.ControlLen > maxControlLen { return 0, syscall.ENOBUFS } - n, sender, senderLen, cms, e := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, msg.NameLen != 0, msg.ControlLen) + n, mflags, sender, senderLen, cms, e := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, msg.NameLen != 0, msg.ControlLen) if e != nil { return 0, syserror.ConvertIntr(e.ToError(), kernel.ERESTARTSYS) } @@ -802,9 +800,7 @@ func recvSingleMsg(t *kernel.Task, s socket.Socket, msgPtr usermem.Addr, flags i } // Copy out the flags to the caller. - // - // TODO: Plumb through actual flags. - if _, err := t.CopyOut(msgPtr+flagsOffset, int32(0)); err != nil { + if _, err := t.CopyOut(msgPtr+flagsOffset, int32(mflags)); err != nil { return 0, err } @@ -856,7 +852,7 @@ func recvFrom(t *kernel.Task, fd kdefs.FD, bufPtr usermem.Addr, bufLen uint64, f flags |= linux.MSG_DONTWAIT } - n, sender, senderLen, cm, e := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, nameLenPtr != 0, 0) + n, _, sender, senderLen, cm, e := s.RecvMsg(t, dst, int(flags), haveDeadline, deadline, nameLenPtr != 0, 0) cm.Unix.Release() if e != nil { return 0, syserror.ConvertIntr(e.ToError(), kernel.ERESTARTSYS) diff --git a/test/syscalls/linux/socket_netlink_route.cc b/test/syscalls/linux/socket_netlink_route.cc index 5f83836df..fa895d841 100644 --- a/test/syscalls/linux/socket_netlink_route.cc +++ b/test/syscalls/linux/socket_netlink_route.cc @@ -220,6 +220,86 @@ TEST(NetlinkRouteTest, GetLinkDump) { EXPECT_TRUE(loopbackFound); } +TEST(NetlinkRouteTest, MsgHdrMsgTrunc) { + FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(NetlinkBoundSocket()); + + struct request { + struct nlmsghdr hdr; + struct ifinfomsg ifm; + }; + + constexpr uint32_t kSeq = 12345; + + struct request req = {}; + req.hdr.nlmsg_len = sizeof(req); + req.hdr.nlmsg_type = RTM_GETLINK; + req.hdr.nlmsg_flags = NLM_F_REQUEST | NLM_F_DUMP; + req.hdr.nlmsg_seq = kSeq; + req.ifm.ifi_family = AF_UNSPEC; + + struct iovec iov = {}; + iov.iov_base = &req; + iov.iov_len = sizeof(req); + + struct msghdr msg = {}; + msg.msg_iov = &iov; + msg.msg_iovlen = 1; + // No destination required; it defaults to pid 0, the kernel. + + ASSERT_THAT(RetryEINTR(sendmsg)(fd.get(), &msg, 0), SyscallSucceeds()); + + // Small enough to ensure that the response doesn't fit. + constexpr size_t kBufferSize = 10; + std::vector buf(kBufferSize); + iov.iov_base = buf.data(); + iov.iov_len = buf.size(); + + ASSERT_THAT(RetryEINTR(recvmsg)(fd.get(), &msg, 0), + SyscallSucceedsWithValue(kBufferSize)); + EXPECT_EQ((msg.msg_flags & MSG_TRUNC), MSG_TRUNC); +} + +TEST(NetlinkRouteTest, MsgTruncMsgHdrMsgTrunc) { + FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(NetlinkBoundSocket()); + + struct request { + struct nlmsghdr hdr; + struct ifinfomsg ifm; + }; + + constexpr uint32_t kSeq = 12345; + + struct request req = {}; + req.hdr.nlmsg_len = sizeof(req); + req.hdr.nlmsg_type = RTM_GETLINK; + req.hdr.nlmsg_flags = NLM_F_REQUEST | NLM_F_DUMP; + req.hdr.nlmsg_seq = kSeq; + req.ifm.ifi_family = AF_UNSPEC; + + struct iovec iov = {}; + iov.iov_base = &req; + iov.iov_len = sizeof(req); + + struct msghdr msg = {}; + msg.msg_iov = &iov; + msg.msg_iovlen = 1; + // No destination required; it defaults to pid 0, the kernel. + + ASSERT_THAT(RetryEINTR(sendmsg)(fd.get(), &msg, 0), SyscallSucceeds()); + + // Small enough to ensure that the response doesn't fit. + constexpr size_t kBufferSize = 10; + std::vector buf(kBufferSize); + iov.iov_base = buf.data(); + iov.iov_len = buf.size(); + + int res = 0; + ASSERT_THAT(res = RetryEINTR(recvmsg)(fd.get(), &msg, MSG_TRUNC), + SyscallSucceeds()); + EXPECT_GT(res, kBufferSize); + EXPECT_EQ((msg.msg_flags & MSG_TRUNC), MSG_TRUNC); +} + TEST(NetlinkRouteTest, ControlMessageIgnored) { FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(NetlinkBoundSocket()); uint32_t port = ASSERT_NO_ERRNO_AND_VALUE(NetlinkPortID(fd.get())); diff --git a/test/syscalls/linux/socket_non_stream.cc b/test/syscalls/linux/socket_non_stream.cc index d49aab363..d170008a4 100644 --- a/test/syscalls/linux/socket_non_stream.cc +++ b/test/syscalls/linux/socket_non_stream.cc @@ -15,6 +15,7 @@ #include "test/syscalls/linux/socket_non_stream.h" #include +#include #include #include "gtest/gtest.h" @@ -89,6 +90,33 @@ TEST_P(NonStreamSocketPairTest, SingleRecv) { EXPECT_EQ(0, memcmp(sent_data1, received_data, sizeof(sent_data1))); } +TEST_P(NonStreamSocketPairTest, RecvmsgMsghdrFlagMsgTrunc) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + char sent_data[10]; + RandomizeBuffer(sent_data, sizeof(sent_data)); + ASSERT_THAT( + RetryEINTR(send)(sockets->first_fd(), sent_data, sizeof(sent_data), 0), + SyscallSucceedsWithValue(sizeof(sent_data))); + + char received_data[sizeof(sent_data) / 2] = {}; + + struct iovec iov; + iov.iov_base = received_data; + iov.iov_len = sizeof(received_data); + struct msghdr msg = {}; + msg.msg_flags = -1; + msg.msg_iov = &iov; + msg.msg_iovlen = 1; + + ASSERT_THAT(RetryEINTR(recvmsg)(sockets->second_fd(), &msg, 0), + SyscallSucceedsWithValue(sizeof(received_data))); + EXPECT_EQ(0, memcmp(received_data, sent_data, sizeof(received_data))); + + // Check that msghdr flags were updated. + EXPECT_EQ(msg.msg_flags, MSG_TRUNC); +} + // Stream sockets allow data sent with multiple sends to be peeked at in a // single recv. Datagram sockets (except for unix sockets) do not. // @@ -142,6 +170,33 @@ TEST_P(NonStreamSocketPairTest, MsgTruncTruncation) { sizeof(sent_data) / 2)); } +TEST_P(NonStreamSocketPairTest, MsgTruncTruncationRecvmsgMsghdrFlagMsgTrunc) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + char sent_data[10]; + RandomizeBuffer(sent_data, sizeof(sent_data)); + ASSERT_THAT( + RetryEINTR(send)(sockets->first_fd(), sent_data, sizeof(sent_data), 0), + SyscallSucceedsWithValue(sizeof(sent_data))); + + char received_data[sizeof(sent_data) / 2] = {}; + + struct iovec iov; + iov.iov_base = received_data; + iov.iov_len = sizeof(received_data); + struct msghdr msg = {}; + msg.msg_flags = -1; + msg.msg_iov = &iov; + msg.msg_iovlen = 1; + + ASSERT_THAT(RetryEINTR(recvmsg)(sockets->second_fd(), &msg, MSG_TRUNC), + SyscallSucceedsWithValue(sizeof(sent_data))); + EXPECT_EQ(0, memcmp(received_data, sent_data, sizeof(received_data))); + + // Check that msghdr flags were updated. + EXPECT_EQ(msg.msg_flags, MSG_TRUNC); +} + TEST_P(NonStreamSocketPairTest, MsgTruncSameSize) { auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); char sent_data[512]; diff --git a/test/syscalls/linux/socket_stream.cc b/test/syscalls/linux/socket_stream.cc index 32e9d958b..c8a8ad0f6 100644 --- a/test/syscalls/linux/socket_stream.cc +++ b/test/syscalls/linux/socket_stream.cc @@ -81,6 +81,33 @@ TEST_P(StreamSocketPairTest, WriteOneSideClosed) { SyscallFailsWithErrno(EPIPE)); } +TEST_P(StreamSocketPairTest, RecvmsgMsghdrFlagsNoMsgTrunc) { + auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); + + char sent_data[10]; + RandomizeBuffer(sent_data, sizeof(sent_data)); + ASSERT_THAT( + RetryEINTR(send)(sockets->first_fd(), sent_data, sizeof(sent_data), 0), + SyscallSucceedsWithValue(sizeof(sent_data))); + + char received_data[sizeof(sent_data) / 2] = {}; + + struct iovec iov; + iov.iov_base = received_data; + iov.iov_len = sizeof(received_data); + struct msghdr msg = {}; + msg.msg_flags = -1; + msg.msg_iov = &iov; + msg.msg_iovlen = 1; + + ASSERT_THAT(RetryEINTR(recvmsg)(sockets->second_fd(), &msg, 0), + SyscallSucceedsWithValue(sizeof(received_data))); + EXPECT_EQ(0, memcmp(received_data, sent_data, sizeof(received_data))); + + // Check that msghdr flags were cleared (MSG_TRUNC was not set). + EXPECT_EQ(msg.msg_flags, 0); +} + TEST_P(StreamSocketPairTest, MsgTrunc) { auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); char sent_data[512]; -- cgit v1.2.3 From f4ce43e1f426148d99c28c1b0e5c43ddda17a8cb Mon Sep 17 00:00:00 2001 From: Nicolas Lacasse Date: Mon, 29 Apr 2019 14:03:04 -0700 Subject: Allow and document bug ids in gVisor codebase. PiperOrigin-RevId: 245818639 Change-Id: I03703ef0fb9b6675955637b9fe2776204c545789 --- CONTRIBUTING.md | 7 +++ pkg/cpuid/cpuid_test.go | 2 +- pkg/dhcp/client.go | 2 +- pkg/log/glog.go | 2 +- pkg/metric/metric.go | 4 +- pkg/segment/set.go | 2 +- pkg/segment/test/set_functions.go | 2 +- pkg/sentry/arch/arch.go | 2 +- pkg/sentry/arch/arch_amd64.go | 4 +- pkg/sentry/arch/arch_x86.go | 2 +- pkg/sentry/arch/signal_amd64.go | 6 +-- pkg/sentry/arch/stack.go | 6 +-- pkg/sentry/context/context.go | 2 +- pkg/sentry/control/proc.go | 2 +- pkg/sentry/fs/README.md | 2 +- pkg/sentry/fs/ashmem/area.go | 4 +- pkg/sentry/fs/binder/binder.go | 22 ++++---- pkg/sentry/fs/dentry.go | 2 +- pkg/sentry/fs/dirent.go | 8 +-- pkg/sentry/fs/file.go | 2 +- pkg/sentry/fs/file_overlay.go | 4 +- pkg/sentry/fs/fsutil/file.go | 8 +-- pkg/sentry/fs/fsutil/inode_cached.go | 4 +- pkg/sentry/fs/gofer/cache_policy.go | 4 +- pkg/sentry/fs/gofer/file.go | 2 +- pkg/sentry/fs/gofer/file_state.go | 2 +- pkg/sentry/fs/gofer/handles.go | 2 +- pkg/sentry/fs/gofer/inode.go | 6 +-- pkg/sentry/fs/gofer/inode_state.go | 2 +- pkg/sentry/fs/gofer/session.go | 2 +- pkg/sentry/fs/gofer/session_state.go | 2 +- pkg/sentry/fs/host/fs.go | 4 +- pkg/sentry/fs/host/inode.go | 10 ++-- pkg/sentry/fs/inode.go | 6 +-- pkg/sentry/fs/inode_operations.go | 2 +- pkg/sentry/fs/inode_overlay.go | 6 +-- pkg/sentry/fs/mount.go | 4 +- pkg/sentry/fs/mount_test.go | 2 +- pkg/sentry/fs/proc/README.md | 12 ++--- pkg/sentry/fs/proc/fds.go | 2 +- pkg/sentry/fs/proc/loadavg.go | 2 +- pkg/sentry/fs/proc/meminfo.go | 6 +-- pkg/sentry/fs/proc/mounts.go | 2 +- pkg/sentry/fs/proc/net.go | 2 +- pkg/sentry/fs/proc/stat.go | 12 ++--- pkg/sentry/fs/proc/sys_net.go | 2 +- pkg/sentry/fs/proc/task.go | 8 +-- pkg/sentry/fs/proc/version.go | 2 +- pkg/sentry/fs/ramfs/dir.go | 2 +- pkg/sentry/fs/tmpfs/fs.go | 2 +- pkg/sentry/fs/tmpfs/inode_file.go | 2 +- pkg/sentry/fs/tmpfs/tmpfs.go | 2 +- pkg/sentry/fs/tty/dir.go | 6 +-- pkg/sentry/fs/tty/fs.go | 2 +- pkg/sentry/fs/tty/master.go | 6 +-- pkg/sentry/fs/tty/slave.go | 6 +-- pkg/sentry/kernel/auth/credentials.go | 2 +- pkg/sentry/kernel/auth/user_namespace.go | 2 +- pkg/sentry/kernel/pending_signals.go | 2 +- pkg/sentry/kernel/ptrace.go | 4 +- pkg/sentry/kernel/rseq.go | 2 +- pkg/sentry/kernel/sched/cpuset.go | 2 +- pkg/sentry/kernel/semaphore/semaphore.go | 6 +-- pkg/sentry/kernel/shm/shm.go | 2 +- pkg/sentry/kernel/syscalls.go | 2 +- pkg/sentry/kernel/task_context.go | 2 +- pkg/sentry/kernel/task_exec.go | 2 +- pkg/sentry/kernel/task_exit.go | 4 +- pkg/sentry/kernel/task_identity.go | 2 +- pkg/sentry/kernel/task_run.go | 2 +- pkg/sentry/kernel/task_signals.go | 4 +- pkg/sentry/kernel/task_stop.go | 2 +- pkg/sentry/loader/loader.go | 2 +- pkg/sentry/loader/vdso.go | 6 +-- pkg/sentry/memmap/memmap.go | 2 +- pkg/sentry/mm/aio_context.go | 2 +- pkg/sentry/mm/procfs.go | 10 ++-- pkg/sentry/mm/special_mappable.go | 2 +- pkg/sentry/mm/syscalls.go | 6 +-- pkg/sentry/mm/vma.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64_unsafe.go | 2 +- pkg/sentry/platform/platform.go | 2 +- pkg/sentry/platform/ptrace/subprocess.go | 2 +- pkg/sentry/platform/ring0/x86.go | 4 +- pkg/sentry/sighandling/sighandling.go | 2 +- pkg/sentry/sighandling/sighandling_unsafe.go | 2 +- pkg/sentry/socket/epsocket/epsocket.go | 32 ++++++------ pkg/sentry/socket/epsocket/save_restore.go | 2 +- pkg/sentry/socket/epsocket/stack.go | 2 +- pkg/sentry/socket/hostinet/socket.go | 2 +- pkg/sentry/socket/netlink/route/protocol.go | 8 +-- pkg/sentry/socket/netlink/socket.go | 10 ++-- pkg/sentry/socket/rpcinet/conn/conn.go | 2 +- pkg/sentry/socket/rpcinet/notifier/notifier.go | 4 +- pkg/sentry/socket/rpcinet/socket.go | 6 +-- pkg/sentry/socket/rpcinet/syscall_rpc.proto | 2 +- pkg/sentry/strace/strace.go | 2 +- pkg/sentry/syscalls/linux/error.go | 2 +- pkg/sentry/syscalls/linux/linux64.go | 60 +++++++++++----------- pkg/sentry/syscalls/linux/sys_aio.go | 2 +- pkg/sentry/syscalls/linux/sys_file.go | 4 +- pkg/sentry/syscalls/linux/sys_mmap.go | 4 +- pkg/sentry/syscalls/linux/sys_read.go | 2 +- pkg/sentry/syscalls/linux/sys_socket.go | 4 +- pkg/sentry/syscalls/linux/sys_thread.go | 2 +- pkg/sentry/syscalls/linux/sys_write.go | 4 +- pkg/sentry/time/calibrated_clock.go | 6 +-- pkg/sentry/time/parameters.go | 2 +- pkg/sentry/usermem/usermem.go | 4 +- pkg/sentry/watchdog/watchdog.go | 2 +- pkg/syserr/syserr.go | 10 ++-- pkg/tcpip/network/ipv4/icmp.go | 2 +- pkg/tcpip/network/ipv6/icmp.go | 4 +- pkg/tcpip/stack/nic.go | 6 +-- pkg/tcpip/stack/stack.go | 4 +- pkg/tcpip/stack/stack_global_state.go | 2 +- pkg/tcpip/stack/transport_test.go | 2 +- pkg/tcpip/tcpip.go | 2 +- pkg/tcpip/transport/raw/raw.go | 2 +- pkg/tcpip/transport/tcp/BUILD | 2 +- pkg/unet/unet.go | 2 +- pkg/unet/unet_test.go | 2 +- runsc/boot/controller.go | 4 +- runsc/boot/fs.go | 6 +-- runsc/boot/loader.go | 2 +- runsc/cmd/checkpoint.go | 2 +- runsc/container/container.go | 2 +- runsc/container/container_test.go | 4 +- runsc/sandbox/sandbox.go | 6 +-- runsc/specutils/specutils.go | 4 +- test/syscalls/BUILD | 6 +-- test/syscalls/build_defs.bzl | 4 +- test/syscalls/linux/32bit.cc | 14 ++--- test/syscalls/linux/aio.cc | 2 +- test/syscalls/linux/chmod.cc | 2 +- test/syscalls/linux/epoll.cc | 2 +- test/syscalls/linux/exec_binary.cc | 12 ++--- test/syscalls/linux/file_base.h | 4 +- test/syscalls/linux/ioctl.cc | 4 +- test/syscalls/linux/ip_socket_test_util.cc | 2 +- test/syscalls/linux/lseek.cc | 2 +- test/syscalls/linux/mkdir.cc | 2 +- test/syscalls/linux/mmap.cc | 18 +++---- test/syscalls/linux/open.cc | 2 +- test/syscalls/linux/partial_bad_buffer.cc | 18 +++---- test/syscalls/linux/pipe.cc | 6 +-- test/syscalls/linux/proc.cc | 32 ++++++------ test/syscalls/linux/proc_pid_smaps.cc | 2 +- test/syscalls/linux/ptrace.cc | 2 +- test/syscalls/linux/pwrite64.cc | 2 +- test/syscalls/linux/readv_socket.cc | 2 +- test/syscalls/linux/rtsignal.cc | 2 +- test/syscalls/linux/socket_inet_loopback.cc | 10 ++-- .../socket_ipv4_udp_unbound_external_networking.cc | 4 +- test/syscalls/linux/socket_netlink_route.cc | 4 +- test/syscalls/linux/socket_stream_blocking.cc | 2 +- test/syscalls/linux/socket_test_util.cc | 2 +- test/syscalls/linux/socket_unix.cc | 16 +++--- test/syscalls/linux/socket_unix_dgram.cc | 2 +- .../linux/socket_unix_dgram_non_blocking.cc | 2 +- test/syscalls/linux/socket_unix_non_stream.cc | 10 ++-- .../linux/socket_unix_unbound_seqpacket.cc | 2 +- test/syscalls/linux/socket_unix_unbound_stream.cc | 4 +- test/syscalls/linux/stat.cc | 2 +- test/syscalls/linux/stat_times.cc | 8 +-- test/syscalls/linux/tcp_socket.cc | 2 +- test/syscalls/linux/tkill.cc | 2 +- test/syscalls/linux/udp_bind.cc | 4 +- test/syscalls/linux/uidgid.cc | 2 +- test/syscalls/linux/utimes.cc | 4 +- test/syscalls/linux/wait.cc | 2 +- test/syscalls/linux/write.cc | 2 +- third_party/gvsync/downgradable_rwmutex_unsafe.go | 2 +- vdso/cycle_clock.h | 2 +- vdso/vdso_amd64.lds | 2 +- vdso/vdso_arm64.lds | 2 +- 176 files changed, 403 insertions(+), 396 deletions(-) (limited to 'pkg/sentry/socket/rpcinet') diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index d6dafc595..238dd6665 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -126,6 +126,13 @@ change. When approved, the change will be submitted by a team member and automatically merged into the repository. +### Bug IDs + +Some TODOs and NOTEs sprinkled throughout the code have associated IDs of the +form b/1234. These correspond to bugs in our internal bug tracker. Eventually +these bugs will be moved to the GitHub Issues, but until then they can simply be +ignored. + ### The small print Contributions made by corporations are covered by a different agreement than the diff --git a/pkg/cpuid/cpuid_test.go b/pkg/cpuid/cpuid_test.go index 35e7b8e50..64ade1cbe 100644 --- a/pkg/cpuid/cpuid_test.go +++ b/pkg/cpuid/cpuid_test.go @@ -78,7 +78,7 @@ func TestTakeFeatureIntersection(t *testing.T) { } } -// TODO: Run this test on a very old platform, and make sure more +// TODO(b/73346484): Run this test on a very old platform, and make sure more // bits are enabled than just FPU and PAE. This test currently may not detect // if HostFeatureSet gives back junk bits. func TestHostFeatureSet(t *testing.T) { diff --git a/pkg/dhcp/client.go b/pkg/dhcp/client.go index 354205e63..2ba79be32 100644 --- a/pkg/dhcp/client.go +++ b/pkg/dhcp/client.go @@ -120,7 +120,7 @@ func (c *Client) Config() Config { // If the server sets a lease limit a timer is set to automatically // renew it. func (c *Client) Request(ctx context.Context, requestedAddr tcpip.Address) (cfg Config, reterr error) { - // TODO: remove calls to {Add,Remove}Address when they're no + // TODO(b/127321246): remove calls to {Add,Remove}Address when they're no // longer required to send and receive broadcast. if err := c.stack.AddAddressWithOptions(c.nicid, ipv4.ProtocolNumber, tcpipHeader.IPv4Any, stack.NeverPrimaryEndpoint); err != nil && err != tcpip.ErrDuplicateAddress { return Config{}, fmt.Errorf("dhcp: AddAddressWithOptions(): %s", err) diff --git a/pkg/log/glog.go b/pkg/log/glog.go index fbb58501b..24d5390d7 100644 --- a/pkg/log/glog.go +++ b/pkg/log/glog.go @@ -144,7 +144,7 @@ func (g GoogleEmitter) Emit(level Level, timestamp time.Time, format string, arg b.writeAll(pid) b.write(' ') - // FIXME: The caller, fabricated. This really sucks, but it + // FIXME(b/73383460): The caller, fabricated. This really sucks, but it // is unacceptable to put runtime.Callers() in the hot path. b.writeAll(caller) b.write(']') diff --git a/pkg/metric/metric.go b/pkg/metric/metric.go index 02af75974..e5eb95f89 100644 --- a/pkg/metric/metric.go +++ b/pkg/metric/metric.go @@ -44,8 +44,8 @@ var ( // // Metrics are not saved across save/restore and thus reset to zero on restore. // -// TODO: Support non-cumulative metrics. -// TODO: Support metric fields. +// TODO(b/67298402): Support non-cumulative metrics. +// TODO(b/67298427): Support metric fields. // type Uint64Metric struct { // value is the actual value of the metric. It must be accessed diff --git a/pkg/segment/set.go b/pkg/segment/set.go index a9a3b8875..74a916ea3 100644 --- a/pkg/segment/set.go +++ b/pkg/segment/set.go @@ -1270,7 +1270,7 @@ func segmentAfterPosition(n *node, i int) Iterator { } func zeroValueSlice(slice []Value) { - // TODO: check if Go is actually smart enough to optimize a + // TODO(jamieliu): check if Go is actually smart enough to optimize a // ClearValue that assigns nil to a memset here for i := range slice { Functions{}.ClearValue(&slice[i]) diff --git a/pkg/segment/test/set_functions.go b/pkg/segment/test/set_functions.go index 05ba5fbb9..41f649011 100644 --- a/pkg/segment/test/set_functions.go +++ b/pkg/segment/test/set_functions.go @@ -15,7 +15,7 @@ package segment // Basic numeric constants that we define because the math package doesn't. -// TODO: These should be Math.MaxInt64/MinInt64? +// TODO(nlacasse): These should be Math.MaxInt64/MinInt64? const ( maxInt = int(^uint(0) >> 1) minInt = -maxInt - 1 diff --git a/pkg/sentry/arch/arch.go b/pkg/sentry/arch/arch.go index 4cd7a9af5..16d8eb2b2 100644 --- a/pkg/sentry/arch/arch.go +++ b/pkg/sentry/arch/arch.go @@ -53,7 +53,7 @@ type FloatingPointData byte // Context provides architecture-dependent information for a specific thread. // -// NOTE: Currently we use uintptr here to refer to a generic native +// NOTE(b/34169503): Currently we use uintptr here to refer to a generic native // register value. While this will work for the foreseeable future, it isn't // strictly correct. We may want to create some abstraction that makes this // more clear or enables us to store values of arbitrary widths. This is diff --git a/pkg/sentry/arch/arch_amd64.go b/pkg/sentry/arch/arch_amd64.go index 2507774f7..7ec2f2c84 100644 --- a/pkg/sentry/arch/arch_amd64.go +++ b/pkg/sentry/arch/arch_amd64.go @@ -305,7 +305,7 @@ func (c *context64) PtracePeekUser(addr uintptr) (interface{}, error) { buf := binary.Marshal(nil, usermem.ByteOrder, c.ptraceGetRegs()) return c.Native(uintptr(usermem.ByteOrder.Uint64(buf[addr:]))), nil } - // TODO: debug registers + // TODO(b/34088053): debug registers return c.Native(0), nil } @@ -320,6 +320,6 @@ func (c *context64) PtracePokeUser(addr, data uintptr) error { _, err := c.PtraceSetRegs(bytes.NewBuffer(buf)) return err } - // TODO: debug registers + // TODO(b/34088053): debug registers return nil } diff --git a/pkg/sentry/arch/arch_x86.go b/pkg/sentry/arch/arch_x86.go index c8bf0e7f2..4305fe2cb 100644 --- a/pkg/sentry/arch/arch_x86.go +++ b/pkg/sentry/arch/arch_x86.go @@ -306,7 +306,7 @@ func (s *State) ptraceGetRegs() syscall.PtraceRegs { // FS/GS_TLS_SEL when fs_base/gs_base is a 64-bit value. (We do the // same in PtraceSetRegs.) // - // TODO: Remove this fixup since newer Linux + // TODO(gvisor.dev/issue/168): Remove this fixup since newer Linux // doesn't have this behavior anymore. if regs.Fs == 0 && regs.Fs_base <= 0xffffffff { regs.Fs = _FS_TLS_SEL diff --git a/pkg/sentry/arch/signal_amd64.go b/pkg/sentry/arch/signal_amd64.go index c9de36897..7f76eba27 100644 --- a/pkg/sentry/arch/signal_amd64.go +++ b/pkg/sentry/arch/signal_amd64.go @@ -319,7 +319,7 @@ func (c *context64) NewSignalStack() NativeSignalStack { // From Linux 'arch/x86/include/uapi/asm/sigcontext.h' the following is the // size of the magic cookie at the end of the xsave frame. // -// NOTE: Currently we don't actually populate the fpstate +// NOTE(b/33003106#comment11): Currently we don't actually populate the fpstate // on the signal stack. const _FP_XSTATE_MAGIC2_SIZE = 4 @@ -392,7 +392,7 @@ func (c *context64) SignalSetup(st *Stack, act *SignalAct, info *SignalInfo, alt Sigset: sigset, } - // TODO: Set SignalContext64.Err, Trapno, and Cr2 + // TODO(gvisor.dev/issue/159): Set SignalContext64.Err, Trapno, and Cr2 // based on the fault that caused the signal. For now, leave Err and // Trapno unset and assume CR2 == info.Addr() for SIGSEGVs and // SIGBUSes. @@ -505,7 +505,7 @@ func (c *context64) SignalRestore(st *Stack, rt bool) (linux.SignalSet, SignalSt l := len(c.sigFPState) if l > 0 { c.x86FPState = c.sigFPState[l-1] - // NOTE: State save requires that any slice + // NOTE(cl/133042258): State save requires that any slice // elements from '[len:cap]' to be zero value. c.sigFPState[l-1] = nil c.sigFPState = c.sigFPState[0 : l-1] diff --git a/pkg/sentry/arch/stack.go b/pkg/sentry/arch/stack.go index f2cfb0426..2e33ccdf5 100644 --- a/pkg/sentry/arch/stack.go +++ b/pkg/sentry/arch/stack.go @@ -97,7 +97,7 @@ func (s *Stack) Push(vals ...interface{}) (usermem.Addr, error) { if c < 0 { return 0, fmt.Errorf("bad binary.Size for %T", v) } - // TODO: Use a real context.Context. + // TODO(b/38173783): Use a real context.Context. n, err := usermem.CopyObjectOut(context.Background(), s.IO, s.Bottom-usermem.Addr(c), norm, usermem.IOOpts{}) if err != nil || c != n { return 0, err @@ -121,11 +121,11 @@ func (s *Stack) Pop(vals ...interface{}) (usermem.Addr, error) { var err error if isVaddr { value := s.Arch.Native(uintptr(0)) - // TODO: Use a real context.Context. + // TODO(b/38173783): Use a real context.Context. n, err = usermem.CopyObjectIn(context.Background(), s.IO, s.Bottom, value, usermem.IOOpts{}) *vaddr = usermem.Addr(s.Arch.Value(value)) } else { - // TODO: Use a real context.Context. + // TODO(b/38173783): Use a real context.Context. n, err = usermem.CopyObjectIn(context.Background(), s.IO, s.Bottom, v, usermem.IOOpts{}) } if err != nil { diff --git a/pkg/sentry/context/context.go b/pkg/sentry/context/context.go index 7ed6a5e8a..eefc3e1b4 100644 --- a/pkg/sentry/context/context.go +++ b/pkg/sentry/context/context.go @@ -114,7 +114,7 @@ var bgContext = &logContext{Logger: log.Log()} // Background returns an empty context using the default logger. // // Users should be wary of using a Background context. Please tag any use with -// FIXME and a note to remove this use. +// FIXME(b/38173783) and a note to remove this use. // // Generally, one should use the Task as their context when available, or avoid // having to use a context in places where a Task is unavailable. diff --git a/pkg/sentry/control/proc.go b/pkg/sentry/control/proc.go index e848def14..aca2267a7 100644 --- a/pkg/sentry/control/proc.go +++ b/pkg/sentry/control/proc.go @@ -261,7 +261,7 @@ func (proc *Proc) Ps(args *PsArgs, out *string) error { } // Process contains information about a single process in a Sandbox. -// TODO: Implement TTY field. +// TODO(b/117881927): Implement TTY field. type Process struct { UID auth.KUID `json:"uid"` PID kernel.ThreadID `json:"pid"` diff --git a/pkg/sentry/fs/README.md b/pkg/sentry/fs/README.md index a88a0cd3a..f53ed3eaa 100644 --- a/pkg/sentry/fs/README.md +++ b/pkg/sentry/fs/README.md @@ -59,7 +59,7 @@ two categories: The first is always necessary to save and restore. An application may never have any open file descriptors, but across save and restore it should see a coherent -view of any mount namespace. NOTE: Currently only one "initial" +view of any mount namespace. NOTE(b/63601033): Currently only one "initial" mount namespace is supported. The second is so that system calls across save and restore are coherent with diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go index 651cbc164..1f61c5711 100644 --- a/pkg/sentry/fs/ashmem/area.go +++ b/pkg/sentry/fs/ashmem/area.go @@ -240,7 +240,7 @@ func (a *Area) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArgume return 0, syserror.EINVAL } - // TODO: If personality flag + // TODO(b/30946773,gvisor.dev/issue/153): If personality flag // READ_IMPLIES_EXEC is set, set PROT_EXEC if PORT_READ is set. a.perms = perms @@ -290,7 +290,7 @@ func (a *Area) pinOperation(pin linux.AshmemPin, op uint32) (uintptr, error) { return linux.AshmemNotPurged, nil case linux.AshmemUnpinIoctl: - // TODO: Implement purge on unpin. + // TODO(b/30946773): Implement purge on unpin. a.pb.UnpinRange(r) return 0, nil diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go index a41b5dcae..d9f1559de 100644 --- a/pkg/sentry/fs/binder/binder.go +++ b/pkg/sentry/fs/binder/binder.go @@ -69,7 +69,7 @@ func NewDevice(ctx context.Context, owner fs.FileOwner, fp fs.FilePermissions) * // GetFile implements fs.InodeOperations.GetFile. // -// TODO: Add functionality to GetFile: Additional fields will be +// TODO(b/30946773): Add functionality to GetFile: Additional fields will be // needed in the Device structure, initialize them here. Also, Device will need // to keep track of the created Procs in order to implement BINDER_READ_WRITE // ioctl. @@ -133,7 +133,7 @@ func (bp *Proc) Write(ctx context.Context, file *fs.File, src usermem.IOSequence // Flush implements fs.FileOperations.Flush. // -// TODO: Implement. +// TODO(b/30946773): Implement. func (bp *Proc) Flush(ctx context.Context, file *fs.File) error { return nil } @@ -149,7 +149,7 @@ func (bp *Proc) ConfigureMMap(ctx context.Context, file *fs.File, opts *memmap.M } opts.MaxPerms.Write = false - // TODO: Binder sets VM_DONTCOPY, preventing the created vma + // TODO(b/30946773): Binder sets VM_DONTCOPY, preventing the created vma // from being copied across fork(), but we don't support this yet. As // a result, MMs containing a Binder mapping cannot be forked (MM.Fork will // fail when AddMapping returns EBUSY). @@ -159,7 +159,7 @@ func (bp *Proc) ConfigureMMap(ctx context.Context, file *fs.File, opts *memmap.M // Ioctl implements fs.FileOperations.Ioctl. // -// TODO: Implement. +// TODO(b/30946773): Implement. func (bp *Proc) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { // Switch on ioctl request. switch uint32(args[1].Int()) { @@ -173,22 +173,22 @@ func (bp *Proc) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArgum }) return 0, err case linux.BinderWriteReadIoctl: - // TODO: Implement. + // TODO(b/30946773): Implement. fallthrough case linux.BinderSetIdleTimeoutIoctl: - // TODO: Implement. + // TODO(b/30946773): Implement. fallthrough case linux.BinderSetMaxThreadsIoctl: - // TODO: Implement. + // TODO(b/30946773): Implement. fallthrough case linux.BinderSetIdlePriorityIoctl: - // TODO: Implement. + // TODO(b/30946773): Implement. fallthrough case linux.BinderSetContextMgrIoctl: - // TODO: Implement. + // TODO(b/30946773): Implement. fallthrough case linux.BinderThreadExitIoctl: - // TODO: Implement. + // TODO(b/30946773): Implement. return 0, syserror.ENOSYS default: // Ioctls irrelevant to Binder. @@ -228,7 +228,7 @@ func (bp *Proc) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, // Translate implements memmap.Mappable.Translate. func (bp *Proc) Translate(ctx context.Context, required, optional memmap.MappableRange, at usermem.AccessType) ([]memmap.Translation, error) { - // TODO: In addition to the page initially allocated and mapped + // TODO(b/30946773): In addition to the page initially allocated and mapped // in AddMapping (Linux: binder_mmap), Binder allocates and maps pages for // each transaction (Linux: binder_ioctl => binder_ioctl_write_read => // binder_thread_write => binder_transaction => binder_alloc_buf => diff --git a/pkg/sentry/fs/dentry.go b/pkg/sentry/fs/dentry.go index 4879df4d6..29fb155a4 100644 --- a/pkg/sentry/fs/dentry.go +++ b/pkg/sentry/fs/dentry.go @@ -83,7 +83,7 @@ type DirCtx struct { attrs map[string]DentAttr // DirCursor is the directory cursor. - // TODO: Once Handles are removed this can just live in the + // TODO(b/67778717): Once Handles are removed this can just live in the // respective FileOperations implementations and not need to get // plumbed everywhere. DirCursor *string diff --git a/pkg/sentry/fs/dirent.go b/pkg/sentry/fs/dirent.go index 4bcdf530a..54fc11fe1 100644 --- a/pkg/sentry/fs/dirent.go +++ b/pkg/sentry/fs/dirent.go @@ -318,7 +318,7 @@ func (d *Dirent) SyncAll(ctx context.Context) { // There is nothing to sync for a read-only filesystem. if !d.Inode.MountSource.Flags.ReadOnly { - // FIXME: This should be a mount traversal, not a + // FIXME(b/34856369): This should be a mount traversal, not a // Dirent traversal, because some Inodes that need to be synced // may no longer be reachable by name (after sys_unlink). // @@ -1506,7 +1506,7 @@ func Rename(ctx context.Context, root *Dirent, oldParent *Dirent, oldName string } // Are we frozen? - // TODO: Is this the right errno? + // TODO(jamieliu): Is this the right errno? if oldParent.frozen && !oldParent.Inode.IsVirtual() { return syscall.ENOENT } @@ -1565,7 +1565,7 @@ func Rename(ctx context.Context, root *Dirent, oldParent *Dirent, oldName string } else { // Check constraints on the dirent being replaced. - // NOTE: We don't want to keep replaced alive + // NOTE(b/111808347): We don't want to keep replaced alive // across the Rename, so must call DecRef manually (no defer). // Check that we can delete replaced. @@ -1606,7 +1606,7 @@ func Rename(ctx context.Context, root *Dirent, oldParent *Dirent, oldName string // Allow the file system to drop extra references on replaced. replaced.dropExtendedReference() - // NOTE: Keeping a dirent + // NOTE(b/31798319,b/31867149,b/31867671): Keeping a dirent // open across renames is currently broken for multiple // reasons, so we flush all references on the replaced node and // its children. diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go index 2c2126f17..5d5026661 100644 --- a/pkg/sentry/fs/file.go +++ b/pkg/sentry/fs/file.go @@ -65,7 +65,7 @@ const FileMaxOffset = math.MaxInt64 // under a single abortable mutex which also synchronizes lseek(2), read(2), // and write(2). // -// FIXME: Split synchronization from cancellation. +// FIXME(b/38451980): Split synchronization from cancellation. // // +stateify savable type File struct { diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go index e1f02f0f4..6e680f0a4 100644 --- a/pkg/sentry/fs/file_overlay.go +++ b/pkg/sentry/fs/file_overlay.go @@ -160,7 +160,7 @@ func (f *overlayFileOperations) Seek(ctx context.Context, file *File, whence See // If this was a seek on a directory, we must update the cursor. if seekDir && whence == SeekSet && offset == 0 { // Currently only seeking to 0 on a directory is supported. - // FIXME: Lift directory seeking limitations. + // FIXME(b/33075855): Lift directory seeking limitations. f.dirCursor = "" } return n, nil @@ -329,7 +329,7 @@ func (*overlayFileOperations) ConfigureMMap(ctx context.Context, file *File, opt if !o.isMappableLocked() { return syserror.ENODEV } - // FIXME: This is a copy/paste of fsutil.GenericConfigureMMap, + // FIXME(jamieliu): This is a copy/paste of fsutil.GenericConfigureMMap, // which we can't use because the overlay implementation is in package fs, // so depending on fs/fsutil would create a circular dependency. Move // overlay to fs/overlay. diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index df34dc788..42afdd11c 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -36,7 +36,7 @@ func (FileNoopRelease) Release() {} // // Currently only seeking to 0 on a directory is supported. // -// FIXME: Lift directory seeking limitations. +// FIXME(b/33075855): Lift directory seeking limitations. func SeekWithDirCursor(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64, dirCursor *string) (int64, error) { inode := file.Dirent.Inode current := file.Offset() @@ -50,7 +50,7 @@ func SeekWithDirCursor(ctx context.Context, file *fs.File, whence fs.SeekWhence, if fs.IsCharDevice(inode.StableAttr) { // Ignore seek requests. // - // FIXME: This preserves existing + // FIXME(b/34716638): This preserves existing // behavior but is not universally correct. return 0, nil } @@ -104,7 +104,7 @@ func SeekWithDirCursor(ctx context.Context, file *fs.File, whence fs.SeekWhence, return current, syserror.EINVAL } return sz + offset, nil - // FIXME: This is not universally correct. + // FIXME(b/34778850): This is not universally correct. // Remove SpecialDirectory. case fs.SpecialDirectory: if offset != 0 { @@ -112,7 +112,7 @@ func SeekWithDirCursor(ctx context.Context, file *fs.File, whence fs.SeekWhence, } // SEEK_END to 0 moves the directory "cursor" to the end. // - // FIXME: The ensures that after the seek, + // FIXME(b/35442290): The ensures that after the seek, // reading on the directory will get EOF. But it is not // correct in general because the directory can grow in // size; attempting to read those new entries will be diff --git a/pkg/sentry/fs/fsutil/inode_cached.go b/pkg/sentry/fs/fsutil/inode_cached.go index b690cfe93..ba33b9912 100644 --- a/pkg/sentry/fs/fsutil/inode_cached.go +++ b/pkg/sentry/fs/fsutil/inode_cached.go @@ -479,7 +479,7 @@ func (c *CachingInodeOperations) Read(ctx context.Context, file *fs.File, dst us // common: getting a return value of 0 from a read syscall is the only way // to detect EOF. // - // TODO: Separate out c.attr.Size and use atomics instead of + // TODO(jamieliu): Separate out c.attr.Size and use atomics instead of // c.dataMu. c.dataMu.RLock() size := c.attr.Size @@ -776,7 +776,7 @@ func (c *CachingInodeOperations) Translate(ctx context.Context, required, option var translatedEnd uint64 for seg := c.cache.FindSegment(required.Start); seg.Ok() && seg.Start() < required.End; seg, _ = seg.NextNonEmpty() { segMR := seg.Range().Intersect(optional) - // TODO: Make Translations writable even if writability is + // TODO(jamieliu): Make Translations writable even if writability is // not required if already kept-dirty by another writable translation. perms := usermem.AccessType{ Read: true, diff --git a/pkg/sentry/fs/gofer/cache_policy.go b/pkg/sentry/fs/gofer/cache_policy.go index d7fbb71b7..51c573aef 100644 --- a/pkg/sentry/fs/gofer/cache_policy.go +++ b/pkg/sentry/fs/gofer/cache_policy.go @@ -136,7 +136,7 @@ func (cp cachePolicy) revalidate(ctx context.Context, name string, parent, child // Walk from parent to child again. // - // TODO: If we have a directory FD in the parent + // TODO(b/112031682): If we have a directory FD in the parent // inodeOperations, then we can use fstatat(2) to get the inode // attributes instead of making this RPC. qids, _, mask, attr, err := parentIops.fileState.file.walkGetAttr(ctx, []string{name}) @@ -171,7 +171,7 @@ func (cp cachePolicy) keep(d *fs.Dirent) bool { return false } sattr := d.Inode.StableAttr - // NOTE: Only cache files, directories, and symlinks. + // NOTE(b/31979197): Only cache files, directories, and symlinks. return fs.IsFile(sattr) || fs.IsDir(sattr) || fs.IsSymlink(sattr) } diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go index 80d1e08a6..35caa42cd 100644 --- a/pkg/sentry/fs/gofer/file.go +++ b/pkg/sentry/fs/gofer/file.go @@ -297,7 +297,7 @@ func (f *fileOperations) Flush(ctx context.Context, file *fs.File) error { // We do this because some p9 server implementations of Flush are // over-zealous. // - // FIXME: weaken these implementations and remove this check. + // FIXME(edahlgren): weaken these implementations and remove this check. if !file.Flags().Write { return nil } diff --git a/pkg/sentry/fs/gofer/file_state.go b/pkg/sentry/fs/gofer/file_state.go index f770ca4ea..d0c64003c 100644 --- a/pkg/sentry/fs/gofer/file_state.go +++ b/pkg/sentry/fs/gofer/file_state.go @@ -28,7 +28,7 @@ func (f *fileOperations) afterLoad() { // Manually load the open handles. var err error - // TODO: Context is not plumbed to save/restore. + // TODO(b/38173783): Context is not plumbed to save/restore. f.handles, err = f.inodeOperations.fileState.getHandles(context.Background(), f.flags) if err != nil { return fmt.Errorf("failed to re-open handle: %v", err) diff --git a/pkg/sentry/fs/gofer/handles.go b/pkg/sentry/fs/gofer/handles.go index f32e99ce0..0b33e80c3 100644 --- a/pkg/sentry/fs/gofer/handles.go +++ b/pkg/sentry/fs/gofer/handles.go @@ -49,7 +49,7 @@ func (h *handles) DecRef() { log.Warningf("error closing host file: %v", err) } } - // FIXME: Context is not plumbed here. + // FIXME(b/38173783): Context is not plumbed here. if err := h.File.close(context.Background()); err != nil { log.Warningf("error closing p9 file: %v", err) } diff --git a/pkg/sentry/fs/gofer/inode.go b/pkg/sentry/fs/gofer/inode.go index 29af1010c..1181a24cc 100644 --- a/pkg/sentry/fs/gofer/inode.go +++ b/pkg/sentry/fs/gofer/inode.go @@ -570,13 +570,13 @@ func init() { } // AddLink implements InodeOperations.AddLink, but is currently a noop. -// FIXME: Remove this from InodeOperations altogether. +// FIXME(b/63117438): Remove this from InodeOperations altogether. func (*inodeOperations) AddLink() {} // DropLink implements InodeOperations.DropLink, but is currently a noop. -// FIXME: Remove this from InodeOperations altogether. +// FIXME(b/63117438): Remove this from InodeOperations altogether. func (*inodeOperations) DropLink() {} // NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. -// FIXME: Remove this from InodeOperations altogether. +// FIXME(b/63117438): Remove this from InodeOperations altogether. func (i *inodeOperations) NotifyStatusChange(ctx context.Context) {} diff --git a/pkg/sentry/fs/gofer/inode_state.go b/pkg/sentry/fs/gofer/inode_state.go index ad4d3df58..44d76ba9f 100644 --- a/pkg/sentry/fs/gofer/inode_state.go +++ b/pkg/sentry/fs/gofer/inode_state.go @@ -123,7 +123,7 @@ func (i *inodeFileState) afterLoad() { // beforeSave. return fmt.Errorf("failed to find path for inode number %d. Device %s contains %s", i.sattr.InodeID, i.s.connID, fs.InodeMappings(i.s.inodeMappings)) } - // TODO: Context is not plumbed to save/restore. + // TODO(b/38173783): Context is not plumbed to save/restore. ctx := &dummyClockContext{context.Background()} _, i.file, err = i.s.attach.walk(ctx, splitAbsolutePath(name)) diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go index ed5147c65..4ed688ce5 100644 --- a/pkg/sentry/fs/gofer/session.go +++ b/pkg/sentry/fs/gofer/session.go @@ -134,7 +134,7 @@ type session struct { // socket files. This allows unix domain sockets to be used with paths that // belong to a gofer. // - // TODO: there are few possible races with someone stat'ing the + // TODO(b/77154739): there are few possible races with someone stat'ing the // file and another deleting it concurrently, where the file will not be // reported as socket file. endpoints *endpointMaps `state:"wait"` diff --git a/pkg/sentry/fs/gofer/session_state.go b/pkg/sentry/fs/gofer/session_state.go index 0ad5d63b5..b1f299be5 100644 --- a/pkg/sentry/fs/gofer/session_state.go +++ b/pkg/sentry/fs/gofer/session_state.go @@ -104,7 +104,7 @@ func (s *session) afterLoad() { // If private unix sockets are enabled, create and fill the session's endpoint // maps. if opts.privateunixsocket { - // TODO: Context is not plumbed to save/restore. + // TODO(b/38173783): Context is not plumbed to save/restore. ctx := &dummyClockContext{context.Background()} if err = s.restoreEndpointMaps(ctx); err != nil { diff --git a/pkg/sentry/fs/host/fs.go b/pkg/sentry/fs/host/fs.go index 800649211..de349a41a 100644 --- a/pkg/sentry/fs/host/fs.go +++ b/pkg/sentry/fs/host/fs.go @@ -87,7 +87,7 @@ func (f *Filesystem) Mount(ctx context.Context, _ string, flags fs.MountSourceFl options := fs.GenericMountSourceOptions(data) // Grab the whitelist if one was specified. - // TODO: require another option "testonly" in order to allow + // TODO(edahlgren/mpratt/hzy): require another option "testonly" in order to allow // no whitelist. if wl, ok := options[whitelistKey]; ok { f.paths = strings.Split(wl, "|") @@ -320,7 +320,7 @@ func (m *superOperations) SaveInodeMapping(inode *fs.Inode, path string) { // Keep implements fs.MountSourceOperations.Keep. // -// TODO: It is possible to change the permissions on a +// TODO(b/72455313,b/77596690): It is possible to change the permissions on a // host file while it is in the dirent cache (say from RO to RW), but it is not // possible to re-open the file with more relaxed permissions, since the host // FD is already open and stored in the inode. diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go index 2030edcb4..69c648f67 100644 --- a/pkg/sentry/fs/host/inode.go +++ b/pkg/sentry/fs/host/inode.go @@ -95,7 +95,7 @@ type inodeFileState struct { // ReadToBlocksAt implements fsutil.CachedFileObject.ReadToBlocksAt. func (i *inodeFileState) ReadToBlocksAt(ctx context.Context, dsts safemem.BlockSeq, offset uint64) (uint64, error) { - // TODO: Using safemem.FromIOReader here is wasteful for two + // TODO(jamieliu): Using safemem.FromIOReader here is wasteful for two // reasons: // // - Using preadv instead of iterated preads saves on host system calls. @@ -325,7 +325,7 @@ func (i *inodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flags fs.Fi // canMap returns true if this fs.Inode can be memory mapped. func canMap(inode *fs.Inode) bool { - // FIXME: Some obscure character devices can be mapped. + // FIXME(b/38213152): Some obscure character devices can be mapped. return fs.IsFile(inode.StableAttr) } @@ -428,15 +428,15 @@ func (i *inodeOperations) StatFS(context.Context) (fs.Info, error) { } // AddLink implements fs.InodeOperations.AddLink. -// FIXME: Remove this from InodeOperations altogether. +// FIXME(b/63117438): Remove this from InodeOperations altogether. func (i *inodeOperations) AddLink() {} // DropLink implements fs.InodeOperations.DropLink. -// FIXME: Remove this from InodeOperations altogether. +// FIXME(b/63117438): Remove this from InodeOperations altogether. func (i *inodeOperations) DropLink() {} // NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. -// FIXME: Remove this from InodeOperations altogether. +// FIXME(b/63117438): Remove this from InodeOperations altogether. func (i *inodeOperations) NotifyStatusChange(ctx context.Context) {} // readdirAll returns all of the directory entries in i. diff --git a/pkg/sentry/fs/inode.go b/pkg/sentry/fs/inode.go index d82f9740e..fe411a766 100644 --- a/pkg/sentry/fs/inode.go +++ b/pkg/sentry/fs/inode.go @@ -93,10 +93,10 @@ func (i *Inode) DecRef() { // destroy releases the Inode and releases the msrc reference taken. func (i *Inode) destroy() { - // FIXME: Context is not plumbed here. + // FIXME(b/38173783): Context is not plumbed here. ctx := context.Background() if err := i.WriteOut(ctx); err != nil { - // FIXME: Mark as warning again once noatime is + // FIXME(b/65209558): Mark as warning again once noatime is // properly supported. log.Debugf("Inode %+v, failed to sync all metadata: %v", i.StableAttr, err) } @@ -359,7 +359,7 @@ func (i *Inode) Getlink(ctx context.Context) (*Dirent, error) { // AddLink calls i.InodeOperations.AddLink. func (i *Inode) AddLink() { if i.overlay != nil { - // FIXME: Remove this from InodeOperations altogether. + // FIXME(b/63117438): Remove this from InodeOperations altogether. // // This interface is only used by ramfs to update metadata of // children. These filesystems should _never_ have overlay diff --git a/pkg/sentry/fs/inode_operations.go b/pkg/sentry/fs/inode_operations.go index ceacc7659..ff8b75f31 100644 --- a/pkg/sentry/fs/inode_operations.go +++ b/pkg/sentry/fs/inode_operations.go @@ -118,7 +118,7 @@ type InodeOperations interface { // // The caller must ensure that this operation is permitted. // - // TODO: merge Remove and RemoveDirectory, Remove + // TODO(b/67778723): merge Remove and RemoveDirectory, Remove // just needs a type flag. Remove(ctx context.Context, dir *Inode, name string) error diff --git a/pkg/sentry/fs/inode_overlay.go b/pkg/sentry/fs/inode_overlay.go index 254646176..bda3e1861 100644 --- a/pkg/sentry/fs/inode_overlay.go +++ b/pkg/sentry/fs/inode_overlay.go @@ -142,7 +142,7 @@ func overlayLookup(ctx context.Context, parent *overlayEntry, inode *Inode, name } else { // If we have something from the upper, we can only use it if the types // match. - // NOTE: Allow SpecialDirectories and Directories to merge. + // NOTE(b/112312863): Allow SpecialDirectories and Directories to merge. // This is needed to allow submounts in /proc and /sys. if upperInode.StableAttr.Type == child.Inode.StableAttr.Type || (IsDir(upperInode.StableAttr) && IsDir(child.Inode.StableAttr)) { @@ -226,7 +226,7 @@ func overlayCreate(ctx context.Context, o *overlayEntry, parent *Dirent, name st return nil, err } - // NOTE: Replace the Dirent with a transient Dirent, since + // NOTE(b/71766861): Replace the Dirent with a transient Dirent, since // we are about to create the real Dirent: an overlay Dirent. // // This ensures the *fs.File returned from overlayCreate is in the same @@ -338,7 +338,7 @@ func overlayRename(ctx context.Context, o *overlayEntry, oldParent *Dirent, rena // directory will appear empty in the upper fs, which will then // allow the rename to proceed when it should return ENOTEMPTY. // - // NOTE: Ideally, we'd just pass in the replaced + // NOTE(b/111808347): Ideally, we'd just pass in the replaced // Dirent from Rename, but we must drop the reference on // replaced before we make the rename call, so Rename can't // pass the Dirent to the Inode without significantly diff --git a/pkg/sentry/fs/mount.go b/pkg/sentry/fs/mount.go index 1e245ae5f..4d1693204 100644 --- a/pkg/sentry/fs/mount.go +++ b/pkg/sentry/fs/mount.go @@ -42,7 +42,7 @@ type DirentOperations interface { // MountSourceOperations contains filesystem specific operations. type MountSourceOperations interface { - // TODO: Add: + // TODO(b/67778729): Add: // BlockSize() int64 // FS() Filesystem @@ -101,7 +101,7 @@ func (i InodeMappings) String() string { // amalgamation implies that a mount source cannot be shared by multiple mounts // (e.g. cannot be mounted at different locations). // -// TODO: Move mount-specific information out of MountSource. +// TODO(b/63601033): Move mount-specific information out of MountSource. // // +stateify savable type MountSource struct { diff --git a/pkg/sentry/fs/mount_test.go b/pkg/sentry/fs/mount_test.go index 269d6b9da..d7605b2c9 100644 --- a/pkg/sentry/fs/mount_test.go +++ b/pkg/sentry/fs/mount_test.go @@ -33,7 +33,7 @@ func cacheReallyContains(cache *DirentCache, d *Dirent) bool { } // TestMountSourceOnlyCachedOnce tests that a Dirent that is mounted over only ends -// up in a single Dirent Cache. NOTE: Having a dirent in multiple +// up in a single Dirent Cache. NOTE(b/63848693): Having a dirent in multiple // caches causes major consistency issues. func TestMountSourceOnlyCachedOnce(t *testing.T) { ctx := contexttest.Context(t) diff --git a/pkg/sentry/fs/proc/README.md b/pkg/sentry/fs/proc/README.md index 3cc5f197c..5d4ec6c7b 100644 --- a/pkg/sentry/fs/proc/README.md +++ b/pkg/sentry/fs/proc/README.md @@ -91,7 +91,7 @@ CPU.IO utilization in last 10 minutes | Always zero Num currently running processes | Always zero Total num processes | Always zero -TODO: Populate the columns with accurate statistics. +TODO(b/62345059): Populate the columns with accurate statistics. ### meminfo @@ -128,12 +128,12 @@ Field name | Notes Buffers | Always zero, no block devices SwapCache | Always zero, no swap Inactive(anon) | Always zero, see SwapCache -Unevictable | Always zero TODO -Mlocked | Always zero TODO +Unevictable | Always zero TODO(b/31823263) +Mlocked | Always zero TODO(b/31823263) SwapTotal | Always zero, no swap SwapFree | Always zero, no swap -Dirty | Always zero TODO -Writeback | Always zero TODO +Dirty | Always zero TODO(b/31823263) +Writeback | Always zero TODO(b/31823263) MemAvailable | Uses the same value as MemFree since there is no swap. Slab | Missing SReclaimable | Missing @@ -185,7 +185,7 @@ softirq 0 0 0 0 0 0 0 0 0 0 0 All fields except for `btime` are always zero. -TODO: Populate with accurate fields. +TODO(b/37226836): Populate with accurate fields. ### sys diff --git a/pkg/sentry/fs/proc/fds.go b/pkg/sentry/fs/proc/fds.go index 25da06f5d..f2329e623 100644 --- a/pkg/sentry/fs/proc/fds.go +++ b/pkg/sentry/fs/proc/fds.go @@ -258,7 +258,7 @@ func newFdInfoDir(t *kernel.Task, msrc *fs.MountSource) *fs.Inode { // Lookup loads an fd in /proc/TID/fdinfo into a Dirent. func (fdid *fdInfoDir) Lookup(ctx context.Context, dir *fs.Inode, p string) (*fs.Dirent, error) { inode, err := walkDescriptors(fdid.t, p, func(file *fs.File, fdFlags kernel.FDFlags) *fs.Inode { - // TODO: Using a static inode here means that the + // TODO(b/121266871): Using a static inode here means that the // data can be out-of-date if, for instance, the flags on the // FD change before we read this file. We should switch to // generating the data on Read(). Also, we should include pos, diff --git a/pkg/sentry/fs/proc/loadavg.go b/pkg/sentry/fs/proc/loadavg.go index 78f3a1dc0..3ee0e570a 100644 --- a/pkg/sentry/fs/proc/loadavg.go +++ b/pkg/sentry/fs/proc/loadavg.go @@ -40,7 +40,7 @@ func (d *loadavgData) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) var buf bytes.Buffer - // TODO: Include real data in fields. + // TODO(b/62345059): Include real data in fields. // Column 1-3: CPU and IO utilization of the last 1, 5, and 10 minute periods. // Column 4-5: currently running processes and the total number of processes. // Column 6: the last process ID used. diff --git a/pkg/sentry/fs/proc/meminfo.go b/pkg/sentry/fs/proc/meminfo.go index 620e93ce3..75cbf3e77 100644 --- a/pkg/sentry/fs/proc/meminfo.go +++ b/pkg/sentry/fs/proc/meminfo.go @@ -58,7 +58,7 @@ func (d *meminfoData) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) fmt.Fprintf(&buf, "MemTotal: %8d kB\n", totalSize/1024) memFree := (totalSize - totalUsage) / 1024 // We use MemFree as MemAvailable because we don't swap. - // TODO: When reclaim is implemented the value of MemAvailable + // TODO(rahat): When reclaim is implemented the value of MemAvailable // should change. fmt.Fprintf(&buf, "MemFree: %8d kB\n", memFree) fmt.Fprintf(&buf, "MemAvailable: %8d kB\n", memFree) @@ -72,8 +72,8 @@ func (d *meminfoData) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) fmt.Fprintf(&buf, "Inactive(anon): 0 kB\n") fmt.Fprintf(&buf, "Active(file): %8d kB\n", activeFile/1024) fmt.Fprintf(&buf, "Inactive(file): %8d kB\n", inactiveFile/1024) - fmt.Fprintf(&buf, "Unevictable: 0 kB\n") // TODO - fmt.Fprintf(&buf, "Mlocked: 0 kB\n") // TODO + fmt.Fprintf(&buf, "Unevictable: 0 kB\n") // TODO(b/31823263) + fmt.Fprintf(&buf, "Mlocked: 0 kB\n") // TODO(b/31823263) fmt.Fprintf(&buf, "SwapTotal: 0 kB\n") fmt.Fprintf(&buf, "SwapFree: 0 kB\n") fmt.Fprintf(&buf, "Dirty: 0 kB\n") diff --git a/pkg/sentry/fs/proc/mounts.go b/pkg/sentry/fs/proc/mounts.go index 1e62af8c6..fe62b167b 100644 --- a/pkg/sentry/fs/proc/mounts.go +++ b/pkg/sentry/fs/proc/mounts.go @@ -114,7 +114,7 @@ func (mif *mountInfoFile) ReadSeqFileData(ctx context.Context, handle seqfile.Se // (4) Root: the pathname of the directory in the filesystem // which forms the root of this mount. // - // NOTE: This will always be "/" until we implement + // NOTE(b/78135857): This will always be "/" until we implement // bind mounts. fmt.Fprintf(&buf, "/ ") diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go index 55a958f9e..d24b2d370 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -154,7 +154,7 @@ func (n *netDev) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([]se contents[1] = " face |bytes packets errs drop fifo frame compressed multicast|bytes packets errs drop fifo colls carrier compressed\n" for _, i := range interfaces { - // TODO: Collect stats from each inet.Stack + // TODO(b/71872867): Collect stats from each inet.Stack // implementation (hostinet, epsocket, and rpcinet). // Implements the same format as diff --git a/pkg/sentry/fs/proc/stat.go b/pkg/sentry/fs/proc/stat.go index f2bbef375..18bd8e9b6 100644 --- a/pkg/sentry/fs/proc/stat.go +++ b/pkg/sentry/fs/proc/stat.go @@ -83,7 +83,7 @@ func (s *statData) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([] var buf bytes.Buffer - // TODO: We currently export only zero CPU stats. We could + // TODO(b/37226836): We currently export only zero CPU stats. We could // at least provide some aggregate stats. var cpu cpuStats fmt.Fprintf(&buf, "cpu %s\n", cpu) @@ -100,7 +100,7 @@ func (s *statData) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([] const numInterrupts = 256 // The Kernel doesn't handle real interrupts, so report all zeroes. - // TODO: We could count page faults as #PF. + // TODO(b/37226836): We could count page faults as #PF. fmt.Fprintf(&buf, "intr 0") // total for i := 0; i < numInterrupts; i++ { fmt.Fprintf(&buf, " 0") @@ -108,22 +108,22 @@ func (s *statData) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) ([] fmt.Fprintf(&buf, "\n") // Total number of context switches. - // TODO: Count this. + // TODO(b/37226836): Count this. fmt.Fprintf(&buf, "ctxt 0\n") // CLOCK_REALTIME timestamp from boot, in seconds. fmt.Fprintf(&buf, "btime %d\n", s.k.Timekeeper().BootTime().Seconds()) // Total number of clones. - // TODO: Count this. + // TODO(b/37226836): Count this. fmt.Fprintf(&buf, "processes 0\n") // Number of runnable tasks. - // TODO: Count this. + // TODO(b/37226836): Count this. fmt.Fprintf(&buf, "procs_running 0\n") // Number of tasks waiting on IO. - // TODO: Count this. + // TODO(b/37226836): Count this. fmt.Fprintf(&buf, "procs_blocked 0\n") // Number of each softirq handled. diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index 728a46a74..0ce77f04f 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -39,7 +39,7 @@ const ( // tcpMemInode is used to read/write the size of netstack tcp buffers. // -// TODO: If we have multiple proc mounts, concurrent writes can +// TODO(b/121381035): If we have multiple proc mounts, concurrent writes can // leave netstack and the proc files in an inconsistent state. Since we set the // buffer size from these proc files on restore, we may also race and end up in // an inconsistent state on restore. diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go index 0edcdfce2..9f65a8337 100644 --- a/pkg/sentry/fs/proc/task.go +++ b/pkg/sentry/fs/proc/task.go @@ -77,7 +77,7 @@ func newTaskDir(t *kernel.Task, msrc *fs.MountSource, pidns *kernel.PIDNamespace "fd": newFdDir(t, msrc), "fdinfo": newFdInfoDir(t, msrc), "gid_map": newGIDMap(t, msrc), - // FIXME: create the correct io file for threads. + // FIXME(b/123511468): create the correct io file for threads. "io": newIO(t, msrc), "maps": newMaps(t, msrc), "mountinfo": seqfile.NewSeqFileInode(t, &mountInfoFile{t: t}, msrc), @@ -93,7 +93,7 @@ func newTaskDir(t *kernel.Task, msrc *fs.MountSource, pidns *kernel.PIDNamespace contents["task"] = newSubtasks(t, msrc, pidns) } - // TODO: Set EUID/EGID based on dumpability. + // TODO(b/31916171): Set EUID/EGID based on dumpability. d := &taskDir{ Dir: *ramfs.NewDir(t, contents, fs.RootOwner, fs.FilePermsFromMode(0555)), t: t, @@ -245,7 +245,7 @@ func (e *exe) executable() (d *fs.Dirent, err error) { e.t.WithMuLocked(func(t *kernel.Task) { mm := t.MemoryManager() if mm == nil { - // TODO: Check shouldn't allow Readlink once the + // TODO(b/34851096): Check shouldn't allow Readlink once the // Task is zombied. err = syserror.EACCES return @@ -297,7 +297,7 @@ type namespaceSymlink struct { } func newNamespaceSymlink(t *kernel.Task, msrc *fs.MountSource, name string) *fs.Inode { - // TODO: Namespace symlinks should contain the namespace name and the + // TODO(rahat): Namespace symlinks should contain the namespace name and the // inode number for the namespace instance, so for example user:[123456]. We // currently fake the inode number by sticking the symlink inode in its // place. diff --git a/pkg/sentry/fs/proc/version.go b/pkg/sentry/fs/proc/version.go index b6d49d5e9..58e0c793c 100644 --- a/pkg/sentry/fs/proc/version.go +++ b/pkg/sentry/fs/proc/version.go @@ -65,7 +65,7 @@ func (v *versionData) ReadSeqFileData(ctx context.Context, h seqfile.SeqHandle) // Since we don't really want to expose build information to // applications, those fields are omitted. // - // FIXME: Using Version from the init task SyscallTable + // FIXME(mpratt): Using Version from the init task SyscallTable // disregards the different version a task may have (e.g., in a uts // namespace). ver := init.Leader().SyscallTable().Version diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index 159fd2981..c0400b67d 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -358,7 +358,7 @@ func (d *Dir) CreateDirectory(ctx context.Context, dir *fs.Inode, name string, p _, err := d.createInodeOperationsCommon(ctx, name, func() (*fs.Inode, error) { return d.NewDir(ctx, dir, perms) }) - // TODO: Support updating status times, as those should be + // TODO(nlacasse): Support updating status times, as those should be // updated by links. return err } diff --git a/pkg/sentry/fs/tmpfs/fs.go b/pkg/sentry/fs/tmpfs/fs.go index d0c93028f..8e44421b6 100644 --- a/pkg/sentry/fs/tmpfs/fs.go +++ b/pkg/sentry/fs/tmpfs/fs.go @@ -34,7 +34,7 @@ const ( // GID for the root directory. rootGIDKey = "gid" - // TODO: support a tmpfs size limit. + // TODO(edahlgren/mpratt): support a tmpfs size limit. // size = "size" // Permissions that exceed modeMask will be rejected. diff --git a/pkg/sentry/fs/tmpfs/inode_file.go b/pkg/sentry/fs/tmpfs/inode_file.go index 7c80d711b..4450e1363 100644 --- a/pkg/sentry/fs/tmpfs/inode_file.go +++ b/pkg/sentry/fs/tmpfs/inode_file.go @@ -309,7 +309,7 @@ func (f *fileInodeOperations) read(ctx context.Context, file *fs.File, dst userm // common: getting a return value of 0 from a read syscall is the only way // to detect EOF. // - // TODO: Separate out f.attr.Size and use atomics instead of + // TODO(jamieliu): Separate out f.attr.Size and use atomics instead of // f.dataMu. f.dataMu.RLock() size := f.attr.Size diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go index 555692505..5bb4922cb 100644 --- a/pkg/sentry/fs/tmpfs/tmpfs.go +++ b/pkg/sentry/fs/tmpfs/tmpfs.go @@ -32,7 +32,7 @@ import ( var fsInfo = fs.Info{ Type: linux.TMPFS_MAGIC, - // TODO: allow configuring a tmpfs size and enforce it. + // TODO(b/29637826): allow configuring a tmpfs size and enforce it. TotalBlocks: 0, FreeBlocks: 0, } diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index 33b4c6438..f8713471a 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -66,7 +66,7 @@ type dirInodeOperations struct { // msrc is the super block this directory is on. // - // TODO: Plumb this through instead of storing it here. + // TODO(chrisko): Plumb this through instead of storing it here. msrc *fs.MountSource // mu protects the fields below. @@ -89,7 +89,7 @@ type dirInodeOperations struct { // next is the next pty index to use. // - // TODO: reuse indices when ptys are closed. + // TODO(b/29356795): reuse indices when ptys are closed. next uint32 } @@ -118,7 +118,7 @@ func newDir(ctx context.Context, m *fs.MountSource) *fs.Inode { // N.B. Linux always uses inode id 1 for the directory. See // fs/devpts/inode.c:devpts_fill_super. // - // TODO: Since ptsDevice must be shared between + // TODO(b/75267214): Since ptsDevice must be shared between // different mounts, we must not assign fixed numbers. InodeID: ptsDevice.NextIno(), BlockSize: usermem.PageSize, diff --git a/pkg/sentry/fs/tty/fs.go b/pkg/sentry/fs/tty/fs.go index 43e0e2a04..a53448c47 100644 --- a/pkg/sentry/fs/tty/fs.go +++ b/pkg/sentry/fs/tty/fs.go @@ -43,7 +43,7 @@ func (*filesystem) Name() string { // AllowUserMount allows users to mount(2) this file system. func (*filesystem) AllowUserMount() bool { - // TODO: Users may mount this once the terminals are in a + // TODO(b/29356795): Users may mount this once the terminals are in a // usable state. return false } diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index 7c256abb0..e2686a074 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -51,7 +51,7 @@ func newMasterInode(ctx context.Context, d *dirInodeOperations, owner fs.FileOwn // N.B. Linux always uses inode id 2 for ptmx. See // fs/devpts/inode.c:mknod_ptmx. // - // TODO: Since ptsDevice must be shared between + // TODO(b/75267214): Since ptsDevice must be shared between // different mounts, we must not assign fixed numbers. InodeID: ptsDevice.NextIno(), Type: fs.CharacterDevice, @@ -157,7 +157,7 @@ func (mf *masterFileOperations) Ioctl(ctx context.Context, io usermem.IO, args a // of the slave end. return mf.t.ld.setTermios(ctx, io, args) case linux.TCSETSW: - // TODO: This should drain the output queue first. + // TODO(b/29356795): This should drain the output queue first. return mf.t.ld.setTermios(ctx, io, args) case linux.TIOCGPTN: _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), uint32(mf.t.n), usermem.IOOpts{ @@ -165,7 +165,7 @@ func (mf *masterFileOperations) Ioctl(ctx context.Context, io usermem.IO, args a }) return 0, err case linux.TIOCSPTLCK: - // TODO: Implement pty locking. For now just pretend we do. + // TODO(b/29356795): Implement pty locking. For now just pretend we do. return 0, nil case linux.TIOCGWINSZ: return 0, mf.t.ld.windowSize(ctx, io, args) diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index e8368bcdd..ed080ca0f 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -56,7 +56,7 @@ func newSlaveInode(ctx context.Context, d *dirInodeOperations, t *Terminal, owne // N.B. Linux always uses inode id = tty index + 3. See // fs/devpts/inode.c:devpts_pty_new. // - // TODO: Since ptsDevice must be shared between + // TODO(b/75267214): Since ptsDevice must be shared between // different mounts, we must not assign fixed numbers. InodeID: ptsDevice.NextIno(), Type: fs.CharacterDevice, @@ -137,7 +137,7 @@ func (sf *slaveFileOperations) Ioctl(ctx context.Context, io usermem.IO, args ar case linux.TCSETS: return sf.si.t.ld.setTermios(ctx, io, args) case linux.TCSETSW: - // TODO: This should drain the output queue first. + // TODO(b/29356795): This should drain the output queue first. return sf.si.t.ld.setTermios(ctx, io, args) case linux.TIOCGPTN: _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), uint32(sf.si.t.n), usermem.IOOpts{ @@ -151,7 +151,7 @@ func (sf *slaveFileOperations) Ioctl(ctx context.Context, io usermem.IO, args ar case linux.TIOCSCTTY: // Make the given terminal the controlling terminal of the // calling process. - // TODO: Implement once we have support for job + // TODO(b/129283598): Implement once we have support for job // control. return 0, nil default: diff --git a/pkg/sentry/kernel/auth/credentials.go b/pkg/sentry/kernel/auth/credentials.go index a843b9aab..2055da196 100644 --- a/pkg/sentry/kernel/auth/credentials.go +++ b/pkg/sentry/kernel/auth/credentials.go @@ -125,7 +125,7 @@ func NewUserCredentials(kuid KUID, kgid KGID, extraKGIDs []KGID, capabilities *T creds.EffectiveCaps = capabilities.EffectiveCaps creds.BoundingCaps = capabilities.BoundingCaps creds.InheritableCaps = capabilities.InheritableCaps - // TODO: Support ambient capabilities. + // TODO(nlacasse): Support ambient capabilities. } else { // If no capabilities are specified, grant capabilities consistent with // setresuid + setresgid from NewRootCredentials to the given uid and diff --git a/pkg/sentry/kernel/auth/user_namespace.go b/pkg/sentry/kernel/auth/user_namespace.go index 30957bb9a..159940a69 100644 --- a/pkg/sentry/kernel/auth/user_namespace.go +++ b/pkg/sentry/kernel/auth/user_namespace.go @@ -49,7 +49,7 @@ type UserNamespace struct { gidMapFromParent idMapSet gidMapToParent idMapSet - // TODO: Support disabling setgroups(2). + // TODO(b/27454212): Support disabling setgroups(2). } // NewRootUserNamespace returns a UserNamespace that is appropriate for a diff --git a/pkg/sentry/kernel/pending_signals.go b/pkg/sentry/kernel/pending_signals.go index 373e11772..deff6def9 100644 --- a/pkg/sentry/kernel/pending_signals.go +++ b/pkg/sentry/kernel/pending_signals.go @@ -30,7 +30,7 @@ const ( // rtSignalCap is the maximum number of instances of a given realtime // signal that may be pending. // - // TODO: In Linux, the minimum signal queue size is + // TODO(igudger): In Linux, the minimum signal queue size is // RLIMIT_SIGPENDING, which is by default max_threads/2. rtSignalCap = 32 ) diff --git a/pkg/sentry/kernel/ptrace.go b/pkg/sentry/kernel/ptrace.go index 8d78b2fb3..15f2e2964 100644 --- a/pkg/sentry/kernel/ptrace.go +++ b/pkg/sentry/kernel/ptrace.go @@ -162,7 +162,7 @@ func (t *Task) CanTrace(target *Task, attach bool) bool { if cgid := callerCreds.RealKGID; cgid != targetCreds.RealKGID || cgid != targetCreds.EffectiveKGID || cgid != targetCreds.SavedKGID { return false } - // TODO: dumpability check + // TODO(b/31916171): dumpability check if callerCreds.UserNamespace != targetCreds.UserNamespace { return false } @@ -396,7 +396,7 @@ func (t *Task) ptraceAttach(target *Task, seize bool, opts uintptr) error { if target.stop == (*groupStop)(nil) { target.trapStopPending = true target.endInternalStopLocked() - // TODO: Linux blocks ptrace_attach() until the task has + // TODO(jamieliu): Linux blocks ptrace_attach() until the task has // entered the ptrace-stop (or exited) via JOBCTL_TRAPPING. } target.tg.signalHandlers.mu.Unlock() diff --git a/pkg/sentry/kernel/rseq.go b/pkg/sentry/kernel/rseq.go index 0a954bc16..6d3314e81 100644 --- a/pkg/sentry/kernel/rseq.go +++ b/pkg/sentry/kernel/rseq.go @@ -66,7 +66,7 @@ func (t *Task) SetRSEQCriticalRegion(rscr RSEQCriticalRegion) error { if rscr.CriticalSection.Contains(rscr.Restart) { return syserror.EINVAL } - // TODO: check that rscr.CriticalSection and rscr.Restart are in + // TODO(jamieliu): check that rscr.CriticalSection and rscr.Restart are in // the application address range, for consistency with Linux t.tg.rscr.Store(&rscr) return nil diff --git a/pkg/sentry/kernel/sched/cpuset.go b/pkg/sentry/kernel/sched/cpuset.go index 69aee9127..41ac1067d 100644 --- a/pkg/sentry/kernel/sched/cpuset.go +++ b/pkg/sentry/kernel/sched/cpuset.go @@ -29,7 +29,7 @@ type CPUSet []byte // CPUSetSize returns the size in bytes of a CPUSet that can contain num cpus. func CPUSetSize(num uint) uint { - // NOTE: Applications may expect that the size of a CPUSet in + // NOTE(b/68859821): Applications may expect that the size of a CPUSet in // bytes is always a multiple of sizeof(unsigned long), since this is true // in Linux. Thus we always round up. bytes := (num + bitsPerByte - 1) / bitsPerByte diff --git a/pkg/sentry/kernel/semaphore/semaphore.go b/pkg/sentry/kernel/semaphore/semaphore.go index 29a2eb804..2b7c1a9bc 100644 --- a/pkg/sentry/kernel/semaphore/semaphore.go +++ b/pkg/sentry/kernel/semaphore/semaphore.go @@ -302,7 +302,7 @@ func (s *Set) SetVal(ctx context.Context, num int32, val int16, creds *auth.Cred return syserror.ERANGE } - // TODO: Clear undo entries in all processes + // TODO(b/29354920): Clear undo entries in all processes sem.value = val sem.pid = pid s.changeTime = ktime.NowFromContext(ctx) @@ -336,7 +336,7 @@ func (s *Set) SetValAll(ctx context.Context, vals []uint16, creds *auth.Credenti for i, val := range vals { sem := &s.sems[i] - // TODO: Clear undo entries in all processes + // TODO(b/29354920): Clear undo entries in all processes sem.value = int16(val) sem.pid = pid sem.wakeWaiters() @@ -481,7 +481,7 @@ func (s *Set) executeOps(ctx context.Context, ops []linux.Sembuf, pid int32) (ch } // All operations succeeded, apply them. - // TODO: handle undo operations. + // TODO(b/29354920): handle undo operations. for i, v := range tmpVals { s.sems[i].value = v s.sems[i].wakeWaiters() diff --git a/pkg/sentry/kernel/shm/shm.go b/pkg/sentry/kernel/shm/shm.go index 349f2a26e..d4812a065 100644 --- a/pkg/sentry/kernel/shm/shm.go +++ b/pkg/sentry/kernel/shm/shm.go @@ -427,7 +427,7 @@ func (s *Shm) AddMapping(ctx context.Context, _ memmap.MappingSpace, _ usermem.A func (s *Shm) RemoveMapping(ctx context.Context, _ memmap.MappingSpace, _ usermem.AddrRange, _ uint64, _ bool) { s.mu.Lock() defer s.mu.Unlock() - // TODO: RemoveMapping may be called during task exit, when ctx + // TODO(b/38173783): RemoveMapping may be called during task exit, when ctx // is context.Background. Gracefully handle missing clocks. Failing to // update the detach time in these cases is ok, since no one can observe the // omission. diff --git a/pkg/sentry/kernel/syscalls.go b/pkg/sentry/kernel/syscalls.go index 7eb99718d..293b21249 100644 --- a/pkg/sentry/kernel/syscalls.go +++ b/pkg/sentry/kernel/syscalls.go @@ -165,7 +165,7 @@ type Stracer interface { // // The returned private data is passed to SyscallExit. // - // TODO: remove kernel imports from the strace + // TODO(gvisor.dev/issue/155): remove kernel imports from the strace // package so that the type can be used directly. SyscallEnter(t *Task, sysno uintptr, args arch.SyscallArguments, flags uint32) interface{} diff --git a/pkg/sentry/kernel/task_context.go b/pkg/sentry/kernel/task_context.go index 1b4d4cf2f..ac38dd157 100644 --- a/pkg/sentry/kernel/task_context.go +++ b/pkg/sentry/kernel/task_context.go @@ -60,7 +60,7 @@ func (tc *TaskContext) release() { // Nil out pointers so that if the task is saved after release, it doesn't // follow the pointers to possibly now-invalid objects. if tc.MemoryManager != nil { - // TODO + // TODO(b/38173783) tc.MemoryManager.DecUsers(context.Background()) tc.MemoryManager = nil } diff --git a/pkg/sentry/kernel/task_exec.go b/pkg/sentry/kernel/task_exec.go index 9fca90a1c..b49f902a5 100644 --- a/pkg/sentry/kernel/task_exec.go +++ b/pkg/sentry/kernel/task_exec.go @@ -208,7 +208,7 @@ func (r *runSyscallAfterExecStop) execute(t *Task) taskRunState { t.tc = *r.tc t.mu.Unlock() t.unstopVforkParent() - // NOTE: All locks must be dropped prior to calling Activate. + // NOTE(b/30316266): All locks must be dropped prior to calling Activate. t.MemoryManager().Activate() t.ptraceExec(oldTID) diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index 1a0734ab6..a07956208 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -339,7 +339,7 @@ func (t *Task) exitChildren() { }, true /* group */) other.signalHandlers.mu.Unlock() } - // TODO: The init process waits for all processes in the + // TODO(b/37722272): The init process waits for all processes in the // namespace to exit before completing its own exit // (kernel/pid_namespace.c:zap_pid_ns_processes()). Stop until all // other tasks in the namespace are dead, except possibly for this @@ -692,7 +692,7 @@ func (t *Task) exitNotificationSignal(sig linux.Signal, receiver *Task) *arch.Si info.Code = arch.CLD_EXITED info.SetStatus(int32(t.exitStatus.Code)) } - // TODO: Set utime, stime. + // TODO(b/72102453): Set utime, stime. return info } diff --git a/pkg/sentry/kernel/task_identity.go b/pkg/sentry/kernel/task_identity.go index e105eba13..6c9608f8d 100644 --- a/pkg/sentry/kernel/task_identity.go +++ b/pkg/sentry/kernel/task_identity.go @@ -421,7 +421,7 @@ func (t *Task) SetKeepCaps(k bool) { // updateCredsForExec updates t.creds to reflect an execve(). // -// NOTE: We currently do not implement privileged executables +// NOTE(b/30815691): We currently do not implement privileged executables // (set-user/group-ID bits and file capabilities). This allows us to make a lot // of simplifying assumptions: // diff --git a/pkg/sentry/kernel/task_run.go b/pkg/sentry/kernel/task_run.go index 6b5fe7165..7115aa967 100644 --- a/pkg/sentry/kernel/task_run.go +++ b/pkg/sentry/kernel/task_run.go @@ -110,7 +110,7 @@ func (t *Task) doStop() { return } t.Deactivate() - // NOTE: t.Activate() must be called without any locks held, so + // NOTE(b/30316266): t.Activate() must be called without any locks held, so // this defer must precede the defer for unlocking the signal mutex. defer t.Activate() t.accountTaskGoroutineEnter(TaskGoroutineStopped) diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go index 3a8e61900..7f2e0df72 100644 --- a/pkg/sentry/kernel/task_signals.go +++ b/pkg/sentry/kernel/task_signals.go @@ -509,7 +509,7 @@ func (t *Task) canReceiveSignalLocked(sig linux.Signal) bool { if t.stop != nil { return false } - // - TODO: No special case for when t is also the sending task, + // - TODO(b/38173783): No special case for when t is also the sending task, // because the identity of the sender is unknown. // - Do not choose tasks that have already been interrupted, as they may be // busy handling another signal. @@ -895,7 +895,7 @@ func (t *Task) signalStop(target *Task, code int32, status int32) { sigchld.SetPid(int32(t.tg.pidns.tids[target])) sigchld.SetUid(int32(target.Credentials().RealKUID.In(t.UserNamespace()).OrOverflow())) sigchld.SetStatus(status) - // TODO: Set utime, stime. + // TODO(b/72102453): Set utime, stime. t.sendSignalLocked(sigchld, true /* group */) } } diff --git a/pkg/sentry/kernel/task_stop.go b/pkg/sentry/kernel/task_stop.go index 36846484c..1302cadc1 100644 --- a/pkg/sentry/kernel/task_stop.go +++ b/pkg/sentry/kernel/task_stop.go @@ -69,7 +69,7 @@ import ( // A TaskStop is a condition visible to the task control flow graph that // prevents a task goroutine from running or exiting, i.e. an internal stop. // -// NOTE: Most TaskStops don't contain any data; they're +// NOTE(b/30793614): Most TaskStops don't contain any data; they're // distinguished by their type. The obvious way to implement such a TaskStop // is: // diff --git a/pkg/sentry/loader/loader.go b/pkg/sentry/loader/loader.go index 80ad59dde..79051befa 100644 --- a/pkg/sentry/loader/loader.go +++ b/pkg/sentry/loader/loader.go @@ -70,7 +70,7 @@ func openPath(ctx context.Context, mm *fs.MountNamespace, root, wd *fs.Dirent, m defer d.DecRef() perms := fs.PermMask{ - // TODO: Linux requires only execute + // TODO(gvisor.dev/issue/160): Linux requires only execute // permission, not read. However, our backing filesystems may // prevent us from reading the file without read permission. // diff --git a/pkg/sentry/loader/vdso.go b/pkg/sentry/loader/vdso.go index 18b7e90d8..8c196df84 100644 --- a/pkg/sentry/loader/vdso.go +++ b/pkg/sentry/loader/vdso.go @@ -194,7 +194,7 @@ func validateVDSO(ctx context.Context, f *fs.File, size uint64) (elfInfo, error) // VDSO describes a VDSO. // -// NOTE: to support multiple architectures or operating systems, this +// NOTE(mpratt): to support multiple architectures or operating systems, this // would need to contain a VDSO for each. // // +stateify savable @@ -262,7 +262,7 @@ func PrepareVDSO(mfp pgalloc.MemoryFileProvider) (*VDSO, error) { return &VDSO{ ParamPage: mm.NewSpecialMappable("[vvar]", mfp, paramPage), - // TODO: Don't advertise the VDSO, as + // TODO(gvisor.dev/issue/157): Don't advertise the VDSO, as // some applications may not be able to handle multiple [vdso] // hints. vdso: mm.NewSpecialMappable("", mfp, vdso), @@ -279,7 +279,7 @@ func PrepareVDSO(mfp pgalloc.MemoryFileProvider) (*VDSO, error) { // kernel simply directly maps the entire file into process memory, with very // little real ELF parsing. // -// NOTE: This means that userspace can, and unfortunately does, +// NOTE(b/25323870): This means that userspace can, and unfortunately does, // depend on parts of the ELF that would normally not be mapped. To maintain // compatibility with such binaries, we load the VDSO much like Linux. // diff --git a/pkg/sentry/memmap/memmap.go b/pkg/sentry/memmap/memmap.go index 1ef1f0dd8..3f6f7ebd0 100644 --- a/pkg/sentry/memmap/memmap.go +++ b/pkg/sentry/memmap/memmap.go @@ -356,6 +356,6 @@ type MMapOpts struct { // Hint is the name used for the mapping in /proc/[pid]/maps. If Hint is // empty, MappingIdentity.MappedName() will be used instead. // - // TODO: Replace entirely with MappingIdentity? + // TODO(jamieliu): Replace entirely with MappingIdentity? Hint string } diff --git a/pkg/sentry/mm/aio_context.go b/pkg/sentry/mm/aio_context.go index f7ff06de0..7075792e0 100644 --- a/pkg/sentry/mm/aio_context.go +++ b/pkg/sentry/mm/aio_context.go @@ -331,7 +331,7 @@ func (mm *MemoryManager) NewAIOContext(ctx context.Context, events uint32) (uint Length: aioRingBufferSize, MappingIdentity: m, Mappable: m, - // TODO: Linux does "do_mmap_pgoff(..., PROT_READ | + // TODO(fvoznika): Linux does "do_mmap_pgoff(..., PROT_READ | // PROT_WRITE, ...)" in fs/aio.c:aio_setup_ring(); why do we make this // mapping read-only? Perms: usermem.Read, diff --git a/pkg/sentry/mm/procfs.go b/pkg/sentry/mm/procfs.go index 0c4b8895d..7cdbf6e25 100644 --- a/pkg/sentry/mm/procfs.go +++ b/pkg/sentry/mm/procfs.go @@ -69,7 +69,7 @@ func (mm *MemoryManager) ReadMapsSeqFileData(ctx context.Context, handle seqfile start = *handle.(*usermem.Addr) } for vseg := mm.vmas.LowerBoundSegment(start); vseg.Ok(); vseg = vseg.NextSegment() { - // FIXME: If we use a usermem.Addr for the handle, we get + // FIXME(b/30793614): If we use a usermem.Addr for the handle, we get // "panic: autosave error: type usermem.Addr is not registered". vmaAddr := vseg.End() data = append(data, seqfile.SeqData{ @@ -88,7 +88,7 @@ func (mm *MemoryManager) ReadMapsSeqFileData(ctx context.Context, handle seqfile // // Artifically adjust the seqfile handle so we only output vsyscall entry once. if start != vsyscallEnd { - // FIXME: Can't get a pointer to constant vsyscallEnd. + // FIXME(b/30793614): Can't get a pointer to constant vsyscallEnd. vmaAddr := vsyscallEnd data = append(data, seqfile.SeqData{ Buf: []byte(vsyscallMapsEntry), @@ -134,7 +134,7 @@ func (mm *MemoryManager) appendVMAMapsEntryLocked(ctx context.Context, vseg vmaI if vma.hint != "" { s = vma.hint } else if vma.id != nil { - // FIXME: We are holding mm.mappingMu here, which is + // FIXME(jamieliu): We are holding mm.mappingMu here, which is // consistent with Linux's holding mmap_sem in // fs/proc/task_mmu.c:show_map_vma() => fs/seq_file.c:seq_file_path(). // However, it's not clear that fs.File.MappedName() is actually @@ -162,7 +162,7 @@ func (mm *MemoryManager) ReadSmapsSeqFileData(ctx context.Context, handle seqfil start = *handle.(*usermem.Addr) } for vseg := mm.vmas.LowerBoundSegment(start); vseg.Ok(); vseg = vseg.NextSegment() { - // FIXME: If we use a usermem.Addr for the handle, we get + // FIXME(b/30793614): If we use a usermem.Addr for the handle, we get // "panic: autosave error: type usermem.Addr is not registered". vmaAddr := vseg.End() data = append(data, seqfile.SeqData{ @@ -174,7 +174,7 @@ func (mm *MemoryManager) ReadSmapsSeqFileData(ctx context.Context, handle seqfil // We always emulate vsyscall, so advertise it here. See // ReadMapsSeqFileData for additional commentary. if start != vsyscallEnd { - // FIXME: Can't get a pointer to constant vsyscallEnd. + // FIXME(b/30793614): Can't get a pointer to constant vsyscallEnd. vmaAddr := vsyscallEnd data = append(data, seqfile.SeqData{ Buf: []byte(vsyscallSmapsEntry), diff --git a/pkg/sentry/mm/special_mappable.go b/pkg/sentry/mm/special_mappable.go index cfbf7a104..3b5161998 100644 --- a/pkg/sentry/mm/special_mappable.go +++ b/pkg/sentry/mm/special_mappable.go @@ -136,7 +136,7 @@ func (m *SpecialMappable) Length() uint64 { // NewSharedAnonMappable returns a SpecialMappable that implements the // semantics of mmap(MAP_SHARED|MAP_ANONYMOUS) and mappings of /dev/zero. // -// TODO: The use of SpecialMappable is a lazy code reuse hack. Linux +// TODO(jamieliu): The use of SpecialMappable is a lazy code reuse hack. Linux // uses an ephemeral file created by mm/shmem.c:shmem_zero_setup(); we should // do the same to get non-zero device and inode IDs. func NewSharedAnonMappable(length uint64, mfp pgalloc.MemoryFileProvider) (*SpecialMappable, error) { diff --git a/pkg/sentry/mm/syscalls.go b/pkg/sentry/mm/syscalls.go index cc7eb76d2..7b675b9b5 100644 --- a/pkg/sentry/mm/syscalls.go +++ b/pkg/sentry/mm/syscalls.go @@ -137,7 +137,7 @@ func (mm *MemoryManager) MMap(ctx context.Context, opts memmap.MMapOpts) (userme return 0, err } - // TODO: In Linux, VM_LOCKONFAULT (which may be set on the new + // TODO(jamieliu): In Linux, VM_LOCKONFAULT (which may be set on the new // vma by mlockall(MCL_FUTURE|MCL_ONFAULT) => mm_struct::def_flags) appears // to effectively disable MAP_POPULATE by unsetting FOLL_POPULATE in // mm/util.c:vm_mmap_pgoff() => mm/gup.c:__mm_populate() => @@ -148,7 +148,7 @@ func (mm *MemoryManager) MMap(ctx context.Context, opts memmap.MMapOpts) (userme mm.populateVMAAndUnlock(ctx, vseg, ar, true) case opts.Mappable == nil && length <= privateAllocUnit: - // NOTE: Get pmas and map eagerly in the hope + // NOTE(b/63077076, b/63360184): Get pmas and map eagerly in the hope // that doing so will save on future page faults. We only do this for // anonymous mappings, since otherwise the cost of // memmap.Mappable.Translate is unknown; and only for small mappings, @@ -698,7 +698,7 @@ func (mm *MemoryManager) Brk(ctx context.Context, addr usermem.Addr) (usermem.Ad return mm.brk.End, syserror.EINVAL } - // TODO: This enforces RLIMIT_DATA, but is + // TODO(gvisor.dev/issue/156): This enforces RLIMIT_DATA, but is // slightly more permissive than the usual data limit. In particular, // this only limits the size of the heap; a true RLIMIT_DATA limits the // size of heap + data + bss. The segment sizes need to be plumbed from diff --git a/pkg/sentry/mm/vma.go b/pkg/sentry/mm/vma.go index e9c9a80ea..931995254 100644 --- a/pkg/sentry/mm/vma.go +++ b/pkg/sentry/mm/vma.go @@ -274,7 +274,7 @@ func (mm *MemoryManager) getVMAsLocked(ctx context.Context, ar usermem.AddrRange // Loop invariants: vgap = vseg.PrevGap(); addr < vseg.End(). vma := vseg.ValuePtr() if addr < vseg.Start() { - // TODO: Implement vma.growsDown here. + // TODO(jamieliu): Implement vma.growsDown here. return vbegin, vgap, syserror.EFAULT } diff --git a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go index c0a0af92d..d0f6bb225 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go @@ -62,7 +62,7 @@ func updateSystemValues(fd int) error { // Calculate whether guestPCID is supported. // - // FIXME: These should go through the much more pleasant + // FIXME(ascannell): These should go through the much more pleasant // cpuid package interfaces, once a way to accept raw kvm CPUID entries // is plumbed (or some rough equivalent). for i := 0; i < int(cpuidSupported.nr); i++ { diff --git a/pkg/sentry/platform/platform.go b/pkg/sentry/platform/platform.go index d1c9458ea..0e48417b9 100644 --- a/pkg/sentry/platform/platform.go +++ b/pkg/sentry/platform/platform.go @@ -181,7 +181,7 @@ var ( // this signal both to Contexts and to the sentry itself, under the assumption // that they originate from races with Context.Interrupt(). // -// NOTE: The Go runtime only guarantees that a small subset +// NOTE(b/23420492): The Go runtime only guarantees that a small subset // of signals will be always be unblocked on all threads, one of which // is SIGCHLD. const SignalInterrupt = linux.SIGCHLD diff --git a/pkg/sentry/platform/ptrace/subprocess.go b/pkg/sentry/platform/ptrace/subprocess.go index 82f125073..2a5d699ec 100644 --- a/pkg/sentry/platform/ptrace/subprocess.go +++ b/pkg/sentry/platform/ptrace/subprocess.go @@ -79,7 +79,7 @@ func (tp *threadPool) lookupOrCreate(currentTID int32, newThread func() *thread) // Before creating a new thread, see if we can find a thread // whose system tid has disappeared. // - // TODO: Other parts of this package depend on + // TODO(b/77216482): Other parts of this package depend on // threads never exiting. for origTID, t := range tp.threads { // Signal zero is an easy existence check. diff --git a/pkg/sentry/platform/ring0/x86.go b/pkg/sentry/platform/ring0/x86.go index 7c88010d8..4c6daec22 100644 --- a/pkg/sentry/platform/ring0/x86.go +++ b/pkg/sentry/platform/ring0/x86.go @@ -116,7 +116,7 @@ const ( // // Note that sign-extension semantics apply to the highest order bit. // -// FIXME: This should use the cpuid passed to Init. +// FIXME(b/69382326): This should use the cpuid passed to Init. func VirtualAddressBits() uint32 { ax, _, _, _ := cpuid.HostID(0x80000008, 0) return (ax >> 8) & 0xff @@ -124,7 +124,7 @@ func VirtualAddressBits() uint32 { // PhysicalAddressBits returns the number of bits available for physical addresses. // -// FIXME: This should use the cpuid passed to Init. +// FIXME(b/69382326): This should use the cpuid passed to Init. func PhysicalAddressBits() uint32 { ax, _, _, _ := cpuid.HostID(0x80000008, 0) return ax & 0xff diff --git a/pkg/sentry/sighandling/sighandling.go b/pkg/sentry/sighandling/sighandling.go index 6b5d5f993..571245ce5 100644 --- a/pkg/sentry/sighandling/sighandling.go +++ b/pkg/sentry/sighandling/sighandling.go @@ -86,7 +86,7 @@ func handleSignals(sigchans []chan os.Signal, handler func(linux.Signal), start, // // Otherwise ignore the signal. // - // TODO: Drop in Go 1.12, which uses tgkill + // TODO(b/114489875): Drop in Go 1.12, which uses tgkill // in runtime.raise. switch signal { case linux.SIGHUP, linux.SIGINT, linux.SIGTERM: diff --git a/pkg/sentry/sighandling/sighandling_unsafe.go b/pkg/sentry/sighandling/sighandling_unsafe.go index 5913d47a8..db6e71487 100644 --- a/pkg/sentry/sighandling/sighandling_unsafe.go +++ b/pkg/sentry/sighandling/sighandling_unsafe.go @@ -23,7 +23,7 @@ import ( "gvisor.googlesource.com/gvisor/pkg/abi/linux" ) -// TODO: Move to pkg/abi/linux along with definitions in +// TODO(b/34161764): Move to pkg/abi/linux along with definitions in // pkg/sentry/arch. type sigaction struct { handler uintptr diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 23138d874..768fa0dfa 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -608,7 +608,7 @@ func (s *SocketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { // GetSockOpt implements the linux syscall getsockopt(2) for sockets backed by // tcpip.Endpoint. func (s *SocketOperations) GetSockOpt(t *kernel.Task, level, name, outLen int) (interface{}, *syserr.Error) { - // TODO: Unlike other socket options, SO_TIMESTAMP is + // TODO(b/78348848): Unlike other socket options, SO_TIMESTAMP is // implemented specifically for epsocket.SocketOperations rather than // commonEndpoint. commonEndpoint should be extended to support socket // options where the implementation is not shared, as unix sockets need @@ -658,7 +658,7 @@ func GetSockOpt(t *kernel.Task, s socket.Socket, ep commonEndpoint, family int, // getSockOptSocket implements GetSockOpt when level is SOL_SOCKET. func getSockOptSocket(t *kernel.Task, s socket.Socket, ep commonEndpoint, family int, skType transport.SockType, name, outLen int) (interface{}, *syserr.Error) { - // TODO: Stop rejecting short optLen values in getsockopt. + // TODO(b/124056281): Stop rejecting short optLen values in getsockopt. switch name { case linux.SO_TYPE: if outLen < sizeOfInt32 { @@ -789,7 +789,7 @@ func getSockOptSocket(t *kernel.Task, s socket.Socket, ep commonEndpoint, family return linux.Linger{}, nil case linux.SO_SNDTIMEO: - // TODO: Linux allows shorter lengths for partial results. + // TODO(igudger): Linux allows shorter lengths for partial results. if outLen < linux.SizeOfTimeval { return nil, syserr.ErrInvalidArgument } @@ -797,7 +797,7 @@ func getSockOptSocket(t *kernel.Task, s socket.Socket, ep commonEndpoint, family return linux.NsecToTimeval(s.SendTimeout()), nil case linux.SO_RCVTIMEO: - // TODO: Linux allows shorter lengths for partial results. + // TODO(igudger): Linux allows shorter lengths for partial results. if outLen < linux.SizeOfTimeval { return nil, syserr.ErrInvalidArgument } @@ -894,7 +894,7 @@ func getSockOptTCP(t *kernel.Task, ep commonEndpoint, name, outLen int) (interfa return nil, syserr.TranslateNetstackError(err) } - // TODO: Translate fields once they are added to + // TODO(b/64800844): Translate fields once they are added to // tcpip.TCPInfoOption. info := linux.TCPInfo{} @@ -995,7 +995,7 @@ func getSockOptIP(t *kernel.Task, ep commonEndpoint, name, outLen int) (interfac // SetSockOpt implements the linux syscall setsockopt(2) for sockets backed by // tcpip.Endpoint. func (s *SocketOperations) SetSockOpt(t *kernel.Task, level int, name int, optVal []byte) *syserr.Error { - // TODO: Unlike other socket options, SO_TIMESTAMP is + // TODO(b/78348848): Unlike other socket options, SO_TIMESTAMP is // implemented specifically for epsocket.SocketOperations rather than // commonEndpoint. commonEndpoint should be extended to support socket // options where the implementation is not shared, as unix sockets need @@ -1338,7 +1338,7 @@ func setSockOptIP(t *kernel.Task, ep commonEndpoint, name int, optVal []byte) *s return syserr.TranslateNetstackError(ep.SetSockOpt(tcpip.AddMembershipOption{ NIC: tcpip.NICID(req.InterfaceIndex), - // TODO: Change AddMembership to use the standard + // TODO(igudger): Change AddMembership to use the standard // any address representation. InterfaceAddr: tcpip.Address(req.InterfaceAddr[:]), MulticastAddr: tcpip.Address(req.MulticastAddr[:]), @@ -1352,7 +1352,7 @@ func setSockOptIP(t *kernel.Task, ep commonEndpoint, name int, optVal []byte) *s return syserr.TranslateNetstackError(ep.SetSockOpt(tcpip.RemoveMembershipOption{ NIC: tcpip.NICID(req.InterfaceIndex), - // TODO: Change DropMembership to use the standard + // TODO(igudger): Change DropMembership to use the standard // any address representation. InterfaceAddr: tcpip.Address(req.InterfaceAddr[:]), MulticastAddr: tcpip.Address(req.MulticastAddr[:]), @@ -1380,7 +1380,7 @@ func setSockOptIP(t *kernel.Task, ep commonEndpoint, name int, optVal []byte) *s )) case linux.MCAST_JOIN_GROUP: - // FIXME: Implement MCAST_JOIN_GROUP. + // FIXME(b/124219304): Implement MCAST_JOIN_GROUP. t.Kernel().EmitUnimplementedEvent(t) return syserr.ErrInvalidArgument @@ -1695,7 +1695,7 @@ func (s *SocketOperations) coalescingRead(ctx context.Context, dst usermem.IOSeq // nonBlockingRead issues a non-blocking read. // -// TODO: Support timestamps for stream sockets. +// TODO(b/78348848): Support timestamps for stream sockets. func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSequence, peek, trunc, senderRequested bool) (int, int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { isPacket := s.isPacketBased() @@ -1762,7 +1762,7 @@ func (s *SocketOperations) nonBlockingRead(ctx context.Context, dst usermem.IOSe dst = dst.DropFirst(n) num, err := dst.CopyOutFrom(ctx, safemem.FromVecReaderFunc{func(dsts [][]byte) (int64, error) { n, _, err := s.Endpoint.Peek(dsts) - // TODO: Handle peek timestamp. + // TODO(b/78348848): Handle peek timestamp. if err != nil { return int64(n), syserr.TranslateNetstackError(err).ToError() } @@ -1963,7 +1963,7 @@ func (s *SocketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] func (s *SocketOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { // SIOCGSTAMP is implemented by epsocket rather than all commonEndpoint // sockets. - // TODO: Add a commonEndpoint method to support SIOCGSTAMP. + // TODO(b/78348848): Add a commonEndpoint method to support SIOCGSTAMP. if int(args[1].Int()) == syscall.SIOCGSTAMP { s.readMu.Lock() defer s.readMu.Unlock() @@ -2153,19 +2153,19 @@ func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFRe case syscall.SIOCGIFMAP: // Gets the hardware parameters of the device. - // TODO: Implement. + // TODO(b/71872867): Implement. case syscall.SIOCGIFTXQLEN: // Gets the transmit queue length of the device. - // TODO: Implement. + // TODO(b/71872867): Implement. case syscall.SIOCGIFDSTADDR: // Gets the destination address of a point-to-point device. - // TODO: Implement. + // TODO(b/71872867): Implement. case syscall.SIOCGIFBRDADDR: // Gets the broadcast address of a device. - // TODO: Implement. + // TODO(b/71872867): Implement. case syscall.SIOCGIFNETMASK: // Gets the network mask of a device. diff --git a/pkg/sentry/socket/epsocket/save_restore.go b/pkg/sentry/socket/epsocket/save_restore.go index 34d9a7cf0..f19afb6c0 100644 --- a/pkg/sentry/socket/epsocket/save_restore.go +++ b/pkg/sentry/socket/epsocket/save_restore.go @@ -20,7 +20,7 @@ import ( // afterLoad is invoked by stateify. func (s *Stack) afterLoad() { - s.Stack = stack.StackFromEnv // FIXME + s.Stack = stack.StackFromEnv // FIXME(b/36201077) if s.Stack == nil { panic("can't restore without netstack/tcpip/stack.Stack") } diff --git a/pkg/sentry/socket/epsocket/stack.go b/pkg/sentry/socket/epsocket/stack.go index c0081c819..37c48f4bc 100644 --- a/pkg/sentry/socket/epsocket/stack.go +++ b/pkg/sentry/socket/epsocket/stack.go @@ -77,7 +77,7 @@ func (s *Stack) InterfaceAddrs() map[int32][]inet.InterfaceAddr { Family: family, PrefixLen: uint8(len(a.Address) * 8), Addr: []byte(a.Address), - // TODO: Other fields. + // TODO(b/68878065): Other fields. }) } nicAddrs[int32(id)] = addrs diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index c4848b313..49349074f 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -348,7 +348,7 @@ func (s *socketOperations) SetSockOpt(t *kernel.Task, level int, name int, opt [ func (s *socketOperations) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, int, interface{}, uint32, socket.ControlMessages, *syserr.Error) { // Whitelist flags. // - // FIXME: We can't support MSG_ERRQUEUE because it uses ancillary + // FIXME(jamieliu): We can't support MSG_ERRQUEUE because it uses ancillary // messages that netstack/tcpip/transport/unix doesn't understand. Kill the // Socket interface's dependence on netstack. if flags&^(syscall.MSG_DONTWAIT|syscall.MSG_PEEK|syscall.MSG_TRUNC) != 0 { diff --git a/pkg/sentry/socket/netlink/route/protocol.go b/pkg/sentry/socket/netlink/route/protocol.go index 7e70b09b2..e414b829b 100644 --- a/pkg/sentry/socket/netlink/route/protocol.go +++ b/pkg/sentry/socket/netlink/route/protocol.go @@ -110,7 +110,7 @@ func (p *Protocol) dumpLinks(ctx context.Context, hdr linux.NetlinkMessageHeader m.PutAttr(linux.IFLA_ADDRESS, mac) m.PutAttr(linux.IFLA_BROADCAST, brd) - // TODO: There are many more attributes. + // TODO(b/68878065): There are many more attributes. } return nil @@ -122,7 +122,7 @@ func (p *Protocol) dumpAddrs(ctx context.Context, hdr linux.NetlinkMessageHeader // netlink header and 1 byte protocol family common to all // NETLINK_ROUTE requests. // - // TODO: Filter output by passed protocol family. + // TODO(b/68878065): Filter output by passed protocol family. // The RTM_GETADDR dump response is a set of RTM_NEWADDR messages each // containing an InterfaceAddrMessage followed by a set of netlink @@ -151,7 +151,7 @@ func (p *Protocol) dumpAddrs(ctx context.Context, hdr linux.NetlinkMessageHeader m.PutAttr(linux.IFA_ADDRESS, []byte(a.Addr)) - // TODO: There are many more attributes. + // TODO(b/68878065): There are many more attributes. } } @@ -175,7 +175,7 @@ func (p *Protocol) ProcessMessage(ctx context.Context, hdr linux.NetlinkMessageH } } - // TODO: Only the dump variant of the types below are + // TODO(b/68878065): Only the dump variant of the types below are // supported. if hdr.Flags&linux.NLM_F_DUMP != linux.NLM_F_DUMP { return syserr.ErrNotSupported diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index 0fe9b39b6..a34f9d3ca 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -168,7 +168,7 @@ func (s *Socket) EventUnregister(e *waiter.Entry) { // Ioctl implements fs.FileOperations.Ioctl. func (s *Socket) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { - // TODO: no ioctls supported. + // TODO(b/68878065): no ioctls supported. return 0, syserror.ENOTTY } @@ -319,7 +319,7 @@ func (s *Socket) GetSockOpt(t *kernel.Task, level int, name int, outLen int) (in t.Kernel().EmitUnimplementedEvent(t) } } - // TODO: other sockopts are not supported. + // TODO(b/68878065): other sockopts are not supported. return nil, syserr.ErrProtocolNotAvailable } @@ -369,7 +369,7 @@ func (s *Socket) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *sy } } - // TODO: other sockopts are not supported. + // TODO(b/68878065): other sockopts are not supported. return syserr.ErrProtocolNotAvailable } @@ -389,7 +389,7 @@ func (s *Socket) GetSockName(t *kernel.Task) (interface{}, uint32, *syserr.Error func (s *Socket) GetPeerName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { sa := linux.SockAddrNetlink{ Family: linux.AF_NETLINK, - // TODO: Support non-kernel peers. For now the peer + // TODO(b/68878065): Support non-kernel peers. For now the peer // must be the kernel. PortID: 0, } @@ -540,7 +540,7 @@ func (s *Socket) processMessages(ctx context.Context, buf []byte) *syserr.Error continue } - // TODO: ACKs not supported yet. + // TODO(b/68877377): ACKs not supported yet. if hdr.Flags&linux.NLM_F_ACK == linux.NLM_F_ACK { return syserr.ErrNotSupported } diff --git a/pkg/sentry/socket/rpcinet/conn/conn.go b/pkg/sentry/socket/rpcinet/conn/conn.go index 9c749b888..64106c4b5 100644 --- a/pkg/sentry/socket/rpcinet/conn/conn.go +++ b/pkg/sentry/socket/rpcinet/conn/conn.go @@ -50,7 +50,7 @@ type RPCConnection struct { // NewRPCConnection initializes a RPC connection to a socket gofer. func NewRPCConnection(s *unet.Socket) *RPCConnection { conn := &RPCConnection{socket: s, requests: map[uint64]request{}} - go func() { // S/R-FIXME + go func() { // S/R-FIXME(b/77962828) var nums [16]byte for { for n := 0; n < len(nums); { diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier.go b/pkg/sentry/socket/rpcinet/notifier/notifier.go index d9bda78b0..f06d12231 100644 --- a/pkg/sentry/socket/rpcinet/notifier/notifier.go +++ b/pkg/sentry/socket/rpcinet/notifier/notifier.go @@ -64,7 +64,7 @@ func NewRPCNotifier(cn *conn.RPCConnection) (*Notifier, error) { fdMap: make(map[uint32]*fdInfo), } - go w.waitAndNotify() // S/R-FIXME + go w.waitAndNotify() // S/R-FIXME(b/77962828) return w, nil } @@ -166,7 +166,7 @@ func (n *Notifier) waitAndNotify() error { res := n.rpcConn.Request(id).Result.(*pb.SyscallResponse_EpollWait).EpollWait.Result if e, ok := res.(*pb.EpollWaitResponse_ErrorNumber); ok { err := syscall.Errno(e.ErrorNumber) - // NOTE: I don't think epoll_wait can return EAGAIN but I'm being + // NOTE(magi): I don't think epoll_wait can return EAGAIN but I'm being // conseratively careful here since exiting the notification thread // would be really bad. if err == syscall.EINTR || err == syscall.EAGAIN { diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index 3418a6d75..cf8f69efb 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -288,7 +288,7 @@ func (s *socketOperations) Accept(t *kernel.Task, peerRequested bool, flags int, if blocking && se == syserr.ErrTryAgain { // Register for notifications. e, ch := waiter.NewChannelEntry(nil) - // FIXME: This waiter.EventHUp is a partial + // FIXME(b/119878986): This waiter.EventHUp is a partial // measure, need to figure out how to translate linux events to // internal events. s.EventRegister(&e, waiter.EventIn|waiter.EventHUp) @@ -370,7 +370,7 @@ func (s *socketOperations) Shutdown(t *kernel.Task, how int) *syserr.Error { // We save the shutdown state because of strange differences on linux // related to recvs on blocking vs. non-blocking sockets after a SHUT_RD. // We need to emulate that behavior on the blocking side. - // TODO: There is a possible race that can exist with loopback, + // TODO(b/120096741): There is a possible race that can exist with loopback, // where data could possibly be lost. s.setShutdownFlags(how) @@ -771,7 +771,7 @@ func (s *socketOperations) SendMsg(t *kernel.Task, src usermem.IOSequence, to [] return 0, syserr.FromError(err) } - // TODO: this needs to change to map directly to a SendMsg syscall + // TODO(bgeffon): this needs to change to map directly to a SendMsg syscall // in the RPC. totalWritten := 0 n, err := rpcSendMsg(t, &pb.SyscallRequest_Sendmsg{&pb.SendmsgRequest{ diff --git a/pkg/sentry/socket/rpcinet/syscall_rpc.proto b/pkg/sentry/socket/rpcinet/syscall_rpc.proto index c056e4c9d..9586f5923 100644 --- a/pkg/sentry/socket/rpcinet/syscall_rpc.proto +++ b/pkg/sentry/socket/rpcinet/syscall_rpc.proto @@ -3,7 +3,7 @@ syntax = "proto3"; // package syscall_rpc is a set of networking related system calls that can be // forwarded to a socket gofer. // -// TODO: Document individual RPCs. +// TODO(b/77963526): Document individual RPCs. package syscall_rpc; message SendmsgRequest { diff --git a/pkg/sentry/strace/strace.go b/pkg/sentry/strace/strace.go index a6d870b44..434a200d9 100644 --- a/pkg/sentry/strace/strace.go +++ b/pkg/sentry/strace/strace.go @@ -722,7 +722,7 @@ func (s SyscallMap) Name(sysno uintptr) string { // N.B. This is not in an init function because we can't be sure all syscall // tables are registered with the kernel when init runs. // -// TODO: remove kernel package dependencies from this +// TODO(gvisor.dev/issue/155): remove kernel package dependencies from this // package and have the kernel package self-initialize all syscall tables. func Initialize() { for _, table := range kernel.SyscallTables() { diff --git a/pkg/sentry/syscalls/linux/error.go b/pkg/sentry/syscalls/linux/error.go index 8759e5e32..304a12dde 100644 --- a/pkg/sentry/syscalls/linux/error.go +++ b/pkg/sentry/syscalls/linux/error.go @@ -89,7 +89,7 @@ func handleIOError(t *kernel.Task, partialResult bool, err, intr error, op strin // side is gone. The partial write is returned. EPIPE will be // returned on the next call. // - // TODO: In some cases SIGPIPE should + // TODO(gvisor.dev/issue/161): In some cases SIGPIPE should // also be sent to the application. return nil case syserror.ErrWouldBlock: diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go index be793ca11..b9b4ccbd1 100644 --- a/pkg/sentry/syscalls/linux/linux64.go +++ b/pkg/sentry/syscalls/linux/linux64.go @@ -143,10 +143,10 @@ var AMD64 = &kernel.SyscallTable{ 65: Semop, 66: Semctl, 67: Shmdt, - // 68: @Syscall(Msgget), TODO - // 69: @Syscall(Msgsnd), TODO - // 70: @Syscall(Msgrcv), TODO - // 71: @Syscall(Msgctl), TODO + // 68: @Syscall(Msgget), TODO(b/29354921) + // 69: @Syscall(Msgsnd), TODO(b/29354921) + // 70: @Syscall(Msgrcv), TODO(b/29354921) + // 71: @Syscall(Msgctl), TODO(b/29354921) 72: Fcntl, 73: Flock, 74: Fsync, @@ -197,8 +197,8 @@ var AMD64 = &kernel.SyscallTable{ 119: Setresgid, 120: Getresgid, 121: Getpgid, - // 122: @Syscall(Setfsuid), TODO - // 123: @Syscall(Setfsgid), TODO + // 122: @Syscall(Setfsuid), TODO(b/112851702) + // 123: @Syscall(Setfsgid), TODO(b/112851702) 124: Getsid, 125: Capget, 126: Capset, @@ -217,7 +217,7 @@ var AMD64 = &kernel.SyscallTable{ 136: syscalls.ErrorWithEvent(syscall.ENOSYS), 137: Statfs, 138: Fstatfs, - // 139: @Syscall(Sysfs), TODO + // 139: @Syscall(Sysfs), TODO(gvisor.dev/issue/165) 140: Getpriority, 141: Setpriority, // @Syscall(SchedSetparam, returns:EPERM or ENOSYS, note:Returns EPERM if the process does not have cap_sys_nice; ENOSYS otherwise) @@ -291,7 +291,7 @@ var AMD64 = &kernel.SyscallTable{ // @Syscall(Security, note:Not implemented in Linux) 185: syscalls.Error(syscall.ENOSYS), 186: Gettid, - 187: nil, // @Syscall(Readahead), TODO + 187: nil, // @Syscall(Readahead), TODO(b/29351341) // @Syscall(Setxattr, returns:ENOTSUP, note:Requires filesystem support) 188: syscalls.ErrorWithEvent(syscall.ENOTSUP), // @Syscall(Lsetxattr, returns:ENOTSUP, note:Requires filesystem support) @@ -342,7 +342,7 @@ var AMD64 = &kernel.SyscallTable{ 217: Getdents64, 218: SetTidAddress, 219: RestartSyscall, - // 220: @Syscall(Semtimedop), TODO + // 220: @Syscall(Semtimedop), TODO(b/29354920) 221: Fadvise64, 222: TimerCreate, 223: TimerSettime, @@ -360,16 +360,16 @@ var AMD64 = &kernel.SyscallTable{ 235: Utimes, // @Syscall(Vserver, note:Not implemented by Linux) 236: syscalls.Error(syscall.ENOSYS), // Vserver, not implemented by Linux - // @Syscall(Mbind, returns:EPERM or ENOSYS, note:Returns EPERM if the process does not have cap_sys_nice; ENOSYS otherwise), TODO + // @Syscall(Mbind, returns:EPERM or ENOSYS, note:Returns EPERM if the process does not have cap_sys_nice; ENOSYS otherwise), TODO(b/117792295) 237: syscalls.CapError(linux.CAP_SYS_NICE), // may require cap_sys_nice 238: SetMempolicy, 239: GetMempolicy, - // 240: @Syscall(MqOpen), TODO - // 241: @Syscall(MqUnlink), TODO - // 242: @Syscall(MqTimedsend), TODO - // 243: @Syscall(MqTimedreceive), TODO - // 244: @Syscall(MqNotify), TODO - // 245: @Syscall(MqGetsetattr), TODO + // 240: @Syscall(MqOpen), TODO(b/29354921) + // 241: @Syscall(MqUnlink), TODO(b/29354921) + // 242: @Syscall(MqTimedsend), TODO(b/29354921) + // 243: @Syscall(MqTimedreceive), TODO(b/29354921) + // 244: @Syscall(MqNotify), TODO(b/29354921) + // 245: @Syscall(MqGetsetattr), TODO(b/29354921) 246: syscalls.CapError(linux.CAP_SYS_BOOT), // kexec_load, requires cap_sys_boot 247: Waitid, // @Syscall(AddKey, returns:EACCES, note:Not available to user) @@ -407,22 +407,22 @@ var AMD64 = &kernel.SyscallTable{ 273: syscalls.Error(syscall.ENOSYS), // @Syscall(GetRobustList, note:Obsolete) 274: syscalls.Error(syscall.ENOSYS), - // 275: @Syscall(Splice), TODO - // 276: @Syscall(Tee), TODO + // 275: @Syscall(Splice), TODO(b/29354098) + // 276: @Syscall(Tee), TODO(b/29354098) 277: SyncFileRange, - // 278: @Syscall(Vmsplice), TODO + // 278: @Syscall(Vmsplice), TODO(b/29354098) // @Syscall(MovePages, returns:EPERM or ENOSYS, note:Returns EPERM if the process does not have cap_sys_nice; ENOSYS otherwise) 279: syscalls.CapError(linux.CAP_SYS_NICE), // requires cap_sys_nice (mostly) 280: Utimensat, 281: EpollPwait, - // 282: @Syscall(Signalfd), TODO + // 282: @Syscall(Signalfd), TODO(b/19846426) 283: TimerfdCreate, 284: Eventfd, 285: Fallocate, 286: TimerfdSettime, 287: TimerfdGettime, 288: Accept4, - // 289: @Syscall(Signalfd4), TODO + // 289: @Syscall(Signalfd4), TODO(b/19846426) 290: Eventfd2, 291: EpollCreate1, 292: Dup3, @@ -447,17 +447,17 @@ var AMD64 = &kernel.SyscallTable{ 305: syscalls.CapError(linux.CAP_SYS_TIME), // requires cap_sys_time 306: Syncfs, 307: SendMMsg, - // 308: @Syscall(Setns), TODO + // 308: @Syscall(Setns), TODO(b/29354995) 309: Getcpu, - // 310: @Syscall(ProcessVmReadv), TODO may require cap_sys_ptrace - // 311: @Syscall(ProcessVmWritev), TODO may require cap_sys_ptrace + // 310: @Syscall(ProcessVmReadv), TODO(gvisor.dev/issue/158) may require cap_sys_ptrace + // 311: @Syscall(ProcessVmWritev), TODO(gvisor.dev/issue/158) may require cap_sys_ptrace // @Syscall(Kcmp, returns:EPERM or ENOSYS, note:Requires cap_sys_ptrace) 312: syscalls.CapError(linux.CAP_SYS_PTRACE), // @Syscall(FinitModule, returns:EPERM or ENOSYS, note:Returns EPERM if the process does not have cap_sys_module; ENOSYS otherwise) 313: syscalls.CapError(linux.CAP_SYS_MODULE), - // 314: @Syscall(SchedSetattr), TODO, we have no scheduler - // 315: @Syscall(SchedGetattr), TODO, we have no scheduler - // 316: @Syscall(Renameat2), TODO + // 314: @Syscall(SchedSetattr), TODO(b/118902272), we have no scheduler + // 315: @Syscall(SchedGetattr), TODO(b/118902272), we have no scheduler + // 316: @Syscall(Renameat2), TODO(b/118902772) 317: Seccomp, 318: GetRandom, 319: MemfdCreate, @@ -465,9 +465,9 @@ var AMD64 = &kernel.SyscallTable{ 320: syscalls.CapError(linux.CAP_SYS_BOOT), // @Syscall(Bpf, returns:EPERM or ENOSYS, note:Returns EPERM if the process does not have cap_sys_boot; ENOSYS otherwise) 321: syscalls.CapError(linux.CAP_SYS_ADMIN), // requires cap_sys_admin for all commands - // 322: @Syscall(Execveat), TODO - // 323: @Syscall(Userfaultfd), TODO - // 324: @Syscall(Membarrier), TODO + // 322: @Syscall(Execveat), TODO(b/118901836) + // 323: @Syscall(Userfaultfd), TODO(b/118906345) + // 324: @Syscall(Membarrier), TODO(b/118904897) 325: Mlock2, // Syscalls after 325 are "backports" from versions of Linux after 4.4. // 326: @Syscall(CopyFileRange), diff --git a/pkg/sentry/syscalls/linux/sys_aio.go b/pkg/sentry/syscalls/linux/sys_aio.go index 355071131..61c2647bf 100644 --- a/pkg/sentry/syscalls/linux/sys_aio.go +++ b/pkg/sentry/syscalls/linux/sys_aio.go @@ -120,7 +120,7 @@ func IoDestroy(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sys // Does not exist. return 0, nil, syserror.EINVAL } - // FIXME: Linux blocks until all AIO to the destroyed context is + // FIXME(fvoznika): Linux blocks until all AIO to the destroyed context is // done. return 0, nil, nil } diff --git a/pkg/sentry/syscalls/linux/sys_file.go b/pkg/sentry/syscalls/linux/sys_file.go index 50151f7b6..967464c85 100644 --- a/pkg/sentry/syscalls/linux/sys_file.go +++ b/pkg/sentry/syscalls/linux/sys_file.go @@ -259,7 +259,7 @@ func mknodAt(t *kernel.Task, dirFD kdefs.FD, addr usermem.Addr, mode linux.FileM case linux.ModeCharacterDevice: fallthrough case linux.ModeBlockDevice: - // TODO: We don't support creating block or character + // TODO(b/72101894): We don't support creating block or character // devices at the moment. // // When we start supporting block and character devices, we'll @@ -1532,7 +1532,7 @@ func chown(t *kernel.Task, d *fs.Dirent, uid auth.UID, gid auth.GID) error { owner.GID = kgid } - // FIXME: This is racy; the inode's owner may have changed in + // FIXME(b/62949101): This is racy; the inode's owner may have changed in // the meantime. (Linux holds i_mutex while calling // fs/attr.c:notify_change() => inode_operations::setattr => // inode_change_ok().) diff --git a/pkg/sentry/syscalls/linux/sys_mmap.go b/pkg/sentry/syscalls/linux/sys_mmap.go index 8732861e0..805b251b1 100644 --- a/pkg/sentry/syscalls/linux/sys_mmap.go +++ b/pkg/sentry/syscalls/linux/sys_mmap.go @@ -185,7 +185,7 @@ func Madvise(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sysca case linux.MADV_MERGEABLE, linux.MADV_UNMERGEABLE: fallthrough case linux.MADV_DONTDUMP, linux.MADV_DODUMP: - // TODO: Core dumping isn't implemented, so these are + // TODO(b/72045799): Core dumping isn't implemented, so these are // no-ops. fallthrough case linux.MADV_NORMAL, linux.MADV_RANDOM, linux.MADV_SEQUENTIAL, linux.MADV_WILLNEED: @@ -223,7 +223,7 @@ func GetMempolicy(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel. nodeFlag := flags&linux.MPOL_F_NODE != 0 addrFlag := flags&linux.MPOL_F_ADDR != 0 - // TODO: Once sysfs is implemented, report a single numa node in + // TODO(rahat): Once sysfs is implemented, report a single numa node in // /sys/devices/system/node. if nodemask != 0 && maxnode < 1 { return 0, nil, syserror.EINVAL diff --git a/pkg/sentry/syscalls/linux/sys_read.go b/pkg/sentry/syscalls/linux/sys_read.go index 8105e9b43..50c7d7a74 100644 --- a/pkg/sentry/syscalls/linux/sys_read.go +++ b/pkg/sentry/syscalls/linux/sys_read.go @@ -192,7 +192,7 @@ func Preadv(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscal } // Preadv2 implements linux syscall preadv2(2). -// TODO: Implement RWF_HIPRI functionality. +// TODO(b/120162627): Implement RWF_HIPRI functionality. func Preadv2(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { // While the syscall is // preadv2(int fd, struct iovec* iov, int iov_cnt, off_t offset, int flags) diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index 30ccc3f66..c8748958a 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -317,7 +317,7 @@ func accept(t *kernel.Task, fd kdefs.FD, addr usermem.Addr, addrLen usermem.Addr return 0, syserror.ConvertIntr(e.ToError(), kernel.ERESTARTSYS) } if peerRequested { - // NOTE: Linux does not give you an error if it can't + // 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 == syscall.EINVAL { return 0, err @@ -735,7 +735,7 @@ func recvSingleMsg(t *kernel.Task, s socket.Socket, msgPtr usermem.Addr, flags i return 0, err } - // FIXME: Pretend we have an empty error queue. + // FIXME(b/63594852): Pretend we have an empty error queue. if flags&linux.MSG_ERRQUEUE != 0 { return 0, syscall.EAGAIN } diff --git a/pkg/sentry/syscalls/linux/sys_thread.go b/pkg/sentry/syscalls/linux/sys_thread.go index 61cafefb9..ddcb5b789 100644 --- a/pkg/sentry/syscalls/linux/sys_thread.go +++ b/pkg/sentry/syscalls/linux/sys_thread.go @@ -350,7 +350,7 @@ func Waitid(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscal } si.SetPid(int32(wr.TID)) si.SetUid(int32(wr.UID)) - // TODO: convert kernel.ExitStatus to functions and make + // TODO(b/73541790): convert kernel.ExitStatus to functions and make // WaitResult.Status a linux.WaitStatus s := syscall.WaitStatus(wr.Status) switch { diff --git a/pkg/sentry/syscalls/linux/sys_write.go b/pkg/sentry/syscalls/linux/sys_write.go index a5ad7efb2..e405608c4 100644 --- a/pkg/sentry/syscalls/linux/sys_write.go +++ b/pkg/sentry/syscalls/linux/sys_write.go @@ -192,8 +192,8 @@ func Pwritev(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Sysca } // Pwritev2 implements linux syscall pwritev2(2). -// TODO: Implement RWF_HIPRI functionality. -// TODO: Implement O_SYNC and D_SYNC functionality. +// TODO(b/120162627): Implement RWF_HIPRI functionality. +// TODO(b/120161091): Implement O_SYNC and D_SYNC functionality. func Pwritev2(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { // While the syscall is // pwritev2(int fd, struct iovec* iov, int iov_cnt, off_t offset, int flags) diff --git a/pkg/sentry/time/calibrated_clock.go b/pkg/sentry/time/calibrated_clock.go index c8cf4eca4..a98bcd7de 100644 --- a/pkg/sentry/time/calibrated_clock.go +++ b/pkg/sentry/time/calibrated_clock.go @@ -37,7 +37,7 @@ var fallbackMetric = metric.MustCreateNewUint64Metric("/time/fallback", false /* // clock. type CalibratedClock struct { // mu protects the fields below. - // TODO: consider a sequence counter for read locking. + // TODO(mpratt): consider a sequence counter for read locking. mu sync.RWMutex // ref sample the reference clock that this clock is calibrated @@ -140,7 +140,7 @@ func (c *CalibratedClock) updateParams(actual Parameters) { // N.B. logErrorAdjustment will have already logged the error // at warning level. // - // TODO: We could allow Realtime clock jumps here. + // TODO(mpratt): We could allow Realtime clock jumps here. c.resetLocked("Extreme clock error.") return } @@ -229,7 +229,7 @@ func (c *CalibratedClock) GetTime() (int64, error) { // CalibratedClocks contains calibrated monotonic and realtime clocks. // -// TODO: We know that Linux runs the monotonic and realtime clocks at +// TODO(mpratt): We know that Linux runs the monotonic and realtime clocks at // the same rate, so rather than tracking both individually, we could do one // calibration for both clocks. type CalibratedClocks struct { diff --git a/pkg/sentry/time/parameters.go b/pkg/sentry/time/parameters.go index f3ad58454..8568b1193 100644 --- a/pkg/sentry/time/parameters.go +++ b/pkg/sentry/time/parameters.go @@ -43,7 +43,7 @@ const ( // These statements assume that the host clock does not change. Actual // error will depend upon host clock changes. // - // TODO: make error correction more robust to delayed + // TODO(b/68779214): make error correction more robust to delayed // updates. ApproxUpdateInterval = 1 * time.Second diff --git a/pkg/sentry/usermem/usermem.go b/pkg/sentry/usermem/usermem.go index 99766a803..4c7d5014a 100644 --- a/pkg/sentry/usermem/usermem.go +++ b/pkg/sentry/usermem/usermem.go @@ -28,7 +28,7 @@ import ( // IO provides access to the contents of a virtual memory space. // -// FIXME: Implementations of IO cannot expect ctx to contain any +// FIXME(b/38173783): Implementations of IO cannot expect ctx to contain any // meaningful data. type IO interface { // CopyOut copies len(src) bytes from src to the memory mapped at addr. It @@ -85,7 +85,7 @@ type IO interface { // order. CopyInTo(ctx context.Context, ars AddrRangeSeq, dst safemem.Writer, opts IOOpts) (int64, error) - // TODO: The requirement that CopyOutFrom/CopyInTo call src/dst + // TODO(jamieliu): The requirement that CopyOutFrom/CopyInTo call src/dst // at most once, which is unnecessary in most cases, forces implementations // to gather safemem.Blocks into a single slice to pass to src/dst. Add // CopyOutFromIter/CopyInToIter, which relaxes this restriction, to avoid diff --git a/pkg/sentry/watchdog/watchdog.go b/pkg/sentry/watchdog/watchdog.go index c49b537a5..b4f1e3a4f 100644 --- a/pkg/sentry/watchdog/watchdog.go +++ b/pkg/sentry/watchdog/watchdog.go @@ -236,7 +236,7 @@ func (w *Watchdog) runTurn() { if !ok { // New stuck task detected. // - // TODO: Tasks blocked doing IO may be considered stuck in kernel. + // TODO(b/65849403): Tasks blocked doing IO may be considered stuck in kernel. tc = &offender{lastUpdateTime: lastUpdateTime} stuckTasks.Increment() newTaskFound = true diff --git a/pkg/syserr/syserr.go b/pkg/syserr/syserr.go index dad83e80c..232634dd4 100644 --- a/pkg/syserr/syserr.go +++ b/pkg/syserr/syserr.go @@ -49,7 +49,7 @@ func New(message string, linuxTranslation *linux.Errno) *Error { return err } - // TODO: Remove this. + // TODO(b/34162363): Remove this. errno := linuxTranslation.Number() if errno <= 0 || errno >= len(linuxBackwardsTranslations) { panic(fmt.Sprint("invalid errno: ", errno)) @@ -106,12 +106,12 @@ type linuxBackwardsTranslation struct { ok bool } -// TODO: Remove this. +// TODO(b/34162363): Remove this. var linuxBackwardsTranslations [maxErrno]linuxBackwardsTranslation // ToError translates an Error to a corresponding error value. // -// TODO: Remove this. +// TODO(b/34162363): Remove this. func (e *Error) ToError() error { if e == nil { return nil @@ -138,7 +138,7 @@ func (e *Error) ToLinux() *linux.Errno { return e.errno } -// TODO: Remove or replace most of these errors. +// TODO(b/34162363): Remove or replace most of these errors. // // Some of the errors should be replaced with package specific errors and // others should be removed entirely. @@ -278,7 +278,7 @@ var ( // FromError converts a generic error to an *Error. // -// TODO: Remove this function. +// TODO(b/34162363): Remove this function. func FromError(err error) *Error { if err == nil { return nil diff --git a/pkg/tcpip/network/ipv4/icmp.go b/pkg/tcpip/network/ipv4/icmp.go index ed9a4eee5..1c3acda4b 100644 --- a/pkg/tcpip/network/ipv4/icmp.go +++ b/pkg/tcpip/network/ipv4/icmp.go @@ -64,7 +64,7 @@ func (e *endpoint) handleICMP(r *stack.Route, netHeader buffer.View, vv buffer.V } h := header.ICMPv4(v) - // TODO: Meaningfully handle all ICMP types. + // TODO(b/112892170): Meaningfully handle all ICMP types. switch h.Type() { case header.ICMPv4Echo: received.Echo.Increment() diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go index 3210e6fc7..be28be36d 100644 --- a/pkg/tcpip/network/ipv6/icmp.go +++ b/pkg/tcpip/network/ipv6/icmp.go @@ -73,7 +73,7 @@ func (e *endpoint) handleICMP(r *stack.Route, netHeader buffer.View, vv buffer.V } h := header.ICMPv6(v) - // TODO: Meaningfully handle all ICMP types. + // TODO(b/112892170): Meaningfully handle all ICMP types. switch h.Type() { case header.ICMPv6PacketTooBig: received.PacketTooBig.Increment() @@ -247,7 +247,7 @@ func (*protocol) LinkAddressRequest(addr, localAddr tcpip.Address, linkEP stack. DstAddr: r.RemoteAddress, }) - // TODO: count this in ICMP stats. + // TODO(stijlist): count this in ICMP stats. return linkEP.WritePacket(r, nil /* gso */, hdr, buffer.VectorisedView{}, ProtocolNumber) } diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index 8b6c17a90..c18571b0f 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -176,7 +176,7 @@ func (n *NIC) primaryEndpoint(protocol tcpip.NetworkProtocolNumber) *referencedN for e := list.Front(); e != nil; e = e.Next() { r := e.(*referencedNetworkEndpoint) - // TODO: allow broadcast address when SO_BROADCAST is set. + // TODO(crawshaw): allow broadcast address when SO_BROADCAST is set. switch r.ep.ID().LocalAddress { case header.IPv4Broadcast, header.IPv4Any: continue @@ -476,7 +476,7 @@ func (n *NIC) DeliverNetworkPacket(linkEP LinkEndpoint, remote, _ tcpip.LinkAddr n.mu.RUnlock() if ok && ref.tryIncRef() { r.RemoteAddress = src - // TODO: Update the source NIC as well. + // TODO(b/123449044): Update the source NIC as well. ref.ep.HandlePacket(&r, vv) ref.decRef() } else { @@ -485,7 +485,7 @@ func (n *NIC) DeliverNetworkPacket(linkEP LinkEndpoint, remote, _ tcpip.LinkAddr hdr := buffer.NewPrependableFromView(vv.First()) vv.RemoveFirst() - // TODO: use route.WritePacket. + // TODO(b/128629022): use route.WritePacket. if err := n.linkEP.WritePacket(&r, nil /* gso */, hdr, vv, protocol); err != nil { r.Stats().IP.OutgoingPacketErrors.Increment() } else { diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index 8f7b6f781..cb9ffe9c2 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -476,7 +476,7 @@ func (s *Stack) Stats() tcpip.Stats { // SetForwarding enables or disables the packet forwarding between NICs. func (s *Stack) SetForwarding(enable bool) { - // TODO: Expose via /proc/sys/net/ipv4/ip_forward. + // TODO(igudger, bgeffon): Expose via /proc/sys/net/ipv4/ip_forward. s.mu.Lock() s.forwarding = enable s.mu.Unlock() @@ -484,7 +484,7 @@ func (s *Stack) SetForwarding(enable bool) { // Forwarding returns if the packet forwarding between NICs is enabled. func (s *Stack) Forwarding() bool { - // TODO: Expose via /proc/sys/net/ipv4/ip_forward. + // TODO(igudger, bgeffon): Expose via /proc/sys/net/ipv4/ip_forward. s.mu.RLock() defer s.mu.RUnlock() return s.forwarding diff --git a/pkg/tcpip/stack/stack_global_state.go b/pkg/tcpip/stack/stack_global_state.go index f2c6c9a8d..3d7e4b719 100644 --- a/pkg/tcpip/stack/stack_global_state.go +++ b/pkg/tcpip/stack/stack_global_state.go @@ -15,5 +15,5 @@ package stack // StackFromEnv is the global stack created in restore run. -// FIXME +// FIXME(b/36201077) var StackFromEnv *Stack diff --git a/pkg/tcpip/stack/transport_test.go b/pkg/tcpip/stack/transport_test.go index 0c2589083..2df974bf2 100644 --- a/pkg/tcpip/stack/transport_test.go +++ b/pkg/tcpip/stack/transport_test.go @@ -453,7 +453,7 @@ func TestTransportForwarding(t *testing.T) { s := stack.New([]string{"fakeNet"}, []string{"fakeTrans"}, stack.Options{}) s.SetForwarding(true) - // TODO: Change this to a channel NIC. + // TODO(b/123449044): Change this to a channel NIC. id1 := loopback.New() if err := s.CreateNIC(1, id1); err != nil { t.Fatalf("CreateNIC #1 failed: %v", err) diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go index 80cd6b4e5..b09137f08 100644 --- a/pkg/tcpip/tcpip.go +++ b/pkg/tcpip/tcpip.go @@ -444,7 +444,7 @@ type PasscredOption int // TCPInfoOption is used by GetSockOpt to expose TCP statistics. // -// TODO: Add and populate stat fields. +// TODO(b/64800844): Add and populate stat fields. type TCPInfoOption struct { RTT time.Duration RTTVar time.Duration diff --git a/pkg/tcpip/transport/raw/raw.go b/pkg/tcpip/transport/raw/raw.go index 8dada2e4f..f0f60ce91 100644 --- a/pkg/tcpip/transport/raw/raw.go +++ b/pkg/tcpip/transport/raw/raw.go @@ -100,7 +100,7 @@ type endpoint struct { } // NewEndpoint returns a raw endpoint for the given protocols. -// TODO: IP_HDRINCL, IPPROTO_RAW, and AF_PACKET. +// TODO(b/129292371): IP_HDRINCL, IPPROTO_RAW, and AF_PACKET. func NewEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, transProto tcpip.TransportProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { if netProto != header.IPv4ProtocolNumber { return nil, tcpip.ErrUnknownProtocol diff --git a/pkg/tcpip/transport/tcp/BUILD b/pkg/tcpip/transport/tcp/BUILD index e5c05f8c0..d44d63e95 100644 --- a/pkg/tcpip/transport/tcp/BUILD +++ b/pkg/tcpip/transport/tcp/BUILD @@ -73,7 +73,7 @@ go_test( "tcp_test.go", "tcp_timestamp_test.go", ], - # FIXME + # FIXME(b/68809571) tags = ["flaky"], deps = [ ":tcp", diff --git a/pkg/unet/unet.go b/pkg/unet/unet.go index deeea078d..114fb8c5b 100644 --- a/pkg/unet/unet.go +++ b/pkg/unet/unet.go @@ -211,7 +211,7 @@ func SocketPair(packet bool) (*Socket, *Socket, error) { // variable between our two sockets. We only use SocketPair in tests // anyway. // - // NOTE: This is purely due to the fact that the raw + // NOTE(b/27107811): This is purely due to the fact that the raw // syscall does not serve as a boundary for the sanitizer. var race int32 a, err := NewSocket(fds[0]) diff --git a/pkg/unet/unet_test.go b/pkg/unet/unet_test.go index ecc670925..db5485539 100644 --- a/pkg/unet/unet_test.go +++ b/pkg/unet/unet_test.go @@ -40,7 +40,7 @@ func randomFilename() (string, error) { return "", err } - // NOTE: We try to use relative path if possible. This is + // NOTE(b/26918832): We try to use relative path if possible. This is // to help conforming to the unix path length limit. if rel, err := filepath.Rel(cwd, file); err == nil { return rel, nil diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go index 2488981f9..712c50ee9 100644 --- a/runsc/boot/controller.go +++ b/runsc/boot/controller.go @@ -231,7 +231,7 @@ func (cm *containerManager) Start(args *StartArgs, _ *struct{}) error { } // Prevent CIDs containing ".." from confusing the sentry when creating // /containers/ directory. - // TODO: Once we have multiple independent roots, this + // TODO(b/129293409): Once we have multiple independent roots, this // check won't be necessary. if path.Clean(args.CID) != args.CID { return fmt.Errorf("container ID shouldn't contain directory traversals such as \"..\": %q", args.CID) @@ -352,7 +352,7 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error { return fmt.Errorf("creating network: %v", err) } if eps, ok := networkStack.(*epsocket.Stack); ok { - stack.StackFromEnv = eps.Stack // FIXME + stack.StackFromEnv = eps.Stack // FIXME(b/36201077) } info, err := o.FilePayload.Files[0].Stat() if err != nil { diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go index 761142d98..07061b9b3 100644 --- a/runsc/boot/fs.go +++ b/runsc/boot/fs.go @@ -274,7 +274,7 @@ func getMountNameAndOptions(conf *Config, m specs.Mount, fds *fdDispenser) (stri useOverlay = conf.Overlay && !mountFlags(m.Options).ReadOnly default: - // TODO: Support all the mount types and make this a + // TODO(nlacasse): Support all the mount types and make this a // fatal error. Most applications will "just work" without // them, so this is a warning for now. // we do not support. @@ -425,7 +425,7 @@ func addRestoreMount(conf *Config, renv *fs.RestoreEnvironment, m specs.Mount, f if err != nil { return err } - // TODO: Fix this when we support all the mount types and + // TODO(nlacasse): Fix this when we support all the mount types and // make this a fatal error. if fsName == "" { return nil @@ -475,7 +475,7 @@ func createRestoreEnvironment(spec *specs.Spec, conf *Config, fds *fdDispenser) } } - // TODO: handle '/tmp' properly (see mountTmp()). + // TODO(b/67958150): handle '/tmp' properly (see mountTmp()). if !tmpMounted { tmpMount := specs.Mount{ Type: tmpfs, diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go index 48ecb2626..75ec19c32 100644 --- a/runsc/boot/loader.go +++ b/runsc/boot/loader.go @@ -577,7 +577,7 @@ func (l *Loader) startContainer(k *kernel.Kernel, spec *specs.Spec, conf *Config // sentry currently supports only 1 mount namespace, which is tied to a // single user namespace. Thus we must run in the same user namespace // to access mounts. - // TODO: Create a new mount namespace for the container. + // TODO(b/63601033): Create a new mount namespace for the container. creds := auth.NewUserCredentials( auth.KUID(spec.Process.User.UID), auth.KGID(spec.Process.User.GID), diff --git a/runsc/cmd/checkpoint.go b/runsc/cmd/checkpoint.go index d8f748aa0..f722df055 100644 --- a/runsc/cmd/checkpoint.go +++ b/runsc/cmd/checkpoint.go @@ -105,7 +105,7 @@ func (c *Checkpoint) Execute(_ context.Context, f *flag.FlagSet, args ...interfa return subcommands.ExitSuccess } - // TODO: Make it possible to restore into same container. + // TODO(b/110843694): Make it possible to restore into same container. // For now, we can fake it by destroying the container and making a // new container with the same ID. This hack does not work with docker // which uses the container pid to ensure that the restore-container is diff --git a/runsc/container/container.go b/runsc/container/container.go index 1bed1a97e..a30c217f7 100644 --- a/runsc/container/container.go +++ b/runsc/container/container.go @@ -529,7 +529,7 @@ func (c *Container) WaitPID(pid int32, clearStatus bool) (syscall.WaitStatus, er // SignalContainer sends the signal to the container. If all is true and signal // is SIGKILL, then waits for all processes to exit before returning. // SignalContainer returns an error if the container is already stopped. -// TODO: Distinguish different error types. +// TODO(b/113680494): Distinguish different error types. func (c *Container) SignalContainer(sig syscall.Signal, all bool) error { log.Debugf("Signal container %q: %v", c.ID, sig) // Signaling container in Stopped state is allowed. When all=false, diff --git a/runsc/container/container_test.go b/runsc/container/container_test.go index 9fe584aa3..603c4d929 100644 --- a/runsc/container/container_test.go +++ b/runsc/container/container_test.go @@ -242,10 +242,10 @@ func configs(opts ...configOption) []*boot.Config { case overlay: c.Overlay = true case kvm: - // TODO: KVM tests are flaky. Disable until fixed. + // TODO(b/112165693): KVM tests are flaky. Disable until fixed. continue - // TODO: KVM doesn't work with --race. + // TODO(b/68787993): KVM doesn't work with --race. if testutil.RaceEnabled { continue } diff --git a/runsc/sandbox/sandbox.go b/runsc/sandbox/sandbox.go index 92495c69e..48a0dafe2 100644 --- a/runsc/sandbox/sandbox.go +++ b/runsc/sandbox/sandbox.go @@ -267,7 +267,7 @@ func (s *Sandbox) Event(cid string) (*boot.Event, error) { defer conn.Close() var e boot.Event - // TODO: Pass in the container id (cid) here. The sandbox + // TODO(b/129292330): Pass in the container id (cid) here. The sandbox // should return events only for that container. if err := conn.Call(boot.ContainerEvent, nil, &e); err != nil { return nil, fmt.Errorf("retrieving event data from sandbox: %v", err) @@ -457,7 +457,7 @@ func (s *Sandbox) createSandboxProcess(spec *specs.Spec, conf *boot.Config, bund } if conf.Platform == boot.PlatformPtrace { - // TODO: Also set a new PID namespace so that we limit + // TODO(b/75837838): Also set a new PID namespace so that we limit // access to other host processes. log.Infof("Sandbox will be started in the current PID namespace") } else { @@ -520,7 +520,7 @@ func (s *Sandbox) createSandboxProcess(spec *specs.Spec, conf *boot.Config, bund // root for itself, so it has to have the CAP_SYS_ADMIN // capability. // - // FIXME: The current implementations of + // FIXME(b/122554829): The current implementations of // os/exec doesn't allow to set ambient capabilities if // a process is started in a new user namespace. As a // workaround, we start the sandbox process with the 0 diff --git a/runsc/specutils/specutils.go b/runsc/specutils/specutils.go index 32f81b8d4..ac85bec71 100644 --- a/runsc/specutils/specutils.go +++ b/runsc/specutils/specutils.go @@ -90,7 +90,7 @@ func ValidateSpec(spec *specs.Spec) error { log.Warningf("AppArmor profile %q is being ignored", spec.Process.ApparmorProfile) } - // TODO: Apply seccomp to application inside sandbox. + // TODO(b/72226747): Apply seccomp to application inside sandbox. if spec.Linux != nil && spec.Linux.Seccomp != nil { log.Warningf("Seccomp spec is being ignored") } @@ -220,7 +220,7 @@ func Capabilities(enableRaw bool, specCaps *specs.LinuxCapabilities) (*auth.Task if caps.PermittedCaps, err = capsFromNames(specCaps.Permitted, skipSet); err != nil { return nil, err } - // TODO: Support ambient capabilities. + // TODO(nlacasse): Support ambient capabilities. } return &caps, nil } diff --git a/test/syscalls/BUILD b/test/syscalls/BUILD index 94e0f24e0..d35f59433 100644 --- a/test/syscalls/BUILD +++ b/test/syscalls/BUILD @@ -277,7 +277,7 @@ syscall_test(test = "//test/syscalls/linux:sendfile_test") syscall_test(test = "//test/syscalls/linux:sigaction_test") -# TODO: Enable once the test passes in runsc. +# TODO(b/119826902): Enable once the test passes in runsc. # syscall_test(test = "//test/syscalls/linux:sigaltstack_test") syscall_test(test = "//test/syscalls/linux:sigiret_test") @@ -414,7 +414,7 @@ syscall_test( ) syscall_test( - # NOTE: Large sendmsg may stall a long time. + # NOTE(b/116636318): Large sendmsg may stall a long time. size = "enormous", test = "//test/syscalls/linux:socket_unix_dgram_local_test", ) @@ -437,7 +437,7 @@ syscall_test( ) syscall_test( - # NOTE: Large sendmsg may stall a long time. + # NOTE(b/116636318): Large sendmsg may stall a long time. size = "enormous", test = "//test/syscalls/linux:socket_unix_seqpacket_local_test", ) diff --git a/test/syscalls/build_defs.bzl b/test/syscalls/build_defs.bzl index 610b030b2..cd74a769d 100644 --- a/test/syscalls/build_defs.bzl +++ b/test/syscalls/build_defs.bzl @@ -78,10 +78,10 @@ def _syscall_test( tags += [full_platform, "file_" + file_access] # Add tag to prevent the tests from running in a Bazel sandbox. - # TODO: Make the tests run without this tag. + # TODO(b/120560048): Make the tests run without this tag. tags.append("no-sandbox") - # TODO: KVM tests are tagged "manual" to until the platform is + # TODO(b/112165693): KVM tests are tagged "manual" to until the platform is # more stable. if platform == "kvm": tags += ["manual"] diff --git a/test/syscalls/linux/32bit.cc b/test/syscalls/linux/32bit.cc index 230648c9b..78baf548e 100644 --- a/test/syscalls/linux/32bit.cc +++ b/test/syscalls/linux/32bit.cc @@ -80,11 +80,11 @@ constexpr int kExitCode = 42; TEST(Syscall32Bit, Int80) { switch (GvisorPlatform()) { case Platform::kKVM: - // TODO: 32-bit segments are broken (but not explictly + // TODO(b/111805002): 32-bit segments are broken (but not explictly // disabled). return; case Platform::kPtrace: - // TODO: The ptrace platform does not have a + // TODO(gvisor.dev/issue/167): The ptrace platform does not have a // consistent story here. return; case Platform::kNative: @@ -99,10 +99,10 @@ TEST(Syscall32Bit, Int80) { TEST(Syscall32Bit, Sysenter) { switch (GvisorPlatform()) { case Platform::kKVM: - // TODO: See above. + // TODO(b/111805002): See above. return; case Platform::kPtrace: - // TODO: See above. + // TODO(gvisor.dev/issue/167): See above. return; case Platform::kNative: break; @@ -123,10 +123,10 @@ TEST(Syscall32Bit, Sysenter) { TEST(Syscall32Bit, Syscall) { switch (GvisorPlatform()) { case Platform::kKVM: - // TODO: See above. + // TODO(b/111805002): See above. return; case Platform::kPtrace: - // TODO: See above. + // TODO(gvisor.dev/issue/167): See above. return; case Platform::kNative: break; @@ -207,7 +207,7 @@ void FarCall32() { TEST(Call32Bit, Disallowed) { switch (GvisorPlatform()) { case Platform::kKVM: - // TODO: See above. + // TODO(b/111805002): See above. return; case Platform::kPtrace: // The ptrace platform cannot prevent switching to compatibility mode. diff --git a/test/syscalls/linux/aio.cc b/test/syscalls/linux/aio.cc index 06643ccb8..b96aab9b9 100644 --- a/test/syscalls/linux/aio.cc +++ b/test/syscalls/linux/aio.cc @@ -103,7 +103,7 @@ TEST_F(AIOTest, BasicWrite) { // aio implementation uses aio_ring. gVisor doesn't and returns all zeroes. // Linux implements aio_ring, so skip the zeroes check. // - // TODO: Remove when gVisor implements aio_ring. + // TODO(b/65486370): Remove when gVisor implements aio_ring. auto ring = reinterpret_cast(ctx_); auto magic = IsRunningOnGvisor() ? 0 : AIO_RING_MAGIC; EXPECT_EQ(ring->magic, magic); diff --git a/test/syscalls/linux/chmod.cc b/test/syscalls/linux/chmod.cc index 2f2ff3b7d..2f42fe326 100644 --- a/test/syscalls/linux/chmod.cc +++ b/test/syscalls/linux/chmod.cc @@ -235,7 +235,7 @@ TEST(ChmodTest, FchmodFileToNoPermissionsSucceeds_NoRandomSave) { // Verify that we can get a RW FD after chmod, even if a RO fd is left open. TEST(ChmodTest, ChmodWritableWithOpenFD) { - // FIXME: broken on hostfs. + // FIXME(b/72455313): broken on hostfs. if (IsRunningOnGvisor()) { return; } diff --git a/test/syscalls/linux/epoll.cc b/test/syscalls/linux/epoll.cc index 7b1d83ad8..b4a3bfcba 100644 --- a/test/syscalls/linux/epoll.cc +++ b/test/syscalls/linux/epoll.cc @@ -56,7 +56,7 @@ TEST(EpollTest, AllWritable) { struct epoll_event result[kFDsPerEpoll]; ASSERT_THAT(RetryEINTR(epoll_wait)(epollfd.get(), result, kFDsPerEpoll, -1), SyscallSucceedsWithValue(kFDsPerEpoll)); - // TODO: Why do some tests check epoll_event::data, and others + // TODO(edahlgren): Why do some tests check epoll_event::data, and others // don't? Does Linux actually guarantee that, in any of these test cases, // epoll_wait will necessarily write out the epoll_events in the order that // they were registered? diff --git a/test/syscalls/linux/exec_binary.cc b/test/syscalls/linux/exec_binary.cc index 187696ed9..c10d85398 100644 --- a/test/syscalls/linux/exec_binary.cc +++ b/test/syscalls/linux/exec_binary.cc @@ -285,7 +285,7 @@ ElfBinary<64> StandardElf() { elf.header.e_phoff = sizeof(elf.header); elf.header.e_phentsize = sizeof(decltype(elf)::ElfPhdr); - // TODO: Always include a PT_GNU_STACK segment to + // TODO(gvisor.dev/issue/153): Always include a PT_GNU_STACK segment to // disable executable stacks. With this omitted the stack (and all PROT_READ) // mappings should be executable, but gVisor doesn't support that. decltype(elf)::ElfPhdr phdr = {}; @@ -403,7 +403,7 @@ TEST(ElfTest, DataSegment) { // Linux will allow PT_LOAD segments to overlap. TEST(ElfTest, DirectlyOverlappingSegments) { - // NOTE: see PIEOutOfOrderSegments. + // NOTE(b/37289926): see PIEOutOfOrderSegments. SKIP_IF(IsRunningOnGvisor()); ElfBinary<64> elf = StandardElf(); @@ -439,7 +439,7 @@ TEST(ElfTest, DirectlyOverlappingSegments) { // Linux allows out-of-order PT_LOAD segments. TEST(ElfTest, OutOfOrderSegments) { - // NOTE: see PIEOutOfOrderSegments. + // NOTE(b/37289926): see PIEOutOfOrderSegments. SKIP_IF(IsRunningOnGvisor()); ElfBinary<64> elf = StandardElf(); @@ -670,7 +670,7 @@ TEST(ElfTest, PIENonZeroStart) { } TEST(ElfTest, PIEOutOfOrderSegments) { - // TODO: This triggers a bug in Linux where it computes the size + // TODO(b/37289926): This triggers a bug in Linux where it computes the size // of the binary as 0x20000 - 0x40000 = 0xfffffffffffe0000, which obviously // fails to map. // @@ -1005,7 +1005,7 @@ TEST(ElfTest, NoExecute) { // Execute, but no read permissions on the binary works just fine. TEST(ElfTest, NoRead) { - // TODO: gVisor's backing filesystem may prevent the + // TODO(gvisor.dev/issue/160): gVisor's backing filesystem may prevent the // sentry from reading the executable. SKIP_IF(IsRunningOnGvisor()); @@ -1024,7 +1024,7 @@ TEST(ElfTest, NoRead) { ASSERT_NO_ERRNO(WaitStopped(child)); - // TODO: A task with a non-readable executable is marked + // TODO(gvisor.dev/issue/160): A task with a non-readable executable is marked // non-dumpable, preventing access to proc files. gVisor does not implement // this behavior. } diff --git a/test/syscalls/linux/file_base.h b/test/syscalls/linux/file_base.h index 19c9a5053..43f568111 100644 --- a/test/syscalls/linux/file_base.h +++ b/test/syscalls/linux/file_base.h @@ -52,7 +52,7 @@ class FileTest : public ::testing::Test { test_file_fd_ = ASSERT_NO_ERRNO_AND_VALUE( Open(test_file_name_, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR)); - // FIXME: enable when mknod syscall is supported. + // FIXME(edahlgren): enable when mknod syscall is supported. // test_fifo_name_ = NewTempAbsPath(); // ASSERT_THAT(mknod(test_fifo_name_.c_str()), S_IFIFO|0644, 0, // SyscallSucceeds()); @@ -97,7 +97,7 @@ class FileTest : public ::testing::Test { UnlinkFile(); ClosePipes(); - // FIXME: enable when mknod syscall is supported. + // FIXME(edahlgren): enable when mknod syscall is supported. // close(test_fifo_[0]); // close(test_fifo_[1]); // unlink(test_fifo_name_.c_str()); diff --git a/test/syscalls/linux/ioctl.cc b/test/syscalls/linux/ioctl.cc index de29047e0..c7741a177 100644 --- a/test/syscalls/linux/ioctl.cc +++ b/test/syscalls/linux/ioctl.cc @@ -158,7 +158,7 @@ TEST_F(IoctlTest, FIOASYNCNoTarget) { } TEST_F(IoctlTest, FIOASYNCSelfTarget) { - // FIXME: gVisor erroneously sends SIGIO on close(2), which would + // FIXME(b/120624367): gVisor erroneously sends SIGIO on close(2), which would // kill the test when pair goes out of scope. Temporarily ignore SIGIO so that // that the close signal is ignored. struct sigaction sa; @@ -195,7 +195,7 @@ TEST_F(IoctlTest, FIOASYNCSelfTarget) { // Equivalent to FIOASYNCSelfTarget except that FIOSETOWN is called before // FIOASYNC. TEST_F(IoctlTest, FIOASYNCSelfTarget2) { - // FIXME: gVisor erroneously sends SIGIO on close(2), which would + // FIXME(b/120624367): gVisor erroneously sends SIGIO on close(2), which would // kill the test when pair goes out of scope. Temporarily ignore SIGIO so that // that the close signal is ignored. struct sigaction sa; diff --git a/test/syscalls/linux/ip_socket_test_util.cc b/test/syscalls/linux/ip_socket_test_util.cc index 4ad787cc0..0a149c2e5 100644 --- a/test/syscalls/linux/ip_socket_test_util.cc +++ b/test/syscalls/linux/ip_socket_test_util.cc @@ -24,7 +24,7 @@ namespace gvisor { namespace testing { PosixErrorOr InterfaceIndex(std::string name) { - // TODO: Consider using netlink. + // TODO(igudger): Consider using netlink. ifreq req = {}; memcpy(req.ifr_name, name.c_str(), name.size()); ASSIGN_OR_RETURN_ERRNO(auto sock, Socket(AF_INET, SOCK_DGRAM, 0)); diff --git a/test/syscalls/linux/lseek.cc b/test/syscalls/linux/lseek.cc index fb6a1546e..6a4f1423c 100644 --- a/test/syscalls/linux/lseek.cc +++ b/test/syscalls/linux/lseek.cc @@ -194,7 +194,7 @@ TEST(LseekTest, EtcPasswdDup) { ASSERT_THAT(lseek(fd3.get(), 0, SEEK_CUR), SyscallSucceedsWithValue(1000)); } -// TODO: Add tests where we have donated in sockets. +// TODO(magi): Add tests where we have donated in sockets. } // namespace diff --git a/test/syscalls/linux/mkdir.cc b/test/syscalls/linux/mkdir.cc index 84db45eb3..50807b68f 100644 --- a/test/syscalls/linux/mkdir.cc +++ b/test/syscalls/linux/mkdir.cc @@ -36,7 +36,7 @@ class MkdirTest : public ::testing::Test { // TearDown unlinks created files. void TearDown() override { - // FIXME: We don't currently implement rmdir. + // FIXME(edahlgren): We don't currently implement rmdir. // We do this unconditionally because there's no harm in trying. rmdir(dirname_.c_str()); } diff --git a/test/syscalls/linux/mmap.cc b/test/syscalls/linux/mmap.cc index b500e79a4..a4fb9d1e0 100644 --- a/test/syscalls/linux/mmap.cc +++ b/test/syscalls/linux/mmap.cc @@ -816,7 +816,7 @@ class MMapFileTest : public MMapTest { // MAP_POPULATE allowed. // There isn't a good way to verify it actually did anything. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, MapPopulate) { ASSERT_THAT( Map(0, kPageSize, PROT_READ, MAP_PRIVATE | MAP_POPULATE, fd_.get(), 0), @@ -825,7 +825,7 @@ TEST_F(MMapFileTest, MapPopulate) { // MAP_POPULATE on a short file. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, MapPopulateShort) { ASSERT_THAT(Map(0, 2 * kPageSize, PROT_READ, MAP_PRIVATE | MAP_POPULATE, fd_.get(), 0), @@ -923,7 +923,7 @@ TEST_F(MMapFileTest, WriteSharedOnReadOnlyFd) { // MAP_SHARED PROT_READ not allowed on write-only FDs. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, ReadSharedOnWriteOnlyFd) { const FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(Open(filename_, O_WRONLY)); @@ -936,7 +936,7 @@ TEST_F(MMapFileTest, ReadSharedOnWriteOnlyFd) { // MAP_SHARED PROT_WRITE not allowed on write-only FDs. // The FD must always be readable. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, WriteSharedOnWriteOnlyFd) { const FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(Open(filename_, O_WRONLY)); @@ -1371,7 +1371,7 @@ TEST_F(MMapFileTest, WritePrivate) { // SIGBUS raised when writing past end of file to a private mapping. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, SigBusDeathWritePrivate) { SetupGvisorDeathTest(); @@ -1390,7 +1390,7 @@ TEST_F(MMapFileTest, SigBusDeathWritePrivate) { // SIGBUS raised when reading past end of file on a shared mapping. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, SigBusDeathReadShared) { SetupGvisorDeathTest(); @@ -1410,7 +1410,7 @@ TEST_F(MMapFileTest, SigBusDeathReadShared) { // SIGBUS raised when reading past end of file on a shared mapping. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, SigBusDeathWriteShared) { SetupGvisorDeathTest(); @@ -1459,7 +1459,7 @@ TEST_F(MMapFileTest, NoSigBusOnPageContainingEOFWritePrivate) { // Tests that SIGBUS is not raised when reading from a file-mapped page // containing EOF, *after* the EOF for a shared mapping. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, NoSigBusOnPageContainingEOFReadShared) { uintptr_t addr; ASSERT_THAT(addr = Map(0, 2 * kPageSize, PROT_READ, MAP_SHARED, fd_.get(), 0), @@ -1476,7 +1476,7 @@ TEST_F(MMapFileTest, NoSigBusOnPageContainingEOFReadShared) { // Tests that SIGBUS is not raised when writing to a file-mapped page containing // EOF, *after* the EOF for a shared mapping. // -// FIXME: Parameterize. +// FIXME(b/37222275): Parameterize. TEST_F(MMapFileTest, NoSigBusOnPageContainingEOFWriteShared) { uintptr_t addr; ASSERT_THAT(addr = Map(0, 2 * kPageSize, PROT_READ | PROT_WRITE, MAP_SHARED, diff --git a/test/syscalls/linux/open.cc b/test/syscalls/linux/open.cc index cdc226300..22e4666c2 100644 --- a/test/syscalls/linux/open.cc +++ b/test/syscalls/linux/open.cc @@ -279,7 +279,7 @@ TEST_F(OpenTest, Null) { ASSERT_THAT(open(&c, O_RDONLY), SyscallFailsWithErrno(ENOENT)); } -// NOTE: While the man pages specify that this behavior should be +// NOTE(b/119785738): While the man pages specify that this behavior should be // undefined, Linux truncates the file on opening read only if we have write // permission, so we will too. TEST_F(OpenTest, CanTruncateReadOnly) { diff --git a/test/syscalls/linux/partial_bad_buffer.cc b/test/syscalls/linux/partial_bad_buffer.cc index 073a6b8c1..71288ebc4 100644 --- a/test/syscalls/linux/partial_bad_buffer.cc +++ b/test/syscalls/linux/partial_bad_buffer.cc @@ -158,7 +158,7 @@ TEST_F(PartialBadBufferTest, PreadvSmall) { } TEST_F(PartialBadBufferTest, WriteBig) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); @@ -168,7 +168,7 @@ TEST_F(PartialBadBufferTest, WriteBig) { } TEST_F(PartialBadBufferTest, WriteSmall) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); @@ -178,7 +178,7 @@ TEST_F(PartialBadBufferTest, WriteSmall) { } TEST_F(PartialBadBufferTest, PwriteBig) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); @@ -188,7 +188,7 @@ TEST_F(PartialBadBufferTest, PwriteBig) { } TEST_F(PartialBadBufferTest, PwriteSmall) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); @@ -198,7 +198,7 @@ TEST_F(PartialBadBufferTest, PwriteSmall) { } TEST_F(PartialBadBufferTest, WritevBig) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); @@ -211,7 +211,7 @@ TEST_F(PartialBadBufferTest, WritevBig) { } TEST_F(PartialBadBufferTest, WritevSmall) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); @@ -224,7 +224,7 @@ TEST_F(PartialBadBufferTest, WritevSmall) { } TEST_F(PartialBadBufferTest, PwritevBig) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); @@ -238,7 +238,7 @@ TEST_F(PartialBadBufferTest, PwritevBig) { } TEST_F(PartialBadBufferTest, PwritevSmall) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); @@ -279,7 +279,7 @@ TEST_F(PartialBadBufferTest, GetdentsOneEntry) { // Verify that when write returns EFAULT the kernel hasn't silently written // the initial valid bytes. TEST_F(PartialBadBufferTest, WriteEfaultIsntPartial) { - // FIXME: The sentry write syscalls will return immediately + // FIXME(b/24788078): The sentry write syscalls will return immediately // if Access returns an error, but Access may not return an error // and the sentry will instead perform a partial write. SKIP_IF(IsRunningOnGvisor()); diff --git a/test/syscalls/linux/pipe.cc b/test/syscalls/linux/pipe.cc index c49ec9f09..abd10b11b 100644 --- a/test/syscalls/linux/pipe.cc +++ b/test/syscalls/linux/pipe.cc @@ -36,7 +36,7 @@ namespace { // Buffer size of a pipe. // -// TODO: Get this from F_GETPIPE_SZ. +// TODO(b/35762278): Get this from F_GETPIPE_SZ. constexpr int kPipeSize = 65536; class PipeTest : public ::testing::Test { @@ -316,7 +316,7 @@ TEST_F(PipeTest, BlockWriteClosed) { // Blocking write returns EPIPE when read end is closed even if something has // been written. // -// FIXME: Pipe writes blocking early allows S/R to interrupt the +// FIXME(b/35924046): Pipe writes blocking early allows S/R to interrupt the // write(2) call before the buffer is full. Then the next call will will return // non-zero instead of EPIPE. TEST_F(PipeTest, BlockPartialWriteClosed_NoRandomSave) { @@ -329,7 +329,7 @@ TEST_F(PipeTest, BlockPartialWriteClosed_NoRandomSave) { // Write more than fits in the buffer. Blocks then returns partial write // when the other end is closed. The next call returns EPIPE. if (IsRunningOnGvisor()) { - // FIXME: Pipe writes block early on gVisor, resulting in a + // FIXME(b/35924046): Pipe writes block early on gVisor, resulting in a // shorter than expected partial write. ASSERT_THAT(write(wfd, buf.data(), buf.size()), SyscallSucceedsWithValue(::testing::Gt(0))); diff --git a/test/syscalls/linux/proc.cc b/test/syscalls/linux/proc.cc index 3ec31ae8b..7ba274226 100644 --- a/test/syscalls/linux/proc.cc +++ b/test/syscalls/linux/proc.cc @@ -61,7 +61,7 @@ #include "test/util/thread_util.h" #include "test/util/timer_util.h" -// NOTE: No, this isn't really a syscall but this is a really simple +// NOTE(magi): No, this isn't really a syscall but this is a really simple // way to get it tested on both gVisor, PTrace and Linux. using ::testing::AllOf; @@ -489,7 +489,7 @@ TEST(ProcSelfMaps, Map1) { } TEST(ProcSelfMaps, Map2) { - // NOTE: The permissions must be different or the pages will get merged. + // NOTE(magi): The permissions must be different or the pages will get merged. Mapping map1 = ASSERT_NO_ERRNO_AND_VALUE( MmapAnon(kPageSize, PROT_READ | PROT_EXEC, MAP_PRIVATE)); Mapping map2 = @@ -564,7 +564,7 @@ TEST(ProcSelfMaps, MapUnmap) { } TEST(ProcSelfMaps, Mprotect) { - // FIXME: Linux's mprotect() sometimes fails to merge VMAs in this + // FIXME(jamieliu): Linux's mprotect() sometimes fails to merge VMAs in this // case. SKIP_IF(!IsRunningOnGvisor()); @@ -977,7 +977,7 @@ void MapPopulateRSS(int prot, uint64_t* before, uint64_t* after) { *after = ASSERT_NO_ERRNO_AND_VALUE(CurrentRSS()); } -// TODO: Test for PROT_READ + MAP_POPULATE anonymous mappings. Their +// TODO(b/73896574): Test for PROT_READ + MAP_POPULATE anonymous mappings. Their // semantics are more subtle: // // Small pages -> Zero page mapped, not counted in RSS @@ -1140,7 +1140,7 @@ TEST(ProcPidStatusTest, ValuesAreTabDelimited) { // Threads properly counts running threads. // -// TODO: Test zombied threads while the thread group leader is still +// TODO(mpratt): Test zombied threads while the thread group leader is still // running with generalized fork and clone children from the wait test. TEST(ProcPidStatusTest, Threads) { char buf[4096] = {}; @@ -1274,7 +1274,7 @@ TEST(ProcPidSymlink, SubprocessRunning) { SyscallSucceedsWithValue(sizeof(buf))); } -// FIXME: Inconsistent behavior between gVisor and linux +// FIXME(gvisor.dev/issue/164): Inconsistent behavior between gVisor and linux // on proc files. TEST(ProcPidSymlink, SubprocessZombied) { ASSERT_NO_ERRNO(SetCapability(CAP_DAC_OVERRIDE, false)); @@ -1298,13 +1298,13 @@ TEST(ProcPidSymlink, SubprocessZombied) { SyscallFailsWithErrno(want)); } - // FIXME: Inconsistent behavior between gVisor and linux + // FIXME(gvisor.dev/issue/164): Inconsistent behavior between gVisor and linux // on proc files. // 4.17 & gVisor: Syscall succeeds and returns 1 // EXPECT_THAT(ReadlinkWhileZombied("ns/pid", buf, sizeof(buf)), // SyscallFailsWithErrno(EACCES)); - // FIXME: Inconsistent behavior between gVisor and linux + // FIXME(gvisor.dev/issue/164): Inconsistent behavior between gVisor and linux // on proc files. // 4.17 & gVisor: Syscall succeeds and returns 1. // EXPECT_THAT(ReadlinkWhileZombied("ns/user", buf, sizeof(buf)), @@ -1313,7 +1313,7 @@ TEST(ProcPidSymlink, SubprocessZombied) { // Test whether /proc/PID/ symlinks can be read for an exited process. TEST(ProcPidSymlink, SubprocessExited) { - // FIXME: These all succeed on gVisor. + // FIXME(gvisor.dev/issue/164): These all succeed on gVisor. SKIP_IF(IsRunningOnGvisor()); char buf[1]; @@ -1404,7 +1404,7 @@ TEST(ProcPidFile, SubprocessZombie) { EXPECT_THAT(ReadWhileZombied("uid_map", buf, sizeof(buf)), SyscallSucceedsWithValue(sizeof(buf))); - // FIXME: Inconsistent behavior between gVisor and linux + // FIXME(gvisor.dev/issue/164): Inconsistent behavior between gVisor and linux // on proc files. // gVisor & 4.17: Succeeds and returns 1. // EXPECT_THAT(ReadWhileZombied("io", buf, sizeof(buf)), @@ -1415,7 +1415,7 @@ TEST(ProcPidFile, SubprocessZombie) { TEST(ProcPidFile, SubprocessExited) { char buf[1]; - // FIXME: Inconsistent behavior between kernels + // FIXME(gvisor.dev/issue/164): Inconsistent behavior between kernels // gVisor: Fails with ESRCH. // 4.17: Succeeds and returns 1. // EXPECT_THAT(ReadWhileExited("auxv", buf, sizeof(buf)), @@ -1425,7 +1425,7 @@ TEST(ProcPidFile, SubprocessExited) { SyscallFailsWithErrno(ESRCH)); if (!IsRunningOnGvisor()) { - // FIXME: Succeeds on gVisor. + // FIXME(gvisor.dev/issue/164): Succeeds on gVisor. EXPECT_THAT(ReadWhileExited("comm", buf, sizeof(buf)), SyscallFailsWithErrno(ESRCH)); } @@ -1434,25 +1434,25 @@ TEST(ProcPidFile, SubprocessExited) { SyscallSucceedsWithValue(sizeof(buf))); if (!IsRunningOnGvisor()) { - // FIXME: Succeeds on gVisor. + // FIXME(gvisor.dev/issue/164): Succeeds on gVisor. EXPECT_THAT(ReadWhileExited("io", buf, sizeof(buf)), SyscallFailsWithErrno(ESRCH)); } if (!IsRunningOnGvisor()) { - // FIXME: Returns EOF on gVisor. + // FIXME(gvisor.dev/issue/164): Returns EOF on gVisor. EXPECT_THAT(ReadWhileExited("maps", buf, sizeof(buf)), SyscallFailsWithErrno(ESRCH)); } if (!IsRunningOnGvisor()) { - // FIXME: Succeeds on gVisor. + // FIXME(gvisor.dev/issue/164): Succeeds on gVisor. EXPECT_THAT(ReadWhileExited("stat", buf, sizeof(buf)), SyscallFailsWithErrno(ESRCH)); } if (!IsRunningOnGvisor()) { - // FIXME: Succeeds on gVisor. + // FIXME(gvisor.dev/issue/164): Succeeds on gVisor. EXPECT_THAT(ReadWhileExited("status", buf, sizeof(buf)), SyscallFailsWithErrno(ESRCH)); } diff --git a/test/syscalls/linux/proc_pid_smaps.cc b/test/syscalls/linux/proc_pid_smaps.cc index 5f9c42ce5..cf5c462f3 100644 --- a/test/syscalls/linux/proc_pid_smaps.cc +++ b/test/syscalls/linux/proc_pid_smaps.cc @@ -82,7 +82,7 @@ struct ProcPidSmapsEntry { // Given the value part of a /proc/[pid]/smaps field containing a value in kB // (for example, " 4 kB", returns the value in kB (in this example, 4). PosixErrorOr SmapsValueKb(absl::string_view value) { - // TODO: let us use RE2 or + // TODO(jamieliu): let us use RE2 or std::pair parts = absl::StrSplit(value, ' ', absl::SkipEmpty()); if (parts.second != "kB") { diff --git a/test/syscalls/linux/ptrace.cc b/test/syscalls/linux/ptrace.cc index 1c9d7d4f4..e0c56f1fc 100644 --- a/test/syscalls/linux/ptrace.cc +++ b/test/syscalls/linux/ptrace.cc @@ -823,7 +823,7 @@ TEST(PtraceTest, TEST(PtraceTest, Int3) { switch (GvisorPlatform()) { case Platform::kKVM: - // TODO: int3 isn't handled properly. + // TODO(b/124248694): int3 isn't handled properly. return; default: break; diff --git a/test/syscalls/linux/pwrite64.cc b/test/syscalls/linux/pwrite64.cc index 60ae6de1f..485b1e48d 100644 --- a/test/syscalls/linux/pwrite64.cc +++ b/test/syscalls/linux/pwrite64.cc @@ -30,7 +30,7 @@ namespace { // This test is currently very rudimentary. // -// TODO: +// TODO(edahlgren): // * bad buffer states (EFAULT). // * bad fds (wrong permission, wrong type of file, EBADF). // * check offset is not incremented. diff --git a/test/syscalls/linux/readv_socket.cc b/test/syscalls/linux/readv_socket.cc index 2c129b7e8..cf22c395e 100644 --- a/test/syscalls/linux/readv_socket.cc +++ b/test/syscalls/linux/readv_socket.cc @@ -41,7 +41,7 @@ class ReadvSocketTest : public SocketTest { ASSERT_THAT(write(test_unix_seqpacket_socket_[1], kReadvTestData, kReadvTestDataSize), SyscallSucceedsWithValue(kReadvTestDataSize)); - // FIXME: Enable when possible. + // FIXME(b/69821513): Enable when possible. // ASSERT_THAT(write(test_tcp_socket_[1], kReadvTestData, // kReadvTestDataSize), // SyscallSucceedsWithValue(kReadvTestDataSize)); diff --git a/test/syscalls/linux/rtsignal.cc b/test/syscalls/linux/rtsignal.cc index 1f2fed7cc..ff948f9d5 100644 --- a/test/syscalls/linux/rtsignal.cc +++ b/test/syscalls/linux/rtsignal.cc @@ -75,7 +75,7 @@ class RtSignalTest : public ::testing::Test { static int rt_sigqueueinfo(pid_t tgid, int sig, siginfo_t* uinfo) { int ret; do { - // NOTE: rt_sigqueueinfo(2) could return EAGAIN for RT signals. + // NOTE(b/25434735): rt_sigqueueinfo(2) could return EAGAIN for RT signals. ret = syscall(SYS_rt_sigqueueinfo, tgid, sig, uinfo); } while (ret == -1 && errno == EAGAIN); return ret; diff --git a/test/syscalls/linux/socket_inet_loopback.cc b/test/syscalls/linux/socket_inet_loopback.cc index cdc5c0ce8..14d7827c2 100644 --- a/test/syscalls/linux/socket_inet_loopback.cc +++ b/test/syscalls/linux/socket_inet_loopback.cc @@ -221,7 +221,7 @@ TEST_P(SocketInetReusePortTest, TcpPortReuseMultiThread) { std::atomic connects_received = ATOMIC_VAR_INIT(0); std::unique_ptr listen_thread[kThreadCount]; int accept_counts[kThreadCount] = {}; - // TODO: figure how to not disable S/R for the whole test. + // TODO(avagin): figure how to not disable S/R for the whole test. // We need to take into account that this test executes a lot of system // calls from many threads. DisableSave ds; @@ -325,7 +325,7 @@ TEST_P(SocketInetReusePortTest, UdpPortReuseMultiThread) { std::atomic packets_received = ATOMIC_VAR_INIT(0); std::unique_ptr receiver_thread[kThreadCount]; int packets_per_socket[kThreadCount] = {}; - // TODO: figure how to not disable S/R for the whole test. + // TODO(avagin): figure how to not disable S/R for the whole test. DisableSave ds; // Too expensive. for (int i = 0; i < kThreadCount; i++) { @@ -642,7 +642,7 @@ TEST_P(SocketMultiProtocolInetLoopbackTest, V6OnlyV6AnyReservesV6) { TEST_P(SocketMultiProtocolInetLoopbackTest, V6EphemeralPortReserved) { auto const& param = GetParam(); - // FIXME + // FIXME(b/114268588) SKIP_IF(IsRunningOnGvisor() && param.type == SOCK_STREAM); for (int i = 0; true; i++) { @@ -743,7 +743,7 @@ TEST_P(SocketMultiProtocolInetLoopbackTest, V6EphemeralPortReserved) { TEST_P(SocketMultiProtocolInetLoopbackTest, V4MappedEphemeralPortReserved) { auto const& param = GetParam(); - // FIXME + // FIXME(b/114268588) SKIP_IF(IsRunningOnGvisor() && param.type == SOCK_STREAM); for (int i = 0; true; i++) { @@ -867,7 +867,7 @@ TEST_P(SocketMultiProtocolInetLoopbackTest, V4MappedEphemeralPortReserved) { TEST_P(SocketMultiProtocolInetLoopbackTest, V4EphemeralPortReserved) { auto const& param = GetParam(); - // FIXME + // FIXME(b/114268588) SKIP_IF(IsRunningOnGvisor() && param.type == SOCK_STREAM); for (int i = 0; true; i++) { diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc index 8b4fc57b6..9dd9e1bd6 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc @@ -244,7 +244,7 @@ TestAddress V4Multicast() { // set interface or group membership. TEST_P(IPv4UDPUnboundExternalNetworkingSocketTest, TestSendMulticastSelfNoGroup) { - // FIXME: A group membership is not required for external + // FIXME(b/125485338): A group membership is not required for external // multicast on gVisor. SKIP_IF(IsRunningOnGvisor()); @@ -371,7 +371,7 @@ TEST_P(IPv4UDPUnboundExternalNetworkingSocketTest, // Check that multicast packets won't be delivered to another socket with no // set interface or group membership. TEST_P(IPv4UDPUnboundExternalNetworkingSocketTest, TestSendMulticastNoGroup) { - // FIXME: A group membership is not required for external + // FIXME(b/125485338): A group membership is not required for external // multicast on gVisor. SKIP_IF(IsRunningOnGvisor()); diff --git a/test/syscalls/linux/socket_netlink_route.cc b/test/syscalls/linux/socket_netlink_route.cc index 8d2e7d333..ed4ae1c71 100644 --- a/test/syscalls/linux/socket_netlink_route.cc +++ b/test/syscalls/linux/socket_netlink_route.cc @@ -180,7 +180,7 @@ void CheckGetLinkResponse(const struct nlmsghdr* hdr, int seq, int port) { // RTM_NEWLINK contains at least the header and ifinfomsg. EXPECT_GE(hdr->nlmsg_len, NLMSG_SPACE(sizeof(struct ifinfomsg))); - // TODO: Check ifinfomsg contents and following attrs. + // TODO(mpratt): Check ifinfomsg contents and following attrs. } TEST(NetlinkRouteTest, GetLinkDump) { @@ -370,7 +370,7 @@ TEST(NetlinkRouteTest, GetAddrDump) { // RTM_NEWADDR contains at least the header and ifaddrmsg. EXPECT_GE(hdr->nlmsg_len, sizeof(*hdr) + sizeof(struct ifaddrmsg)); - // TODO: Check ifaddrmsg contents and following attrs. + // TODO(mpratt): Check ifaddrmsg contents and following attrs. })); } diff --git a/test/syscalls/linux/socket_stream_blocking.cc b/test/syscalls/linux/socket_stream_blocking.cc index 8b3f6a647..f0f86c01c 100644 --- a/test/syscalls/linux/socket_stream_blocking.cc +++ b/test/syscalls/linux/socket_stream_blocking.cc @@ -33,7 +33,7 @@ namespace gvisor { namespace testing { TEST_P(BlockingStreamSocketPairTest, BlockPartialWriteClosed) { - // FIXME: gVisor doesn't support SO_SNDBUF on UDS, nor does it + // FIXME(b/35921550): gVisor doesn't support SO_SNDBUF on UDS, nor does it // enforce any limit; it will write arbitrary amounts of data without // blocking. SKIP_IF(IsRunningOnGvisor()); diff --git a/test/syscalls/linux/socket_test_util.cc b/test/syscalls/linux/socket_test_util.cc index 035087566..0be23e541 100644 --- a/test/syscalls/linux/socket_test_util.cc +++ b/test/syscalls/linux/socket_test_util.cc @@ -353,7 +353,7 @@ PosixErrorOr> CreateTCPAcceptBindSocketPair( } MaybeSave(); // Successful accept. - // FIXME + // FIXME(b/110484944) if (connect_result == -1) { absl::SleepFor(absl::Seconds(1)); } diff --git a/test/syscalls/linux/socket_unix.cc b/test/syscalls/linux/socket_unix.cc index 7332b768e..fafb23ad1 100644 --- a/test/syscalls/linux/socket_unix.cc +++ b/test/syscalls/linux/socket_unix.cc @@ -186,7 +186,7 @@ TEST_P(UnixSocketPairTest, BasicFDPassNoSpace) { // BasicFDPassNoSpaceMsgCtrunc sends an FD, but does not provide any space to // receive it. It then verifies that the MSG_CTRUNC flag is set in the msghdr. TEST_P(UnixSocketPairTest, BasicFDPassNoSpaceMsgCtrunc) { - // FIXME: Support MSG_CTRUNC. + // FIXME(gvisor.dev/issue/206): Support MSG_CTRUNC. SKIP_IF(IsRunningOnGvisor()); auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); @@ -224,7 +224,7 @@ TEST_P(UnixSocketPairTest, BasicFDPassNoSpaceMsgCtrunc) { // accomidate the FD, but msg_control is set to NULL. In this case, msg_control // should override msg_controllen. TEST_P(UnixSocketPairTest, BasicFDPassNullControlMsgCtrunc) { - // FIXME: Fix handling of NULL msg_control. + // FIXME(gvisor.dev/issue/207): Fix handling of NULL msg_control. SKIP_IF(IsRunningOnGvisor()); auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); @@ -259,7 +259,7 @@ TEST_P(UnixSocketPairTest, BasicFDPassNullControlMsgCtrunc) { // space to receive it. It then verifies that the MSG_CTRUNC flag is set in the // msghdr. TEST_P(UnixSocketPairTest, BasicFDPassNotEnoughSpaceMsgCtrunc) { - // FIXME: Support MSG_CTRUNC. + // FIXME(gvisor.dev/issue/206): Support MSG_CTRUNC. SKIP_IF(IsRunningOnGvisor()); auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); @@ -296,7 +296,7 @@ TEST_P(UnixSocketPairTest, BasicFDPassNotEnoughSpaceMsgCtrunc) { // space to receive two of them. It then verifies that the MSG_CTRUNC flag is // set in the msghdr. TEST_P(UnixSocketPairTest, BasicThreeFDPassTruncationMsgCtrunc) { - // FIXME: Support MSG_CTRUNC. + // FIXME(gvisor.dev/issue/206): Support MSG_CTRUNC. SKIP_IF(IsRunningOnGvisor()); auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); @@ -408,7 +408,7 @@ TEST_P(UnixSocketPairTest, BasicFDPassUnalignedRecvNoMsgTrunc) { // provides enough space to receive one of them. It then verifies that the // MSG_CTRUNC flag is set in the msghdr. TEST_P(UnixSocketPairTest, BasicTwoFDPassUnalignedRecvTruncationMsgTrunc) { - // FIXME: Support MSG_CTRUNC. + // FIXME(gvisor.dev/issue/206): Support MSG_CTRUNC. SKIP_IF(IsRunningOnGvisor()); auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); @@ -1010,7 +1010,7 @@ TEST_P(UnixSocketPairTest, CredPassNoMsgCtrunc) { // the data without providing space for any credentials and verifies that // MSG_CTRUNC is set in the msghdr. TEST_P(UnixSocketPairTest, CredPassNoSpaceMsgCtrunc) { - // FIXME: Support MSG_CTRUNC. + // FIXME(gvisor.dev/issue/206): Support MSG_CTRUNC. SKIP_IF(IsRunningOnGvisor()); auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); @@ -1061,7 +1061,7 @@ TEST_P(UnixSocketPairTest, CredPassNoSpaceMsgCtrunc) { // the data while providing enough space for only the first field of the // credentials and verifies that MSG_CTRUNC is set in the msghdr. TEST_P(UnixSocketPairTest, CredPassTruncatedMsgCtrunc) { - // FIXME: Support MSG_CTRUNC. + // FIXME(gvisor.dev/issue/206): Support MSG_CTRUNC. SKIP_IF(IsRunningOnGvisor()); auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); @@ -1615,7 +1615,7 @@ TEST_P(UnixSocketPairTest, SocketShutdown) { } TEST_P(UnixSocketPairTest, SocketReopenFromProcfs) { - // TODO: We should be returning ENXIO and NOT EIO. + // TODO(b/122310852): We should be returning ENXIO and NOT EIO. SKIP_IF(IsRunningOnGvisor()); auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair()); diff --git a/test/syscalls/linux/socket_unix_dgram.cc b/test/syscalls/linux/socket_unix_dgram.cc index c17d3990f..5dd5e6d77 100644 --- a/test/syscalls/linux/socket_unix_dgram.cc +++ b/test/syscalls/linux/socket_unix_dgram.cc @@ -28,7 +28,7 @@ namespace testing { namespace { TEST_P(DgramUnixSocketPairTest, WriteOneSideClosed) { - // FIXME: gVisor datagram sockets return EPIPE instead of + // FIXME(b/35925052): gVisor datagram sockets return EPIPE instead of // ECONNREFUSED. SKIP_IF(IsRunningOnGvisor()); diff --git a/test/syscalls/linux/socket_unix_dgram_non_blocking.cc b/test/syscalls/linux/socket_unix_dgram_non_blocking.cc index 460eb8320..3becb513d 100644 --- a/test/syscalls/linux/socket_unix_dgram_non_blocking.cc +++ b/test/syscalls/linux/socket_unix_dgram_non_blocking.cc @@ -31,7 +31,7 @@ using NonBlockingDgramUnixSocketPairTest = SocketPairTest; TEST_P(NonBlockingDgramUnixSocketPairTest, ReadOneSideClosed) { if (IsRunningOnGvisor()) { - // FIXME: gVisor datagram sockets return 0 instead of + // FIXME(b/70803293): gVisor datagram sockets return 0 instead of // EAGAIN. return; } diff --git a/test/syscalls/linux/socket_unix_non_stream.cc b/test/syscalls/linux/socket_unix_non_stream.cc index 8e0cbee4c..a565978f9 100644 --- a/test/syscalls/linux/socket_unix_non_stream.cc +++ b/test/syscalls/linux/socket_unix_non_stream.cc @@ -47,7 +47,7 @@ TEST_P(UnixNonStreamSocketPairTest, RecvMsgTooLarge) { const int ret = RetryEINTR(write)(sockets->second_fd(), write_buf.data(), write_buf.size()); if (ret < 0 && errno == ENOBUFS) { - // NOTE: Linux may stall the write for a long time and + // NOTE(b/116636318): Linux may stall the write for a long time and // ultimately return ENOBUFS. Allow this error, since a retry will likely // result in the same error. return; @@ -136,7 +136,7 @@ TEST_P(UnixNonStreamSocketPairTest, FragmentedSendMsg) { // N.B. At minimum, the socketpair gofer should provide a socket that is // already the correct size. // - // TODO: When internal UDS support SO_SNDBUF, we can assert that + // TODO(b/35921550): When internal UDS support SO_SNDBUF, we can assert that // we always get the right SO_SNDBUF on gVisor. GTEST_SKIP() << "SO_SNDBUF = " << actual_sndbuf << ", want " << sndbuf; } @@ -156,7 +156,7 @@ TEST_P(UnixNonStreamSocketPairTest, FragmentedSendMsg) { msg.msg_iov = &iov; msg.msg_iovlen = 1; - // NOTE: Linux has poor behavior in the presence of + // NOTE(b/116636318,b/115833655): Linux has poor behavior in the presence of // physical memory fragmentation. As a result, this may stall for a long time // and ultimately return ENOBUFS. Allow this error, since it means that we // made it to the host kernel and started the sendmsg. @@ -192,7 +192,7 @@ TEST_P(UnixNonStreamSocketPairTest, FragmentedRecvMsg) { // N.B. At minimum, the socketpair gofer should provide a socket that is // already the correct size. // - // TODO: When internal UDS support SO_SNDBUF, we can assert that + // TODO(b/35921550): When internal UDS support SO_SNDBUF, we can assert that // we always get the right SO_SNDBUF on gVisor. GTEST_SKIP() << "SO_SNDBUF = " << actual_sndbuf << ", want " << sndbuf; } @@ -201,7 +201,7 @@ TEST_P(UnixNonStreamSocketPairTest, FragmentedRecvMsg) { const int ret = RetryEINTR(write)(sockets->first_fd(), write_buf.data(), write_buf.size()); if (ret < 0 && errno == ENOBUFS) { - // NOTE: Linux may stall the write for a long time and + // NOTE(b/116636318): Linux may stall the write for a long time and // ultimately return ENOBUFS. Allow this error, since a retry will likely // result in the same error. return; diff --git a/test/syscalls/linux/socket_unix_unbound_seqpacket.cc b/test/syscalls/linux/socket_unix_unbound_seqpacket.cc index 270d7203f..21209b244 100644 --- a/test/syscalls/linux/socket_unix_unbound_seqpacket.cc +++ b/test/syscalls/linux/socket_unix_unbound_seqpacket.cc @@ -42,7 +42,7 @@ TEST_P(UnboundUnixSeqpacketSocketPairTest, SendtoWithoutConnect) { } TEST_P(UnboundUnixSeqpacketSocketPairTest, SendtoWithoutConnectIgnoresAddr) { - // FIXME: gVisor tries to find /foo/bar and thus returns ENOENT. + // FIXME(b/68223466): gVisor tries to find /foo/bar and thus returns ENOENT. if (IsRunningOnGvisor()) { return; } diff --git a/test/syscalls/linux/socket_unix_unbound_stream.cc b/test/syscalls/linux/socket_unix_unbound_stream.cc index 4db5b4be1..b95f9569e 100644 --- a/test/syscalls/linux/socket_unix_unbound_stream.cc +++ b/test/syscalls/linux/socket_unix_unbound_stream.cc @@ -269,7 +269,7 @@ TEST_P(UnixStreamSocketPairTest, SinglePeek) { // 9f389e35674f5b086edd70ed524ca0f287259725 which changes this behavior. We // used to target 3.11 compatibility, so disable this test on newer kernels. // - // NOTE: Bring this up to Linux 4.4 compatibility. + // NOTE(b/118902768): Bring this up to Linux 4.4 compatibility. auto version = ASSERT_NO_ERRNO_AND_VALUE(GetKernelVersion()); SKIP_IF(version.major > 4 || (version.major == 4 && version.minor >= 3)); } @@ -686,7 +686,7 @@ TEST_P(UnboundUnixStreamSocketPairTest, SendtoWithoutConnect) { } TEST_P(UnboundUnixStreamSocketPairTest, SendtoWithoutConnectIgnoresAddr) { - // FIXME: gVisor tries to find /foo/bar and thus returns ENOENT. + // FIXME(b/68223466): gVisor tries to find /foo/bar and thus returns ENOENT. if (IsRunningOnGvisor()) { return; } diff --git a/test/syscalls/linux/stat.cc b/test/syscalls/linux/stat.cc index 48a2059de..746318d09 100644 --- a/test/syscalls/linux/stat.cc +++ b/test/syscalls/linux/stat.cc @@ -416,7 +416,7 @@ TEST_F(StatTest, ZeroLinksOpenFdRegularFileChild_NoRandomSave) { EXPECT_EQ(st_child_before.st_gid, st_child_fd.st_gid); EXPECT_EQ(st_child_before.st_size, st_child_fd.st_size); - // TODO: This isn't ideal but since fstatfs(2) will always return + // TODO(b/34861058): This isn't ideal but since fstatfs(2) will always return // OVERLAYFS_SUPER_MAGIC we have no way to know if this fs is backed by a // gofer which doesn't support links. EXPECT_TRUE(st_child_fd.st_nlink == 0 || st_child_fd.st_nlink == 1); diff --git a/test/syscalls/linux/stat_times.cc b/test/syscalls/linux/stat_times.cc index 442957c65..8346e9a8e 100644 --- a/test/syscalls/linux/stat_times.cc +++ b/test/syscalls/linux/stat_times.cc @@ -68,7 +68,7 @@ TEST_F(StatTimesTest, FileCreationTimes) { TEST_F(StatTimesTest, FileCtimeChanges) { auto file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); - MaybeSave(); // FIXME: ctime is inconsistent. + MaybeSave(); // FIXME(b/69865927): ctime is inconsistent. absl::Time atime, mtime, ctime; std::tie(atime, mtime, ctime) = GetTime(file); @@ -150,7 +150,7 @@ TEST_F(StatTimesTest, FileAtimeChanges) { const auto file = ASSERT_NO_ERRNO_AND_VALUE( TempPath::CreateFileWith(GetAbsoluteTestTmpdir(), contents, 0666)); - MaybeSave(); // FIXME: ctime is inconsistent. + MaybeSave(); // FIXME(b/69865927): ctime is inconsistent. absl::Time atime, mtime, ctime; std::tie(atime, mtime, ctime) = GetTime(file); @@ -184,7 +184,7 @@ TEST_F(StatTimesTest, DirAtimeChanges) { const auto file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFileIn(dir.path())); - MaybeSave(); // FIXME: ctime is inconsistent. + MaybeSave(); // FIXME(b/69865927): ctime is inconsistent. absl::Time atime, mtime, ctime; std::tie(atime, mtime, ctime) = GetTime(dir); @@ -193,7 +193,7 @@ TEST_F(StatTimesTest, DirAtimeChanges) { const absl::Time before = absl::Now() - absl::Seconds(1); - // NOTE: Keep an fd open. This ensures that the inode backing the + // NOTE(b/37756234): Keep an fd open. This ensures that the inode backing the // directory won't be destroyed before the final GetTime to avoid writing out // timestamps and causing side effects. const auto fd = ASSERT_NO_ERRNO_AND_VALUE(Open(dir.path(), O_RDONLY, 0)); diff --git a/test/syscalls/linux/tcp_socket.cc b/test/syscalls/linux/tcp_socket.cc index 1057f5892..33620a874 100644 --- a/test/syscalls/linux/tcp_socket.cc +++ b/test/syscalls/linux/tcp_socket.cc @@ -191,7 +191,7 @@ TEST_P(TcpSocketTest, SenderAddressIgnoredOnPeek) { TEST_P(TcpSocketTest, SendtoAddressIgnored) { struct sockaddr_storage addr; memset(&addr, 0, sizeof(addr)); - addr.ss_family = GetParam(); // FIXME + addr.ss_family = GetParam(); // FIXME(b/63803955) char data = '\0'; EXPECT_THAT( diff --git a/test/syscalls/linux/tkill.cc b/test/syscalls/linux/tkill.cc index 9842ccc9b..3e8ce5327 100644 --- a/test/syscalls/linux/tkill.cc +++ b/test/syscalls/linux/tkill.cc @@ -32,7 +32,7 @@ namespace { static int tkill(pid_t tid, int sig) { int ret; do { - // NOTE: tkill(2) could return EAGAIN for RT signals. + // NOTE(b/25434735): tkill(2) could return EAGAIN for RT signals. ret = syscall(SYS_tkill, tid, sig); } while (ret == -1 && errno == EAGAIN); return ret; diff --git a/test/syscalls/linux/udp_bind.cc b/test/syscalls/linux/udp_bind.cc index 902be47d3..547eb2a6c 100644 --- a/test/syscalls/linux/udp_bind.cc +++ b/test/syscalls/linux/udp_bind.cc @@ -286,7 +286,7 @@ INSTANTIATE_TEST_SUITE_P( []() { SendtoTestParam param = {}; param.description = "connected IPv6 sendto IPv4 mapped IPv6"; - // TODO: Determine if this inconsistent behavior is worth + // TODO(igudger): Determine if this inconsistent behavior is worth // implementing. param.skip_on_gvisor = true; param.send_domain = AF_INET6; @@ -299,7 +299,7 @@ INSTANTIATE_TEST_SUITE_P( []() { SendtoTestParam param = {}; param.description = "connected IPv6 sendto IPv4"; - // TODO: Determine if this inconsistent behavior is worth + // TODO(igudger): Determine if this inconsistent behavior is worth // implementing. param.skip_on_gvisor = true; param.send_domain = AF_INET6; diff --git a/test/syscalls/linux/uidgid.cc b/test/syscalls/linux/uidgid.cc index c0c1f2960..d78a09b1e 100644 --- a/test/syscalls/linux/uidgid.cc +++ b/test/syscalls/linux/uidgid.cc @@ -169,7 +169,7 @@ TEST(UidGidRootTest, SetgidNotFromThreadGroupLeader) { SKIP_IF(!ASSERT_NO_ERRNO_AND_VALUE(IsRoot())); const gid_t gid = FLAGS_scratch_gid1; - // NOTE: Do setgid in a separate thread so that we can test if + // NOTE(b/64676707): Do setgid in a separate thread so that we can test if // info.si_pid is set correctly. ScopedThread([gid] { ASSERT_THAT(setgid(gid), SyscallSucceeds()); }); EXPECT_NO_ERRNO(CheckGIDs(gid, gid, gid)); diff --git a/test/syscalls/linux/utimes.cc b/test/syscalls/linux/utimes.cc index d95ee74ec..bf776cd93 100644 --- a/test/syscalls/linux/utimes.cc +++ b/test/syscalls/linux/utimes.cc @@ -33,7 +33,7 @@ namespace testing { namespace { -// TODO: utimes(nullptr) does not pick the "now" time in the +// TODO(b/36516566): utimes(nullptr) does not pick the "now" time in the // application's time domain, so when asserting that times are within a window, // we expand the window to allow for differences between the time domains. constexpr absl::Duration kClockSlack = absl::Milliseconds(100); @@ -235,7 +235,7 @@ void TestUtimensat(int dirFd, std::string const& path) { EXPECT_LE(mtime3, after); if (!IsRunningOnGvisor()) { - // FIXME: Gofers set atime and mtime to different "now" times. + // FIXME(b/36516566): Gofers set atime and mtime to different "now" times. EXPECT_EQ(atime3, mtime3); } } diff --git a/test/syscalls/linux/wait.cc b/test/syscalls/linux/wait.cc index cfab8a976..fcd606bec 100644 --- a/test/syscalls/linux/wait.cc +++ b/test/syscalls/linux/wait.cc @@ -40,7 +40,7 @@ using ::testing::UnorderedElementsAre; // These unit tests focus on the wait4(2) system call, but include a basic // checks for the i386 waitpid(2) syscall, which is a subset of wait4(2). // -// NOTE: Some functionality is not tested as +// NOTE(b/22640830,b/27680907,b/29049891): Some functionality is not tested as // it is not currently supported by gVisor: // * UID in waitid(2) siginfo. // * Process groups. diff --git a/test/syscalls/linux/write.cc b/test/syscalls/linux/write.cc index 432bd6066..7f80b2fa8 100644 --- a/test/syscalls/linux/write.cc +++ b/test/syscalls/linux/write.cc @@ -33,7 +33,7 @@ namespace testing { namespace { // This test is currently very rudimentary. // -// TODO: +// TODO(edahlgren): // * bad buffer states (EFAULT). // * bad fds (wrong permission, wrong type of file, EBADF). // * check offset is incremented. diff --git a/third_party/gvsync/downgradable_rwmutex_unsafe.go b/third_party/gvsync/downgradable_rwmutex_unsafe.go index a63a0d084..131f0a2ba 100644 --- a/third_party/gvsync/downgradable_rwmutex_unsafe.go +++ b/third_party/gvsync/downgradable_rwmutex_unsafe.go @@ -49,7 +49,7 @@ func (rw *DowngradableRWMutex) RLock() { // RUnlock undoes a single RLock call. func (rw *DowngradableRWMutex) RUnlock() { if RaceEnabled { - // TODO: Why does this need to be ReleaseMerge instead of + // TODO(jamieliu): Why does this need to be ReleaseMerge instead of // Release? IIUC this establishes Unlock happens-before RUnlock, which // seems unnecessary. RaceReleaseMerge(unsafe.Pointer(&rw.writerSem)) diff --git a/vdso/cycle_clock.h b/vdso/cycle_clock.h index 26d6690c0..309e07a3f 100644 --- a/vdso/cycle_clock.h +++ b/vdso/cycle_clock.h @@ -23,7 +23,7 @@ namespace vdso { #if __x86_64__ -// TODO: The appropriate barrier instruction to use with rdtsc on +// TODO(b/74613497): The appropriate barrier instruction to use with rdtsc on // x86_64 depends on the vendor. Intel processors can use lfence but AMD may // need mfence, depending on MSR_F10H_DECFG_LFENCE_SERIALIZE_BIT. diff --git a/vdso/vdso_amd64.lds b/vdso/vdso_amd64.lds index 166779931..e2615ae9e 100644 --- a/vdso/vdso_amd64.lds +++ b/vdso/vdso_amd64.lds @@ -56,7 +56,7 @@ SECTIONS { .altinstr_replacement : { *(.altinstr_replacement) } /* - * TODO: Remove this alignment? Then the VDSO would fit + * TODO(gvisor.dev/issue/157): Remove this alignment? Then the VDSO would fit * in a single page. */ . = ALIGN(0x1000); diff --git a/vdso/vdso_arm64.lds b/vdso/vdso_arm64.lds index 19f8efa01..469185468 100644 --- a/vdso/vdso_arm64.lds +++ b/vdso/vdso_arm64.lds @@ -59,7 +59,7 @@ SECTIONS { .altinstr_replacement : { *(.altinstr_replacement) } /* - * TODO: Remove this alignment? Then the VDSO would fit + * TODO(gvisor.dev/issue/157): Remove this alignment? Then the VDSO would fit * in a single page. */ . = ALIGN(0x1000); -- cgit v1.2.3 From 4d52a5520101a88424fb63dd99412a1db33fbd06 Mon Sep 17 00:00:00 2001 From: Michael Pratt Date: Mon, 29 Apr 2019 14:25:05 -0700 Subject: Change copyright notice to "The gVisor Authors" Based on the guidelines at https://opensource.google.com/docs/releasing/authors/. 1. $ rg -l "Google LLC" | xargs sed -i 's/Google LLC.*/The gVisor Authors./' 2. Manual fixup of "Google Inc" references. 3. Add AUTHORS file. Authors may request to be added to this file. 4. Point netstack AUTHORS to gVisor AUTHORS. Drop CONTRIBUTORS. Fixes #209 PiperOrigin-RevId: 245823212 Change-Id: I64530b24ad021a7d683137459cafc510f5ee1de9 --- AUTHORS | 8 ++++++++ kokoro/run_build.sh | 2 +- kokoro/run_tests.sh | 2 +- pkg/abi/abi.go | 2 +- pkg/abi/abi_linux.go | 2 +- pkg/abi/flag.go | 2 +- pkg/abi/linux/aio.go | 2 +- pkg/abi/linux/ashmem.go | 2 +- pkg/abi/linux/audit.go | 2 +- pkg/abi/linux/binder.go | 2 +- pkg/abi/linux/bpf.go | 2 +- pkg/abi/linux/capability.go | 2 +- pkg/abi/linux/dev.go | 2 +- pkg/abi/linux/elf.go | 2 +- pkg/abi/linux/errors.go | 2 +- pkg/abi/linux/eventfd.go | 2 +- pkg/abi/linux/exec.go | 2 +- pkg/abi/linux/fcntl.go | 2 +- pkg/abi/linux/file.go | 2 +- pkg/abi/linux/fs.go | 2 +- pkg/abi/linux/futex.go | 2 +- pkg/abi/linux/inotify.go | 2 +- pkg/abi/linux/ioctl.go | 2 +- pkg/abi/linux/ip.go | 2 +- pkg/abi/linux/ipc.go | 2 +- pkg/abi/linux/limits.go | 2 +- pkg/abi/linux/linux.go | 2 +- pkg/abi/linux/mm.go | 2 +- pkg/abi/linux/netdevice.go | 2 +- pkg/abi/linux/netlink.go | 2 +- pkg/abi/linux/netlink_route.go | 2 +- pkg/abi/linux/poll.go | 2 +- pkg/abi/linux/prctl.go | 2 +- pkg/abi/linux/ptrace.go | 2 +- pkg/abi/linux/rusage.go | 2 +- pkg/abi/linux/sched.go | 2 +- pkg/abi/linux/seccomp.go | 2 +- pkg/abi/linux/sem.go | 2 +- pkg/abi/linux/shm.go | 2 +- pkg/abi/linux/signal.go | 2 +- pkg/abi/linux/socket.go | 2 +- pkg/abi/linux/tcp.go | 2 +- pkg/abi/linux/time.go | 2 +- pkg/abi/linux/timer.go | 2 +- pkg/abi/linux/tty.go | 2 +- pkg/abi/linux/uio.go | 2 +- pkg/abi/linux/utsname.go | 2 +- pkg/amutex/amutex.go | 2 +- pkg/amutex/amutex_test.go | 2 +- pkg/atomicbitops/atomic_bitops.go | 2 +- pkg/atomicbitops/atomic_bitops_amd64.s | 2 +- pkg/atomicbitops/atomic_bitops_common.go | 2 +- pkg/atomicbitops/atomic_bitops_test.go | 2 +- pkg/binary/binary.go | 2 +- pkg/binary/binary_test.go | 2 +- pkg/bits/bits.go | 2 +- pkg/bits/bits_template.go | 2 +- pkg/bits/uint64_arch_amd64.go | 2 +- pkg/bits/uint64_arch_amd64_asm.s | 2 +- pkg/bits/uint64_arch_generic.go | 2 +- pkg/bits/uint64_test.go | 2 +- pkg/bpf/bpf.go | 2 +- pkg/bpf/decoder.go | 2 +- pkg/bpf/decoder_test.go | 2 +- pkg/bpf/input_bytes.go | 2 +- pkg/bpf/interpreter.go | 2 +- pkg/bpf/interpreter_test.go | 2 +- pkg/bpf/program_builder.go | 2 +- pkg/bpf/program_builder_test.go | 2 +- pkg/compressio/compressio.go | 2 +- pkg/compressio/compressio_test.go | 2 +- pkg/control/client/client.go | 2 +- pkg/control/server/server.go | 2 +- pkg/cpuid/cpu_amd64.s | 2 +- pkg/cpuid/cpuid.go | 2 +- pkg/cpuid/cpuid_parse_test.go | 2 +- pkg/cpuid/cpuid_test.go | 2 +- pkg/dhcp/client.go | 2 +- pkg/dhcp/dhcp.go | 2 +- pkg/dhcp/dhcp_string.go | 2 +- pkg/dhcp/dhcp_test.go | 2 +- pkg/dhcp/server.go | 2 +- pkg/eventchannel/event.go | 2 +- pkg/eventchannel/event.proto | 2 +- pkg/fd/fd.go | 2 +- pkg/fd/fd_test.go | 2 +- pkg/fdnotifier/fdnotifier.go | 2 +- pkg/fdnotifier/poll_unsafe.go | 2 +- pkg/gate/gate.go | 2 +- pkg/gate/gate_test.go | 2 +- pkg/ilist/list.go | 2 +- pkg/ilist/list_test.go | 2 +- pkg/linewriter/linewriter.go | 2 +- pkg/linewriter/linewriter_test.go | 2 +- pkg/log/glog.go | 2 +- pkg/log/glog_unsafe.go | 2 +- pkg/log/json.go | 2 +- pkg/log/json_k8s.go | 2 +- pkg/log/json_test.go | 2 +- pkg/log/log.go | 2 +- pkg/log/log_test.go | 2 +- pkg/metric/metric.go | 2 +- pkg/metric/metric.proto | 2 +- pkg/metric/metric_test.go | 2 +- pkg/p9/buffer.go | 2 +- pkg/p9/buffer_test.go | 2 +- pkg/p9/client.go | 2 +- pkg/p9/client_file.go | 2 +- pkg/p9/client_test.go | 2 +- pkg/p9/file.go | 2 +- pkg/p9/handlers.go | 2 +- pkg/p9/local_server/local_server.go | 2 +- pkg/p9/messages.go | 2 +- pkg/p9/messages_test.go | 2 +- pkg/p9/p9.go | 2 +- pkg/p9/p9_test.go | 2 +- pkg/p9/p9test/client_test.go | 2 +- pkg/p9/p9test/p9test.go | 2 +- pkg/p9/path_tree.go | 2 +- pkg/p9/pool.go | 2 +- pkg/p9/pool_test.go | 2 +- pkg/p9/server.go | 2 +- pkg/p9/transport.go | 2 +- pkg/p9/transport_test.go | 2 +- pkg/p9/version.go | 2 +- pkg/p9/version_test.go | 2 +- pkg/rand/rand.go | 2 +- pkg/rand/rand_linux.go | 2 +- pkg/refs/refcounter.go | 2 +- pkg/refs/refcounter_state.go | 2 +- pkg/refs/refcounter_test.go | 2 +- pkg/seccomp/seccomp.go | 2 +- pkg/seccomp/seccomp_rules.go | 2 +- pkg/seccomp/seccomp_test.go | 2 +- pkg/seccomp/seccomp_test_victim.go | 2 +- pkg/seccomp/seccomp_unsafe.go | 2 +- pkg/secio/full_reader.go | 2 +- pkg/secio/secio.go | 2 +- pkg/secio/secio_test.go | 2 +- pkg/segment/range.go | 2 +- pkg/segment/set.go | 2 +- pkg/segment/set_state.go | 2 +- pkg/segment/test/segment_test.go | 2 +- pkg/segment/test/set_functions.go | 2 +- pkg/sentry/arch/aligned.go | 2 +- pkg/sentry/arch/arch.go | 2 +- pkg/sentry/arch/arch_amd64.go | 2 +- pkg/sentry/arch/arch_amd64.s | 2 +- pkg/sentry/arch/arch_state_x86.go | 2 +- pkg/sentry/arch/arch_x86.go | 2 +- pkg/sentry/arch/auxv.go | 2 +- pkg/sentry/arch/registers.proto | 2 +- pkg/sentry/arch/signal_act.go | 2 +- pkg/sentry/arch/signal_amd64.go | 2 +- pkg/sentry/arch/signal_info.go | 2 +- pkg/sentry/arch/signal_stack.go | 2 +- pkg/sentry/arch/stack.go | 2 +- pkg/sentry/arch/syscalls_amd64.go | 2 +- pkg/sentry/context/context.go | 2 +- pkg/sentry/context/contexttest/contexttest.go | 2 +- pkg/sentry/control/control.go | 2 +- pkg/sentry/control/pprof.go | 2 +- pkg/sentry/control/proc.go | 2 +- pkg/sentry/control/proc_test.go | 2 +- pkg/sentry/control/state.go | 2 +- pkg/sentry/device/device.go | 2 +- pkg/sentry/device/device_test.go | 2 +- pkg/sentry/fs/anon/anon.go | 2 +- pkg/sentry/fs/anon/device.go | 2 +- pkg/sentry/fs/ashmem/area.go | 2 +- pkg/sentry/fs/ashmem/device.go | 2 +- pkg/sentry/fs/ashmem/pin_board.go | 2 +- pkg/sentry/fs/ashmem/pin_board_test.go | 2 +- pkg/sentry/fs/attr.go | 2 +- pkg/sentry/fs/binder/binder.go | 2 +- pkg/sentry/fs/context.go | 2 +- pkg/sentry/fs/copy_up.go | 2 +- pkg/sentry/fs/copy_up_test.go | 2 +- pkg/sentry/fs/dentry.go | 2 +- pkg/sentry/fs/dev/dev.go | 2 +- pkg/sentry/fs/dev/device.go | 2 +- pkg/sentry/fs/dev/fs.go | 2 +- pkg/sentry/fs/dev/full.go | 2 +- pkg/sentry/fs/dev/null.go | 2 +- pkg/sentry/fs/dev/random.go | 2 +- pkg/sentry/fs/dirent.go | 2 +- pkg/sentry/fs/dirent_cache.go | 2 +- pkg/sentry/fs/dirent_cache_limiter.go | 2 +- pkg/sentry/fs/dirent_cache_test.go | 2 +- pkg/sentry/fs/dirent_refs_test.go | 2 +- pkg/sentry/fs/dirent_state.go | 2 +- pkg/sentry/fs/fdpipe/pipe.go | 2 +- pkg/sentry/fs/fdpipe/pipe_opener.go | 2 +- pkg/sentry/fs/fdpipe/pipe_opener_test.go | 2 +- pkg/sentry/fs/fdpipe/pipe_state.go | 2 +- pkg/sentry/fs/fdpipe/pipe_test.go | 2 +- pkg/sentry/fs/file.go | 2 +- pkg/sentry/fs/file_operations.go | 2 +- pkg/sentry/fs/file_overlay.go | 2 +- pkg/sentry/fs/file_overlay_test.go | 2 +- pkg/sentry/fs/file_state.go | 2 +- pkg/sentry/fs/file_test.go | 2 +- pkg/sentry/fs/filesystems.go | 2 +- pkg/sentry/fs/filetest/filetest.go | 2 +- pkg/sentry/fs/flags.go | 2 +- pkg/sentry/fs/fs.go | 2 +- pkg/sentry/fs/fsutil/dirty_set.go | 2 +- pkg/sentry/fs/fsutil/dirty_set_test.go | 2 +- pkg/sentry/fs/fsutil/file.go | 2 +- pkg/sentry/fs/fsutil/file_range_set.go | 2 +- pkg/sentry/fs/fsutil/frame_ref_set.go | 2 +- pkg/sentry/fs/fsutil/fsutil.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper_state.go | 2 +- pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go | 2 +- pkg/sentry/fs/fsutil/host_mappable.go | 2 +- pkg/sentry/fs/fsutil/inode.go | 2 +- pkg/sentry/fs/fsutil/inode_cached.go | 2 +- pkg/sentry/fs/fsutil/inode_cached_test.go | 2 +- pkg/sentry/fs/gofer/attr.go | 2 +- pkg/sentry/fs/gofer/cache_policy.go | 2 +- pkg/sentry/fs/gofer/context_file.go | 2 +- pkg/sentry/fs/gofer/device.go | 2 +- pkg/sentry/fs/gofer/file.go | 2 +- pkg/sentry/fs/gofer/file_state.go | 2 +- pkg/sentry/fs/gofer/fs.go | 2 +- pkg/sentry/fs/gofer/gofer_test.go | 2 +- pkg/sentry/fs/gofer/handles.go | 2 +- pkg/sentry/fs/gofer/inode.go | 2 +- pkg/sentry/fs/gofer/inode_state.go | 2 +- pkg/sentry/fs/gofer/path.go | 2 +- pkg/sentry/fs/gofer/session.go | 2 +- pkg/sentry/fs/gofer/session_state.go | 2 +- pkg/sentry/fs/gofer/socket.go | 2 +- pkg/sentry/fs/gofer/util.go | 2 +- pkg/sentry/fs/host/control.go | 2 +- pkg/sentry/fs/host/descriptor.go | 2 +- pkg/sentry/fs/host/descriptor_state.go | 2 +- pkg/sentry/fs/host/descriptor_test.go | 2 +- pkg/sentry/fs/host/device.go | 2 +- pkg/sentry/fs/host/file.go | 2 +- pkg/sentry/fs/host/fs.go | 2 +- pkg/sentry/fs/host/fs_test.go | 2 +- pkg/sentry/fs/host/inode.go | 2 +- pkg/sentry/fs/host/inode_state.go | 2 +- pkg/sentry/fs/host/inode_test.go | 2 +- pkg/sentry/fs/host/ioctl_unsafe.go | 2 +- pkg/sentry/fs/host/socket.go | 2 +- pkg/sentry/fs/host/socket_iovec.go | 2 +- pkg/sentry/fs/host/socket_state.go | 2 +- pkg/sentry/fs/host/socket_test.go | 2 +- pkg/sentry/fs/host/socket_unsafe.go | 2 +- pkg/sentry/fs/host/tty.go | 2 +- pkg/sentry/fs/host/util.go | 2 +- pkg/sentry/fs/host/util_unsafe.go | 2 +- pkg/sentry/fs/host/wait_test.go | 2 +- pkg/sentry/fs/inode.go | 2 +- pkg/sentry/fs/inode_inotify.go | 2 +- pkg/sentry/fs/inode_operations.go | 2 +- pkg/sentry/fs/inode_overlay.go | 2 +- pkg/sentry/fs/inode_overlay_test.go | 2 +- pkg/sentry/fs/inotify.go | 2 +- pkg/sentry/fs/inotify_event.go | 2 +- pkg/sentry/fs/inotify_watch.go | 2 +- pkg/sentry/fs/lock/lock.go | 2 +- pkg/sentry/fs/lock/lock_range_test.go | 2 +- pkg/sentry/fs/lock/lock_set_functions.go | 2 +- pkg/sentry/fs/lock/lock_test.go | 2 +- pkg/sentry/fs/mock.go | 2 +- pkg/sentry/fs/mount.go | 2 +- pkg/sentry/fs/mount_overlay.go | 2 +- pkg/sentry/fs/mount_test.go | 2 +- pkg/sentry/fs/mounts.go | 2 +- pkg/sentry/fs/mounts_test.go | 2 +- pkg/sentry/fs/offset.go | 2 +- pkg/sentry/fs/overlay.go | 2 +- pkg/sentry/fs/path.go | 2 +- pkg/sentry/fs/path_test.go | 2 +- pkg/sentry/fs/proc/cpuinfo.go | 2 +- pkg/sentry/fs/proc/device/device.go | 2 +- pkg/sentry/fs/proc/exec_args.go | 2 +- pkg/sentry/fs/proc/fds.go | 2 +- pkg/sentry/fs/proc/filesystems.go | 2 +- pkg/sentry/fs/proc/fs.go | 2 +- pkg/sentry/fs/proc/inode.go | 2 +- pkg/sentry/fs/proc/loadavg.go | 2 +- pkg/sentry/fs/proc/meminfo.go | 2 +- pkg/sentry/fs/proc/mounts.go | 2 +- pkg/sentry/fs/proc/net.go | 2 +- pkg/sentry/fs/proc/net_test.go | 2 +- pkg/sentry/fs/proc/proc.go | 2 +- pkg/sentry/fs/proc/rpcinet_proc.go | 2 +- pkg/sentry/fs/proc/seqfile/seqfile.go | 2 +- pkg/sentry/fs/proc/seqfile/seqfile_test.go | 2 +- pkg/sentry/fs/proc/stat.go | 2 +- pkg/sentry/fs/proc/sys.go | 2 +- pkg/sentry/fs/proc/sys_net.go | 2 +- pkg/sentry/fs/proc/sys_net_state.go | 2 +- pkg/sentry/fs/proc/sys_net_test.go | 2 +- pkg/sentry/fs/proc/task.go | 2 +- pkg/sentry/fs/proc/uid_gid_map.go | 2 +- pkg/sentry/fs/proc/uptime.go | 2 +- pkg/sentry/fs/proc/version.go | 2 +- pkg/sentry/fs/ramfs/dir.go | 2 +- pkg/sentry/fs/ramfs/socket.go | 2 +- pkg/sentry/fs/ramfs/symlink.go | 2 +- pkg/sentry/fs/ramfs/tree.go | 2 +- pkg/sentry/fs/ramfs/tree_test.go | 2 +- pkg/sentry/fs/restore.go | 2 +- pkg/sentry/fs/save.go | 2 +- pkg/sentry/fs/seek.go | 2 +- pkg/sentry/fs/sync.go | 2 +- pkg/sentry/fs/sys/device.go | 2 +- pkg/sentry/fs/sys/devices.go | 2 +- pkg/sentry/fs/sys/fs.go | 2 +- pkg/sentry/fs/sys/sys.go | 2 +- pkg/sentry/fs/timerfd/timerfd.go | 2 +- pkg/sentry/fs/tmpfs/device.go | 2 +- pkg/sentry/fs/tmpfs/file_regular.go | 2 +- pkg/sentry/fs/tmpfs/file_test.go | 2 +- pkg/sentry/fs/tmpfs/fs.go | 2 +- pkg/sentry/fs/tmpfs/inode_file.go | 2 +- pkg/sentry/fs/tmpfs/tmpfs.go | 2 +- pkg/sentry/fs/tty/dir.go | 2 +- pkg/sentry/fs/tty/fs.go | 2 +- pkg/sentry/fs/tty/line_discipline.go | 2 +- pkg/sentry/fs/tty/master.go | 2 +- pkg/sentry/fs/tty/queue.go | 2 +- pkg/sentry/fs/tty/slave.go | 2 +- pkg/sentry/fs/tty/terminal.go | 2 +- pkg/sentry/fs/tty/tty_test.go | 2 +- pkg/sentry/hostcpu/getcpu_amd64.s | 2 +- pkg/sentry/hostcpu/hostcpu.go | 2 +- pkg/sentry/hostcpu/hostcpu_test.go | 2 +- pkg/sentry/inet/context.go | 2 +- pkg/sentry/inet/inet.go | 2 +- pkg/sentry/inet/test_stack.go | 2 +- pkg/sentry/kernel/abstract_socket_namespace.go | 2 +- pkg/sentry/kernel/auth/auth.go | 2 +- pkg/sentry/kernel/auth/capability_set.go | 2 +- pkg/sentry/kernel/auth/context.go | 2 +- pkg/sentry/kernel/auth/credentials.go | 2 +- pkg/sentry/kernel/auth/id.go | 2 +- pkg/sentry/kernel/auth/id_map.go | 2 +- pkg/sentry/kernel/auth/id_map_functions.go | 2 +- pkg/sentry/kernel/auth/user_namespace.go | 2 +- pkg/sentry/kernel/context.go | 2 +- pkg/sentry/kernel/contexttest/contexttest.go | 2 +- pkg/sentry/kernel/epoll/epoll.go | 2 +- pkg/sentry/kernel/epoll/epoll_state.go | 2 +- pkg/sentry/kernel/epoll/epoll_test.go | 2 +- pkg/sentry/kernel/eventfd/eventfd.go | 2 +- pkg/sentry/kernel/eventfd/eventfd_test.go | 2 +- pkg/sentry/kernel/fasync/fasync.go | 2 +- pkg/sentry/kernel/fd_map.go | 2 +- pkg/sentry/kernel/fd_map_test.go | 2 +- pkg/sentry/kernel/fs_context.go | 2 +- pkg/sentry/kernel/futex/futex.go | 2 +- pkg/sentry/kernel/futex/futex_test.go | 2 +- pkg/sentry/kernel/ipc_namespace.go | 2 +- pkg/sentry/kernel/kdefs/kdefs.go | 2 +- pkg/sentry/kernel/kernel.go | 2 +- pkg/sentry/kernel/kernel_state.go | 2 +- pkg/sentry/kernel/memevent/memory_events.go | 2 +- pkg/sentry/kernel/memevent/memory_events.proto | 2 +- pkg/sentry/kernel/pending_signals.go | 2 +- pkg/sentry/kernel/pending_signals_state.go | 2 +- pkg/sentry/kernel/pipe/buffers.go | 2 +- pkg/sentry/kernel/pipe/device.go | 2 +- pkg/sentry/kernel/pipe/node.go | 2 +- pkg/sentry/kernel/pipe/node_test.go | 2 +- pkg/sentry/kernel/pipe/pipe.go | 2 +- pkg/sentry/kernel/pipe/pipe_test.go | 2 +- pkg/sentry/kernel/pipe/reader.go | 2 +- pkg/sentry/kernel/pipe/reader_writer.go | 2 +- pkg/sentry/kernel/pipe/writer.go | 2 +- pkg/sentry/kernel/posixtimer.go | 2 +- pkg/sentry/kernel/ptrace.go | 2 +- pkg/sentry/kernel/ptrace_amd64.go | 2 +- pkg/sentry/kernel/ptrace_arm64.go | 2 +- pkg/sentry/kernel/rseq.go | 2 +- pkg/sentry/kernel/sched/cpuset.go | 2 +- pkg/sentry/kernel/sched/cpuset_test.go | 2 +- pkg/sentry/kernel/sched/sched.go | 2 +- pkg/sentry/kernel/seccomp.go | 2 +- pkg/sentry/kernel/semaphore/semaphore.go | 2 +- pkg/sentry/kernel/semaphore/semaphore_test.go | 2 +- pkg/sentry/kernel/sessions.go | 2 +- pkg/sentry/kernel/shm/device.go | 2 +- pkg/sentry/kernel/shm/shm.go | 2 +- pkg/sentry/kernel/signal.go | 2 +- pkg/sentry/kernel/signal_handlers.go | 2 +- pkg/sentry/kernel/syscalls.go | 2 +- pkg/sentry/kernel/syscalls_state.go | 2 +- pkg/sentry/kernel/syslog.go | 2 +- pkg/sentry/kernel/table_test.go | 2 +- pkg/sentry/kernel/task.go | 2 +- pkg/sentry/kernel/task_acct.go | 2 +- pkg/sentry/kernel/task_block.go | 2 +- pkg/sentry/kernel/task_clone.go | 2 +- pkg/sentry/kernel/task_context.go | 2 +- pkg/sentry/kernel/task_exec.go | 2 +- pkg/sentry/kernel/task_exit.go | 2 +- pkg/sentry/kernel/task_futex.go | 2 +- pkg/sentry/kernel/task_identity.go | 2 +- pkg/sentry/kernel/task_log.go | 2 +- pkg/sentry/kernel/task_net.go | 2 +- pkg/sentry/kernel/task_run.go | 2 +- pkg/sentry/kernel/task_sched.go | 2 +- pkg/sentry/kernel/task_signals.go | 2 +- pkg/sentry/kernel/task_start.go | 2 +- pkg/sentry/kernel/task_stop.go | 2 +- pkg/sentry/kernel/task_syscall.go | 2 +- pkg/sentry/kernel/task_test.go | 2 +- pkg/sentry/kernel/task_usermem.go | 2 +- pkg/sentry/kernel/thread_group.go | 2 +- pkg/sentry/kernel/threads.go | 2 +- pkg/sentry/kernel/time/context.go | 2 +- pkg/sentry/kernel/time/time.go | 2 +- pkg/sentry/kernel/timekeeper.go | 2 +- pkg/sentry/kernel/timekeeper_state.go | 2 +- pkg/sentry/kernel/timekeeper_test.go | 2 +- pkg/sentry/kernel/uncaught_signal.proto | 2 +- pkg/sentry/kernel/uts_namespace.go | 2 +- pkg/sentry/kernel/vdso.go | 2 +- pkg/sentry/kernel/version.go | 2 +- pkg/sentry/limits/context.go | 2 +- pkg/sentry/limits/limits.go | 2 +- pkg/sentry/limits/limits_test.go | 2 +- pkg/sentry/limits/linux.go | 2 +- pkg/sentry/loader/elf.go | 2 +- pkg/sentry/loader/interpreter.go | 2 +- pkg/sentry/loader/loader.go | 2 +- pkg/sentry/loader/vdso.go | 2 +- pkg/sentry/loader/vdso_state.go | 2 +- pkg/sentry/memmap/mapping_set.go | 2 +- pkg/sentry/memmap/mapping_set_test.go | 2 +- pkg/sentry/memmap/memmap.go | 2 +- pkg/sentry/memutil/memutil.go | 2 +- pkg/sentry/memutil/memutil_unsafe.go | 2 +- pkg/sentry/mm/address_space.go | 2 +- pkg/sentry/mm/aio_context.go | 2 +- pkg/sentry/mm/aio_context_state.go | 2 +- pkg/sentry/mm/debug.go | 2 +- pkg/sentry/mm/io.go | 2 +- pkg/sentry/mm/lifecycle.go | 2 +- pkg/sentry/mm/metadata.go | 2 +- pkg/sentry/mm/mm.go | 2 +- pkg/sentry/mm/mm_test.go | 2 +- pkg/sentry/mm/pma.go | 2 +- pkg/sentry/mm/procfs.go | 2 +- pkg/sentry/mm/save_restore.go | 2 +- pkg/sentry/mm/shm.go | 2 +- pkg/sentry/mm/special_mappable.go | 2 +- pkg/sentry/mm/syscalls.go | 2 +- pkg/sentry/mm/vma.go | 2 +- pkg/sentry/pgalloc/context.go | 2 +- pkg/sentry/pgalloc/pgalloc.go | 2 +- pkg/sentry/pgalloc/pgalloc_test.go | 2 +- pkg/sentry/pgalloc/pgalloc_unsafe.go | 2 +- pkg/sentry/pgalloc/save_restore.go | 2 +- pkg/sentry/platform/context.go | 2 +- pkg/sentry/platform/interrupt/interrupt.go | 2 +- pkg/sentry/platform/interrupt/interrupt_test.go | 2 +- pkg/sentry/platform/kvm/address_space.go | 2 +- pkg/sentry/platform/kvm/allocator.go | 2 +- pkg/sentry/platform/kvm/bluepill.go | 2 +- pkg/sentry/platform/kvm/bluepill_amd64.go | 2 +- pkg/sentry/platform/kvm/bluepill_amd64.s | 2 +- pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/bluepill_fault.go | 2 +- pkg/sentry/platform/kvm/bluepill_unsafe.go | 2 +- pkg/sentry/platform/kvm/context.go | 2 +- pkg/sentry/platform/kvm/kvm.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64.go | 2 +- pkg/sentry/platform/kvm/kvm_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/kvm_const.go | 2 +- pkg/sentry/platform/kvm/kvm_test.go | 2 +- pkg/sentry/platform/kvm/machine.go | 2 +- pkg/sentry/platform/kvm/machine_amd64.go | 2 +- pkg/sentry/platform/kvm/machine_amd64_unsafe.go | 2 +- pkg/sentry/platform/kvm/machine_unsafe.go | 2 +- pkg/sentry/platform/kvm/physical_map.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil_amd64.go | 2 +- pkg/sentry/platform/kvm/testutil/testutil_amd64.s | 2 +- pkg/sentry/platform/kvm/virtual_map.go | 2 +- pkg/sentry/platform/kvm/virtual_map_test.go | 2 +- pkg/sentry/platform/mmap_min_addr.go | 2 +- pkg/sentry/platform/platform.go | 2 +- pkg/sentry/platform/procid/procid.go | 2 +- pkg/sentry/platform/procid/procid_amd64.s | 2 +- pkg/sentry/platform/procid/procid_arm64.s | 2 +- pkg/sentry/platform/procid/procid_net_test.go | 2 +- pkg/sentry/platform/procid/procid_test.go | 2 +- pkg/sentry/platform/ptrace/ptrace.go | 2 +- pkg/sentry/platform/ptrace/ptrace_unsafe.go | 2 +- pkg/sentry/platform/ptrace/stub_amd64.s | 2 +- pkg/sentry/platform/ptrace/stub_unsafe.go | 2 +- pkg/sentry/platform/ptrace/subprocess.go | 2 +- pkg/sentry/platform/ptrace/subprocess_amd64.go | 2 +- pkg/sentry/platform/ptrace/subprocess_linux.go | 2 +- pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go | 2 +- pkg/sentry/platform/ptrace/subprocess_unsafe.go | 2 +- pkg/sentry/platform/ring0/defs.go | 2 +- pkg/sentry/platform/ring0/defs_amd64.go | 2 +- pkg/sentry/platform/ring0/entry_amd64.go | 2 +- pkg/sentry/platform/ring0/entry_amd64.s | 2 +- pkg/sentry/platform/ring0/gen_offsets/main.go | 2 +- pkg/sentry/platform/ring0/kernel.go | 2 +- pkg/sentry/platform/ring0/kernel_amd64.go | 2 +- pkg/sentry/platform/ring0/kernel_unsafe.go | 2 +- pkg/sentry/platform/ring0/lib_amd64.go | 2 +- pkg/sentry/platform/ring0/lib_amd64.s | 2 +- pkg/sentry/platform/ring0/offsets_amd64.go | 2 +- pkg/sentry/platform/ring0/pagetables/allocator.go | 2 +- pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_test.go | 2 +- pkg/sentry/platform/ring0/pagetables/pagetables_x86.go | 2 +- pkg/sentry/platform/ring0/pagetables/pcids_x86.go | 2 +- pkg/sentry/platform/ring0/pagetables/walker_amd64.go | 2 +- pkg/sentry/platform/ring0/ring0.go | 2 +- pkg/sentry/platform/ring0/x86.go | 2 +- pkg/sentry/platform/safecopy/atomic_amd64.s | 2 +- pkg/sentry/platform/safecopy/safecopy.go | 2 +- pkg/sentry/platform/safecopy/safecopy_test.go | 2 +- pkg/sentry/platform/safecopy/safecopy_unsafe.go | 2 +- pkg/sentry/platform/safecopy/sighandler_amd64.s | 2 +- pkg/sentry/platform/safecopy/sighandler_arm64.s | 2 +- pkg/sentry/safemem/block_unsafe.go | 2 +- pkg/sentry/safemem/io.go | 2 +- pkg/sentry/safemem/io_test.go | 2 +- pkg/sentry/safemem/safemem.go | 2 +- pkg/sentry/safemem/seq_test.go | 2 +- pkg/sentry/safemem/seq_unsafe.go | 2 +- pkg/sentry/sighandling/sighandling.go | 2 +- pkg/sentry/sighandling/sighandling_unsafe.go | 2 +- pkg/sentry/socket/control/control.go | 2 +- pkg/sentry/socket/epsocket/device.go | 2 +- pkg/sentry/socket/epsocket/epsocket.go | 2 +- pkg/sentry/socket/epsocket/provider.go | 2 +- pkg/sentry/socket/epsocket/save_restore.go | 2 +- pkg/sentry/socket/epsocket/stack.go | 2 +- pkg/sentry/socket/hostinet/device.go | 2 +- pkg/sentry/socket/hostinet/hostinet.go | 2 +- pkg/sentry/socket/hostinet/save_restore.go | 2 +- pkg/sentry/socket/hostinet/socket.go | 2 +- pkg/sentry/socket/hostinet/socket_unsafe.go | 2 +- pkg/sentry/socket/hostinet/stack.go | 2 +- pkg/sentry/socket/netlink/message.go | 2 +- pkg/sentry/socket/netlink/port/port.go | 2 +- pkg/sentry/socket/netlink/port/port_test.go | 2 +- pkg/sentry/socket/netlink/provider.go | 2 +- pkg/sentry/socket/netlink/route/protocol.go | 2 +- pkg/sentry/socket/netlink/socket.go | 2 +- pkg/sentry/socket/rpcinet/conn/conn.go | 2 +- pkg/sentry/socket/rpcinet/device.go | 2 +- pkg/sentry/socket/rpcinet/notifier/notifier.go | 2 +- pkg/sentry/socket/rpcinet/rpcinet.go | 2 +- pkg/sentry/socket/rpcinet/socket.go | 2 +- pkg/sentry/socket/rpcinet/stack.go | 2 +- pkg/sentry/socket/rpcinet/stack_unsafe.go | 2 +- pkg/sentry/socket/socket.go | 2 +- pkg/sentry/socket/unix/device.go | 2 +- pkg/sentry/socket/unix/io.go | 2 +- pkg/sentry/socket/unix/transport/connectioned.go | 2 +- pkg/sentry/socket/unix/transport/connectioned_state.go | 2 +- pkg/sentry/socket/unix/transport/connectionless.go | 2 +- pkg/sentry/socket/unix/transport/queue.go | 2 +- pkg/sentry/socket/unix/transport/unix.go | 2 +- pkg/sentry/socket/unix/unix.go | 2 +- pkg/sentry/state/state.go | 2 +- pkg/sentry/state/state_metadata.go | 2 +- pkg/sentry/state/state_unsafe.go | 2 +- pkg/sentry/strace/capability.go | 2 +- pkg/sentry/strace/clone.go | 2 +- pkg/sentry/strace/futex.go | 2 +- pkg/sentry/strace/linux64.go | 2 +- pkg/sentry/strace/open.go | 2 +- pkg/sentry/strace/poll.go | 2 +- pkg/sentry/strace/ptrace.go | 2 +- pkg/sentry/strace/signal.go | 2 +- pkg/sentry/strace/socket.go | 2 +- pkg/sentry/strace/strace.go | 2 +- pkg/sentry/strace/strace.proto | 2 +- pkg/sentry/strace/syscalls.go | 2 +- pkg/sentry/syscalls/epoll.go | 2 +- pkg/sentry/syscalls/linux/error.go | 2 +- pkg/sentry/syscalls/linux/flags.go | 2 +- pkg/sentry/syscalls/linux/linux64.go | 2 +- pkg/sentry/syscalls/linux/sigset.go | 2 +- pkg/sentry/syscalls/linux/sys_aio.go | 2 +- pkg/sentry/syscalls/linux/sys_capability.go | 2 +- pkg/sentry/syscalls/linux/sys_epoll.go | 2 +- pkg/sentry/syscalls/linux/sys_eventfd.go | 2 +- pkg/sentry/syscalls/linux/sys_file.go | 2 +- pkg/sentry/syscalls/linux/sys_futex.go | 2 +- pkg/sentry/syscalls/linux/sys_getdents.go | 2 +- pkg/sentry/syscalls/linux/sys_identity.go | 2 +- pkg/sentry/syscalls/linux/sys_inotify.go | 2 +- pkg/sentry/syscalls/linux/sys_lseek.go | 2 +- pkg/sentry/syscalls/linux/sys_mmap.go | 2 +- pkg/sentry/syscalls/linux/sys_mount.go | 2 +- pkg/sentry/syscalls/linux/sys_pipe.go | 2 +- pkg/sentry/syscalls/linux/sys_poll.go | 2 +- pkg/sentry/syscalls/linux/sys_prctl.go | 2 +- pkg/sentry/syscalls/linux/sys_random.go | 2 +- pkg/sentry/syscalls/linux/sys_read.go | 2 +- pkg/sentry/syscalls/linux/sys_rlimit.go | 2 +- pkg/sentry/syscalls/linux/sys_rusage.go | 2 +- pkg/sentry/syscalls/linux/sys_sched.go | 2 +- pkg/sentry/syscalls/linux/sys_seccomp.go | 2 +- pkg/sentry/syscalls/linux/sys_sem.go | 2 +- pkg/sentry/syscalls/linux/sys_shm.go | 2 +- pkg/sentry/syscalls/linux/sys_signal.go | 2 +- pkg/sentry/syscalls/linux/sys_socket.go | 2 +- pkg/sentry/syscalls/linux/sys_stat.go | 2 +- pkg/sentry/syscalls/linux/sys_sync.go | 2 +- pkg/sentry/syscalls/linux/sys_sysinfo.go | 2 +- pkg/sentry/syscalls/linux/sys_syslog.go | 2 +- pkg/sentry/syscalls/linux/sys_thread.go | 2 +- pkg/sentry/syscalls/linux/sys_time.go | 2 +- pkg/sentry/syscalls/linux/sys_timer.go | 2 +- pkg/sentry/syscalls/linux/sys_timerfd.go | 2 +- pkg/sentry/syscalls/linux/sys_tls.go | 2 +- pkg/sentry/syscalls/linux/sys_utsname.go | 2 +- pkg/sentry/syscalls/linux/sys_write.go | 2 +- pkg/sentry/syscalls/linux/timespec.go | 2 +- pkg/sentry/syscalls/syscalls.go | 2 +- pkg/sentry/time/calibrated_clock.go | 2 +- pkg/sentry/time/calibrated_clock_test.go | 2 +- pkg/sentry/time/clock_id.go | 2 +- pkg/sentry/time/clocks.go | 2 +- pkg/sentry/time/muldiv_amd64.s | 2 +- pkg/sentry/time/muldiv_arm64.s | 2 +- pkg/sentry/time/parameters.go | 2 +- pkg/sentry/time/parameters_test.go | 2 +- pkg/sentry/time/sampler.go | 2 +- pkg/sentry/time/sampler_test.go | 2 +- pkg/sentry/time/sampler_unsafe.go | 2 +- pkg/sentry/time/tsc_amd64.s | 2 +- pkg/sentry/time/tsc_arm64.s | 2 +- pkg/sentry/unimpl/events.go | 2 +- pkg/sentry/unimpl/unimplemented_syscall.proto | 2 +- pkg/sentry/uniqueid/context.go | 2 +- pkg/sentry/usage/cpu.go | 2 +- pkg/sentry/usage/io.go | 2 +- pkg/sentry/usage/memory.go | 2 +- pkg/sentry/usage/memory_unsafe.go | 2 +- pkg/sentry/usage/usage.go | 2 +- pkg/sentry/usermem/access_type.go | 2 +- pkg/sentry/usermem/addr.go | 2 +- pkg/sentry/usermem/addr_range_seq_test.go | 2 +- pkg/sentry/usermem/addr_range_seq_unsafe.go | 2 +- pkg/sentry/usermem/bytes_io.go | 2 +- pkg/sentry/usermem/bytes_io_unsafe.go | 2 +- pkg/sentry/usermem/usermem.go | 2 +- pkg/sentry/usermem/usermem_arm64.go | 2 +- pkg/sentry/usermem/usermem_test.go | 2 +- pkg/sentry/usermem/usermem_unsafe.go | 2 +- pkg/sentry/usermem/usermem_x86.go | 2 +- pkg/sentry/watchdog/watchdog.go | 2 +- pkg/sleep/commit_amd64.s | 2 +- pkg/sleep/commit_asm.go | 2 +- pkg/sleep/commit_noasm.go | 2 +- pkg/sleep/empty.s | 2 +- pkg/sleep/sleep_test.go | 2 +- pkg/sleep/sleep_unsafe.go | 2 +- pkg/state/decode.go | 2 +- pkg/state/encode.go | 2 +- pkg/state/encode_unsafe.go | 2 +- pkg/state/map.go | 2 +- pkg/state/object.proto | 2 +- pkg/state/printer.go | 2 +- pkg/state/state.go | 2 +- pkg/state/state_test.go | 2 +- pkg/state/statefile/statefile.go | 2 +- pkg/state/statefile/statefile_test.go | 2 +- pkg/state/stats.go | 2 +- pkg/syserr/host_linux.go | 2 +- pkg/syserr/netstack.go | 2 +- pkg/syserr/syserr.go | 2 +- pkg/syserror/syserror.go | 2 +- pkg/syserror/syserror_test.go | 2 +- pkg/tcpip/adapters/gonet/gonet.go | 2 +- pkg/tcpip/adapters/gonet/gonet_test.go | 2 +- pkg/tcpip/buffer/prependable.go | 2 +- pkg/tcpip/buffer/view.go | 2 +- pkg/tcpip/buffer/view_test.go | 2 +- pkg/tcpip/checker/checker.go | 2 +- pkg/tcpip/hash/jenkins/jenkins.go | 2 +- pkg/tcpip/hash/jenkins/jenkins_test.go | 2 +- pkg/tcpip/header/arp.go | 2 +- pkg/tcpip/header/checksum.go | 2 +- pkg/tcpip/header/eth.go | 2 +- pkg/tcpip/header/gue.go | 2 +- pkg/tcpip/header/icmpv4.go | 2 +- pkg/tcpip/header/icmpv6.go | 2 +- pkg/tcpip/header/interfaces.go | 2 +- pkg/tcpip/header/ipv4.go | 2 +- pkg/tcpip/header/ipv6.go | 2 +- pkg/tcpip/header/ipv6_fragment.go | 2 +- pkg/tcpip/header/ipversion_test.go | 2 +- pkg/tcpip/header/tcp.go | 2 +- pkg/tcpip/header/tcp_test.go | 2 +- pkg/tcpip/header/udp.go | 2 +- pkg/tcpip/link/channel/channel.go | 2 +- pkg/tcpip/link/fdbased/endpoint.go | 2 +- pkg/tcpip/link/fdbased/endpoint_test.go | 2 +- pkg/tcpip/link/fdbased/endpoint_unsafe.go | 2 +- pkg/tcpip/link/fdbased/mmap.go | 2 +- pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go | 2 +- pkg/tcpip/link/loopback/loopback.go | 2 +- pkg/tcpip/link/muxed/injectable.go | 2 +- pkg/tcpip/link/muxed/injectable_test.go | 2 +- pkg/tcpip/link/rawfile/blockingpoll_amd64.s | 2 +- pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go | 2 +- pkg/tcpip/link/rawfile/blockingpoll_unsafe.go | 2 +- pkg/tcpip/link/rawfile/errors.go | 2 +- pkg/tcpip/link/rawfile/rawfile_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe_test.go | 2 +- pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/pipe/rx.go | 2 +- pkg/tcpip/link/sharedmem/pipe/tx.go | 2 +- pkg/tcpip/link/sharedmem/queue/queue_test.go | 2 +- pkg/tcpip/link/sharedmem/queue/rx.go | 2 +- pkg/tcpip/link/sharedmem/queue/tx.go | 2 +- pkg/tcpip/link/sharedmem/rx.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem_test.go | 2 +- pkg/tcpip/link/sharedmem/sharedmem_unsafe.go | 2 +- pkg/tcpip/link/sharedmem/tx.go | 2 +- pkg/tcpip/link/sniffer/pcap.go | 2 +- pkg/tcpip/link/sniffer/sniffer.go | 2 +- pkg/tcpip/link/tun/tun_unsafe.go | 2 +- pkg/tcpip/link/waitable/waitable.go | 2 +- pkg/tcpip/link/waitable/waitable_test.go | 2 +- pkg/tcpip/network/arp/arp.go | 2 +- pkg/tcpip/network/arp/arp_test.go | 2 +- pkg/tcpip/network/fragmentation/frag_heap.go | 2 +- pkg/tcpip/network/fragmentation/frag_heap_test.go | 2 +- pkg/tcpip/network/fragmentation/fragmentation.go | 2 +- pkg/tcpip/network/fragmentation/fragmentation_test.go | 2 +- pkg/tcpip/network/fragmentation/reassembler.go | 2 +- pkg/tcpip/network/fragmentation/reassembler_test.go | 2 +- pkg/tcpip/network/hash/hash.go | 2 +- pkg/tcpip/network/ip_test.go | 2 +- pkg/tcpip/network/ipv4/icmp.go | 2 +- pkg/tcpip/network/ipv4/ipv4.go | 2 +- pkg/tcpip/network/ipv4/ipv4_test.go | 2 +- pkg/tcpip/network/ipv6/icmp.go | 2 +- pkg/tcpip/network/ipv6/icmp_test.go | 2 +- pkg/tcpip/network/ipv6/ipv6.go | 2 +- pkg/tcpip/ports/ports.go | 2 +- pkg/tcpip/ports/ports_test.go | 2 +- pkg/tcpip/sample/tun_tcp_connect/main.go | 2 +- pkg/tcpip/sample/tun_tcp_echo/main.go | 2 +- pkg/tcpip/seqnum/seqnum.go | 2 +- pkg/tcpip/stack/linkaddrcache.go | 2 +- pkg/tcpip/stack/linkaddrcache_test.go | 2 +- pkg/tcpip/stack/nic.go | 2 +- pkg/tcpip/stack/registration.go | 2 +- pkg/tcpip/stack/route.go | 2 +- pkg/tcpip/stack/stack.go | 2 +- pkg/tcpip/stack/stack_global_state.go | 2 +- pkg/tcpip/stack/stack_test.go | 2 +- pkg/tcpip/stack/transport_demuxer.go | 2 +- pkg/tcpip/stack/transport_test.go | 2 +- pkg/tcpip/tcpip.go | 2 +- pkg/tcpip/tcpip_test.go | 2 +- pkg/tcpip/time.s | 2 +- pkg/tcpip/time_unsafe.go | 2 +- pkg/tcpip/transport/icmp/endpoint.go | 2 +- pkg/tcpip/transport/icmp/endpoint_state.go | 2 +- pkg/tcpip/transport/icmp/protocol.go | 2 +- pkg/tcpip/transport/raw/raw.go | 2 +- pkg/tcpip/transport/raw/state.go | 2 +- pkg/tcpip/transport/tcp/accept.go | 2 +- pkg/tcpip/transport/tcp/connect.go | 2 +- pkg/tcpip/transport/tcp/cubic.go | 2 +- pkg/tcpip/transport/tcp/dual_stack_test.go | 2 +- pkg/tcpip/transport/tcp/endpoint.go | 2 +- pkg/tcpip/transport/tcp/endpoint_state.go | 2 +- pkg/tcpip/transport/tcp/forwarder.go | 2 +- pkg/tcpip/transport/tcp/protocol.go | 2 +- pkg/tcpip/transport/tcp/rcv.go | 2 +- pkg/tcpip/transport/tcp/reno.go | 2 +- pkg/tcpip/transport/tcp/sack.go | 2 +- pkg/tcpip/transport/tcp/sack_scoreboard.go | 2 +- pkg/tcpip/transport/tcp/sack_scoreboard_test.go | 2 +- pkg/tcpip/transport/tcp/segment.go | 2 +- pkg/tcpip/transport/tcp/segment_heap.go | 2 +- pkg/tcpip/transport/tcp/segment_queue.go | 2 +- pkg/tcpip/transport/tcp/segment_state.go | 2 +- pkg/tcpip/transport/tcp/snd.go | 2 +- pkg/tcpip/transport/tcp/snd_state.go | 2 +- pkg/tcpip/transport/tcp/tcp_sack_test.go | 2 +- pkg/tcpip/transport/tcp/tcp_test.go | 2 +- pkg/tcpip/transport/tcp/tcp_timestamp_test.go | 2 +- pkg/tcpip/transport/tcp/testing/context/context.go | 2 +- pkg/tcpip/transport/tcp/timer.go | 2 +- pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go | 2 +- pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go | 2 +- pkg/tcpip/transport/udp/endpoint.go | 2 +- pkg/tcpip/transport/udp/endpoint_state.go | 2 +- pkg/tcpip/transport/udp/forwarder.go | 2 +- pkg/tcpip/transport/udp/protocol.go | 2 +- pkg/tcpip/transport/udp/udp_test.go | 2 +- pkg/tmutex/tmutex.go | 2 +- pkg/tmutex/tmutex_test.go | 2 +- pkg/unet/unet.go | 2 +- pkg/unet/unet_test.go | 2 +- pkg/unet/unet_unsafe.go | 2 +- pkg/urpc/urpc.go | 2 +- pkg/urpc/urpc_test.go | 2 +- pkg/waiter/waiter.go | 2 +- pkg/waiter/waiter_test.go | 2 +- runsc/boot/compat.go | 2 +- runsc/boot/compat_amd64.go | 2 +- runsc/boot/compat_test.go | 2 +- runsc/boot/config.go | 2 +- runsc/boot/controller.go | 2 +- runsc/boot/debug.go | 2 +- runsc/boot/events.go | 2 +- runsc/boot/fds.go | 2 +- runsc/boot/filter/config.go | 2 +- runsc/boot/filter/extra_filters.go | 2 +- runsc/boot/filter/extra_filters_msan.go | 2 +- runsc/boot/filter/extra_filters_race.go | 2 +- runsc/boot/filter/filter.go | 2 +- runsc/boot/fs.go | 2 +- runsc/boot/limits.go | 2 +- runsc/boot/loader.go | 2 +- runsc/boot/loader_test.go | 2 +- runsc/boot/network.go | 2 +- runsc/boot/strace.go | 2 +- runsc/cgroup/cgroup.go | 2 +- runsc/cgroup/cgroup_test.go | 2 +- runsc/cmd/boot.go | 2 +- runsc/cmd/capability.go | 2 +- runsc/cmd/capability_test.go | 2 +- runsc/cmd/checkpoint.go | 2 +- runsc/cmd/chroot.go | 2 +- runsc/cmd/cmd.go | 2 +- runsc/cmd/create.go | 2 +- runsc/cmd/debug.go | 2 +- runsc/cmd/delete.go | 2 +- runsc/cmd/delete_test.go | 2 +- runsc/cmd/do.go | 2 +- runsc/cmd/events.go | 2 +- runsc/cmd/exec.go | 2 +- runsc/cmd/exec_test.go | 2 +- runsc/cmd/gofer.go | 2 +- runsc/cmd/gofer_test.go | 2 +- runsc/cmd/kill.go | 2 +- runsc/cmd/list.go | 2 +- runsc/cmd/path.go | 2 +- runsc/cmd/pause.go | 2 +- runsc/cmd/ps.go | 2 +- runsc/cmd/restore.go | 2 +- runsc/cmd/resume.go | 2 +- runsc/cmd/run.go | 2 +- runsc/cmd/spec.go | 2 +- runsc/cmd/start.go | 2 +- runsc/cmd/state.go | 2 +- runsc/cmd/wait.go | 2 +- runsc/console/console.go | 2 +- runsc/container/console_test.go | 2 +- runsc/container/container.go | 2 +- runsc/container/container_test.go | 2 +- runsc/container/hook.go | 2 +- runsc/container/multi_container_test.go | 2 +- runsc/container/shared_volume_test.go | 2 +- runsc/container/status.go | 2 +- runsc/container/test_app.go | 2 +- runsc/fsgofer/filter/config.go | 2 +- runsc/fsgofer/filter/extra_filters.go | 2 +- runsc/fsgofer/filter/extra_filters_msan.go | 2 +- runsc/fsgofer/filter/extra_filters_race.go | 2 +- runsc/fsgofer/filter/filter.go | 2 +- runsc/fsgofer/fsgofer.go | 2 +- runsc/fsgofer/fsgofer_test.go | 2 +- runsc/fsgofer/fsgofer_unsafe.go | 2 +- runsc/main.go | 2 +- runsc/sandbox/network.go | 2 +- runsc/sandbox/network_unsafe.go | 2 +- runsc/sandbox/sandbox.go | 2 +- runsc/specutils/fs.go | 2 +- runsc/specutils/namespace.go | 2 +- runsc/specutils/specutils.go | 2 +- runsc/specutils/specutils_test.go | 2 +- runsc/test/image/image.go | 2 +- runsc/test/image/image_test.go | 2 +- runsc/test/image/mysql.sql | 2 +- runsc/test/image/ruby.rb | 2 +- runsc/test/image/ruby.sh | 2 +- runsc/test/install.sh | 2 +- runsc/test/integration/exec_test.go | 2 +- runsc/test/integration/integration.go | 2 +- runsc/test/integration/integration_test.go | 2 +- runsc/test/root/cgroup_test.go | 2 +- runsc/test/root/chroot_test.go | 2 +- runsc/test/root/crictl_test.go | 2 +- runsc/test/root/root.go | 2 +- runsc/test/root/testdata/busybox.go | 2 +- runsc/test/root/testdata/containerd_config.go | 2 +- runsc/test/root/testdata/httpd.go | 2 +- runsc/test/root/testdata/httpd_mount_paths.go | 2 +- runsc/test/root/testdata/sandbox.go | 2 +- runsc/test/testutil/crictl.go | 2 +- runsc/test/testutil/docker.go | 2 +- runsc/test/testutil/testutil.go | 2 +- runsc/test/testutil/testutil_race.go | 2 +- runsc/tools/dockercfg/dockercfg.go | 2 +- runsc/version.go | 2 +- test/syscalls/gtest/gtest.go | 2 +- test/syscalls/linux/32bit.cc | 2 +- test/syscalls/linux/accept_bind.cc | 2 +- test/syscalls/linux/accept_bind_stream.cc | 2 +- test/syscalls/linux/access.cc | 2 +- test/syscalls/linux/affinity.cc | 2 +- test/syscalls/linux/aio.cc | 2 +- test/syscalls/linux/alarm.cc | 2 +- test/syscalls/linux/arch_prctl.cc | 2 +- test/syscalls/linux/bad.cc | 2 +- test/syscalls/linux/base_poll_test.cc | 2 +- test/syscalls/linux/base_poll_test.h | 2 +- test/syscalls/linux/bind.cc | 2 +- test/syscalls/linux/brk.cc | 2 +- test/syscalls/linux/chdir.cc | 2 +- test/syscalls/linux/chmod.cc | 2 +- test/syscalls/linux/chown.cc | 2 +- test/syscalls/linux/chroot.cc | 2 +- test/syscalls/linux/clock_getres.cc | 2 +- test/syscalls/linux/clock_gettime.cc | 2 +- test/syscalls/linux/clock_nanosleep.cc | 2 +- test/syscalls/linux/concurrency.cc | 2 +- test/syscalls/linux/creat.cc | 2 +- test/syscalls/linux/dev.cc | 2 +- test/syscalls/linux/dup.cc | 2 +- test/syscalls/linux/epoll.cc | 2 +- test/syscalls/linux/eventfd.cc | 2 +- test/syscalls/linux/exceptions.cc | 2 +- test/syscalls/linux/exec.cc | 2 +- test/syscalls/linux/exec.h | 2 +- test/syscalls/linux/exec_assert_closed_workload.cc | 2 +- test/syscalls/linux/exec_basic_workload.cc | 2 +- test/syscalls/linux/exec_binary.cc | 2 +- test/syscalls/linux/exec_proc_exe_workload.cc | 2 +- test/syscalls/linux/exec_state_workload.cc | 2 +- test/syscalls/linux/exit.cc | 2 +- test/syscalls/linux/exit_script.sh | 2 +- test/syscalls/linux/fadvise64.cc | 2 +- test/syscalls/linux/fallocate.cc | 2 +- test/syscalls/linux/fault.cc | 2 +- test/syscalls/linux/fchdir.cc | 2 +- test/syscalls/linux/fcntl.cc | 2 +- test/syscalls/linux/file_base.h | 2 +- test/syscalls/linux/flock.cc | 2 +- test/syscalls/linux/fork.cc | 2 +- test/syscalls/linux/fpsig_fork.cc | 2 +- test/syscalls/linux/fpsig_nested.cc | 2 +- test/syscalls/linux/fsync.cc | 2 +- test/syscalls/linux/futex.cc | 2 +- test/syscalls/linux/getcpu.cc | 2 +- test/syscalls/linux/getdents.cc | 2 +- test/syscalls/linux/getrandom.cc | 2 +- test/syscalls/linux/getrusage.cc | 2 +- test/syscalls/linux/inotify.cc | 2 +- test/syscalls/linux/ioctl.cc | 2 +- test/syscalls/linux/ip_socket_test_util.cc | 2 +- test/syscalls/linux/ip_socket_test_util.h | 2 +- test/syscalls/linux/itimer.cc | 2 +- test/syscalls/linux/kill.cc | 2 +- test/syscalls/linux/link.cc | 2 +- test/syscalls/linux/lseek.cc | 2 +- test/syscalls/linux/madvise.cc | 2 +- test/syscalls/linux/memfd.cc | 2 +- test/syscalls/linux/memory_accounting.cc | 2 +- test/syscalls/linux/mempolicy.cc | 2 +- test/syscalls/linux/mincore.cc | 2 +- test/syscalls/linux/mkdir.cc | 2 +- test/syscalls/linux/mknod.cc | 2 +- test/syscalls/linux/mlock.cc | 2 +- test/syscalls/linux/mmap.cc | 2 +- test/syscalls/linux/mount.cc | 2 +- test/syscalls/linux/mremap.cc | 2 +- test/syscalls/linux/msync.cc | 2 +- test/syscalls/linux/munmap.cc | 2 +- test/syscalls/linux/open.cc | 2 +- test/syscalls/linux/open_create.cc | 2 +- test/syscalls/linux/partial_bad_buffer.cc | 2 +- test/syscalls/linux/pause.cc | 2 +- test/syscalls/linux/pipe.cc | 2 +- test/syscalls/linux/poll.cc | 2 +- test/syscalls/linux/ppoll.cc | 2 +- test/syscalls/linux/prctl.cc | 2 +- test/syscalls/linux/prctl_setuid.cc | 2 +- test/syscalls/linux/pread64.cc | 2 +- test/syscalls/linux/preadv.cc | 2 +- test/syscalls/linux/preadv2.cc | 2 +- test/syscalls/linux/priority.cc | 2 +- test/syscalls/linux/priority_execve.cc | 2 +- test/syscalls/linux/proc.cc | 2 +- test/syscalls/linux/proc_net.cc | 2 +- test/syscalls/linux/proc_net_unix.cc | 2 +- test/syscalls/linux/proc_pid_smaps.cc | 2 +- test/syscalls/linux/proc_pid_uid_gid_map.cc | 2 +- test/syscalls/linux/pselect.cc | 2 +- test/syscalls/linux/ptrace.cc | 2 +- test/syscalls/linux/pty.cc | 2 +- test/syscalls/linux/pwrite64.cc | 2 +- test/syscalls/linux/pwritev2.cc | 2 +- test/syscalls/linux/raw_socket_ipv4.cc | 2 +- test/syscalls/linux/read.cc | 2 +- test/syscalls/linux/readv.cc | 2 +- test/syscalls/linux/readv_common.cc | 2 +- test/syscalls/linux/readv_common.h | 2 +- test/syscalls/linux/readv_socket.cc | 2 +- test/syscalls/linux/rename.cc | 2 +- test/syscalls/linux/rlimits.cc | 2 +- test/syscalls/linux/rtsignal.cc | 2 +- test/syscalls/linux/sched.cc | 2 +- test/syscalls/linux/sched_yield.cc | 2 +- test/syscalls/linux/seccomp.cc | 2 +- test/syscalls/linux/select.cc | 2 +- test/syscalls/linux/semaphore.cc | 2 +- test/syscalls/linux/sendfile.cc | 2 +- test/syscalls/linux/sendfile_socket.cc | 2 +- test/syscalls/linux/shm.cc | 2 +- test/syscalls/linux/sigaction.cc | 2 +- test/syscalls/linux/sigaltstack.cc | 2 +- test/syscalls/linux/sigaltstack_check.cc | 2 +- test/syscalls/linux/sigiret.cc | 2 +- test/syscalls/linux/sigprocmask.cc | 2 +- test/syscalls/linux/sigstop.cc | 2 +- test/syscalls/linux/sigtimedwait.cc | 2 +- test/syscalls/linux/socket_abstract.cc | 2 +- test/syscalls/linux/socket_blocking.cc | 2 +- test/syscalls/linux/socket_blocking.h | 2 +- test/syscalls/linux/socket_filesystem.cc | 2 +- test/syscalls/linux/socket_generic.cc | 2 +- test/syscalls/linux/socket_generic.h | 2 +- test/syscalls/linux/socket_inet_loopback.cc | 2 +- test/syscalls/linux/socket_ip_loopback_blocking.cc | 2 +- test/syscalls/linux/socket_ip_tcp_generic.cc | 2 +- test/syscalls/linux/socket_ip_tcp_generic.h | 2 +- test/syscalls/linux/socket_ip_tcp_generic_loopback.cc | 2 +- test/syscalls/linux/socket_ip_tcp_loopback.cc | 2 +- test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc | 2 +- test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc | 2 +- test/syscalls/linux/socket_ip_tcp_udp_generic.cc | 2 +- test/syscalls/linux/socket_ip_udp_generic.cc | 2 +- test/syscalls/linux/socket_ip_udp_generic.h | 2 +- test/syscalls/linux/socket_ip_udp_loopback.cc | 2 +- test/syscalls/linux/socket_ip_udp_loopback_blocking.cc | 2 +- test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc | 2 +- .../syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc | 2 +- test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h | 2 +- .../linux/socket_ipv4_tcp_unbound_external_networking_test.cc | 2 +- test/syscalls/linux/socket_ipv4_udp_unbound.cc | 2 +- test/syscalls/linux/socket_ipv4_udp_unbound.h | 2 +- .../syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc | 2 +- test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h | 2 +- .../linux/socket_ipv4_udp_unbound_external_networking_test.cc | 2 +- test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc | 2 +- test/syscalls/linux/socket_netdevice.cc | 2 +- test/syscalls/linux/socket_netlink_route.cc | 2 +- test/syscalls/linux/socket_netlink_util.cc | 2 +- test/syscalls/linux/socket_netlink_util.h | 2 +- test/syscalls/linux/socket_non_blocking.cc | 2 +- test/syscalls/linux/socket_non_blocking.h | 2 +- test/syscalls/linux/socket_non_stream.cc | 2 +- test/syscalls/linux/socket_non_stream.h | 2 +- test/syscalls/linux/socket_non_stream_blocking.cc | 2 +- test/syscalls/linux/socket_non_stream_blocking.h | 2 +- test/syscalls/linux/socket_stream.cc | 2 +- test/syscalls/linux/socket_stream.h | 2 +- test/syscalls/linux/socket_stream_blocking.cc | 2 +- test/syscalls/linux/socket_stream_blocking.h | 2 +- test/syscalls/linux/socket_stream_nonblock.cc | 2 +- test/syscalls/linux/socket_stream_nonblock.h | 2 +- test/syscalls/linux/socket_test_util.cc | 2 +- test/syscalls/linux/socket_test_util.h | 2 +- test/syscalls/linux/socket_unix.cc | 2 +- test/syscalls/linux/socket_unix.h | 2 +- test/syscalls/linux/socket_unix_abstract.cc | 2 +- test/syscalls/linux/socket_unix_abstract_nonblock.cc | 2 +- test/syscalls/linux/socket_unix_blocking_local.cc | 2 +- test/syscalls/linux/socket_unix_dgram.cc | 2 +- test/syscalls/linux/socket_unix_dgram.h | 2 +- test/syscalls/linux/socket_unix_dgram_local.cc | 2 +- test/syscalls/linux/socket_unix_dgram_non_blocking.cc | 2 +- test/syscalls/linux/socket_unix_domain.cc | 2 +- test/syscalls/linux/socket_unix_filesystem.cc | 2 +- test/syscalls/linux/socket_unix_filesystem_nonblock.cc | 2 +- test/syscalls/linux/socket_unix_non_stream.cc | 2 +- test/syscalls/linux/socket_unix_non_stream.h | 2 +- test/syscalls/linux/socket_unix_non_stream_blocking_local.cc | 2 +- test/syscalls/linux/socket_unix_pair.cc | 2 +- test/syscalls/linux/socket_unix_pair_nonblock.cc | 2 +- test/syscalls/linux/socket_unix_seqpacket.cc | 2 +- test/syscalls/linux/socket_unix_seqpacket.h | 2 +- test/syscalls/linux/socket_unix_seqpacket_local.cc | 2 +- test/syscalls/linux/socket_unix_stream.cc | 2 +- test/syscalls/linux/socket_unix_stream_blocking_local.cc | 2 +- test/syscalls/linux/socket_unix_stream_local.cc | 2 +- test/syscalls/linux/socket_unix_stream_nonblock_local.cc | 2 +- test/syscalls/linux/socket_unix_unbound_abstract.cc | 2 +- test/syscalls/linux/socket_unix_unbound_dgram.cc | 2 +- test/syscalls/linux/socket_unix_unbound_filesystem.cc | 2 +- test/syscalls/linux/socket_unix_unbound_seqpacket.cc | 2 +- test/syscalls/linux/socket_unix_unbound_stream.cc | 2 +- test/syscalls/linux/stat.cc | 2 +- test/syscalls/linux/stat_times.cc | 2 +- test/syscalls/linux/statfs.cc | 2 +- test/syscalls/linux/sticky.cc | 2 +- test/syscalls/linux/symlink.cc | 2 +- test/syscalls/linux/sync.cc | 2 +- test/syscalls/linux/sync_file_range.cc | 2 +- test/syscalls/linux/sysinfo.cc | 2 +- test/syscalls/linux/syslog.cc | 2 +- test/syscalls/linux/sysret.cc | 2 +- test/syscalls/linux/tcp_socket.cc | 2 +- test/syscalls/linux/temp_umask.h | 2 +- test/syscalls/linux/tgkill.cc | 2 +- test/syscalls/linux/time.cc | 2 +- test/syscalls/linux/timerfd.cc | 2 +- test/syscalls/linux/timers.cc | 2 +- test/syscalls/linux/tkill.cc | 2 +- test/syscalls/linux/truncate.cc | 2 +- test/syscalls/linux/udp_bind.cc | 2 +- test/syscalls/linux/udp_socket.cc | 2 +- test/syscalls/linux/uidgid.cc | 2 +- test/syscalls/linux/uname.cc | 2 +- test/syscalls/linux/unix_domain_socket_test_util.cc | 2 +- test/syscalls/linux/unix_domain_socket_test_util.h | 2 +- test/syscalls/linux/unlink.cc | 2 +- test/syscalls/linux/unshare.cc | 2 +- test/syscalls/linux/utimes.cc | 2 +- test/syscalls/linux/vdso.cc | 2 +- test/syscalls/linux/vdso_clock_gettime.cc | 2 +- test/syscalls/linux/vfork.cc | 2 +- test/syscalls/linux/vsyscall.cc | 2 +- test/syscalls/linux/wait.cc | 2 +- test/syscalls/linux/write.cc | 2 +- test/syscalls/syscall_test_runner.go | 2 +- test/syscalls/syscall_test_runner.sh | 2 +- test/util/capability_util.cc | 2 +- test/util/capability_util.h | 2 +- test/util/cleanup.h | 2 +- test/util/epoll_util.cc | 2 +- test/util/epoll_util.h | 2 +- test/util/eventfd_util.h | 2 +- test/util/file_descriptor.h | 2 +- test/util/fs_util.cc | 2 +- test/util/fs_util.h | 2 +- test/util/fs_util_test.cc | 2 +- test/util/logging.cc | 2 +- test/util/logging.h | 2 +- test/util/memory_util.h | 2 +- test/util/mount_util.h | 2 +- test/util/multiprocess_util.cc | 2 +- test/util/multiprocess_util.h | 2 +- test/util/posix_error.cc | 2 +- test/util/posix_error.h | 2 +- test/util/posix_error_test.cc | 2 +- test/util/proc_util.cc | 2 +- test/util/proc_util.h | 2 +- test/util/proc_util_test.cc | 2 +- test/util/rlimit_util.cc | 2 +- test/util/rlimit_util.h | 2 +- test/util/save_util.cc | 2 +- test/util/save_util.h | 2 +- test/util/signal_util.cc | 2 +- test/util/signal_util.h | 2 +- test/util/temp_path.cc | 2 +- test/util/temp_path.h | 2 +- test/util/test_main.cc | 2 +- test/util/test_util.cc | 2 +- test/util/test_util.h | 2 +- test/util/test_util_test.cc | 2 +- test/util/thread_util.h | 2 +- test/util/timer_util.cc | 2 +- test/util/timer_util.h | 2 +- third_party/gvsync/atomicptr_unsafe.go | 2 +- third_party/gvsync/atomicptrtest/atomicptr_test.go | 2 +- third_party/gvsync/downgradable_rwmutex_test.go | 2 +- third_party/gvsync/downgradable_rwmutex_unsafe.go | 2 +- third_party/gvsync/gvsync.go | 2 +- third_party/gvsync/memmove_unsafe.go | 2 +- third_party/gvsync/norace_unsafe.go | 2 +- third_party/gvsync/race_unsafe.go | 2 +- third_party/gvsync/seqatomic_unsafe.go | 2 +- third_party/gvsync/seqatomictest/seqatomic_test.go | 2 +- third_party/gvsync/seqcount.go | 2 +- third_party/gvsync/seqcount_test.go | 2 +- tools/go_generics/generics.go | 2 +- tools/go_generics/generics_tests/all_stmts/input.go | 2 +- tools/go_generics/generics_tests/all_stmts/output/output.go | 2 +- tools/go_generics/generics_tests/all_types/input.go | 2 +- tools/go_generics/generics_tests/all_types/lib/lib.go | 2 +- tools/go_generics/generics_tests/all_types/output/output.go | 2 +- tools/go_generics/generics_tests/consts/input.go | 2 +- tools/go_generics/generics_tests/consts/output/output.go | 2 +- tools/go_generics/generics_tests/imports/input.go | 2 +- tools/go_generics/generics_tests/imports/output/output.go | 2 +- tools/go_generics/generics_tests/remove_typedef/input.go | 2 +- tools/go_generics/generics_tests/remove_typedef/output/output.go | 2 +- tools/go_generics/generics_tests/simple/input.go | 2 +- tools/go_generics/generics_tests/simple/output/output.go | 2 +- tools/go_generics/globals/globals_visitor.go | 2 +- tools/go_generics/globals/scope.go | 2 +- tools/go_generics/go_generics_unittest.sh | 2 +- tools/go_generics/go_merge/main.go | 2 +- tools/go_generics/imports.go | 2 +- tools/go_generics/remove.go | 2 +- tools/go_generics/rules_tests/template.go | 2 +- tools/go_generics/rules_tests/template_test.go | 2 +- tools/go_stateify/main.go | 2 +- tools/tag_release.sh | 2 +- tools/workspace_status.sh | 2 +- vdso/barrier.h | 2 +- vdso/check_vdso.py | 2 +- vdso/compiler.h | 2 +- vdso/cycle_clock.h | 2 +- vdso/seqlock.h | 2 +- vdso/syscalls.h | 2 +- vdso/vdso.cc | 2 +- vdso/vdso_time.cc | 2 +- vdso/vdso_time.h | 2 +- 1235 files changed, 1242 insertions(+), 1234 deletions(-) create mode 100644 AUTHORS (limited to 'pkg/sentry/socket/rpcinet') diff --git a/AUTHORS b/AUTHORS new file mode 100644 index 000000000..01ba46567 --- /dev/null +++ b/AUTHORS @@ -0,0 +1,8 @@ +# This is the list of gVisor authors for copyright purposes. +# +# This does not necessarily list everyone who has contributed code, since in +# some cases, their employer may be the copyright holder. To see the full list +# of contributors, see the revision history in source control. +# +# Please send a patch if you would like to be included in this list. +Google LLC diff --git a/kokoro/run_build.sh b/kokoro/run_build.sh index 89e24b037..63fffda48 100755 --- a/kokoro/run_build.sh +++ b/kokoro/run_build.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# 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. diff --git a/kokoro/run_tests.sh b/kokoro/run_tests.sh index 8a3ce7402..08f678e39 100755 --- a/kokoro/run_tests.sh +++ b/kokoro/run_tests.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# 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. diff --git a/pkg/abi/abi.go b/pkg/abi/abi.go index 7770f0405..d56c481c9 100644 --- a/pkg/abi/abi.go +++ b/pkg/abi/abi.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/abi_linux.go b/pkg/abi/abi_linux.go index 9d9f361a4..3059479bd 100644 --- a/pkg/abi/abi_linux.go +++ b/pkg/abi/abi_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/flag.go b/pkg/abi/flag.go index b48757da8..dcdd66d4e 100644 --- a/pkg/abi/flag.go +++ b/pkg/abi/flag.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/aio.go b/pkg/abi/linux/aio.go index 1b7ca714a..3c6e0079d 100644 --- a/pkg/abi/linux/aio.go +++ b/pkg/abi/linux/aio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/ashmem.go b/pkg/abi/linux/ashmem.go index ced1e44d4..2a722abe0 100644 --- a/pkg/abi/linux/ashmem.go +++ b/pkg/abi/linux/ashmem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/audit.go b/pkg/abi/linux/audit.go index b39ba4515..6cca69af9 100644 --- a/pkg/abi/linux/audit.go +++ b/pkg/abi/linux/audit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/binder.go b/pkg/abi/linux/binder.go index 522dc6f53..63b08324a 100644 --- a/pkg/abi/linux/binder.go +++ b/pkg/abi/linux/binder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/bpf.go b/pkg/abi/linux/bpf.go index d9cd09948..aa3d3ce70 100644 --- a/pkg/abi/linux/bpf.go +++ b/pkg/abi/linux/bpf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/capability.go b/pkg/abi/linux/capability.go index 7d96f013e..c120cac64 100644 --- a/pkg/abi/linux/capability.go +++ b/pkg/abi/linux/capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/dev.go b/pkg/abi/linux/dev.go index 5b1199aac..421e11256 100644 --- a/pkg/abi/linux/dev.go +++ b/pkg/abi/linux/dev.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/elf.go b/pkg/abi/linux/elf.go index 928067c04..fb1c679d2 100644 --- a/pkg/abi/linux/elf.go +++ b/pkg/abi/linux/elf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/errors.go b/pkg/abi/linux/errors.go index e5f6f3f07..93f85a864 100644 --- a/pkg/abi/linux/errors.go +++ b/pkg/abi/linux/errors.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/eventfd.go b/pkg/abi/linux/eventfd.go index 5614f5cf1..9c479fc8f 100644 --- a/pkg/abi/linux/eventfd.go +++ b/pkg/abi/linux/eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/exec.go b/pkg/abi/linux/exec.go index a07c29243..579d46c41 100644 --- a/pkg/abi/linux/exec.go +++ b/pkg/abi/linux/exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/fcntl.go b/pkg/abi/linux/fcntl.go index c8558933a..cc8f2702d 100644 --- a/pkg/abi/linux/fcntl.go +++ b/pkg/abi/linux/fcntl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/file.go b/pkg/abi/linux/file.go index 46b10ca97..753fec3ed 100644 --- a/pkg/abi/linux/file.go +++ b/pkg/abi/linux/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/fs.go b/pkg/abi/linux/fs.go index a9f2ba132..c82ab9b5b 100644 --- a/pkg/abi/linux/fs.go +++ b/pkg/abi/linux/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/futex.go b/pkg/abi/linux/futex.go index afdf4123b..08bfde3b5 100644 --- a/pkg/abi/linux/futex.go +++ b/pkg/abi/linux/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/inotify.go b/pkg/abi/linux/inotify.go index 79c5d3593..2d08194ba 100644 --- a/pkg/abi/linux/inotify.go +++ b/pkg/abi/linux/inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/ioctl.go b/pkg/abi/linux/ioctl.go index 191b26e4d..04bb767dc 100644 --- a/pkg/abi/linux/ioctl.go +++ b/pkg/abi/linux/ioctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/ip.go b/pkg/abi/linux/ip.go index 77ac1062c..31e56ffa6 100644 --- a/pkg/abi/linux/ip.go +++ b/pkg/abi/linux/ip.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/ipc.go b/pkg/abi/linux/ipc.go index 10681768b..2ef8d6cbb 100644 --- a/pkg/abi/linux/ipc.go +++ b/pkg/abi/linux/ipc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/limits.go b/pkg/abi/linux/limits.go index e0aa5b31d..c74dfcd53 100644 --- a/pkg/abi/linux/limits.go +++ b/pkg/abi/linux/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/linux.go b/pkg/abi/linux/linux.go index d365f693d..8a8f831cd 100644 --- a/pkg/abi/linux/linux.go +++ b/pkg/abi/linux/linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/mm.go b/pkg/abi/linux/mm.go index eda8d9788..0b02f938a 100644 --- a/pkg/abi/linux/mm.go +++ b/pkg/abi/linux/mm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/netdevice.go b/pkg/abi/linux/netdevice.go index e3b6b1e40..aef1acf75 100644 --- a/pkg/abi/linux/netdevice.go +++ b/pkg/abi/linux/netdevice.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/netlink.go b/pkg/abi/linux/netlink.go index 25c5e17fd..5e718c363 100644 --- a/pkg/abi/linux/netlink.go +++ b/pkg/abi/linux/netlink.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/netlink_route.go b/pkg/abi/linux/netlink_route.go index 4200b6506..630dc339a 100644 --- a/pkg/abi/linux/netlink_route.go +++ b/pkg/abi/linux/netlink_route.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/poll.go b/pkg/abi/linux/poll.go index 9f0b15d1c..c04d26e4c 100644 --- a/pkg/abi/linux/poll.go +++ b/pkg/abi/linux/poll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/prctl.go b/pkg/abi/linux/prctl.go index db3206f36..dae2de290 100644 --- a/pkg/abi/linux/prctl.go +++ b/pkg/abi/linux/prctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/ptrace.go b/pkg/abi/linux/ptrace.go index 7db4f5464..23e605ab2 100644 --- a/pkg/abi/linux/ptrace.go +++ b/pkg/abi/linux/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/rusage.go b/pkg/abi/linux/rusage.go index 7fea4b589..d8302dc85 100644 --- a/pkg/abi/linux/rusage.go +++ b/pkg/abi/linux/rusage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/sched.go b/pkg/abi/linux/sched.go index ef96a3801..193d9a242 100644 --- a/pkg/abi/linux/sched.go +++ b/pkg/abi/linux/sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/seccomp.go b/pkg/abi/linux/seccomp.go index 8673a27bf..4eeb5cd7a 100644 --- a/pkg/abi/linux/seccomp.go +++ b/pkg/abi/linux/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/sem.go b/pkg/abi/linux/sem.go index b80c93daf..de422c519 100644 --- a/pkg/abi/linux/sem.go +++ b/pkg/abi/linux/sem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/shm.go b/pkg/abi/linux/shm.go index 82a80e609..e45aadb10 100644 --- a/pkg/abi/linux/shm.go +++ b/pkg/abi/linux/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/signal.go b/pkg/abi/linux/signal.go index 395f9f31e..9cbd77dda 100644 --- a/pkg/abi/linux/signal.go +++ b/pkg/abi/linux/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/socket.go b/pkg/abi/linux/socket.go index 6fa4e7c3e..417840731 100644 --- a/pkg/abi/linux/socket.go +++ b/pkg/abi/linux/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/tcp.go b/pkg/abi/linux/tcp.go index 67908deb9..174d470e2 100644 --- a/pkg/abi/linux/tcp.go +++ b/pkg/abi/linux/tcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/time.go b/pkg/abi/linux/time.go index bbd21e726..fa9ee27e1 100644 --- a/pkg/abi/linux/time.go +++ b/pkg/abi/linux/time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/timer.go b/pkg/abi/linux/timer.go index a6f420bdb..e32d09e10 100644 --- a/pkg/abi/linux/timer.go +++ b/pkg/abi/linux/timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/tty.go b/pkg/abi/linux/tty.go index bff882d89..8ac02aee8 100644 --- a/pkg/abi/linux/tty.go +++ b/pkg/abi/linux/tty.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/uio.go b/pkg/abi/linux/uio.go index 7e00d9959..1fd1e9802 100644 --- a/pkg/abi/linux/uio.go +++ b/pkg/abi/linux/uio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/abi/linux/utsname.go b/pkg/abi/linux/utsname.go index f80ed7d4a..60f220a67 100644 --- a/pkg/abi/linux/utsname.go +++ b/pkg/abi/linux/utsname.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/amutex/amutex.go b/pkg/amutex/amutex.go index 26b674435..85e819304 100644 --- a/pkg/amutex/amutex.go +++ b/pkg/amutex/amutex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/amutex/amutex_test.go b/pkg/amutex/amutex_test.go index 104e0dab1..6a0af006e 100644 --- a/pkg/amutex/amutex_test.go +++ b/pkg/amutex/amutex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/atomicbitops/atomic_bitops.go b/pkg/atomicbitops/atomic_bitops.go index 9a57f9599..63aa2b7f1 100644 --- a/pkg/atomicbitops/atomic_bitops.go +++ b/pkg/atomicbitops/atomic_bitops.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/atomicbitops/atomic_bitops_amd64.s b/pkg/atomicbitops/atomic_bitops_amd64.s index b37e3aad3..db0972001 100644 --- a/pkg/atomicbitops/atomic_bitops_amd64.s +++ b/pkg/atomicbitops/atomic_bitops_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/atomicbitops/atomic_bitops_common.go b/pkg/atomicbitops/atomic_bitops_common.go index b03242baa..b2a943dcb 100644 --- a/pkg/atomicbitops/atomic_bitops_common.go +++ b/pkg/atomicbitops/atomic_bitops_common.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/atomicbitops/atomic_bitops_test.go b/pkg/atomicbitops/atomic_bitops_test.go index ee6207cb3..965e9be79 100644 --- a/pkg/atomicbitops/atomic_bitops_test.go +++ b/pkg/atomicbitops/atomic_bitops_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/binary/binary.go b/pkg/binary/binary.go index 02f7e9fb8..631785f7b 100644 --- a/pkg/binary/binary.go +++ b/pkg/binary/binary.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/binary/binary_test.go b/pkg/binary/binary_test.go index 200961c70..4d609a438 100644 --- a/pkg/binary/binary_test.go +++ b/pkg/binary/binary_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bits/bits.go b/pkg/bits/bits.go index eb3c80f49..a26433ad6 100644 --- a/pkg/bits/bits.go +++ b/pkg/bits/bits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bits/bits_template.go b/pkg/bits/bits_template.go index 8c578cca2..93a435b80 100644 --- a/pkg/bits/bits_template.go +++ b/pkg/bits/bits_template.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bits/uint64_arch_amd64.go b/pkg/bits/uint64_arch_amd64.go index 1fef89394..faccaa61a 100644 --- a/pkg/bits/uint64_arch_amd64.go +++ b/pkg/bits/uint64_arch_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bits/uint64_arch_amd64_asm.s b/pkg/bits/uint64_arch_amd64_asm.s index 8c7322f0f..8ff364181 100644 --- a/pkg/bits/uint64_arch_amd64_asm.s +++ b/pkg/bits/uint64_arch_amd64_asm.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bits/uint64_arch_generic.go b/pkg/bits/uint64_arch_generic.go index cfb47400b..7dd2d1480 100644 --- a/pkg/bits/uint64_arch_generic.go +++ b/pkg/bits/uint64_arch_generic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bits/uint64_test.go b/pkg/bits/uint64_test.go index d6dbaf602..1b018d808 100644 --- a/pkg/bits/uint64_test.go +++ b/pkg/bits/uint64_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bpf/bpf.go b/pkg/bpf/bpf.go index 98d44d911..eb546f48f 100644 --- a/pkg/bpf/bpf.go +++ b/pkg/bpf/bpf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bpf/decoder.go b/pkg/bpf/decoder.go index ae6b8839a..45c192215 100644 --- a/pkg/bpf/decoder.go +++ b/pkg/bpf/decoder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bpf/decoder_test.go b/pkg/bpf/decoder_test.go index f093e1e41..8c4bdad21 100644 --- a/pkg/bpf/decoder_test.go +++ b/pkg/bpf/decoder_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bpf/input_bytes.go b/pkg/bpf/input_bytes.go index 745c0749b..86b216cfc 100644 --- a/pkg/bpf/input_bytes.go +++ b/pkg/bpf/input_bytes.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bpf/interpreter.go b/pkg/bpf/interpreter.go index 86c7add4d..86de523a2 100644 --- a/pkg/bpf/interpreter.go +++ b/pkg/bpf/interpreter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bpf/interpreter_test.go b/pkg/bpf/interpreter_test.go index c46a43991..67b00ffe3 100644 --- a/pkg/bpf/interpreter_test.go +++ b/pkg/bpf/interpreter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bpf/program_builder.go b/pkg/bpf/program_builder.go index b4ce228e1..fc9d27203 100644 --- a/pkg/bpf/program_builder.go +++ b/pkg/bpf/program_builder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/bpf/program_builder_test.go b/pkg/bpf/program_builder_test.go index 0e0b79d88..5b2ad67de 100644 --- a/pkg/bpf/program_builder_test.go +++ b/pkg/bpf/program_builder_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/compressio/compressio.go b/pkg/compressio/compressio.go index 4daaa82b6..8c14ccbfa 100644 --- a/pkg/compressio/compressio.go +++ b/pkg/compressio/compressio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/compressio/compressio_test.go b/pkg/compressio/compressio_test.go index 1bbabee79..86dc47e44 100644 --- a/pkg/compressio/compressio_test.go +++ b/pkg/compressio/compressio_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/control/client/client.go b/pkg/control/client/client.go index 0d0c9f148..3fec27846 100644 --- a/pkg/control/client/client.go +++ b/pkg/control/client/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/control/server/server.go b/pkg/control/server/server.go index c46b5d70b..1a15da1a8 100644 --- a/pkg/control/server/server.go +++ b/pkg/control/server/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/cpuid/cpu_amd64.s b/pkg/cpuid/cpu_amd64.s index 905c1d12e..ac80d3c8a 100644 --- a/pkg/cpuid/cpu_amd64.s +++ b/pkg/cpuid/cpu_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/cpuid/cpuid.go b/pkg/cpuid/cpuid.go index 61441150e..3eb2bcd2b 100644 --- a/pkg/cpuid/cpuid.go +++ b/pkg/cpuid/cpuid.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/cpuid/cpuid_parse_test.go b/pkg/cpuid/cpuid_parse_test.go index e8f87a10e..dd9969db4 100644 --- a/pkg/cpuid/cpuid_parse_test.go +++ b/pkg/cpuid/cpuid_parse_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/cpuid/cpuid_test.go b/pkg/cpuid/cpuid_test.go index 64ade1cbe..6ae14d2da 100644 --- a/pkg/cpuid/cpuid_test.go +++ b/pkg/cpuid/cpuid_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/dhcp/client.go b/pkg/dhcp/client.go index 2ba79be32..b7cde3819 100644 --- a/pkg/dhcp/client.go +++ b/pkg/dhcp/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/dhcp/dhcp.go b/pkg/dhcp/dhcp.go index 6945bcd35..f96ffd891 100644 --- a/pkg/dhcp/dhcp.go +++ b/pkg/dhcp/dhcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/dhcp/dhcp_string.go b/pkg/dhcp/dhcp_string.go index 8533895bd..29ce98593 100644 --- a/pkg/dhcp/dhcp_string.go +++ b/pkg/dhcp/dhcp_string.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/dhcp/dhcp_test.go b/pkg/dhcp/dhcp_test.go index e1d8ef603..751626bb0 100644 --- a/pkg/dhcp/dhcp_test.go +++ b/pkg/dhcp/dhcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/dhcp/server.go b/pkg/dhcp/server.go index 9549ff705..6a1972860 100644 --- a/pkg/dhcp/server.go +++ b/pkg/dhcp/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/eventchannel/event.go b/pkg/eventchannel/event.go index 41a7b5ed3..4c8ae573b 100644 --- a/pkg/eventchannel/event.go +++ b/pkg/eventchannel/event.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/eventchannel/event.proto b/pkg/eventchannel/event.proto index c1679c7e7..34468f072 100644 --- a/pkg/eventchannel/event.proto +++ b/pkg/eventchannel/event.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/fd/fd.go b/pkg/fd/fd.go index d40758c22..2785243a2 100644 --- a/pkg/fd/fd.go +++ b/pkg/fd/fd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/fd/fd_test.go b/pkg/fd/fd_test.go index 42bb3ef6c..5fb0ad47d 100644 --- a/pkg/fd/fd_test.go +++ b/pkg/fd/fd_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/fdnotifier/fdnotifier.go b/pkg/fdnotifier/fdnotifier.go index aa4906ca0..f0b028b0b 100644 --- a/pkg/fdnotifier/fdnotifier.go +++ b/pkg/fdnotifier/fdnotifier.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/fdnotifier/poll_unsafe.go b/pkg/fdnotifier/poll_unsafe.go index 05be9aeb5..bc5e0ac44 100644 --- a/pkg/fdnotifier/poll_unsafe.go +++ b/pkg/fdnotifier/poll_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/gate/gate.go b/pkg/gate/gate.go index 48122bf5a..bda6aae09 100644 --- a/pkg/gate/gate.go +++ b/pkg/gate/gate.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/gate/gate_test.go b/pkg/gate/gate_test.go index 95620fa8e..7467e7d07 100644 --- a/pkg/gate/gate_test.go +++ b/pkg/gate/gate_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/ilist/list.go b/pkg/ilist/list.go index 51c9b6df3..019caadca 100644 --- a/pkg/ilist/list.go +++ b/pkg/ilist/list.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/ilist/list_test.go b/pkg/ilist/list_test.go index f37946dc2..3f9abfb56 100644 --- a/pkg/ilist/list_test.go +++ b/pkg/ilist/list_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/linewriter/linewriter.go b/pkg/linewriter/linewriter.go index 5fbd4e779..cd6e4e2ce 100644 --- a/pkg/linewriter/linewriter.go +++ b/pkg/linewriter/linewriter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/linewriter/linewriter_test.go b/pkg/linewriter/linewriter_test.go index 9140ee6af..96dc7e6e0 100644 --- a/pkg/linewriter/linewriter_test.go +++ b/pkg/linewriter/linewriter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/log/glog.go b/pkg/log/glog.go index 24d5390d7..5732785b4 100644 --- a/pkg/log/glog.go +++ b/pkg/log/glog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/log/glog_unsafe.go b/pkg/log/glog_unsafe.go index bb06aa7d3..ea17ae349 100644 --- a/pkg/log/glog_unsafe.go +++ b/pkg/log/glog_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/log/json.go b/pkg/log/json.go index 96bd13d87..a278c8fc8 100644 --- a/pkg/log/json.go +++ b/pkg/log/json.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/log/json_k8s.go b/pkg/log/json_k8s.go index 9c2f8d2b7..c2c019915 100644 --- a/pkg/log/json_k8s.go +++ b/pkg/log/json_k8s.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/log/json_test.go b/pkg/log/json_test.go index b8c7a795e..f25224fe1 100644 --- a/pkg/log/json_test.go +++ b/pkg/log/json_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/log/log.go b/pkg/log/log.go index b8d456aae..7d563241e 100644 --- a/pkg/log/log.go +++ b/pkg/log/log.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/log/log_test.go b/pkg/log/log_test.go index a59d457dd..0634e7c1f 100644 --- a/pkg/log/log_test.go +++ b/pkg/log/log_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/metric/metric.go b/pkg/metric/metric.go index e5eb95f89..803709cc4 100644 --- a/pkg/metric/metric.go +++ b/pkg/metric/metric.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/metric/metric.proto b/pkg/metric/metric.proto index 917fda1ac..a2c2bd1ba 100644 --- a/pkg/metric/metric.proto +++ b/pkg/metric/metric.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/metric/metric_test.go b/pkg/metric/metric_test.go index 40034a589..b8b124c83 100644 --- a/pkg/metric/metric_test.go +++ b/pkg/metric/metric_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/buffer.go b/pkg/p9/buffer.go index b7bb14ef9..4c8c6555d 100644 --- a/pkg/p9/buffer.go +++ b/pkg/p9/buffer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/buffer_test.go b/pkg/p9/buffer_test.go index 18d55e5c0..a9c75f86b 100644 --- a/pkg/p9/buffer_test.go +++ b/pkg/p9/buffer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/client.go b/pkg/p9/client.go index 67887874a..2f9c716d0 100644 --- a/pkg/p9/client.go +++ b/pkg/p9/client.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/client_file.go b/pkg/p9/client_file.go index 992d1daf7..63c65129a 100644 --- a/pkg/p9/client_file.go +++ b/pkg/p9/client_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/client_test.go b/pkg/p9/client_test.go index f7145452d..fc49729d8 100644 --- a/pkg/p9/client_test.go +++ b/pkg/p9/client_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/file.go b/pkg/p9/file.go index 55ceb52e1..a52a0f3e7 100644 --- a/pkg/p9/file.go +++ b/pkg/p9/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/handlers.go b/pkg/p9/handlers.go index c1d1ac1e8..6da2ce4e3 100644 --- a/pkg/p9/handlers.go +++ b/pkg/p9/handlers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/local_server/local_server.go b/pkg/p9/local_server/local_server.go index 69b90c6cd..f4077a9d4 100644 --- a/pkg/p9/local_server/local_server.go +++ b/pkg/p9/local_server/local_server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/messages.go b/pkg/p9/messages.go index 97decd3cc..833defbd6 100644 --- a/pkg/p9/messages.go +++ b/pkg/p9/messages.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/messages_test.go b/pkg/p9/messages_test.go index 68395a396..10a0587cf 100644 --- a/pkg/p9/messages_test.go +++ b/pkg/p9/messages_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/p9.go b/pkg/p9/p9.go index 4ea9f2f9a..78c7d3f86 100644 --- a/pkg/p9/p9.go +++ b/pkg/p9/p9.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/p9_test.go b/pkg/p9/p9_test.go index 02498346c..8dda6cc64 100644 --- a/pkg/p9/p9_test.go +++ b/pkg/p9/p9_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/p9test/client_test.go b/pkg/p9/p9test/client_test.go index 242d81b95..e00dd03ab 100644 --- a/pkg/p9/p9test/client_test.go +++ b/pkg/p9/p9test/client_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/p9test/p9test.go b/pkg/p9/p9test/p9test.go index f9bacbf84..1c8eff200 100644 --- a/pkg/p9/p9test/p9test.go +++ b/pkg/p9/p9test/p9test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/path_tree.go b/pkg/p9/path_tree.go index 60b20578e..f37ad4ab2 100644 --- a/pkg/p9/path_tree.go +++ b/pkg/p9/path_tree.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/pool.go b/pkg/p9/pool.go index 34ed898e8..52de889e1 100644 --- a/pkg/p9/pool.go +++ b/pkg/p9/pool.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/pool_test.go b/pkg/p9/pool_test.go index 71052d8c4..e4746b8da 100644 --- a/pkg/p9/pool_test.go +++ b/pkg/p9/pool_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/server.go b/pkg/p9/server.go index 3ef151595..b2a86d8fa 100644 --- a/pkg/p9/server.go +++ b/pkg/p9/server.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/transport.go b/pkg/p9/transport.go index bafb377de..ef59077ff 100644 --- a/pkg/p9/transport.go +++ b/pkg/p9/transport.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/transport_test.go b/pkg/p9/transport_test.go index b7b7825bd..c833d1c9c 100644 --- a/pkg/p9/transport_test.go +++ b/pkg/p9/transport_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/version.go b/pkg/p9/version.go index ceb6fabbf..a36a499a1 100644 --- a/pkg/p9/version.go +++ b/pkg/p9/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/p9/version_test.go b/pkg/p9/version_test.go index c053614c9..291e8580e 100644 --- a/pkg/p9/version_test.go +++ b/pkg/p9/version_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/rand/rand.go b/pkg/rand/rand.go index 593a14380..a2714784d 100644 --- a/pkg/rand/rand.go +++ b/pkg/rand/rand.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/rand/rand_linux.go b/pkg/rand/rand_linux.go index 7ebe8f3b0..2b92db3e6 100644 --- a/pkg/rand/rand_linux.go +++ b/pkg/rand/rand_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/refs/refcounter.go b/pkg/refs/refcounter.go index 8f08c74c7..20f515391 100644 --- a/pkg/refs/refcounter.go +++ b/pkg/refs/refcounter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/refs/refcounter_state.go b/pkg/refs/refcounter_state.go index 136f06fbf..7c99fd2b5 100644 --- a/pkg/refs/refcounter_state.go +++ b/pkg/refs/refcounter_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/refs/refcounter_test.go b/pkg/refs/refcounter_test.go index abaa87453..ffd3d3f07 100644 --- a/pkg/refs/refcounter_test.go +++ b/pkg/refs/refcounter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/seccomp/seccomp.go b/pkg/seccomp/seccomp.go index e113f3574..50c9409e4 100644 --- a/pkg/seccomp/seccomp.go +++ b/pkg/seccomp/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/seccomp/seccomp_rules.go b/pkg/seccomp/seccomp_rules.go index a9278c64b..29eec8db1 100644 --- a/pkg/seccomp/seccomp_rules.go +++ b/pkg/seccomp/seccomp_rules.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/seccomp/seccomp_test.go b/pkg/seccomp/seccomp_test.go index 11ed90eb4..47ecac6f7 100644 --- a/pkg/seccomp/seccomp_test.go +++ b/pkg/seccomp/seccomp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/seccomp/seccomp_test_victim.go b/pkg/seccomp/seccomp_test_victim.go index dd5ed0041..afc2f755f 100644 --- a/pkg/seccomp/seccomp_test_victim.go +++ b/pkg/seccomp/seccomp_test_victim.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/seccomp/seccomp_unsafe.go b/pkg/seccomp/seccomp_unsafe.go index a31c6471d..ccd40d9db 100644 --- a/pkg/seccomp/seccomp_unsafe.go +++ b/pkg/seccomp/seccomp_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/secio/full_reader.go b/pkg/secio/full_reader.go index 90b1772a7..aed2564bd 100644 --- a/pkg/secio/full_reader.go +++ b/pkg/secio/full_reader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/secio/secio.go b/pkg/secio/secio.go index e5f74a497..b43226035 100644 --- a/pkg/secio/secio.go +++ b/pkg/secio/secio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/secio/secio_test.go b/pkg/secio/secio_test.go index 8304c4f74..d1d905187 100644 --- a/pkg/secio/secio_test.go +++ b/pkg/secio/secio_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/segment/range.go b/pkg/segment/range.go index 057bcd7ff..4d4aeffef 100644 --- a/pkg/segment/range.go +++ b/pkg/segment/range.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/segment/set.go b/pkg/segment/set.go index 74a916ea3..982eb3fdd 100644 --- a/pkg/segment/set.go +++ b/pkg/segment/set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/segment/set_state.go b/pkg/segment/set_state.go index b86e1b75f..76de92591 100644 --- a/pkg/segment/set_state.go +++ b/pkg/segment/set_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/segment/test/segment_test.go b/pkg/segment/test/segment_test.go index 0825105db..f19a005f3 100644 --- a/pkg/segment/test/segment_test.go +++ b/pkg/segment/test/segment_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/segment/test/set_functions.go b/pkg/segment/test/set_functions.go index 41f649011..bcddb39bb 100644 --- a/pkg/segment/test/set_functions.go +++ b/pkg/segment/test/set_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/aligned.go b/pkg/sentry/arch/aligned.go index c88c034f6..df01a903d 100644 --- a/pkg/sentry/arch/aligned.go +++ b/pkg/sentry/arch/aligned.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/arch.go b/pkg/sentry/arch/arch.go index 16d8eb2b2..53f0c9018 100644 --- a/pkg/sentry/arch/arch.go +++ b/pkg/sentry/arch/arch.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/arch_amd64.go b/pkg/sentry/arch/arch_amd64.go index 7ec2f2c84..135c2ee1f 100644 --- a/pkg/sentry/arch/arch_amd64.go +++ b/pkg/sentry/arch/arch_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/arch_amd64.s b/pkg/sentry/arch/arch_amd64.s index fa9857df7..bd61402cf 100644 --- a/pkg/sentry/arch/arch_amd64.s +++ b/pkg/sentry/arch/arch_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/arch_state_x86.go b/pkg/sentry/arch/arch_state_x86.go index 01949049d..bb52d8db0 100644 --- a/pkg/sentry/arch/arch_state_x86.go +++ b/pkg/sentry/arch/arch_state_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/arch_x86.go b/pkg/sentry/arch/arch_x86.go index 4305fe2cb..4d167ce98 100644 --- a/pkg/sentry/arch/arch_x86.go +++ b/pkg/sentry/arch/arch_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/auxv.go b/pkg/sentry/arch/auxv.go index 5df65a691..80c923103 100644 --- a/pkg/sentry/arch/auxv.go +++ b/pkg/sentry/arch/auxv.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/registers.proto b/pkg/sentry/arch/registers.proto index f4c2f7043..9dc83e241 100644 --- a/pkg/sentry/arch/registers.proto +++ b/pkg/sentry/arch/registers.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/signal_act.go b/pkg/sentry/arch/signal_act.go index ad098c746..f9ca2e74e 100644 --- a/pkg/sentry/arch/signal_act.go +++ b/pkg/sentry/arch/signal_act.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/signal_amd64.go b/pkg/sentry/arch/signal_amd64.go index 7f76eba27..aa030fd70 100644 --- a/pkg/sentry/arch/signal_amd64.go +++ b/pkg/sentry/arch/signal_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/signal_info.go b/pkg/sentry/arch/signal_info.go index fa0ecbec5..f93ee8b46 100644 --- a/pkg/sentry/arch/signal_info.go +++ b/pkg/sentry/arch/signal_info.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/signal_stack.go b/pkg/sentry/arch/signal_stack.go index c02ae3b7c..a442f9fdc 100644 --- a/pkg/sentry/arch/signal_stack.go +++ b/pkg/sentry/arch/signal_stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/stack.go b/pkg/sentry/arch/stack.go index 2e33ccdf5..7e6324e82 100644 --- a/pkg/sentry/arch/stack.go +++ b/pkg/sentry/arch/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/arch/syscalls_amd64.go b/pkg/sentry/arch/syscalls_amd64.go index 47c31d4b9..8b4f23007 100644 --- a/pkg/sentry/arch/syscalls_amd64.go +++ b/pkg/sentry/arch/syscalls_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/context/context.go b/pkg/sentry/context/context.go index eefc3e1b4..d70f3a5c3 100644 --- a/pkg/sentry/context/context.go +++ b/pkg/sentry/context/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/context/contexttest/contexttest.go b/pkg/sentry/context/contexttest/contexttest.go index a29087775..a42038711 100644 --- a/pkg/sentry/context/contexttest/contexttest.go +++ b/pkg/sentry/context/contexttest/contexttest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/control/control.go b/pkg/sentry/control/control.go index 32d30b6ea..6060b9b4f 100644 --- a/pkg/sentry/control/control.go +++ b/pkg/sentry/control/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/control/pprof.go b/pkg/sentry/control/pprof.go index 1af092af3..94ed149f2 100644 --- a/pkg/sentry/control/pprof.go +++ b/pkg/sentry/control/pprof.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/control/proc.go b/pkg/sentry/control/proc.go index aca2267a7..f7f02a3e1 100644 --- a/pkg/sentry/control/proc.go +++ b/pkg/sentry/control/proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/control/proc_test.go b/pkg/sentry/control/proc_test.go index 5d52cd829..b7895d03c 100644 --- a/pkg/sentry/control/proc_test.go +++ b/pkg/sentry/control/proc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/control/state.go b/pkg/sentry/control/state.go index b6bbf69fa..11efcaba1 100644 --- a/pkg/sentry/control/state.go +++ b/pkg/sentry/control/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/device/device.go b/pkg/sentry/device/device.go index ae4fa1d93..458d03b30 100644 --- a/pkg/sentry/device/device.go +++ b/pkg/sentry/device/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/device/device_test.go b/pkg/sentry/device/device_test.go index 5d8805c2f..e3f51ce4f 100644 --- a/pkg/sentry/device/device_test.go +++ b/pkg/sentry/device/device_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/anon/anon.go b/pkg/sentry/fs/anon/anon.go index a5e8c4f0d..a6ea8b9e7 100644 --- a/pkg/sentry/fs/anon/anon.go +++ b/pkg/sentry/fs/anon/anon.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/anon/device.go b/pkg/sentry/fs/anon/device.go index 2d1249299..5927bd11e 100644 --- a/pkg/sentry/fs/anon/device.go +++ b/pkg/sentry/fs/anon/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go index 1f61c5711..b53746519 100644 --- a/pkg/sentry/fs/ashmem/area.go +++ b/pkg/sentry/fs/ashmem/area.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ashmem/device.go b/pkg/sentry/fs/ashmem/device.go index 5369d1b0d..5e005bc2e 100644 --- a/pkg/sentry/fs/ashmem/device.go +++ b/pkg/sentry/fs/ashmem/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ashmem/pin_board.go b/pkg/sentry/fs/ashmem/pin_board.go index 7c997f533..bdf23b371 100644 --- a/pkg/sentry/fs/ashmem/pin_board.go +++ b/pkg/sentry/fs/ashmem/pin_board.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ashmem/pin_board_test.go b/pkg/sentry/fs/ashmem/pin_board_test.go index 736e628dc..24f5d86d6 100644 --- a/pkg/sentry/fs/ashmem/pin_board_test.go +++ b/pkg/sentry/fs/ashmem/pin_board_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/attr.go b/pkg/sentry/fs/attr.go index 3523b068a..591e35e6a 100644 --- a/pkg/sentry/fs/attr.go +++ b/pkg/sentry/fs/attr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go index d9f1559de..acbbd5466 100644 --- a/pkg/sentry/fs/binder/binder.go +++ b/pkg/sentry/fs/binder/binder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/context.go b/pkg/sentry/fs/context.go index 4869428a8..c80ea0175 100644 --- a/pkg/sentry/fs/context.go +++ b/pkg/sentry/fs/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/copy_up.go b/pkg/sentry/fs/copy_up.go index ba69e718d..ee2d3d115 100644 --- a/pkg/sentry/fs/copy_up.go +++ b/pkg/sentry/fs/copy_up.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/copy_up_test.go b/pkg/sentry/fs/copy_up_test.go index 98a0b7638..54810afca 100644 --- a/pkg/sentry/fs/copy_up_test.go +++ b/pkg/sentry/fs/copy_up_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dentry.go b/pkg/sentry/fs/dentry.go index 29fb155a4..fe656cc24 100644 --- a/pkg/sentry/fs/dentry.go +++ b/pkg/sentry/fs/dentry.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dev/dev.go b/pkg/sentry/fs/dev/dev.go index fbc750a71..34ac01173 100644 --- a/pkg/sentry/fs/dev/dev.go +++ b/pkg/sentry/fs/dev/dev.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dev/device.go b/pkg/sentry/fs/dev/device.go index 3cecdf6e2..9f4e41fc9 100644 --- a/pkg/sentry/fs/dev/device.go +++ b/pkg/sentry/fs/dev/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dev/fs.go b/pkg/sentry/fs/dev/fs.go index cf4e7d00f..6096a40f8 100644 --- a/pkg/sentry/fs/dev/fs.go +++ b/pkg/sentry/fs/dev/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dev/full.go b/pkg/sentry/fs/dev/full.go index 82da9aae9..6b11afa44 100644 --- a/pkg/sentry/fs/dev/full.go +++ b/pkg/sentry/fs/dev/full.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dev/null.go b/pkg/sentry/fs/dev/null.go index 5d306d352..069212b6d 100644 --- a/pkg/sentry/fs/dev/null.go +++ b/pkg/sentry/fs/dev/null.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dev/random.go b/pkg/sentry/fs/dev/random.go index ffd5cf6c3..de0f3e5e5 100644 --- a/pkg/sentry/fs/dev/random.go +++ b/pkg/sentry/fs/dev/random.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dirent.go b/pkg/sentry/fs/dirent.go index 54fc11fe1..c0bc261a2 100644 --- a/pkg/sentry/fs/dirent.go +++ b/pkg/sentry/fs/dirent.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dirent_cache.go b/pkg/sentry/fs/dirent_cache.go index d26a06971..71f2d11de 100644 --- a/pkg/sentry/fs/dirent_cache.go +++ b/pkg/sentry/fs/dirent_cache.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dirent_cache_limiter.go b/pkg/sentry/fs/dirent_cache_limiter.go index 024c7b2d5..ebb80bd50 100644 --- a/pkg/sentry/fs/dirent_cache_limiter.go +++ b/pkg/sentry/fs/dirent_cache_limiter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dirent_cache_test.go b/pkg/sentry/fs/dirent_cache_test.go index 93e8d415f..395c879f5 100644 --- a/pkg/sentry/fs/dirent_cache_test.go +++ b/pkg/sentry/fs/dirent_cache_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dirent_refs_test.go b/pkg/sentry/fs/dirent_refs_test.go index 325404e27..db88d850e 100644 --- a/pkg/sentry/fs/dirent_refs_test.go +++ b/pkg/sentry/fs/dirent_refs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/dirent_state.go b/pkg/sentry/fs/dirent_state.go index 5cf151dab..18652b809 100644 --- a/pkg/sentry/fs/dirent_state.go +++ b/pkg/sentry/fs/dirent_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go index 98483ab68..95e66ea8d 100644 --- a/pkg/sentry/fs/fdpipe/pipe.go +++ b/pkg/sentry/fs/fdpipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fdpipe/pipe_opener.go b/pkg/sentry/fs/fdpipe/pipe_opener.go index 92ab6ff0e..0cabe2e18 100644 --- a/pkg/sentry/fs/fdpipe/pipe_opener.go +++ b/pkg/sentry/fs/fdpipe/pipe_opener.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fdpipe/pipe_opener_test.go b/pkg/sentry/fs/fdpipe/pipe_opener_test.go index 69516e048..8c8b1b40c 100644 --- a/pkg/sentry/fs/fdpipe/pipe_opener_test.go +++ b/pkg/sentry/fs/fdpipe/pipe_opener_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fdpipe/pipe_state.go b/pkg/sentry/fs/fdpipe/pipe_state.go index 4395666ad..8b347aa11 100644 --- a/pkg/sentry/fs/fdpipe/pipe_state.go +++ b/pkg/sentry/fs/fdpipe/pipe_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fdpipe/pipe_test.go b/pkg/sentry/fs/fdpipe/pipe_test.go index 7e3ee5257..b59a6aa0e 100644 --- a/pkg/sentry/fs/fdpipe/pipe_test.go +++ b/pkg/sentry/fs/fdpipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go index 5d5026661..62b35dabc 100644 --- a/pkg/sentry/fs/file.go +++ b/pkg/sentry/fs/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/file_operations.go b/pkg/sentry/fs/file_operations.go index e0fa5135f..ab0acb6eb 100644 --- a/pkg/sentry/fs/file_operations.go +++ b/pkg/sentry/fs/file_operations.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go index 6e680f0a4..948ce9c6f 100644 --- a/pkg/sentry/fs/file_overlay.go +++ b/pkg/sentry/fs/file_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/file_overlay_test.go b/pkg/sentry/fs/file_overlay_test.go index a4ac58763..6a2b8007c 100644 --- a/pkg/sentry/fs/file_overlay_test.go +++ b/pkg/sentry/fs/file_overlay_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/file_state.go b/pkg/sentry/fs/file_state.go index 1c3bae3e8..523182d59 100644 --- a/pkg/sentry/fs/file_state.go +++ b/pkg/sentry/fs/file_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/file_test.go b/pkg/sentry/fs/file_test.go index f3ed9a70b..d867a0257 100644 --- a/pkg/sentry/fs/file_test.go +++ b/pkg/sentry/fs/file_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/filesystems.go b/pkg/sentry/fs/filesystems.go index a6b27c402..acd84dfcc 100644 --- a/pkg/sentry/fs/filesystems.go +++ b/pkg/sentry/fs/filesystems.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/filetest/filetest.go b/pkg/sentry/fs/filetest/filetest.go index 388a1ce36..f6b827800 100644 --- a/pkg/sentry/fs/filetest/filetest.go +++ b/pkg/sentry/fs/filetest/filetest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/flags.go b/pkg/sentry/fs/flags.go index bf2a20b33..5c8cb773f 100644 --- a/pkg/sentry/fs/flags.go +++ b/pkg/sentry/fs/flags.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fs.go b/pkg/sentry/fs/fs.go index 119689776..632055cce 100644 --- a/pkg/sentry/fs/fs.go +++ b/pkg/sentry/fs/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/dirty_set.go b/pkg/sentry/fs/fsutil/dirty_set.go index 5add16ac4..9cd196d7d 100644 --- a/pkg/sentry/fs/fsutil/dirty_set.go +++ b/pkg/sentry/fs/fsutil/dirty_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/dirty_set_test.go b/pkg/sentry/fs/fsutil/dirty_set_test.go index f5c9d9215..d9c68baa3 100644 --- a/pkg/sentry/fs/fsutil/dirty_set_test.go +++ b/pkg/sentry/fs/fsutil/dirty_set_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index 42afdd11c..e355d8594 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/file_range_set.go b/pkg/sentry/fs/fsutil/file_range_set.go index 32ebf64ff..b5ac6c71c 100644 --- a/pkg/sentry/fs/fsutil/file_range_set.go +++ b/pkg/sentry/fs/fsutil/file_range_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/frame_ref_set.go b/pkg/sentry/fs/fsutil/frame_ref_set.go index b6e783614..6565c28c8 100644 --- a/pkg/sentry/fs/fsutil/frame_ref_set.go +++ b/pkg/sentry/fs/fsutil/frame_ref_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/fsutil.go b/pkg/sentry/fs/fsutil/fsutil.go index 319c4841b..c9587b1d9 100644 --- a/pkg/sentry/fs/fsutil/fsutil.go +++ b/pkg/sentry/fs/fsutil/fsutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper.go b/pkg/sentry/fs/fsutil/host_file_mapper.go index 9599665f0..2bdfc0db6 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper_state.go b/pkg/sentry/fs/fsutil/host_file_mapper_state.go index bbd15b30b..576d2a3df 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper_state.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go b/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go index 86df76822..7167be263 100644 --- a/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go +++ b/pkg/sentry/fs/fsutil/host_file_mapper_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/host_mappable.go b/pkg/sentry/fs/fsutil/host_mappable.go index 4a182baa1..28686f3b3 100644 --- a/pkg/sentry/fs/fsutil/host_mappable.go +++ b/pkg/sentry/fs/fsutil/host_mappable.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go index 468171a9b..b6366d906 100644 --- a/pkg/sentry/fs/fsutil/inode.go +++ b/pkg/sentry/fs/fsutil/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/inode_cached.go b/pkg/sentry/fs/fsutil/inode_cached.go index ba33b9912..919d2534c 100644 --- a/pkg/sentry/fs/fsutil/inode_cached.go +++ b/pkg/sentry/fs/fsutil/inode_cached.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/fsutil/inode_cached_test.go b/pkg/sentry/fs/fsutil/inode_cached_test.go index 2a8a1639c..661ec41f6 100644 --- a/pkg/sentry/fs/fsutil/inode_cached_test.go +++ b/pkg/sentry/fs/fsutil/inode_cached_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/attr.go b/pkg/sentry/fs/gofer/attr.go index 98700d014..c572f3396 100644 --- a/pkg/sentry/fs/gofer/attr.go +++ b/pkg/sentry/fs/gofer/attr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/cache_policy.go b/pkg/sentry/fs/gofer/cache_policy.go index 51c573aef..35cd0c1d6 100644 --- a/pkg/sentry/fs/gofer/cache_policy.go +++ b/pkg/sentry/fs/gofer/cache_policy.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/context_file.go b/pkg/sentry/fs/gofer/context_file.go index 455953237..d512afefc 100644 --- a/pkg/sentry/fs/gofer/context_file.go +++ b/pkg/sentry/fs/gofer/context_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/device.go b/pkg/sentry/fs/gofer/device.go index 52c5acf48..1de6c247c 100644 --- a/pkg/sentry/fs/gofer/device.go +++ b/pkg/sentry/fs/gofer/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go index 35caa42cd..bc2be546e 100644 --- a/pkg/sentry/fs/gofer/file.go +++ b/pkg/sentry/fs/gofer/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/file_state.go b/pkg/sentry/fs/gofer/file_state.go index d0c64003c..31264e065 100644 --- a/pkg/sentry/fs/gofer/file_state.go +++ b/pkg/sentry/fs/gofer/file_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/fs.go b/pkg/sentry/fs/gofer/fs.go index adff0abac..6ab89fcc2 100644 --- a/pkg/sentry/fs/gofer/fs.go +++ b/pkg/sentry/fs/gofer/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/gofer_test.go b/pkg/sentry/fs/gofer/gofer_test.go index 36201f017..29d34da7e 100644 --- a/pkg/sentry/fs/gofer/gofer_test.go +++ b/pkg/sentry/fs/gofer/gofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/handles.go b/pkg/sentry/fs/gofer/handles.go index 0b33e80c3..c7098cd36 100644 --- a/pkg/sentry/fs/gofer/handles.go +++ b/pkg/sentry/fs/gofer/handles.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/inode.go b/pkg/sentry/fs/gofer/inode.go index 1181a24cc..f6f20844d 100644 --- a/pkg/sentry/fs/gofer/inode.go +++ b/pkg/sentry/fs/gofer/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/inode_state.go b/pkg/sentry/fs/gofer/inode_state.go index 44d76ba9f..ac22ee4b1 100644 --- a/pkg/sentry/fs/gofer/inode_state.go +++ b/pkg/sentry/fs/gofer/inode_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/path.go b/pkg/sentry/fs/gofer/path.go index 8ae33d286..4cbf9e9d9 100644 --- a/pkg/sentry/fs/gofer/path.go +++ b/pkg/sentry/fs/gofer/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go index 4ed688ce5..4cb65e7c6 100644 --- a/pkg/sentry/fs/gofer/session.go +++ b/pkg/sentry/fs/gofer/session.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/session_state.go b/pkg/sentry/fs/gofer/session_state.go index b1f299be5..68fbf3417 100644 --- a/pkg/sentry/fs/gofer/session_state.go +++ b/pkg/sentry/fs/gofer/session_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/socket.go b/pkg/sentry/fs/gofer/socket.go index ce6d3d5c3..cbd5b9a84 100644 --- a/pkg/sentry/fs/gofer/socket.go +++ b/pkg/sentry/fs/gofer/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/gofer/util.go b/pkg/sentry/fs/gofer/util.go index 1a759370d..d0e1096ce 100644 --- a/pkg/sentry/fs/gofer/util.go +++ b/pkg/sentry/fs/gofer/util.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/control.go b/pkg/sentry/fs/host/control.go index 0753640a2..480f0c8f4 100644 --- a/pkg/sentry/fs/host/control.go +++ b/pkg/sentry/fs/host/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/descriptor.go b/pkg/sentry/fs/host/descriptor.go index 554e1693a..ffcd57a94 100644 --- a/pkg/sentry/fs/host/descriptor.go +++ b/pkg/sentry/fs/host/descriptor.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/descriptor_state.go b/pkg/sentry/fs/host/descriptor_state.go index 530c0109f..8167390a9 100644 --- a/pkg/sentry/fs/host/descriptor_state.go +++ b/pkg/sentry/fs/host/descriptor_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/descriptor_test.go b/pkg/sentry/fs/host/descriptor_test.go index 5dec84ab2..ff08e43af 100644 --- a/pkg/sentry/fs/host/descriptor_test.go +++ b/pkg/sentry/fs/host/descriptor_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/device.go b/pkg/sentry/fs/host/device.go index b5adedf44..055024c44 100644 --- a/pkg/sentry/fs/host/device.go +++ b/pkg/sentry/fs/host/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go index 2a8f285ff..82e2ae3b9 100644 --- a/pkg/sentry/fs/host/file.go +++ b/pkg/sentry/fs/host/file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/fs.go b/pkg/sentry/fs/host/fs.go index de349a41a..b1b8dc0b6 100644 --- a/pkg/sentry/fs/host/fs.go +++ b/pkg/sentry/fs/host/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/fs_test.go b/pkg/sentry/fs/host/fs_test.go index c83b29a16..16c89ddf1 100644 --- a/pkg/sentry/fs/host/fs_test.go +++ b/pkg/sentry/fs/host/fs_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go index 69c648f67..20e077f77 100644 --- a/pkg/sentry/fs/host/inode.go +++ b/pkg/sentry/fs/host/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/inode_state.go b/pkg/sentry/fs/host/inode_state.go index b7c1a9581..26cc755bc 100644 --- a/pkg/sentry/fs/host/inode_state.go +++ b/pkg/sentry/fs/host/inode_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/inode_test.go b/pkg/sentry/fs/host/inode_test.go index 9f1561bd5..ad1878b5a 100644 --- a/pkg/sentry/fs/host/inode_test.go +++ b/pkg/sentry/fs/host/inode_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/ioctl_unsafe.go b/pkg/sentry/fs/host/ioctl_unsafe.go index 175dca613..b5a85c4d9 100644 --- a/pkg/sentry/fs/host/ioctl_unsafe.go +++ b/pkg/sentry/fs/host/ioctl_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/socket.go b/pkg/sentry/fs/host/socket.go index be2c3581f..3034e9441 100644 --- a/pkg/sentry/fs/host/socket.go +++ b/pkg/sentry/fs/host/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/socket_iovec.go b/pkg/sentry/fs/host/socket_iovec.go index d4ce4a8c1..5efbb3ae8 100644 --- a/pkg/sentry/fs/host/socket_iovec.go +++ b/pkg/sentry/fs/host/socket_iovec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/socket_state.go b/pkg/sentry/fs/host/socket_state.go index 2932c1f16..5676c451a 100644 --- a/pkg/sentry/fs/host/socket_state.go +++ b/pkg/sentry/fs/host/socket_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/socket_test.go b/pkg/sentry/fs/host/socket_test.go index 83e8e1b3c..cc760a7e1 100644 --- a/pkg/sentry/fs/host/socket_test.go +++ b/pkg/sentry/fs/host/socket_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/socket_unsafe.go b/pkg/sentry/fs/host/socket_unsafe.go index f35e2492d..8873705c0 100644 --- a/pkg/sentry/fs/host/socket_unsafe.go +++ b/pkg/sentry/fs/host/socket_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/tty.go b/pkg/sentry/fs/host/tty.go index c5cb75df7..e45b339f5 100644 --- a/pkg/sentry/fs/host/tty.go +++ b/pkg/sentry/fs/host/tty.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/util.go b/pkg/sentry/fs/host/util.go index 40c450660..94ff7708e 100644 --- a/pkg/sentry/fs/host/util.go +++ b/pkg/sentry/fs/host/util.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/util_unsafe.go b/pkg/sentry/fs/host/util_unsafe.go index a8721d197..b95a57c3f 100644 --- a/pkg/sentry/fs/host/util_unsafe.go +++ b/pkg/sentry/fs/host/util_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/host/wait_test.go b/pkg/sentry/fs/host/wait_test.go index 9ca8c399f..afcb74724 100644 --- a/pkg/sentry/fs/host/wait_test.go +++ b/pkg/sentry/fs/host/wait_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/inode.go b/pkg/sentry/fs/inode.go index fe411a766..d764ef93d 100644 --- a/pkg/sentry/fs/inode.go +++ b/pkg/sentry/fs/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/inode_inotify.go b/pkg/sentry/fs/inode_inotify.go index d2b653bc7..0f2a66a79 100644 --- a/pkg/sentry/fs/inode_inotify.go +++ b/pkg/sentry/fs/inode_inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/inode_operations.go b/pkg/sentry/fs/inode_operations.go index ff8b75f31..ac287e1e4 100644 --- a/pkg/sentry/fs/inode_operations.go +++ b/pkg/sentry/fs/inode_operations.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/inode_overlay.go b/pkg/sentry/fs/inode_overlay.go index bda3e1861..3d015328e 100644 --- a/pkg/sentry/fs/inode_overlay.go +++ b/pkg/sentry/fs/inode_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/inode_overlay_test.go b/pkg/sentry/fs/inode_overlay_test.go index fa8accf6c..66b3da2d0 100644 --- a/pkg/sentry/fs/inode_overlay_test.go +++ b/pkg/sentry/fs/inode_overlay_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/inotify.go b/pkg/sentry/fs/inotify.go index 59fa662f3..2652582c3 100644 --- a/pkg/sentry/fs/inotify.go +++ b/pkg/sentry/fs/inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/inotify_event.go b/pkg/sentry/fs/inotify_event.go index f09928b68..d52f956e4 100644 --- a/pkg/sentry/fs/inotify_event.go +++ b/pkg/sentry/fs/inotify_event.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/inotify_watch.go b/pkg/sentry/fs/inotify_watch.go index d33e7e498..a0b488467 100644 --- a/pkg/sentry/fs/inotify_watch.go +++ b/pkg/sentry/fs/inotify_watch.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/lock/lock.go b/pkg/sentry/fs/lock/lock.go index 5ff800d2d..f2aee4512 100644 --- a/pkg/sentry/fs/lock/lock.go +++ b/pkg/sentry/fs/lock/lock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/lock/lock_range_test.go b/pkg/sentry/fs/lock/lock_range_test.go index b0ab882b9..6221199d1 100644 --- a/pkg/sentry/fs/lock/lock_range_test.go +++ b/pkg/sentry/fs/lock/lock_range_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/lock/lock_set_functions.go b/pkg/sentry/fs/lock/lock_set_functions.go index 395592a4b..8a3ace0c1 100644 --- a/pkg/sentry/fs/lock/lock_set_functions.go +++ b/pkg/sentry/fs/lock/lock_set_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/lock/lock_test.go b/pkg/sentry/fs/lock/lock_test.go index 67fa4b1dd..ba002aeb7 100644 --- a/pkg/sentry/fs/lock/lock_test.go +++ b/pkg/sentry/fs/lock/lock_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/mock.go b/pkg/sentry/fs/mock.go index 118e30f63..cf359a1f1 100644 --- a/pkg/sentry/fs/mock.go +++ b/pkg/sentry/fs/mock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/mount.go b/pkg/sentry/fs/mount.go index 4d1693204..a169ea4c9 100644 --- a/pkg/sentry/fs/mount.go +++ b/pkg/sentry/fs/mount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/mount_overlay.go b/pkg/sentry/fs/mount_overlay.go index fb60a1aec..535f812c8 100644 --- a/pkg/sentry/fs/mount_overlay.go +++ b/pkg/sentry/fs/mount_overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/mount_test.go b/pkg/sentry/fs/mount_test.go index d7605b2c9..9f7fbeff2 100644 --- a/pkg/sentry/fs/mount_test.go +++ b/pkg/sentry/fs/mount_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/mounts.go b/pkg/sentry/fs/mounts.go index f6f7be0aa..01eb4607e 100644 --- a/pkg/sentry/fs/mounts.go +++ b/pkg/sentry/fs/mounts.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/mounts_test.go b/pkg/sentry/fs/mounts_test.go index 54000614f..56d726dd1 100644 --- a/pkg/sentry/fs/mounts_test.go +++ b/pkg/sentry/fs/mounts_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/offset.go b/pkg/sentry/fs/offset.go index 38aee765a..3f68da149 100644 --- a/pkg/sentry/fs/offset.go +++ b/pkg/sentry/fs/offset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/overlay.go b/pkg/sentry/fs/overlay.go index f3e2d5cbe..db89a5f70 100644 --- a/pkg/sentry/fs/overlay.go +++ b/pkg/sentry/fs/overlay.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/path.go b/pkg/sentry/fs/path.go index 52139b648..e4dc02dbb 100644 --- a/pkg/sentry/fs/path.go +++ b/pkg/sentry/fs/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/path_test.go b/pkg/sentry/fs/path_test.go index 4ba1498f6..e6f57ebba 100644 --- a/pkg/sentry/fs/path_test.go +++ b/pkg/sentry/fs/path_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/cpuinfo.go b/pkg/sentry/fs/proc/cpuinfo.go index f756c45bf..15031234e 100644 --- a/pkg/sentry/fs/proc/cpuinfo.go +++ b/pkg/sentry/fs/proc/cpuinfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/device/device.go b/pkg/sentry/fs/proc/device/device.go index 04b687bcf..0de466c73 100644 --- a/pkg/sentry/fs/proc/device/device.go +++ b/pkg/sentry/fs/proc/device/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/exec_args.go b/pkg/sentry/fs/proc/exec_args.go index fc21dfbbd..d49dad685 100644 --- a/pkg/sentry/fs/proc/exec_args.go +++ b/pkg/sentry/fs/proc/exec_args.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/fds.go b/pkg/sentry/fs/proc/fds.go index f2329e623..744b31c74 100644 --- a/pkg/sentry/fs/proc/fds.go +++ b/pkg/sentry/fs/proc/fds.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/filesystems.go b/pkg/sentry/fs/proc/filesystems.go index c050a00be..7bb081d0e 100644 --- a/pkg/sentry/fs/proc/filesystems.go +++ b/pkg/sentry/fs/proc/filesystems.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/fs.go b/pkg/sentry/fs/proc/fs.go index 666a2d054..7c5f8484a 100644 --- a/pkg/sentry/fs/proc/fs.go +++ b/pkg/sentry/fs/proc/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/inode.go b/pkg/sentry/fs/proc/inode.go index 8dde2ea46..b03807043 100644 --- a/pkg/sentry/fs/proc/inode.go +++ b/pkg/sentry/fs/proc/inode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/loadavg.go b/pkg/sentry/fs/proc/loadavg.go index 3ee0e570a..2dfe7089a 100644 --- a/pkg/sentry/fs/proc/loadavg.go +++ b/pkg/sentry/fs/proc/loadavg.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/meminfo.go b/pkg/sentry/fs/proc/meminfo.go index 75cbf3e77..d2b9b92c7 100644 --- a/pkg/sentry/fs/proc/meminfo.go +++ b/pkg/sentry/fs/proc/meminfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/mounts.go b/pkg/sentry/fs/proc/mounts.go index fe62b167b..37ed30724 100644 --- a/pkg/sentry/fs/proc/mounts.go +++ b/pkg/sentry/fs/proc/mounts.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go index d24b2d370..4a107c739 100644 --- a/pkg/sentry/fs/proc/net.go +++ b/pkg/sentry/fs/proc/net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/net_test.go b/pkg/sentry/fs/proc/net_test.go index 94677cc1d..9aed5fdca 100644 --- a/pkg/sentry/fs/proc/net_test.go +++ b/pkg/sentry/fs/proc/net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/proc.go b/pkg/sentry/fs/proc/proc.go index 64e1e1998..196fa5128 100644 --- a/pkg/sentry/fs/proc/proc.go +++ b/pkg/sentry/fs/proc/proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go index 81f64a28b..db53686f6 100644 --- a/pkg/sentry/fs/proc/rpcinet_proc.go +++ b/pkg/sentry/fs/proc/rpcinet_proc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/seqfile/seqfile.go b/pkg/sentry/fs/proc/seqfile/seqfile.go index 0a0eb45e2..10ea1f55d 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/seqfile/seqfile_test.go b/pkg/sentry/fs/proc/seqfile/seqfile_test.go index 35403ab7f..c4de565eb 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile_test.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/stat.go b/pkg/sentry/fs/proc/stat.go index 18bd8e9b6..397f9ec6b 100644 --- a/pkg/sentry/fs/proc/stat.go +++ b/pkg/sentry/fs/proc/stat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go index a7bc9198e..b889ed625 100644 --- a/pkg/sentry/fs/proc/sys.go +++ b/pkg/sentry/fs/proc/sys.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index 0ce77f04f..e49794a48 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/sys_net_state.go b/pkg/sentry/fs/proc/sys_net_state.go index 5f481a1cf..6eba709c6 100644 --- a/pkg/sentry/fs/proc/sys_net_state.go +++ b/pkg/sentry/fs/proc/sys_net_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/sys_net_test.go b/pkg/sentry/fs/proc/sys_net_test.go index ea0d94fce..78135ba13 100644 --- a/pkg/sentry/fs/proc/sys_net_test.go +++ b/pkg/sentry/fs/proc/sys_net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go index 9f65a8337..0f400e80f 100644 --- a/pkg/sentry/fs/proc/task.go +++ b/pkg/sentry/fs/proc/task.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/uid_gid_map.go b/pkg/sentry/fs/proc/uid_gid_map.go index d433632cf..d649da0f1 100644 --- a/pkg/sentry/fs/proc/uid_gid_map.go +++ b/pkg/sentry/fs/proc/uid_gid_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/uptime.go b/pkg/sentry/fs/proc/uptime.go index d7ae26fcf..1ddf9fafa 100644 --- a/pkg/sentry/fs/proc/uptime.go +++ b/pkg/sentry/fs/proc/uptime.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/proc/version.go b/pkg/sentry/fs/proc/version.go index 58e0c793c..a5479990c 100644 --- a/pkg/sentry/fs/proc/version.go +++ b/pkg/sentry/fs/proc/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ramfs/dir.go b/pkg/sentry/fs/ramfs/dir.go index c0400b67d..a6b6a5c33 100644 --- a/pkg/sentry/fs/ramfs/dir.go +++ b/pkg/sentry/fs/ramfs/dir.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go index 5bcb6c364..9406a07ca 100644 --- a/pkg/sentry/fs/ramfs/socket.go +++ b/pkg/sentry/fs/ramfs/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ramfs/symlink.go b/pkg/sentry/fs/ramfs/symlink.go index 35dabdad2..f7835fe05 100644 --- a/pkg/sentry/fs/ramfs/symlink.go +++ b/pkg/sentry/fs/ramfs/symlink.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ramfs/tree.go b/pkg/sentry/fs/ramfs/tree.go index c1ac8a78b..8c6b31f70 100644 --- a/pkg/sentry/fs/ramfs/tree.go +++ b/pkg/sentry/fs/ramfs/tree.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/ramfs/tree_test.go b/pkg/sentry/fs/ramfs/tree_test.go index 8bee9cfc1..27abeb6ba 100644 --- a/pkg/sentry/fs/ramfs/tree_test.go +++ b/pkg/sentry/fs/ramfs/tree_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/restore.go b/pkg/sentry/fs/restore.go index a6645b41e..f10168125 100644 --- a/pkg/sentry/fs/restore.go +++ b/pkg/sentry/fs/restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/save.go b/pkg/sentry/fs/save.go index 90988d385..2eaf6ab69 100644 --- a/pkg/sentry/fs/save.go +++ b/pkg/sentry/fs/save.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/seek.go b/pkg/sentry/fs/seek.go index 72f3fb632..0f43918ad 100644 --- a/pkg/sentry/fs/seek.go +++ b/pkg/sentry/fs/seek.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/sync.go b/pkg/sentry/fs/sync.go index 6dcc2fe8d..1fff8059c 100644 --- a/pkg/sentry/fs/sync.go +++ b/pkg/sentry/fs/sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/sys/device.go b/pkg/sentry/fs/sys/device.go index 38ecd0c18..128d3a9d9 100644 --- a/pkg/sentry/fs/sys/device.go +++ b/pkg/sentry/fs/sys/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/sys/devices.go b/pkg/sentry/fs/sys/devices.go index 8b728a4e4..db91de435 100644 --- a/pkg/sentry/fs/sys/devices.go +++ b/pkg/sentry/fs/sys/devices.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/sys/fs.go b/pkg/sentry/fs/sys/fs.go index 44ae43754..f0c2322e0 100644 --- a/pkg/sentry/fs/sys/fs.go +++ b/pkg/sentry/fs/sys/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/sys/sys.go b/pkg/sentry/fs/sys/sys.go index c5b56fe69..d20ef91fa 100644 --- a/pkg/sentry/fs/sys/sys.go +++ b/pkg/sentry/fs/sys/sys.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go index ef9a08854..749961f51 100644 --- a/pkg/sentry/fs/timerfd/timerfd.go +++ b/pkg/sentry/fs/timerfd/timerfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tmpfs/device.go b/pkg/sentry/fs/tmpfs/device.go index aade93c26..179c3a46f 100644 --- a/pkg/sentry/fs/tmpfs/device.go +++ b/pkg/sentry/fs/tmpfs/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go index d0c9b8bea..1ef256511 100644 --- a/pkg/sentry/fs/tmpfs/file_regular.go +++ b/pkg/sentry/fs/tmpfs/file_regular.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tmpfs/file_test.go b/pkg/sentry/fs/tmpfs/file_test.go index 743061190..b44c06556 100644 --- a/pkg/sentry/fs/tmpfs/file_test.go +++ b/pkg/sentry/fs/tmpfs/file_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tmpfs/fs.go b/pkg/sentry/fs/tmpfs/fs.go index 8e44421b6..b7c29a4d1 100644 --- a/pkg/sentry/fs/tmpfs/fs.go +++ b/pkg/sentry/fs/tmpfs/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tmpfs/inode_file.go b/pkg/sentry/fs/tmpfs/inode_file.go index 4450e1363..f89d86c83 100644 --- a/pkg/sentry/fs/tmpfs/inode_file.go +++ b/pkg/sentry/fs/tmpfs/inode_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go index 5bb4922cb..832914453 100644 --- a/pkg/sentry/fs/tmpfs/tmpfs.go +++ b/pkg/sentry/fs/tmpfs/tmpfs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index f8713471a..0fc777e67 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tty/fs.go b/pkg/sentry/fs/tty/fs.go index a53448c47..701b2f7d9 100644 --- a/pkg/sentry/fs/tty/fs.go +++ b/pkg/sentry/fs/tty/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tty/line_discipline.go b/pkg/sentry/fs/tty/line_discipline.go index c4a364edb..20d29d130 100644 --- a/pkg/sentry/fs/tty/line_discipline.go +++ b/pkg/sentry/fs/tty/line_discipline.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index e2686a074..45e167e5f 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tty/queue.go b/pkg/sentry/fs/tty/queue.go index 5e88d84d9..11fb92be3 100644 --- a/pkg/sentry/fs/tty/queue.go +++ b/pkg/sentry/fs/tty/queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index ed080ca0f..0ae57a02c 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tty/terminal.go b/pkg/sentry/fs/tty/terminal.go index 79f9d76d7..2b4160ba5 100644 --- a/pkg/sentry/fs/tty/terminal.go +++ b/pkg/sentry/fs/tty/terminal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/fs/tty/tty_test.go b/pkg/sentry/fs/tty/tty_test.go index ad535838f..d2e75a511 100644 --- a/pkg/sentry/fs/tty/tty_test.go +++ b/pkg/sentry/fs/tty/tty_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/hostcpu/getcpu_amd64.s b/pkg/sentry/hostcpu/getcpu_amd64.s index 409db1450..aa00316da 100644 --- a/pkg/sentry/hostcpu/getcpu_amd64.s +++ b/pkg/sentry/hostcpu/getcpu_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/hostcpu/hostcpu.go b/pkg/sentry/hostcpu/hostcpu.go index 3adc847bb..d78f78402 100644 --- a/pkg/sentry/hostcpu/hostcpu.go +++ b/pkg/sentry/hostcpu/hostcpu.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/hostcpu/hostcpu_test.go b/pkg/sentry/hostcpu/hostcpu_test.go index 38de0e1f6..7d6885c9e 100644 --- a/pkg/sentry/hostcpu/hostcpu_test.go +++ b/pkg/sentry/hostcpu/hostcpu_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/inet/context.go b/pkg/sentry/inet/context.go index d05e96f15..8550c4793 100644 --- a/pkg/sentry/inet/context.go +++ b/pkg/sentry/inet/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/inet/inet.go b/pkg/sentry/inet/inet.go index 8206377cc..7c104fd47 100644 --- a/pkg/sentry/inet/inet.go +++ b/pkg/sentry/inet/inet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/inet/test_stack.go b/pkg/sentry/inet/test_stack.go index 05c1a1792..624371eb6 100644 --- a/pkg/sentry/inet/test_stack.go +++ b/pkg/sentry/inet/test_stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/abstract_socket_namespace.go b/pkg/sentry/kernel/abstract_socket_namespace.go index 1ea2cee36..5ce52e66c 100644 --- a/pkg/sentry/kernel/abstract_socket_namespace.go +++ b/pkg/sentry/kernel/abstract_socket_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/auth/auth.go b/pkg/sentry/kernel/auth/auth.go index 19f15fd36..847d121aa 100644 --- a/pkg/sentry/kernel/auth/auth.go +++ b/pkg/sentry/kernel/auth/auth.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/auth/capability_set.go b/pkg/sentry/kernel/auth/capability_set.go index 88d6243aa..7a0c967cd 100644 --- a/pkg/sentry/kernel/auth/capability_set.go +++ b/pkg/sentry/kernel/auth/capability_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/auth/context.go b/pkg/sentry/kernel/auth/context.go index f7e945599..16d110610 100644 --- a/pkg/sentry/kernel/auth/context.go +++ b/pkg/sentry/kernel/auth/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/auth/credentials.go b/pkg/sentry/kernel/auth/credentials.go index 2055da196..1511a0324 100644 --- a/pkg/sentry/kernel/auth/credentials.go +++ b/pkg/sentry/kernel/auth/credentials.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/auth/id.go b/pkg/sentry/kernel/auth/id.go index e5bed44d7..0a58ba17c 100644 --- a/pkg/sentry/kernel/auth/id.go +++ b/pkg/sentry/kernel/auth/id.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/auth/id_map.go b/pkg/sentry/kernel/auth/id_map.go index 43f439825..e5d6028d6 100644 --- a/pkg/sentry/kernel/auth/id_map.go +++ b/pkg/sentry/kernel/auth/id_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/auth/id_map_functions.go b/pkg/sentry/kernel/auth/id_map_functions.go index 8f1a189ec..432dbfb6d 100644 --- a/pkg/sentry/kernel/auth/id_map_functions.go +++ b/pkg/sentry/kernel/auth/id_map_functions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/auth/user_namespace.go b/pkg/sentry/kernel/auth/user_namespace.go index 159940a69..a40dd668f 100644 --- a/pkg/sentry/kernel/auth/user_namespace.go +++ b/pkg/sentry/kernel/auth/user_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/context.go b/pkg/sentry/kernel/context.go index b629521eb..a1a084eab 100644 --- a/pkg/sentry/kernel/context.go +++ b/pkg/sentry/kernel/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/contexttest/contexttest.go b/pkg/sentry/kernel/contexttest/contexttest.go index eb56a6a07..ae67e2a25 100644 --- a/pkg/sentry/kernel/contexttest/contexttest.go +++ b/pkg/sentry/kernel/contexttest/contexttest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go index befefb11c..2399ae6f2 100644 --- a/pkg/sentry/kernel/epoll/epoll.go +++ b/pkg/sentry/kernel/epoll/epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/epoll/epoll_state.go b/pkg/sentry/kernel/epoll/epoll_state.go index f6e3e4825..4c3c38f9e 100644 --- a/pkg/sentry/kernel/epoll/epoll_state.go +++ b/pkg/sentry/kernel/epoll/epoll_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/epoll/epoll_test.go b/pkg/sentry/kernel/epoll/epoll_test.go index d89c1b745..49b781b69 100644 --- a/pkg/sentry/kernel/epoll/epoll_test.go +++ b/pkg/sentry/kernel/epoll/epoll_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/eventfd/eventfd.go b/pkg/sentry/kernel/eventfd/eventfd.go index b448ad813..5d3139eef 100644 --- a/pkg/sentry/kernel/eventfd/eventfd.go +++ b/pkg/sentry/kernel/eventfd/eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/eventfd/eventfd_test.go b/pkg/sentry/kernel/eventfd/eventfd_test.go index 14e8996d9..1159638e5 100644 --- a/pkg/sentry/kernel/eventfd/eventfd_test.go +++ b/pkg/sentry/kernel/eventfd/eventfd_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/fasync/fasync.go b/pkg/sentry/kernel/fasync/fasync.go index 298d988ea..84cd08501 100644 --- a/pkg/sentry/kernel/fasync/fasync.go +++ b/pkg/sentry/kernel/fasync/fasync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/fd_map.go b/pkg/sentry/kernel/fd_map.go index 715f4714d..c5636d233 100644 --- a/pkg/sentry/kernel/fd_map.go +++ b/pkg/sentry/kernel/fd_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/fd_map_test.go b/pkg/sentry/kernel/fd_map_test.go index 9e76f0a2d..22db4c7cf 100644 --- a/pkg/sentry/kernel/fd_map_test.go +++ b/pkg/sentry/kernel/fd_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/fs_context.go b/pkg/sentry/kernel/fs_context.go index 3cf0db280..d8115f59a 100644 --- a/pkg/sentry/kernel/fs_context.go +++ b/pkg/sentry/kernel/fs_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/futex/futex.go b/pkg/sentry/kernel/futex/futex.go index cd7d51621..bb38eb81e 100644 --- a/pkg/sentry/kernel/futex/futex.go +++ b/pkg/sentry/kernel/futex/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/futex/futex_test.go b/pkg/sentry/kernel/futex/futex_test.go index 9d44ee8e5..2de5239bf 100644 --- a/pkg/sentry/kernel/futex/futex_test.go +++ b/pkg/sentry/kernel/futex/futex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/ipc_namespace.go b/pkg/sentry/kernel/ipc_namespace.go index 9ceb9bd92..ebe12812c 100644 --- a/pkg/sentry/kernel/ipc_namespace.go +++ b/pkg/sentry/kernel/ipc_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/kdefs/kdefs.go b/pkg/sentry/kernel/kdefs/kdefs.go index 8eafe810b..304da2032 100644 --- a/pkg/sentry/kernel/kdefs/kdefs.go +++ b/pkg/sentry/kernel/kdefs/kdefs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index a1b2d7161..0468dd678 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/kernel_state.go b/pkg/sentry/kernel/kernel_state.go index aae6f9ad2..48c3ff5a9 100644 --- a/pkg/sentry/kernel/kernel_state.go +++ b/pkg/sentry/kernel/kernel_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/memevent/memory_events.go b/pkg/sentry/kernel/memevent/memory_events.go index d09d6debf..0e2cee807 100644 --- a/pkg/sentry/kernel/memevent/memory_events.go +++ b/pkg/sentry/kernel/memevent/memory_events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/memevent/memory_events.proto b/pkg/sentry/kernel/memevent/memory_events.proto index 43b8deb76..bf8029ff5 100644 --- a/pkg/sentry/kernel/memevent/memory_events.proto +++ b/pkg/sentry/kernel/memevent/memory_events.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pending_signals.go b/pkg/sentry/kernel/pending_signals.go index deff6def9..c93f6598a 100644 --- a/pkg/sentry/kernel/pending_signals.go +++ b/pkg/sentry/kernel/pending_signals.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pending_signals_state.go b/pkg/sentry/kernel/pending_signals_state.go index 72be6702f..2c902c7e3 100644 --- a/pkg/sentry/kernel/pending_signals_state.go +++ b/pkg/sentry/kernel/pending_signals_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/buffers.go b/pkg/sentry/kernel/pipe/buffers.go index 54e059f8b..ba53fd482 100644 --- a/pkg/sentry/kernel/pipe/buffers.go +++ b/pkg/sentry/kernel/pipe/buffers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/device.go b/pkg/sentry/kernel/pipe/device.go index eec5c5de8..eb59e15a1 100644 --- a/pkg/sentry/kernel/pipe/device.go +++ b/pkg/sentry/kernel/pipe/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/node.go b/pkg/sentry/kernel/pipe/node.go index 1336b6293..99188dddf 100644 --- a/pkg/sentry/kernel/pipe/node.go +++ b/pkg/sentry/kernel/pipe/node.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/node_test.go b/pkg/sentry/kernel/pipe/node_test.go index ad103b195..7ddecdad8 100644 --- a/pkg/sentry/kernel/pipe/node_test.go +++ b/pkg/sentry/kernel/pipe/node_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/pipe.go b/pkg/sentry/kernel/pipe/pipe.go index 357d1162e..bd7649d2f 100644 --- a/pkg/sentry/kernel/pipe/pipe.go +++ b/pkg/sentry/kernel/pipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/pipe_test.go b/pkg/sentry/kernel/pipe/pipe_test.go index 3b9895927..de340c40c 100644 --- a/pkg/sentry/kernel/pipe/pipe_test.go +++ b/pkg/sentry/kernel/pipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/reader.go b/pkg/sentry/kernel/pipe/reader.go index f27379969..48fab45d1 100644 --- a/pkg/sentry/kernel/pipe/reader.go +++ b/pkg/sentry/kernel/pipe/reader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/reader_writer.go b/pkg/sentry/kernel/pipe/reader_writer.go index 1090432d7..ddcc5e09a 100644 --- a/pkg/sentry/kernel/pipe/reader_writer.go +++ b/pkg/sentry/kernel/pipe/reader_writer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/pipe/writer.go b/pkg/sentry/kernel/pipe/writer.go index 6fea9769c..0f29fbc43 100644 --- a/pkg/sentry/kernel/pipe/writer.go +++ b/pkg/sentry/kernel/pipe/writer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/posixtimer.go b/pkg/sentry/kernel/posixtimer.go index 40b5acca3..a016b4087 100644 --- a/pkg/sentry/kernel/posixtimer.go +++ b/pkg/sentry/kernel/posixtimer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/ptrace.go b/pkg/sentry/kernel/ptrace.go index 15f2e2964..4423e7efd 100644 --- a/pkg/sentry/kernel/ptrace.go +++ b/pkg/sentry/kernel/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/ptrace_amd64.go b/pkg/sentry/kernel/ptrace_amd64.go index 1f88efca3..048eeaa3f 100644 --- a/pkg/sentry/kernel/ptrace_amd64.go +++ b/pkg/sentry/kernel/ptrace_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google Inc. +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/ptrace_arm64.go b/pkg/sentry/kernel/ptrace_arm64.go index 4636405e6..4899c813f 100644 --- a/pkg/sentry/kernel/ptrace_arm64.go +++ b/pkg/sentry/kernel/ptrace_arm64.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google Inc. +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/kernel/rseq.go b/pkg/sentry/kernel/rseq.go index 6d3314e81..c4fb2c56c 100644 --- a/pkg/sentry/kernel/rseq.go +++ b/pkg/sentry/kernel/rseq.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/sched/cpuset.go b/pkg/sentry/kernel/sched/cpuset.go index 41ac1067d..c6c436690 100644 --- a/pkg/sentry/kernel/sched/cpuset.go +++ b/pkg/sentry/kernel/sched/cpuset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/sched/cpuset_test.go b/pkg/sentry/kernel/sched/cpuset_test.go index a036ed513..3af9f1197 100644 --- a/pkg/sentry/kernel/sched/cpuset_test.go +++ b/pkg/sentry/kernel/sched/cpuset_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/sched/sched.go b/pkg/sentry/kernel/sched/sched.go index e59909baf..de18c9d02 100644 --- a/pkg/sentry/kernel/sched/sched.go +++ b/pkg/sentry/kernel/sched/sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/seccomp.go b/pkg/sentry/kernel/seccomp.go index 4bed4d373..cc75eb08a 100644 --- a/pkg/sentry/kernel/seccomp.go +++ b/pkg/sentry/kernel/seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/semaphore/semaphore.go b/pkg/sentry/kernel/semaphore/semaphore.go index 2b7c1a9bc..9d0620e02 100644 --- a/pkg/sentry/kernel/semaphore/semaphore.go +++ b/pkg/sentry/kernel/semaphore/semaphore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/semaphore/semaphore_test.go b/pkg/sentry/kernel/semaphore/semaphore_test.go index 2e51e6ee5..abfcd0fb4 100644 --- a/pkg/sentry/kernel/semaphore/semaphore_test.go +++ b/pkg/sentry/kernel/semaphore/semaphore_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/sessions.go b/pkg/sentry/kernel/sessions.go index 070c2f930..610e199da 100644 --- a/pkg/sentry/kernel/sessions.go +++ b/pkg/sentry/kernel/sessions.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/shm/device.go b/pkg/sentry/kernel/shm/device.go index bbc653ed8..3cb759072 100644 --- a/pkg/sentry/kernel/shm/device.go +++ b/pkg/sentry/kernel/shm/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/shm/shm.go b/pkg/sentry/kernel/shm/shm.go index d4812a065..00393b5f0 100644 --- a/pkg/sentry/kernel/shm/shm.go +++ b/pkg/sentry/kernel/shm/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/signal.go b/pkg/sentry/kernel/signal.go index 22a56c6fc..b528ec0dc 100644 --- a/pkg/sentry/kernel/signal.go +++ b/pkg/sentry/kernel/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/signal_handlers.go b/pkg/sentry/kernel/signal_handlers.go index 60cbe85b8..ce8bcb5e5 100644 --- a/pkg/sentry/kernel/signal_handlers.go +++ b/pkg/sentry/kernel/signal_handlers.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/syscalls.go b/pkg/sentry/kernel/syscalls.go index 293b21249..0572053db 100644 --- a/pkg/sentry/kernel/syscalls.go +++ b/pkg/sentry/kernel/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/syscalls_state.go b/pkg/sentry/kernel/syscalls_state.go index 981455d46..00358326b 100644 --- a/pkg/sentry/kernel/syscalls_state.go +++ b/pkg/sentry/kernel/syscalls_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/syslog.go b/pkg/sentry/kernel/syslog.go index 2aecf3eea..175d1b247 100644 --- a/pkg/sentry/kernel/syslog.go +++ b/pkg/sentry/kernel/syslog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/table_test.go b/pkg/sentry/kernel/table_test.go index 3b29d3c6a..8f7cdb9f3 100644 --- a/pkg/sentry/kernel/table_test.go +++ b/pkg/sentry/kernel/table_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go index ed2175c37..f9378c2de 100644 --- a/pkg/sentry/kernel/task.go +++ b/pkg/sentry/kernel/task.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_acct.go b/pkg/sentry/kernel/task_acct.go index 24230af89..1ca2a82eb 100644 --- a/pkg/sentry/kernel/task_acct.go +++ b/pkg/sentry/kernel/task_acct.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_block.go b/pkg/sentry/kernel/task_block.go index e5027e551..30a7f6b1e 100644 --- a/pkg/sentry/kernel/task_block.go +++ b/pkg/sentry/kernel/task_block.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go index daf974920..bba8ddd39 100644 --- a/pkg/sentry/kernel/task_clone.go +++ b/pkg/sentry/kernel/task_clone.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_context.go b/pkg/sentry/kernel/task_context.go index ac38dd157..bbd294141 100644 --- a/pkg/sentry/kernel/task_context.go +++ b/pkg/sentry/kernel/task_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_exec.go b/pkg/sentry/kernel/task_exec.go index b49f902a5..5d1425d5c 100644 --- a/pkg/sentry/kernel/task_exec.go +++ b/pkg/sentry/kernel/task_exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index a07956208..6e9701b01 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_futex.go b/pkg/sentry/kernel/task_futex.go index 351cf47d7..f98097c2c 100644 --- a/pkg/sentry/kernel/task_futex.go +++ b/pkg/sentry/kernel/task_futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_identity.go b/pkg/sentry/kernel/task_identity.go index 6c9608f8d..17f08729a 100644 --- a/pkg/sentry/kernel/task_identity.go +++ b/pkg/sentry/kernel/task_identity.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_log.go b/pkg/sentry/kernel/task_log.go index f4c881c2d..e0e57e8bd 100644 --- a/pkg/sentry/kernel/task_log.go +++ b/pkg/sentry/kernel/task_log.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_net.go b/pkg/sentry/kernel/task_net.go index fc7cefc1f..04c684c1a 100644 --- a/pkg/sentry/kernel/task_net.go +++ b/pkg/sentry/kernel/task_net.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_run.go b/pkg/sentry/kernel/task_run.go index 7115aa967..4549b437e 100644 --- a/pkg/sentry/kernel/task_run.go +++ b/pkg/sentry/kernel/task_run.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_sched.go b/pkg/sentry/kernel/task_sched.go index 3d654bf93..5455f6ea9 100644 --- a/pkg/sentry/kernel/task_sched.go +++ b/pkg/sentry/kernel/task_sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go index 7f2e0df72..654cf7525 100644 --- a/pkg/sentry/kernel/task_signals.go +++ b/pkg/sentry/kernel/task_signals.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go index b7534c0a2..b42531e57 100644 --- a/pkg/sentry/kernel/task_start.go +++ b/pkg/sentry/kernel/task_start.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_stop.go b/pkg/sentry/kernel/task_stop.go index 1302cadc1..e735a5dd0 100644 --- a/pkg/sentry/kernel/task_stop.go +++ b/pkg/sentry/kernel/task_stop.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_syscall.go b/pkg/sentry/kernel/task_syscall.go index 52f5fde8d..a9283d0df 100644 --- a/pkg/sentry/kernel/task_syscall.go +++ b/pkg/sentry/kernel/task_syscall.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_test.go b/pkg/sentry/kernel/task_test.go index 3f37f505d..b895361d0 100644 --- a/pkg/sentry/kernel/task_test.go +++ b/pkg/sentry/kernel/task_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/task_usermem.go b/pkg/sentry/kernel/task_usermem.go index cb68799d3..461bd7316 100644 --- a/pkg/sentry/kernel/task_usermem.go +++ b/pkg/sentry/kernel/task_usermem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/thread_group.go b/pkg/sentry/kernel/thread_group.go index 58f3a7ec9..8bd53928e 100644 --- a/pkg/sentry/kernel/thread_group.go +++ b/pkg/sentry/kernel/thread_group.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/threads.go b/pkg/sentry/kernel/threads.go index 4fd6cf4e2..656bbd46c 100644 --- a/pkg/sentry/kernel/threads.go +++ b/pkg/sentry/kernel/threads.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/time/context.go b/pkg/sentry/kernel/time/context.go index 3675ea20d..c0660d362 100644 --- a/pkg/sentry/kernel/time/context.go +++ b/pkg/sentry/kernel/time/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/time/time.go b/pkg/sentry/kernel/time/time.go index ca0f4ba2e..3846cf1ea 100644 --- a/pkg/sentry/kernel/time/time.go +++ b/pkg/sentry/kernel/time/time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/timekeeper.go b/pkg/sentry/kernel/timekeeper.go index d7bd85e78..505a4fa4f 100644 --- a/pkg/sentry/kernel/timekeeper.go +++ b/pkg/sentry/kernel/timekeeper.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/timekeeper_state.go b/pkg/sentry/kernel/timekeeper_state.go index f3a3ed543..6ce358a05 100644 --- a/pkg/sentry/kernel/timekeeper_state.go +++ b/pkg/sentry/kernel/timekeeper_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/timekeeper_test.go b/pkg/sentry/kernel/timekeeper_test.go index 6084bcb18..a92ad689e 100644 --- a/pkg/sentry/kernel/timekeeper_test.go +++ b/pkg/sentry/kernel/timekeeper_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/uncaught_signal.proto b/pkg/sentry/kernel/uncaught_signal.proto index c7f6a1978..0bdb062cb 100644 --- a/pkg/sentry/kernel/uncaught_signal.proto +++ b/pkg/sentry/kernel/uncaught_signal.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/uts_namespace.go b/pkg/sentry/kernel/uts_namespace.go index ed5f0c031..96fe3cbb9 100644 --- a/pkg/sentry/kernel/uts_namespace.go +++ b/pkg/sentry/kernel/uts_namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/vdso.go b/pkg/sentry/kernel/vdso.go index 3a35f1d00..d40ad74f4 100644 --- a/pkg/sentry/kernel/vdso.go +++ b/pkg/sentry/kernel/vdso.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/kernel/version.go b/pkg/sentry/kernel/version.go index 8d2f14209..5640dd71d 100644 --- a/pkg/sentry/kernel/version.go +++ b/pkg/sentry/kernel/version.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/limits/context.go b/pkg/sentry/limits/context.go index bf413eb7d..9200edb52 100644 --- a/pkg/sentry/limits/context.go +++ b/pkg/sentry/limits/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/limits/limits.go b/pkg/sentry/limits/limits.go index b0571739f..b6c22656b 100644 --- a/pkg/sentry/limits/limits.go +++ b/pkg/sentry/limits/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/limits/limits_test.go b/pkg/sentry/limits/limits_test.go index 945428163..658a20f56 100644 --- a/pkg/sentry/limits/limits_test.go +++ b/pkg/sentry/limits/limits_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/limits/linux.go b/pkg/sentry/limits/linux.go index e09d0d2fb..a2b401e3d 100644 --- a/pkg/sentry/limits/linux.go +++ b/pkg/sentry/limits/linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/loader/elf.go b/pkg/sentry/loader/elf.go index 385ad0102..97e32c8ba 100644 --- a/pkg/sentry/loader/elf.go +++ b/pkg/sentry/loader/elf.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/loader/interpreter.go b/pkg/sentry/loader/interpreter.go index 35b83654d..b88062ae5 100644 --- a/pkg/sentry/loader/interpreter.go +++ b/pkg/sentry/loader/interpreter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/loader/loader.go b/pkg/sentry/loader/loader.go index 79051befa..dc1a52398 100644 --- a/pkg/sentry/loader/loader.go +++ b/pkg/sentry/loader/loader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/loader/vdso.go b/pkg/sentry/loader/vdso.go index 8c196df84..207d8ed3d 100644 --- a/pkg/sentry/loader/vdso.go +++ b/pkg/sentry/loader/vdso.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/loader/vdso_state.go b/pkg/sentry/loader/vdso_state.go index b327f0e1e..db378e90a 100644 --- a/pkg/sentry/loader/vdso_state.go +++ b/pkg/sentry/loader/vdso_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/memmap/mapping_set.go b/pkg/sentry/memmap/mapping_set.go index bd07e9aac..3cf2b338f 100644 --- a/pkg/sentry/memmap/mapping_set.go +++ b/pkg/sentry/memmap/mapping_set.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/memmap/mapping_set_test.go b/pkg/sentry/memmap/mapping_set_test.go index 45d1d4688..c702555ce 100644 --- a/pkg/sentry/memmap/mapping_set_test.go +++ b/pkg/sentry/memmap/mapping_set_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/memmap/memmap.go b/pkg/sentry/memmap/memmap.go index 3f6f7ebd0..0106c857d 100644 --- a/pkg/sentry/memmap/memmap.go +++ b/pkg/sentry/memmap/memmap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/memutil/memutil.go b/pkg/sentry/memutil/memutil.go index 286d50ca4..a4154c42a 100644 --- a/pkg/sentry/memutil/memutil.go +++ b/pkg/sentry/memutil/memutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/memutil/memutil_unsafe.go b/pkg/sentry/memutil/memutil_unsafe.go index bc2c72f55..92eab8a26 100644 --- a/pkg/sentry/memutil/memutil_unsafe.go +++ b/pkg/sentry/memutil/memutil_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/address_space.go b/pkg/sentry/mm/address_space.go index 4dddcf7b5..06f587fde 100644 --- a/pkg/sentry/mm/address_space.go +++ b/pkg/sentry/mm/address_space.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/aio_context.go b/pkg/sentry/mm/aio_context.go index 7075792e0..5c61acf36 100644 --- a/pkg/sentry/mm/aio_context.go +++ b/pkg/sentry/mm/aio_context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/aio_context_state.go b/pkg/sentry/mm/aio_context_state.go index 192a6f744..c37fc9f7b 100644 --- a/pkg/sentry/mm/aio_context_state.go +++ b/pkg/sentry/mm/aio_context_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/debug.go b/pkg/sentry/mm/debug.go index d075ee1ca..fe58cfc4c 100644 --- a/pkg/sentry/mm/debug.go +++ b/pkg/sentry/mm/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/io.go b/pkg/sentry/mm/io.go index 81787a6fd..e4c057d28 100644 --- a/pkg/sentry/mm/io.go +++ b/pkg/sentry/mm/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/lifecycle.go b/pkg/sentry/mm/lifecycle.go index 2fe03172c..e6aa6f9ef 100644 --- a/pkg/sentry/mm/lifecycle.go +++ b/pkg/sentry/mm/lifecycle.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/metadata.go b/pkg/sentry/mm/metadata.go index 5ef1ba0b1..9768e51f1 100644 --- a/pkg/sentry/mm/metadata.go +++ b/pkg/sentry/mm/metadata.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/mm.go b/pkg/sentry/mm/mm.go index a3417a46e..d25aa5136 100644 --- a/pkg/sentry/mm/mm.go +++ b/pkg/sentry/mm/mm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/mm_test.go b/pkg/sentry/mm/mm_test.go index ae4fba478..f4917419f 100644 --- a/pkg/sentry/mm/mm_test.go +++ b/pkg/sentry/mm/mm_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/pma.go b/pkg/sentry/mm/pma.go index 0cca743ef..ece561ff0 100644 --- a/pkg/sentry/mm/pma.go +++ b/pkg/sentry/mm/pma.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/procfs.go b/pkg/sentry/mm/procfs.go index 7cdbf6e25..c8302a553 100644 --- a/pkg/sentry/mm/procfs.go +++ b/pkg/sentry/mm/procfs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/save_restore.go b/pkg/sentry/mm/save_restore.go index 46e0e0754..0385957bd 100644 --- a/pkg/sentry/mm/save_restore.go +++ b/pkg/sentry/mm/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/shm.go b/pkg/sentry/mm/shm.go index 3bc48c7e7..12913007b 100644 --- a/pkg/sentry/mm/shm.go +++ b/pkg/sentry/mm/shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/special_mappable.go b/pkg/sentry/mm/special_mappable.go index 3b5161998..687959005 100644 --- a/pkg/sentry/mm/special_mappable.go +++ b/pkg/sentry/mm/special_mappable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/syscalls.go b/pkg/sentry/mm/syscalls.go index 7b675b9b5..a25318abb 100644 --- a/pkg/sentry/mm/syscalls.go +++ b/pkg/sentry/mm/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/mm/vma.go b/pkg/sentry/mm/vma.go index 931995254..ad901344b 100644 --- a/pkg/sentry/mm/vma.go +++ b/pkg/sentry/mm/vma.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/pgalloc/context.go b/pkg/sentry/pgalloc/context.go index adc97e78f..cb9809b1f 100644 --- a/pkg/sentry/pgalloc/context.go +++ b/pkg/sentry/pgalloc/context.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google Inc. +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/pgalloc/pgalloc.go b/pkg/sentry/pgalloc/pgalloc.go index 0754e608f..411dafa07 100644 --- a/pkg/sentry/pgalloc/pgalloc.go +++ b/pkg/sentry/pgalloc/pgalloc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/pgalloc/pgalloc_test.go b/pkg/sentry/pgalloc/pgalloc_test.go index 726623c1a..14a39bb9e 100644 --- a/pkg/sentry/pgalloc/pgalloc_test.go +++ b/pkg/sentry/pgalloc/pgalloc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/pgalloc/pgalloc_unsafe.go b/pkg/sentry/pgalloc/pgalloc_unsafe.go index 33b0a68a8..a4b5d581c 100644 --- a/pkg/sentry/pgalloc/pgalloc_unsafe.go +++ b/pkg/sentry/pgalloc/pgalloc_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/pgalloc/save_restore.go b/pkg/sentry/pgalloc/save_restore.go index 21024e656..cf169af55 100644 --- a/pkg/sentry/pgalloc/save_restore.go +++ b/pkg/sentry/pgalloc/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/context.go b/pkg/sentry/platform/context.go index cca21a23e..793f57fd7 100644 --- a/pkg/sentry/platform/context.go +++ b/pkg/sentry/platform/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/interrupt/interrupt.go b/pkg/sentry/platform/interrupt/interrupt.go index 9c83f41eb..a4651f500 100644 --- a/pkg/sentry/platform/interrupt/interrupt.go +++ b/pkg/sentry/platform/interrupt/interrupt.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/interrupt/interrupt_test.go b/pkg/sentry/platform/interrupt/interrupt_test.go index fb3284395..0ecdf6e7a 100644 --- a/pkg/sentry/platform/interrupt/interrupt_test.go +++ b/pkg/sentry/platform/interrupt/interrupt_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/address_space.go b/pkg/sentry/platform/kvm/address_space.go index f2f7ab1e8..689122175 100644 --- a/pkg/sentry/platform/kvm/address_space.go +++ b/pkg/sentry/platform/kvm/address_space.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/allocator.go b/pkg/sentry/platform/kvm/allocator.go index b25cad155..42bcc9733 100644 --- a/pkg/sentry/platform/kvm/allocator.go +++ b/pkg/sentry/platform/kvm/allocator.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/bluepill.go b/pkg/sentry/platform/kvm/bluepill.go index f24f1c662..a926e6f8b 100644 --- a/pkg/sentry/platform/kvm/bluepill.go +++ b/pkg/sentry/platform/kvm/bluepill.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.go b/pkg/sentry/platform/kvm/bluepill_amd64.go index 6520682d7..c258408f9 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64.go +++ b/pkg/sentry/platform/kvm/bluepill_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.s b/pkg/sentry/platform/kvm/bluepill_amd64.s index 65b01f358..2bc34a435 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64.s +++ b/pkg/sentry/platform/kvm/bluepill_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go b/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go index 21de2488e..92fde7ee0 100644 --- a/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/bluepill_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/bluepill_fault.go b/pkg/sentry/platform/kvm/bluepill_fault.go index e79a30ef2..3c452f5ba 100644 --- a/pkg/sentry/platform/kvm/bluepill_fault.go +++ b/pkg/sentry/platform/kvm/bluepill_fault.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/bluepill_unsafe.go b/pkg/sentry/platform/kvm/bluepill_unsafe.go index 2605f8c93..4184939e5 100644 --- a/pkg/sentry/platform/kvm/bluepill_unsafe.go +++ b/pkg/sentry/platform/kvm/bluepill_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/context.go b/pkg/sentry/platform/kvm/context.go index c75a4b415..0eb0020f7 100644 --- a/pkg/sentry/platform/kvm/context.go +++ b/pkg/sentry/platform/kvm/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/kvm.go b/pkg/sentry/platform/kvm/kvm.go index c5a4435b1..ed0521c3f 100644 --- a/pkg/sentry/platform/kvm/kvm.go +++ b/pkg/sentry/platform/kvm/kvm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/kvm_amd64.go b/pkg/sentry/platform/kvm/kvm_amd64.go index 70d0ac63b..61493ccaf 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64.go +++ b/pkg/sentry/platform/kvm/kvm_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go index d0f6bb225..46c4b9113 100644 --- a/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/kvm_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/kvm_const.go b/pkg/sentry/platform/kvm/kvm_const.go index cac8d9937..d05f05c29 100644 --- a/pkg/sentry/platform/kvm/kvm_const.go +++ b/pkg/sentry/platform/kvm/kvm_const.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/kvm_test.go b/pkg/sentry/platform/kvm/kvm_test.go index 361200622..e83db71e9 100644 --- a/pkg/sentry/platform/kvm/kvm_test.go +++ b/pkg/sentry/platform/kvm/kvm_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/machine.go b/pkg/sentry/platform/kvm/machine.go index b8b3c9a4a..f5953b96e 100644 --- a/pkg/sentry/platform/kvm/machine.go +++ b/pkg/sentry/platform/kvm/machine.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/machine_amd64.go b/pkg/sentry/platform/kvm/machine_amd64.go index ccfe837b5..b6821122a 100644 --- a/pkg/sentry/platform/kvm/machine_amd64.go +++ b/pkg/sentry/platform/kvm/machine_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go index 69ba67ced..06a2e3b0c 100644 --- a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go +++ b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/machine_unsafe.go b/pkg/sentry/platform/kvm/machine_unsafe.go index 22ae60b63..452d88d7f 100644 --- a/pkg/sentry/platform/kvm/machine_unsafe.go +++ b/pkg/sentry/platform/kvm/machine_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/physical_map.go b/pkg/sentry/platform/kvm/physical_map.go index 9d7dca5b3..450eb8201 100644 --- a/pkg/sentry/platform/kvm/physical_map.go +++ b/pkg/sentry/platform/kvm/physical_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/testutil/testutil.go b/pkg/sentry/platform/kvm/testutil/testutil.go index 0d496561d..6cf2359a3 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil.go +++ b/pkg/sentry/platform/kvm/testutil/testutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/testutil/testutil_amd64.go b/pkg/sentry/platform/kvm/testutil/testutil_amd64.go index fcba33813..203d71528 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil_amd64.go +++ b/pkg/sentry/platform/kvm/testutil/testutil_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/testutil/testutil_amd64.s b/pkg/sentry/platform/kvm/testutil/testutil_amd64.s index f1da41a44..491ec0c2a 100644 --- a/pkg/sentry/platform/kvm/testutil/testutil_amd64.s +++ b/pkg/sentry/platform/kvm/testutil/testutil_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/virtual_map.go b/pkg/sentry/platform/kvm/virtual_map.go index 0343e9267..28a1b4414 100644 --- a/pkg/sentry/platform/kvm/virtual_map.go +++ b/pkg/sentry/platform/kvm/virtual_map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/kvm/virtual_map_test.go b/pkg/sentry/platform/kvm/virtual_map_test.go index 935e0eb93..d03ec654a 100644 --- a/pkg/sentry/platform/kvm/virtual_map_test.go +++ b/pkg/sentry/platform/kvm/virtual_map_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/mmap_min_addr.go b/pkg/sentry/platform/mmap_min_addr.go index 1bcc1f8e9..90976735b 100644 --- a/pkg/sentry/platform/mmap_min_addr.go +++ b/pkg/sentry/platform/mmap_min_addr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/platform.go b/pkg/sentry/platform/platform.go index 0e48417b9..ae37276ad 100644 --- a/pkg/sentry/platform/platform.go +++ b/pkg/sentry/platform/platform.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/procid/procid.go b/pkg/sentry/platform/procid/procid.go index 3f49ab093..78b92422c 100644 --- a/pkg/sentry/platform/procid/procid.go +++ b/pkg/sentry/platform/procid/procid.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/procid/procid_amd64.s b/pkg/sentry/platform/procid/procid_amd64.s index ef3439c03..272c9fc14 100644 --- a/pkg/sentry/platform/procid/procid_amd64.s +++ b/pkg/sentry/platform/procid/procid_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/procid/procid_arm64.s b/pkg/sentry/platform/procid/procid_arm64.s index 02e907b6b..7a1684a18 100644 --- a/pkg/sentry/platform/procid/procid_arm64.s +++ b/pkg/sentry/platform/procid/procid_arm64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/procid/procid_net_test.go b/pkg/sentry/platform/procid/procid_net_test.go index e8dcc479d..b628e2285 100644 --- a/pkg/sentry/platform/procid/procid_net_test.go +++ b/pkg/sentry/platform/procid/procid_net_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/procid/procid_test.go b/pkg/sentry/platform/procid/procid_test.go index 7a57c7cdc..88dd0b3ae 100644 --- a/pkg/sentry/platform/procid/procid_test.go +++ b/pkg/sentry/platform/procid/procid_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/ptrace.go b/pkg/sentry/platform/ptrace/ptrace.go index 3c0713e95..6a890dd81 100644 --- a/pkg/sentry/platform/ptrace/ptrace.go +++ b/pkg/sentry/platform/ptrace/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/ptrace_unsafe.go b/pkg/sentry/platform/ptrace/ptrace_unsafe.go index 223b23199..585f6c1fb 100644 --- a/pkg/sentry/platform/ptrace/ptrace_unsafe.go +++ b/pkg/sentry/platform/ptrace/ptrace_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/stub_amd64.s b/pkg/sentry/platform/ptrace/stub_amd64.s index 63f98e40d..64c718d21 100644 --- a/pkg/sentry/platform/ptrace/stub_amd64.s +++ b/pkg/sentry/platform/ptrace/stub_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/stub_unsafe.go b/pkg/sentry/platform/ptrace/stub_unsafe.go index 48c16c4a1..54d5021a9 100644 --- a/pkg/sentry/platform/ptrace/stub_unsafe.go +++ b/pkg/sentry/platform/ptrace/stub_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/subprocess.go b/pkg/sentry/platform/ptrace/subprocess.go index 2a5d699ec..83b43057f 100644 --- a/pkg/sentry/platform/ptrace/subprocess.go +++ b/pkg/sentry/platform/ptrace/subprocess.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/subprocess_amd64.go b/pkg/sentry/platform/ptrace/subprocess_amd64.go index d23a1133e..77a0e908f 100644 --- a/pkg/sentry/platform/ptrace/subprocess_amd64.go +++ b/pkg/sentry/platform/ptrace/subprocess_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/subprocess_linux.go b/pkg/sentry/platform/ptrace/subprocess_linux.go index e2aab8135..2c07b4ac3 100644 --- a/pkg/sentry/platform/ptrace/subprocess_linux.go +++ b/pkg/sentry/platform/ptrace/subprocess_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go index 0c9263060..1bf7eab28 100644 --- a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go +++ b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ptrace/subprocess_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_unsafe.go index ca6c4ac97..17736b05b 100644 --- a/pkg/sentry/platform/ptrace/subprocess_unsafe.go +++ b/pkg/sentry/platform/ptrace/subprocess_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/defs.go b/pkg/sentry/platform/ring0/defs.go index 98d0a6de0..5bbd4612d 100644 --- a/pkg/sentry/platform/ring0/defs.go +++ b/pkg/sentry/platform/ring0/defs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/defs_amd64.go b/pkg/sentry/platform/ring0/defs_amd64.go index 67242b92b..413c3dbc4 100644 --- a/pkg/sentry/platform/ring0/defs_amd64.go +++ b/pkg/sentry/platform/ring0/defs_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/entry_amd64.go b/pkg/sentry/platform/ring0/entry_amd64.go index 4a9affe64..a5ce67885 100644 --- a/pkg/sentry/platform/ring0/entry_amd64.go +++ b/pkg/sentry/platform/ring0/entry_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/entry_amd64.s b/pkg/sentry/platform/ring0/entry_amd64.s index afb040a6f..8cb8c4996 100644 --- a/pkg/sentry/platform/ring0/entry_amd64.s +++ b/pkg/sentry/platform/ring0/entry_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/gen_offsets/main.go b/pkg/sentry/platform/ring0/gen_offsets/main.go index 11c49855f..a4927da2f 100644 --- a/pkg/sentry/platform/ring0/gen_offsets/main.go +++ b/pkg/sentry/platform/ring0/gen_offsets/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/kernel.go b/pkg/sentry/platform/ring0/kernel.go index 19ac6eb7c..900c0bba7 100644 --- a/pkg/sentry/platform/ring0/kernel.go +++ b/pkg/sentry/platform/ring0/kernel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/kernel_amd64.go b/pkg/sentry/platform/ring0/kernel_amd64.go index 5ed4342dd..3577b5127 100644 --- a/pkg/sentry/platform/ring0/kernel_amd64.go +++ b/pkg/sentry/platform/ring0/kernel_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/kernel_unsafe.go b/pkg/sentry/platform/ring0/kernel_unsafe.go index faf4240e5..16955ad91 100644 --- a/pkg/sentry/platform/ring0/kernel_unsafe.go +++ b/pkg/sentry/platform/ring0/kernel_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/lib_amd64.go b/pkg/sentry/platform/ring0/lib_amd64.go index 2b95a0141..9c5f26962 100644 --- a/pkg/sentry/platform/ring0/lib_amd64.go +++ b/pkg/sentry/platform/ring0/lib_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/lib_amd64.s b/pkg/sentry/platform/ring0/lib_amd64.s index 98a130525..75d742750 100644 --- a/pkg/sentry/platform/ring0/lib_amd64.s +++ b/pkg/sentry/platform/ring0/lib_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/offsets_amd64.go b/pkg/sentry/platform/ring0/offsets_amd64.go index 806e07ec0..85cc3fdad 100644 --- a/pkg/sentry/platform/ring0/offsets_amd64.go +++ b/pkg/sentry/platform/ring0/offsets_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/allocator.go b/pkg/sentry/platform/ring0/pagetables/allocator.go index ee6e90a11..23fd5c352 100644 --- a/pkg/sentry/platform/ring0/pagetables/allocator.go +++ b/pkg/sentry/platform/ring0/pagetables/allocator.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go b/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go index f48647b3a..1b996b4e2 100644 --- a/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go +++ b/pkg/sentry/platform/ring0/pagetables/allocator_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables.go b/pkg/sentry/platform/ring0/pagetables/pagetables.go index c7207ec18..e5dcaada7 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go index 746f614e5..7aa6c524e 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go index 2f82c4353..a1ec4b109 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_amd64_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_test.go b/pkg/sentry/platform/ring0/pagetables/pagetables_test.go index 3e5dc7dc7..36e424495 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_test.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go b/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go index 6bd8c3584..ff427fbe9 100644 --- a/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go +++ b/pkg/sentry/platform/ring0/pagetables/pagetables_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/pcids_x86.go b/pkg/sentry/platform/ring0/pagetables/pcids_x86.go index 0d9a51aa5..0f029f25d 100644 --- a/pkg/sentry/platform/ring0/pagetables/pcids_x86.go +++ b/pkg/sentry/platform/ring0/pagetables/pcids_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/pagetables/walker_amd64.go b/pkg/sentry/platform/ring0/pagetables/walker_amd64.go index c4c71d23e..8f9dacd93 100644 --- a/pkg/sentry/platform/ring0/pagetables/walker_amd64.go +++ b/pkg/sentry/platform/ring0/pagetables/walker_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/ring0.go b/pkg/sentry/platform/ring0/ring0.go index 10c51e88d..cdeb1b43a 100644 --- a/pkg/sentry/platform/ring0/ring0.go +++ b/pkg/sentry/platform/ring0/ring0.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/ring0/x86.go b/pkg/sentry/platform/ring0/x86.go index 4c6daec22..7e5ceafdb 100644 --- a/pkg/sentry/platform/ring0/x86.go +++ b/pkg/sentry/platform/ring0/x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/safecopy/atomic_amd64.s b/pkg/sentry/platform/safecopy/atomic_amd64.s index f90b4bfd1..a0cd78f33 100644 --- a/pkg/sentry/platform/safecopy/atomic_amd64.s +++ b/pkg/sentry/platform/safecopy/atomic_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/safecopy/safecopy.go b/pkg/sentry/platform/safecopy/safecopy.go index 69c66a3b7..5126871eb 100644 --- a/pkg/sentry/platform/safecopy/safecopy.go +++ b/pkg/sentry/platform/safecopy/safecopy.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/safecopy/safecopy_test.go b/pkg/sentry/platform/safecopy/safecopy_test.go index 1a682d28a..5818f7f9b 100644 --- a/pkg/sentry/platform/safecopy/safecopy_test.go +++ b/pkg/sentry/platform/safecopy/safecopy_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/safecopy/safecopy_unsafe.go b/pkg/sentry/platform/safecopy/safecopy_unsafe.go index f84527484..eef028e68 100644 --- a/pkg/sentry/platform/safecopy/safecopy_unsafe.go +++ b/pkg/sentry/platform/safecopy/safecopy_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/safecopy/sighandler_amd64.s b/pkg/sentry/platform/safecopy/sighandler_amd64.s index db7701a29..475ae48e9 100644 --- a/pkg/sentry/platform/safecopy/sighandler_amd64.s +++ b/pkg/sentry/platform/safecopy/sighandler_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/platform/safecopy/sighandler_arm64.s b/pkg/sentry/platform/safecopy/sighandler_arm64.s index cdfca8207..53e4ac2c1 100644 --- a/pkg/sentry/platform/safecopy/sighandler_arm64.s +++ b/pkg/sentry/platform/safecopy/sighandler_arm64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/safemem/block_unsafe.go b/pkg/sentry/safemem/block_unsafe.go index c3a9780d2..1f72deb61 100644 --- a/pkg/sentry/safemem/block_unsafe.go +++ b/pkg/sentry/safemem/block_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/safemem/io.go b/pkg/sentry/safemem/io.go index 6cb52439f..5c3d73eb7 100644 --- a/pkg/sentry/safemem/io.go +++ b/pkg/sentry/safemem/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/safemem/io_test.go b/pkg/sentry/safemem/io_test.go index 2eda8c3bb..629741bee 100644 --- a/pkg/sentry/safemem/io_test.go +++ b/pkg/sentry/safemem/io_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/safemem/safemem.go b/pkg/sentry/safemem/safemem.go index 090932d3e..3e70d33a2 100644 --- a/pkg/sentry/safemem/safemem.go +++ b/pkg/sentry/safemem/safemem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/safemem/seq_test.go b/pkg/sentry/safemem/seq_test.go index fddcaf714..eba4bb535 100644 --- a/pkg/sentry/safemem/seq_test.go +++ b/pkg/sentry/safemem/seq_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/safemem/seq_unsafe.go b/pkg/sentry/safemem/seq_unsafe.go index 83a6b7183..354a95dde 100644 --- a/pkg/sentry/safemem/seq_unsafe.go +++ b/pkg/sentry/safemem/seq_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/sighandling/sighandling.go b/pkg/sentry/sighandling/sighandling.go index 571245ce5..659b43363 100644 --- a/pkg/sentry/sighandling/sighandling.go +++ b/pkg/sentry/sighandling/sighandling.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/sighandling/sighandling_unsafe.go b/pkg/sentry/sighandling/sighandling_unsafe.go index db6e71487..aca77888a 100644 --- a/pkg/sentry/sighandling/sighandling_unsafe.go +++ b/pkg/sentry/sighandling/sighandling_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/control/control.go b/pkg/sentry/socket/control/control.go index d44f5e88a..abda364c9 100644 --- a/pkg/sentry/socket/control/control.go +++ b/pkg/sentry/socket/control/control.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/epsocket/device.go b/pkg/sentry/socket/epsocket/device.go index 3cc138eb0..ab4083efe 100644 --- a/pkg/sentry/socket/epsocket/device.go +++ b/pkg/sentry/socket/epsocket/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 768fa0dfa..520d82f68 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/epsocket/provider.go b/pkg/sentry/socket/epsocket/provider.go index 0d9c2df24..5a89a63fb 100644 --- a/pkg/sentry/socket/epsocket/provider.go +++ b/pkg/sentry/socket/epsocket/provider.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/epsocket/save_restore.go b/pkg/sentry/socket/epsocket/save_restore.go index f19afb6c0..feaafb7cc 100644 --- a/pkg/sentry/socket/epsocket/save_restore.go +++ b/pkg/sentry/socket/epsocket/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/epsocket/stack.go b/pkg/sentry/socket/epsocket/stack.go index 37c48f4bc..edefa225b 100644 --- a/pkg/sentry/socket/epsocket/stack.go +++ b/pkg/sentry/socket/epsocket/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/hostinet/device.go b/pkg/sentry/socket/hostinet/device.go index c5133f3bb..4267e3691 100644 --- a/pkg/sentry/socket/hostinet/device.go +++ b/pkg/sentry/socket/hostinet/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/hostinet/hostinet.go b/pkg/sentry/socket/hostinet/hostinet.go index 7858892ab..0d6f51d2b 100644 --- a/pkg/sentry/socket/hostinet/hostinet.go +++ b/pkg/sentry/socket/hostinet/hostinet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/hostinet/save_restore.go b/pkg/sentry/socket/hostinet/save_restore.go index 3827f082a..1dec33897 100644 --- a/pkg/sentry/socket/hostinet/save_restore.go +++ b/pkg/sentry/socket/hostinet/save_restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index 49349074f..71884d3db 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/hostinet/socket_unsafe.go b/pkg/sentry/socket/hostinet/socket_unsafe.go index 59c8910ca..eed0c7837 100644 --- a/pkg/sentry/socket/hostinet/socket_unsafe.go +++ b/pkg/sentry/socket/hostinet/socket_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/hostinet/stack.go b/pkg/sentry/socket/hostinet/stack.go index 4ce73c1f1..9c45991ba 100644 --- a/pkg/sentry/socket/hostinet/stack.go +++ b/pkg/sentry/socket/hostinet/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/netlink/message.go b/pkg/sentry/socket/netlink/message.go index a95172cba..5bd3b49ce 100644 --- a/pkg/sentry/socket/netlink/message.go +++ b/pkg/sentry/socket/netlink/message.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/netlink/port/port.go b/pkg/sentry/socket/netlink/port/port.go index 20b9a6e37..e9d3275b1 100644 --- a/pkg/sentry/socket/netlink/port/port.go +++ b/pkg/sentry/socket/netlink/port/port.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/netlink/port/port_test.go b/pkg/sentry/socket/netlink/port/port_test.go index 49b3b48ab..516f6cd6c 100644 --- a/pkg/sentry/socket/netlink/port/port_test.go +++ b/pkg/sentry/socket/netlink/port/port_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/netlink/provider.go b/pkg/sentry/socket/netlink/provider.go index 06786bd50..76cf12fd4 100644 --- a/pkg/sentry/socket/netlink/provider.go +++ b/pkg/sentry/socket/netlink/provider.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/netlink/route/protocol.go b/pkg/sentry/socket/netlink/route/protocol.go index e414b829b..9f0a81403 100644 --- a/pkg/sentry/socket/netlink/route/protocol.go +++ b/pkg/sentry/socket/netlink/route/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index a34f9d3ca..dc688eb00 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/rpcinet/conn/conn.go b/pkg/sentry/socket/rpcinet/conn/conn.go index 64106c4b5..f537c7f63 100644 --- a/pkg/sentry/socket/rpcinet/conn/conn.go +++ b/pkg/sentry/socket/rpcinet/conn/conn.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/rpcinet/device.go b/pkg/sentry/socket/rpcinet/device.go index d2b9f9222..44c0a39b7 100644 --- a/pkg/sentry/socket/rpcinet/device.go +++ b/pkg/sentry/socket/rpcinet/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/rpcinet/notifier/notifier.go b/pkg/sentry/socket/rpcinet/notifier/notifier.go index f06d12231..601e05994 100644 --- a/pkg/sentry/socket/rpcinet/notifier/notifier.go +++ b/pkg/sentry/socket/rpcinet/notifier/notifier.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/rpcinet/rpcinet.go b/pkg/sentry/socket/rpcinet/rpcinet.go index 6c98e6acb..5d4fd4dac 100644 --- a/pkg/sentry/socket/rpcinet/rpcinet.go +++ b/pkg/sentry/socket/rpcinet/rpcinet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index cf8f69efb..c028ed4dd 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/rpcinet/stack.go b/pkg/sentry/socket/rpcinet/stack.go index cb8344ec6..a1be711df 100644 --- a/pkg/sentry/socket/rpcinet/stack.go +++ b/pkg/sentry/socket/rpcinet/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/rpcinet/stack_unsafe.go b/pkg/sentry/socket/rpcinet/stack_unsafe.go index d04fb2069..e53f578ba 100644 --- a/pkg/sentry/socket/rpcinet/stack_unsafe.go +++ b/pkg/sentry/socket/rpcinet/stack_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go index 62ba13782..7e840b452 100644 --- a/pkg/sentry/socket/socket.go +++ b/pkg/sentry/socket/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/unix/device.go b/pkg/sentry/socket/unix/device.go index 41820dbb3..734d39ee6 100644 --- a/pkg/sentry/socket/unix/device.go +++ b/pkg/sentry/socket/unix/device.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/unix/io.go b/pkg/sentry/socket/unix/io.go index 7d80e4393..382911d51 100644 --- a/pkg/sentry/socket/unix/io.go +++ b/pkg/sentry/socket/unix/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/unix/transport/connectioned.go b/pkg/sentry/socket/unix/transport/connectioned.go index 62641bb34..18e492862 100644 --- a/pkg/sentry/socket/unix/transport/connectioned.go +++ b/pkg/sentry/socket/unix/transport/connectioned.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/unix/transport/connectioned_state.go b/pkg/sentry/socket/unix/transport/connectioned_state.go index 608a6a97a..7e02a5db8 100644 --- a/pkg/sentry/socket/unix/transport/connectioned_state.go +++ b/pkg/sentry/socket/unix/transport/connectioned_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/unix/transport/connectionless.go b/pkg/sentry/socket/unix/transport/connectionless.go index 728863f3f..43ff875e4 100644 --- a/pkg/sentry/socket/unix/transport/connectionless.go +++ b/pkg/sentry/socket/unix/transport/connectionless.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/unix/transport/queue.go b/pkg/sentry/socket/unix/transport/queue.go index 45a58c600..b650caae7 100644 --- a/pkg/sentry/socket/unix/transport/queue.go +++ b/pkg/sentry/socket/unix/transport/queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/unix/transport/unix.go b/pkg/sentry/socket/unix/transport/unix.go index 12b1576bd..d5f7f7aa8 100644 --- a/pkg/sentry/socket/unix/transport/unix.go +++ b/pkg/sentry/socket/unix/transport/unix.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index 01efd24d3..e9607aa01 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/state/state.go b/pkg/sentry/state/state.go index 224f8b709..27fde505b 100644 --- a/pkg/sentry/state/state.go +++ b/pkg/sentry/state/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/state/state_metadata.go b/pkg/sentry/state/state_metadata.go index 7f047b808..b8e128c40 100644 --- a/pkg/sentry/state/state_metadata.go +++ b/pkg/sentry/state/state_metadata.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/state/state_unsafe.go b/pkg/sentry/state/state_unsafe.go index f02e12b2a..7745b6ac6 100644 --- a/pkg/sentry/state/state_unsafe.go +++ b/pkg/sentry/state/state_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/capability.go b/pkg/sentry/strace/capability.go index 9001181e7..f85d6636e 100644 --- a/pkg/sentry/strace/capability.go +++ b/pkg/sentry/strace/capability.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/clone.go b/pkg/sentry/strace/clone.go index e18ce84dc..ff6a432c6 100644 --- a/pkg/sentry/strace/clone.go +++ b/pkg/sentry/strace/clone.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/futex.go b/pkg/sentry/strace/futex.go index f4aa7fcad..24301bda6 100644 --- a/pkg/sentry/strace/futex.go +++ b/pkg/sentry/strace/futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/linux64.go b/pkg/sentry/strace/linux64.go index 6043b8cb1..3650fd6e1 100644 --- a/pkg/sentry/strace/linux64.go +++ b/pkg/sentry/strace/linux64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/open.go b/pkg/sentry/strace/open.go index 3bf348d7a..140727b02 100644 --- a/pkg/sentry/strace/open.go +++ b/pkg/sentry/strace/open.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/poll.go b/pkg/sentry/strace/poll.go index b6b05423c..15605187d 100644 --- a/pkg/sentry/strace/poll.go +++ b/pkg/sentry/strace/poll.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/strace/ptrace.go b/pkg/sentry/strace/ptrace.go index 8c4b79227..485aacb8a 100644 --- a/pkg/sentry/strace/ptrace.go +++ b/pkg/sentry/strace/ptrace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/signal.go b/pkg/sentry/strace/signal.go index 524be0e15..f82460e1c 100644 --- a/pkg/sentry/strace/signal.go +++ b/pkg/sentry/strace/signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/socket.go b/pkg/sentry/strace/socket.go index 4c1a9d469..dbe53b9a2 100644 --- a/pkg/sentry/strace/socket.go +++ b/pkg/sentry/strace/socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/strace.go b/pkg/sentry/strace/strace.go index 434a200d9..f4c1be4ce 100644 --- a/pkg/sentry/strace/strace.go +++ b/pkg/sentry/strace/strace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/strace.proto b/pkg/sentry/strace/strace.proto index f1fc539d6..4b2f73a5f 100644 --- a/pkg/sentry/strace/strace.proto +++ b/pkg/sentry/strace/strace.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/strace/syscalls.go b/pkg/sentry/strace/syscalls.go index 8c897fcbe..eae2d6c12 100644 --- a/pkg/sentry/strace/syscalls.go +++ b/pkg/sentry/strace/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/epoll.go b/pkg/sentry/syscalls/epoll.go index b90d191b7..ec1eab331 100644 --- a/pkg/sentry/syscalls/epoll.go +++ b/pkg/sentry/syscalls/epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/error.go b/pkg/sentry/syscalls/linux/error.go index 304a12dde..1ba3695fb 100644 --- a/pkg/sentry/syscalls/linux/error.go +++ b/pkg/sentry/syscalls/linux/error.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/flags.go b/pkg/sentry/syscalls/linux/flags.go index d2aec963a..d83e12971 100644 --- a/pkg/sentry/syscalls/linux/flags.go +++ b/pkg/sentry/syscalls/linux/flags.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go index b9b4ccbd1..9a460ebdf 100644 --- a/pkg/sentry/syscalls/linux/linux64.go +++ b/pkg/sentry/syscalls/linux/linux64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sigset.go b/pkg/sentry/syscalls/linux/sigset.go index a033b7c70..5438b664b 100644 --- a/pkg/sentry/syscalls/linux/sigset.go +++ b/pkg/sentry/syscalls/linux/sigset.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_aio.go b/pkg/sentry/syscalls/linux/sys_aio.go index 61c2647bf..1b27b2415 100644 --- a/pkg/sentry/syscalls/linux/sys_aio.go +++ b/pkg/sentry/syscalls/linux/sys_aio.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_capability.go b/pkg/sentry/syscalls/linux/sys_capability.go index cf972dc28..622cb8d0d 100644 --- a/pkg/sentry/syscalls/linux/sys_capability.go +++ b/pkg/sentry/syscalls/linux/sys_capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_epoll.go b/pkg/sentry/syscalls/linux/sys_epoll.go index 200c46355..1467feb4e 100644 --- a/pkg/sentry/syscalls/linux/sys_epoll.go +++ b/pkg/sentry/syscalls/linux/sys_epoll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_eventfd.go b/pkg/sentry/syscalls/linux/sys_eventfd.go index 903172890..ca4ead488 100644 --- a/pkg/sentry/syscalls/linux/sys_eventfd.go +++ b/pkg/sentry/syscalls/linux/sys_eventfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_file.go b/pkg/sentry/syscalls/linux/sys_file.go index 967464c85..893322647 100644 --- a/pkg/sentry/syscalls/linux/sys_file.go +++ b/pkg/sentry/syscalls/linux/sys_file.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_futex.go b/pkg/sentry/syscalls/linux/sys_futex.go index f0c89cba4..7cef4b50c 100644 --- a/pkg/sentry/syscalls/linux/sys_futex.go +++ b/pkg/sentry/syscalls/linux/sys_futex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_getdents.go b/pkg/sentry/syscalls/linux/sys_getdents.go index 4b441b31b..1b597d5bc 100644 --- a/pkg/sentry/syscalls/linux/sys_getdents.go +++ b/pkg/sentry/syscalls/linux/sys_getdents.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_identity.go b/pkg/sentry/syscalls/linux/sys_identity.go index 8d594aa83..27e765a2d 100644 --- a/pkg/sentry/syscalls/linux/sys_identity.go +++ b/pkg/sentry/syscalls/linux/sys_identity.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_inotify.go b/pkg/sentry/syscalls/linux/sys_inotify.go index 26a505782..20269a769 100644 --- a/pkg/sentry/syscalls/linux/sys_inotify.go +++ b/pkg/sentry/syscalls/linux/sys_inotify.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_lseek.go b/pkg/sentry/syscalls/linux/sys_lseek.go index ad3bfd761..8aadc6d8c 100644 --- a/pkg/sentry/syscalls/linux/sys_lseek.go +++ b/pkg/sentry/syscalls/linux/sys_lseek.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_mmap.go b/pkg/sentry/syscalls/linux/sys_mmap.go index 805b251b1..64a6e639c 100644 --- a/pkg/sentry/syscalls/linux/sys_mmap.go +++ b/pkg/sentry/syscalls/linux/sys_mmap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_mount.go b/pkg/sentry/syscalls/linux/sys_mount.go index e110a553f..cf613bad0 100644 --- a/pkg/sentry/syscalls/linux/sys_mount.go +++ b/pkg/sentry/syscalls/linux/sys_mount.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_pipe.go b/pkg/sentry/syscalls/linux/sys_pipe.go index 3652c429e..036845c13 100644 --- a/pkg/sentry/syscalls/linux/sys_pipe.go +++ b/pkg/sentry/syscalls/linux/sys_pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_poll.go b/pkg/sentry/syscalls/linux/sys_poll.go index 17b6768e5..e32099dd4 100644 --- a/pkg/sentry/syscalls/linux/sys_poll.go +++ b/pkg/sentry/syscalls/linux/sys_poll.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_prctl.go b/pkg/sentry/syscalls/linux/sys_prctl.go index 7a29bd9b7..117ae1a0e 100644 --- a/pkg/sentry/syscalls/linux/sys_prctl.go +++ b/pkg/sentry/syscalls/linux/sys_prctl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_random.go b/pkg/sentry/syscalls/linux/sys_random.go index 452dff058..fc3959a7e 100644 --- a/pkg/sentry/syscalls/linux/sys_random.go +++ b/pkg/sentry/syscalls/linux/sys_random.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_read.go b/pkg/sentry/syscalls/linux/sys_read.go index 50c7d7a74..48b0fd49d 100644 --- a/pkg/sentry/syscalls/linux/sys_read.go +++ b/pkg/sentry/syscalls/linux/sys_read.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_rlimit.go b/pkg/sentry/syscalls/linux/sys_rlimit.go index 443334693..8b0379779 100644 --- a/pkg/sentry/syscalls/linux/sys_rlimit.go +++ b/pkg/sentry/syscalls/linux/sys_rlimit.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_rusage.go b/pkg/sentry/syscalls/linux/sys_rusage.go index ab07c77f9..003d718da 100644 --- a/pkg/sentry/syscalls/linux/sys_rusage.go +++ b/pkg/sentry/syscalls/linux/sys_rusage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_sched.go b/pkg/sentry/syscalls/linux/sys_sched.go index e679a6694..8aea03abe 100644 --- a/pkg/sentry/syscalls/linux/sys_sched.go +++ b/pkg/sentry/syscalls/linux/sys_sched.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_seccomp.go b/pkg/sentry/syscalls/linux/sys_seccomp.go index f08fdf5cb..b4262162a 100644 --- a/pkg/sentry/syscalls/linux/sys_seccomp.go +++ b/pkg/sentry/syscalls/linux/sys_seccomp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_sem.go b/pkg/sentry/syscalls/linux/sys_sem.go index 86f850ef1..5bd61ab87 100644 --- a/pkg/sentry/syscalls/linux/sys_sem.go +++ b/pkg/sentry/syscalls/linux/sys_sem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_shm.go b/pkg/sentry/syscalls/linux/sys_shm.go index a0d3a73c5..d0eceac7c 100644 --- a/pkg/sentry/syscalls/linux/sys_shm.go +++ b/pkg/sentry/syscalls/linux/sys_shm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_signal.go b/pkg/sentry/syscalls/linux/sys_signal.go index a539354c5..7fbeb4fcd 100644 --- a/pkg/sentry/syscalls/linux/sys_signal.go +++ b/pkg/sentry/syscalls/linux/sys_signal.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_socket.go b/pkg/sentry/syscalls/linux/sys_socket.go index c8748958a..69862f110 100644 --- a/pkg/sentry/syscalls/linux/sys_socket.go +++ b/pkg/sentry/syscalls/linux/sys_socket.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_stat.go b/pkg/sentry/syscalls/linux/sys_stat.go index 49c225011..10fc201ef 100644 --- a/pkg/sentry/syscalls/linux/sys_stat.go +++ b/pkg/sentry/syscalls/linux/sys_stat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_sync.go b/pkg/sentry/syscalls/linux/sys_sync.go index 68488330f..4352482fb 100644 --- a/pkg/sentry/syscalls/linux/sys_sync.go +++ b/pkg/sentry/syscalls/linux/sys_sync.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_sysinfo.go b/pkg/sentry/syscalls/linux/sys_sysinfo.go index 6f7acf98f..ecf88edc1 100644 --- a/pkg/sentry/syscalls/linux/sys_sysinfo.go +++ b/pkg/sentry/syscalls/linux/sys_sysinfo.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_syslog.go b/pkg/sentry/syscalls/linux/sys_syslog.go index 7193b7aed..9efc58d34 100644 --- a/pkg/sentry/syscalls/linux/sys_syslog.go +++ b/pkg/sentry/syscalls/linux/sys_syslog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_thread.go b/pkg/sentry/syscalls/linux/sys_thread.go index ddcb5b789..23c2f7035 100644 --- a/pkg/sentry/syscalls/linux/sys_thread.go +++ b/pkg/sentry/syscalls/linux/sys_thread.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_time.go b/pkg/sentry/syscalls/linux/sys_time.go index 063fbb106..b4f2609c0 100644 --- a/pkg/sentry/syscalls/linux/sys_time.go +++ b/pkg/sentry/syscalls/linux/sys_time.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_timer.go b/pkg/sentry/syscalls/linux/sys_timer.go index 6baf4599b..04ea7a4e9 100644 --- a/pkg/sentry/syscalls/linux/sys_timer.go +++ b/pkg/sentry/syscalls/linux/sys_timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_timerfd.go b/pkg/sentry/syscalls/linux/sys_timerfd.go index f70d13682..ec0155cbb 100644 --- a/pkg/sentry/syscalls/linux/sys_timerfd.go +++ b/pkg/sentry/syscalls/linux/sys_timerfd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_tls.go b/pkg/sentry/syscalls/linux/sys_tls.go index 8ea78093b..1e8312e00 100644 --- a/pkg/sentry/syscalls/linux/sys_tls.go +++ b/pkg/sentry/syscalls/linux/sys_tls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_utsname.go b/pkg/sentry/syscalls/linux/sys_utsname.go index f7545b965..fa81fe10e 100644 --- a/pkg/sentry/syscalls/linux/sys_utsname.go +++ b/pkg/sentry/syscalls/linux/sys_utsname.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/sys_write.go b/pkg/sentry/syscalls/linux/sys_write.go index e405608c4..1da72d606 100644 --- a/pkg/sentry/syscalls/linux/sys_write.go +++ b/pkg/sentry/syscalls/linux/sys_write.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/linux/timespec.go b/pkg/sentry/syscalls/linux/timespec.go index 752ec326d..fa6fcdc0b 100644 --- a/pkg/sentry/syscalls/linux/timespec.go +++ b/pkg/sentry/syscalls/linux/timespec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/syscalls/syscalls.go b/pkg/sentry/syscalls/syscalls.go index 425ce900c..5d10b3824 100644 --- a/pkg/sentry/syscalls/syscalls.go +++ b/pkg/sentry/syscalls/syscalls.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/calibrated_clock.go b/pkg/sentry/time/calibrated_clock.go index a98bcd7de..c27e391c9 100644 --- a/pkg/sentry/time/calibrated_clock.go +++ b/pkg/sentry/time/calibrated_clock.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/calibrated_clock_test.go b/pkg/sentry/time/calibrated_clock_test.go index a9237630e..d6622bfe2 100644 --- a/pkg/sentry/time/calibrated_clock_test.go +++ b/pkg/sentry/time/calibrated_clock_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/clock_id.go b/pkg/sentry/time/clock_id.go index 1317a5dad..724f59dd9 100644 --- a/pkg/sentry/time/clock_id.go +++ b/pkg/sentry/time/clock_id.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/clocks.go b/pkg/sentry/time/clocks.go index e26386520..837e86094 100644 --- a/pkg/sentry/time/clocks.go +++ b/pkg/sentry/time/clocks.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/muldiv_amd64.s b/pkg/sentry/time/muldiv_amd64.s index bfcb8c724..028c6684e 100644 --- a/pkg/sentry/time/muldiv_amd64.s +++ b/pkg/sentry/time/muldiv_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/muldiv_arm64.s b/pkg/sentry/time/muldiv_arm64.s index 5fa82a136..5ad57a8a3 100644 --- a/pkg/sentry/time/muldiv_arm64.s +++ b/pkg/sentry/time/muldiv_arm64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/parameters.go b/pkg/sentry/time/parameters.go index 8568b1193..63cf7c4a3 100644 --- a/pkg/sentry/time/parameters.go +++ b/pkg/sentry/time/parameters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/parameters_test.go b/pkg/sentry/time/parameters_test.go index 4a0c4e880..e1b9084ac 100644 --- a/pkg/sentry/time/parameters_test.go +++ b/pkg/sentry/time/parameters_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/sampler.go b/pkg/sentry/time/sampler.go index 445690d49..2140a99b7 100644 --- a/pkg/sentry/time/sampler.go +++ b/pkg/sentry/time/sampler.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/sampler_test.go b/pkg/sentry/time/sampler_test.go index ec0e442b6..3e70a1134 100644 --- a/pkg/sentry/time/sampler_test.go +++ b/pkg/sentry/time/sampler_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/sampler_unsafe.go b/pkg/sentry/time/sampler_unsafe.go index 0f8eb4fc8..e76180217 100644 --- a/pkg/sentry/time/sampler_unsafe.go +++ b/pkg/sentry/time/sampler_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/tsc_amd64.s b/pkg/sentry/time/tsc_amd64.s index e53d477f7..6a8eed664 100644 --- a/pkg/sentry/time/tsc_amd64.s +++ b/pkg/sentry/time/tsc_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/time/tsc_arm64.s b/pkg/sentry/time/tsc_arm64.s index c1c9760ef..da9fa4112 100644 --- a/pkg/sentry/time/tsc_arm64.s +++ b/pkg/sentry/time/tsc_arm64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/unimpl/events.go b/pkg/sentry/unimpl/events.go index f78f8c981..d92766e2d 100644 --- a/pkg/sentry/unimpl/events.go +++ b/pkg/sentry/unimpl/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/unimpl/unimplemented_syscall.proto b/pkg/sentry/unimpl/unimplemented_syscall.proto index 41579b016..0d7a94be7 100644 --- a/pkg/sentry/unimpl/unimplemented_syscall.proto +++ b/pkg/sentry/unimpl/unimplemented_syscall.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/uniqueid/context.go b/pkg/sentry/uniqueid/context.go index 399d98c29..e55b89689 100644 --- a/pkg/sentry/uniqueid/context.go +++ b/pkg/sentry/uniqueid/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usage/cpu.go b/pkg/sentry/usage/cpu.go index cbd7cfe19..bfc282d69 100644 --- a/pkg/sentry/usage/cpu.go +++ b/pkg/sentry/usage/cpu.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usage/io.go b/pkg/sentry/usage/io.go index 8e27a0a88..dfcd3a49d 100644 --- a/pkg/sentry/usage/io.go +++ b/pkg/sentry/usage/io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usage/memory.go b/pkg/sentry/usage/memory.go index 5be9ed9c6..c316f1597 100644 --- a/pkg/sentry/usage/memory.go +++ b/pkg/sentry/usage/memory.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usage/memory_unsafe.go b/pkg/sentry/usage/memory_unsafe.go index a3ae668a5..9e0014ca0 100644 --- a/pkg/sentry/usage/memory_unsafe.go +++ b/pkg/sentry/usage/memory_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usage/usage.go b/pkg/sentry/usage/usage.go index ab327f8e2..e3d33a965 100644 --- a/pkg/sentry/usage/usage.go +++ b/pkg/sentry/usage/usage.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/access_type.go b/pkg/sentry/usermem/access_type.go index 9e6a27bcf..9c1742a59 100644 --- a/pkg/sentry/usermem/access_type.go +++ b/pkg/sentry/usermem/access_type.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/addr.go b/pkg/sentry/usermem/addr.go index 2a75aa60c..e79210804 100644 --- a/pkg/sentry/usermem/addr.go +++ b/pkg/sentry/usermem/addr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/addr_range_seq_test.go b/pkg/sentry/usermem/addr_range_seq_test.go index bd6a1ec8a..82f735026 100644 --- a/pkg/sentry/usermem/addr_range_seq_test.go +++ b/pkg/sentry/usermem/addr_range_seq_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/addr_range_seq_unsafe.go b/pkg/sentry/usermem/addr_range_seq_unsafe.go index f5fd446fa..c09337c15 100644 --- a/pkg/sentry/usermem/addr_range_seq_unsafe.go +++ b/pkg/sentry/usermem/addr_range_seq_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/bytes_io.go b/pkg/sentry/usermem/bytes_io.go index 274f568d0..f98d82168 100644 --- a/pkg/sentry/usermem/bytes_io.go +++ b/pkg/sentry/usermem/bytes_io.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/bytes_io_unsafe.go b/pkg/sentry/usermem/bytes_io_unsafe.go index 7add8bc82..bb49d2ff3 100644 --- a/pkg/sentry/usermem/bytes_io_unsafe.go +++ b/pkg/sentry/usermem/bytes_io_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/usermem.go b/pkg/sentry/usermem/usermem.go index 4c7d5014a..31e4d6ada 100644 --- a/pkg/sentry/usermem/usermem.go +++ b/pkg/sentry/usermem/usermem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/usermem_arm64.go b/pkg/sentry/usermem/usermem_arm64.go index 7fd4ce963..fdfc30a66 100644 --- a/pkg/sentry/usermem/usermem_arm64.go +++ b/pkg/sentry/usermem/usermem_arm64.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_test.go b/pkg/sentry/usermem/usermem_test.go index 1991a9641..4a07118b7 100644 --- a/pkg/sentry/usermem/usermem_test.go +++ b/pkg/sentry/usermem/usermem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/usermem/usermem_unsafe.go b/pkg/sentry/usermem/usermem_unsafe.go index 3895e7871..876783e78 100644 --- a/pkg/sentry/usermem/usermem_unsafe.go +++ b/pkg/sentry/usermem/usermem_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/sentry/usermem/usermem_x86.go b/pkg/sentry/usermem/usermem_x86.go index 9ec90f9ff..8059b72d2 100644 --- a/pkg/sentry/usermem/usermem_x86.go +++ b/pkg/sentry/usermem/usermem_x86.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sentry/watchdog/watchdog.go b/pkg/sentry/watchdog/watchdog.go index b4f1e3a4f..2fc4472dd 100644 --- a/pkg/sentry/watchdog/watchdog.go +++ b/pkg/sentry/watchdog/watchdog.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sleep/commit_amd64.s b/pkg/sleep/commit_amd64.s index d08df7f37..bc4ac2c3c 100644 --- a/pkg/sleep/commit_amd64.s +++ b/pkg/sleep/commit_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sleep/commit_asm.go b/pkg/sleep/commit_asm.go index 90eef4cbc..35e2cc337 100644 --- a/pkg/sleep/commit_asm.go +++ b/pkg/sleep/commit_asm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sleep/commit_noasm.go b/pkg/sleep/commit_noasm.go index 967d22e24..686b1da3d 100644 --- a/pkg/sleep/commit_noasm.go +++ b/pkg/sleep/commit_noasm.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sleep/empty.s b/pkg/sleep/empty.s index 85d52cd9c..fb37360ac 100644 --- a/pkg/sleep/empty.s +++ b/pkg/sleep/empty.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sleep/sleep_test.go b/pkg/sleep/sleep_test.go index 8feb9ffc2..130806c86 100644 --- a/pkg/sleep/sleep_test.go +++ b/pkg/sleep/sleep_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/sleep/sleep_unsafe.go b/pkg/sleep/sleep_unsafe.go index 45fb6f0ea..62e0abc34 100644 --- a/pkg/sleep/sleep_unsafe.go +++ b/pkg/sleep/sleep_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/decode.go b/pkg/state/decode.go index 54b5ad8b8..73a59f871 100644 --- a/pkg/state/decode.go +++ b/pkg/state/decode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/encode.go b/pkg/state/encode.go index fe8512bbf..b0714170b 100644 --- a/pkg/state/encode.go +++ b/pkg/state/encode.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/encode_unsafe.go b/pkg/state/encode_unsafe.go index be94742a8..457e6dbb7 100644 --- a/pkg/state/encode_unsafe.go +++ b/pkg/state/encode_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/map.go b/pkg/state/map.go index 0035d7250..1fb9b47b8 100644 --- a/pkg/state/map.go +++ b/pkg/state/map.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/object.proto b/pkg/state/object.proto index d3b46ea97..952289069 100644 --- a/pkg/state/object.proto +++ b/pkg/state/object.proto @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/printer.go b/pkg/state/printer.go index aee4b69fb..5174c3ba3 100644 --- a/pkg/state/printer.go +++ b/pkg/state/printer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/state.go b/pkg/state/state.go index 4486f83a7..cf7df803a 100644 --- a/pkg/state/state.go +++ b/pkg/state/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/state_test.go b/pkg/state/state_test.go index 22bcad9e1..7c24bbcda 100644 --- a/pkg/state/state_test.go +++ b/pkg/state/state_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/statefile/statefile.go b/pkg/state/statefile/statefile.go index c21e3bb0e..ad4e3b43e 100644 --- a/pkg/state/statefile/statefile.go +++ b/pkg/state/statefile/statefile.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/statefile/statefile_test.go b/pkg/state/statefile/statefile_test.go index b4f400e01..60b769895 100644 --- a/pkg/state/statefile/statefile_test.go +++ b/pkg/state/statefile/statefile_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/state/stats.go b/pkg/state/stats.go index 17ca258fc..eb51cda47 100644 --- a/pkg/state/stats.go +++ b/pkg/state/stats.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/syserr/host_linux.go b/pkg/syserr/host_linux.go index 74bbe9f5b..fc6ef60a1 100644 --- a/pkg/syserr/host_linux.go +++ b/pkg/syserr/host_linux.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/syserr/netstack.go b/pkg/syserr/netstack.go index 1a23919ef..bd489b424 100644 --- a/pkg/syserr/netstack.go +++ b/pkg/syserr/netstack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/syserr/syserr.go b/pkg/syserr/syserr.go index 232634dd4..4ddbd3322 100644 --- a/pkg/syserr/syserr.go +++ b/pkg/syserr/syserr.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/syserror/syserror.go b/pkg/syserror/syserror.go index 5558cccff..345653544 100644 --- a/pkg/syserror/syserror.go +++ b/pkg/syserror/syserror.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/syserror/syserror_test.go b/pkg/syserror/syserror_test.go index 0f0da5781..f2a10ee7b 100644 --- a/pkg/syserror/syserror_test.go +++ b/pkg/syserror/syserror_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/adapters/gonet/gonet.go b/pkg/tcpip/adapters/gonet/gonet.go index 628e28f57..df8bf435d 100644 --- a/pkg/tcpip/adapters/gonet/gonet.go +++ b/pkg/tcpip/adapters/gonet/gonet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/adapters/gonet/gonet_test.go b/pkg/tcpip/adapters/gonet/gonet_test.go index e84f73feb..2c81c5697 100644 --- a/pkg/tcpip/adapters/gonet/gonet_test.go +++ b/pkg/tcpip/adapters/gonet/gonet_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/buffer/prependable.go b/pkg/tcpip/buffer/prependable.go index d3a9a0f88..43cbbc74c 100644 --- a/pkg/tcpip/buffer/prependable.go +++ b/pkg/tcpip/buffer/prependable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/buffer/view.go b/pkg/tcpip/buffer/view.go index 43cbb9461..1a9d40778 100644 --- a/pkg/tcpip/buffer/view.go +++ b/pkg/tcpip/buffer/view.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/buffer/view_test.go b/pkg/tcpip/buffer/view_test.go index 74a0a96fc..ebc3a17b7 100644 --- a/pkg/tcpip/buffer/view_test.go +++ b/pkg/tcpip/buffer/view_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/checker/checker.go b/pkg/tcpip/checker/checker.go index 5dfb3ca1d..6e7edf3ab 100644 --- a/pkg/tcpip/checker/checker.go +++ b/pkg/tcpip/checker/checker.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/hash/jenkins/jenkins.go b/pkg/tcpip/hash/jenkins/jenkins.go index e66d5f12b..52c22230e 100644 --- a/pkg/tcpip/hash/jenkins/jenkins.go +++ b/pkg/tcpip/hash/jenkins/jenkins.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/hash/jenkins/jenkins_test.go b/pkg/tcpip/hash/jenkins/jenkins_test.go index 9d86174aa..4c78b5808 100644 --- a/pkg/tcpip/hash/jenkins/jenkins_test.go +++ b/pkg/tcpip/hash/jenkins/jenkins_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/arp.go b/pkg/tcpip/header/arp.go index 22b259ccb..55fe7292c 100644 --- a/pkg/tcpip/header/arp.go +++ b/pkg/tcpip/header/arp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/checksum.go b/pkg/tcpip/header/checksum.go index 2e8c65fac..2eaa7938a 100644 --- a/pkg/tcpip/header/checksum.go +++ b/pkg/tcpip/header/checksum.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/eth.go b/pkg/tcpip/header/eth.go index 77365bc41..76143f454 100644 --- a/pkg/tcpip/header/eth.go +++ b/pkg/tcpip/header/eth.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/gue.go b/pkg/tcpip/header/gue.go index 2ad13955a..10d358c0e 100644 --- a/pkg/tcpip/header/gue.go +++ b/pkg/tcpip/header/gue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/icmpv4.go b/pkg/tcpip/header/icmpv4.go index 3ac89cdae..782e1053c 100644 --- a/pkg/tcpip/header/icmpv4.go +++ b/pkg/tcpip/header/icmpv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/icmpv6.go b/pkg/tcpip/header/icmpv6.go index e317975e8..d0b10d849 100644 --- a/pkg/tcpip/header/icmpv6.go +++ b/pkg/tcpip/header/icmpv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/interfaces.go b/pkg/tcpip/header/interfaces.go index ac327d8a5..fb250ea30 100644 --- a/pkg/tcpip/header/interfaces.go +++ b/pkg/tcpip/header/interfaces.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/ipv4.go b/pkg/tcpip/header/ipv4.go index c3b8fb00e..96e461491 100644 --- a/pkg/tcpip/header/ipv4.go +++ b/pkg/tcpip/header/ipv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/ipv6.go b/pkg/tcpip/header/ipv6.go index 3d24736c7..66820a466 100644 --- a/pkg/tcpip/header/ipv6.go +++ b/pkg/tcpip/header/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/ipv6_fragment.go b/pkg/tcpip/header/ipv6_fragment.go index e36d5177b..6d896355a 100644 --- a/pkg/tcpip/header/ipv6_fragment.go +++ b/pkg/tcpip/header/ipv6_fragment.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/ipversion_test.go b/pkg/tcpip/header/ipversion_test.go index 8301ba5cf..0c830180e 100644 --- a/pkg/tcpip/header/ipversion_test.go +++ b/pkg/tcpip/header/ipversion_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/tcp.go b/pkg/tcpip/header/tcp.go index e656ebb15..0cd89b992 100644 --- a/pkg/tcpip/header/tcp.go +++ b/pkg/tcpip/header/tcp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/tcp_test.go b/pkg/tcpip/header/tcp_test.go index 7cd98df3b..9a2b99489 100644 --- a/pkg/tcpip/header/tcp_test.go +++ b/pkg/tcpip/header/tcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/header/udp.go b/pkg/tcpip/header/udp.go index e8c860436..2205fec18 100644 --- a/pkg/tcpip/header/udp.go +++ b/pkg/tcpip/header/udp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/channel/channel.go b/pkg/tcpip/link/channel/channel.go index f7501a1bc..ee9dd8700 100644 --- a/pkg/tcpip/link/channel/channel.go +++ b/pkg/tcpip/link/channel/channel.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/fdbased/endpoint.go b/pkg/tcpip/link/fdbased/endpoint.go index 8f4d67074..4da376774 100644 --- a/pkg/tcpip/link/fdbased/endpoint.go +++ b/pkg/tcpip/link/fdbased/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/fdbased/endpoint_test.go b/pkg/tcpip/link/fdbased/endpoint_test.go index c8b037d57..31138e4ac 100644 --- a/pkg/tcpip/link/fdbased/endpoint_test.go +++ b/pkg/tcpip/link/fdbased/endpoint_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/fdbased/endpoint_unsafe.go b/pkg/tcpip/link/fdbased/endpoint_unsafe.go index 36e7fe5a9..97a477b61 100644 --- a/pkg/tcpip/link/fdbased/endpoint_unsafe.go +++ b/pkg/tcpip/link/fdbased/endpoint_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/mmap.go b/pkg/tcpip/link/fdbased/mmap.go index f1e71c233..430c85a42 100644 --- a/pkg/tcpip/link/fdbased/mmap.go +++ b/pkg/tcpip/link/fdbased/mmap.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go b/pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go index e5ac7996d..135da2498 100644 --- a/pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go +++ b/pkg/tcpip/link/fdbased/mmap_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/loopback/loopback.go b/pkg/tcpip/link/loopback/loopback.go index 2dc4bcfda..2c1148123 100644 --- a/pkg/tcpip/link/loopback/loopback.go +++ b/pkg/tcpip/link/loopback/loopback.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/muxed/injectable.go b/pkg/tcpip/link/muxed/injectable.go index b3e71c7fc..be07b7c29 100644 --- a/pkg/tcpip/link/muxed/injectable.go +++ b/pkg/tcpip/link/muxed/injectable.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/muxed/injectable_test.go b/pkg/tcpip/link/muxed/injectable_test.go index 031449a05..5d40dfacc 100644 --- a/pkg/tcpip/link/muxed/injectable_test.go +++ b/pkg/tcpip/link/muxed/injectable_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_amd64.s b/pkg/tcpip/link/rawfile/blockingpoll_amd64.s index 9dade5421..b54131573 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_amd64.s +++ b/pkg/tcpip/link/rawfile/blockingpoll_amd64.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go b/pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go index 3ba96a123..0b51982c6 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go +++ b/pkg/tcpip/link/rawfile/blockingpoll_amd64_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go b/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go index 94ddad8ea..4eab77c74 100644 --- a/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go +++ b/pkg/tcpip/link/rawfile/blockingpoll_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/rawfile/errors.go b/pkg/tcpip/link/rawfile/errors.go index 7359849b1..8bde41637 100644 --- a/pkg/tcpip/link/rawfile/errors.go +++ b/pkg/tcpip/link/rawfile/errors.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/rawfile/rawfile_unsafe.go b/pkg/tcpip/link/rawfile/rawfile_unsafe.go index fe2779125..86db7a487 100644 --- a/pkg/tcpip/link/rawfile/rawfile_unsafe.go +++ b/pkg/tcpip/link/rawfile/rawfile_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe.go b/pkg/tcpip/link/sharedmem/pipe/pipe.go index e014324cc..74c9f0311 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe_test.go b/pkg/tcpip/link/sharedmem/pipe/pipe_test.go index 30742ccb1..59ef69a8b 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe_test.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go b/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go index f491d74a2..62d17029e 100644 --- a/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go +++ b/pkg/tcpip/link/sharedmem/pipe/pipe_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/pipe/rx.go b/pkg/tcpip/link/sharedmem/pipe/rx.go index 8d641c76f..f22e533ac 100644 --- a/pkg/tcpip/link/sharedmem/pipe/rx.go +++ b/pkg/tcpip/link/sharedmem/pipe/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/pipe/tx.go b/pkg/tcpip/link/sharedmem/pipe/tx.go index e75175d98..9841eb231 100644 --- a/pkg/tcpip/link/sharedmem/pipe/tx.go +++ b/pkg/tcpip/link/sharedmem/pipe/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/queue/queue_test.go b/pkg/tcpip/link/sharedmem/queue/queue_test.go index 391165bc3..d3f8f4b8b 100644 --- a/pkg/tcpip/link/sharedmem/queue/queue_test.go +++ b/pkg/tcpip/link/sharedmem/queue/queue_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/queue/rx.go b/pkg/tcpip/link/sharedmem/queue/rx.go index d3a5da08a..d9aecf2d9 100644 --- a/pkg/tcpip/link/sharedmem/queue/rx.go +++ b/pkg/tcpip/link/sharedmem/queue/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/queue/tx.go b/pkg/tcpip/link/sharedmem/queue/tx.go index 845108db1..a24dccd11 100644 --- a/pkg/tcpip/link/sharedmem/queue/tx.go +++ b/pkg/tcpip/link/sharedmem/queue/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/rx.go b/pkg/tcpip/link/sharedmem/rx.go index 3eeab769e..215cb607f 100644 --- a/pkg/tcpip/link/sharedmem/rx.go +++ b/pkg/tcpip/link/sharedmem/rx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/sharedmem.go b/pkg/tcpip/link/sharedmem/sharedmem.go index 6e6aa5a13..e34b780f8 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem.go +++ b/pkg/tcpip/link/sharedmem/sharedmem.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/sharedmem_test.go b/pkg/tcpip/link/sharedmem/sharedmem_test.go index 1f44e224c..65b9d7085 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem_test.go +++ b/pkg/tcpip/link/sharedmem/sharedmem_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go b/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go index b91adbaf7..f7e816a41 100644 --- a/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go +++ b/pkg/tcpip/link/sharedmem/sharedmem_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sharedmem/tx.go b/pkg/tcpip/link/sharedmem/tx.go index 37da34831..ac3577aa6 100644 --- a/pkg/tcpip/link/sharedmem/tx.go +++ b/pkg/tcpip/link/sharedmem/tx.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sniffer/pcap.go b/pkg/tcpip/link/sniffer/pcap.go index 3d0d8d852..c16c19647 100644 --- a/pkg/tcpip/link/sniffer/pcap.go +++ b/pkg/tcpip/link/sniffer/pcap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/sniffer/sniffer.go b/pkg/tcpip/link/sniffer/sniffer.go index 462a6e3a3..e87ae07d7 100644 --- a/pkg/tcpip/link/sniffer/sniffer.go +++ b/pkg/tcpip/link/sniffer/sniffer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/tun/tun_unsafe.go b/pkg/tcpip/link/tun/tun_unsafe.go index e4c589dda..09ca9b527 100644 --- a/pkg/tcpip/link/tun/tun_unsafe.go +++ b/pkg/tcpip/link/tun/tun_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/waitable/waitable.go b/pkg/tcpip/link/waitable/waitable.go index bd9f9845b..21690a226 100644 --- a/pkg/tcpip/link/waitable/waitable.go +++ b/pkg/tcpip/link/waitable/waitable.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/link/waitable/waitable_test.go b/pkg/tcpip/link/waitable/waitable_test.go index a2df6be95..62054fb7f 100644 --- a/pkg/tcpip/link/waitable/waitable_test.go +++ b/pkg/tcpip/link/waitable/waitable_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/arp/arp.go b/pkg/tcpip/network/arp/arp.go index 975919e80..a3f2bce3e 100644 --- a/pkg/tcpip/network/arp/arp.go +++ b/pkg/tcpip/network/arp/arp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/arp/arp_test.go b/pkg/tcpip/network/arp/arp_test.go index 14b9cb8b6..1b971b1a3 100644 --- a/pkg/tcpip/network/arp/arp_test.go +++ b/pkg/tcpip/network/arp/arp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/fragmentation/frag_heap.go b/pkg/tcpip/network/fragmentation/frag_heap.go index 55615c8e6..9ad3e5a8a 100644 --- a/pkg/tcpip/network/fragmentation/frag_heap.go +++ b/pkg/tcpip/network/fragmentation/frag_heap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/fragmentation/frag_heap_test.go b/pkg/tcpip/network/fragmentation/frag_heap_test.go index 1b1b72e88..3a2486ba8 100644 --- a/pkg/tcpip/network/fragmentation/frag_heap_test.go +++ b/pkg/tcpip/network/fragmentation/frag_heap_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/fragmentation/fragmentation.go b/pkg/tcpip/network/fragmentation/fragmentation.go index a5dda0398..e90edb375 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation.go +++ b/pkg/tcpip/network/fragmentation/fragmentation.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/fragmentation/fragmentation_test.go b/pkg/tcpip/network/fragmentation/fragmentation_test.go index 5bf3463a9..99ded68a3 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation_test.go +++ b/pkg/tcpip/network/fragmentation/fragmentation_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/fragmentation/reassembler.go b/pkg/tcpip/network/fragmentation/reassembler.go index c9ad2bef6..04f9ab964 100644 --- a/pkg/tcpip/network/fragmentation/reassembler.go +++ b/pkg/tcpip/network/fragmentation/reassembler.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/fragmentation/reassembler_test.go b/pkg/tcpip/network/fragmentation/reassembler_test.go index a2bc9707a..7eee0710d 100644 --- a/pkg/tcpip/network/fragmentation/reassembler_test.go +++ b/pkg/tcpip/network/fragmentation/reassembler_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/hash/hash.go b/pkg/tcpip/network/hash/hash.go index 07960ddf0..0c91905dc 100644 --- a/pkg/tcpip/network/hash/hash.go +++ b/pkg/tcpip/network/hash/hash.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/ip_test.go b/pkg/tcpip/network/ip_test.go index 522009fac..4b822e2c6 100644 --- a/pkg/tcpip/network/ip_test.go +++ b/pkg/tcpip/network/ip_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/ipv4/icmp.go b/pkg/tcpip/network/ipv4/icmp.go index 1c3acda4b..9cb81245a 100644 --- a/pkg/tcpip/network/ipv4/icmp.go +++ b/pkg/tcpip/network/ipv4/icmp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go index cbdca98a5..c6af0db79 100644 --- a/pkg/tcpip/network/ipv4/ipv4.go +++ b/pkg/tcpip/network/ipv4/ipv4.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/ipv4/ipv4_test.go b/pkg/tcpip/network/ipv4/ipv4_test.go index 42e85564e..146143ab3 100644 --- a/pkg/tcpip/network/ipv4/ipv4_test.go +++ b/pkg/tcpip/network/ipv4/ipv4_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go index be28be36d..9c011e107 100644 --- a/pkg/tcpip/network/ipv6/icmp.go +++ b/pkg/tcpip/network/ipv6/icmp.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/ipv6/icmp_test.go b/pkg/tcpip/network/ipv6/icmp_test.go index 8b57a0641..d8737a616 100644 --- a/pkg/tcpip/network/ipv6/icmp_test.go +++ b/pkg/tcpip/network/ipv6/icmp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go index 9a743ea80..4b8cd496b 100644 --- a/pkg/tcpip/network/ipv6/ipv6.go +++ b/pkg/tcpip/network/ipv6/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/ports/ports.go b/pkg/tcpip/ports/ports.go index d212a5792..a1712b590 100644 --- a/pkg/tcpip/ports/ports.go +++ b/pkg/tcpip/ports/ports.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/ports/ports_test.go b/pkg/tcpip/ports/ports_test.go index 01e7320b4..8466c661b 100644 --- a/pkg/tcpip/ports/ports_test.go +++ b/pkg/tcpip/ports/ports_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/sample/tun_tcp_connect/main.go b/pkg/tcpip/sample/tun_tcp_connect/main.go index cf8900c4d..1681de56e 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/main.go +++ b/pkg/tcpip/sample/tun_tcp_connect/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/sample/tun_tcp_echo/main.go b/pkg/tcpip/sample/tun_tcp_echo/main.go index da6202f97..642607f83 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/main.go +++ b/pkg/tcpip/sample/tun_tcp_echo/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/seqnum/seqnum.go b/pkg/tcpip/seqnum/seqnum.go index f2b988839..b40a3c212 100644 --- a/pkg/tcpip/seqnum/seqnum.go +++ b/pkg/tcpip/seqnum/seqnum.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/linkaddrcache.go b/pkg/tcpip/stack/linkaddrcache.go index 40e4bdb4a..42b9768ae 100644 --- a/pkg/tcpip/stack/linkaddrcache.go +++ b/pkg/tcpip/stack/linkaddrcache.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/linkaddrcache_test.go b/pkg/tcpip/stack/linkaddrcache_test.go index 77a09ca86..91b2ffea8 100644 --- a/pkg/tcpip/stack/linkaddrcache_test.go +++ b/pkg/tcpip/stack/linkaddrcache_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index c18571b0f..8008d9870 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/registration.go b/pkg/tcpip/stack/registration.go index 6e1660051..c70533a35 100644 --- a/pkg/tcpip/stack/registration.go +++ b/pkg/tcpip/stack/registration.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/route.go b/pkg/tcpip/stack/route.go index 8ae562dcd..3d4c282a9 100644 --- a/pkg/tcpip/stack/route.go +++ b/pkg/tcpip/stack/route.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index cb9ffe9c2..f204ca790 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/stack_global_state.go b/pkg/tcpip/stack/stack_global_state.go index 3d7e4b719..dfec4258a 100644 --- a/pkg/tcpip/stack/stack_global_state.go +++ b/pkg/tcpip/stack/stack_global_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/stack_test.go b/pkg/tcpip/stack/stack_test.go index b5375df3c..351f63221 100644 --- a/pkg/tcpip/stack/stack_test.go +++ b/pkg/tcpip/stack/stack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/transport_demuxer.go b/pkg/tcpip/stack/transport_demuxer.go index a8ac18e72..e8b562ad9 100644 --- a/pkg/tcpip/stack/transport_demuxer.go +++ b/pkg/tcpip/stack/transport_demuxer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/stack/transport_test.go b/pkg/tcpip/stack/transport_test.go index 2df974bf2..8d74f1543 100644 --- a/pkg/tcpip/stack/transport_test.go +++ b/pkg/tcpip/stack/transport_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go index b09137f08..9367c8c02 100644 --- a/pkg/tcpip/tcpip.go +++ b/pkg/tcpip/tcpip.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/tcpip_test.go b/pkg/tcpip/tcpip_test.go index 1f7b04398..ebb1c1b56 100644 --- a/pkg/tcpip/tcpip_test.go +++ b/pkg/tcpip/tcpip_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/time.s b/pkg/tcpip/time.s index 85d52cd9c..fb37360ac 100644 --- a/pkg/tcpip/time.s +++ b/pkg/tcpip/time.s @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/time_unsafe.go b/pkg/tcpip/time_unsafe.go index 7ec5741af..1a307483b 100644 --- a/pkg/tcpip/time_unsafe.go +++ b/pkg/tcpip/time_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/icmp/endpoint.go b/pkg/tcpip/transport/icmp/endpoint.go index 8f2e3aa20..00840cfcf 100644 --- a/pkg/tcpip/transport/icmp/endpoint.go +++ b/pkg/tcpip/transport/icmp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/icmp/endpoint_state.go b/pkg/tcpip/transport/icmp/endpoint_state.go index 8a7909246..332b3cd33 100644 --- a/pkg/tcpip/transport/icmp/endpoint_state.go +++ b/pkg/tcpip/transport/icmp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/icmp/protocol.go b/pkg/tcpip/transport/icmp/protocol.go index 09ee2f892..954fde9d8 100644 --- a/pkg/tcpip/transport/icmp/protocol.go +++ b/pkg/tcpip/transport/icmp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/raw/raw.go b/pkg/tcpip/transport/raw/raw.go index f0f60ce91..7004c7ff4 100644 --- a/pkg/tcpip/transport/raw/raw.go +++ b/pkg/tcpip/transport/raw/raw.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/raw/state.go b/pkg/tcpip/transport/raw/state.go index e3891a8b8..e8907ebb1 100644 --- a/pkg/tcpip/transport/raw/state.go +++ b/pkg/tcpip/transport/raw/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go index a3894ed8f..e506d7133 100644 --- a/pkg/tcpip/transport/tcp/accept.go +++ b/pkg/tcpip/transport/tcp/accept.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go index 6c4a4d95e..eaa67aeb7 100644 --- a/pkg/tcpip/transport/tcp/connect.go +++ b/pkg/tcpip/transport/tcp/connect.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/cubic.go b/pkg/tcpip/transport/tcp/cubic.go index 003525d86..e618cd2b9 100644 --- a/pkg/tcpip/transport/tcp/cubic.go +++ b/pkg/tcpip/transport/tcp/cubic.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/dual_stack_test.go b/pkg/tcpip/transport/tcp/dual_stack_test.go index 2886cc707..43bcfa070 100644 --- a/pkg/tcpip/transport/tcp/dual_stack_test.go +++ b/pkg/tcpip/transport/tcp/dual_stack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go index 09eff5be1..982f491cc 100644 --- a/pkg/tcpip/transport/tcp/endpoint.go +++ b/pkg/tcpip/transport/tcp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/endpoint_state.go b/pkg/tcpip/transport/tcp/endpoint_state.go index 7f9dabb4d..27b0be046 100644 --- a/pkg/tcpip/transport/tcp/endpoint_state.go +++ b/pkg/tcpip/transport/tcp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/forwarder.go b/pkg/tcpip/transport/tcp/forwarder.go index 6a7efaf1d..e088e24cb 100644 --- a/pkg/tcpip/transport/tcp/forwarder.go +++ b/pkg/tcpip/transport/tcp/forwarder.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go index b5fb160bc..b86473891 100644 --- a/pkg/tcpip/transport/tcp/protocol.go +++ b/pkg/tcpip/transport/tcp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/rcv.go b/pkg/tcpip/transport/tcp/rcv.go index fa6bdddba..b08a0e356 100644 --- a/pkg/tcpip/transport/tcp/rcv.go +++ b/pkg/tcpip/transport/tcp/rcv.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/reno.go b/pkg/tcpip/transport/tcp/reno.go index e4f8b7d5a..f83ebc717 100644 --- a/pkg/tcpip/transport/tcp/reno.go +++ b/pkg/tcpip/transport/tcp/reno.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/sack.go b/pkg/tcpip/transport/tcp/sack.go index 24e48fe7b..6a013d99b 100644 --- a/pkg/tcpip/transport/tcp/sack.go +++ b/pkg/tcpip/transport/tcp/sack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/sack_scoreboard.go b/pkg/tcpip/transport/tcp/sack_scoreboard.go index 21878ad82..99560d5b4 100644 --- a/pkg/tcpip/transport/tcp/sack_scoreboard.go +++ b/pkg/tcpip/transport/tcp/sack_scoreboard.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/sack_scoreboard_test.go b/pkg/tcpip/transport/tcp/sack_scoreboard_test.go index 3cf2ff451..8f6890cdf 100644 --- a/pkg/tcpip/transport/tcp/sack_scoreboard_test.go +++ b/pkg/tcpip/transport/tcp/sack_scoreboard_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/segment.go b/pkg/tcpip/transport/tcp/segment.go index c603fe713..187effb6b 100644 --- a/pkg/tcpip/transport/tcp/segment.go +++ b/pkg/tcpip/transport/tcp/segment.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/segment_heap.go b/pkg/tcpip/transport/tcp/segment_heap.go index 98422fadf..9fd061d7d 100644 --- a/pkg/tcpip/transport/tcp/segment_heap.go +++ b/pkg/tcpip/transport/tcp/segment_heap.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/segment_queue.go b/pkg/tcpip/transport/tcp/segment_queue.go index 0c637d7ad..3b020e580 100644 --- a/pkg/tcpip/transport/tcp/segment_queue.go +++ b/pkg/tcpip/transport/tcp/segment_queue.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/segment_state.go b/pkg/tcpip/transport/tcp/segment_state.go index 68b049f06..dd7e14aa6 100644 --- a/pkg/tcpip/transport/tcp/segment_state.go +++ b/pkg/tcpip/transport/tcp/segment_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go index 6317748cf..50743670e 100644 --- a/pkg/tcpip/transport/tcp/snd.go +++ b/pkg/tcpip/transport/tcp/snd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/snd_state.go b/pkg/tcpip/transport/tcp/snd_state.go index 86bbd643f..12eff8afc 100644 --- a/pkg/tcpip/transport/tcp/snd_state.go +++ b/pkg/tcpip/transport/tcp/snd_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/tcp_sack_test.go b/pkg/tcpip/transport/tcp/tcp_sack_test.go index 06b0702c5..dbfbd5c4f 100644 --- a/pkg/tcpip/transport/tcp/tcp_sack_test.go +++ b/pkg/tcpip/transport/tcp/tcp_sack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go index c5732ad1c..a8b290dae 100644 --- a/pkg/tcpip/transport/tcp/tcp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go index 87c640967..039bbcfba 100644 --- a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/testing/context/context.go b/pkg/tcpip/transport/tcp/testing/context/context.go index 6e2fed880..fa721a7f8 100644 --- a/pkg/tcpip/transport/tcp/testing/context/context.go +++ b/pkg/tcpip/transport/tcp/testing/context/context.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcp/timer.go b/pkg/tcpip/transport/tcp/timer.go index 38240d2d5..fc1c7cbd2 100644 --- a/pkg/tcpip/transport/tcp/timer.go +++ b/pkg/tcpip/transport/tcp/timer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go index b94568fb1..f1dcd36d5 100644 --- a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go +++ b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go index aaeae9b18..435e136de 100644 --- a/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go +++ b/pkg/tcpip/transport/tcpconntrack/tcp_conntrack_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go index 1f9251de3..db65a4e88 100644 --- a/pkg/tcpip/transport/udp/endpoint.go +++ b/pkg/tcpip/transport/udp/endpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/udp/endpoint_state.go b/pkg/tcpip/transport/udp/endpoint_state.go index b2daaf751..163dcbc13 100644 --- a/pkg/tcpip/transport/udp/endpoint_state.go +++ b/pkg/tcpip/transport/udp/endpoint_state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/udp/forwarder.go b/pkg/tcpip/transport/udp/forwarder.go index d80c47e34..25bdd2929 100644 --- a/pkg/tcpip/transport/udp/forwarder.go +++ b/pkg/tcpip/transport/udp/forwarder.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/pkg/tcpip/transport/udp/protocol.go b/pkg/tcpip/transport/udp/protocol.go index 616a9f388..8b47cce17 100644 --- a/pkg/tcpip/transport/udp/protocol.go +++ b/pkg/tcpip/transport/udp/protocol.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go index 2f4e94c58..86a8fa19b 100644 --- a/pkg/tcpip/transport/udp/udp_test.go +++ b/pkg/tcpip/transport/udp/udp_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tmutex/tmutex.go b/pkg/tmutex/tmutex.go index df61d89f5..c4685020d 100644 --- a/pkg/tmutex/tmutex.go +++ b/pkg/tmutex/tmutex.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/tmutex/tmutex_test.go b/pkg/tmutex/tmutex_test.go index a4537cb3b..ce34c7962 100644 --- a/pkg/tmutex/tmutex_test.go +++ b/pkg/tmutex/tmutex_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/unet/unet.go b/pkg/unet/unet.go index 114fb8c5b..2aa1af4ff 100644 --- a/pkg/unet/unet.go +++ b/pkg/unet/unet.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/unet/unet_test.go b/pkg/unet/unet_test.go index db5485539..763b23c7c 100644 --- a/pkg/unet/unet_test.go +++ b/pkg/unet/unet_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/unet/unet_unsafe.go b/pkg/unet/unet_unsafe.go index 1d6ec286c..fa0916439 100644 --- a/pkg/unet/unet_unsafe.go +++ b/pkg/unet/unet_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/urpc/urpc.go b/pkg/urpc/urpc.go index 719f0e92f..0f155ec74 100644 --- a/pkg/urpc/urpc.go +++ b/pkg/urpc/urpc.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/urpc/urpc_test.go b/pkg/urpc/urpc_test.go index f1b9a85ca..5bf2c5ed2 100644 --- a/pkg/urpc/urpc_test.go +++ b/pkg/urpc/urpc_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/waiter/waiter.go b/pkg/waiter/waiter.go index a6c9dff3c..8a65ed164 100644 --- a/pkg/waiter/waiter.go +++ b/pkg/waiter/waiter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/pkg/waiter/waiter_test.go b/pkg/waiter/waiter_test.go index 60853f9c1..c1b94a4f3 100644 --- a/pkg/waiter/waiter_test.go +++ b/pkg/waiter/waiter_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/compat.go b/runsc/boot/compat.go index b3499bcde..c1b33c551 100644 --- a/runsc/boot/compat.go +++ b/runsc/boot/compat.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/compat_amd64.go b/runsc/boot/compat_amd64.go index 0c9472f18..99df5e614 100644 --- a/runsc/boot/compat_amd64.go +++ b/runsc/boot/compat_amd64.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/compat_test.go b/runsc/boot/compat_test.go index f1940dd72..ccec3d20c 100644 --- a/runsc/boot/compat_test.go +++ b/runsc/boot/compat_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/config.go b/runsc/boot/config.go index ba47effc1..b6771de30 100644 --- a/runsc/boot/config.go +++ b/runsc/boot/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go index 712c50ee9..ab7c58838 100644 --- a/runsc/boot/controller.go +++ b/runsc/boot/controller.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/debug.go b/runsc/boot/debug.go index d224d08b7..79f7387ac 100644 --- a/runsc/boot/debug.go +++ b/runsc/boot/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/events.go b/runsc/boot/events.go index 717adfedd..ffd99f5e9 100644 --- a/runsc/boot/events.go +++ b/runsc/boot/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/fds.go b/runsc/boot/fds.go index a3d21d963..4e428b49c 100644 --- a/runsc/boot/fds.go +++ b/runsc/boot/fds.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/filter/config.go b/runsc/boot/filter/config.go index 9c72e3b1a..652da1cef 100644 --- a/runsc/boot/filter/config.go +++ b/runsc/boot/filter/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/filter/extra_filters.go b/runsc/boot/filter/extra_filters.go index 67f3101fe..5c5ec4e06 100644 --- a/runsc/boot/filter/extra_filters.go +++ b/runsc/boot/filter/extra_filters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/filter/extra_filters_msan.go b/runsc/boot/filter/extra_filters_msan.go index fb95283ab..ac5a0f1aa 100644 --- a/runsc/boot/filter/extra_filters_msan.go +++ b/runsc/boot/filter/extra_filters_msan.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/filter/extra_filters_race.go b/runsc/boot/filter/extra_filters_race.go index 02a122c95..ba3c1ce87 100644 --- a/runsc/boot/filter/extra_filters_race.go +++ b/runsc/boot/filter/extra_filters_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/filter/filter.go b/runsc/boot/filter/filter.go index fb197f9b1..17479e0dd 100644 --- a/runsc/boot/filter/filter.go +++ b/runsc/boot/filter/filter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go index 07061b9b3..aeb1c52cc 100644 --- a/runsc/boot/fs.go +++ b/runsc/boot/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/limits.go b/runsc/boot/limits.go index 32e62cdf7..3364aa5e6 100644 --- a/runsc/boot/limits.go +++ b/runsc/boot/limits.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go index 75ec19c32..0b5be0a42 100644 --- a/runsc/boot/loader.go +++ b/runsc/boot/loader.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/loader_test.go b/runsc/boot/loader_test.go index 01578cfc5..9a864ad3f 100644 --- a/runsc/boot/loader_test.go +++ b/runsc/boot/loader_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/network.go b/runsc/boot/network.go index 35baa36ad..598ec969e 100644 --- a/runsc/boot/network.go +++ b/runsc/boot/network.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/boot/strace.go b/runsc/boot/strace.go index 028bcc1f4..19c7f8fbd 100644 --- a/runsc/boot/strace.go +++ b/runsc/boot/strace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cgroup/cgroup.go b/runsc/cgroup/cgroup.go index 2b338b6c6..7431b17d6 100644 --- a/runsc/cgroup/cgroup.go +++ b/runsc/cgroup/cgroup.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cgroup/cgroup_test.go b/runsc/cgroup/cgroup_test.go index ecc184f74..548c80e9a 100644 --- a/runsc/cgroup/cgroup_test.go +++ b/runsc/cgroup/cgroup_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/boot.go b/runsc/cmd/boot.go index ff2fa2fb9..ac937f7bc 100644 --- a/runsc/cmd/boot.go +++ b/runsc/cmd/boot.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/capability.go b/runsc/cmd/capability.go index e5da021e5..312e5b471 100644 --- a/runsc/cmd/capability.go +++ b/runsc/cmd/capability.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/capability_test.go b/runsc/cmd/capability_test.go index dd278b32d..ee74d33d8 100644 --- a/runsc/cmd/capability_test.go +++ b/runsc/cmd/capability_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/checkpoint.go b/runsc/cmd/checkpoint.go index f722df055..96d3c3378 100644 --- a/runsc/cmd/checkpoint.go +++ b/runsc/cmd/checkpoint.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/chroot.go b/runsc/cmd/chroot.go index ed1dafef1..1a774db04 100644 --- a/runsc/cmd/chroot.go +++ b/runsc/cmd/chroot.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/cmd/cmd.go b/runsc/cmd/cmd.go index 208cf5304..aa7b1a636 100644 --- a/runsc/cmd/cmd.go +++ b/runsc/cmd/cmd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/create.go b/runsc/cmd/create.go index 30c8fa283..629c198fd 100644 --- a/runsc/cmd/create.go +++ b/runsc/cmd/create.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/debug.go b/runsc/cmd/debug.go index 3ee9a9b49..000f694c7 100644 --- a/runsc/cmd/debug.go +++ b/runsc/cmd/debug.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/delete.go b/runsc/cmd/delete.go index 3206b267a..9039723e9 100644 --- a/runsc/cmd/delete.go +++ b/runsc/cmd/delete.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/delete_test.go b/runsc/cmd/delete_test.go index 4a5b4774a..45fc91016 100644 --- a/runsc/cmd/delete_test.go +++ b/runsc/cmd/delete_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/do.go b/runsc/cmd/do.go index 343461130..67d415733 100644 --- a/runsc/cmd/do.go +++ b/runsc/cmd/do.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/events.go b/runsc/cmd/events.go index 208d2f74b..c6bc8fc3a 100644 --- a/runsc/cmd/events.go +++ b/runsc/cmd/events.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/exec.go b/runsc/cmd/exec.go index 718d01067..ad2508405 100644 --- a/runsc/cmd/exec.go +++ b/runsc/cmd/exec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/exec_test.go b/runsc/cmd/exec_test.go index 686c5e150..6f0f258c0 100644 --- a/runsc/cmd/exec_test.go +++ b/runsc/cmd/exec_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/gofer.go b/runsc/cmd/gofer.go index 82487887c..bccb29397 100644 --- a/runsc/cmd/gofer.go +++ b/runsc/cmd/gofer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/gofer_test.go b/runsc/cmd/gofer_test.go index 8e692feb9..cbea7f127 100644 --- a/runsc/cmd/gofer_test.go +++ b/runsc/cmd/gofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/kill.go b/runsc/cmd/kill.go index e67f82473..aed5f3291 100644 --- a/runsc/cmd/kill.go +++ b/runsc/cmd/kill.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/list.go b/runsc/cmd/list.go index 1dcea2af0..1f5ca2473 100644 --- a/runsc/cmd/list.go +++ b/runsc/cmd/list.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/path.go b/runsc/cmd/path.go index 1276f0dbd..0e9ef7fa5 100644 --- a/runsc/cmd/path.go +++ b/runsc/cmd/path.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/pause.go b/runsc/cmd/pause.go index 2c93e5f3e..11b36aa10 100644 --- a/runsc/cmd/pause.go +++ b/runsc/cmd/pause.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/ps.go b/runsc/cmd/ps.go index 060d796f2..3a3e6f17a 100644 --- a/runsc/cmd/ps.go +++ b/runsc/cmd/ps.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/restore.go b/runsc/cmd/restore.go index 66b23c38e..27b06713a 100644 --- a/runsc/cmd/restore.go +++ b/runsc/cmd/restore.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/resume.go b/runsc/cmd/resume.go index 5551d1450..9a2ade41e 100644 --- a/runsc/cmd/resume.go +++ b/runsc/cmd/resume.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/run.go b/runsc/cmd/run.go index be1c1b678..4d5f5c139 100644 --- a/runsc/cmd/run.go +++ b/runsc/cmd/run.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/spec.go b/runsc/cmd/spec.go index 063bd39c5..344da13ba 100644 --- a/runsc/cmd/spec.go +++ b/runsc/cmd/spec.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/start.go b/runsc/cmd/start.go index 9e2e0c11d..657726251 100644 --- a/runsc/cmd/start.go +++ b/runsc/cmd/start.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/state.go b/runsc/cmd/state.go index c3ef65ab5..f0d449b19 100644 --- a/runsc/cmd/state.go +++ b/runsc/cmd/state.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/cmd/wait.go b/runsc/cmd/wait.go index 6498dd15c..a55a682f3 100644 --- a/runsc/cmd/wait.go +++ b/runsc/cmd/wait.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/console/console.go b/runsc/console/console.go index 2eb9a8807..64b23639a 100644 --- a/runsc/console/console.go +++ b/runsc/console/console.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/container/console_test.go b/runsc/container/console_test.go index 0b0dfb4cb..b8af27c15 100644 --- a/runsc/container/console_test.go +++ b/runsc/container/console_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/container/container.go b/runsc/container/container.go index a30c217f7..884bbc0fb 100644 --- a/runsc/container/container.go +++ b/runsc/container/container.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/container/container_test.go b/runsc/container/container_test.go index 603c4d929..9458dbb90 100644 --- a/runsc/container/container_test.go +++ b/runsc/container/container_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/container/hook.go b/runsc/container/hook.go index 6b9e5550a..acae6781e 100644 --- a/runsc/container/hook.go +++ b/runsc/container/hook.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/container/multi_container_test.go b/runsc/container/multi_container_test.go index 8922e6dbe..e554237cf 100644 --- a/runsc/container/multi_container_test.go +++ b/runsc/container/multi_container_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/container/shared_volume_test.go b/runsc/container/shared_volume_test.go index 8f81ed630..9d5a592a5 100644 --- a/runsc/container/shared_volume_test.go +++ b/runsc/container/shared_volume_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/container/status.go b/runsc/container/status.go index 234ffb0dd..91d9112f1 100644 --- a/runsc/container/status.go +++ b/runsc/container/status.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/container/test_app.go b/runsc/container/test_app.go index b5071ada6..62923f1ef 100644 --- a/runsc/container/test_app.go +++ b/runsc/container/test_app.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/fsgofer/filter/config.go b/runsc/fsgofer/filter/config.go index 75a087848..a1ad49fb2 100644 --- a/runsc/fsgofer/filter/config.go +++ b/runsc/fsgofer/filter/config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/fsgofer/filter/extra_filters.go b/runsc/fsgofer/filter/extra_filters.go index 67f3101fe..5c5ec4e06 100644 --- a/runsc/fsgofer/filter/extra_filters.go +++ b/runsc/fsgofer/filter/extra_filters.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/fsgofer/filter/extra_filters_msan.go b/runsc/fsgofer/filter/extra_filters_msan.go index 7e142b790..553060bc3 100644 --- a/runsc/fsgofer/filter/extra_filters_msan.go +++ b/runsc/fsgofer/filter/extra_filters_msan.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/fsgofer/filter/extra_filters_race.go b/runsc/fsgofer/filter/extra_filters_race.go index 3cd29472a..28555f898 100644 --- a/runsc/fsgofer/filter/extra_filters_race.go +++ b/runsc/fsgofer/filter/extra_filters_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/fsgofer/filter/filter.go b/runsc/fsgofer/filter/filter.go index c120d57a6..ff8154369 100644 --- a/runsc/fsgofer/filter/filter.go +++ b/runsc/fsgofer/filter/filter.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/fsgofer/fsgofer.go b/runsc/fsgofer/fsgofer.go index c964a2a3b..158f22ddc 100644 --- a/runsc/fsgofer/fsgofer.go +++ b/runsc/fsgofer/fsgofer.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/fsgofer/fsgofer_test.go b/runsc/fsgofer/fsgofer_test.go index e74df7ede..695836927 100644 --- a/runsc/fsgofer/fsgofer_test.go +++ b/runsc/fsgofer/fsgofer_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/fsgofer/fsgofer_unsafe.go b/runsc/fsgofer/fsgofer_unsafe.go index 94413db86..58af5e44d 100644 --- a/runsc/fsgofer/fsgofer_unsafe.go +++ b/runsc/fsgofer/fsgofer_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/main.go b/runsc/main.go index b35726a74..11bc73f75 100644 --- a/runsc/main.go +++ b/runsc/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/sandbox/network.go b/runsc/sandbox/network.go index 6c6b665a0..2a68d7043 100644 --- a/runsc/sandbox/network.go +++ b/runsc/sandbox/network.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/sandbox/network_unsafe.go b/runsc/sandbox/network_unsafe.go index f7447f002..2a2a0fb7e 100644 --- a/runsc/sandbox/network_unsafe.go +++ b/runsc/sandbox/network_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/runsc/sandbox/sandbox.go b/runsc/sandbox/sandbox.go index 48a0dafe2..dac35ca0b 100644 --- a/runsc/sandbox/sandbox.go +++ b/runsc/sandbox/sandbox.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/specutils/fs.go b/runsc/specutils/fs.go index 98c3b19c0..1f3afb4e4 100644 --- a/runsc/specutils/fs.go +++ b/runsc/specutils/fs.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/specutils/namespace.go b/runsc/specutils/namespace.go index 35da789f4..7d194335c 100644 --- a/runsc/specutils/namespace.go +++ b/runsc/specutils/namespace.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/specutils/specutils.go b/runsc/specutils/specutils.go index ac85bec71..c72207fb4 100644 --- a/runsc/specutils/specutils.go +++ b/runsc/specutils/specutils.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/specutils/specutils_test.go b/runsc/specutils/specutils_test.go index 02af6e6ad..2c86fffe8 100644 --- a/runsc/specutils/specutils_test.go +++ b/runsc/specutils/specutils_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/image/image.go b/runsc/test/image/image.go index bcb6f876f..297f1ab92 100644 --- a/runsc/test/image/image.go +++ b/runsc/test/image/image.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/image/image_test.go b/runsc/test/image/image_test.go index f7e750d71..0c45602f9 100644 --- a/runsc/test/image/image_test.go +++ b/runsc/test/image/image_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/image/mysql.sql b/runsc/test/image/mysql.sql index c1271e719..51554b98d 100644 --- a/runsc/test/image/mysql.sql +++ b/runsc/test/image/mysql.sql @@ -1,4 +1,4 @@ -# Copyright 2018 Google LLC +# 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. diff --git a/runsc/test/image/ruby.rb b/runsc/test/image/ruby.rb index 25d1ac129..aced49c6d 100644 --- a/runsc/test/image/ruby.rb +++ b/runsc/test/image/ruby.rb @@ -1,4 +1,4 @@ -# Copyright 2018 Google LLC +# 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. diff --git a/runsc/test/image/ruby.sh b/runsc/test/image/ruby.sh index d3a9b5656..ebe8d5b0e 100644 --- a/runsc/test/image/ruby.sh +++ b/runsc/test/image/ruby.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# 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. diff --git a/runsc/test/install.sh b/runsc/test/install.sh index 32e1e884e..457df2d26 100755 --- a/runsc/test/install.sh +++ b/runsc/test/install.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# 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. diff --git a/runsc/test/integration/exec_test.go b/runsc/test/integration/exec_test.go index d87957e2d..7af064d79 100644 --- a/runsc/test/integration/exec_test.go +++ b/runsc/test/integration/exec_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/integration/integration.go b/runsc/test/integration/integration.go index e15321c87..4cd5f6c24 100644 --- a/runsc/test/integration/integration.go +++ b/runsc/test/integration/integration.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/integration/integration_test.go b/runsc/test/integration/integration_test.go index 4a2770d48..b2e86aacc 100644 --- a/runsc/test/integration/integration_test.go +++ b/runsc/test/integration/integration_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/cgroup_test.go b/runsc/test/root/cgroup_test.go index 91839048c..edb6dee1d 100644 --- a/runsc/test/root/cgroup_test.go +++ b/runsc/test/root/cgroup_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/chroot_test.go b/runsc/test/root/chroot_test.go index 0deca0532..da2f473b9 100644 --- a/runsc/test/root/chroot_test.go +++ b/runsc/test/root/chroot_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/crictl_test.go b/runsc/test/root/crictl_test.go index 37fe53ba3..3cc176104 100644 --- a/runsc/test/root/crictl_test.go +++ b/runsc/test/root/crictl_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/root.go b/runsc/test/root/root.go index 586ea0fe3..349c752cc 100644 --- a/runsc/test/root/root.go +++ b/runsc/test/root/root.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/testdata/busybox.go b/runsc/test/root/testdata/busybox.go index 544571c63..e4dbd2843 100644 --- a/runsc/test/root/testdata/busybox.go +++ b/runsc/test/root/testdata/busybox.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/testdata/containerd_config.go b/runsc/test/root/testdata/containerd_config.go index 949354987..e12f1ec88 100644 --- a/runsc/test/root/testdata/containerd_config.go +++ b/runsc/test/root/testdata/containerd_config.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/testdata/httpd.go b/runsc/test/root/testdata/httpd.go index f65b1da5d..45d5e33d4 100644 --- a/runsc/test/root/testdata/httpd.go +++ b/runsc/test/root/testdata/httpd.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/testdata/httpd_mount_paths.go b/runsc/test/root/testdata/httpd_mount_paths.go index 5ca14340e..ac3f4446a 100644 --- a/runsc/test/root/testdata/httpd_mount_paths.go +++ b/runsc/test/root/testdata/httpd_mount_paths.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/root/testdata/sandbox.go b/runsc/test/root/testdata/sandbox.go index 194242a27..0db210370 100644 --- a/runsc/test/root/testdata/sandbox.go +++ b/runsc/test/root/testdata/sandbox.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/testutil/crictl.go b/runsc/test/testutil/crictl.go index 84bb4475a..4f9ee0c05 100644 --- a/runsc/test/testutil/crictl.go +++ b/runsc/test/testutil/crictl.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/testutil/docker.go b/runsc/test/testutil/docker.go index b651319ed..29ef505b4 100644 --- a/runsc/test/testutil/docker.go +++ b/runsc/test/testutil/docker.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/testutil/testutil.go b/runsc/test/testutil/testutil.go index 79f0a8b6b..6a4c045a8 100644 --- a/runsc/test/testutil/testutil.go +++ b/runsc/test/testutil/testutil.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/test/testutil/testutil_race.go b/runsc/test/testutil/testutil_race.go index 9267af150..86db6ffa1 100644 --- a/runsc/test/testutil/testutil_race.go +++ b/runsc/test/testutil/testutil_race.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/tools/dockercfg/dockercfg.go b/runsc/tools/dockercfg/dockercfg.go index cc7a67816..6fb134558 100644 --- a/runsc/tools/dockercfg/dockercfg.go +++ b/runsc/tools/dockercfg/dockercfg.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/runsc/version.go b/runsc/version.go index 4894f2de6..ce0573a9b 100644 --- a/runsc/version.go +++ b/runsc/version.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/gtest/gtest.go b/test/syscalls/gtest/gtest.go index dfe5037cd..bdec8eb07 100644 --- a/test/syscalls/gtest/gtest.go +++ b/test/syscalls/gtest/gtest.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/32bit.cc b/test/syscalls/linux/32bit.cc index 78baf548e..a7cbee06b 100644 --- a/test/syscalls/linux/32bit.cc +++ b/test/syscalls/linux/32bit.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/accept_bind.cc b/test/syscalls/linux/accept_bind.cc index c2bb4a7ce..56377feab 100644 --- a/test/syscalls/linux/accept_bind.cc +++ b/test/syscalls/linux/accept_bind.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/accept_bind_stream.cc b/test/syscalls/linux/accept_bind_stream.cc index 1501e526e..b6cdb3f4f 100644 --- a/test/syscalls/linux/accept_bind_stream.cc +++ b/test/syscalls/linux/accept_bind_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/access.cc b/test/syscalls/linux/access.cc index 6ea070a5d..bcc25cef4 100644 --- a/test/syscalls/linux/access.cc +++ b/test/syscalls/linux/access.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/affinity.cc b/test/syscalls/linux/affinity.cc index 81bd9bcb5..f2d8375b6 100644 --- a/test/syscalls/linux/affinity.cc +++ b/test/syscalls/linux/affinity.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/aio.cc b/test/syscalls/linux/aio.cc index b96aab9b9..68dc05417 100644 --- a/test/syscalls/linux/aio.cc +++ b/test/syscalls/linux/aio.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/alarm.cc b/test/syscalls/linux/alarm.cc index e0ddbb415..d89269985 100644 --- a/test/syscalls/linux/alarm.cc +++ b/test/syscalls/linux/alarm.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/arch_prctl.cc b/test/syscalls/linux/arch_prctl.cc index 5687ceb86..81bf5a775 100644 --- a/test/syscalls/linux/arch_prctl.cc +++ b/test/syscalls/linux/arch_prctl.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/bad.cc b/test/syscalls/linux/bad.cc index a2634a8bf..f246a799e 100644 --- a/test/syscalls/linux/bad.cc +++ b/test/syscalls/linux/bad.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/base_poll_test.cc b/test/syscalls/linux/base_poll_test.cc index bba0108ea..ab7a19dd0 100644 --- a/test/syscalls/linux/base_poll_test.cc +++ b/test/syscalls/linux/base_poll_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/base_poll_test.h b/test/syscalls/linux/base_poll_test.h index 9b9b81933..088831f9f 100644 --- a/test/syscalls/linux/base_poll_test.h +++ b/test/syscalls/linux/base_poll_test.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/bind.cc b/test/syscalls/linux/bind.cc index f5aa9c500..de8cca53b 100644 --- a/test/syscalls/linux/bind.cc +++ b/test/syscalls/linux/bind.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/brk.cc b/test/syscalls/linux/brk.cc index 33d353959..a03a44465 100644 --- a/test/syscalls/linux/brk.cc +++ b/test/syscalls/linux/brk.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/chdir.cc b/test/syscalls/linux/chdir.cc index a4b54f0ee..3182c228b 100644 --- a/test/syscalls/linux/chdir.cc +++ b/test/syscalls/linux/chdir.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/chmod.cc b/test/syscalls/linux/chmod.cc index 2f42fe326..79e98597f 100644 --- a/test/syscalls/linux/chmod.cc +++ b/test/syscalls/linux/chmod.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/chown.cc b/test/syscalls/linux/chown.cc index ad892cf6a..eb1762ddf 100644 --- a/test/syscalls/linux/chown.cc +++ b/test/syscalls/linux/chown.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/chroot.cc b/test/syscalls/linux/chroot.cc index 6c200f63e..a4354ff62 100644 --- a/test/syscalls/linux/chroot.cc +++ b/test/syscalls/linux/chroot.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/clock_getres.cc b/test/syscalls/linux/clock_getres.cc index 8f8842299..c408b936c 100644 --- a/test/syscalls/linux/clock_getres.cc +++ b/test/syscalls/linux/clock_getres.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/clock_gettime.cc b/test/syscalls/linux/clock_gettime.cc index 4ecb5f5b1..082ae1c39 100644 --- a/test/syscalls/linux/clock_gettime.cc +++ b/test/syscalls/linux/clock_gettime.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/clock_nanosleep.cc b/test/syscalls/linux/clock_nanosleep.cc index 61c67a5ff..52a69d230 100644 --- a/test/syscalls/linux/clock_nanosleep.cc +++ b/test/syscalls/linux/clock_nanosleep.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/concurrency.cc b/test/syscalls/linux/concurrency.cc index 7978845c1..4e0a13f8b 100644 --- a/test/syscalls/linux/concurrency.cc +++ b/test/syscalls/linux/concurrency.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/creat.cc b/test/syscalls/linux/creat.cc index df2cc0d5c..3c270d6da 100644 --- a/test/syscalls/linux/creat.cc +++ b/test/syscalls/linux/creat.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/dev.cc b/test/syscalls/linux/dev.cc index a140d3b30..b86ebe233 100644 --- a/test/syscalls/linux/dev.cc +++ b/test/syscalls/linux/dev.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/dup.cc b/test/syscalls/linux/dup.cc index e8de2f4c4..4f773bc75 100644 --- a/test/syscalls/linux/dup.cc +++ b/test/syscalls/linux/dup.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/epoll.cc b/test/syscalls/linux/epoll.cc index b4a3bfcba..a4f8f3cec 100644 --- a/test/syscalls/linux/epoll.cc +++ b/test/syscalls/linux/epoll.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/eventfd.cc b/test/syscalls/linux/eventfd.cc index 8111da30e..5e5c39d44 100644 --- a/test/syscalls/linux/eventfd.cc +++ b/test/syscalls/linux/eventfd.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exceptions.cc b/test/syscalls/linux/exceptions.cc index 3f0aa8bf1..0da4c817d 100644 --- a/test/syscalls/linux/exceptions.cc +++ b/test/syscalls/linux/exceptions.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exec.cc b/test/syscalls/linux/exec.cc index 30bc4b608..06c322a99 100644 --- a/test/syscalls/linux/exec.cc +++ b/test/syscalls/linux/exec.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exec.h b/test/syscalls/linux/exec.h index b82bfffd1..5c0f7e654 100644 --- a/test/syscalls/linux/exec.h +++ b/test/syscalls/linux/exec.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exec_assert_closed_workload.cc b/test/syscalls/linux/exec_assert_closed_workload.cc index 4448431e1..95643618d 100644 --- a/test/syscalls/linux/exec_assert_closed_workload.cc +++ b/test/syscalls/linux/exec_assert_closed_workload.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exec_basic_workload.cc b/test/syscalls/linux/exec_basic_workload.cc index d4bdf511f..1bbd6437e 100644 --- a/test/syscalls/linux/exec_basic_workload.cc +++ b/test/syscalls/linux/exec_basic_workload.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exec_binary.cc b/test/syscalls/linux/exec_binary.cc index c10d85398..bdd6eb10b 100644 --- a/test/syscalls/linux/exec_binary.cc +++ b/test/syscalls/linux/exec_binary.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exec_proc_exe_workload.cc b/test/syscalls/linux/exec_proc_exe_workload.cc index b9a4ac749..b3fbd5042 100644 --- a/test/syscalls/linux/exec_proc_exe_workload.cc +++ b/test/syscalls/linux/exec_proc_exe_workload.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exec_state_workload.cc b/test/syscalls/linux/exec_state_workload.cc index b66e22565..725c2977f 100644 --- a/test/syscalls/linux/exec_state_workload.cc +++ b/test/syscalls/linux/exec_state_workload.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exit.cc b/test/syscalls/linux/exit.cc index 7246a7b3b..99de2b376 100644 --- a/test/syscalls/linux/exit.cc +++ b/test/syscalls/linux/exit.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/exit_script.sh b/test/syscalls/linux/exit_script.sh index f014fcf99..527518e06 100755 --- a/test/syscalls/linux/exit_script.sh +++ b/test/syscalls/linux/exit_script.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# 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. diff --git a/test/syscalls/linux/fadvise64.cc b/test/syscalls/linux/fadvise64.cc index 041e8b7b6..2af7aa6d9 100644 --- a/test/syscalls/linux/fadvise64.cc +++ b/test/syscalls/linux/fadvise64.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/fallocate.cc b/test/syscalls/linux/fallocate.cc index e51538734..61b8acc7a 100644 --- a/test/syscalls/linux/fallocate.cc +++ b/test/syscalls/linux/fallocate.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/fault.cc b/test/syscalls/linux/fault.cc index cfa7d0d1f..f6e19026f 100644 --- a/test/syscalls/linux/fault.cc +++ b/test/syscalls/linux/fault.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/fchdir.cc b/test/syscalls/linux/fchdir.cc index 2b13e36c3..08bcae1e8 100644 --- a/test/syscalls/linux/fchdir.cc +++ b/test/syscalls/linux/fchdir.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/fcntl.cc b/test/syscalls/linux/fcntl.cc index 32a90a163..2f8e7c9dd 100644 --- a/test/syscalls/linux/fcntl.cc +++ b/test/syscalls/linux/fcntl.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/file_base.h b/test/syscalls/linux/file_base.h index 43f568111..b5b972c07 100644 --- a/test/syscalls/linux/file_base.h +++ b/test/syscalls/linux/file_base.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/flock.cc b/test/syscalls/linux/flock.cc index 1388d3839..d89cfcbd7 100644 --- a/test/syscalls/linux/flock.cc +++ b/test/syscalls/linux/flock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/fork.cc b/test/syscalls/linux/fork.cc index 73ac885b5..dd6e1a422 100644 --- a/test/syscalls/linux/fork.cc +++ b/test/syscalls/linux/fork.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/fpsig_fork.cc b/test/syscalls/linux/fpsig_fork.cc index e8f1dfa8a..e7e9f06a1 100644 --- a/test/syscalls/linux/fpsig_fork.cc +++ b/test/syscalls/linux/fpsig_fork.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/fpsig_nested.cc b/test/syscalls/linux/fpsig_nested.cc index 2fa40b42d..395463aed 100644 --- a/test/syscalls/linux/fpsig_nested.cc +++ b/test/syscalls/linux/fpsig_nested.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/fsync.cc b/test/syscalls/linux/fsync.cc index b34229248..e7e057f06 100644 --- a/test/syscalls/linux/fsync.cc +++ b/test/syscalls/linux/fsync.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/futex.cc b/test/syscalls/linux/futex.cc index c7a709a0a..bfec95466 100644 --- a/test/syscalls/linux/futex.cc +++ b/test/syscalls/linux/futex.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/getcpu.cc b/test/syscalls/linux/getcpu.cc index 3a52b25fa..f4d94bd6a 100644 --- a/test/syscalls/linux/getcpu.cc +++ b/test/syscalls/linux/getcpu.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/getdents.cc b/test/syscalls/linux/getdents.cc index e8a7bcd43..d146c8db7 100644 --- a/test/syscalls/linux/getdents.cc +++ b/test/syscalls/linux/getdents.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/getrandom.cc b/test/syscalls/linux/getrandom.cc index be5325497..f97f60029 100644 --- a/test/syscalls/linux/getrandom.cc +++ b/test/syscalls/linux/getrandom.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/getrusage.cc b/test/syscalls/linux/getrusage.cc index 1ae603858..9bdb1e4cd 100644 --- a/test/syscalls/linux/getrusage.cc +++ b/test/syscalls/linux/getrusage.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/inotify.cc b/test/syscalls/linux/inotify.cc index b99d339e5..6a3539e22 100644 --- a/test/syscalls/linux/inotify.cc +++ b/test/syscalls/linux/inotify.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/ioctl.cc b/test/syscalls/linux/ioctl.cc index c7741a177..c525d41d2 100644 --- a/test/syscalls/linux/ioctl.cc +++ b/test/syscalls/linux/ioctl.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/ip_socket_test_util.cc b/test/syscalls/linux/ip_socket_test_util.cc index 0a149c2e5..7612919d4 100644 --- a/test/syscalls/linux/ip_socket_test_util.cc +++ b/test/syscalls/linux/ip_socket_test_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/ip_socket_test_util.h b/test/syscalls/linux/ip_socket_test_util.h index cac790e64..6898effb8 100644 --- a/test/syscalls/linux/ip_socket_test_util.h +++ b/test/syscalls/linux/ip_socket_test_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/itimer.cc b/test/syscalls/linux/itimer.cc index ddfbc28fc..57ffd1595 100644 --- a/test/syscalls/linux/itimer.cc +++ b/test/syscalls/linux/itimer.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/kill.cc b/test/syscalls/linux/kill.cc index cd98de41f..18ad923b8 100644 --- a/test/syscalls/linux/kill.cc +++ b/test/syscalls/linux/kill.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/link.cc b/test/syscalls/linux/link.cc index ed74437bc..a91703070 100644 --- a/test/syscalls/linux/link.cc +++ b/test/syscalls/linux/link.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/lseek.cc b/test/syscalls/linux/lseek.cc index 6a4f1423c..a8af8e545 100644 --- a/test/syscalls/linux/lseek.cc +++ b/test/syscalls/linux/lseek.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/madvise.cc b/test/syscalls/linux/madvise.cc index a79c8c75d..f6ad4d18b 100644 --- a/test/syscalls/linux/madvise.cc +++ b/test/syscalls/linux/madvise.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/memfd.cc b/test/syscalls/linux/memfd.cc index c2513682d..7e103124b 100644 --- a/test/syscalls/linux/memfd.cc +++ b/test/syscalls/linux/memfd.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/memory_accounting.cc b/test/syscalls/linux/memory_accounting.cc index b4b680c34..a6e20f9c3 100644 --- a/test/syscalls/linux/memory_accounting.cc +++ b/test/syscalls/linux/memory_accounting.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/mempolicy.cc b/test/syscalls/linux/mempolicy.cc index 9f8033bdf..4ac4cb88f 100644 --- a/test/syscalls/linux/mempolicy.cc +++ b/test/syscalls/linux/mempolicy.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/mincore.cc b/test/syscalls/linux/mincore.cc index c572bf5ec..5c1240c89 100644 --- a/test/syscalls/linux/mincore.cc +++ b/test/syscalls/linux/mincore.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/mkdir.cc b/test/syscalls/linux/mkdir.cc index 50807b68f..cf138d328 100644 --- a/test/syscalls/linux/mkdir.cc +++ b/test/syscalls/linux/mkdir.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/mknod.cc b/test/syscalls/linux/mknod.cc index 361ca299b..b1675b9c7 100644 --- a/test/syscalls/linux/mknod.cc +++ b/test/syscalls/linux/mknod.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/mlock.cc b/test/syscalls/linux/mlock.cc index a492b2404..aee4f7d1a 100644 --- a/test/syscalls/linux/mlock.cc +++ b/test/syscalls/linux/mlock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/mmap.cc b/test/syscalls/linux/mmap.cc index a4fb9d1e0..5b5b4c2e8 100644 --- a/test/syscalls/linux/mmap.cc +++ b/test/syscalls/linux/mmap.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/mount.cc b/test/syscalls/linux/mount.cc index 201b83e87..3a17672aa 100644 --- a/test/syscalls/linux/mount.cc +++ b/test/syscalls/linux/mount.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/mremap.cc b/test/syscalls/linux/mremap.cc index 01116c1ab..7298d4ca8 100644 --- a/test/syscalls/linux/mremap.cc +++ b/test/syscalls/linux/mremap.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/msync.cc b/test/syscalls/linux/msync.cc index 5afbfce72..ac7146017 100644 --- a/test/syscalls/linux/msync.cc +++ b/test/syscalls/linux/msync.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/munmap.cc b/test/syscalls/linux/munmap.cc index e20039950..067241f4d 100644 --- a/test/syscalls/linux/munmap.cc +++ b/test/syscalls/linux/munmap.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/open.cc b/test/syscalls/linux/open.cc index 22e4666c2..42646bb02 100644 --- a/test/syscalls/linux/open.cc +++ b/test/syscalls/linux/open.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/open_create.cc b/test/syscalls/linux/open_create.cc index b2cbd63d1..e5a85ef9d 100644 --- a/test/syscalls/linux/open_create.cc +++ b/test/syscalls/linux/open_create.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/partial_bad_buffer.cc b/test/syscalls/linux/partial_bad_buffer.cc index 71288ebc4..83b1ad4e4 100644 --- a/test/syscalls/linux/partial_bad_buffer.cc +++ b/test/syscalls/linux/partial_bad_buffer.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/pause.cc b/test/syscalls/linux/pause.cc index 4e1148c24..8c05efd6f 100644 --- a/test/syscalls/linux/pause.cc +++ b/test/syscalls/linux/pause.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/pipe.cc b/test/syscalls/linux/pipe.cc index abd10b11b..8698295b3 100644 --- a/test/syscalls/linux/pipe.cc +++ b/test/syscalls/linux/pipe.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/poll.cc b/test/syscalls/linux/poll.cc index cd2161bb1..9e5aa7fd0 100644 --- a/test/syscalls/linux/poll.cc +++ b/test/syscalls/linux/poll.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/ppoll.cc b/test/syscalls/linux/ppoll.cc index f8c388c00..8245a11e8 100644 --- a/test/syscalls/linux/ppoll.cc +++ b/test/syscalls/linux/ppoll.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/prctl.cc b/test/syscalls/linux/prctl.cc index 854dec714..bce42dc74 100644 --- a/test/syscalls/linux/prctl.cc +++ b/test/syscalls/linux/prctl.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/prctl_setuid.cc b/test/syscalls/linux/prctl_setuid.cc index c1b561464..00dd6523e 100644 --- a/test/syscalls/linux/prctl_setuid.cc +++ b/test/syscalls/linux/prctl_setuid.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/pread64.cc b/test/syscalls/linux/pread64.cc index 4e5bcfcde..5e3eb1735 100644 --- a/test/syscalls/linux/pread64.cc +++ b/test/syscalls/linux/pread64.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/preadv.cc b/test/syscalls/linux/preadv.cc index 4a31123d8..eebd129f2 100644 --- a/test/syscalls/linux/preadv.cc +++ b/test/syscalls/linux/preadv.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/preadv2.cc b/test/syscalls/linux/preadv2.cc index 58a4f9224..aac960130 100644 --- a/test/syscalls/linux/preadv2.cc +++ b/test/syscalls/linux/preadv2.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/priority.cc b/test/syscalls/linux/priority.cc index 3906c7132..1d9bdfa70 100644 --- a/test/syscalls/linux/priority.cc +++ b/test/syscalls/linux/priority.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/priority_execve.cc b/test/syscalls/linux/priority_execve.cc index 5604bd3d0..5cb343bad 100644 --- a/test/syscalls/linux/priority_execve.cc +++ b/test/syscalls/linux/priority_execve.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/proc.cc b/test/syscalls/linux/proc.cc index 7ba274226..654f26242 100644 --- a/test/syscalls/linux/proc.cc +++ b/test/syscalls/linux/proc.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/proc_net.cc b/test/syscalls/linux/proc_net.cc index 6060d0644..03d0665eb 100644 --- a/test/syscalls/linux/proc_net.cc +++ b/test/syscalls/linux/proc_net.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/proc_net_unix.cc b/test/syscalls/linux/proc_net_unix.cc index ea7c93012..6d745f728 100644 --- a/test/syscalls/linux/proc_net_unix.cc +++ b/test/syscalls/linux/proc_net_unix.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/proc_pid_smaps.cc b/test/syscalls/linux/proc_pid_smaps.cc index cf5c462f3..7f2e8f203 100644 --- a/test/syscalls/linux/proc_pid_smaps.cc +++ b/test/syscalls/linux/proc_pid_smaps.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/proc_pid_uid_gid_map.cc b/test/syscalls/linux/proc_pid_uid_gid_map.cc index 96c58c564..df70b7eb9 100644 --- a/test/syscalls/linux/proc_pid_uid_gid_map.cc +++ b/test/syscalls/linux/proc_pid_uid_gid_map.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/pselect.cc b/test/syscalls/linux/pselect.cc index 3294f6c14..4e43c4d7f 100644 --- a/test/syscalls/linux/pselect.cc +++ b/test/syscalls/linux/pselect.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/ptrace.cc b/test/syscalls/linux/ptrace.cc index e0c56f1fc..4c212836c 100644 --- a/test/syscalls/linux/ptrace.cc +++ b/test/syscalls/linux/ptrace.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/pty.cc b/test/syscalls/linux/pty.cc index 5b2dc9ccb..0485d187c 100644 --- a/test/syscalls/linux/pty.cc +++ b/test/syscalls/linux/pty.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/pwrite64.cc b/test/syscalls/linux/pwrite64.cc index 485b1e48d..e1603fc2d 100644 --- a/test/syscalls/linux/pwrite64.cc +++ b/test/syscalls/linux/pwrite64.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/pwritev2.cc b/test/syscalls/linux/pwritev2.cc index a6949f08e..db519f4e0 100644 --- a/test/syscalls/linux/pwritev2.cc +++ b/test/syscalls/linux/pwritev2.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/raw_socket_ipv4.cc b/test/syscalls/linux/raw_socket_ipv4.cc index 8b8d032cb..e20b5cb50 100644 --- a/test/syscalls/linux/raw_socket_ipv4.cc +++ b/test/syscalls/linux/raw_socket_ipv4.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/read.cc b/test/syscalls/linux/read.cc index eb1b5bc10..4430fa3c2 100644 --- a/test/syscalls/linux/read.cc +++ b/test/syscalls/linux/read.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/readv.cc b/test/syscalls/linux/readv.cc index 0b933673a..f327ec3a9 100644 --- a/test/syscalls/linux/readv.cc +++ b/test/syscalls/linux/readv.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/readv_common.cc b/test/syscalls/linux/readv_common.cc index 349b80d7f..35d2dd9e3 100644 --- a/test/syscalls/linux/readv_common.cc +++ b/test/syscalls/linux/readv_common.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/readv_common.h b/test/syscalls/linux/readv_common.h index e261d545a..b16179fca 100644 --- a/test/syscalls/linux/readv_common.h +++ b/test/syscalls/linux/readv_common.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/readv_socket.cc b/test/syscalls/linux/readv_socket.cc index cf22c395e..3c315cc02 100644 --- a/test/syscalls/linux/readv_socket.cc +++ b/test/syscalls/linux/readv_socket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/rename.cc b/test/syscalls/linux/rename.cc index c0cbc7cd9..c9d76c2e2 100644 --- a/test/syscalls/linux/rename.cc +++ b/test/syscalls/linux/rename.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/rlimits.cc b/test/syscalls/linux/rlimits.cc index 7b255d0f6..860f0f688 100644 --- a/test/syscalls/linux/rlimits.cc +++ b/test/syscalls/linux/rlimits.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/rtsignal.cc b/test/syscalls/linux/rtsignal.cc index ff948f9d5..81d193ffd 100644 --- a/test/syscalls/linux/rtsignal.cc +++ b/test/syscalls/linux/rtsignal.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sched.cc b/test/syscalls/linux/sched.cc index 60cb6c443..735e99411 100644 --- a/test/syscalls/linux/sched.cc +++ b/test/syscalls/linux/sched.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sched_yield.cc b/test/syscalls/linux/sched_yield.cc index fc45aa5c2..5d24f5b58 100644 --- a/test/syscalls/linux/sched_yield.cc +++ b/test/syscalls/linux/sched_yield.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/seccomp.cc b/test/syscalls/linux/seccomp.cc index 27740d7ef..e77586852 100644 --- a/test/syscalls/linux/seccomp.cc +++ b/test/syscalls/linux/seccomp.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/select.cc b/test/syscalls/linux/select.cc index 41e6043cc..88c010aec 100644 --- a/test/syscalls/linux/select.cc +++ b/test/syscalls/linux/select.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/semaphore.cc b/test/syscalls/linux/semaphore.cc index 1c47b6851..421318fcb 100644 --- a/test/syscalls/linux/semaphore.cc +++ b/test/syscalls/linux/semaphore.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sendfile.cc b/test/syscalls/linux/sendfile.cc index 15fd01ff0..2fbb3f4ef 100644 --- a/test/syscalls/linux/sendfile.cc +++ b/test/syscalls/linux/sendfile.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sendfile_socket.cc b/test/syscalls/linux/sendfile_socket.cc index e2ccf17ce..66adda515 100644 --- a/test/syscalls/linux/sendfile_socket.cc +++ b/test/syscalls/linux/sendfile_socket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/shm.cc b/test/syscalls/linux/shm.cc index 2c0f9b04a..eb7a3966f 100644 --- a/test/syscalls/linux/shm.cc +++ b/test/syscalls/linux/shm.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sigaction.cc b/test/syscalls/linux/sigaction.cc index cdd2dbf31..9a53fd3e0 100644 --- a/test/syscalls/linux/sigaction.cc +++ b/test/syscalls/linux/sigaction.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sigaltstack.cc b/test/syscalls/linux/sigaltstack.cc index 5741720f4..7d4a12c1d 100644 --- a/test/syscalls/linux/sigaltstack.cc +++ b/test/syscalls/linux/sigaltstack.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sigaltstack_check.cc b/test/syscalls/linux/sigaltstack_check.cc index b71f812a8..5ac1b661d 100644 --- a/test/syscalls/linux/sigaltstack_check.cc +++ b/test/syscalls/linux/sigaltstack_check.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sigiret.cc b/test/syscalls/linux/sigiret.cc index 1b7cecccb..a47c781ea 100644 --- a/test/syscalls/linux/sigiret.cc +++ b/test/syscalls/linux/sigiret.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sigprocmask.cc b/test/syscalls/linux/sigprocmask.cc index 1aea1ecb8..654c6a47f 100644 --- a/test/syscalls/linux/sigprocmask.cc +++ b/test/syscalls/linux/sigprocmask.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sigstop.cc b/test/syscalls/linux/sigstop.cc index e21d23d51..9c7210e17 100644 --- a/test/syscalls/linux/sigstop.cc +++ b/test/syscalls/linux/sigstop.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sigtimedwait.cc b/test/syscalls/linux/sigtimedwait.cc index 1df9c013f..1e5bf5942 100644 --- a/test/syscalls/linux/sigtimedwait.cc +++ b/test/syscalls/linux/sigtimedwait.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_abstract.cc b/test/syscalls/linux/socket_abstract.cc index 639cd4e59..2faf678f7 100644 --- a/test/syscalls/linux/socket_abstract.cc +++ b/test/syscalls/linux/socket_abstract.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_blocking.cc b/test/syscalls/linux/socket_blocking.cc index c1bca467f..00c50d1bf 100644 --- a/test/syscalls/linux/socket_blocking.cc +++ b/test/syscalls/linux/socket_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_blocking.h b/test/syscalls/linux/socket_blocking.h index 5cddee54b..db26e5ef5 100644 --- a/test/syscalls/linux/socket_blocking.h +++ b/test/syscalls/linux/socket_blocking.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_filesystem.cc b/test/syscalls/linux/socket_filesystem.cc index 2653be158..f7cb72df4 100644 --- a/test/syscalls/linux/socket_filesystem.cc +++ b/test/syscalls/linux/socket_filesystem.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_generic.cc b/test/syscalls/linux/socket_generic.cc index d04d5abe0..f99f3fe62 100644 --- a/test/syscalls/linux/socket_generic.cc +++ b/test/syscalls/linux/socket_generic.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_generic.h b/test/syscalls/linux/socket_generic.h index cd826abcf..00ae7bfc3 100644 --- a/test/syscalls/linux/socket_generic.h +++ b/test/syscalls/linux/socket_generic.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_inet_loopback.cc b/test/syscalls/linux/socket_inet_loopback.cc index 14d7827c2..f86a0f30c 100644 --- a/test/syscalls/linux/socket_inet_loopback.cc +++ b/test/syscalls/linux/socket_inet_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_loopback_blocking.cc b/test/syscalls/linux/socket_ip_loopback_blocking.cc index 9cec7a71d..d7fc20aad 100644 --- a/test/syscalls/linux/socket_ip_loopback_blocking.cc +++ b/test/syscalls/linux/socket_ip_loopback_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_tcp_generic.cc b/test/syscalls/linux/socket_ip_tcp_generic.cc index 54f00cd9b..5b198f49d 100644 --- a/test/syscalls/linux/socket_ip_tcp_generic.cc +++ b/test/syscalls/linux/socket_ip_tcp_generic.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_tcp_generic.h b/test/syscalls/linux/socket_ip_tcp_generic.h index f38500d14..a3eff3c73 100644 --- a/test/syscalls/linux/socket_ip_tcp_generic.h +++ b/test/syscalls/linux/socket_ip_tcp_generic.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_tcp_generic_loopback.cc b/test/syscalls/linux/socket_ip_tcp_generic_loopback.cc index 1963d5deb..2c6ae17bf 100644 --- a/test/syscalls/linux/socket_ip_tcp_generic_loopback.cc +++ b/test/syscalls/linux/socket_ip_tcp_generic_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_tcp_loopback.cc b/test/syscalls/linux/socket_ip_tcp_loopback.cc index 7e36c35d2..831de53b8 100644 --- a/test/syscalls/linux/socket_ip_tcp_loopback.cc +++ b/test/syscalls/linux/socket_ip_tcp_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc b/test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc index 9e2a18d3e..d1ea8ef12 100644 --- a/test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc +++ b/test/syscalls/linux/socket_ip_tcp_loopback_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc b/test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc index 54053360f..96c1b3b3d 100644 --- a/test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc +++ b/test/syscalls/linux/socket_ip_tcp_loopback_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_tcp_udp_generic.cc b/test/syscalls/linux/socket_ip_tcp_udp_generic.cc index 5bf1de7c6..251817a9f 100644 --- a/test/syscalls/linux/socket_ip_tcp_udp_generic.cc +++ b/test/syscalls/linux/socket_ip_tcp_udp_generic.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_udp_generic.cc b/test/syscalls/linux/socket_ip_udp_generic.cc index ac15154f2..044394ba7 100644 --- a/test/syscalls/linux/socket_ip_udp_generic.cc +++ b/test/syscalls/linux/socket_ip_udp_generic.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_udp_generic.h b/test/syscalls/linux/socket_ip_udp_generic.h index 8b8fc7c6e..106c54e9f 100644 --- a/test/syscalls/linux/socket_ip_udp_generic.h +++ b/test/syscalls/linux/socket_ip_udp_generic.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_udp_loopback.cc b/test/syscalls/linux/socket_ip_udp_loopback.cc index 0e4463649..fc124e9ef 100644 --- a/test/syscalls/linux/socket_ip_udp_loopback.cc +++ b/test/syscalls/linux/socket_ip_udp_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_udp_loopback_blocking.cc b/test/syscalls/linux/socket_ip_udp_loopback_blocking.cc index 0c3b669bf..1c3d1c0ad 100644 --- a/test/syscalls/linux/socket_ip_udp_loopback_blocking.cc +++ b/test/syscalls/linux/socket_ip_udp_loopback_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc b/test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc index 7bf8597fe..7554b08d5 100644 --- a/test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc +++ b/test/syscalls/linux/socket_ip_udp_loopback_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc index 8e1c13ff4..3a068aacf 100644 --- a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc +++ b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h index b23de08d1..fb582b224 100644 --- a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h +++ b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking_test.cc b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking_test.cc index 773d84b13..040bb176e 100644 --- a/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking_test.cc +++ b/test/syscalls/linux/socket_ipv4_tcp_unbound_external_networking_test.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound.cc b/test/syscalls/linux/socket_ipv4_udp_unbound.cc index c99958ed5..709172580 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound.h b/test/syscalls/linux/socket_ipv4_udp_unbound.h index a780c0144..8e07bfbbf 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound.h +++ b/test/syscalls/linux/socket_ipv4_udp_unbound.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc index 9dd9e1bd6..53dcd58cd 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h index 5cf9fa8eb..45e1d37ea 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking_test.cc b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking_test.cc index 535a5fa10..ffbb8e6eb 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking_test.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_external_networking_test.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc b/test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc index d6a8e428c..cb0105471 100644 --- a/test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc +++ b/test/syscalls/linux/socket_ipv4_udp_unbound_loopback.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_netdevice.cc b/test/syscalls/linux/socket_netdevice.cc index b4e9fe51b..6a5fa8965 100644 --- a/test/syscalls/linux/socket_netdevice.cc +++ b/test/syscalls/linux/socket_netdevice.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_netlink_route.cc b/test/syscalls/linux/socket_netlink_route.cc index ed4ae1c71..c8693225f 100644 --- a/test/syscalls/linux/socket_netlink_route.cc +++ b/test/syscalls/linux/socket_netlink_route.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_netlink_util.cc b/test/syscalls/linux/socket_netlink_util.cc index edf549544..728d25434 100644 --- a/test/syscalls/linux/socket_netlink_util.cc +++ b/test/syscalls/linux/socket_netlink_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_netlink_util.h b/test/syscalls/linux/socket_netlink_util.h index 44b1f148c..bea449107 100644 --- a/test/syscalls/linux/socket_netlink_util.h +++ b/test/syscalls/linux/socket_netlink_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_non_blocking.cc b/test/syscalls/linux/socket_non_blocking.cc index 1bcc6fb7f..73e6dc618 100644 --- a/test/syscalls/linux/socket_non_blocking.cc +++ b/test/syscalls/linux/socket_non_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_non_blocking.h b/test/syscalls/linux/socket_non_blocking.h index 287e096bb..bd3e02fd2 100644 --- a/test/syscalls/linux/socket_non_blocking.h +++ b/test/syscalls/linux/socket_non_blocking.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_non_stream.cc b/test/syscalls/linux/socket_non_stream.cc index d170008a4..3c599b6e8 100644 --- a/test/syscalls/linux/socket_non_stream.cc +++ b/test/syscalls/linux/socket_non_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_non_stream.h b/test/syscalls/linux/socket_non_stream.h index 02dd2a958..469fbe6a2 100644 --- a/test/syscalls/linux/socket_non_stream.h +++ b/test/syscalls/linux/socket_non_stream.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_non_stream_blocking.cc b/test/syscalls/linux/socket_non_stream_blocking.cc index 9e92628c3..76127d181 100644 --- a/test/syscalls/linux/socket_non_stream_blocking.cc +++ b/test/syscalls/linux/socket_non_stream_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_non_stream_blocking.h b/test/syscalls/linux/socket_non_stream_blocking.h index bde355452..6e205a039 100644 --- a/test/syscalls/linux/socket_non_stream_blocking.h +++ b/test/syscalls/linux/socket_non_stream_blocking.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_stream.cc b/test/syscalls/linux/socket_stream.cc index c8a8ad0f6..0417dd347 100644 --- a/test/syscalls/linux/socket_stream.cc +++ b/test/syscalls/linux/socket_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_stream.h b/test/syscalls/linux/socket_stream.h index 35e591e17..b837b8f8c 100644 --- a/test/syscalls/linux/socket_stream.h +++ b/test/syscalls/linux/socket_stream.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_stream_blocking.cc b/test/syscalls/linux/socket_stream_blocking.cc index f0f86c01c..8367460d2 100644 --- a/test/syscalls/linux/socket_stream_blocking.cc +++ b/test/syscalls/linux/socket_stream_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_stream_blocking.h b/test/syscalls/linux/socket_stream_blocking.h index 06113ad03..9fd19ff90 100644 --- a/test/syscalls/linux/socket_stream_blocking.h +++ b/test/syscalls/linux/socket_stream_blocking.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_stream_nonblock.cc b/test/syscalls/linux/socket_stream_nonblock.cc index a3202ffe4..b00748b97 100644 --- a/test/syscalls/linux/socket_stream_nonblock.cc +++ b/test/syscalls/linux/socket_stream_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_stream_nonblock.h b/test/syscalls/linux/socket_stream_nonblock.h index 491f53848..c3b7fad91 100644 --- a/test/syscalls/linux/socket_stream_nonblock.h +++ b/test/syscalls/linux/socket_stream_nonblock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_test_util.cc b/test/syscalls/linux/socket_test_util.cc index 0be23e541..da69de37c 100644 --- a/test/syscalls/linux/socket_test_util.cc +++ b/test/syscalls/linux/socket_test_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_test_util.h b/test/syscalls/linux/socket_test_util.h index dfabdf179..058313986 100644 --- a/test/syscalls/linux/socket_test_util.h +++ b/test/syscalls/linux/socket_test_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix.cc b/test/syscalls/linux/socket_unix.cc index fafb23ad1..bb3397fa2 100644 --- a/test/syscalls/linux/socket_unix.cc +++ b/test/syscalls/linux/socket_unix.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix.h b/test/syscalls/linux/socket_unix.h index d2a16afb2..3625cc404 100644 --- a/test/syscalls/linux/socket_unix.h +++ b/test/syscalls/linux/socket_unix.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_abstract.cc b/test/syscalls/linux/socket_unix_abstract.cc index c4a3c889c..8241bf997 100644 --- a/test/syscalls/linux/socket_unix_abstract.cc +++ b/test/syscalls/linux/socket_unix_abstract.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_abstract_nonblock.cc b/test/syscalls/linux/socket_unix_abstract_nonblock.cc index a69ee027e..9de0f6dfe 100644 --- a/test/syscalls/linux/socket_unix_abstract_nonblock.cc +++ b/test/syscalls/linux/socket_unix_abstract_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_blocking_local.cc b/test/syscalls/linux/socket_unix_blocking_local.cc index 57af118c5..320915b0f 100644 --- a/test/syscalls/linux/socket_unix_blocking_local.cc +++ b/test/syscalls/linux/socket_unix_blocking_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_dgram.cc b/test/syscalls/linux/socket_unix_dgram.cc index 5dd5e6d77..3e0f611d2 100644 --- a/test/syscalls/linux/socket_unix_dgram.cc +++ b/test/syscalls/linux/socket_unix_dgram.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_dgram.h b/test/syscalls/linux/socket_unix_dgram.h index 722a3d8e6..0764ef85b 100644 --- a/test/syscalls/linux/socket_unix_dgram.h +++ b/test/syscalls/linux/socket_unix_dgram.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_dgram_local.cc b/test/syscalls/linux/socket_unix_dgram_local.cc index da8f59704..4ba2c80ae 100644 --- a/test/syscalls/linux/socket_unix_dgram_local.cc +++ b/test/syscalls/linux/socket_unix_dgram_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_dgram_non_blocking.cc b/test/syscalls/linux/socket_unix_dgram_non_blocking.cc index 3becb513d..9fe86cee8 100644 --- a/test/syscalls/linux/socket_unix_dgram_non_blocking.cc +++ b/test/syscalls/linux/socket_unix_dgram_non_blocking.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_domain.cc b/test/syscalls/linux/socket_unix_domain.cc index f081c601f..fa3efc7f8 100644 --- a/test/syscalls/linux/socket_unix_domain.cc +++ b/test/syscalls/linux/socket_unix_domain.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_filesystem.cc b/test/syscalls/linux/socket_unix_filesystem.cc index 6a67da75f..5dbe67773 100644 --- a/test/syscalls/linux/socket_unix_filesystem.cc +++ b/test/syscalls/linux/socket_unix_filesystem.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_filesystem_nonblock.cc b/test/syscalls/linux/socket_unix_filesystem_nonblock.cc index c13a1e564..137db53c4 100644 --- a/test/syscalls/linux/socket_unix_filesystem_nonblock.cc +++ b/test/syscalls/linux/socket_unix_filesystem_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_non_stream.cc b/test/syscalls/linux/socket_unix_non_stream.cc index a565978f9..dafe82494 100644 --- a/test/syscalls/linux/socket_unix_non_stream.cc +++ b/test/syscalls/linux/socket_unix_non_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_non_stream.h b/test/syscalls/linux/socket_unix_non_stream.h index e4214d949..7478ab172 100644 --- a/test/syscalls/linux/socket_unix_non_stream.h +++ b/test/syscalls/linux/socket_unix_non_stream.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_non_stream_blocking_local.cc b/test/syscalls/linux/socket_unix_non_stream_blocking_local.cc index 6c435669b..98cf1fe8a 100644 --- a/test/syscalls/linux/socket_unix_non_stream_blocking_local.cc +++ b/test/syscalls/linux/socket_unix_non_stream_blocking_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_pair.cc b/test/syscalls/linux/socket_unix_pair.cc index c575fdcb2..bacfc11e4 100644 --- a/test/syscalls/linux/socket_unix_pair.cc +++ b/test/syscalls/linux/socket_unix_pair.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_pair_nonblock.cc b/test/syscalls/linux/socket_unix_pair_nonblock.cc index 1ae7f9b5e..583506f08 100644 --- a/test/syscalls/linux/socket_unix_pair_nonblock.cc +++ b/test/syscalls/linux/socket_unix_pair_nonblock.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_seqpacket.cc b/test/syscalls/linux/socket_unix_seqpacket.cc index ad0af77e9..6f6367dd5 100644 --- a/test/syscalls/linux/socket_unix_seqpacket.cc +++ b/test/syscalls/linux/socket_unix_seqpacket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_seqpacket.h b/test/syscalls/linux/socket_unix_seqpacket.h index da8eb2b2b..30d9b9edf 100644 --- a/test/syscalls/linux/socket_unix_seqpacket.h +++ b/test/syscalls/linux/socket_unix_seqpacket.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_seqpacket_local.cc b/test/syscalls/linux/socket_unix_seqpacket_local.cc index e6484d9b4..b903a9e8f 100644 --- a/test/syscalls/linux/socket_unix_seqpacket_local.cc +++ b/test/syscalls/linux/socket_unix_seqpacket_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_stream.cc b/test/syscalls/linux/socket_unix_stream.cc index 95f454251..659c93945 100644 --- a/test/syscalls/linux/socket_unix_stream.cc +++ b/test/syscalls/linux/socket_unix_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_stream_blocking_local.cc b/test/syscalls/linux/socket_unix_stream_blocking_local.cc index ec0fc6955..ce0f1e50d 100644 --- a/test/syscalls/linux/socket_unix_stream_blocking_local.cc +++ b/test/syscalls/linux/socket_unix_stream_blocking_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_stream_local.cc b/test/syscalls/linux/socket_unix_stream_local.cc index bf4c5f2eb..6b840189c 100644 --- a/test/syscalls/linux/socket_unix_stream_local.cc +++ b/test/syscalls/linux/socket_unix_stream_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_stream_nonblock_local.cc b/test/syscalls/linux/socket_unix_stream_nonblock_local.cc index df80b105a..ebec4e0ec 100644 --- a/test/syscalls/linux/socket_unix_stream_nonblock_local.cc +++ b/test/syscalls/linux/socket_unix_stream_nonblock_local.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_unbound_abstract.cc b/test/syscalls/linux/socket_unix_unbound_abstract.cc index b6fe7a9ce..4b5832de8 100644 --- a/test/syscalls/linux/socket_unix_unbound_abstract.cc +++ b/test/syscalls/linux/socket_unix_unbound_abstract.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_unbound_dgram.cc b/test/syscalls/linux/socket_unix_unbound_dgram.cc index 1ec11a08d..2ddc5c11f 100644 --- a/test/syscalls/linux/socket_unix_unbound_dgram.cc +++ b/test/syscalls/linux/socket_unix_unbound_dgram.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_unbound_filesystem.cc b/test/syscalls/linux/socket_unix_unbound_filesystem.cc index d09142aa6..8cb03c450 100644 --- a/test/syscalls/linux/socket_unix_unbound_filesystem.cc +++ b/test/syscalls/linux/socket_unix_unbound_filesystem.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_unbound_seqpacket.cc b/test/syscalls/linux/socket_unix_unbound_seqpacket.cc index 21209b244..0575f2e1d 100644 --- a/test/syscalls/linux/socket_unix_unbound_seqpacket.cc +++ b/test/syscalls/linux/socket_unix_unbound_seqpacket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/socket_unix_unbound_stream.cc b/test/syscalls/linux/socket_unix_unbound_stream.cc index b95f9569e..091d546b3 100644 --- a/test/syscalls/linux/socket_unix_unbound_stream.cc +++ b/test/syscalls/linux/socket_unix_unbound_stream.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/stat.cc b/test/syscalls/linux/stat.cc index 746318d09..80ba67496 100644 --- a/test/syscalls/linux/stat.cc +++ b/test/syscalls/linux/stat.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/stat_times.cc b/test/syscalls/linux/stat_times.cc index 8346e9a8e..9b53739a0 100644 --- a/test/syscalls/linux/stat_times.cc +++ b/test/syscalls/linux/stat_times.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/statfs.cc b/test/syscalls/linux/statfs.cc index e1e7fc707..aca51d30f 100644 --- a/test/syscalls/linux/statfs.cc +++ b/test/syscalls/linux/statfs.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sticky.cc b/test/syscalls/linux/sticky.cc index 58cf0d014..59fb5dfe6 100644 --- a/test/syscalls/linux/sticky.cc +++ b/test/syscalls/linux/sticky.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/symlink.cc b/test/syscalls/linux/symlink.cc index 318917f4b..494072a9b 100644 --- a/test/syscalls/linux/symlink.cc +++ b/test/syscalls/linux/symlink.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sync.cc b/test/syscalls/linux/sync.cc index 5b777b6eb..fe479390d 100644 --- a/test/syscalls/linux/sync.cc +++ b/test/syscalls/linux/sync.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sync_file_range.cc b/test/syscalls/linux/sync_file_range.cc index d11f58481..36cc42043 100644 --- a/test/syscalls/linux/sync_file_range.cc +++ b/test/syscalls/linux/sync_file_range.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sysinfo.cc b/test/syscalls/linux/sysinfo.cc index a0dd82640..1a71256da 100644 --- a/test/syscalls/linux/sysinfo.cc +++ b/test/syscalls/linux/sysinfo.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/syslog.cc b/test/syscalls/linux/syslog.cc index 5bd0d1cc3..9a7407d96 100644 --- a/test/syscalls/linux/syslog.cc +++ b/test/syscalls/linux/syslog.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/sysret.cc b/test/syscalls/linux/sysret.cc index 8e10220eb..819fa655a 100644 --- a/test/syscalls/linux/sysret.cc +++ b/test/syscalls/linux/sysret.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/tcp_socket.cc b/test/syscalls/linux/tcp_socket.cc index 33620a874..e3f9f9f9d 100644 --- a/test/syscalls/linux/tcp_socket.cc +++ b/test/syscalls/linux/tcp_socket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/temp_umask.h b/test/syscalls/linux/temp_umask.h index f202dfa59..81a25440c 100644 --- a/test/syscalls/linux/temp_umask.h +++ b/test/syscalls/linux/temp_umask.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/tgkill.cc b/test/syscalls/linux/tgkill.cc index 2d258ef11..80acae5de 100644 --- a/test/syscalls/linux/tgkill.cc +++ b/test/syscalls/linux/tgkill.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/time.cc b/test/syscalls/linux/time.cc index 5a3dfd026..c7eead17e 100644 --- a/test/syscalls/linux/time.cc +++ b/test/syscalls/linux/time.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/timerfd.cc b/test/syscalls/linux/timerfd.cc index b85321795..9df53612f 100644 --- a/test/syscalls/linux/timerfd.cc +++ b/test/syscalls/linux/timerfd.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/timers.cc b/test/syscalls/linux/timers.cc index 14506eb12..fd42e81e1 100644 --- a/test/syscalls/linux/timers.cc +++ b/test/syscalls/linux/timers.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/tkill.cc b/test/syscalls/linux/tkill.cc index 3e8ce5327..bae377c69 100644 --- a/test/syscalls/linux/tkill.cc +++ b/test/syscalls/linux/tkill.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/truncate.cc b/test/syscalls/linux/truncate.cc index 2616a9147..e5cc5d97c 100644 --- a/test/syscalls/linux/truncate.cc +++ b/test/syscalls/linux/truncate.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/udp_bind.cc b/test/syscalls/linux/udp_bind.cc index 547eb2a6c..6d92bdbeb 100644 --- a/test/syscalls/linux/udp_bind.cc +++ b/test/syscalls/linux/udp_bind.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/udp_socket.cc b/test/syscalls/linux/udp_socket.cc index f39281d5c..31db8a2ad 100644 --- a/test/syscalls/linux/udp_socket.cc +++ b/test/syscalls/linux/udp_socket.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/uidgid.cc b/test/syscalls/linux/uidgid.cc index d78a09b1e..bf1ca8679 100644 --- a/test/syscalls/linux/uidgid.cc +++ b/test/syscalls/linux/uidgid.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/uname.cc b/test/syscalls/linux/uname.cc index d22a34bd7..0a5d91017 100644 --- a/test/syscalls/linux/uname.cc +++ b/test/syscalls/linux/uname.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/unix_domain_socket_test_util.cc b/test/syscalls/linux/unix_domain_socket_test_util.cc index 2d7a530b9..6f49e3660 100644 --- a/test/syscalls/linux/unix_domain_socket_test_util.cc +++ b/test/syscalls/linux/unix_domain_socket_test_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/unix_domain_socket_test_util.h b/test/syscalls/linux/unix_domain_socket_test_util.h index 1b09aeae7..aae990245 100644 --- a/test/syscalls/linux/unix_domain_socket_test_util.h +++ b/test/syscalls/linux/unix_domain_socket_test_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/unlink.cc b/test/syscalls/linux/unlink.cc index b10aae025..b6f65e027 100644 --- a/test/syscalls/linux/unlink.cc +++ b/test/syscalls/linux/unlink.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/unshare.cc b/test/syscalls/linux/unshare.cc index 9dd6ec4b6..e32619efe 100644 --- a/test/syscalls/linux/unshare.cc +++ b/test/syscalls/linux/unshare.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/utimes.cc b/test/syscalls/linux/utimes.cc index bf776cd93..80716859a 100644 --- a/test/syscalls/linux/utimes.cc +++ b/test/syscalls/linux/utimes.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/vdso.cc b/test/syscalls/linux/vdso.cc index 0f6e1c7c6..19c80add8 100644 --- a/test/syscalls/linux/vdso.cc +++ b/test/syscalls/linux/vdso.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/vdso_clock_gettime.cc b/test/syscalls/linux/vdso_clock_gettime.cc index 0e936594b..759a50569 100644 --- a/test/syscalls/linux/vdso_clock_gettime.cc +++ b/test/syscalls/linux/vdso_clock_gettime.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/vfork.cc b/test/syscalls/linux/vfork.cc index 9999a909e..631a53654 100644 --- a/test/syscalls/linux/vfork.cc +++ b/test/syscalls/linux/vfork.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/vsyscall.cc b/test/syscalls/linux/vsyscall.cc index cb6840cc6..2c2303358 100644 --- a/test/syscalls/linux/vsyscall.cc +++ b/test/syscalls/linux/vsyscall.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/wait.cc b/test/syscalls/linux/wait.cc index fcd606bec..50d0725a7 100644 --- a/test/syscalls/linux/wait.cc +++ b/test/syscalls/linux/wait.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/linux/write.cc b/test/syscalls/linux/write.cc index 7f80b2fa8..9b219cfd6 100644 --- a/test/syscalls/linux/write.cc +++ b/test/syscalls/linux/write.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/syscall_test_runner.go b/test/syscalls/syscall_test_runner.go index c4af28103..28f312b8b 100644 --- a/test/syscalls/syscall_test_runner.go +++ b/test/syscalls/syscall_test_runner.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/syscalls/syscall_test_runner.sh b/test/syscalls/syscall_test_runner.sh index 87d62786b..864bb2de4 100755 --- a/test/syscalls/syscall_test_runner.sh +++ b/test/syscalls/syscall_test_runner.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# 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. diff --git a/test/util/capability_util.cc b/test/util/capability_util.cc index d1dd95e76..5d733887b 100644 --- a/test/util/capability_util.cc +++ b/test/util/capability_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/capability_util.h b/test/util/capability_util.h index 8708f5e69..e968a2583 100644 --- a/test/util/capability_util.h +++ b/test/util/capability_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/cleanup.h b/test/util/cleanup.h index fb4724f97..c76482ef4 100644 --- a/test/util/cleanup.h +++ b/test/util/cleanup.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/epoll_util.cc b/test/util/epoll_util.cc index 0b95aa8cd..2e5051468 100644 --- a/test/util/epoll_util.cc +++ b/test/util/epoll_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/epoll_util.h b/test/util/epoll_util.h index 521e7a3d3..f233b37d5 100644 --- a/test/util/epoll_util.h +++ b/test/util/epoll_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/eventfd_util.h b/test/util/eventfd_util.h index 1fdb07d3b..cb9ce829c 100644 --- a/test/util/eventfd_util.h +++ b/test/util/eventfd_util.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/file_descriptor.h b/test/util/file_descriptor.h index be8812d01..fc5caa55b 100644 --- a/test/util/file_descriptor.h +++ b/test/util/file_descriptor.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/fs_util.cc b/test/util/fs_util.cc index 6bd424417..bc90bd78e 100644 --- a/test/util/fs_util.cc +++ b/test/util/fs_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/fs_util.h b/test/util/fs_util.h index 9412b2f71..eb7cdaa24 100644 --- a/test/util/fs_util.h +++ b/test/util/fs_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/fs_util_test.cc b/test/util/fs_util_test.cc index ce70d58aa..4e12076a1 100644 --- a/test/util/fs_util_test.cc +++ b/test/util/fs_util_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/logging.cc b/test/util/logging.cc index 86ea71df3..cc71d77b0 100644 --- a/test/util/logging.cc +++ b/test/util/logging.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/logging.h b/test/util/logging.h index 6e957b172..589166fab 100644 --- a/test/util/logging.h +++ b/test/util/logging.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/memory_util.h b/test/util/memory_util.h index 8f6e99ba6..8c77778ea 100644 --- a/test/util/memory_util.h +++ b/test/util/memory_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/mount_util.h b/test/util/mount_util.h index 468170646..7782e6bf2 100644 --- a/test/util/mount_util.h +++ b/test/util/mount_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/multiprocess_util.cc b/test/util/multiprocess_util.cc index 12637db8c..95f5f3b4f 100644 --- a/test/util/multiprocess_util.cc +++ b/test/util/multiprocess_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/multiprocess_util.h b/test/util/multiprocess_util.h index ba5f2601f..0aecd3439 100644 --- a/test/util/multiprocess_util.h +++ b/test/util/multiprocess_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/posix_error.cc b/test/util/posix_error.cc index ead9ede16..cebf7e0ac 100644 --- a/test/util/posix_error.cc +++ b/test/util/posix_error.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/posix_error.h b/test/util/posix_error.h index 2a66e2e94..b604f4f8f 100644 --- a/test/util/posix_error.h +++ b/test/util/posix_error.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/posix_error_test.cc b/test/util/posix_error_test.cc index c5427b8e5..d67270842 100644 --- a/test/util/posix_error_test.cc +++ b/test/util/posix_error_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/proc_util.cc b/test/util/proc_util.cc index 2d9eb1986..9d4db37c3 100644 --- a/test/util/proc_util.cc +++ b/test/util/proc_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/proc_util.h b/test/util/proc_util.h index e1ee2db9c..af209a51e 100644 --- a/test/util/proc_util.h +++ b/test/util/proc_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/proc_util_test.cc b/test/util/proc_util_test.cc index 75335415a..71dd2355e 100644 --- a/test/util/proc_util_test.cc +++ b/test/util/proc_util_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/rlimit_util.cc b/test/util/rlimit_util.cc index a9912c372..684253f78 100644 --- a/test/util/rlimit_util.cc +++ b/test/util/rlimit_util.cc @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/rlimit_util.h b/test/util/rlimit_util.h index fa5cc70dc..873252a32 100644 --- a/test/util/rlimit_util.h +++ b/test/util/rlimit_util.h @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/test/util/save_util.cc b/test/util/save_util.cc index 5540e2146..05f52b80d 100644 --- a/test/util/save_util.cc +++ b/test/util/save_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/save_util.h b/test/util/save_util.h index 919e4af3d..90460701e 100644 --- a/test/util/save_util.h +++ b/test/util/save_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/signal_util.cc b/test/util/signal_util.cc index 3e2df32a6..26738864f 100644 --- a/test/util/signal_util.cc +++ b/test/util/signal_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/signal_util.h b/test/util/signal_util.h index 80f1808f6..7fd2af015 100644 --- a/test/util/signal_util.h +++ b/test/util/signal_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/temp_path.cc b/test/util/temp_path.cc index 48ce82d20..c5d8fc635 100644 --- a/test/util/temp_path.cc +++ b/test/util/temp_path.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/temp_path.h b/test/util/temp_path.h index 33eb6a72c..89302e0fd 100644 --- a/test/util/temp_path.h +++ b/test/util/temp_path.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/test_main.cc b/test/util/test_main.cc index 4c6b5e860..5c7ee0064 100644 --- a/test/util/test_main.cc +++ b/test/util/test_main.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/test_util.cc b/test/util/test_util.cc index 9b7cfa4dc..c52fd9a4a 100644 --- a/test/util/test_util.cc +++ b/test/util/test_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/test_util.h b/test/util/test_util.h index 905412b24..8f5eb5089 100644 --- a/test/util/test_util.h +++ b/test/util/test_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/test_util_test.cc b/test/util/test_util_test.cc index 5889651d1..b7300d9e5 100644 --- a/test/util/test_util_test.cc +++ b/test/util/test_util_test.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/thread_util.h b/test/util/thread_util.h index df09ac8cf..860e77531 100644 --- a/test/util/thread_util.h +++ b/test/util/thread_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/timer_util.cc b/test/util/timer_util.cc index 681fafb69..43a26b0d3 100644 --- a/test/util/timer_util.cc +++ b/test/util/timer_util.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/test/util/timer_util.h b/test/util/timer_util.h index 9bdc51a57..2cebfa5d1 100644 --- a/test/util/timer_util.h +++ b/test/util/timer_util.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/third_party/gvsync/atomicptr_unsafe.go b/third_party/gvsync/atomicptr_unsafe.go index da9f16240..53a943282 100644 --- a/third_party/gvsync/atomicptr_unsafe.go +++ b/third_party/gvsync/atomicptr_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/atomicptrtest/atomicptr_test.go b/third_party/gvsync/atomicptrtest/atomicptr_test.go index 15d0936d4..8fdc5112e 100644 --- a/third_party/gvsync/atomicptrtest/atomicptr_test.go +++ b/third_party/gvsync/atomicptrtest/atomicptr_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/downgradable_rwmutex_test.go b/third_party/gvsync/downgradable_rwmutex_test.go index 6517dd5dc..40c384b8b 100644 --- a/third_party/gvsync/downgradable_rwmutex_test.go +++ b/third_party/gvsync/downgradable_rwmutex_test.go @@ -1,5 +1,5 @@ // Copyright 2009 The Go Authors. All rights reserved. -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/downgradable_rwmutex_unsafe.go b/third_party/gvsync/downgradable_rwmutex_unsafe.go index 131f0a2ba..4d43eb765 100644 --- a/third_party/gvsync/downgradable_rwmutex_unsafe.go +++ b/third_party/gvsync/downgradable_rwmutex_unsafe.go @@ -1,5 +1,5 @@ // Copyright 2009 The Go Authors. All rights reserved. -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/gvsync.go b/third_party/gvsync/gvsync.go index 46a2565fd..3bbef13c3 100644 --- a/third_party/gvsync/gvsync.go +++ b/third_party/gvsync/gvsync.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/memmove_unsafe.go b/third_party/gvsync/memmove_unsafe.go index d483fc739..4c8aa9ab6 100644 --- a/third_party/gvsync/memmove_unsafe.go +++ b/third_party/gvsync/memmove_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/norace_unsafe.go b/third_party/gvsync/norace_unsafe.go index f9c88d13f..e3852db8c 100644 --- a/third_party/gvsync/norace_unsafe.go +++ b/third_party/gvsync/norace_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/race_unsafe.go b/third_party/gvsync/race_unsafe.go index 2cdcdf7f7..13c02a830 100644 --- a/third_party/gvsync/race_unsafe.go +++ b/third_party/gvsync/race_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/seqatomic_unsafe.go b/third_party/gvsync/seqatomic_unsafe.go index ef61503e2..c52d378f1 100644 --- a/third_party/gvsync/seqatomic_unsafe.go +++ b/third_party/gvsync/seqatomic_unsafe.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/seqatomictest/seqatomic_test.go b/third_party/gvsync/seqatomictest/seqatomic_test.go index d0c373bae..2da73cf96 100644 --- a/third_party/gvsync/seqatomictest/seqatomic_test.go +++ b/third_party/gvsync/seqatomictest/seqatomic_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/third_party/gvsync/seqcount.go b/third_party/gvsync/seqcount.go index c7ae91cfa..2c9c2c3d6 100644 --- a/third_party/gvsync/seqcount.go +++ b/third_party/gvsync/seqcount.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/third_party/gvsync/seqcount_test.go b/third_party/gvsync/seqcount_test.go index ee6579ed8..085e574b3 100644 --- a/third_party/gvsync/seqcount_test.go +++ b/third_party/gvsync/seqcount_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 Google LLC +// Copyright 2019 The gVisor Authors. // // Use of this source code is governed by a BSD-style // license that can be found in the LICENSE file. diff --git a/tools/go_generics/generics.go b/tools/go_generics/generics.go index eaf5c4970..ca414d8cb 100644 --- a/tools/go_generics/generics.go +++ b/tools/go_generics/generics.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/all_stmts/input.go b/tools/go_generics/generics_tests/all_stmts/input.go index 19184a3fe..4791d1ff1 100644 --- a/tools/go_generics/generics_tests/all_stmts/input.go +++ b/tools/go_generics/generics_tests/all_stmts/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/all_stmts/output/output.go b/tools/go_generics/generics_tests/all_stmts/output/output.go index 51582346c..a53d84535 100644 --- a/tools/go_generics/generics_tests/all_stmts/output/output.go +++ b/tools/go_generics/generics_tests/all_stmts/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/all_types/input.go b/tools/go_generics/generics_tests/all_types/input.go index ed6e97c29..3575d02ec 100644 --- a/tools/go_generics/generics_tests/all_types/input.go +++ b/tools/go_generics/generics_tests/all_types/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/all_types/lib/lib.go b/tools/go_generics/generics_tests/all_types/lib/lib.go index 7e73e678e..988786496 100644 --- a/tools/go_generics/generics_tests/all_types/lib/lib.go +++ b/tools/go_generics/generics_tests/all_types/lib/lib.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/all_types/output/output.go b/tools/go_generics/generics_tests/all_types/output/output.go index ec09a6be4..41fd147a1 100644 --- a/tools/go_generics/generics_tests/all_types/output/output.go +++ b/tools/go_generics/generics_tests/all_types/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/consts/input.go b/tools/go_generics/generics_tests/consts/input.go index 394bcc262..04b95fcc6 100644 --- a/tools/go_generics/generics_tests/consts/input.go +++ b/tools/go_generics/generics_tests/consts/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/consts/output/output.go b/tools/go_generics/generics_tests/consts/output/output.go index 91a07fdc2..18d316cc9 100644 --- a/tools/go_generics/generics_tests/consts/output/output.go +++ b/tools/go_generics/generics_tests/consts/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/imports/input.go b/tools/go_generics/generics_tests/imports/input.go index 22e6641a6..0f032c2a1 100644 --- a/tools/go_generics/generics_tests/imports/input.go +++ b/tools/go_generics/generics_tests/imports/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/imports/output/output.go b/tools/go_generics/generics_tests/imports/output/output.go index 2555c0004..2488ca58c 100644 --- a/tools/go_generics/generics_tests/imports/output/output.go +++ b/tools/go_generics/generics_tests/imports/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/remove_typedef/input.go b/tools/go_generics/generics_tests/remove_typedef/input.go index d9c9b8530..cf632bae7 100644 --- a/tools/go_generics/generics_tests/remove_typedef/input.go +++ b/tools/go_generics/generics_tests/remove_typedef/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/remove_typedef/output/output.go b/tools/go_generics/generics_tests/remove_typedef/output/output.go index f111a9426..d44fd8e1c 100644 --- a/tools/go_generics/generics_tests/remove_typedef/output/output.go +++ b/tools/go_generics/generics_tests/remove_typedef/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/simple/input.go b/tools/go_generics/generics_tests/simple/input.go index 711687cf5..2a917f16c 100644 --- a/tools/go_generics/generics_tests/simple/input.go +++ b/tools/go_generics/generics_tests/simple/input.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/generics_tests/simple/output/output.go b/tools/go_generics/generics_tests/simple/output/output.go index 139c9bf9d..6bfa0b25b 100644 --- a/tools/go_generics/generics_tests/simple/output/output.go +++ b/tools/go_generics/generics_tests/simple/output/output.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/globals/globals_visitor.go b/tools/go_generics/globals/globals_visitor.go index daaa17b1d..7ae48c662 100644 --- a/tools/go_generics/globals/globals_visitor.go +++ b/tools/go_generics/globals/globals_visitor.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/globals/scope.go b/tools/go_generics/globals/scope.go index b75a91689..96c965ea2 100644 --- a/tools/go_generics/globals/scope.go +++ b/tools/go_generics/globals/scope.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/go_generics_unittest.sh b/tools/go_generics/go_generics_unittest.sh index e7553a071..44b22db91 100755 --- a/tools/go_generics/go_generics_unittest.sh +++ b/tools/go_generics/go_generics_unittest.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# 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. diff --git a/tools/go_generics/go_merge/main.go b/tools/go_generics/go_merge/main.go index 2f83facf8..f6a331123 100644 --- a/tools/go_generics/go_merge/main.go +++ b/tools/go_generics/go_merge/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/imports.go b/tools/go_generics/imports.go index 57f7c3dce..3a7230c97 100644 --- a/tools/go_generics/imports.go +++ b/tools/go_generics/imports.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/remove.go b/tools/go_generics/remove.go index 139d03955..568a6bbd3 100644 --- a/tools/go_generics/remove.go +++ b/tools/go_generics/remove.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/rules_tests/template.go b/tools/go_generics/rules_tests/template.go index f3f31ae8e..aace61da1 100644 --- a/tools/go_generics/rules_tests/template.go +++ b/tools/go_generics/rules_tests/template.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_generics/rules_tests/template_test.go b/tools/go_generics/rules_tests/template_test.go index 3a38c8629..b2a3446ef 100644 --- a/tools/go_generics/rules_tests/template_test.go +++ b/tools/go_generics/rules_tests/template_test.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/go_stateify/main.go b/tools/go_stateify/main.go index 9e2c8e106..db7a7107b 100644 --- a/tools/go_stateify/main.go +++ b/tools/go_stateify/main.go @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/tools/tag_release.sh b/tools/tag_release.sh index 6906a952f..02a49cdf1 100755 --- a/tools/tag_release.sh +++ b/tools/tag_release.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2019 Google LLC +# Copyright 2019 The gVisor Authors. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/tools/workspace_status.sh b/tools/workspace_status.sh index a0e646e45..64a905fc9 100755 --- a/tools/workspace_status.sh +++ b/tools/workspace_status.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright 2018 Google LLC +# 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. diff --git a/vdso/barrier.h b/vdso/barrier.h index 5b6c763f6..edba4afb5 100644 --- a/vdso/barrier.h +++ b/vdso/barrier.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/vdso/check_vdso.py b/vdso/check_vdso.py index 6f7d7e7ec..e41b09709 100644 --- a/vdso/check_vdso.py +++ b/vdso/check_vdso.py @@ -1,4 +1,4 @@ -# Copyright 2018 Google LLC +# 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. diff --git a/vdso/compiler.h b/vdso/compiler.h index d65f148fb..54a510000 100644 --- a/vdso/compiler.h +++ b/vdso/compiler.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/vdso/cycle_clock.h b/vdso/cycle_clock.h index 309e07a3f..5d3fbb257 100644 --- a/vdso/cycle_clock.h +++ b/vdso/cycle_clock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/vdso/seqlock.h b/vdso/seqlock.h index ab2f3fda3..7a173174b 100644 --- a/vdso/seqlock.h +++ b/vdso/seqlock.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/vdso/syscalls.h b/vdso/syscalls.h index 90fb424ce..f5865bb72 100644 --- a/vdso/syscalls.h +++ b/vdso/syscalls.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/vdso/vdso.cc b/vdso/vdso.cc index 550729035..6265ad217 100644 --- a/vdso/vdso.cc +++ b/vdso/vdso.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/vdso/vdso_time.cc b/vdso/vdso_time.cc index 9fc262f60..1bb4bb86b 100644 --- a/vdso/vdso_time.cc +++ b/vdso/vdso_time.cc @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. diff --git a/vdso/vdso_time.h b/vdso/vdso_time.h index 464dadff2..70d079efc 100644 --- a/vdso/vdso_time.h +++ b/vdso/vdso_time.h @@ -1,4 +1,4 @@ -// Copyright 2018 Google LLC +// 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. -- cgit v1.2.3 From 9cdae51feca5cee9faa198161b92a0aeece52d6c Mon Sep 17 00:00:00 2001 From: Adin Scannell Date: Tue, 21 May 2019 15:17:05 -0700 Subject: Add basic plumbing for splice and stub implementation. This does not actually implement an efficient splice or sendfile. Rather, it adds a generic plumbing to the file internals so that this can be added. All file implementations use the stub fileutil.NoSplice implementation, which causes sendfile and splice to fall back to an internal copy. A basic splice system call interface is added, along with a test. PiperOrigin-RevId: 249335960 Change-Id: Ic5568be2af0a505c19e7aec66d5af2480ab0939b --- pkg/abi/linux/BUILD | 1 + pkg/abi/linux/splice.go | 23 ++ pkg/sentry/fs/BUILD | 4 +- pkg/sentry/fs/ashmem/area.go | 3 +- pkg/sentry/fs/binder/binder.go | 3 +- pkg/sentry/fs/dev/full.go | 3 +- pkg/sentry/fs/dev/null.go | 4 +- pkg/sentry/fs/dev/random.go | 1 + pkg/sentry/fs/fdpipe/pipe.go | 1 + pkg/sentry/fs/file.go | 139 +++++++---- pkg/sentry/fs/file_operations.go | 47 ++++ pkg/sentry/fs/file_overlay.go | 79 ++++--- pkg/sentry/fs/file_test.go | 24 -- pkg/sentry/fs/filetest/filetest.go | 1 + pkg/sentry/fs/fsutil/file.go | 16 ++ pkg/sentry/fs/fsutil/inode.go | 3 +- pkg/sentry/fs/gofer/file.go | 5 +- pkg/sentry/fs/host/file.go | 1 + pkg/sentry/fs/inotify.go | 10 + pkg/sentry/fs/proc/exec_args.go | 3 +- pkg/sentry/fs/proc/rpcinet_proc.go | 3 +- pkg/sentry/fs/proc/seqfile/seqfile.go | 1 + pkg/sentry/fs/proc/sys.go | 3 +- pkg/sentry/fs/proc/sys_net.go | 6 +- pkg/sentry/fs/proc/task.go | 10 +- pkg/sentry/fs/proc/uid_gid_map.go | 1 + pkg/sentry/fs/proc/uptime.go | 5 +- pkg/sentry/fs/ramfs/socket.go | 7 +- pkg/sentry/fs/ramfs/symlink.go | 7 +- pkg/sentry/fs/splice.go | 187 +++++++++++++++ pkg/sentry/fs/timerfd/timerfd.go | 5 +- pkg/sentry/fs/tmpfs/file_regular.go | 3 +- pkg/sentry/fs/tty/dir.go | 3 +- pkg/sentry/fs/tty/master.go | 3 +- pkg/sentry/fs/tty/slave.go | 3 +- pkg/sentry/kernel/epoll/epoll.go | 3 +- pkg/sentry/kernel/eventfd/eventfd.go | 5 +- pkg/sentry/kernel/pipe/reader_writer.go | 3 +- pkg/sentry/loader/vdso.go | 3 +- pkg/sentry/socket/epsocket/epsocket.go | 3 +- pkg/sentry/socket/hostinet/socket.go | 3 +- pkg/sentry/socket/netlink/socket.go | 3 +- pkg/sentry/socket/rpcinet/socket.go | 3 +- pkg/sentry/socket/unix/unix.go | 3 +- pkg/sentry/syscalls/linux/BUILD | 1 + pkg/sentry/syscalls/linux/linux64.go | 2 +- pkg/sentry/syscalls/linux/sys_file.go | 98 -------- pkg/sentry/syscalls/linux/sys_splice.go | 293 +++++++++++++++++++++++ test/syscalls/BUILD | 2 + test/syscalls/linux/BUILD | 16 ++ test/syscalls/linux/splice.cc | 404 ++++++++++++++++++++++++++++++++ 51 files changed, 1221 insertions(+), 242 deletions(-) create mode 100644 pkg/abi/linux/splice.go delete mode 100644 pkg/sentry/fs/file_test.go create mode 100644 pkg/sentry/fs/splice.go create mode 100644 pkg/sentry/syscalls/linux/sys_splice.go create mode 100644 test/syscalls/linux/splice.cc (limited to 'pkg/sentry/socket/rpcinet') diff --git a/pkg/abi/linux/BUILD b/pkg/abi/linux/BUILD index fdf193873..96e8d4641 100644 --- a/pkg/abi/linux/BUILD +++ b/pkg/abi/linux/BUILD @@ -45,6 +45,7 @@ go_library( "shm.go", "signal.go", "socket.go", + "splice.go", "tcp.go", "time.go", "timer.go", diff --git a/pkg/abi/linux/splice.go b/pkg/abi/linux/splice.go new file mode 100644 index 000000000..650eb87e8 --- /dev/null +++ b/pkg/abi/linux/splice.go @@ -0,0 +1,23 @@ +// Copyright 2019 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package linux + +// Constants for splice(2), sendfile(2) and tee(2). +const ( + SPLICE_F_MOVE = 1 << iota + SPLICE_F_NONBLOCK + SPLICE_F_MORE + SPLICE_F_GIFT +) diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD index 1fd9e30f6..142a00840 100644 --- a/pkg/sentry/fs/BUILD +++ b/pkg/sentry/fs/BUILD @@ -40,6 +40,7 @@ go_library( "restore.go", "save.go", "seek.go", + "splice.go", "sync.go", ], importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs", @@ -51,6 +52,7 @@ go_library( "//pkg/metric", "//pkg/p9", "//pkg/refs", + "//pkg/secio", "//pkg/sentry/arch", "//pkg/sentry/context", "//pkg/sentry/device", @@ -66,7 +68,6 @@ go_library( "//pkg/sentry/usermem", "//pkg/state", "//pkg/syserror", - "//pkg/tcpip", "//pkg/waiter", ], ) @@ -122,7 +123,6 @@ go_test( srcs = [ "dirent_cache_test.go", "dirent_refs_test.go", - "file_test.go", "mount_test.go", "path_test.go", ], diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go index b53746519..b4b0cc08b 100644 --- a/pkg/sentry/fs/ashmem/area.go +++ b/pkg/sentry/fs/ashmem/area.go @@ -42,11 +42,12 @@ const ( // // +stateify savable type Area struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` ad *Device diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go index a992253e6..c78f1fc40 100644 --- a/pkg/sentry/fs/binder/binder.go +++ b/pkg/sentry/fs/binder/binder.go @@ -86,10 +86,11 @@ func (bd *Device) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) // // +stateify savable type Proc struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` bd *Device task *kernel.Task diff --git a/pkg/sentry/fs/dev/full.go b/pkg/sentry/fs/dev/full.go index 17d68b5c4..8f6c6da2d 100644 --- a/pkg/sentry/fs/dev/full.go +++ b/pkg/sentry/fs/dev/full.go @@ -60,6 +60,7 @@ func (f *fullDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.Fi // +stateify savable type fullFileOperations struct { + waiter.AlwaysReady `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` @@ -68,8 +69,8 @@ type fullFileOperations struct { fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` readZeros `state:"nosave"` - waiter.AlwaysReady `state:"nosave"` } var _ fs.FileOperations = (*fullFileOperations)(nil) diff --git a/pkg/sentry/fs/dev/null.go b/pkg/sentry/fs/dev/null.go index ee13183c8..3f1accef8 100644 --- a/pkg/sentry/fs/dev/null.go +++ b/pkg/sentry/fs/dev/null.go @@ -64,6 +64,7 @@ type nullFileOperations struct { fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRead `state:"nosave"` @@ -104,14 +105,15 @@ func (zd *zeroDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.F type zeroFileOperations struct { fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNoopWrite `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` - readZeros `state:"nosave"` waiter.AlwaysReady `state:"nosave"` + readZeros `state:"nosave"` } var _ fs.FileOperations = (*zeroFileOperations)(nil) diff --git a/pkg/sentry/fs/dev/random.go b/pkg/sentry/fs/dev/random.go index b0a412382..e5a01a906 100644 --- a/pkg/sentry/fs/dev/random.go +++ b/pkg/sentry/fs/dev/random.go @@ -61,6 +61,7 @@ type randomFileOperations struct { fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go index 95e66ea8d..4ef7ea08a 100644 --- a/pkg/sentry/fs/fdpipe/pipe.go +++ b/pkg/sentry/fs/fdpipe/pipe.go @@ -43,6 +43,7 @@ type pipeOperations struct { fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` waiter.Queue `state:"nosave"` diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go index 62b35dabc..8f1baca23 100644 --- a/pkg/sentry/fs/file.go +++ b/pkg/sentry/fs/file.go @@ -21,7 +21,6 @@ import ( "time" "gvisor.googlesource.com/gvisor/pkg/amutex" - "gvisor.googlesource.com/gvisor/pkg/log" "gvisor.googlesource.com/gvisor/pkg/metric" "gvisor.googlesource.com/gvisor/pkg/refs" "gvisor.googlesource.com/gvisor/pkg/sentry/context" @@ -35,8 +34,13 @@ import ( ) var ( - // RecordWaitTime controls writing metrics for filesystem reads. Enabling this comes at a small - // CPU cost due to performing two monotonic clock reads per read call. + // RecordWaitTime controls writing metrics for filesystem reads. + // Enabling this comes at a small CPU cost due to performing two + // monotonic clock reads per read call. + // + // Note that this is only performed in the direct read path, and may + // not be consistently applied for other forms of reads, such as + // splice. RecordWaitTime = false reads = metric.MustCreateNewUint64Metric("/fs/reads", false /* sync */, "Number of file reads.") @@ -306,14 +310,28 @@ func (f *File) Writev(ctx context.Context, src usermem.IOSequence) (int64, error return 0, syserror.ErrInterrupted } - offset, err := f.checkWriteLocked(ctx, &src, f.offset) - if err != nil { + // Handle append mode. + if f.Flags().Append { + if err := f.offsetForAppend(ctx, &f.offset); err != nil { + f.mu.Unlock() + return 0, err + } + } + + // Enforce file limits. + limit, ok := f.checkLimit(ctx, f.offset) + switch { + case ok && limit == 0: f.mu.Unlock() - return 0, err + return 0, syserror.ErrExceedsFileSizeLimit + case ok: + src = src.TakeFirst64(limit) } - n, err := f.FileOperations.Write(ctx, f, src, offset) + + // We must hold the lock during the write. + n, err := f.FileOperations.Write(ctx, f, src, f.offset) if n >= 0 { - atomic.StoreInt64(&f.offset, offset+n) + atomic.StoreInt64(&f.offset, f.offset+n) } f.mu.Unlock() return n, err @@ -325,51 +343,67 @@ func (f *File) Writev(ctx context.Context, src usermem.IOSequence) (int64, error // // Otherwise same as Writev. func (f *File) Pwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) { - if !f.mu.Lock(ctx) { - return 0, syserror.ErrInterrupted + // "POSIX requires that opening a file with the O_APPEND flag should + // have no effect on the location at which pwrite() writes data. + // However, on Linux, if a file is opened with O_APPEND, pwrite() + // appends data to the end of the file, regardless of the value of + // offset." + if f.Flags().Append { + if !f.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer f.mu.Unlock() + if err := f.offsetForAppend(ctx, &offset); err != nil { + f.mu.Unlock() + return 0, err + } } - offset, err := f.checkWriteLocked(ctx, &src, offset) - if err != nil { - f.mu.Unlock() - return 0, err + // Enforce file limits. + limit, ok := f.checkLimit(ctx, offset) + switch { + case ok && limit == 0: + return 0, syserror.ErrExceedsFileSizeLimit + case ok: + src = src.TakeFirst64(limit) } - n, err := f.FileOperations.Write(ctx, f, src, offset) - f.mu.Unlock() - return n, err + + return f.FileOperations.Write(ctx, f, src, offset) } -// checkWriteLocked returns the offset to write at or an error if the write -// would not succeed. May update src to fit a write operation into a file -// size limit. -func (f *File) checkWriteLocked(ctx context.Context, src *usermem.IOSequence, offset int64) (int64, error) { - // Handle append only files. Note that this is still racy for network - // filesystems. - if f.Flags().Append { - uattr, err := f.Dirent.Inode.UnstableAttr(ctx) - if err != nil { - // This is an odd error, most likely it is evidence - // that something is terribly wrong with the filesystem. - // Return a generic EIO error. - log.Warningf("Failed to check write of inode %#v: %v", f.Dirent.Inode.StableAttr, err) - return offset, syserror.EIO - } - offset = uattr.Size +// offsetForAppend sets the given offset to the end of the file. +// +// Precondition: the underlying file mutex should be held. +func (f *File) offsetForAppend(ctx context.Context, offset *int64) error { + uattr, err := f.Dirent.Inode.UnstableAttr(ctx) + if err != nil { + // This is an odd error, we treat it as evidence that + // something is terribly wrong with the filesystem. + return syserror.EIO } - // Is this a regular file? + // Update the offset. + *offset = uattr.Size + + return nil +} + +// checkLimit checks the offset that the write will be performed at. The +// returned boolean indicates that the write must be limited. The returned +// integer indicates the new maximum write length. +func (f *File) checkLimit(ctx context.Context, offset int64) (int64, bool) { if IsRegular(f.Dirent.Inode.StableAttr) { // Enforce size limits. fileSizeLimit := limits.FromContext(ctx).Get(limits.FileSize).Cur if fileSizeLimit <= math.MaxInt64 { if offset >= int64(fileSizeLimit) { - return offset, syserror.ErrExceedsFileSizeLimit + return 0, true } - *src = src.TakeFirst64(int64(fileSizeLimit) - offset) + return int64(fileSizeLimit) - offset, true } } - return offset, nil + return 0, false } // Fsync calls f.FileOperations.Fsync with f as the File. @@ -466,8 +500,13 @@ func (f *File) Async(newAsync func() FileAsync) FileAsync { return f.async } -// FileReader implements io.Reader and io.ReaderAt. -type FileReader struct { +// lockedReader implements io.Reader and io.ReaderAt. +// +// Note this reads the underlying file using the file operations directly. It +// is the responsibility of the caller to ensure that locks are appropriately +// held and offsets updated if required. This should be used only by internal +// functions that perform these operations and checks at other times. +type lockedReader struct { // Ctx is the context for the file reader. Ctx context.Context @@ -476,19 +515,21 @@ type FileReader struct { } // Read implements io.Reader.Read. -func (r *FileReader) Read(buf []byte) (int, error) { - n, err := r.File.Readv(r.Ctx, usermem.BytesIOSequence(buf)) +func (r *lockedReader) Read(buf []byte) (int, error) { + n, err := r.File.FileOperations.Read(r.Ctx, r.File, usermem.BytesIOSequence(buf), r.File.offset) return int(n), err } // ReadAt implements io.Reader.ReadAt. -func (r *FileReader) ReadAt(buf []byte, offset int64) (int, error) { - n, err := r.File.Preadv(r.Ctx, usermem.BytesIOSequence(buf), offset) +func (r *lockedReader) ReadAt(buf []byte, offset int64) (int, error) { + n, err := r.File.FileOperations.Read(r.Ctx, r.File, usermem.BytesIOSequence(buf), offset) return int(n), err } -// FileWriter implements io.Writer and io.WriterAt. -type FileWriter struct { +// lockedWriter implements io.Writer and io.WriterAt. +// +// The same constraints as lockedReader apply; see above. +type lockedWriter struct { // Ctx is the context for the file writer. Ctx context.Context @@ -497,13 +538,13 @@ type FileWriter struct { } // Write implements io.Writer.Write. -func (w *FileWriter) Write(buf []byte) (int, error) { - n, err := w.File.Writev(w.Ctx, usermem.BytesIOSequence(buf)) +func (w *lockedWriter) Write(buf []byte) (int, error) { + n, err := w.File.FileOperations.Write(w.Ctx, w.File, usermem.BytesIOSequence(buf), w.File.offset) return int(n), err } // WriteAt implements io.Writer.WriteAt. -func (w *FileWriter) WriteAt(buf []byte, offset int64) (int, error) { - n, err := w.File.Pwritev(w.Ctx, usermem.BytesIOSequence(buf), offset) +func (w *lockedWriter) WriteAt(buf []byte, offset int64) (int, error) { + n, err := w.File.FileOperations.Write(w.Ctx, w.File, usermem.BytesIOSequence(buf), offset) return int(n), err } diff --git a/pkg/sentry/fs/file_operations.go b/pkg/sentry/fs/file_operations.go index ab0acb6eb..0f2dfa273 100644 --- a/pkg/sentry/fs/file_operations.go +++ b/pkg/sentry/fs/file_operations.go @@ -22,6 +22,38 @@ import ( "gvisor.googlesource.com/gvisor/pkg/waiter" ) +// SpliceOpts define how a splice works. +type SpliceOpts struct { + // Length is the length of the splice operation. + Length int64 + + // SrcOffset indicates whether the existing source file offset should + // be used. If this is true, then the Start value below is used. + // + // When passed to FileOperations object, this should always be true as + // the offset will be provided by a layer above, unless the object in + // question is a pipe or socket. This value can be relied upon for such + // an indicator. + SrcOffset bool + + // SrcStart is the start of the source file. This is used only if + // SrcOffset is false. + SrcStart int64 + + // Dup indicates that the contents should not be consumed from the + // source (e.g. in the case of a socket or a pipe), but duplicated. + Dup bool + + // DstOffset indicates that the destination file offset should be used. + // + // See SrcOffset for additional information. + DstOffset bool + + // DstStart is the start of the destination file. This is used only if + // DstOffset is false. + DstStart int64 +} + // FileOperations are operations on a File that diverge per file system. // // Operations that take a *File may use only the following interfaces: @@ -67,6 +99,15 @@ type FileOperations interface { // Read must not be called if !FileFlags.Read. Read(ctx context.Context, file *File, dst usermem.IOSequence, offset int64) (int64, error) + // WriteTo is a variant of read that takes another file as a + // destination. For a splice (copy or move from one file to another), + // first a WriteTo on the source is attempted, followed by a ReadFrom + // on the destination, following by a buffered copy with standard Read + // and Write operations. + // + // The same preconditions as Read apply. + WriteTo(ctx context.Context, file *File, dst *File, opts SpliceOpts) (int64, error) + // Write writes src to file at offset and returns the number of bytes // written which must be greater than or equal to 0. Like Read, file // systems that do not support writing at an offset (i.e. pipefs, sockfs) @@ -81,6 +122,12 @@ type FileOperations interface { // Write must not be called if !FileFlags.Write. Write(ctx context.Context, file *File, src usermem.IOSequence, offset int64) (int64, error) + // ReadFrom is a variant of write that takes a another file as a + // source. See WriteTo for details regarding how this is called. + // + // The same preconditions as Write apply; FileFlags.Write must be set. + ReadFrom(ctx context.Context, file *File, src *File, opts SpliceOpts) (int64, error) + // Fsync writes buffered modifications of file and/or flushes in-flight // operations to backing storage based on syncType. The range to sync is // [start, end]. The end is inclusive so that the last byte of a maximally diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go index 948ce9c6f..273de1e14 100644 --- a/pkg/sentry/fs/file_overlay.go +++ b/pkg/sentry/fs/file_overlay.go @@ -17,7 +17,6 @@ package fs import ( "sync" - "gvisor.googlesource.com/gvisor/pkg/log" "gvisor.googlesource.com/gvisor/pkg/refs" "gvisor.googlesource.com/gvisor/pkg/sentry/arch" "gvisor.googlesource.com/gvisor/pkg/sentry/context" @@ -222,31 +221,50 @@ func (f *overlayFileOperations) IterateDir(ctx context.Context, dirCtx *DirCtx, return offset + n, err } -// Read implements FileOperations.Read. -func (f *overlayFileOperations) Read(ctx context.Context, file *File, dst usermem.IOSequence, offset int64) (int64, error) { - o := file.Dirent.Inode.overlay +// onTop performs the given operation on the top-most available layer. +func (f *overlayFileOperations) onTop(ctx context.Context, file *File, fn func(*File, FileOperations) error) error { + file.Dirent.Inode.overlay.copyMu.RLock() + defer file.Dirent.Inode.overlay.copyMu.RUnlock() - o.copyMu.RLock() - defer o.copyMu.RUnlock() + // Only lower layer is available. + if file.Dirent.Inode.overlay.upper == nil { + return fn(f.lower, f.lower.FileOperations) + } - if o.upper != nil { - // We may need to acquire an open file handle to read from if - // copy up has occurred. Otherwise we risk reading from the - // wrong source. - f.upperMu.Lock() - if f.upper == nil { - var err error - f.upper, err = overlayFile(ctx, o.upper, file.Flags()) - if err != nil { - f.upperMu.Unlock() - log.Warningf("failed to acquire handle with flags %v: %v", file.Flags(), err) - return 0, syserror.EIO - } + f.upperMu.Lock() + if f.upper == nil { + upper, err := overlayFile(ctx, file.Dirent.Inode.overlay.upper, file.Flags()) + if err != nil { + // Something very wrong; return a generic filesystem + // error to avoid propagating internals. + f.upperMu.Unlock() + return syserror.EIO } - f.upperMu.Unlock() - return f.upper.FileOperations.Read(ctx, f.upper, dst, offset) + + // Save upper file. + f.upper = upper } - return f.lower.FileOperations.Read(ctx, f.lower, dst, offset) + f.upperMu.Unlock() + + return fn(f.upper, f.upper.FileOperations) +} + +// Read implements FileOperations.Read. +func (f *overlayFileOperations) Read(ctx context.Context, file *File, dst usermem.IOSequence, offset int64) (n int64, err error) { + err = f.onTop(ctx, file, func(file *File, ops FileOperations) error { + n, err = ops.Read(ctx, file, dst, offset) + return err // Will overwrite itself. + }) + return +} + +// WriteTo implements FileOperations.WriteTo. +func (f *overlayFileOperations) WriteTo(ctx context.Context, file *File, dst *File, opts SpliceOpts) (n int64, err error) { + err = f.onTop(ctx, file, func(file *File, ops FileOperations) error { + n, err = ops.WriteTo(ctx, file, dst, opts) + return err // Will overwrite itself. + }) + return } // Write implements FileOperations.Write. @@ -257,15 +275,20 @@ func (f *overlayFileOperations) Write(ctx context.Context, file *File, src userm return f.upper.FileOperations.Write(ctx, f.upper, src, offset) } +// ReadFrom implements FileOperations.ReadFrom. +func (f *overlayFileOperations) ReadFrom(ctx context.Context, file *File, src *File, opts SpliceOpts) (n int64, err error) { + // See above; f.upper must be non-nil. + return f.upper.FileOperations.ReadFrom(ctx, f.upper, src, opts) +} + // Fsync implements FileOperations.Fsync. -func (f *overlayFileOperations) Fsync(ctx context.Context, file *File, start, end int64, syncType SyncType) error { - var err error +func (f *overlayFileOperations) Fsync(ctx context.Context, file *File, start, end int64, syncType SyncType) (err error) { f.upperMu.Lock() if f.upper != nil { err = f.upper.FileOperations.Fsync(ctx, f.upper, start, end, syncType) } f.upperMu.Unlock() - if f.lower != nil { + if err == nil && f.lower != nil { // N.B. Fsync on the lower filesystem can cause writes of file // attributes (i.e. access time) despite the fact that we must // treat the lower filesystem as read-only. @@ -277,15 +300,14 @@ func (f *overlayFileOperations) Fsync(ctx context.Context, file *File, start, en } // Flush implements FileOperations.Flush. -func (f *overlayFileOperations) Flush(ctx context.Context, file *File) error { +func (f *overlayFileOperations) Flush(ctx context.Context, file *File) (err error) { // Flush whatever handles we have. - var err error f.upperMu.Lock() if f.upper != nil { err = f.upper.FileOperations.Flush(ctx, f.upper) } f.upperMu.Unlock() - if f.lower != nil { + if err == nil && f.lower != nil { err = f.lower.FileOperations.Flush(ctx, f.lower) } return err @@ -329,6 +351,7 @@ func (*overlayFileOperations) ConfigureMMap(ctx context.Context, file *File, opt if !o.isMappableLocked() { return syserror.ENODEV } + // FIXME(jamieliu): This is a copy/paste of fsutil.GenericConfigureMMap, // which we can't use because the overlay implementation is in package fs, // so depending on fs/fsutil would create a circular dependency. Move diff --git a/pkg/sentry/fs/file_test.go b/pkg/sentry/fs/file_test.go deleted file mode 100644 index d867a0257..000000000 --- a/pkg/sentry/fs/file_test.go +++ /dev/null @@ -1,24 +0,0 @@ -// 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. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package fs - -import "io" - -var ( - _ = io.Reader(&FileReader{}) - _ = io.ReaderAt(&FileReader{}) - _ = io.Writer(&FileWriter{}) - _ = io.WriterAt(&FileWriter{}) -) diff --git a/pkg/sentry/fs/filetest/filetest.go b/pkg/sentry/fs/filetest/filetest.go index f6b827800..c0b1b088d 100644 --- a/pkg/sentry/fs/filetest/filetest.go +++ b/pkg/sentry/fs/filetest/filetest.go @@ -38,6 +38,7 @@ type TestFileOperations struct { fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` waiter.AlwaysReady `state:"nosave"` } diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go index e355d8594..9381963d0 100644 --- a/pkg/sentry/fs/fsutil/file.go +++ b/pkg/sentry/fs/fsutil/file.go @@ -223,6 +223,20 @@ func (FileNoIoctl) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallAr return 0, syserror.ENOTTY } +// FileNoSplice implements fs.FileOperations.ReadFrom and +// fs.FileOperations.WriteTo for files that don't support splice. +type FileNoSplice struct{} + +// WriteTo implements fs.FileOperations.WriteTo. +func (FileNoSplice) WriteTo(context.Context, *fs.File, *fs.File, fs.SpliceOpts) (int64, error) { + return 0, syserror.ENOSYS +} + +// ReadFrom implements fs.FileOperations.ReadFrom. +func (FileNoSplice) ReadFrom(context.Context, *fs.File, *fs.File, fs.SpliceOpts) (int64, error) { + return 0, syserror.ENOSYS +} + // DirFileOperations implements most of fs.FileOperations for directories, // except for Readdir and UnstableAttr which the embedding type must implement. type DirFileOperations struct { @@ -233,6 +247,7 @@ type DirFileOperations struct { FileNoopFlush FileNoopFsync FileNoopRelease + FileNoSplice } // Read implements fs.FileOperations.Read @@ -303,6 +318,7 @@ type NoReadWriteFile struct { FileNoWrite `state:"nosave"` FileNotDirReaddir `state:"nosave"` FileUseInodeUnstableAttr `state:"nosave"` + FileNoSplice `state:"nosave"` } var _ fs.FileOperations = (*NoReadWriteFile)(nil) diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go index a22b6ce9c..925887335 100644 --- a/pkg/sentry/fs/fsutil/inode.go +++ b/pkg/sentry/fs/fsutil/inode.go @@ -250,16 +250,17 @@ func (i *InodeSimpleExtendedAttributes) Listxattr(_ *fs.Inode) (map[string]struc // // +stateify savable type staticFile struct { - waiter.AlwaysReady `state:"nosave"` FileGenericSeek `state:"nosave"` FileNoIoctl `state:"nosave"` FileNoMMap `state:"nosave"` + FileNoSplice `state:"nosave"` FileNoopFsync `state:"nosave"` FileNoopFlush `state:"nosave"` FileNoopRelease `state:"nosave"` FileNoopWrite `state:"nosave"` FileNotDirReaddir `state:"nosave"` FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` FileStaticContentReader } diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go index bc2be546e..fb4f50113 100644 --- a/pkg/sentry/fs/gofer/file.go +++ b/pkg/sentry/fs/gofer/file.go @@ -46,8 +46,9 @@ var ( // // +stateify savable type fileOperations struct { - fsutil.FileNoIoctl `state:"nosave"` - waiter.AlwaysReady `state:"nosave"` + fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoSplice `state:"nosplice"` + waiter.AlwaysReady `state:"nosave"` // inodeOperations is the inodeOperations backing the file. It is protected // by a reference held by File.Dirent.Inode which is stable until diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go index 82e2ae3b9..ad0a3ec85 100644 --- a/pkg/sentry/fs/host/file.go +++ b/pkg/sentry/fs/host/file.go @@ -37,6 +37,7 @@ import ( // +stateify savable type fileOperations struct { fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoSplice `state:"nosplice"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` diff --git a/pkg/sentry/fs/inotify.go b/pkg/sentry/fs/inotify.go index 2652582c3..7dfd31020 100644 --- a/pkg/sentry/fs/inotify.go +++ b/pkg/sentry/fs/inotify.go @@ -171,11 +171,21 @@ func (i *Inotify) Read(ctx context.Context, _ *File, dst usermem.IOSequence, _ i return writeLen, nil } +// WriteTo implements FileOperations.WriteTo. +func (*Inotify) WriteTo(context.Context, *File, *File, SpliceOpts) (int64, error) { + return 0, syserror.ENOSYS +} + // Fsync implements FileOperations.Fsync. func (*Inotify) Fsync(context.Context, *File, int64, int64, SyncType) error { return syserror.EINVAL } +// ReadFrom implements FileOperations.ReadFrom. +func (*Inotify) ReadFrom(context.Context, *File, *File, SpliceOpts) (int64, error) { + return 0, syserror.ENOSYS +} + // Flush implements FileOperations.Flush. func (*Inotify) Flush(context.Context, *File) error { return nil diff --git a/pkg/sentry/fs/proc/exec_args.go b/pkg/sentry/fs/proc/exec_args.go index d49dad685..cb28f6bc3 100644 --- a/pkg/sentry/fs/proc/exec_args.go +++ b/pkg/sentry/fs/proc/exec_args.go @@ -77,16 +77,17 @@ func (i *execArgInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs. // +stateify savable type execArgFile struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopWrite `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` // arg is the type of exec argument this file contains. arg execArgType diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go index db53686f6..e36c0bfa6 100644 --- a/pkg/sentry/fs/proc/rpcinet_proc.go +++ b/pkg/sentry/fs/proc/rpcinet_proc.go @@ -60,15 +60,16 @@ func (i *rpcInetInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs. // rpcInetFile implements fs.FileOperations as RPCs. type rpcInetFile struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` inode *rpcInetInode } diff --git a/pkg/sentry/fs/proc/seqfile/seqfile.go b/pkg/sentry/fs/proc/seqfile/seqfile.go index 6b0ae9e60..8364d86ed 100644 --- a/pkg/sentry/fs/proc/seqfile/seqfile.go +++ b/pkg/sentry/fs/proc/seqfile/seqfile.go @@ -187,6 +187,7 @@ type seqFileOperations struct { fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go index b889ed625..59846af4f 100644 --- a/pkg/sentry/fs/proc/sys.go +++ b/pkg/sentry/fs/proc/sys.go @@ -134,7 +134,6 @@ var _ fs.InodeOperations = (*hostname)(nil) // +stateify savable type hostnameFile struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` fsutil.FileNoSeek `state:"nosave"` @@ -143,7 +142,9 @@ type hostnameFile struct { fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoWrite `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` } // Read implements fs.FileOperations.Read. diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go index e49794a48..dbf1a987c 100644 --- a/pkg/sentry/fs/proc/sys_net.go +++ b/pkg/sentry/fs/proc/sys_net.go @@ -85,15 +85,16 @@ func (m *tcpMemInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.F // +stateify savable type tcpMemFile struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` tcpMemInode *tcpMemInode } @@ -198,15 +199,16 @@ func (s *tcpSack) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileF // +stateify savable type tcpSackFile struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` tcpSack *tcpSack diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go index 66d76d194..494b195cd 100644 --- a/pkg/sentry/fs/proc/task.go +++ b/pkg/sentry/fs/proc/task.go @@ -672,16 +672,17 @@ func (c *comm) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlag // +stateify savable type commFile struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` t *kernel.Task } @@ -728,16 +729,17 @@ func (a *auxvec) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl // +stateify savable type auxvecFile struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` t *kernel.Task } diff --git a/pkg/sentry/fs/proc/uid_gid_map.go b/pkg/sentry/fs/proc/uid_gid_map.go index 5df3cee13..a14b1b45f 100644 --- a/pkg/sentry/fs/proc/uid_gid_map.go +++ b/pkg/sentry/fs/proc/uid_gid_map.go @@ -85,6 +85,7 @@ type idMapFileOperations struct { fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` diff --git a/pkg/sentry/fs/proc/uptime.go b/pkg/sentry/fs/proc/uptime.go index 1ddf9fafa..35c3851e1 100644 --- a/pkg/sentry/fs/proc/uptime.go +++ b/pkg/sentry/fs/proc/uptime.go @@ -54,16 +54,17 @@ func (u *uptime) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl // +stateify savable type uptimeFile struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` startTime ktime.Time } diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go index a7cb1bb86..7d8bca70e 100644 --- a/pkg/sentry/fs/ramfs/socket.go +++ b/pkg/sentry/fs/ramfs/socket.go @@ -70,13 +70,14 @@ func (s *Socket) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl type socketFileOperations struct { fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoRead `state:"nosave"` + fsutil.FileNoSeek `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoRead `state:"nosave"` - fsutil.FileNoSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` waiter.AlwaysReady `state:"nosave"` } diff --git a/pkg/sentry/fs/ramfs/symlink.go b/pkg/sentry/fs/ramfs/symlink.go index dd2585b02..21c246169 100644 --- a/pkg/sentry/fs/ramfs/symlink.go +++ b/pkg/sentry/fs/ramfs/symlink.go @@ -91,13 +91,14 @@ func (s *Symlink) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileF type symlinkFileOperations struct { fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoRead `state:"nosave"` + fsutil.FileNoSeek `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoWrite `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` - fsutil.FileNoRead `state:"nosave"` - fsutil.FileNoSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoWrite `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` waiter.AlwaysReady `state:"nosave"` } diff --git a/pkg/sentry/fs/splice.go b/pkg/sentry/fs/splice.go new file mode 100644 index 000000000..65937f44d --- /dev/null +++ b/pkg/sentry/fs/splice.go @@ -0,0 +1,187 @@ +// Copyright 2019 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fs + +import ( + "io" + "sync/atomic" + + "gvisor.googlesource.com/gvisor/pkg/secio" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/syserror" +) + +// Splice moves data to this file, directly from another. +// +// Offsets are updated only if DstOffset and SrcOffset are set. +func Splice(ctx context.Context, dst *File, src *File, opts SpliceOpts) (int64, error) { + // Verify basic file flag permissions. + if !dst.Flags().Write || !src.Flags().Read { + return 0, syserror.EBADF + } + + // Check whether or not the objects being sliced are stream-oriented + // (i.e. pipes or sockets). If yes, we elide checks and offset locks. + srcPipe := IsPipe(src.Dirent.Inode.StableAttr) || IsSocket(src.Dirent.Inode.StableAttr) + dstPipe := IsPipe(dst.Dirent.Inode.StableAttr) || IsSocket(dst.Dirent.Inode.StableAttr) + + if !dstPipe && !opts.DstOffset && !srcPipe && !opts.SrcOffset { + switch { + case dst.UniqueID < src.UniqueID: + // Acquire dst first. + if !dst.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer dst.mu.Unlock() + if !src.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer src.mu.Unlock() + case dst.UniqueID > src.UniqueID: + // Acquire src first. + if !src.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer src.mu.Unlock() + if !dst.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer dst.mu.Unlock() + case dst.UniqueID == src.UniqueID: + // Acquire only one lock; it's the same file. This is a + // bit of a edge case, but presumably it's possible. + if !dst.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer dst.mu.Unlock() + } + // Use both offsets (locked). + opts.DstStart = dst.offset + opts.SrcStart = src.offset + } else if !dstPipe && !opts.DstOffset { + // Acquire only dst. + if !dst.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer dst.mu.Unlock() + opts.DstStart = dst.offset // Safe: locked. + } else if !srcPipe && !opts.SrcOffset { + // Acquire only src. + if !src.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer src.mu.Unlock() + opts.SrcStart = src.offset // Safe: locked. + } + + // Check append-only mode and the limit. + if !dstPipe { + if dst.Flags().Append { + if opts.DstOffset { + // We need to acquire the lock. + if !dst.mu.Lock(ctx) { + return 0, syserror.ErrInterrupted + } + defer dst.mu.Unlock() + } + // Figure out the appropriate offset to use. + if err := dst.offsetForAppend(ctx, &opts.DstStart); err != nil { + return 0, err + } + } + + // Enforce file limits. + limit, ok := dst.checkLimit(ctx, opts.DstStart) + switch { + case ok && limit == 0: + return 0, syserror.ErrExceedsFileSizeLimit + case ok && limit < opts.Length: + opts.Length = limit // Cap the write. + } + } + + // Attempt to do a WriteTo; this is likely the most efficient. + // + // The underlying implementation may be able to donate buffers. + newOpts := SpliceOpts{ + Length: opts.Length, + SrcStart: opts.SrcStart, + SrcOffset: !srcPipe, + Dup: opts.Dup, + DstStart: opts.DstStart, + DstOffset: !dstPipe, + } + n, err := src.FileOperations.WriteTo(ctx, src, dst, newOpts) + if n == 0 && err != nil { + // 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 donate + n, err = dst.FileOperations.ReadFrom(ctx, dst, src, newOpts) + } + if n == 0 && err != nil { + // If we've failed up to here, and at least one of the sources + // is a pipe or socket, then we can't properly support dup. + // Return an error indicating that this operation is not + // supported. + if (srcPipe || dstPipe) && newOpts.Dup { + return 0, syserror.EINVAL + } + + // We failed to splice the files. But that's fine; we just fall + // back to a slow path in this case. This copies without doing + // any mode changes, so should still be more efficient. + var ( + r io.Reader + w io.Writer + ) + fw := &lockedWriter{ + Ctx: ctx, + File: dst, + } + if newOpts.DstOffset { + // Use the provided offset. + w = secio.NewOffsetWriter(fw, newOpts.DstStart) + } else { + // Writes will proceed with no offset. + w = fw + } + fr := &lockedReader{ + Ctx: ctx, + File: src, + } + if newOpts.SrcOffset { + // Limit to the given offset and length. + r = io.NewSectionReader(fr, opts.SrcStart, opts.Length) + } else { + // Limit just to the given length. + r = &io.LimitedReader{fr, opts.Length} + } + + // Copy between the two. + n, err = io.Copy(w, r) + } + + // Update offsets, if required. + if n > 0 { + if !dstPipe && !opts.DstOffset { + atomic.StoreInt64(&dst.offset, dst.offset+n) + } + if !srcPipe && !opts.SrcOffset { + atomic.StoreInt64(&src.offset, src.offset+n) + } + } + + return n, err +} diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go index 749961f51..bce5f091d 100644 --- a/pkg/sentry/fs/timerfd/timerfd.go +++ b/pkg/sentry/fs/timerfd/timerfd.go @@ -36,9 +36,10 @@ type TimerOperations struct { fsutil.FileZeroSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` events waiter.Queue `state:"zerovalue"` diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go index 1ef256511..d1c163879 100644 --- a/pkg/sentry/fs/tmpfs/file_regular.go +++ b/pkg/sentry/fs/tmpfs/file_regular.go @@ -28,14 +28,15 @@ import ( // // +stateify savable type regularFileOperations struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoopFsync `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` // iops is the InodeOperations of a regular tmpfs file. It is // guaranteed to be the same as file.Dirent.Inode.InodeOperations, diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go index 8dc40e1f2..2603354c4 100644 --- a/pkg/sentry/fs/tty/dir.go +++ b/pkg/sentry/fs/tty/dir.go @@ -286,14 +286,15 @@ func (d *dirInodeOperations) masterClose(t *Terminal) { // // +stateify savable type dirFileOperations struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileGenericSeek `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` // di is the inode operations. di *dirInodeOperations diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go index 45e167e5f..afdf44cd1 100644 --- a/pkg/sentry/fs/tty/master.go +++ b/pkg/sentry/fs/tty/master.go @@ -98,8 +98,9 @@ type masterFileOperations struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` // d is the containing dir. diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go index 0ae57a02c..2abf32e57 100644 --- a/pkg/sentry/fs/tty/slave.go +++ b/pkg/sentry/fs/tty/slave.go @@ -87,8 +87,9 @@ type slaveFileOperations struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` // si is the inode operations. diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go index 2399ae6f2..bbacba1f4 100644 --- a/pkg/sentry/kernel/epoll/epoll.go +++ b/pkg/sentry/kernel/epoll/epoll.go @@ -102,8 +102,9 @@ type EventPoll struct { fsutil.FileNotDirReaddir `state:"zerovalue"` fsutil.FileNoFsync `state:"zerovalue"` fsutil.FileNoopFlush `state:"zerovalue"` - fsutil.FileNoMMap `state:"zerovalue"` fsutil.FileNoIoctl `state:"zerovalue"` + fsutil.FileNoMMap `state:"zerovalue"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` // Wait queue is used to notify interested parties when the event poll diff --git a/pkg/sentry/kernel/eventfd/eventfd.go b/pkg/sentry/kernel/eventfd/eventfd.go index 5d3139eef..2f900be38 100644 --- a/pkg/sentry/kernel/eventfd/eventfd.go +++ b/pkg/sentry/kernel/eventfd/eventfd.go @@ -42,9 +42,10 @@ type EventOperations struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` - fsutil.FileNoMMap `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` + fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` // Mutex that protects accesses to the fields of this event. diff --git a/pkg/sentry/kernel/pipe/reader_writer.go b/pkg/sentry/kernel/pipe/reader_writer.go index ddcc5e09a..59899be49 100644 --- a/pkg/sentry/kernel/pipe/reader_writer.go +++ b/pkg/sentry/kernel/pipe/reader_writer.go @@ -38,8 +38,9 @@ type ReaderWriter struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` *Pipe } diff --git a/pkg/sentry/loader/vdso.go b/pkg/sentry/loader/vdso.go index 207d8ed3d..4e73527cf 100644 --- a/pkg/sentry/loader/vdso.go +++ b/pkg/sentry/loader/vdso.go @@ -52,15 +52,16 @@ func (f *fileContext) Value(key interface{}) interface{} { // byteReader implements fs.FileOperations for reading from a []byte source. type byteReader struct { - waiter.AlwaysReady `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` fsutil.FileNoIoctl `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoopRelease `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FilePipeSeek `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` + waiter.AlwaysReady `state:"nosave"` data []byte } diff --git a/pkg/sentry/socket/epsocket/epsocket.go b/pkg/sentry/socket/epsocket/epsocket.go index 520d82f68..31a449cf2 100644 --- a/pkg/sentry/socket/epsocket/epsocket.go +++ b/pkg/sentry/socket/epsocket/epsocket.go @@ -212,9 +212,10 @@ type commonEndpoint interface { type SocketOperations struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` - fsutil.FileNoFsync `state:"nosave"` fsutil.FileNoopFlush `state:"nosave"` + fsutil.FileNoFsync `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` socket.SendReceiveTimeout *waiter.Queue diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go index 71884d3db..41f9693bb 100644 --- a/pkg/sentry/socket/hostinet/socket.go +++ b/pkg/sentry/socket/hostinet/socket.go @@ -49,8 +49,9 @@ type socketOperations struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` socket.SendReceiveTimeout diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go index dc688eb00..afd06ca33 100644 --- a/pkg/sentry/socket/netlink/socket.go +++ b/pkg/sentry/socket/netlink/socket.go @@ -68,8 +68,9 @@ type Socket struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` socket.SendReceiveTimeout diff --git a/pkg/sentry/socket/rpcinet/socket.go b/pkg/sentry/socket/rpcinet/socket.go index c028ed4dd..55e0b6665 100644 --- a/pkg/sentry/socket/rpcinet/socket.go +++ b/pkg/sentry/socket/rpcinet/socket.go @@ -48,8 +48,9 @@ type socketOperations struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` socket.SendReceiveTimeout diff --git a/pkg/sentry/socket/unix/unix.go b/pkg/sentry/socket/unix/unix.go index 26788ec31..931056d51 100644 --- a/pkg/sentry/socket/unix/unix.go +++ b/pkg/sentry/socket/unix/unix.go @@ -48,8 +48,9 @@ type SocketOperations struct { fsutil.FilePipeSeek `state:"nosave"` fsutil.FileNotDirReaddir `state:"nosave"` fsutil.FileNoFsync `state:"nosave"` - fsutil.FileNoopFlush `state:"nosave"` fsutil.FileNoMMap `state:"nosave"` + fsutil.FileNoSplice `state:"nosave"` + fsutil.FileNoopFlush `state:"nosave"` fsutil.FileUseInodeUnstableAttr `state:"nosave"` refs.AtomicRefCount socket.SendReceiveTimeout diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD index 6e2843b36..f76989ae2 100644 --- a/pkg/sentry/syscalls/linux/BUILD +++ b/pkg/sentry/syscalls/linux/BUILD @@ -34,6 +34,7 @@ go_library( "sys_shm.go", "sys_signal.go", "sys_socket.go", + "sys_splice.go", "sys_stat.go", "sys_sync.go", "sys_sysinfo.go", diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go index 9a460ebdf..3e4d312af 100644 --- a/pkg/sentry/syscalls/linux/linux64.go +++ b/pkg/sentry/syscalls/linux/linux64.go @@ -407,7 +407,7 @@ var AMD64 = &kernel.SyscallTable{ 273: syscalls.Error(syscall.ENOSYS), // @Syscall(GetRobustList, note:Obsolete) 274: syscalls.Error(syscall.ENOSYS), - // 275: @Syscall(Splice), TODO(b/29354098) + 275: Splice, // 276: @Syscall(Tee), TODO(b/29354098) 277: SyncFileRange, // 278: @Syscall(Vmsplice), TODO(b/29354098) diff --git a/pkg/sentry/syscalls/linux/sys_file.go b/pkg/sentry/syscalls/linux/sys_file.go index 1764bb4b6..8a80cd430 100644 --- a/pkg/sentry/syscalls/linux/sys_file.go +++ b/pkg/sentry/syscalls/linux/sys_file.go @@ -15,7 +15,6 @@ package linux import ( - "io" "syscall" "gvisor.googlesource.com/gvisor/pkg/abi/linux" @@ -2025,103 +2024,6 @@ func Flock(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall return 0, nil, nil } -// Sendfile implements linux system call sendfile(2). -func Sendfile(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { - outFD := kdefs.FD(args[0].Int()) - inFD := kdefs.FD(args[1].Int()) - offsetAddr := args[2].Pointer() - count := int64(args[3].SizeT()) - - // Don't send a negative number of bytes. - if count < 0 { - return 0, nil, syserror.EINVAL - } - - if count > int64(kernel.MAX_RW_COUNT) { - count = int64(kernel.MAX_RW_COUNT) - } - - // Get files. - outFile := t.FDMap().GetFile(outFD) - if outFile == nil { - return 0, nil, syserror.EBADF - } - defer outFile.DecRef() - - inFile := t.FDMap().GetFile(inFD) - if inFile == nil { - return 0, nil, syserror.EBADF - } - defer inFile.DecRef() - - // Verify that the outfile is writable. - outFlags := outFile.Flags() - if !outFlags.Write { - return 0, nil, syserror.EBADF - } - - // Verify that the outfile Append flag is not set. - if outFlags.Append { - return 0, nil, syserror.EINVAL - } - - // Verify that we have a regular infile. - // http://elixir.free-electrons.com/linux/latest/source/fs/splice.c#L933 - if !fs.IsRegular(inFile.Dirent.Inode.StableAttr) { - return 0, nil, syserror.EINVAL - } - - // Verify that the infile is readable. - if !inFile.Flags().Read { - return 0, nil, syserror.EBADF - } - - // Setup for sending data. - var n int64 - var err error - w := &fs.FileWriter{t, outFile} - hasOffset := offsetAddr != 0 - // If we have a provided offset. - if hasOffset { - // Verify that when offset address is not null, infile must be seekable - if !inFile.Flags().Pread { - return 0, nil, syserror.ESPIPE - } - // Copy in the offset. - var offset int64 - if _, err := t.CopyIn(offsetAddr, &offset); err != nil { - return 0, nil, err - } - if offset < 0 { - return 0, nil, syserror.EINVAL - } - // Send data using Preadv. - r := io.NewSectionReader(&fs.FileReader{t, inFile}, offset, count) - n, err = io.Copy(w, r) - // Copy out the new offset. - if _, err := t.CopyOut(offsetAddr, n+offset); err != nil { - return 0, nil, err - } - // If we don't have a provided offset. - } else { - // Send data using readv. - inOff := inFile.Offset() - r := &io.LimitedReader{R: &fs.FileReader{t, inFile}, N: count} - n, err = io.Copy(w, r) - inOff += n - if inFile.Offset() != inOff { - // Adjust file position in case more bytes were read than written. - if _, err := inFile.Seek(t, fs.SeekSet, inOff); err != nil { - return 0, nil, syserror.EIO - } - } - } - - // We can only pass a single file to handleIOError, so pick inFile - // arbitrarily. - return uintptr(n), nil, handleIOError(t, n != 0, err, kernel.ERESTARTSYS, "sendfile", inFile) -} - const ( memfdPrefix = "/memfd:" memfdAllFlags = uint32(linux.MFD_CLOEXEC | linux.MFD_ALLOW_SEALING) diff --git a/pkg/sentry/syscalls/linux/sys_splice.go b/pkg/sentry/syscalls/linux/sys_splice.go new file mode 100644 index 000000000..37303606f --- /dev/null +++ b/pkg/sentry/syscalls/linux/sys_splice.go @@ -0,0 +1,293 @@ +// Copyright 2019 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package linux + +import ( + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" + "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// doSplice implements a blocking splice operation. +func doSplice(t *kernel.Task, outFile, inFile *fs.File, opts fs.SpliceOpts, nonBlocking bool) (int64, error) { + var ( + total int64 + n int64 + err error + ch chan struct{} + inW bool + outW bool + ) + for opts.Length > 0 { + n, err = fs.Splice(t, outFile, inFile, opts) + opts.Length -= n + total += n + if err != syserror.ErrWouldBlock { + break + } else if err == syserror.ErrWouldBlock && nonBlocking { + break + } + + // Are we a registered waiter? + if ch == nil { + ch = make(chan struct{}, 1) + } + if !inW && inFile.Readiness(EventMaskRead) == 0 && !inFile.Flags().NonBlocking { + w, _ := waiter.NewChannelEntry(ch) + inFile.EventRegister(&w, EventMaskRead) + defer inFile.EventUnregister(&w) + inW = true // Registered. + } else if !outW && outFile.Readiness(EventMaskWrite) == 0 && !outFile.Flags().NonBlocking { + w, _ := waiter.NewChannelEntry(ch) + outFile.EventRegister(&w, EventMaskWrite) + defer outFile.EventUnregister(&w) + outW = true // Registered. + } + + // Was anything registered? If no, everything is non-blocking. + if !inW && !outW { + break + } + + // Block until there's data. + if err = t.Block(ch); err != nil { + break + } + } + + return total, err +} + +// Sendfile implements linux system call sendfile(2). +func Sendfile(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { + outFD := kdefs.FD(args[0].Int()) + inFD := kdefs.FD(args[1].Int()) + offsetAddr := args[2].Pointer() + count := int64(args[3].SizeT()) + + // Don't send a negative number of bytes. + if count < 0 { + return 0, nil, syserror.EINVAL + } + + // Get files. + outFile := t.FDMap().GetFile(outFD) + if outFile == nil { + return 0, nil, syserror.EBADF + } + defer outFile.DecRef() + + inFile := t.FDMap().GetFile(inFD) + if inFile == nil { + return 0, nil, syserror.EBADF + } + defer inFile.DecRef() + + // Verify that the outfile Append flag is not set. Note that fs.Splice + // itself validates that the output file is writable. + if outFile.Flags().Append { + return 0, nil, syserror.EBADF + } + + // Verify that we have a regular infile. This is a requirement; the + // same check appears in Linux (fs/splice.c:splice_direct_to_actor). + if !fs.IsRegular(inFile.Dirent.Inode.StableAttr) { + return 0, nil, syserror.EINVAL + } + + var ( + n int64 + err error + ) + if offsetAddr != 0 { + // Verify that when offset address is not null, infile must be + // seekable. The fs.Splice routine itself validates basic read. + if !inFile.Flags().Pread { + return 0, nil, syserror.ESPIPE + } + + // Copy in the offset. + var offset int64 + if _, err := t.CopyIn(offsetAddr, &offset); err != nil { + return 0, nil, err + } + + // The offset must be valid. + if offset < 0 { + return 0, nil, syserror.EINVAL + } + + // Do the splice. + n, err = doSplice(t, outFile, inFile, fs.SpliceOpts{ + Length: count, + SrcOffset: true, + SrcStart: offset, + }, false) + + // Copy out the new offset. + if _, err := t.CopyOut(offsetAddr, n+offset); err != nil { + return 0, nil, err + } + } else { + // Send data using splice. + n, err = doSplice(t, outFile, inFile, fs.SpliceOpts{ + Length: count, + }, false) + } + + // We can only pass a single file to handleIOError, so pick inFile + // arbitrarily. This is used only for debugging purposes. + return uintptr(n), nil, handleIOError(t, n != 0, err, kernel.ERESTARTSYS, "sendfile", inFile) +} + +// Splice implements splice(2). +func Splice(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { + inFD := kdefs.FD(args[0].Int()) + inOffset := args[1].Pointer() + outFD := kdefs.FD(args[2].Int()) + outOffset := args[3].Pointer() + count := int64(args[4].SizeT()) + flags := args[5].Int() + + // Check for invalid flags. + if flags&^(linux.SPLICE_F_MOVE|linux.SPLICE_F_NONBLOCK|linux.SPLICE_F_MORE|linux.SPLICE_F_GIFT) != 0 { + return 0, nil, syserror.EINVAL + } + + // Only non-blocking is meaningful. Note that unlike in Linux, this + // flag is applied consistently. We will have either fully blocking or + // non-blocking behavior below, regardless of the underlying files + // being spliced to. It's unclear if this is a bug or not yet. + nonBlocking := (flags & linux.SPLICE_F_NONBLOCK) != 0 + + // Get files. + outFile := t.FDMap().GetFile(outFD) + if outFile == nil { + return 0, nil, syserror.EBADF + } + defer outFile.DecRef() + + inFile := t.FDMap().GetFile(inFD) + if inFile == nil { + return 0, nil, syserror.EBADF + } + defer inFile.DecRef() + + // Construct our options. + // + // Note that exactly one of the underlying buffers must be a pipe. We + // don't actually have this constraint internally, but we enforce it + // for the semantics of the call. + opts := fs.SpliceOpts{ + Length: count, + } + switch { + case fs.IsPipe(inFile.Dirent.Inode.StableAttr) && !fs.IsPipe(outFile.Dirent.Inode.StableAttr): + if inOffset != 0 { + return 0, nil, syserror.ESPIPE + } + if outOffset != 0 { + var offset int64 + if _, err := t.CopyIn(outOffset, &offset); err != nil { + return 0, nil, err + } + // Use the destination offset. + opts.DstOffset = true + opts.DstStart = offset + } + case !fs.IsPipe(inFile.Dirent.Inode.StableAttr) && fs.IsPipe(outFile.Dirent.Inode.StableAttr): + if outOffset != 0 { + return 0, nil, syserror.ESPIPE + } + if inOffset != 0 { + var offset int64 + if _, err := t.CopyIn(inOffset, &offset); err != nil { + return 0, nil, err + } + // Use the source offset. + opts.SrcOffset = true + opts.SrcStart = offset + } + case fs.IsPipe(inFile.Dirent.Inode.StableAttr) && fs.IsPipe(outFile.Dirent.Inode.StableAttr): + if inOffset != 0 || outOffset != 0 { + return 0, nil, syserror.ESPIPE + } + default: + return 0, nil, syserror.EINVAL + } + + // We may not refer to the same pipe; otherwise it's a continuous loop. + if inFile.Dirent.Inode.StableAttr.InodeID == outFile.Dirent.Inode.StableAttr.InodeID { + return 0, nil, syserror.EINVAL + } + + // Splice data. + n, err := doSplice(t, outFile, inFile, opts, nonBlocking) + + // See above; inFile is chosen arbitrarily here. + return uintptr(n), nil, handleIOError(t, n != 0, err, kernel.ERESTARTSYS, "splice", inFile) +} + +// Tee imlements tee(2). +func Tee(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { + inFD := kdefs.FD(args[0].Int()) + outFD := kdefs.FD(args[1].Int()) + count := int64(args[2].SizeT()) + flags := args[3].Int() + + // Check for invalid flags. + if flags&^(linux.SPLICE_F_MOVE|linux.SPLICE_F_NONBLOCK|linux.SPLICE_F_MORE|linux.SPLICE_F_GIFT) != 0 { + return 0, nil, syserror.EINVAL + } + + // Only non-blocking is meaningful. + nonBlocking := (flags & linux.SPLICE_F_NONBLOCK) != 0 + + // Get files. + outFile := t.FDMap().GetFile(outFD) + if outFile == nil { + return 0, nil, syserror.EBADF + } + defer outFile.DecRef() + + inFile := t.FDMap().GetFile(inFD) + if inFile == nil { + return 0, nil, syserror.EBADF + } + defer inFile.DecRef() + + // All files must be pipes. + if !fs.IsPipe(inFile.Dirent.Inode.StableAttr) || !fs.IsPipe(outFile.Dirent.Inode.StableAttr) { + return 0, nil, syserror.EINVAL + } + + // We may not refer to the same pipe; see above. + if inFile.Dirent.Inode.StableAttr.InodeID == outFile.Dirent.Inode.StableAttr.InodeID { + return 0, nil, syserror.EINVAL + } + + // Splice data. + n, err := doSplice(t, outFile, inFile, fs.SpliceOpts{ + Length: count, + Dup: true, + }, nonBlocking) + + // See above; inFile is chosen arbitrarily here. + return uintptr(n), nil, handleIOError(t, n != 0, err, kernel.ERESTARTSYS, "tee", inFile) +} diff --git a/test/syscalls/BUILD b/test/syscalls/BUILD index 79be06494..b531d7629 100644 --- a/test/syscalls/BUILD +++ b/test/syscalls/BUILD @@ -277,6 +277,8 @@ syscall_test(test = "//test/syscalls/linux:sendfile_socket_test") syscall_test(test = "//test/syscalls/linux:sendfile_test") +syscall_test(test = "//test/syscalls/linux:splice_test") + syscall_test(test = "//test/syscalls/linux:sigaction_test") # TODO(b/119826902): Enable once the test passes in runsc. diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD index ee40be569..d4e49bb3a 100644 --- a/test/syscalls/linux/BUILD +++ b/test/syscalls/linux/BUILD @@ -1747,6 +1747,22 @@ cc_binary( ], ) +cc_binary( + name = "splice_test", + testonly = 1, + srcs = ["splice.cc"], + linkstatic = 1, + deps = [ + "//test/util:file_descriptor", + "//test/util:temp_path", + "//test/util:test_main", + "//test/util:test_util", + "//test/util:thread_util", + "@com_google_absl//absl/strings", + "@com_google_googletest//:gtest", + ], +) + cc_binary( name = "sigaction_test", testonly = 1, diff --git a/test/syscalls/linux/splice.cc b/test/syscalls/linux/splice.cc new file mode 100644 index 000000000..1875f4533 --- /dev/null +++ b/test/syscalls/linux/splice.cc @@ -0,0 +1,404 @@ +// Copyright 2019 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +#include "gmock/gmock.h" +#include "gtest/gtest.h" +#include "absl/strings/string_view.h" +#include "test/util/file_descriptor.h" +#include "test/util/temp_path.h" +#include "test/util/test_util.h" +#include "test/util/thread_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +TEST(SpliceTest, TwoRegularFiles) { + // Create temp files. + const TempPath in_file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + const TempPath out_file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + + // Open the input file as read only. + const FileDescriptor inf = + ASSERT_NO_ERRNO_AND_VALUE(Open(in_file.path(), O_RDONLY)); + + // Open the output file as write only. + const FileDescriptor outf = + ASSERT_NO_ERRNO_AND_VALUE(Open(out_file.path(), O_WRONLY)); + + // Verify that it is rejected as expected; regardless of offsets. + loff_t in_offset = 0; + loff_t out_offset = 0; + EXPECT_THAT(splice(inf.get(), &in_offset, outf.get(), &out_offset, 1, 0), + SyscallFailsWithErrno(EINVAL)); + EXPECT_THAT(splice(inf.get(), nullptr, outf.get(), &out_offset, 1, 0), + SyscallFailsWithErrno(EINVAL)); + EXPECT_THAT(splice(inf.get(), &in_offset, outf.get(), nullptr, 1, 0), + SyscallFailsWithErrno(EINVAL)); + EXPECT_THAT(splice(inf.get(), nullptr, outf.get(), nullptr, 1, 0), + SyscallFailsWithErrno(EINVAL)); +} + +TEST(SpliceTest, SamePipe) { + // Create a new pipe. + int fds[2]; + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor rfd(fds[0]); + const FileDescriptor wfd(fds[1]); + + // Fill the pipe. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ASSERT_THAT(write(wfd.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + + // Attempt to splice to itself. + EXPECT_THAT(splice(rfd.get(), nullptr, wfd.get(), nullptr, kPageSize, 0), + SyscallFailsWithErrno(EINVAL)); +} + +TEST(TeeTest, SamePipe) { + SKIP_IF(IsRunningOnGvisor()); + + // Create a new pipe. + int fds[2]; + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor rfd(fds[0]); + const FileDescriptor wfd(fds[1]); + + // Fill the pipe. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ASSERT_THAT(write(wfd.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + + // Attempt to tee to itself. + EXPECT_THAT(tee(rfd.get(), wfd.get(), kPageSize, 0), + SyscallFailsWithErrno(EINVAL)); +} + +TEST(TeeTest, RegularFile) { + SKIP_IF(IsRunningOnGvisor()); + + // Open some file. + const TempPath in_file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + const FileDescriptor inf = + ASSERT_NO_ERRNO_AND_VALUE(Open(in_file.path(), O_RDWR)); + + // Create a new pipe. + int fds[2]; + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor rfd(fds[0]); + const FileDescriptor wfd(fds[1]); + + // Attempt to tee from the file. + EXPECT_THAT(tee(inf.get(), wfd.get(), kPageSize, 0), + SyscallFailsWithErrno(EINVAL)); + EXPECT_THAT(tee(rfd.get(), inf.get(), kPageSize, 0), + SyscallFailsWithErrno(EINVAL)); +} + +TEST(SpliceTest, PipeOffsets) { + // Create two new pipes. + int first[2], second[2]; + ASSERT_THAT(pipe(first), SyscallSucceeds()); + const FileDescriptor rfd1(first[0]); + const FileDescriptor wfd1(first[1]); + ASSERT_THAT(pipe(second), SyscallSucceeds()); + const FileDescriptor rfd2(second[0]); + const FileDescriptor wfd2(second[1]); + + // All pipe offsets should be rejected. + loff_t in_offset = 0; + loff_t out_offset = 0; + EXPECT_THAT(splice(rfd1.get(), &in_offset, wfd2.get(), &out_offset, 1, 0), + SyscallFailsWithErrno(ESPIPE)); + EXPECT_THAT(splice(rfd1.get(), nullptr, wfd2.get(), &out_offset, 1, 0), + SyscallFailsWithErrno(ESPIPE)); + EXPECT_THAT(splice(rfd1.get(), &in_offset, wfd2.get(), nullptr, 1, 0), + SyscallFailsWithErrno(ESPIPE)); +} + +TEST(SpliceTest, ToPipe) { + // Open the input file. + const TempPath in_file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + const FileDescriptor inf = + ASSERT_NO_ERRNO_AND_VALUE(Open(in_file.path(), O_RDWR)); + + // Fill with some random data. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ASSERT_THAT(write(inf.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + ASSERT_THAT(lseek(inf.get(), 0, SEEK_SET), SyscallSucceedsWithValue(0)); + + // Create a new pipe. + int fds[2]; + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor rfd(fds[0]); + const FileDescriptor wfd(fds[1]); + + // Splice to the pipe. + EXPECT_THAT(splice(inf.get(), nullptr, wfd.get(), nullptr, kPageSize, 0), + SyscallSucceedsWithValue(kPageSize)); + + // Contents should be equal. + std::vector rbuf(kPageSize); + ASSERT_THAT(read(rfd.get(), rbuf.data(), rbuf.size()), + SyscallSucceedsWithValue(kPageSize)); + EXPECT_EQ(memcmp(rbuf.data(), buf.data(), buf.size()), 0); +} + +TEST(SpliceTest, ToPipeOffset) { + // Open the input file. + const TempPath in_file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + const FileDescriptor inf = + ASSERT_NO_ERRNO_AND_VALUE(Open(in_file.path(), O_RDWR)); + + // Fill with some random data. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ASSERT_THAT(write(inf.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + + // Create a new pipe. + int fds[2]; + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor rfd(fds[0]); + const FileDescriptor wfd(fds[1]); + + // Splice to the pipe. + loff_t in_offset = kPageSize / 2; + EXPECT_THAT( + splice(inf.get(), &in_offset, wfd.get(), nullptr, kPageSize / 2, 0), + SyscallSucceedsWithValue(kPageSize / 2)); + + // Contents should be equal to only the second part. + std::vector rbuf(kPageSize / 2); + ASSERT_THAT(read(rfd.get(), rbuf.data(), rbuf.size()), + SyscallSucceedsWithValue(kPageSize / 2)); + EXPECT_EQ(memcmp(rbuf.data(), buf.data() + (kPageSize / 2), rbuf.size()), 0); +} + +TEST(SpliceTest, FromPipe) { + // Create a new pipe. + int fds[2]; + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor rfd(fds[0]); + const FileDescriptor wfd(fds[1]); + + // Fill with some random data. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ASSERT_THAT(write(wfd.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + + // Open the input file. + const TempPath out_file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + const FileDescriptor outf = + ASSERT_NO_ERRNO_AND_VALUE(Open(out_file.path(), O_RDWR)); + + // Splice to the output file. + EXPECT_THAT(splice(rfd.get(), nullptr, outf.get(), nullptr, kPageSize, 0), + SyscallSucceedsWithValue(kPageSize)); + + // The offset of the output should be equal to kPageSize. We assert that and + // reset to zero so that we can read the contents and ensure they match. + EXPECT_THAT(lseek(outf.get(), 0, SEEK_CUR), + SyscallSucceedsWithValue(kPageSize)); + ASSERT_THAT(lseek(outf.get(), 0, SEEK_SET), SyscallSucceedsWithValue(0)); + + // Contents should be equal. + std::vector rbuf(kPageSize); + ASSERT_THAT(read(outf.get(), rbuf.data(), rbuf.size()), + SyscallSucceedsWithValue(kPageSize)); + EXPECT_EQ(memcmp(rbuf.data(), buf.data(), buf.size()), 0); +} + +TEST(SpliceTest, FromPipeOffset) { + // Create a new pipe. + int fds[2]; + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor rfd(fds[0]); + const FileDescriptor wfd(fds[1]); + + // Fill with some random data. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ASSERT_THAT(write(wfd.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + + // Open the input file. + const TempPath out_file = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + const FileDescriptor outf = + ASSERT_NO_ERRNO_AND_VALUE(Open(out_file.path(), O_RDWR)); + + // Splice to the output file. + loff_t out_offset = kPageSize / 2; + EXPECT_THAT(splice(rfd.get(), nullptr, outf.get(), &out_offset, kPageSize, 0), + SyscallSucceedsWithValue(kPageSize)); + + // Content should reflect the splice. We write to a specific offset in the + // file, so the internals should now be allocated sparsely. + std::vector rbuf(kPageSize); + ASSERT_THAT(read(outf.get(), rbuf.data(), rbuf.size()), + SyscallSucceedsWithValue(kPageSize)); + std::vector zbuf(kPageSize / 2); + memset(zbuf.data(), 0, zbuf.size()); + EXPECT_EQ(memcmp(rbuf.data(), zbuf.data(), zbuf.size()), 0); + EXPECT_EQ(memcmp(rbuf.data() + kPageSize / 2, buf.data(), kPageSize / 2), 0); +} + +TEST(SpliceTest, TwoPipes) { + // Create two new pipes. + int first[2], second[2]; + ASSERT_THAT(pipe(first), SyscallSucceeds()); + const FileDescriptor rfd1(first[0]); + const FileDescriptor wfd1(first[1]); + ASSERT_THAT(pipe(second), SyscallSucceeds()); + const FileDescriptor rfd2(second[0]); + const FileDescriptor wfd2(second[1]); + + // Fill with some random data. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ASSERT_THAT(write(wfd1.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + + // Splice to the second pipe, using two operations. + EXPECT_THAT( + splice(rfd1.get(), nullptr, wfd2.get(), nullptr, kPageSize / 2, 0), + SyscallSucceedsWithValue(kPageSize / 2)); + EXPECT_THAT( + splice(rfd1.get(), nullptr, wfd2.get(), nullptr, kPageSize / 2, 0), + SyscallSucceedsWithValue(kPageSize / 2)); + + // Content should reflect the splice. + std::vector rbuf(kPageSize); + ASSERT_THAT(read(rfd2.get(), rbuf.data(), rbuf.size()), + SyscallSucceedsWithValue(kPageSize)); + EXPECT_EQ(memcmp(rbuf.data(), buf.data(), kPageSize), 0); +} + +TEST(SpliceTest, Blocking) { + // Create two new pipes. + int first[2], second[2]; + ASSERT_THAT(pipe(first), SyscallSucceeds()); + const FileDescriptor rfd1(first[0]); + const FileDescriptor wfd1(first[1]); + ASSERT_THAT(pipe(second), SyscallSucceeds()); + const FileDescriptor rfd2(second[0]); + const FileDescriptor wfd2(second[1]); + + // This thread writes to the main pipe. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ScopedThread t([&]() { + ASSERT_THAT(write(wfd1.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + }); + + // Attempt a splice immediately; it should block. + EXPECT_THAT(splice(rfd1.get(), nullptr, wfd2.get(), nullptr, kPageSize, 0), + SyscallSucceedsWithValue(kPageSize)); + + // Thread should be joinable. + t.Join(); + + // Content should reflect the splice. + std::vector rbuf(kPageSize); + ASSERT_THAT(read(rfd2.get(), rbuf.data(), rbuf.size()), + SyscallSucceedsWithValue(kPageSize)); + EXPECT_EQ(memcmp(rbuf.data(), buf.data(), kPageSize), 0); +} + +TEST(TeeTest, Blocking) { + SKIP_IF(IsRunningOnGvisor()); + + // Create two new pipes. + int first[2], second[2]; + ASSERT_THAT(pipe(first), SyscallSucceeds()); + const FileDescriptor rfd1(first[0]); + const FileDescriptor wfd1(first[1]); + ASSERT_THAT(pipe(second), SyscallSucceeds()); + const FileDescriptor rfd2(second[0]); + const FileDescriptor wfd2(second[1]); + + // This thread writes to the main pipe. + std::vector buf(kPageSize); + RandomizeBuffer(buf.data(), buf.size()); + ScopedThread t([&]() { + ASSERT_THAT(write(wfd1.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(kPageSize)); + }); + + // Attempt a tee immediately; it should block. + EXPECT_THAT(tee(rfd1.get(), wfd2.get(), kPageSize, 0), + SyscallSucceedsWithValue(kPageSize)); + + // Thread should be joinable. + t.Join(); + + // Content should reflect the splice, in both pipes. + std::vector rbuf(kPageSize); + ASSERT_THAT(read(rfd2.get(), rbuf.data(), rbuf.size()), + SyscallSucceedsWithValue(kPageSize)); + EXPECT_EQ(memcmp(rbuf.data(), buf.data(), kPageSize), 0); + ASSERT_THAT(read(rfd1.get(), rbuf.data(), rbuf.size()), + SyscallSucceedsWithValue(kPageSize)); + EXPECT_EQ(memcmp(rbuf.data(), buf.data(), kPageSize), 0); +} + +TEST(SpliceTest, NonBlocking) { + // Create two new pipes. + int first[2], second[2]; + ASSERT_THAT(pipe(first), SyscallSucceeds()); + const FileDescriptor rfd1(first[0]); + const FileDescriptor wfd1(first[1]); + ASSERT_THAT(pipe(second), SyscallSucceeds()); + const FileDescriptor rfd2(second[0]); + const FileDescriptor wfd2(second[1]); + + // Splice with no data to back it. + EXPECT_THAT(splice(rfd1.get(), nullptr, wfd2.get(), nullptr, kPageSize, + SPLICE_F_NONBLOCK), + SyscallFailsWithErrno(EAGAIN)); +} + +TEST(TeeTest, NonBlocking) { + SKIP_IF(IsRunningOnGvisor()); + + // Create two new pipes. + int first[2], second[2]; + ASSERT_THAT(pipe(first), SyscallSucceeds()); + const FileDescriptor rfd1(first[0]); + const FileDescriptor wfd1(first[1]); + ASSERT_THAT(pipe(second), SyscallSucceeds()); + const FileDescriptor rfd2(second[0]); + const FileDescriptor wfd2(second[1]); + + // Splice with no data to back it. + EXPECT_THAT(tee(rfd1.get(), wfd2.get(), kPageSize, SPLICE_F_NONBLOCK), + SyscallFailsWithErrno(EAGAIN)); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3