diff options
author | Googler <noreply@google.com> | 2018-04-27 10:37:02 -0700 |
---|---|---|
committer | Adin Scannell <ascannell@google.com> | 2018-04-28 01:44:26 -0400 |
commit | d02b74a5dcfed4bfc8f2f8e545bca4d2afabb296 (patch) | |
tree | 54f95eef73aee6bacbfc736fffc631be2605ed53 /pkg/sentry/fs/gofer | |
parent | f70210e742919f40aa2f0934a22f1c9ba6dada62 (diff) |
Check in gVisor.
PiperOrigin-RevId: 194583126
Change-Id: Ica1d8821a90f74e7e745962d71801c598c652463
Diffstat (limited to 'pkg/sentry/fs/gofer')
-rw-r--r-- | pkg/sentry/fs/gofer/BUILD | 90 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/attr.go | 162 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/context_file.go | 190 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/device.go | 20 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/file.go | 255 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/file_state.go | 37 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/fs.go | 252 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/gofer_test.go | 776 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/handles.go | 144 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/inode.go | 554 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/inode_state.go | 141 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/path.go | 331 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/session.go | 251 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/session_state.go | 90 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/socket.go | 127 | ||||
-rw-r--r-- | pkg/sentry/fs/gofer/util.go | 60 |
16 files changed, 3480 insertions, 0 deletions
diff --git a/pkg/sentry/fs/gofer/BUILD b/pkg/sentry/fs/gofer/BUILD new file mode 100644 index 000000000..ca42b0a54 --- /dev/null +++ b/pkg/sentry/fs/gofer/BUILD @@ -0,0 +1,90 @@ +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 = "gofer_state", + srcs = [ + "file.go", + "file_state.go", + "fs.go", + "inode.go", + "inode_state.go", + "session.go", + "session_state.go", + ], + out = "gofer_state.go", + package = "gofer", +) + +go_library( + name = "gofer", + srcs = [ + "attr.go", + "context_file.go", + "device.go", + "file.go", + "file_state.go", + "fs.go", + "gofer_state.go", + "handles.go", + "inode.go", + "inode_state.go", + "path.go", + "session.go", + "session_state.go", + "socket.go", + "util.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs/gofer", + visibility = ["//pkg/sentry:internal"], + deps = [ + "//pkg/abi/linux", + "//pkg/amutex", + "//pkg/fd", + "//pkg/log", + "//pkg/metric", + "//pkg/p9", + "//pkg/refs", + "//pkg/secio", + "//pkg/sentry/context", + "//pkg/sentry/device", + "//pkg/sentry/fs", + "//pkg/sentry/fs/fdpipe", + "//pkg/sentry/fs/fsutil", + "//pkg/sentry/fs/host", + "//pkg/sentry/fs/lock", + "//pkg/sentry/kernel/auth", + "//pkg/sentry/kernel/time", + "//pkg/sentry/memmap", + "//pkg/sentry/platform", + "//pkg/sentry/safemem", + "//pkg/sentry/uniqueid", + "//pkg/sentry/usermem", + "//pkg/state", + "//pkg/syserror", + "//pkg/tcpip", + "//pkg/tcpip/transport/unix", + "//pkg/unet", + "//pkg/waiter", + ], +) + +go_test( + name = "gofer_test", + size = "small", + 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/attr.go b/pkg/sentry/fs/gofer/attr.go new file mode 100644 index 000000000..5e24767f9 --- /dev/null +++ b/pkg/sentry/fs/gofer/attr.go @@ -0,0 +1,162 @@ +// 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 gofer + +import ( + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth" + ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +// getattr returns the 9p attributes of the p9.File. On success, Mode, Size, and RDev +// are guaranteed to be masked as valid. +func getattr(ctx context.Context, file contextFile) (p9.QID, p9.AttrMask, p9.Attr, error) { + // Retrieve attributes over the wire. + qid, valid, attr, err := file.getAttr(ctx, p9.AttrMaskAll()) + if err != nil { + return qid, valid, attr, err + } + + // Require mode, size, and raw device id. + if !valid.Mode || !valid.Size || !valid.RDev { + return qid, valid, attr, syscall.EIO + } + + return qid, valid, attr, nil +} + +func unstable(ctx context.Context, valid p9.AttrMask, pattr p9.Attr, mounter fs.FileOwner, client *p9.Client) fs.UnstableAttr { + return fs.UnstableAttr{ + Size: int64(pattr.Size), + Usage: int64(pattr.Size), + Perms: perms(valid, pattr, client), + Owner: owner(mounter, valid, pattr), + AccessTime: atime(ctx, valid, pattr), + ModificationTime: mtime(ctx, valid, pattr), + StatusChangeTime: ctime(ctx, valid, pattr), + Links: links(valid, pattr), + } +} + +func perms(valid p9.AttrMask, pattr p9.Attr, client *p9.Client) fs.FilePermissions { + if pattr.Mode.IsDir() && !p9.VersionSupportsMultiUser(client.Version()) { + // If user and group permissions bits are not supplied, use + // "other" bits to supplement them. + // + // Older Gofer's fake directories only have "other" permission, + // but will often be accessed via user or group permissions. + if pattr.Mode&0770 == 0 { + other := pattr.Mode & 07 + pattr.Mode = pattr.Mode | other<<3 | other<<6 + } + } + return fs.FilePermsFromP9(pattr.Mode) +} + +func owner(mounter fs.FileOwner, valid p9.AttrMask, pattr p9.Attr) fs.FileOwner { + // Unless the file returned its UID and GID, it belongs to the mounting + // task's EUID/EGID. + owner := mounter + if valid.UID { + owner.UID = auth.KUID(pattr.UID) + } + if valid.GID { + owner.GID = auth.KGID(pattr.GID) + } + return owner +} + +// bsize returns a block size from 9p attributes. +func bsize(pattr p9.Attr) int64 { + if pattr.BlockSize > 0 { + return int64(pattr.BlockSize) + } + // Some files may have no clue of their block size. Better not to report + // something misleading or buggy and have a safe default. + return usermem.PageSize +} + +// ntype returns an fs.InodeType from 9p attributes. +func ntype(pattr p9.Attr) fs.InodeType { + switch { + case pattr.Mode.IsNamedPipe(): + return fs.Pipe + case pattr.Mode.IsDir(): + return fs.Directory + case pattr.Mode.IsSymlink(): + return fs.Symlink + case pattr.Mode.IsCharacterDevice(): + return fs.CharacterDevice + case pattr.Mode.IsBlockDevice(): + return fs.BlockDevice + case pattr.Mode.IsSocket(): + return fs.Socket + case pattr.Mode.IsRegular(): + fallthrough + default: + return fs.RegularFile + } +} + +// ctime returns a change time from 9p attributes. +func ctime(ctx context.Context, valid p9.AttrMask, pattr p9.Attr) ktime.Time { + if valid.CTime { + return ktime.FromUnix(int64(pattr.CTimeSeconds), int64(pattr.CTimeNanoSeconds)) + } + // Approximate ctime with mtime if ctime isn't available. + return mtime(ctx, valid, pattr) +} + +// atime returns an access time from 9p attributes. +func atime(ctx context.Context, valid p9.AttrMask, pattr p9.Attr) ktime.Time { + if valid.ATime { + return ktime.FromUnix(int64(pattr.ATimeSeconds), int64(pattr.ATimeNanoSeconds)) + } + return ktime.NowFromContext(ctx) +} + +// mtime returns a modification time from 9p attributes. +func mtime(ctx context.Context, valid p9.AttrMask, pattr p9.Attr) ktime.Time { + if valid.MTime { + return ktime.FromUnix(int64(pattr.MTimeSeconds), int64(pattr.MTimeNanoSeconds)) + } + return ktime.NowFromContext(ctx) +} + +// links returns a hard link count from 9p attributes. +func links(valid p9.AttrMask, pattr p9.Attr) uint64 { + // For gofer file systems that support link count (such as a local file gofer), + // we return the link count reported by the underlying file system. + if valid.NLink { + return pattr.NLink + } + + // This node is likely backed by a file system that doesn't support links. + // We could readdir() and count children directories to provide an accurate + // link count. However this may be expensive since the gofer may be backed by remote + // storage. Instead, simply return 2 links for directories and 1 for everything else + // since no one relies on an accurate link count for gofer-based file systems. + switch ntype(pattr) { + case fs.Directory: + return 2 + default: + return 1 + } +} diff --git a/pkg/sentry/fs/gofer/context_file.go b/pkg/sentry/fs/gofer/context_file.go new file mode 100644 index 000000000..d4b6f6eb7 --- /dev/null +++ b/pkg/sentry/fs/gofer/context_file.go @@ -0,0 +1,190 @@ +// 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 gofer + +import ( + "gvisor.googlesource.com/gvisor/pkg/fd" + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" +) + +// contextFile is a wrapper around p9.File that notifies the context that +// it's about to sleep before calling the Gofer over P9. +type contextFile struct { + file p9.File +} + +func (c *contextFile) walk(ctx context.Context, names []string) ([]p9.QID, contextFile, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + q, f, err := c.file.Walk(names) + if err != nil { + return nil, contextFile{}, err + } + return q, contextFile{file: f}, nil +} + +func (c *contextFile) statFS(ctx context.Context) (p9.FSStat, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.StatFS() +} + +func (c *contextFile) getAttr(ctx context.Context, req p9.AttrMask) (p9.QID, p9.AttrMask, p9.Attr, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.GetAttr(req) +} + +func (c *contextFile) setAttr(ctx context.Context, valid p9.SetAttrMask, attr p9.SetAttr) error { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + 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) + + return c.file.Rename(directory.file, name) +} + +func (c *contextFile) close(ctx context.Context) error { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Close() +} + +func (c *contextFile) open(ctx context.Context, mode p9.OpenFlags) (*fd.FD, p9.QID, uint32, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Open(mode) +} + +func (c *contextFile) readAt(ctx context.Context, p []byte, offset uint64) (int, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.ReadAt(p, offset) +} + +func (c *contextFile) writeAt(ctx context.Context, p []byte, offset uint64) (int, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.WriteAt(p, offset) +} + +func (c *contextFile) fsync(ctx context.Context) error { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.FSync() +} + +func (c *contextFile) create(ctx context.Context, name string, flags p9.OpenFlags, permissions p9.FileMode, uid p9.UID, gid p9.GID) (*fd.FD, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + fd, _, _, _, err := c.file.Create(name, flags, permissions, uid, gid) + return fd, err +} + +func (c *contextFile) mkdir(ctx context.Context, name string, permissions p9.FileMode, uid p9.UID, gid p9.GID) (p9.QID, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Mkdir(name, permissions, uid, gid) +} + +func (c *contextFile) symlink(ctx context.Context, oldName string, newName string, uid p9.UID, gid p9.GID) (p9.QID, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Symlink(oldName, newName, uid, gid) +} + +func (c *contextFile) link(ctx context.Context, target *contextFile, newName string) error { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Link(target.file, newName) +} + +func (c *contextFile) mknod(ctx context.Context, name string, permissions p9.FileMode, major uint32, minor uint32, uid p9.UID, gid p9.GID) (p9.QID, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Mknod(name, permissions, major, minor, uid, gid) +} + +func (c *contextFile) unlinkAt(ctx context.Context, name string, flags uint32) error { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.UnlinkAt(name, flags) +} + +func (c *contextFile) readdir(ctx context.Context, offset uint64, count uint32) ([]p9.Dirent, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Readdir(offset, count) +} + +func (c *contextFile) readlink(ctx context.Context) (string, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Readlink() +} + +func (c *contextFile) flush(ctx context.Context) error { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Flush() +} + +func (c *contextFile) walkGetAttr(ctx context.Context, names []string) ([]p9.QID, contextFile, p9.AttrMask, p9.Attr, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + q, f, m, a, err := c.file.WalkGetAttr(names) + if err != nil { + return nil, contextFile{}, p9.AttrMask{}, p9.Attr{}, err + } + return q, contextFile{file: f}, m, a, nil +} + +func (c *contextFile) connect(ctx context.Context, flags p9.ConnectFlags) (*fd.FD, error) { + ctx.UninterruptibleSleepStart(false) + defer ctx.UninterruptibleSleepFinish(false) + + return c.file.Connect(flags) +} diff --git a/pkg/sentry/fs/gofer/device.go b/pkg/sentry/fs/gofer/device.go new file mode 100644 index 000000000..fac7306d4 --- /dev/null +++ b/pkg/sentry/fs/gofer/device.go @@ -0,0 +1,20 @@ +// 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 gofer + +import "gvisor.googlesource.com/gvisor/pkg/sentry/device" + +// goferDevice is the gofer virtual device. +var goferDevice = device.NewAnonMultiDevice() diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go new file mode 100644 index 000000000..07c9bf01d --- /dev/null +++ b/pkg/sentry/fs/gofer/file.go @@ -0,0 +1,255 @@ +// 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 gofer + +import ( + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/log" + "gvisor.googlesource.com/gvisor/pkg/metric" + "gvisor.googlesource.com/gvisor/pkg/p9" + "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/fs/fsutil" + "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" +) + +var openedWX = metric.MustCreateNewUint64Metric("/gofer/opened_write_execute_file", true /* sync */, "Number of times a writable+executable file was opened from a gofer.") + +// fileOperations implements fs.FileOperations for a remote file system. +type fileOperations struct { + fsutil.NoIoctl `state:"nosave"` + 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 + // FileOperations.Release is called. + inodeOperations *inodeOperations `state:"wait"` + + // dirCursor is the directory cursor. + dirCursor string + + // handles are the opened remote file system handles, which may + // be shared with other files. + handles *handles `state:"nosave"` + + // flags are the flags used to open handles. + flags fs.FileFlags `state:"wait"` +} + +// fileOperations implements fs.FileOperations. +var _ fs.FileOperations = (*fileOperations)(nil) + +// NewFile returns a file. NewFile is not appropriate with host pipes and sockets. +func NewFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags, i *inodeOperations, handles *handles) *fs.File { + // Remote file systems enforce readability/writability at an offset, + // see fs/9p/vfs_inode.c:v9fs_vfs_atomic_open -> fs/open.c:finish_open. + flags.Pread = true + flags.Pwrite = true + + f := &fileOperations{ + inodeOperations: i, + handles: handles, + flags: flags, + } + if flags.Write { + if err := dirent.Inode.CheckPermission(ctx, fs.PermMask{Execute: true}); err == nil { + name, _ := dirent.FullName(fs.RootFromContext(ctx)) + openedWX.Increment() + log.Warningf("Opened a writable executable: %q", name) + } + } + return fs.NewFile(ctx, dirent, flags, f) +} + +// Release implements fs.FileOpeations.Release. +func (f *fileOperations) Release() { + f.handles.DecRef() +} + +// Readdir implements fs.FileOperations.Readdir. +func (f *fileOperations) 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: &f.dirCursor, + } + n, err := fs.DirentReaddir(ctx, file.Dirent, f, root, dirCtx, file.Offset()) + if f.inodeOperations.session().cachePolicy != cacheNone { + f.inodeOperations.cachingInodeOps.TouchAccessTime(ctx, file.Dirent.Inode) + } + return n, err +} + +// IterateDir implements fs.DirIterator.IterateDir. +func (f *fileOperations) IterateDir(ctx context.Context, dirCtx *fs.DirCtx, offset int) (int, error) { + f.inodeOperations.readdirMu.Lock() + defer f.inodeOperations.readdirMu.Unlock() + + // Fetch directory entries if needed. + if f.inodeOperations.readdirCache == nil || f.inodeOperations.session().cachePolicy == cacheNone { + entries, err := f.readdirAll(ctx) + if err != nil { + return offset, err + } + + // Cache the readdir result. + f.inodeOperations.readdirCache = fs.NewSortedDentryMap(entries) + } + + // Serialize the entries. + n, err := fs.GenericReaddir(dirCtx, f.inodeOperations.readdirCache) + return offset + n, err +} + +// readdirAll fetches fs.DentAttrs for f, using the attributes of g. +func (f *fileOperations) readdirAll(ctx context.Context) (map[string]fs.DentAttr, error) { + entries := make(map[string]fs.DentAttr) + var readOffset uint64 + for { + // We choose some arbitrary high number of directory entries (64k) and call + // Readdir until we've exhausted them all. + dirents, err := f.handles.File.readdir(ctx, readOffset, 64*1024) + if err != nil { + return nil, err + } + if len(dirents) == 0 { + // We're done, we reached EOF. + break + } + + // The last dirent contains the offset into the next set of dirents. The gofer + // returns the offset as an index into directories, not as a byte offset, because + // converting a byte offset to an index into directories entries is a huge pain. + // But everything is fine if we're consistent. + readOffset = dirents[len(dirents)-1].Offset + + for _, dirent := range dirents { + if dirent.Name == "." || dirent.Name == ".." { + // These must not be included in Readdir results. + continue + } + + // Find a best approximation of the type. + var nt fs.InodeType + switch dirent.Type { + case p9.TypeDir: + nt = fs.Directory + case p9.TypeSymlink: + nt = fs.Symlink + default: + nt = fs.RegularFile + } + + // Install the DentAttr. + entries[dirent.Name] = fs.DentAttr{ + Type: nt, + // Construct the key to find the virtual inode. + // Directory entries reside on the same Device + // and SecondaryDevice as their parent. + InodeID: goferDevice.Map(device.MultiDeviceKey{ + Device: f.inodeOperations.fileState.key.Device, + SecondaryDevice: f.inodeOperations.fileState.key.SecondaryDevice, + Inode: dirent.QID.Path, + }), + } + } + } + + return entries, nil +} + +// Write implements fs.FileOperations.Write. +func (f *fileOperations) Write(ctx context.Context, file *fs.File, src usermem.IOSequence, offset int64) (int64, error) { + if fs.IsDir(file.Dirent.Inode.StableAttr) { + // Not all remote file systems enforce this so this client does. + return 0, syserror.EISDIR + } + + // Do cached IO for regular files only. Some character devices expect no caching. + isFile := fs.IsFile(file.Dirent.Inode.StableAttr) + if f.inodeOperations.session().cachePolicy == cacheNone || !isFile { + return src.CopyInTo(ctx, f.handles.readWriterAt(ctx, offset)) + } + return f.inodeOperations.cachingInodeOps.Write(ctx, src, offset) +} + +// Read implements fs.FileOperations.Read. +func (f *fileOperations) Read(ctx context.Context, file *fs.File, dst usermem.IOSequence, offset int64) (int64, error) { + if fs.IsDir(file.Dirent.Inode.StableAttr) { + // Not all remote file systems enforce this so this client does. + return 0, syserror.EISDIR + } + + // Do cached IO for regular files only. Some character devices expect no caching. + isFile := fs.IsFile(file.Dirent.Inode.StableAttr) + if f.inodeOperations.session().cachePolicy == cacheNone || !isFile { + return dst.CopyOutFrom(ctx, f.handles.readWriterAt(ctx, offset)) + } + return f.inodeOperations.cachingInodeOps.Read(ctx, file, dst, offset) +} + +// Fsync implements fs.FileOperations.Fsync. +func (f *fileOperations) Fsync(ctx context.Context, file *fs.File, start int64, end int64, syncType fs.SyncType) error { + switch syncType { + case fs.SyncAll, fs.SyncData: + if err := file.Dirent.Inode.WriteOut(ctx); err != nil { + return err + } + fallthrough + case fs.SyncBackingStorage: + // Sync remote caches. + if f.handles.Host != nil { + // Sync the host fd directly. + return syscall.Fsync(f.handles.Host.FD()) + } + // Otherwise sync on the p9.File handle. + return f.handles.File.fsync(ctx) + } + panic("invalid sync type") +} + +// Flush implements fs.FileOperations.Flush. +func (f *fileOperations) Flush(ctx context.Context, file *fs.File) error { + // If this file is not opened writable then there is nothing to flush. + // We do this because some p9 server implementations of Flush are + // over-zealous. + // + // FIXME: weaken these implementations and remove this check. + if !file.Flags().Write { + return nil + } + // Execute the flush. + return f.handles.File.flush(ctx) +} + +// ConfigureMMap implements fs.FileOperations.ConfigureMMap. +func (f *fileOperations) ConfigureMMap(ctx context.Context, file *fs.File, opts *memmap.MMapOpts) error { + if !isFileCachable(f.inodeOperations.session(), file.Dirent.Inode) { + return syserror.ENODEV + } + return fsutil.GenericConfigureMMap(file, f.inodeOperations.cachingInodeOps, opts) +} + +// 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/gofer/file_state.go b/pkg/sentry/fs/gofer/file_state.go new file mode 100644 index 000000000..1d63e33ec --- /dev/null +++ b/pkg/sentry/fs/gofer/file_state.go @@ -0,0 +1,37 @@ +// 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 gofer + +import ( + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" +) + +// afterLoad is invoked by stateify. +func (f *fileOperations) afterLoad() { + load := func() { + f.inodeOperations.fileState.waitForLoad() + + // Manually load the open handles. + var err error + // TODO: Context is not plumbed to save/restore. + f.handles, err = newHandles(context.Background(), f.inodeOperations.fileState.file, f.flags) + if err != nil { + panic("failed to re-open handle: " + err.Error()) + } + f.inodeOperations.fileState.setHandlesForCachedIO(f.flags, f.handles) + } + fs.Async(load) +} diff --git a/pkg/sentry/fs/gofer/fs.go b/pkg/sentry/fs/gofer/fs.go new file mode 100644 index 000000000..0a1a49bbd --- /dev/null +++ b/pkg/sentry/fs/gofer/fs.go @@ -0,0 +1,252 @@ +// 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 gofer implements a remote 9p filesystem. +package gofer + +import ( + "errors" + "fmt" + "strconv" + + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" +) + +// The following are options defined by the Linux 9p client that we support, +// see Documentation/filesystems/9p.txt. +const ( + // The transport method. + transportKey = "trans" + + // The file tree to access when the file server + // is exporting several file systems. Stands for "attach name". + anameKey = "aname" + + // The caching policy. + cacheKey = "cache" + + // The file descriptor for reading with trans=fd. + readFDKey = "rfdno" + + // The file descriptor for writing with trans=fd. + writeFDKey = "wfdno" + + // The number of bytes to use for a 9p packet payload. + msizeKey = "msize" + + // The 9p protocol version. + versionKey = "version" + + // If set to true allows the creation of unix domain sockets inside the + // sandbox using files backed by the gofer. If set to false, unix sockets + // cannot be bound to gofer files without an overlay on top. + privateUnixSocketKey = "privateunixsocket" +) + +// cachePolicy is a 9p cache policy. +type cachePolicy string + +const ( + // Use virtual file system cache. + cacheAll cachePolicy = "fscache" + + // TODO: fully support cache=none. + cacheNone cachePolicy = "none" + + // defaultCache is cacheAll. Note this diverges from the 9p Linux + // client whose default is "none". See TODO above. + defaultCache = cacheAll +) + +// defaultAname is the default attach name. +const defaultAname = "/" + +// defaultMSize is the message size used for chunking large read and write requests. +// This has been tested to give good enough performance up to 64M. +const defaultMSize = 1024 * 1024 // 1M + +// defaultVersion is the default 9p protocol version. Will negotiate downwards with +// file server if needed. +var defaultVersion = p9.HighestVersionString() + +// Number of names of non-children to cache, preventing unneeded walks. 64 is +// plenty for nodejs, which seems to stat about 4 children on every require(). +const nonChildrenCacheSize = 64 + +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='") + + // ErrNoWriteFD is returned when there is no 'wfdno' option. + ErrNoWriteFD = errors.New("missing required option: 'wfdno='") +) + +// filesystem is a 9p client. +type filesystem struct{} + +func init() { + fs.RegisterFilesystem(&filesystem{}) +} + +// FilesystemName is the name under which the filesystem is registered. +// The name matches fs/9p/vfs_super.c:v9fs_fs_type.name. +const FilesystemName = "9p" + +// Name is the name of the filesystem. +func (*filesystem) Name() string { + return FilesystemName +} + +// AllowUserMount prohibits users from using mount(2) with this file system. +func (*filesystem) AllowUserMount() bool { + return false +} + +// Flags returns that there is nothing special about this file system. +// +// The 9p Linux client returns FS_RENAME_DOES_D_MOVE, see fs/9p/vfs_super.c. +func (*filesystem) Flags() fs.FilesystemFlags { + return 0 +} + +// Mount returns an attached 9p client that can be positioned in the vfs. +func (f *filesystem) Mount(ctx context.Context, device string, flags fs.MountSourceFlags, data string) (*fs.Inode, error) { + // Parse and validate the mount options. + o, err := options(data) + if err != nil { + return nil, err + } + + // Construct the 9p root to mount. We intentionally diverge from Linux in that + // the first Tversion and Tattach requests are done lazily. + return Root(ctx, device, f, flags, o) +} + +// opts are parsed 9p mount options. +type opts struct { + fd int + aname string + policy cachePolicy + msize uint32 + version string + privateunixsocket bool +} + +// options parses mount(2) data into structured options. +func options(data string) (opts, error) { + var o opts + + // Parse generic comma-separated key=value options, this file system expects them. + options := fs.GenericMountSourceOptions(data) + + // Check for the required 'trans=fd' option. + trans, ok := options[transportKey] + if !ok { + return o, ErrNoTransport + } + if trans != "fd" { + return o, fmt.Errorf("unsupported transport: 'trans=%s'", trans) + } + delete(options, transportKey) + + // Check for the required 'rfdno=' option. + srfd, ok := options[readFDKey] + if !ok { + return o, ErrNoReadFD + } + delete(options, readFDKey) + + // Check for the required 'wfdno=' option. + swfd, ok := options[writeFDKey] + if !ok { + return o, ErrNoWriteFD + } + delete(options, writeFDKey) + + // Parse the read fd. + rfd, err := strconv.Atoi(srfd) + if err != nil { + return o, fmt.Errorf("invalid fd for 'rfdno=%s': %v", srfd, err) + } + + // Parse the write fd. + wfd, err := strconv.Atoi(swfd) + if err != nil { + return o, fmt.Errorf("invalid fd for 'wfdno=%s': %v", swfd, err) + } + + // Require that the read and write fd are the same. + if rfd != wfd { + return o, fmt.Errorf("fd in 'rfdno=%d' and 'wfdno=%d' must match", rfd, wfd) + } + o.fd = rfd + + // Parse the attach name. + o.aname = defaultAname + if an, ok := options[anameKey]; ok { + o.aname = an + delete(options, anameKey) + } + + // Parse the cache policy. Reject unsupported policies. + o.policy = cacheAll + if cp, ok := options[cacheKey]; ok { + if cachePolicy(cp) != cacheAll && cachePolicy(cp) != cacheNone { + return o, fmt.Errorf("unsupported cache mode: 'cache=%s'", cp) + } + o.policy = cachePolicy(cp) + delete(options, cacheKey) + } + + // Parse the message size. Reject malformed options. + o.msize = uint32(defaultMSize) + if m, ok := options[msizeKey]; ok { + i, err := strconv.ParseUint(m, 10, 32) + if err != nil { + return o, fmt.Errorf("invalid message size for 'msize=%s': %v", m, err) + } + o.msize = uint32(i) + delete(options, msizeKey) + } + + // Parse the protocol version. + o.version = defaultVersion + if v, ok := options[versionKey]; ok { + o.version = v + delete(options, versionKey) + } + + // Parse the unix socket policy. Reject non-booleans. + if v, ok := options[privateUnixSocketKey]; ok { + b, err := strconv.ParseBool(v) + if err != nil { + return o, fmt.Errorf("invalid boolean value for '%s=%s': %v", privateUnixSocketKey, v, err) + } + o.privateunixsocket = b + delete(options, privateUnixSocketKey) + } + + // Fail to attach if the caller wanted us to do something that we + // don't support. + if len(options) > 0 { + return o, fmt.Errorf("unsupported mount options: %v", options) + } + + return o, nil +} diff --git a/pkg/sentry/fs/gofer/gofer_test.go b/pkg/sentry/fs/gofer/gofer_test.go new file mode 100644 index 000000000..58a2e2ef5 --- /dev/null +++ b/pkg/sentry/fs/gofer/gofer_test.go @@ -0,0 +1,776 @@ +// 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 gofer + +import ( + "errors" + "fmt" + "io" + "syscall" + "testing" + + "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" +) + +// A errMock is an error that comes from bad usage of the mock. +var errMock = errors.New("mock error") + +// 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{ + Valid: p9.AttrMask{Mode: true, Size: true, RDev: true}, + Attr: p9.Attr{Mode: mode, Size: size, RDev: 0}, + }, + } +} + +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, 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: cacheNone, + } + + rootFile := goodMockFile(mode, size) + sattr, rootInodeOperations := newInodeOperations(ctx, s, contextFile{file: rootFile}, p9.QID{}, rootFile.GetAttrMock.Valid, rootFile.GetAttrMock.Attr) + m := fs.NewMountSource(s, &filesystem{}, fs.MountSourceFlags{}) + return rootFile, fs.NewInode(rootInodeOperations, m, sattr), nil +} + +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 fails (function fails)", + fileName: "ppp", + want: syscall.ENOENT, + }, + } + + ctx := contexttest.Context(t) + for _, test := range tests { + // Set up mock. + rootFile, rootInode, err := root(ctx, p9.PermissionsMask, 0) + if err != nil { + t.Errorf("TestWalk %s failed: root error got %v, want nil", test.name, err) + } + + 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) + + // Unwrap the InodeOperations. + var newInodeOperations fs.InodeOperations + if dirent != nil { + if dirent.IsNegative() { + err = syscall.ENOENT + } else { + newInodeOperations = dirent.Inode.InodeOperations + } + } + + // Check return values. + if err != test.want { + t.Errorf("TestWalk %s failed: got %v, want %v", test.name, err, test.want) + } + if err == nil && newInodeOperations == nil { + t.Errorf("TestWalk %s failed: expected either non-nil err or non-nil node, but both are nil", test.name) + } + + // Check mock parameters. + if !rootFile.WalkGetAttrMock.Called { + t.Errorf("TestWalk %s failed: GetAttr not called; error: %v", test.name, err) + } else if rootFile.WalkGetAttrMock.Names[0] != test.fileName { + t.Errorf("TestWalk %s failed: file name not set", test.name) + } + } +} + +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 { + // Set up mock. + rootFile, rootInode, err := root(ctx, p9.PermissionsMask, 0) + if err != nil { + t.Errorf("TestSetTimestamps %s failed: root error got %v, want nil", test.name, err) + } + + // Call function. + err = rootInode.SetTimestamps(ctx, nil /* Dirent */, test.ts) + + // Check return values. + if err != nil { + t.Errorf("TestSetTimestamps %s failed: got %v, want nil", test.name, err) + } + + // Check mock parameters. + if !(test.ts.ATimeOmit && test.ts.MTimeOmit) && !rootFile.SetAttrMock.Called { + t.Errorf("TestSetTimestamps %s failed: SetAttr not called", test.name) + continue + } + + // 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("TestSetTimestamps %s failed: ATime got set true in mask, wanted false", test.name) + } + } else { + if got, want := rootFile.SetAttrMock.Valid.ATimeNotSystemTime, !test.ts.ATimeSetSystemTime; got != want { + t.Errorf("TestSetTimestamps %s failed: got ATimeNotSystemTime %v, want %v", test.name, got, want) + } + if !test.ts.ATimeSetSystemTime && !test.ts.ATime.Equal(atimeGiven) { + t.Errorf("TestSetTimestamps %s failed: ATime got %v, want %v", test.name, atimeGiven, test.ts.ATime) + } + } + + mtimeGiven := ktime.FromUnix(int64(attr.MTimeSeconds), int64(attr.MTimeNanoSeconds)) + if test.ts.MTimeOmit { + if rootFile.SetAttrMock.Valid.MTime { + t.Errorf("TestSetTimestamps %s failed: MTime got set true in mask, wanted false", test.name) + } + } else { + if got, want := rootFile.SetAttrMock.Valid.MTimeNotSystemTime, !test.ts.MTimeSetSystemTime; got != want { + t.Errorf("TestSetTimestamps %s failed: got MTimeNotSystemTime %v, want %v", test.name, got, want) + } + if !test.ts.MTimeSetSystemTime && !test.ts.MTime.Equal(mtimeGiven) { + t.Errorf("TestSetTimestamps %s failed: MTime got %v, want %v", test.name, 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 { + // Set up mock. + rootFile, rootInode, err := root(ctx, 0, 0) + if err != nil { + t.Errorf("TestSetPermissions %s failed: root error got %v, want nil", test.name, err) + } + rootFile.SetAttrMock.Err = test.setAttrErr + + ok := rootInode.SetPermissions(ctx, nil /* Dirent */, test.perms) + + // Check return value. + if ok != test.want { + t.Errorf("TestSetPermissions %s failed: got %v, want %v", test.name, ok, test.want) + } + + // Check mock parameters. + pattr := rootFile.SetAttrMock.Attr + if !rootFile.SetAttrMock.Called { + t.Errorf("TestSetPermissions %s failed: SetAttr not called", test.name) + continue + } + if !rootFile.SetAttrMock.Valid.Permissions { + t.Errorf("TestSetPermissions %s failed: SetAttr did not get right request (got false, expected SetAttrMask.Permissions true)", + test.name) + } + if got := fs.FilePermsFromP9(pattr.Permissions); got != test.perms { + t.Errorf("TestSetPermissions %s failed: SetAttr did not get right permissions -- got %v, want %v", + test.name, got, test.perms) + } + } +} + +func TestClose(t *testing.T) { + ctx := contexttest.Context(t) + // Set up mock. + rootFile, rootInode, err := root(ctx, p9.PermissionsMask, 0) + if err != nil { + t.Errorf("TestClose failed: root error got %v, want nil", 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, p9.PermissionsMask, 0) + if err != nil { + t.Errorf("TestRename failed: root error got %v, want nil", 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 { + 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("TestRename %s failed: got %v, want %v", test.name, err, test.want) + } + + // Check mock parameters. + if got, want := mockFile.RenameMock.Called, test.renameCalled; got != want { + t.Errorf("TestRename %s failed: renameCalled got %v want %v", test.name, 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 { + // Set up mock. + rootFile, rootInode, err := root(ctx, test.mode, 1024) + if err != nil { + t.Errorf("TestPreadv %s failed: root error got %v, want nil", test.name, 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("TestPreadv %s failed: got %v, want %v", test.name, err, test.want) + } + + // Check mock parameters. + if test.readAtCalled != openFile.Called { + t.Errorf("TestPreadv %s failed: ReadAt called: %v, but expected opposite", test.name, 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 { + // Set up mock. + rootFile, rootInode, err := root(ctx, test.mode, 0) + if err != nil { + t.Errorf("TestReadlink %s failed: root error got %v, want nil", test.name, 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("TestReadlink %s failed: got %v, want %v", test.name, err, test.want) + } + + // Check mock parameters. + if test.readlinkCalled && !rootFile.ReadlinkMock.Called { + t.Errorf("TestReadlink %s failed: Readlink not called", test.name) + } + } +} + +// 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 { + // Set up mock. + _, rootInode, err := root(ctx, test.mode, 0) + if err != nil { + t.Errorf("TestPwritev %s failed: root error got %v, want nil", test.name, 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("TestPwritev %s failed: got %v, want %v", test.name, err, test.want) + } + + // Check mock parameters. + if test.writeAtCalled != openFile.Called { + t.Errorf("TestPwritev %s failed: WriteAt called: %v, but expected opposite", test.name, openFile.Called) + continue + } + if openFile.Called && test.writeAtErr != nil && openFile.LengthWritten != test.sliceSize { + t.Errorf("TestPwritev %s failed: wrote %d bytes, expected %d bytes written", test.name, openFile.LengthWritten, test.sliceSize) + } + } +} diff --git a/pkg/sentry/fs/gofer/handles.go b/pkg/sentry/fs/gofer/handles.go new file mode 100644 index 000000000..a660c9230 --- /dev/null +++ b/pkg/sentry/fs/gofer/handles.go @@ -0,0 +1,144 @@ +// 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 gofer + +import ( + "io" + + "gvisor.googlesource.com/gvisor/pkg/fd" + "gvisor.googlesource.com/gvisor/pkg/log" + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/refs" + "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" +) + +// handles are the open handles of a gofer file. They are reference counted to +// support open handle sharing between files for read only filesystems. +// +// If Host != nil then it will be used exclusively over File. +type handles struct { + refs.AtomicRefCount + + // File is a p9.File handle. Must not be nil. + File contextFile + + // Host is an *fd.FD handle. May be nil. + Host *fd.FD +} + +// DecRef drops a reference on handles. +func (h *handles) DecRef() { + h.DecRefWithDestructor(func() { + if h.Host != nil { + if err := h.Host.Close(); err != nil { + log.Warningf("error closing host file: %v", err) + } + } + // FIXME: Context is not plumbed here. + if err := h.File.close(context.Background()); err != nil { + log.Warningf("error closing p9 file: %v", err) + } + }) +} + +func newHandles(ctx context.Context, file contextFile, flags fs.FileFlags) (*handles, error) { + _, newFile, err := file.walk(ctx, nil) + if err != nil { + return nil, err + } + + switch { + case flags.Read && flags.Write: + hostFile, _, _, err := newFile.open(ctx, p9.ReadWrite) + if err != nil { + newFile.close(ctx) + return nil, err + } + h := &handles{ + File: newFile, + Host: hostFile, + } + return h, nil + case flags.Read && !flags.Write: + hostFile, _, _, err := newFile.open(ctx, p9.ReadOnly) + if err != nil { + newFile.close(ctx) + return nil, err + } + h := &handles{ + File: newFile, + Host: hostFile, + } + return h, nil + case !flags.Read && flags.Write: + hostFile, _, _, err := newFile.open(ctx, p9.WriteOnly) + if err != nil { + newFile.close(ctx) + return nil, err + } + h := &handles{ + File: newFile, + Host: hostFile, + } + return h, nil + default: + panic("impossible fs.FileFlags") + } +} + +type handleReadWriter struct { + ctx context.Context + h *handles + off int64 +} + +func (h *handles) readWriterAt(ctx context.Context, offset int64) *handleReadWriter { + return &handleReadWriter{ctx, h, offset} +} + +// ReadToBlocks implements safemem.Reader.ReadToBlocks. +func (rw *handleReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error) { + var r io.Reader + if rw.h.Host != nil { + r = secio.NewOffsetReader(rw.h.Host, rw.off) + } else { + r = &p9.ReadWriterFile{File: rw.h.File.file, Offset: uint64(rw.off)} + } + + rw.ctx.UninterruptibleSleepStart(false) + defer rw.ctx.UninterruptibleSleepFinish(false) + n, err := safemem.FromIOReader{r}.ReadToBlocks(dsts) + rw.off += int64(n) + return n, err +} + +// WriteFromBlocks implements safemem.Writer.WriteFromBlocks. +func (rw *handleReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) { + var w io.Writer + if rw.h.Host != nil { + w = secio.NewOffsetWriter(rw.h.Host, rw.off) + } else { + w = &p9.ReadWriterFile{File: rw.h.File.file, Offset: uint64(rw.off)} + } + + rw.ctx.UninterruptibleSleepStart(false) + defer rw.ctx.UninterruptibleSleepFinish(false) + n, err := safemem.FromIOWriter{w}.WriteFromBlocks(srcs) + rw.off += int64(n) + return n, err +} diff --git a/pkg/sentry/fs/gofer/inode.go b/pkg/sentry/fs/gofer/inode.go new file mode 100644 index 000000000..454242923 --- /dev/null +++ b/pkg/sentry/fs/gofer/inode.go @@ -0,0 +1,554 @@ +// 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 gofer + +import ( + "errors" + "sync" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/fd" + "gvisor.googlesource.com/gvisor/pkg/log" + "gvisor.googlesource.com/gvisor/pkg/p9" + "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/fs/fdpipe" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/host" + "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" + "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" + "gvisor.googlesource.com/gvisor/pkg/syserror" +) + +// inodeOperations implements fs.InodeOperations. +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 + // and cachingInodeOps (below). + fileState *inodeFileState `state:"wait"` + + // cachingInodeOps implement memmap.Mappable for inodeOperations. + cachingInodeOps *fsutil.CachingInodeOperations + + // readdirMu protects readdirCache and concurrent Readdirs. + readdirMu sync.Mutex `state:"nosave"` + + // readdirCache is a cache of readdir results in the form of + // a fs.SortedDentryMap. + // + // Starts out as nil, and is initialized under readdirMu lazily; + // invalidating the cache means setting it to nil. + readdirCache *fs.SortedDentryMap `state:"nosave"` +} + +// inodeFileState implements fs.CachedFileObject and otherwise fully +// encapsulates state that needs to be manually loaded on restore for +// this file object. +// +// This unfortunate structure exists because fs.CachingInodeOperations +// defines afterLoad and therefore cannot be lazily loaded (to break a +// circular load dependency between it and inodeOperations). Even with +// lazy loading, this approach defines the dependencies between objects +// and the expected load behavior more concretely. +type inodeFileState struct { + // s is common file system state for Gofers. + s *session `state:"wait"` + + // MultiDeviceKey consists of: + // + // * Device: file system device from a specific gofer. + // * SecondaryDevice: unique identifier of the attach point. + // * Inode: the inode of this resource, unique per Device.= + // + // These fields combined enable consistent hashing of virtual inodes + // on goferDevice. + key device.MultiDeviceKey `state:"nosave"` + + // file is the p9 file that contains a single unopened fid. + file contextFile `state:"nosave"` + + // sattr caches the stable attributes. + sattr fs.StableAttr `state:"wait"` + + // handlesMu protects the below fields. + handlesMu sync.RWMutex `state:"nosave"` + + // Do minimal open handle caching: only for read only filesystems. + readonly *handles `state:"nosave"` + + // Maintain readthrough handles for populating page caches. + readthrough *handles `state:"nosave"` + + // Maintain writeback handles for syncing from page caches. + writeback *handles `state:"nosave"` + + // writebackRW indicates whether writeback is opened read-write. If + // it is not and a read-write handle could replace writeback (above), + // then writeback is replaced with the read-write handle. This + // ensures that files that were first opened write-only and then + // later are opened read-write to be mapped can in fact be mapped. + writebackRW bool + + // loading is acquired when the inodeFileState begins an asynchronous + // load. It releases when the load is complete. Callers that require all + // state to be available should call waitForLoad() to ensure that. + loading sync.Mutex `state:".(struct{})"` + + // savedUAttr is only allocated during S/R. It points to the save-time + // unstable attributes and is used to validate restore-time ones. + // + // Note that these unstable attributes are only used to detect cross-S/R + // external file system metadata changes. They may differ from the + // cached unstable attributes in cachingInodeOps, as that might differ + // from the external file system attributes if there had been WriteOut + // failures. S/R is transparent to Sentry and the latter will continue + // using its cached values after restore. + savedUAttr *fs.UnstableAttr +} + +// Release releases file handles. +func (i *inodeFileState) Release(ctx context.Context) { + i.file.close(ctx) + if i.readonly != nil { + i.readonly.DecRef() + } + if i.readthrough != nil { + i.readthrough.DecRef() + } + if i.writeback != nil { + i.writeback.DecRef() + } +} + +// setHandlesForCachedIO installs file handles for reading and writing +// through fs.CachingInodeOperations. +func (i *inodeFileState) setHandlesForCachedIO(flags fs.FileFlags, h *handles) { + i.handlesMu.Lock() + defer i.handlesMu.Unlock() + + if flags.Read { + if i.readthrough == nil { + h.IncRef() + i.readthrough = h + } + } + if flags.Write { + if i.writeback == nil { + h.IncRef() + i.writeback = h + } else if !i.writebackRW && flags.Read { + i.writeback.DecRef() + h.IncRef() + i.writeback = h + } + if flags.Read { + i.writebackRW = true + } + } +} + +// getCachedHandles returns any cached handles which would accelerate +// performance generally. These handles should only be used if the mount +// supports caching. This is distinct from fs.CachingInodeOperations +// which is used for a limited set of file types (those that can be mapped). +func (i *inodeFileState) getCachedHandles(ctx context.Context, flags fs.FileFlags, msrc *fs.MountSource) (*handles, bool) { + i.handlesMu.Lock() + defer i.handlesMu.Unlock() + + if flags.Read && !flags.Write && msrc.Flags.ReadOnly { + if i.readonly != nil { + i.readonly.IncRef() + return i.readonly, true + } + h, err := newHandles(ctx, i.file, flags) + if err != nil { + return nil, false + } + i.readonly = h + i.readonly.IncRef() + return i.readonly, true + } + + return nil, false +} + +// ReadToBlocksAt implements fsutil.CachedFileObject.ReadToBlocksAt. +func (i *inodeFileState) ReadToBlocksAt(ctx context.Context, dsts safemem.BlockSeq, offset uint64) (uint64, error) { + i.handlesMu.RLock() + defer i.handlesMu.RUnlock() + return i.readthrough.readWriterAt(ctx, int64(offset)).ReadToBlocks(dsts) +} + +// WriteFromBlocksAt implements fsutil.CachedFileObject.WriteFromBlocksAt. +func (i *inodeFileState) WriteFromBlocksAt(ctx context.Context, srcs safemem.BlockSeq, offset uint64) (uint64, error) { + i.handlesMu.RLock() + defer i.handlesMu.RUnlock() + return i.writeback.readWriterAt(ctx, int64(offset)).WriteFromBlocks(srcs) +} + +// SetMaskedAttributes implements fsutil.CachedFileObject.SetMaskedAttributes. +func (i *inodeFileState) SetMaskedAttributes(ctx context.Context, mask fs.AttrMask, attr fs.UnstableAttr) error { + if mask.Empty() { + return nil + } + as, ans := attr.AccessTime.Unix() + ms, mns := attr.ModificationTime.Unix() + // An update of status change time is implied by mask.AccessTime + // or mask.ModificationTime. Updating status change time to a + // time earlier than the system time is not possible. + return i.file.setAttr( + ctx, + p9.SetAttrMask{ + Permissions: mask.Perms, + Size: mask.Size, + UID: mask.UID, + GID: mask.GID, + ATime: mask.AccessTime, + ATimeNotSystemTime: true, + MTime: mask.ModificationTime, + MTimeNotSystemTime: true, + }, p9.SetAttr{ + Permissions: p9.FileMode(attr.Perms.LinuxMode()), + UID: p9.UID(attr.Owner.UID), + GID: p9.GID(attr.Owner.GID), + Size: uint64(attr.Size), + ATimeSeconds: uint64(as), + ATimeNanoSeconds: uint64(ans), + MTimeSeconds: uint64(ms), + MTimeNanoSeconds: uint64(mns), + }) +} + +// Sync implements fsutil.CachedFileObject.Sync. +func (i *inodeFileState) Sync(ctx context.Context) error { + i.handlesMu.RLock() + defer i.handlesMu.RUnlock() + if i.writeback == nil { + return nil + } + return i.writeback.File.fsync(ctx) +} + +// FD implements fsutil.CachedFileObject.FD. +// +// FD meets the requirements of fsutil.CachedFileObject.FD because p9.File.Open +// returns a host file descriptor to back _both_ readthrough and writeback or +// not at all (e.g. both are nil). +func (i *inodeFileState) FD() int { + i.handlesMu.RLock() + defer i.handlesMu.RUnlock() + + // Assert that the file was actually opened. + if i.writeback == nil && i.readthrough == nil { + panic("cannot get host FD for a file that was never opened") + } + // If this file is mapped, then it must have been opened + // read-write and i.writeback was upgraded to a read-write + // handle. Prefer that to map. + if i.writeback != nil { + if i.writeback.Host == nil { + return -1 + } + return int(i.writeback.Host.FD()) + } + // Otherwise the file may only have been opened readable + // so far. That's the only way it can be accessed. + if i.readthrough.Host == nil { + return -1 + } + return int(i.readthrough.Host.FD()) +} + +// waitForLoad makes sure any restore-issued loading is done. +func (i *inodeFileState) waitForLoad() { + // This is not a no-op. The loading mutex is hold upon restore until + // all loading actions are done. + i.loading.Lock() + i.loading.Unlock() +} + +func (i *inodeFileState) unstableAttr(ctx context.Context) (fs.UnstableAttr, error) { + _, valid, pattr, err := getattr(ctx, i.file) + if err != nil { + return fs.UnstableAttr{}, err + } + return unstable(ctx, valid, pattr, i.s.mounter, i.s.client), nil +} + +// session extracts the gofer's session from the MountSource. +func (i *inodeOperations) session() *session { + return i.fileState.s +} + +// Release implements fs.InodeOperations.Release. +func (i *inodeOperations) Release(ctx context.Context) { + i.fileState.Release(ctx) + i.cachingInodeOps.Release() +} + +// Mappable implements fs.InodeOperations.Mappable. +func (i *inodeOperations) Mappable(inode *fs.Inode) memmap.Mappable { + if i.session().cachePolicy == cacheNone || !fs.IsFile(inode.StableAttr) { + return nil + } + return i.cachingInodeOps +} + +func isCachable(session *session, inode *fs.Inode) bool { + return session.cachePolicy != cacheNone && (fs.IsFile(inode.StableAttr) || fs.IsDir(inode.StableAttr)) +} + +func isFileCachable(session *session, inode *fs.Inode) bool { + return session.cachePolicy != cacheNone && fs.IsFile(inode.StableAttr) +} + +// UnstableAttr implements fs.InodeOperations.UnstableAttr. +func (i *inodeOperations) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) { + if isCachable(i.session(), inode) { + return i.cachingInodeOps.UnstableAttr(ctx, inode) + } + return i.fileState.unstableAttr(ctx) +} + +// 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) +} + +// GetFile implements fs.InodeOperations.GetFile. +func (i *inodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + switch d.Inode.StableAttr.Type { + case fs.Socket: + return i.getFileSocket(ctx, d, flags) + case fs.Pipe: + return i.getFilePipe(ctx, d, flags) + default: + return i.getFileDefault(ctx, d, flags) + } +} + +func (i *inodeOperations) getFileSocket(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + f, err := i.fileState.file.connect(ctx, p9.AnonymousSocket) + if err != nil { + return nil, syscall.EIO + } + fsf, err := host.NewSocketWithDirent(ctx, d, f, flags) + if err != nil { + f.Close() + return nil, err + } + return fsf, nil +} + +func (i *inodeOperations) getFilePipe(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + // Try to open as a host pipe. + if pipeOps, err := fdpipe.Open(ctx, i, flags); err != errNotHostFile { + return fs.NewFile(ctx, d, flags, pipeOps), err + } + + // If the error is due to the fact that this was never a host pipe, then back + // this file with its dirent. + h, err := newHandles(ctx, i.fileState.file, flags) + if err != nil { + return nil, err + } + return NewFile(ctx, d, flags, i, h), nil +} + +// errNotHostFile indicates that the file is not a host file. +var errNotHostFile = errors.New("not a host file") + +// NonBlockingOpen implements fdpipe.NonBlockingOpener for opening host named pipes. +func (i *inodeOperations) NonBlockingOpen(ctx context.Context, p fs.PermMask) (*fd.FD, error) { + i.fileState.waitForLoad() + + // Get a cloned fid which we will open. + _, newFile, err := i.fileState.file.walk(ctx, nil) + if err != nil { + log.Warningf("Open Walk failed: %v", err) + return nil, err + } + defer newFile.close(ctx) + + flags, err := openFlagsFromPerms(p) + if err != nil { + log.Warningf("Open flags %s parsing failed: %v", p, err) + return nil, err + } + hostFile, _, _, err := newFile.open(ctx, flags) + // If the host file returned is nil and the error is nil, + // then this was never a host file to begin with, and should + // be treated like a remote file. + if hostFile == nil && err == nil { + return nil, errNotHostFile + } + return hostFile, err +} + +func (i *inodeOperations) getFileDefault(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { + if !isFileCachable(i.session(), d.Inode) { + h, err := newHandles(ctx, i.fileState.file, flags) + if err != nil { + return nil, err + } + return NewFile(ctx, d, flags, i, h), nil + } + + h, ok := i.fileState.getCachedHandles(ctx, flags, d.Inode.MountSource) + if !ok { + var err error + h, err = newHandles(ctx, i.fileState.file, flags) + if err != nil { + return nil, err + } + } + i.fileState.setHandlesForCachedIO(flags, h) + + return NewFile(ctx, d, flags, i, h), nil +} + +// SetPermissions implements fs.InodeOperations.SetPermissions. +func (i *inodeOperations) SetPermissions(ctx context.Context, inode *fs.Inode, p fs.FilePermissions) bool { + if isCachable(i.session(), inode) { + return i.cachingInodeOps.SetPermissions(ctx, inode, p) + } + + mask := p9.SetAttrMask{Permissions: true} + pattr := p9.SetAttr{Permissions: p9.FileMode(p.LinuxMode())} + // Execute the chmod. + return i.fileState.file.setAttr(ctx, mask, pattr) == nil +} + +// SetOwner implements fs.InodeOperations.SetOwner. +func (i *inodeOperations) SetOwner(ctx context.Context, inode *fs.Inode, owner fs.FileOwner) error { + // Save the roundtrip. + if !owner.UID.Ok() && !owner.GID.Ok() { + return nil + } + + if isCachable(i.session(), inode) { + return i.cachingInodeOps.SetOwner(ctx, inode, owner) + } + + var mask p9.SetAttrMask + var attr p9.SetAttr + if owner.UID.Ok() { + mask.UID = true + attr.UID = p9.UID(owner.UID) + } + if owner.GID.Ok() { + mask.GID = true + attr.GID = p9.GID(owner.GID) + } + return i.fileState.file.setAttr(ctx, mask, attr) +} + +// SetTimestamps implements fs.InodeOperations.SetTimestamps. +func (i *inodeOperations) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error { + if isCachable(i.session(), inode) { + return i.cachingInodeOps.SetTimestamps(ctx, inode, ts) + } + + return utimes(ctx, i.fileState.file, ts) +} + +// Truncate implements fs.InodeOperations.Truncate. +func (i *inodeOperations) Truncate(ctx context.Context, inode *fs.Inode, length int64) error { + // This can only be called for files anyway. + if isFileCachable(i.session(), inode) { + return i.cachingInodeOps.Truncate(ctx, inode, length) + } + + return i.fileState.file.setAttr(ctx, p9.SetAttrMask{Size: true}, p9.SetAttr{Size: uint64(length)}) +} + +// WriteOut implements fs.InodeOperations.WriteOut. +func (i *inodeOperations) WriteOut(ctx context.Context, inode *fs.Inode) error { + if !isCachable(i.session(), inode) { + return nil + } + + return i.cachingInodeOps.WriteOut(ctx, inode) +} + +// Readlink implements fs.InodeOperations.Readlink. +func (i *inodeOperations) Readlink(ctx context.Context, inode *fs.Inode) (string, error) { + if !fs.IsSymlink(inode.StableAttr) { + return "", syscall.ENOLINK + } + return i.fileState.file.readlink(ctx) +} + +// Getlink implementfs fs.InodeOperations.Getlink. +func (i *inodeOperations) Getlink(context.Context, *fs.Inode) (*fs.Dirent, error) { + if !fs.IsSymlink(i.fileState.sattr) { + return nil, syserror.ENOLINK + } + return nil, fs.ErrResolveViaReadlink +} + +// StatFS makes a StatFS request. +func (i *inodeOperations) StatFS(ctx context.Context) (fs.Info, error) { + fsstat, err := i.fileState.file.statFS(ctx) + if err != nil { + return fs.Info{}, err + } + + info := fs.Info{ + // This is primarily for distinguishing a gofer file system in + // tests. Testing is important, so instead of defining + // something completely random, use a standard value. + Type: linux.V9FS_MAGIC, + TotalBlocks: fsstat.Blocks, + FreeBlocks: fsstat.BlocksFree, + TotalFiles: fsstat.Files, + FreeFiles: fsstat.FilesFree, + } + + // If blocks available is non-zero, prefer that. + if fsstat.BlocksAvailable != 0 { + info.FreeBlocks = fsstat.BlocksAvailable + } + + return info, nil +} + +func init() { + syserror.AddErrorUnwrapper(func(err error) (syscall.Errno, bool) { + if _, ok := err.(p9.ErrSocket); ok { + // Treat as an I/O error. + return syscall.EIO, true + } + return 0, false + }) +} + +// AddLink implements InodeOperations.AddLink, but is currently a noop. +// FIXME: Remove this from InodeOperations altogether. +func (*inodeOperations) AddLink() {} + +// DropLink implements InodeOperations.DropLink, but is currently a noop. +// FIXME: Remove this from InodeOperations altogether. +func (*inodeOperations) DropLink() {} + +// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange. +// FIXME: 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 new file mode 100644 index 000000000..997a7d1c1 --- /dev/null +++ b/pkg/sentry/fs/gofer/inode_state.go @@ -0,0 +1,141 @@ +// 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 gofer + +import ( + "fmt" + "strings" + + "gvisor.googlesource.com/gvisor/pkg/p9" + "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/kernel/time" +) + +// Some fs implementations may not support atime, ctime, or mtime in getattr. +// The unstable() logic would try to use clock time for them. However, we do not +// want to use such time during S/R as that would cause restore timestamp +// checking failure. Hence a dummy stable-time clock is needed. +// +// Note that application-visible UnstableAttrs either come from CachingInodeOps +// (in which case they are saved), or they are requested from the gofer on each +// stat (for non-caching), so the dummy time only affects the modification +// timestamp check. +type dummyClock struct { + time.Clock +} + +// Now returns a stable dummy time. +func (d *dummyClock) Now() time.Time { + return time.Time{} +} + +type dummyClockContext struct { + context.Context +} + +// Value implements context.Context +func (d *dummyClockContext) Value(key interface{}) interface{} { + switch key { + case time.CtxRealtimeClock: + return &dummyClock{} + default: + return d.Context.Value(key) + } +} + +// beforeSave is invoked by stateify. +func (i *inodeFileState) beforeSave() { + if _, ok := i.s.inodeMappings[i.sattr.InodeID]; !ok { + panic(fmt.Sprintf("failed to find path for inode number %d. Device %s contains %s", i.sattr.InodeID, i.s.connID, fs.InodeMappings(i.s.inodeMappings))) + } + if i.sattr.Type == fs.RegularFile { + uattr, err := i.unstableAttr(&dummyClockContext{context.Background()}) + if err != nil { + panic(fmt.Sprintf("failed to get unstable atttribute of %s: %v", i.s.inodeMappings[i.sattr.InodeID], err)) + } + i.savedUAttr = &uattr + } +} + +// saveLoading is invoked by stateify. +func (i *inodeFileState) saveLoading() struct{} { + return struct{}{} +} + +// loadLoading is invoked by stateify. +func (i *inodeFileState) loadLoading(_ struct{}) { + i.loading.Lock() +} + +// afterLoad is invoked by stateify. +func (i *inodeFileState) afterLoad() { + load := func() { + // See comment on i.loading(). + defer i.loading.Unlock() + + // Manually restore the p9.File. + name, ok := i.s.inodeMappings[i.sattr.InodeID] + if !ok { + // This should be impossible, see assertion in + // beforeSave. + panic(fmt.Sprintf("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. + ctx := &dummyClockContext{context.Background()} + var err error + _, i.file, err = i.s.attach.walk(ctx, strings.Split(name, "/")) + if err != nil { + panic(fmt.Sprintf("failed to walk to %q: %v", name, err)) + } + + // Remap the saved inode number into the gofer device using the + // actual device and actual inode that exists in our new + // environment. + qid, mask, attrs, err := i.file.getAttr(ctx, p9.AttrMaskAll()) + if err != nil { + panic(fmt.Sprintf("failed to get file attributes of %s: %v", name, err)) + } + if !mask.RDev { + panic(fmt.Sprintf("file %s lacks device", name)) + } + i.key = device.MultiDeviceKey{ + Device: attrs.RDev, + SecondaryDevice: i.s.connID, + Inode: qid.Path, + } + if !goferDevice.Load(i.key, i.sattr.InodeID) { + panic(fmt.Sprintf("gofer device %s -> %d conflict in gofer device mappings: %s", i.key, i.sattr.InodeID, goferDevice)) + } + + if i.sattr.Type == fs.RegularFile { + env, ok := fs.CurrentRestoreEnvironment() + if !ok { + panic("missing restore environment") + } + uattr := unstable(ctx, mask, attrs, i.s.mounter, i.s.client) + if env.ValidateFileSize && uattr.Size != i.savedUAttr.Size { + panic(fmt.Errorf("file size has changed for %s: previously %d, now %d", i.s.inodeMappings[i.sattr.InodeID], i.savedUAttr.Size, uattr.Size)) + } + if env.ValidateFileTimestamp && uattr.ModificationTime != i.savedUAttr.ModificationTime { + panic(fmt.Errorf("file modification time has changed for %s: previously %v, now %v", i.s.inodeMappings[i.sattr.InodeID], i.savedUAttr.ModificationTime, uattr.ModificationTime)) + } + i.savedUAttr = nil + } + } + + fs.Async(load) +} diff --git a/pkg/sentry/fs/gofer/path.go b/pkg/sentry/fs/gofer/path.go new file mode 100644 index 000000000..d696f1561 --- /dev/null +++ b/pkg/sentry/fs/gofer/path.go @@ -0,0 +1,331 @@ +// 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 gofer + +import ( + "fmt" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/p9" + "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/syserror" + "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" +) + +// Lookup loads an Inode at name into a Dirent based on the session's cache +// policy. +func (i *inodeOperations) Lookup(ctx context.Context, dir *fs.Inode, name string) (*fs.Dirent, error) { + if i.session().cachePolicy != cacheNone { + // Check to see if we have readdirCache that indicates the + // child does not exist. Avoid holding readdirMu longer than + // we need to. + i.readdirMu.Lock() + if i.readdirCache != nil && !i.readdirCache.Contains(name) { + // No such child. Return a negative dirent. + i.readdirMu.Unlock() + return fs.NewNegativeDirent(name), nil + } + i.readdirMu.Unlock() + } + + // Get a p9.File for name. + qids, newFile, mask, p9attr, err := i.fileState.file.walkGetAttr(ctx, []string{name}) + if err != nil { + if err == syscall.ENOENT { + if i.session().cachePolicy != cacheNone { + // Return a negative Dirent. It will stay cached until something + // is created over it. + return fs.NewNegativeDirent(name), nil + } + return nil, syserror.ENOENT + } + return nil, err + } + + // Construct the Inode operations. + sattr, node := newInodeOperations(ctx, i.fileState.s, newFile, qids[0], mask, p9attr) + + // Construct a positive Dirent. + return fs.NewDirent(fs.NewInode(node, dir.MountSource, sattr), name), nil +} + +// Creates a new Inode at name and returns its File based on the session's cache policy. +// +// Ownership is currently ignored. +func (i *inodeOperations) Create(ctx context.Context, dir *fs.Inode, name string, flags fs.FileFlags, perm fs.FilePermissions) (*fs.File, error) { + // Create replaces the directory fid with the newly created/opened + // file, so clone this directory so it doesn't change out from under + // this node. + _, newFile, err := i.fileState.file.walk(ctx, nil) + if err != nil { + return nil, err + } + + // Map the FileFlags to p9 OpenFlags. + var openFlags p9.OpenFlags + switch { + case flags.Read && flags.Write: + openFlags = p9.ReadWrite + case flags.Read: + openFlags = p9.ReadOnly + case flags.Write: + openFlags = p9.WriteOnly + default: + panic(fmt.Sprintf("Create called with unknown or unset open flags: %v", flags)) + } + + owner := fs.FileOwnerFromContext(ctx) + hostFile, err := newFile.create(ctx, name, openFlags, p9.FileMode(perm.LinuxMode()), p9.UID(owner.UID), p9.GID(owner.GID)) + if err != nil { + // Could not create the file. + return nil, err + } + + i.touchModificationTime(ctx) + + // Get the attributes of the file. + qid, mask, p9attr, err := getattr(ctx, newFile) + if err != nil { + newFile.close(ctx) + return nil, err + } + + // Get an unopened p9.File for the file we created so that it can be + // cloned and re-opened multiple times after creation. + _, unopened, err := i.fileState.file.walk(ctx, []string{name}) + if err != nil { + newFile.close(ctx) + return nil, err + } + + // Construct the InodeOperations. + sattr, iops := newInodeOperations(ctx, i.fileState.s, unopened, qid, mask, p9attr) + + // Construct the positive Dirent. + d := fs.NewDirent(fs.NewInode(iops, dir.MountSource, sattr), name) + defer d.DecRef() + + // Construct the new file, caching the handles if allowed. + h := &handles{ + File: newFile, + Host: hostFile, + } + if isFileCachable(iops.session(), d.Inode) { + iops.fileState.setHandlesForCachedIO(flags, h) + } + return NewFile(ctx, d, flags, iops, h), nil +} + +// CreateLink uses Create to create a symlink between oldname and newname. +func (i *inodeOperations) CreateLink(ctx context.Context, dir *fs.Inode, oldname string, newname string) error { + owner := fs.FileOwnerFromContext(ctx) + if _, err := i.fileState.file.symlink(ctx, oldname, newname, p9.UID(owner.UID), p9.GID(owner.GID)); err != nil { + return err + } + i.touchModificationTime(ctx) + return nil +} + +// CreateHardLink implements InodeOperations.CreateHardLink. +func (i *inodeOperations) CreateHardLink(ctx context.Context, _ *fs.Inode, target *fs.Inode, newName string) error { + targetOpts, ok := target.InodeOperations.(*inodeOperations) + if !ok { + return syscall.EXDEV + } + + if err := i.fileState.file.link(ctx, &targetOpts.fileState.file, newName); err != nil { + return err + } + // TODO: Don't increase link count because we can't properly accounts for links + // with gofers. + i.touchModificationTime(ctx) + return nil +} + +// CreateDirectory uses Create to create a directory named s under inodeOperations. +func (i *inodeOperations) CreateDirectory(ctx context.Context, dir *fs.Inode, s string, perm fs.FilePermissions) error { + owner := fs.FileOwnerFromContext(ctx) + if _, err := i.fileState.file.mkdir(ctx, s, p9.FileMode(perm.LinuxMode()), p9.UID(owner.UID), p9.GID(owner.GID)); err != nil { + return err + } + if i.session().cachePolicy == cacheAll { + // Increase link count. + i.cachingInodeOps.IncLinks(ctx) + + // Invalidate readdir cache. + i.markDirectoryDirty() + } + return nil +} + +// Bind implements InodeOperations. +func (i *inodeOperations) Bind(ctx context.Context, dir *fs.Inode, name string, ep unix.BoundEndpoint, perm fs.FilePermissions) error { + if i.session().endpoints == nil { + return syscall.EOPNOTSUPP + } + + // Create replaces the directory fid with the newly created/opened + // file, so clone this directory so it doesn't change out from under + // this node. + _, newFile, err := i.fileState.file.walk(ctx, nil) + if err != nil { + return err + } + + // Stabilize the endpoint map while creation is in progress. + unlock := i.session().endpoints.lock() + defer unlock() + + // Create a regular file in the gofer and then mark it as a socket by + // adding this inode key in the 'endpoints' map. + owner := fs.FileOwnerFromContext(ctx) + hostFile, err := newFile.create(ctx, name, p9.ReadWrite, p9.FileMode(perm.LinuxMode()), p9.UID(owner.UID), p9.GID(owner.GID)) + if err != nil { + return err + } + // We're not going to use this file. + hostFile.Close() + + i.touchModificationTime(ctx) + + // Get the attributes of the file to create inode key. + qid, _, attr, err := getattr(ctx, newFile) + if err != nil { + newFile.close(ctx) + return err + } + + key := device.MultiDeviceKey{ + Device: attr.RDev, + SecondaryDevice: i.session().connID, + Inode: qid.Path, + } + i.session().endpoints.add(key, ep) + + return nil +} + +// CreateFifo implements fs.InodeOperations.CreateFifo. Gofer nodes do not support the +// creation of fifos and always returns EOPNOTSUPP. +func (*inodeOperations) CreateFifo(context.Context, *fs.Inode, string, fs.FilePermissions) error { + return syscall.EOPNOTSUPP +} + +// Remove implements InodeOperations.Remove. +func (i *inodeOperations) Remove(ctx context.Context, dir *fs.Inode, name string) error { + var key device.MultiDeviceKey + removeSocket := false + if i.session().endpoints != nil { + // Find out if file being deleted is a socket that needs to be + // removed from endpoint map. + if d, err := i.Lookup(ctx, dir, name); err == nil { + defer d.DecRef() + if fs.IsSocket(d.Inode.StableAttr) { + child := d.Inode.InodeOperations.(*inodeOperations) + key = child.fileState.key + removeSocket = true + + // Stabilize the endpoint map while deletion is in progress. + unlock := i.session().endpoints.lock() + defer unlock() + } + } + } + + if err := i.fileState.file.unlinkAt(ctx, name, 0); err != nil { + return err + } + if removeSocket { + i.session().endpoints.remove(key) + } + i.touchModificationTime(ctx) + + return nil +} + +// Remove implements InodeOperations.RemoveDirectory. +func (i *inodeOperations) RemoveDirectory(ctx context.Context, dir *fs.Inode, name string) error { + // 0x200 = AT_REMOVEDIR. + if err := i.fileState.file.unlinkAt(ctx, name, 0x200); err != nil { + return err + } + if i.session().cachePolicy == cacheAll { + // Decrease link count and updates atime. + i.cachingInodeOps.DecLinks(ctx) + + // Invalidate readdir cache. + i.markDirectoryDirty() + } + return nil +} + +// Rename renames this node. +func (i *inodeOperations) Rename(ctx context.Context, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error { + // Unwrap the new parent to a *inodeOperations. + newParentInodeOperations, ok := newParent.InodeOperations.(*inodeOperations) + if !ok { + return syscall.EXDEV + } + + // Unwrap the old parent to a *inodeOperations. + oldParentInodeOperations, ok := oldParent.InodeOperations.(*inodeOperations) + if !ok { + return syscall.EXDEV + } + + // Do the rename. + if err := i.fileState.file.rename(ctx, newParentInodeOperations.fileState.file, newName); err != nil { + return err + } + + // Update cached state. + if i.session().cachePolicy == cacheAll { + // Is the renamed entity a directory? Fix link counts. + if fs.IsDir(i.fileState.sattr) { + oldParentInodeOperations.cachingInodeOps.DecLinks(ctx) + newParentInodeOperations.cachingInodeOps.IncLinks(ctx) + } + + // Mark old directory dirty. + oldParentInodeOperations.markDirectoryDirty() + if oldParent != newParent { + // Mark new directory dirty. + newParentInodeOperations.markDirectoryDirty() + } + } + return nil +} + +func (i *inodeOperations) touchModificationTime(ctx context.Context) { + if i.session().cachePolicy == cacheAll { + i.cachingInodeOps.TouchModificationTime(ctx) + + // Invalidate readdir cache. + i.markDirectoryDirty() + } +} + +// markDirectoryDirty marks any cached data dirty for this directory. This is necessary in order +// to ensure that this node does not retain stale state throughout its lifetime across multiple +// open directory handles. +// +// Currently this means invalidating any readdir caches. +func (i *inodeOperations) markDirectoryDirty() { + i.readdirMu.Lock() + defer i.readdirMu.Unlock() + i.readdirCache = nil +} diff --git a/pkg/sentry/fs/gofer/session.go b/pkg/sentry/fs/gofer/session.go new file mode 100644 index 000000000..ab3b964e0 --- /dev/null +++ b/pkg/sentry/fs/gofer/session.go @@ -0,0 +1,251 @@ +// 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 gofer + +import ( + "sync" + + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/refs" + "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/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" + "gvisor.googlesource.com/gvisor/pkg/unet" +) + +type endpointMap struct { + mu sync.RWMutex + m map[device.MultiDeviceKey]unix.BoundEndpoint +} + +// add adds the endpoint to the map. +// +// Precondition: map must have been locked with 'lock'. +func (e *endpointMap) add(key device.MultiDeviceKey, ep unix.BoundEndpoint) { + e.m[key] = ep +} + +// remove deletes the key from the map. +// +// Precondition: map must have been locked with 'lock'. +func (e *endpointMap) remove(key device.MultiDeviceKey) { + delete(e.m, key) +} + +// lock blocks other addition and removal operations from happening while +// the backing file is being created or deleted. Returns a function that unlocks +// the endpoint map. +func (e *endpointMap) lock() func() { + e.mu.Lock() + return func() { e.mu.Unlock() } +} + +func (e *endpointMap) get(key device.MultiDeviceKey) unix.BoundEndpoint { + e.mu.RLock() + ep := e.m[key] + e.mu.RUnlock() + return ep +} + +// session holds state for each 9p session established during sys_mount. +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"` + + // version is the value of the version mount option, see fs/gofer/fs.go. + version string `state:"wait"` + + // cachePolicy is the cache policy. It may be either cacheAll or cacheNone. + cachePolicy cachePolicy `state:"wait"` + + // aname is the value of the aname mount option, see fs/gofer/fs.go. + aname string `state:"wait"` + + // The client associated with this session. This will be initialized lazily. + client *p9.Client `state:"nosave"` + + // The p9.File pointing to attachName via the client. This will be initialized + // lazily. + attach contextFile `state:"nosave"` + + // Flags provided to the mount. + superBlockFlags fs.MountSourceFlags `state:"wait"` + + // connID is a unique identifier for the session connection. + connID string `state:"wait"` + + // inodeMappings contains mappings of fs.Inodes associated with this session + // to paths relative to the attach point, where inodeMappings is keyed by + // Inode.StableAttr.InodeID. + inodeMappings map[uint64]string `state:"wait"` + + // mounter is the EUID/EGID that mounted this file system. + mounter fs.FileOwner `state:"wait"` + + // endpoints is used to map inodes that represent socket files to their + // corresponding endpoint. Socket files are created as regular files in the + // gofer and their presence in this map indicate that they should indeed be + // 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 + // file and another deleting it concurrently, where the file will not be + // reported as socket file. + endpoints *endpointMap `state:"wait"` +} + +// Destroy tears down the session. +func (s *session) Destroy() { + s.conn.Close() +} + +// Revalidate returns true if the cache policy is does not allow for VFS caching. +func (s *session) Revalidate(*fs.Dirent) bool { + return s.cachePolicy == cacheNone +} + +// TakeRefs takes an extra reference on dirent if possible. +func (s *session) Keep(dirent *fs.Dirent) bool { + // NOTE: Only cache files and directories. + sattr := dirent.Inode.StableAttr + return s.cachePolicy != cacheNone && (fs.IsFile(sattr) || fs.IsDir(sattr)) +} + +// ResetInodeMappings implements fs.MountSourceOperations.ResetInodeMappings. +func (s *session) ResetInodeMappings() { + s.inodeMappings = make(map[uint64]string) +} + +// SaveInodeMapping implements fs.MountSourceOperations.SaveInodeMapping. +func (s *session) SaveInodeMapping(inode *fs.Inode, path string) { + // This is very unintuitive. We *CANNOT* trust the inode's StableAttrs, + // because overlay copyUp may have changed them out from under us. + // So much for "immutable". + sattr := inode.InodeOperations.(*inodeOperations).fileState.sattr + s.inodeMappings[sattr.InodeID] = path +} + +// newInodeOperations creates a new 9p fs.InodeOperations backed by a p9.File and attributes +// (p9.QID, p9.AttrMask, p9.Attr). +func newInodeOperations(ctx context.Context, s *session, file contextFile, qid p9.QID, valid p9.AttrMask, attr p9.Attr) (fs.StableAttr, *inodeOperations) { + deviceKey := device.MultiDeviceKey{ + Device: attr.RDev, + SecondaryDevice: s.connID, + Inode: qid.Path, + } + + sattr := fs.StableAttr{ + Type: ntype(attr), + DeviceID: goferDevice.DeviceID(), + InodeID: goferDevice.Map(deviceKey), + BlockSize: bsize(attr), + } + + if s.endpoints != nil { + // If unix sockets are allowed on this filesystem, check if this file is + // supposed to be a socket file. + if s.endpoints.get(deviceKey) != nil { + sattr.Type = fs.Socket + } + } + + fileState := &inodeFileState{ + s: s, + file: file, + sattr: sattr, + key: deviceKey, + } + + uattr := unstable(ctx, valid, attr, s.mounter, s.client) + return sattr, &inodeOperations{ + fileState: fileState, + cachingInodeOps: fsutil.NewCachingInodeOperations(ctx, fileState, uattr, s.superBlockFlags.ForcePageCache), + } +} + +// Root returns the root of a 9p mount. This mount is bound to a 9p server +// based on conn. Otherwise configuration parameters are: +// +// * dev: connection id +// * filesystem: the filesystem backing the mount +// * superBlockFlags: the mount flags describing general mount options +// * opts: parsed 9p mount options +func Root(ctx context.Context, dev string, filesystem fs.Filesystem, superBlockFlags fs.MountSourceFlags, o opts) (*fs.Inode, error) { + // The mounting EUID/EGID will be cached by this file system. This will + // be used to assign ownership to files that the Gofer owns. + mounter := fs.FileOwnerFromContext(ctx) + + conn, err := unet.NewSocket(o.fd) + if err != nil { + return nil, err + } + + // Construct the session. + s := &session{ + connID: dev, + conn: conn, + msize: o.msize, + version: o.version, + cachePolicy: o.policy, + aname: o.aname, + superBlockFlags: superBlockFlags, + mounter: mounter, + } + + if o.privateunixsocket { + s.endpoints = &endpointMap{m: make(map[device.MultiDeviceKey]unix.BoundEndpoint)} + } + + // Construct the MountSource with the session and superBlockFlags. + m := fs.NewMountSource(s, filesystem, superBlockFlags) + + // Send the Tversion request. + s.client, err = p9.NewClient(s.conn, s.msize, s.version) + if err != nil { + // Drop our reference on the session, it needs to be torn down. + s.DecRef() + return nil, err + } + + // Notify that we're about to call the Gofer and block. + ctx.UninterruptibleSleepStart(false) + // Send the Tattach request. + s.attach.file, err = s.client.Attach(s.aname) + ctx.UninterruptibleSleepFinish(false) + if err != nil { + // Same as above. + s.DecRef() + return nil, err + } + + qid, valid, attr, err := s.attach.getAttr(ctx, p9.AttrMaskAll()) + if err != nil { + s.attach.close(ctx) + // Same as above, but after we execute the Close request. + s.DecRef() + return nil, err + } + + sattr, iops := newInodeOperations(ctx, s, s.attach, qid, valid, attr) + return fs.NewInode(iops, m, sattr), nil +} diff --git a/pkg/sentry/fs/gofer/session_state.go b/pkg/sentry/fs/gofer/session_state.go new file mode 100644 index 000000000..4d993a219 --- /dev/null +++ b/pkg/sentry/fs/gofer/session_state.go @@ -0,0 +1,90 @@ +// 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 gofer + +import ( + "fmt" + + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/unet" +) + +// afterLoad is invoked by stateify. +func (s *session) afterLoad() { + // The restore environment contains the 9p connection of this mount. + fsys := filesystem{} + env, ok := fs.CurrentRestoreEnvironment() + if !ok { + panic("failed to find restore environment") + } + mounts, ok := env.MountSources[fsys.Name()] + if !ok { + panic("failed to find mounts for filesystem type " + fsys.Name()) + } + var args fs.MountArgs + var found bool + for _, mount := range mounts { + if mount.Dev == s.connID { + args = mount + found = true + } + } + if !found { + panic(fmt.Sprintf("no connection for connection id %q", s.connID)) + } + + // Validate the mount flags and options. + opts, err := options(args.Data) + if err != nil { + panic("failed to parse mount options: " + err.Error()) + } + if opts.msize != s.msize { + panic(fmt.Sprintf("new message size %v, want %v", opts.msize, s.msize)) + } + if opts.version != s.version { + panic(fmt.Sprintf("new version %v, want %v", opts.version, s.version)) + } + if opts.policy != s.cachePolicy { + panic(fmt.Sprintf("new cache policy %v, want %v", opts.policy, s.cachePolicy)) + } + if opts.aname != s.aname { + panic(fmt.Sprintf("new attach name %v, want %v", opts.aname, s.aname)) + } + if opts.privateunixsocket != (s.endpoints != nil) { + panic(fmt.Sprintf("new privateunixsocket option %v, want %v", opts.privateunixsocket, s.endpoints != nil)) + } + if args.Flags != s.superBlockFlags { + panic(fmt.Sprintf("new mount flags %v, want %v", args.Flags, s.superBlockFlags)) + } + + // Manually restore the connection. + s.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) + if err != nil { + panic(fmt.Sprintf("failed to connect client to server: %v", err)) + } + + // Manually restore the attach point. + s.attach.file, err = s.client.Attach(s.aname) + if err != nil { + panic(fmt.Sprintf("failed to attach to aname: %v", err)) + } +} diff --git a/pkg/sentry/fs/gofer/socket.go b/pkg/sentry/fs/gofer/socket.go new file mode 100644 index 000000000..954000ef0 --- /dev/null +++ b/pkg/sentry/fs/gofer/socket.go @@ -0,0 +1,127 @@ +// 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 gofer + +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/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 { + if !fs.IsSocket(i.fileState.sattr) { + return nil + } + + if i.session().endpoints != nil { + ep := i.session().endpoints.get(i.fileState.key) + if ep != nil { + return ep + } + + // Not found in endpoints map, it may be a gofer backed unix socket... + } + + inode.IncRef() + return &endpoint{inode, i.fileState.file.file, path} +} + +// endpoint is a Gofer-backed unix.BoundEndpoint. +// +// An endpoint's lifetime is the time between when InodeOperations.BoundEndpoint() +// is called and either BoundEndpoint.BidirectionalConnect or +// BoundEndpoint.UnidirectionalConnect is called. +type endpoint struct { + // inode is the filesystem inode which produced this endpoint. + inode *fs.Inode + + // file is the p9 file that contains a single unopened fid. + file p9.File + + // path is the sentry path where this endpoint is bound. + path string +} + +func unixSockToP9(t unix.SockType) (p9.ConnectFlags, bool) { + switch t { + case unix.SockStream: + return p9.StreamSocket, true + case unix.SockSeqpacket: + return p9.SeqpacketSocket, true + case unix.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 { + cf, ok := unixSockToP9(ce.Type()) + if !ok { + return tcpip.ErrConnectionRefused + } + + // No lock ordering required as only the ConnectingEndpoint has a mutex. + ce.Lock() + defer ce.Unlock() + + // Check connecting state. + if ce.Connected() { + return tcpip.ErrAlreadyConnected + } + if ce.Listening() { + return tcpip.ErrInvalidEndpointState + } + + hostFile, err := e.file.Connect(cf) + if err != nil { + return tcpip.ErrConnectionRefused + } + + r, c, terr := host.NewConnectedEndpoint(hostFile, ce.WaiterQueue(), e.path) + if terr != nil { + return terr + } + returnConnect(r, c) + return nil +} + +// UnidirectionalConnect implements unix.BoundEndpoint.UnidirectionalConnect. +func (e *endpoint) UnidirectionalConnect() (unix.ConnectedEndpoint, *tcpip.Error) { + hostFile, err := e.file.Connect(p9.DgramSocket) + if err != nil { + return nil, tcpip.ErrConnectionRefused + } + + r, c, terr := host.NewConnectedEndpoint(hostFile, &waiter.Queue{}, e.path) + if terr != nil { + return nil, terr + } + + // We don't need the receiver. + r.CloseRecv() + r.Release() + + return c, nil +} + +// Release implements unix.BoundEndpoint.Release. +func (e *endpoint) Release() { + e.inode.DecRef() +} diff --git a/pkg/sentry/fs/gofer/util.go b/pkg/sentry/fs/gofer/util.go new file mode 100644 index 000000000..d9ed8c81e --- /dev/null +++ b/pkg/sentry/fs/gofer/util.go @@ -0,0 +1,60 @@ +// 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 gofer + +import ( + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/p9" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" +) + +func utimes(ctx context.Context, file contextFile, ts fs.TimeSpec) error { + if ts.ATimeOmit && ts.MTimeOmit { + return nil + } + mask := p9.SetAttrMask{ + ATime: !ts.ATimeOmit, + ATimeNotSystemTime: !ts.ATimeSetSystemTime, + MTime: !ts.MTimeOmit, + MTimeNotSystemTime: !ts.MTimeSetSystemTime, + } + as, ans := ts.ATime.Unix() + ms, mns := ts.MTime.Unix() + attr := p9.SetAttr{ + ATimeSeconds: uint64(as), + ATimeNanoSeconds: uint64(ans), + MTimeSeconds: uint64(ms), + MTimeNanoSeconds: uint64(mns), + } + // 9p2000.L SetAttr: "If a time bit is set without the corresponding SET bit, + // the current system time on the server is used instead of the value sent + // in the request." + return file.setAttr(ctx, mask, attr) +} + +func openFlagsFromPerms(p fs.PermMask) (p9.OpenFlags, error) { + switch { + case p.Read && p.Write: + return p9.ReadWrite, nil + case p.Write: + return p9.WriteOnly, nil + case p.Read: + return p9.ReadOnly, nil + default: + return 0, syscall.EINVAL + } +} |