summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fs/gofer
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/fs/gofer')
-rw-r--r--pkg/sentry/fs/gofer/attr.go162
-rw-r--r--pkg/sentry/fs/gofer/cache_policy.go183
-rw-r--r--pkg/sentry/fs/gofer/context_file.go190
-rw-r--r--pkg/sentry/fs/gofer/device.go20
-rw-r--r--pkg/sentry/fs/gofer/file.go333
-rw-r--r--pkg/sentry/fs/gofer/file_state.go39
-rw-r--r--pkg/sentry/fs/gofer/fs.go247
-rwxr-xr-xpkg/sentry/fs/gofer/gofer_state_autogen.go113
-rw-r--r--pkg/sentry/fs/gofer/handles.go129
-rw-r--r--pkg/sentry/fs/gofer/inode.go606
-rw-r--r--pkg/sentry/fs/gofer/inode_state.go172
-rw-r--r--pkg/sentry/fs/gofer/path.go433
-rw-r--r--pkg/sentry/fs/gofer/session.go361
-rw-r--r--pkg/sentry/fs/gofer/session_state.go115
-rw-r--r--pkg/sentry/fs/gofer/socket.go141
-rw-r--r--pkg/sentry/fs/gofer/util.go60
16 files changed, 3304 insertions, 0 deletions
diff --git a/pkg/sentry/fs/gofer/attr.go b/pkg/sentry/fs/gofer/attr.go
new file mode 100644
index 000000000..c572f3396
--- /dev/null
+++ b/pkg/sentry/fs/gofer/attr.go
@@ -0,0 +1,162 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package 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/cache_policy.go b/pkg/sentry/fs/gofer/cache_policy.go
new file mode 100644
index 000000000..c59344589
--- /dev/null
+++ b/pkg/sentry/fs/gofer/cache_policy.go
@@ -0,0 +1,183 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package gofer
+
+import (
+ "fmt"
+
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+)
+
+// cachePolicy is a 9p cache policy. It has methods that determine what to
+// cache (if anything) for a given inode.
+type cachePolicy int
+
+const (
+ // Cache nothing.
+ cacheNone cachePolicy = iota
+
+ // Use virtual file system cache for everything.
+ cacheAll
+
+ // Use virtual file system cache for everything, but send writes to the
+ // fs agent immediately.
+ cacheAllWritethrough
+
+ // Use the (host) page cache for reads/writes, but don't cache anything
+ // else. This allows the sandbox filesystem to stay in sync with any
+ // changes to the remote filesystem.
+ //
+ // This policy should *only* be used with remote filesystems that
+ // donate their host FDs to the sandbox and thus use the host page
+ // cache, otherwise the dirent state will be inconsistent.
+ cacheRemoteRevalidating
+)
+
+// String returns the string name of the cache policy.
+func (cp cachePolicy) String() string {
+ switch cp {
+ case cacheNone:
+ return "cacheNone"
+ case cacheAll:
+ return "cacheAll"
+ case cacheAllWritethrough:
+ return "cacheAllWritethrough"
+ case cacheRemoteRevalidating:
+ return "cacheRemoteRevalidating"
+ default:
+ return "unknown"
+ }
+}
+
+func parseCachePolicy(policy string) (cachePolicy, error) {
+ switch policy {
+ case "fscache":
+ return cacheAll, nil
+ case "none":
+ return cacheNone, nil
+ case "fscache_writethrough":
+ return cacheAllWritethrough, nil
+ case "remote_revalidating":
+ return cacheRemoteRevalidating, nil
+ }
+ return cacheNone, fmt.Errorf("unsupported cache mode: %s", policy)
+}
+
+// cacheUAtters determines whether unstable attributes should be cached for the
+// given inode.
+func (cp cachePolicy) cacheUAttrs(inode *fs.Inode) bool {
+ if !fs.IsFile(inode.StableAttr) && !fs.IsDir(inode.StableAttr) {
+ return false
+ }
+ return cp == cacheAll || cp == cacheAllWritethrough
+}
+
+// cacheReaddir determines whether readdir results should be cached.
+func (cp cachePolicy) cacheReaddir() bool {
+ return cp == cacheAll || cp == cacheAllWritethrough
+}
+
+// useCachingInodeOps determines whether the page cache should be used for the
+// given inode. If the remote filesystem donates host FDs to the sentry, then
+// the host kernel's page cache will be used, otherwise we will use a
+// sentry-internal page cache.
+func (cp cachePolicy) useCachingInodeOps(inode *fs.Inode) bool {
+ // Do cached IO for regular files only. Some "character devices" expect
+ // no caching.
+ if !fs.IsFile(inode.StableAttr) {
+ return false
+ }
+ return cp == cacheAll || cp == cacheAllWritethrough
+}
+
+// writeThough indicates whether writes to the file should be synced to the
+// gofer immediately.
+func (cp cachePolicy) writeThrough(inode *fs.Inode) bool {
+ return cp == cacheNone || cp == cacheAllWritethrough
+}
+
+// revalidate revalidates the child Inode if the cache policy allows it.
+//
+// Depending on the cache policy, revalidate will walk from the parent to the
+// child inode, and if any unstable attributes have changed, will update the
+// cached attributes on the child inode. If the walk fails, or the returned
+// inode id is different from the one being revalidated, then the entire Dirent
+// must be reloaded.
+func (cp cachePolicy) revalidate(ctx context.Context, name string, parent, child *fs.Inode) bool {
+ if cp == cacheAll || cp == cacheAllWritethrough {
+ return false
+ }
+
+ if cp == cacheNone {
+ return true
+ }
+
+ childIops, ok := child.InodeOperations.(*inodeOperations)
+ if !ok {
+ panic(fmt.Sprintf("revalidating inode operations of unknown type %T", child.InodeOperations))
+ }
+ parentIops, ok := parent.InodeOperations.(*inodeOperations)
+ if !ok {
+ panic(fmt.Sprintf("revalidating inode operations with parent of unknown type %T", parent.InodeOperations))
+ }
+
+ // Walk from parent to child again.
+ //
+ // TODO(b/112031682): If we have a directory FD in the parent
+ // inodeOperations, then we can use fstatat(2) to get the inode
+ // attributes instead of making this RPC.
+ qids, f, mask, attr, err := parentIops.fileState.file.walkGetAttr(ctx, []string{name})
+ if err != nil {
+ // Can't look up the name. Trigger reload.
+ return true
+ }
+ f.close(ctx)
+
+ // If the Path has changed, then we are not looking at the file file.
+ // We must reload.
+ if qids[0].Path != childIops.fileState.key.Inode {
+ return true
+ }
+
+ // If we are not caching unstable attrs, then there is nothing to
+ // update on this inode.
+ if !cp.cacheUAttrs(child) {
+ return false
+ }
+
+ // Update the inode's cached unstable attrs.
+ s := childIops.session()
+ childIops.cachingInodeOps.UpdateUnstable(unstable(ctx, mask, attr, s.mounter, s.client))
+
+ return false
+}
+
+// keep indicates that dirents should be kept pinned in the dirent tree even if
+// there are no application references on the file.
+func (cp cachePolicy) keep(d *fs.Dirent) bool {
+ if cp == cacheNone {
+ return false
+ }
+ sattr := d.Inode.StableAttr
+ // NOTE(b/31979197): Only cache files, directories, and symlinks.
+ return fs.IsFile(sattr) || fs.IsDir(sattr) || fs.IsSymlink(sattr)
+}
+
+// cacheNegativeDirents indicates that negative dirents should be held in the
+// dirent tree.
+func (cp cachePolicy) cacheNegativeDirents() bool {
+ return cp == cacheAll || cp == cacheAllWritethrough
+}
diff --git a/pkg/sentry/fs/gofer/context_file.go b/pkg/sentry/fs/gofer/context_file.go
new file mode 100644
index 000000000..be53ac4d9
--- /dev/null
+++ b/pkg/sentry/fs/gofer/context_file.go
@@ -0,0 +1,190 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package 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)
+
+ q, f, err := c.file.Walk(names)
+ if err != nil {
+ ctx.UninterruptibleSleepFinish(false)
+ return nil, contextFile{}, err
+ }
+ ctx.UninterruptibleSleepFinish(false)
+ return q, contextFile{file: f}, nil
+}
+
+func (c *contextFile) statFS(ctx context.Context) (p9.FSStat, error) {
+ ctx.UninterruptibleSleepStart(false)
+ s, err := c.file.StatFS()
+ ctx.UninterruptibleSleepFinish(false)
+ return s, err
+}
+
+func (c *contextFile) getAttr(ctx context.Context, req p9.AttrMask) (p9.QID, p9.AttrMask, p9.Attr, error) {
+ ctx.UninterruptibleSleepStart(false)
+ q, m, a, err := c.file.GetAttr(req)
+ ctx.UninterruptibleSleepFinish(false)
+ return q, m, a, err
+}
+
+func (c *contextFile) setAttr(ctx context.Context, valid p9.SetAttrMask, attr p9.SetAttr) error {
+ ctx.UninterruptibleSleepStart(false)
+ err := c.file.SetAttr(valid, attr)
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+}
+
+func (c *contextFile) allocate(ctx context.Context, mode p9.AllocateMode, offset, length uint64) error {
+ ctx.UninterruptibleSleepStart(false)
+ err := c.file.Allocate(mode, offset, length)
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+}
+
+func (c *contextFile) rename(ctx context.Context, directory contextFile, name string) error {
+ ctx.UninterruptibleSleepStart(false)
+ err := c.file.Rename(directory.file, name)
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+}
+
+func (c *contextFile) close(ctx context.Context) error {
+ ctx.UninterruptibleSleepStart(false)
+ err := c.file.Close()
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+}
+
+func (c *contextFile) open(ctx context.Context, mode p9.OpenFlags) (*fd.FD, p9.QID, uint32, error) {
+ ctx.UninterruptibleSleepStart(false)
+ f, q, u, err := c.file.Open(mode)
+ ctx.UninterruptibleSleepFinish(false)
+ return f, q, u, err
+}
+
+func (c *contextFile) readAt(ctx context.Context, p []byte, offset uint64) (int, error) {
+ ctx.UninterruptibleSleepStart(false)
+ n, err := c.file.ReadAt(p, offset)
+ ctx.UninterruptibleSleepFinish(false)
+ return n, err
+}
+
+func (c *contextFile) writeAt(ctx context.Context, p []byte, offset uint64) (int, error) {
+ ctx.UninterruptibleSleepStart(false)
+ n, err := c.file.WriteAt(p, offset)
+ ctx.UninterruptibleSleepFinish(false)
+ return n, err
+}
+
+func (c *contextFile) fsync(ctx context.Context) error {
+ ctx.UninterruptibleSleepStart(false)
+ err := c.file.FSync()
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+}
+
+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)
+ fd, _, _, _, err := c.file.Create(name, flags, permissions, uid, gid)
+ ctx.UninterruptibleSleepFinish(false)
+ 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)
+ q, err := c.file.Mkdir(name, permissions, uid, gid)
+ ctx.UninterruptibleSleepFinish(false)
+ return q, err
+}
+
+func (c *contextFile) symlink(ctx context.Context, oldName string, newName string, uid p9.UID, gid p9.GID) (p9.QID, error) {
+ ctx.UninterruptibleSleepStart(false)
+ q, err := c.file.Symlink(oldName, newName, uid, gid)
+ ctx.UninterruptibleSleepFinish(false)
+ return q, err
+}
+
+func (c *contextFile) link(ctx context.Context, target *contextFile, newName string) error {
+ ctx.UninterruptibleSleepStart(false)
+ err := c.file.Link(target.file, newName)
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+}
+
+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)
+ q, err := c.file.Mknod(name, permissions, major, minor, uid, gid)
+ ctx.UninterruptibleSleepFinish(false)
+ return q, err
+}
+
+func (c *contextFile) unlinkAt(ctx context.Context, name string, flags uint32) error {
+ ctx.UninterruptibleSleepStart(false)
+ err := c.file.UnlinkAt(name, flags)
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+}
+
+func (c *contextFile) readdir(ctx context.Context, offset uint64, count uint32) ([]p9.Dirent, error) {
+ ctx.UninterruptibleSleepStart(false)
+ d, err := c.file.Readdir(offset, count)
+ ctx.UninterruptibleSleepFinish(false)
+ return d, err
+}
+
+func (c *contextFile) readlink(ctx context.Context) (string, error) {
+ ctx.UninterruptibleSleepStart(false)
+ s, err := c.file.Readlink()
+ ctx.UninterruptibleSleepFinish(false)
+ return s, err
+}
+
+func (c *contextFile) flush(ctx context.Context) error {
+ ctx.UninterruptibleSleepStart(false)
+ err := c.file.Flush()
+ ctx.UninterruptibleSleepFinish(false)
+ return err
+}
+
+func (c *contextFile) walkGetAttr(ctx context.Context, names []string) ([]p9.QID, contextFile, p9.AttrMask, p9.Attr, error) {
+ ctx.UninterruptibleSleepStart(false)
+ q, f, m, a, err := c.file.WalkGetAttr(names)
+ if err != nil {
+ ctx.UninterruptibleSleepFinish(false)
+ return nil, contextFile{}, p9.AttrMask{}, p9.Attr{}, err
+ }
+ ctx.UninterruptibleSleepFinish(false)
+ return q, contextFile{file: f}, m, a, nil
+}
+
+func (c *contextFile) connect(ctx context.Context, flags p9.ConnectFlags) (*fd.FD, error) {
+ ctx.UninterruptibleSleepStart(false)
+ f, err := c.file.Connect(flags)
+ ctx.UninterruptibleSleepFinish(false)
+ return f, err
+}
diff --git a/pkg/sentry/fs/gofer/device.go b/pkg/sentry/fs/gofer/device.go
new file mode 100644
index 000000000..1de6c247c
--- /dev/null
+++ b/pkg/sentry/fs/gofer/device.go
@@ -0,0 +1,20 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package 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..fb4f50113
--- /dev/null
+++ b/pkg/sentry/fs/gofer/file.go
@@ -0,0 +1,333 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package gofer
+
+import (
+ "fmt"
+ "syscall"
+ "time"
+
+ "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 (
+ opensWX = metric.MustCreateNewUint64Metric("/gofer/opened_write_execute_file", true /* sync */, "Number of times a writable+executable file was opened from a gofer.")
+ opens9P = metric.MustCreateNewUint64Metric("/gofer/opens_9p", false /* sync */, "Number of times a 9P file was opened from a gofer.")
+ opensHost = metric.MustCreateNewUint64Metric("/gofer/opens_host", false /* sync */, "Number of times a host file was opened from a gofer.")
+ reads9P = metric.MustCreateNewUint64Metric("/gofer/reads_9p", false /* sync */, "Number of 9P file reads from a gofer.")
+ readWait9P = metric.MustCreateNewUint64Metric("/gofer/read_wait_9p", false /* sync */, "Time waiting on 9P file reads from a gofer, in nanoseconds.")
+ readsHost = metric.MustCreateNewUint64Metric("/gofer/reads_host", false /* sync */, "Number of host file reads from a gofer.")
+ readWaitHost = metric.MustCreateNewUint64Metric("/gofer/read_wait_host", false /* sync */, "Time waiting on host file reads from a gofer, in nanoseconds.")
+)
+
+// fileOperations implements fs.FileOperations for a remote file system.
+//
+// +stateify savable
+type fileOperations struct {
+ fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosplice"`
+ waiter.AlwaysReady `state:"nosave"`
+
+ // inodeOperations is the inodeOperations backing the file. It is protected
+ // by a reference held by File.Dirent.Inode which is stable until
+ // 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.
+//
+// The `name` argument is only used to log a warning if we are returning a
+// writeable+executable file. (A metric counter is incremented in this case as
+// well.) Note that we cannot call d.BaseName() directly in this function,
+// because that would lead to a lock order violation, since this is called in
+// d.Create which holds d.mu, while d.BaseName() takes d.parent.mu, and the two
+// locks must be taken in the opposite order.
+func NewFile(ctx context.Context, dirent *fs.Dirent, name string, 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
+
+ if fs.IsFile(dirent.Inode.StableAttr) {
+ // If cache policy is "remote revalidating", then we must
+ // ensure that we have a host FD. Otherwise, the
+ // sentry-internal page cache will be used, and we can end up
+ // in an inconsistent state if the remote file changes.
+ cp := dirent.Inode.InodeOperations.(*inodeOperations).session().cachePolicy
+ if cp == cacheRemoteRevalidating && handles.Host == nil {
+ panic(fmt.Sprintf("remote-revalidating cache policy requires gofer to donate host FD, but file %q did not have host FD", name))
+ }
+ }
+
+ f := &fileOperations{
+ inodeOperations: i,
+ handles: handles,
+ flags: flags,
+ }
+ if flags.Write {
+ if err := dirent.Inode.CheckPermission(ctx, fs.PermMask{Execute: true}); err == nil {
+ opensWX.Increment()
+ log.Warningf("Opened a writable executable: %q", name)
+ }
+ }
+ if handles.Host != nil {
+ opensHost.Increment()
+ } else {
+ opens9P.Increment()
+ }
+ 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)
+ if root != nil {
+ 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.cacheUAttrs(file.Dirent.Inode) {
+ 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.session().cachePolicy.cacheReaddir() || f.inodeOperations.readdirCache == nil {
+ 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
+ }
+ cp := f.inodeOperations.session().cachePolicy
+ if cp.useCachingInodeOps(file.Dirent.Inode) {
+ n, err := f.inodeOperations.cachingInodeOps.Write(ctx, src, offset)
+ if err != nil {
+ return n, err
+ }
+ if cp.writeThrough(file.Dirent.Inode) {
+ // Write out the file.
+ err = f.inodeOperations.cachingInodeOps.WriteOut(ctx, file.Dirent.Inode)
+ }
+ return n, err
+ }
+ if f.inodeOperations.fileState.hostMappable != nil {
+ return f.inodeOperations.fileState.hostMappable.Write(ctx, src, offset)
+ }
+ return src.CopyInTo(ctx, f.handles.readWriterAt(ctx, offset))
+}
+
+// incrementReadCounters increments the read counters for the read starting at the given time. We
+// use this function rather than using a defer in Read() to avoid the performance hit of defer.
+func (f *fileOperations) incrementReadCounters(start time.Time) {
+ if f.handles.Host != nil {
+ readsHost.Increment()
+ fs.IncrementWait(readWaitHost, start)
+ } else {
+ reads9P.Increment()
+ fs.IncrementWait(readWait9P, start)
+ }
+}
+
+// Read implements fs.FileOperations.Read.
+func (f *fileOperations) Read(ctx context.Context, file *fs.File, dst usermem.IOSequence, offset int64) (int64, error) {
+ var start time.Time
+ if fs.RecordWaitTime {
+ start = time.Now()
+ }
+ if fs.IsDir(file.Dirent.Inode.StableAttr) {
+ // Not all remote file systems enforce this so this client does.
+ f.incrementReadCounters(start)
+ return 0, syserror.EISDIR
+ }
+
+ if f.inodeOperations.session().cachePolicy.useCachingInodeOps(file.Dirent.Inode) {
+ n, err := f.inodeOperations.cachingInodeOps.Read(ctx, file, dst, offset)
+ f.incrementReadCounters(start)
+ return n, err
+ }
+ n, err := dst.CopyOutFrom(ctx, f.handles.readWriterAt(ctx, offset))
+ f.incrementReadCounters(start)
+ return n, err
+}
+
+// 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(edahlgren): 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 {
+ return f.inodeOperations.configureMMap(file, opts)
+}
+
+// UnstableAttr implements fs.FileOperations.UnstableAttr.
+func (f *fileOperations) UnstableAttr(ctx context.Context, file *fs.File) (fs.UnstableAttr, error) {
+ s := f.inodeOperations.session()
+ if s.cachePolicy.cacheUAttrs(file.Dirent.Inode) {
+ return f.inodeOperations.cachingInodeOps.UnstableAttr(ctx, file.Dirent.Inode)
+ }
+ // Use f.handles.File, which represents 9P fids that have been opened,
+ // instead of inodeFileState.file, which represents 9P fids that have not.
+ // This may be significantly more efficient in some implementations.
+ _, valid, pattr, err := getattr(ctx, f.handles.File)
+ if err != nil {
+ return fs.UnstableAttr{}, err
+ }
+ return unstable(ctx, valid, pattr, s.mounter, s.client), nil
+}
+
+// Seek implements fs.FileOperations.Seek.
+func (f *fileOperations) Seek(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64) (int64, error) {
+ return fsutil.SeekWithDirCursor(ctx, file, whence, offset, &f.dirCursor)
+}
diff --git a/pkg/sentry/fs/gofer/file_state.go b/pkg/sentry/fs/gofer/file_state.go
new file mode 100644
index 000000000..31264e065
--- /dev/null
+++ b/pkg/sentry/fs/gofer/file_state.go
@@ -0,0 +1,39 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package gofer
+
+import (
+ "fmt"
+
+ "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() error {
+ f.inodeOperations.fileState.waitForLoad()
+
+ // Manually load the open handles.
+ var err error
+ // TODO(b/38173783): Context is not plumbed to save/restore.
+ f.handles, err = f.inodeOperations.fileState.getHandles(context.Background(), f.flags)
+ if err != nil {
+ return fmt.Errorf("failed to re-open handle: %v", err)
+ }
+ return nil
+ }
+ fs.Async(fs.CatchError(load))
+}
diff --git a/pkg/sentry/fs/gofer/fs.go b/pkg/sentry/fs/gofer/fs.go
new file mode 100644
index 000000000..6ab89fcc2
--- /dev/null
+++ b/pkg/sentry/fs/gofer/fs.go
@@ -0,0 +1,247 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Package 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"
+)
+
+// 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='")
+
+ // ErrFileNoReadFD is returned when there is no 'rfdno' option.
+ ErrFileNoReadFD = errors.New("missing required option: 'rfdno='")
+
+ // ErrFileNoWriteFD is returned when there is no 'wfdno' option.
+ ErrFileNoWriteFD = errors.New("missing required option: 'wfdno='")
+)
+
+// filesystem is a 9p client.
+//
+// +stateify savable
+type filesystem struct{}
+
+var _ fs.Filesystem = (*filesystem)(nil)
+
+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
+}
+
+// AllowUserList allows this filesystem to be listed in /proc/filesystems.
+func (*filesystem) AllowUserList() bool {
+ return true
+}
+
+// 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, _ interface{}) (*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, ErrFileNoReadFD
+ }
+ delete(options, readFDKey)
+
+ // Check for the required 'wfdno=' option.
+ swfd, ok := options[writeFDKey]
+ if !ok {
+ return o, ErrFileNoWriteFD
+ }
+ 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 policy, ok := options[cacheKey]; ok {
+ cp, err := parseCachePolicy(policy)
+ if err != nil {
+ return o, err
+ }
+ o.policy = 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_state_autogen.go b/pkg/sentry/fs/gofer/gofer_state_autogen.go
new file mode 100755
index 000000000..f274d0c39
--- /dev/null
+++ b/pkg/sentry/fs/gofer/gofer_state_autogen.go
@@ -0,0 +1,113 @@
+// automatically generated by stateify.
+
+package gofer
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/state"
+)
+
+func (x *fileOperations) beforeSave() {}
+func (x *fileOperations) save(m state.Map) {
+ x.beforeSave()
+ m.Save("inodeOperations", &x.inodeOperations)
+ m.Save("dirCursor", &x.dirCursor)
+ m.Save("flags", &x.flags)
+}
+
+func (x *fileOperations) load(m state.Map) {
+ m.LoadWait("inodeOperations", &x.inodeOperations)
+ m.Load("dirCursor", &x.dirCursor)
+ m.LoadWait("flags", &x.flags)
+ m.AfterLoad(x.afterLoad)
+}
+
+func (x *filesystem) beforeSave() {}
+func (x *filesystem) save(m state.Map) {
+ x.beforeSave()
+}
+
+func (x *filesystem) afterLoad() {}
+func (x *filesystem) load(m state.Map) {
+}
+
+func (x *inodeOperations) beforeSave() {}
+func (x *inodeOperations) save(m state.Map) {
+ x.beforeSave()
+ m.Save("fileState", &x.fileState)
+ m.Save("cachingInodeOps", &x.cachingInodeOps)
+}
+
+func (x *inodeOperations) afterLoad() {}
+func (x *inodeOperations) load(m state.Map) {
+ m.LoadWait("fileState", &x.fileState)
+ m.Load("cachingInodeOps", &x.cachingInodeOps)
+}
+
+func (x *inodeFileState) save(m state.Map) {
+ x.beforeSave()
+ var loading struct{} = x.saveLoading()
+ m.SaveValue("loading", loading)
+ m.Save("s", &x.s)
+ m.Save("sattr", &x.sattr)
+ m.Save("savedUAttr", &x.savedUAttr)
+ m.Save("hostMappable", &x.hostMappable)
+}
+
+func (x *inodeFileState) load(m state.Map) {
+ m.LoadWait("s", &x.s)
+ m.LoadWait("sattr", &x.sattr)
+ m.Load("savedUAttr", &x.savedUAttr)
+ m.Load("hostMappable", &x.hostMappable)
+ m.LoadValue("loading", new(struct{}), func(y interface{}) { x.loadLoading(y.(struct{})) })
+ m.AfterLoad(x.afterLoad)
+}
+
+func (x *endpointMaps) beforeSave() {}
+func (x *endpointMaps) save(m state.Map) {
+ x.beforeSave()
+ m.Save("direntMap", &x.direntMap)
+ m.Save("pathMap", &x.pathMap)
+}
+
+func (x *endpointMaps) afterLoad() {}
+func (x *endpointMaps) load(m state.Map) {
+ m.Load("direntMap", &x.direntMap)
+ m.Load("pathMap", &x.pathMap)
+}
+
+func (x *session) save(m state.Map) {
+ x.beforeSave()
+ m.Save("AtomicRefCount", &x.AtomicRefCount)
+ m.Save("msize", &x.msize)
+ m.Save("version", &x.version)
+ m.Save("cachePolicy", &x.cachePolicy)
+ m.Save("aname", &x.aname)
+ m.Save("superBlockFlags", &x.superBlockFlags)
+ m.Save("connID", &x.connID)
+ m.Save("inodeMappings", &x.inodeMappings)
+ m.Save("mounter", &x.mounter)
+ m.Save("endpoints", &x.endpoints)
+}
+
+func (x *session) load(m state.Map) {
+ m.Load("AtomicRefCount", &x.AtomicRefCount)
+ m.LoadWait("msize", &x.msize)
+ m.LoadWait("version", &x.version)
+ m.LoadWait("cachePolicy", &x.cachePolicy)
+ m.LoadWait("aname", &x.aname)
+ m.LoadWait("superBlockFlags", &x.superBlockFlags)
+ m.LoadWait("connID", &x.connID)
+ m.LoadWait("inodeMappings", &x.inodeMappings)
+ m.LoadWait("mounter", &x.mounter)
+ m.LoadWait("endpoints", &x.endpoints)
+ m.AfterLoad(x.afterLoad)
+}
+
+func init() {
+ state.Register("gofer.fileOperations", (*fileOperations)(nil), state.Fns{Save: (*fileOperations).save, Load: (*fileOperations).load})
+ state.Register("gofer.filesystem", (*filesystem)(nil), state.Fns{Save: (*filesystem).save, Load: (*filesystem).load})
+ state.Register("gofer.inodeOperations", (*inodeOperations)(nil), state.Fns{Save: (*inodeOperations).save, Load: (*inodeOperations).load})
+ state.Register("gofer.inodeFileState", (*inodeFileState)(nil), state.Fns{Save: (*inodeFileState).save, Load: (*inodeFileState).load})
+ state.Register("gofer.endpointMaps", (*endpointMaps)(nil), state.Fns{Save: (*endpointMaps).save, Load: (*endpointMaps).load})
+ state.Register("gofer.session", (*session)(nil), state.Fns{Save: (*session).save, Load: (*session).load})
+}
diff --git a/pkg/sentry/fs/gofer/handles.go b/pkg/sentry/fs/gofer/handles.go
new file mode 100644
index 000000000..c7098cd36
--- /dev/null
+++ b/pkg/sentry/fs/gofer/handles.go
@@ -0,0 +1,129 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package 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(b/38173783): 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
+ }
+
+ var p9flags p9.OpenFlags
+ switch {
+ case flags.Read && flags.Write:
+ p9flags = p9.ReadWrite
+ case flags.Read && !flags.Write:
+ p9flags = p9.ReadOnly
+ case !flags.Read && flags.Write:
+ p9flags = p9.WriteOnly
+ default:
+ panic("impossible fs.FileFlags")
+ }
+
+ hostFile, _, _, err := newFile.open(ctx, p9flags)
+ if err != nil {
+ newFile.close(ctx)
+ return nil, err
+ }
+ h := &handles{
+ File: newFile,
+ Host: hostFile,
+ }
+ return h, nil
+}
+
+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..dcb3b2880
--- /dev/null
+++ b/pkg/sentry/fs/gofer/inode.go
@@ -0,0 +1,606 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package 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.
+//
+// +stateify savable
+type inodeOperations struct {
+ fsutil.InodeNotVirtual `state:"nosave"`
+ fsutil.InodeNoExtendedAttributes `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.
+//
+// +stateify savable
+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"`
+
+ // If readHandles is non-nil, it holds handles that are either read-only or
+ // read/write. If writeHandles is non-nil, it holds write-only handles if
+ // writeHandlesRW is false, and read/write handles if writeHandlesRW is
+ // true.
+ //
+ // Once readHandles becomes non-nil, it can't be changed until
+ // inodeFileState.Release(), because of a defect in the
+ // fsutil.CachedFileObject interface: there's no way for the caller of
+ // fsutil.CachedFileObject.FD() to keep the returned FD open, so if we
+ // racily replace readHandles after inodeFileState.FD() has returned
+ // readHandles.Host.FD(), fsutil.CachingInodeOperations may use a closed
+ // FD. writeHandles can be changed if writeHandlesRW is false, since
+ // inodeFileState.FD() can't return a write-only FD, but can't be changed
+ // if writeHandlesRW is true for the same reason.
+ readHandles *handles `state:"nosave"`
+ writeHandles *handles `state:"nosave"`
+ writeHandlesRW bool `state:"nosave"`
+
+ // 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
+
+ // hostMappable is created when using 'cacheRemoteRevalidating' to map pages
+ // directly from host.
+ hostMappable *fsutil.HostMappable
+}
+
+// Release releases file handles.
+func (i *inodeFileState) Release(ctx context.Context) {
+ i.file.close(ctx)
+ if i.readHandles != nil {
+ i.readHandles.DecRef()
+ }
+ if i.writeHandles != nil {
+ i.writeHandles.DecRef()
+ }
+}
+
+func (i *inodeFileState) canShareHandles() bool {
+ // Only share handles for regular files, since for other file types,
+ // distinct handles may have special semantics even if they represent the
+ // same file. Disable handle sharing for cache policy cacheNone, since this
+ // is legacy behavior.
+ return fs.IsFile(i.sattr) && i.s.cachePolicy != cacheNone
+}
+
+// Preconditions: i.handlesMu must be locked for writing.
+func (i *inodeFileState) setSharedHandlesLocked(flags fs.FileFlags, h *handles) {
+ if flags.Read && i.readHandles == nil {
+ h.IncRef()
+ i.readHandles = h
+ }
+ if flags.Write {
+ if i.writeHandles == nil {
+ h.IncRef()
+ i.writeHandles = h
+ i.writeHandlesRW = flags.Read
+ } else if !i.writeHandlesRW && flags.Read {
+ // Upgrade i.writeHandles.
+ i.writeHandles.DecRef()
+ h.IncRef()
+ i.writeHandles = h
+ i.writeHandlesRW = flags.Read
+ }
+ }
+}
+
+// getHandles returns a set of handles for a new file using i opened with the
+// given flags.
+func (i *inodeFileState) getHandles(ctx context.Context, flags fs.FileFlags) (*handles, error) {
+ if !i.canShareHandles() {
+ return newHandles(ctx, i.file, flags)
+ }
+ i.handlesMu.Lock()
+ defer i.handlesMu.Unlock()
+ // Do we already have usable shared handles?
+ if flags.Write {
+ if i.writeHandles != nil && (i.writeHandlesRW || !flags.Read) {
+ i.writeHandles.IncRef()
+ return i.writeHandles, nil
+ }
+ } else if i.readHandles != nil {
+ i.readHandles.IncRef()
+ return i.readHandles, nil
+ }
+ // No; get new handles and cache them for future sharing.
+ h, err := newHandles(ctx, i.file, flags)
+ if err != nil {
+ return nil, err
+ }
+ i.setSharedHandlesLocked(flags, h)
+ return h, nil
+}
+
+// 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.readHandles.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.writeHandles.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 i.skipSetAttr(mask) {
+ 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),
+ })
+}
+
+// skipSetAttr checks if attribute change can be skipped. It can be skipped
+// when:
+// - Mask is empty
+// - Mask contains only attributes that cannot be set in the gofer
+// - Mask contains only atime and/or mtime, and host FD exists
+//
+// Updates to atime and mtime can be skipped because cached value will be
+// "close enough" to host value, given that operation went directly to host FD.
+// Skipping atime updates is particularly important to reduce the number of
+// operations sent to the Gofer for readonly files.
+func (i *inodeFileState) skipSetAttr(mask fs.AttrMask) bool {
+ // First remove attributes that cannot be updated.
+ cpy := mask
+ cpy.Type = false
+ cpy.DeviceID = false
+ cpy.InodeID = false
+ cpy.BlockSize = false
+ cpy.Usage = false
+ cpy.Links = false
+ if cpy.Empty() {
+ return true
+ }
+
+ // Then check if more than just atime and mtime is being set.
+ cpy.AccessTime = false
+ cpy.ModificationTime = false
+ if !cpy.Empty() {
+ return false
+ }
+
+ i.handlesMu.RLock()
+ defer i.handlesMu.RUnlock()
+ return (i.readHandles != nil && i.readHandles.Host != nil) ||
+ (i.writeHandles != nil && i.writeHandles.Host != nil)
+}
+
+// Sync implements fsutil.CachedFileObject.Sync.
+func (i *inodeFileState) Sync(ctx context.Context) error {
+ i.handlesMu.RLock()
+ defer i.handlesMu.RUnlock()
+ if i.writeHandles == nil {
+ return nil
+ }
+ return i.writeHandles.File.fsync(ctx)
+}
+
+// FD implements fsutil.CachedFileObject.FD.
+func (i *inodeFileState) FD() int {
+ i.handlesMu.RLock()
+ defer i.handlesMu.RUnlock()
+ if i.writeHandlesRW && i.writeHandles != nil && i.writeHandles.Host != nil {
+ return int(i.writeHandles.Host.FD())
+ }
+ if i.readHandles != nil && i.readHandles.Host != nil {
+ return int(i.readHandles.Host.FD())
+ }
+ return -1
+}
+
+// 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
+}
+
+func (i *inodeFileState) Allocate(ctx context.Context, offset, length int64) error {
+ i.handlesMu.RLock()
+ defer i.handlesMu.RUnlock()
+
+ // No options are supported for now.
+ mode := p9.AllocateMode{}
+ return i.writeHandles.File.allocate(ctx, mode, uint64(offset), uint64(length))
+}
+
+// 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.cachingInodeOps.Release()
+
+ // Releasing the fileState may make RPCs to the gofer. There is
+ // no need to wait for those to return, so we can do this
+ // asynchronously.
+ //
+ // We use AsyncWithContext to avoid needing to allocate an extra
+ // anonymous function on the heap.
+ fs.AsyncWithContext(ctx, i.fileState.Release)
+}
+
+// Mappable implements fs.InodeOperations.Mappable.
+func (i *inodeOperations) Mappable(inode *fs.Inode) memmap.Mappable {
+ if i.session().cachePolicy.useCachingInodeOps(inode) {
+ return i.cachingInodeOps
+ }
+ // This check is necessary because it's returning an interface type.
+ if i.fileState.hostMappable != nil {
+ return i.fileState.hostMappable
+ }
+ return nil
+}
+
+// UnstableAttr implements fs.InodeOperations.UnstableAttr.
+func (i *inodeOperations) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) {
+ if i.session().cachePolicy.cacheUAttrs(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 that doesn't work, handle it normally.
+ pipeOps, err := fdpipe.Open(ctx, i, flags)
+ if err == errNotHostFile {
+ return i.getFileDefault(ctx, d, flags)
+ }
+ if err != nil {
+ return nil, err
+ }
+ return fs.NewFile(ctx, d, flags, pipeOps), 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) {
+ h, err := i.fileState.getHandles(ctx, flags)
+ if err != nil {
+ return nil, err
+ }
+ return NewFile(ctx, d, d.BaseName(), flags, i, h), nil
+}
+
+// SetPermissions implements fs.InodeOperations.SetPermissions.
+func (i *inodeOperations) SetPermissions(ctx context.Context, inode *fs.Inode, p fs.FilePermissions) bool {
+ if i.session().cachePolicy.cacheUAttrs(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 i.session().cachePolicy.cacheUAttrs(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 i.session().cachePolicy.cacheUAttrs(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 i.session().cachePolicy.useCachingInodeOps(inode) {
+ return i.cachingInodeOps.Truncate(ctx, inode, length)
+ }
+ if i.session().cachePolicy == cacheRemoteRevalidating {
+ return i.fileState.hostMappable.Truncate(ctx, length)
+ }
+
+ return i.fileState.file.setAttr(ctx, p9.SetAttrMask{Size: true}, p9.SetAttr{Size: uint64(length)})
+}
+
+// Allocate implements fs.InodeOperations.Allocate.
+func (i *inodeOperations) Allocate(ctx context.Context, inode *fs.Inode, offset, length int64) error {
+ // This can only be called for files anyway.
+ if i.session().cachePolicy.useCachingInodeOps(inode) {
+ return i.cachingInodeOps.Allocate(ctx, offset, length)
+ }
+ if i.session().cachePolicy == cacheRemoteRevalidating {
+ return i.fileState.hostMappable.Allocate(ctx, offset, length)
+ }
+
+ // No options are supported for now.
+ mode := p9.AllocateMode{}
+ return i.fileState.file.allocate(ctx, mode, uint64(offset), uint64(length))
+}
+
+// WriteOut implements fs.InodeOperations.WriteOut.
+func (i *inodeOperations) WriteOut(ctx context.Context, inode *fs.Inode) error {
+ if !i.session().cachePolicy.cacheUAttrs(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 (i *inodeOperations) configureMMap(file *fs.File, opts *memmap.MMapOpts) error {
+ if i.session().cachePolicy.useCachingInodeOps(file.Dirent.Inode) {
+ return fsutil.GenericConfigureMMap(file, i.cachingInodeOps, opts)
+ }
+ if i.fileState.hostMappable != nil {
+ return fsutil.GenericConfigureMMap(file, i.fileState.hostMappable, opts)
+ }
+ return syserror.ENODEV
+}
+
+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(b/63117438): Remove this from InodeOperations altogether.
+func (*inodeOperations) AddLink() {}
+
+// DropLink implements InodeOperations.DropLink, but is currently a noop.
+// FIXME(b/63117438): Remove this from InodeOperations altogether.
+func (*inodeOperations) DropLink() {}
+
+// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange.
+// FIXME(b/63117438): Remove this from InodeOperations altogether.
+func (i *inodeOperations) NotifyStatusChange(ctx context.Context) {}
diff --git a/pkg/sentry/fs/gofer/inode_state.go b/pkg/sentry/fs/gofer/inode_state.go
new file mode 100644
index 000000000..ac22ee4b1
--- /dev/null
+++ b/pkg/sentry/fs/gofer/inode_state.go
@@ -0,0 +1,172 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package gofer
+
+import (
+ "errors"
+ "fmt"
+ "path/filepath"
+ "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(fs.ErrSaveRejection{fmt.Errorf("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{}{}
+}
+
+// splitAbsolutePath splits the path on slashes ignoring the leading slash.
+func splitAbsolutePath(path string) []string {
+ if len(path) == 0 {
+ panic("There is no path!")
+ }
+ if path != filepath.Clean(path) {
+ panic(fmt.Sprintf("path %q is not clean", path))
+ }
+ // This case is to return {} rather than {""}
+ if path == "/" {
+ return []string{}
+ }
+ if path[0] != '/' {
+ panic(fmt.Sprintf("path %q is not absolute", path))
+ }
+
+ s := strings.Split(path, "/")
+
+ // Since p is absolute, the first component of s
+ // is an empty string. We must remove that.
+ return s[1:]
+}
+
+// loadLoading is invoked by stateify.
+func (i *inodeFileState) loadLoading(_ struct{}) {
+ i.loading.Lock()
+}
+
+// afterLoad is invoked by stateify.
+func (i *inodeFileState) afterLoad() {
+ load := func() (err error) {
+ // See comment on i.loading().
+ defer func() {
+ if err == nil {
+ 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.
+ return fmt.Errorf("failed to find path for inode number %d. Device %s contains %s", i.sattr.InodeID, i.s.connID, fs.InodeMappings(i.s.inodeMappings))
+ }
+ // TODO(b/38173783): Context is not plumbed to save/restore.
+ ctx := &dummyClockContext{context.Background()}
+
+ _, i.file, err = i.s.attach.walk(ctx, splitAbsolutePath(name))
+ if err != nil {
+ return fs.ErrCorruption{fmt.Errorf("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 {
+ return fs.ErrCorruption{fmt.Errorf("failed to get file attributes of %s: %v", name, err)}
+ }
+ if !mask.RDev {
+ return fs.ErrCorruption{fmt.Errorf("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) {
+ return fs.ErrCorruption{fmt.Errorf("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 {
+ return errors.New("missing restore environment")
+ }
+ uattr := unstable(ctx, mask, attrs, i.s.mounter, i.s.client)
+ if env.ValidateFileSize && uattr.Size != i.savedUAttr.Size {
+ return fs.ErrCorruption{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 {
+ return fs.ErrCorruption{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
+ }
+
+ return nil
+ }
+
+ fs.Async(fs.CatchError(load))
+}
diff --git a/pkg/sentry/fs/gofer/path.go b/pkg/sentry/fs/gofer/path.go
new file mode 100644
index 000000000..092f8b586
--- /dev/null
+++ b/pkg/sentry/fs/gofer/path.go
@@ -0,0 +1,433 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package gofer
+
+import (
+ "fmt"
+ "syscall"
+
+ "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/socket/unix/transport"
+ "gvisor.googlesource.com/gvisor/pkg/syserror"
+)
+
+// maxFilenameLen is the maximum length of a filename. This is dictated by 9P's
+// encoding of strings, which uses 2 bytes for the length prefix.
+const maxFilenameLen = (1 << 16) - 1
+
+// 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 len(name) > maxFilenameLen {
+ return nil, syserror.ENAMETOOLONG
+ }
+
+ cp := i.session().cachePolicy
+ if cp.cacheReaddir() {
+ // 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.
+ i.readdirMu.Unlock()
+ if cp.cacheNegativeDirents() {
+ return fs.NewNegativeDirent(name), nil
+ }
+ return nil, syserror.ENOENT
+ }
+ 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 cp.cacheNegativeDirents() {
+ // 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, false)
+
+ // 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) {
+ if len(name) > maxFilenameLen {
+ return nil, syserror.ENAMETOOLONG
+ }
+
+ // 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.
+ newFile.close(ctx)
+ return nil, err
+ }
+
+ i.touchModificationAndStatusChangeTime(ctx, dir)
+
+ // Get an unopened p9.File for the file we created so that it can be cloned
+ // and re-opened multiple times after creation, while also getting its
+ // attributes. Both are required for inodeOperations.
+ qids, unopened, mask, p9attr, err := i.fileState.file.walkGetAttr(ctx, []string{name})
+ if err != nil {
+ newFile.close(ctx)
+ if hostFile != nil {
+ hostFile.Close()
+ }
+ return nil, err
+ }
+ if len(qids) != 1 {
+ log.Warningf("WalkGetAttr(%s) succeeded, but returned %d QIDs (%v), wanted 1", name, len(qids), qids)
+ newFile.close(ctx)
+ if hostFile != nil {
+ hostFile.Close()
+ }
+ unopened.close(ctx)
+ return nil, syserror.EIO
+ }
+ qid := qids[0]
+
+ // Construct the InodeOperations.
+ sattr, iops := newInodeOperations(ctx, i.fileState.s, unopened, qid, mask, p9attr, false)
+
+ // 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 iops.fileState.canShareHandles() {
+ iops.fileState.handlesMu.Lock()
+ iops.fileState.setSharedHandlesLocked(flags, h)
+ iops.fileState.handlesMu.Unlock()
+ }
+ return NewFile(ctx, d, name, 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 {
+ if len(newname) > maxFilenameLen {
+ return syserror.ENAMETOOLONG
+ }
+
+ 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.touchModificationAndStatusChangeTime(ctx, dir)
+ return nil
+}
+
+// CreateHardLink implements InodeOperations.CreateHardLink.
+func (i *inodeOperations) CreateHardLink(ctx context.Context, inode *fs.Inode, target *fs.Inode, newName string) error {
+ if len(newName) > maxFilenameLen {
+ return syserror.ENAMETOOLONG
+ }
+
+ 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
+ }
+ if i.session().cachePolicy.cacheUAttrs(inode) {
+ // Increase link count.
+ targetOpts.cachingInodeOps.IncLinks(ctx)
+ }
+ i.touchModificationAndStatusChangeTime(ctx, inode)
+ 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 {
+ if len(s) > maxFilenameLen {
+ return syserror.ENAMETOOLONG
+ }
+
+ 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.cacheUAttrs(dir) {
+ // Increase link count.
+ //
+ // N.B. This will update the modification time.
+ i.cachingInodeOps.IncLinks(ctx)
+ }
+ if i.session().cachePolicy.cacheReaddir() {
+ // Invalidate readdir cache.
+ i.markDirectoryDirty()
+ }
+ return nil
+}
+
+// Bind implements InodeOperations.Bind.
+func (i *inodeOperations) Bind(ctx context.Context, dir *fs.Inode, name string, ep transport.BoundEndpoint, perm fs.FilePermissions) (*fs.Dirent, error) {
+ if len(name) > maxFilenameLen {
+ return nil, syserror.ENAMETOOLONG
+ }
+
+ if i.session().endpoints == nil {
+ return nil, 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 nil, 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 nil, err
+ }
+ // We're not going to use this file.
+ hostFile.Close()
+
+ i.touchModificationAndStatusChangeTime(ctx, dir)
+
+ // Get the attributes of the file to create inode key.
+ qid, mask, attr, err := getattr(ctx, newFile)
+ if err != nil {
+ newFile.close(ctx)
+ return nil, err
+ }
+
+ key := device.MultiDeviceKey{
+ Device: attr.RDev,
+ SecondaryDevice: i.session().connID,
+ Inode: qid.Path,
+ }
+
+ // Create child dirent.
+
+ // 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, attr, true)
+
+ // Construct the positive Dirent.
+ childDir := fs.NewDirent(fs.NewInode(iops, dir.MountSource, sattr), name)
+ i.session().endpoints.add(key, childDir, ep)
+ return childDir, nil
+}
+
+// CreateFifo implements fs.InodeOperations.CreateFifo.
+func (i *inodeOperations) CreateFifo(ctx context.Context, dir *fs.Inode, name string, perm fs.FilePermissions) error {
+ if len(name) > maxFilenameLen {
+ return syserror.ENAMETOOLONG
+ }
+
+ owner := fs.FileOwnerFromContext(ctx)
+ mode := p9.FileMode(perm.LinuxMode()) | p9.ModeNamedPipe
+
+ // N.B. FIFOs use major/minor numbers 0.
+ if _, err := i.fileState.file.mknod(ctx, name, mode, 0, 0, p9.UID(owner.UID), p9.GID(owner.GID)); err != nil {
+ return err
+ }
+
+ i.touchModificationAndStatusChangeTime(ctx, dir)
+ return nil
+}
+
+// Remove implements InodeOperations.Remove.
+func (i *inodeOperations) Remove(ctx context.Context, dir *fs.Inode, name string) error {
+ if len(name) > maxFilenameLen {
+ return syserror.ENAMETOOLONG
+ }
+
+ 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.touchModificationAndStatusChangeTime(ctx, dir)
+
+ return nil
+}
+
+// Remove implements InodeOperations.RemoveDirectory.
+func (i *inodeOperations) RemoveDirectory(ctx context.Context, dir *fs.Inode, name string) error {
+ if len(name) > maxFilenameLen {
+ return syserror.ENAMETOOLONG
+ }
+
+ // 0x200 = AT_REMOVEDIR.
+ if err := i.fileState.file.unlinkAt(ctx, name, 0x200); err != nil {
+ return err
+ }
+ if i.session().cachePolicy.cacheUAttrs(dir) {
+ // Decrease link count and updates atime.
+ i.cachingInodeOps.DecLinks(ctx)
+ }
+ if i.session().cachePolicy.cacheReaddir() {
+ // Invalidate readdir cache.
+ i.markDirectoryDirty()
+ }
+ return nil
+}
+
+// Rename renames this node.
+func (i *inodeOperations) Rename(ctx context.Context, inode *fs.Inode, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string, replacement bool) error {
+ if len(newName) > maxFilenameLen {
+ return syserror.ENAMETOOLONG
+ }
+
+ // 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
+ }
+
+ // Is the renamed entity a directory? Fix link counts.
+ if fs.IsDir(i.fileState.sattr) {
+ // Update cached state.
+ if i.session().cachePolicy.cacheUAttrs(oldParent) {
+ oldParentInodeOperations.cachingInodeOps.DecLinks(ctx)
+ }
+ if i.session().cachePolicy.cacheUAttrs(newParent) {
+ // Only IncLinks if there is a new addition to
+ // newParent. If this is replacement, then the total
+ // count remains the same.
+ if !replacement {
+ newParentInodeOperations.cachingInodeOps.IncLinks(ctx)
+ }
+ }
+ }
+ if i.session().cachePolicy.cacheReaddir() {
+ // Mark old directory dirty.
+ oldParentInodeOperations.markDirectoryDirty()
+ if oldParent != newParent {
+ // Mark new directory dirty.
+ newParentInodeOperations.markDirectoryDirty()
+ }
+ }
+
+ // Rename always updates ctime.
+ if i.session().cachePolicy.cacheUAttrs(inode) {
+ i.cachingInodeOps.TouchStatusChangeTime(ctx)
+ }
+ return nil
+}
+
+func (i *inodeOperations) touchModificationAndStatusChangeTime(ctx context.Context, inode *fs.Inode) {
+ if i.session().cachePolicy.cacheUAttrs(inode) {
+ i.cachingInodeOps.TouchModificationAndStatusChangeTime(ctx)
+ }
+ if i.session().cachePolicy.cacheReaddir() {
+ // 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..085a358fe
--- /dev/null
+++ b/pkg/sentry/fs/gofer/session.go
@@ -0,0 +1,361 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package gofer
+
+import (
+ "fmt"
+ "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/sentry/socket/unix/transport"
+ "gvisor.googlesource.com/gvisor/pkg/unet"
+)
+
+// DefaultDirentCacheSize is the default dirent cache size for 9P mounts. It can
+// be adjusted independentely from the other dirent caches.
+var DefaultDirentCacheSize uint64 = fs.DefaultDirentCacheSize
+
+// +stateify savable
+type endpointMaps struct {
+ // mu protexts the direntMap, the keyMap, and the pathMap below.
+ mu sync.RWMutex `state:"nosave"`
+
+ // direntMap links sockets to their dirents.
+ // It is filled concurrently with the keyMap and is stored upon save.
+ // Before saving, this map is used to populate the pathMap.
+ direntMap map[transport.BoundEndpoint]*fs.Dirent
+
+ // keyMap links MultiDeviceKeys (containing inode IDs) to their sockets.
+ // It is not stored during save because the inode ID may change upon restore.
+ keyMap map[device.MultiDeviceKey]transport.BoundEndpoint `state:"nosave"`
+
+ // pathMap links the sockets to their paths.
+ // It is filled before saving from the direntMap and is stored upon save.
+ // Upon restore, this map is used to re-populate the keyMap.
+ pathMap map[transport.BoundEndpoint]string
+}
+
+// add adds the endpoint to the maps.
+// A reference is taken on the dirent argument.
+//
+// Precondition: maps must have been locked with 'lock'.
+func (e *endpointMaps) add(key device.MultiDeviceKey, d *fs.Dirent, ep transport.BoundEndpoint) {
+ e.keyMap[key] = ep
+ d.IncRef()
+ e.direntMap[ep] = d
+}
+
+// remove deletes the key from the maps.
+//
+// Precondition: maps must have been locked with 'lock'.
+func (e *endpointMaps) remove(key device.MultiDeviceKey) {
+ endpoint := e.get(key)
+ delete(e.keyMap, key)
+
+ d := e.direntMap[endpoint]
+ d.DecRef()
+ delete(e.direntMap, endpoint)
+}
+
+// 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 *endpointMaps) lock() func() {
+ e.mu.Lock()
+ return func() { e.mu.Unlock() }
+}
+
+// get returns the endpoint mapped to the given key.
+//
+// Precondition: maps must have been locked for reading.
+func (e *endpointMaps) get(key device.MultiDeviceKey) transport.BoundEndpoint {
+ return e.keyMap[key]
+}
+
+// session holds state for each 9p session established during sys_mount.
+//
+// +stateify savable
+type session struct {
+ refs.AtomicRefCount
+
+ // 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.
+ 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(b/77154739): there are few possible races with someone stat'ing the
+ // file and another deleting it concurrently, where the file will not be
+ // reported as socket file.
+ endpoints *endpointMaps `state:"wait"`
+}
+
+// Destroy tears down the session.
+func (s *session) Destroy() {
+ s.client.Close()
+}
+
+// Revalidate implements MountSource.Revalidate.
+func (s *session) Revalidate(ctx context.Context, name string, parent, child *fs.Inode) bool {
+ return s.cachePolicy.revalidate(ctx, name, parent, child)
+}
+
+// Keep implements MountSource.Keep.
+func (s *session) Keep(d *fs.Dirent) bool {
+ return s.cachePolicy.keep(d)
+}
+
+// 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).
+//
+// Endpoints lock must not be held if socket == false.
+func newInodeOperations(ctx context.Context, s *session, file contextFile, qid p9.QID, valid p9.AttrMask, attr p9.Attr, socket bool) (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 socket {
+ sattr.Type = fs.Socket
+ } else {
+ // If unix sockets are allowed on this filesystem, check if this file is
+ // supposed to be a socket file.
+ unlock := s.endpoints.lock()
+ if s.endpoints.get(deviceKey) != nil {
+ sattr.Type = fs.Socket
+ }
+ unlock()
+ }
+ }
+
+ fileState := &inodeFileState{
+ s: s,
+ file: file,
+ sattr: sattr,
+ key: deviceKey,
+ }
+ if s.cachePolicy == cacheRemoteRevalidating && fs.IsFile(sattr) {
+ fileState.hostMappable = fsutil.NewHostMappable(fileState)
+ }
+
+ 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,
+ msize: o.msize,
+ version: o.version,
+ cachePolicy: o.policy,
+ aname: o.aname,
+ superBlockFlags: superBlockFlags,
+ mounter: mounter,
+ }
+
+ if o.privateunixsocket {
+ s.endpoints = newEndpointMaps()
+ }
+
+ // Construct the MountSource with the session and superBlockFlags.
+ m := fs.NewMountSource(s, filesystem, superBlockFlags)
+
+ // Given that gofer files can consume host FDs, restrict the number
+ // of files that can be held by the cache.
+ m.SetDirentCacheMaxSize(DefaultDirentCacheSize)
+ m.SetDirentCacheLimiter(fs.DirentCacheLimiterFromContext(ctx))
+
+ // Send the Tversion request.
+ s.client, err = p9.NewClient(conn, s.msize, s.version)
+ if err != nil {
+ // Drop our reference on the session, it needs to be torn down.
+ s.DecRef()
+ 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, false)
+ return fs.NewInode(iops, m, sattr), nil
+}
+
+// newEndpointMaps creates a new endpointMaps.
+func newEndpointMaps() *endpointMaps {
+ return &endpointMaps{
+ direntMap: make(map[transport.BoundEndpoint]*fs.Dirent),
+ keyMap: make(map[device.MultiDeviceKey]transport.BoundEndpoint),
+ pathMap: make(map[transport.BoundEndpoint]string),
+ }
+}
+
+// fillKeyMap populates key and dirent maps upon restore from saved
+// pathmap.
+func (s *session) fillKeyMap(ctx context.Context) error {
+ unlock := s.endpoints.lock()
+ defer unlock()
+
+ for ep, dirPath := range s.endpoints.pathMap {
+ _, file, err := s.attach.walk(ctx, splitAbsolutePath(dirPath))
+ if err != nil {
+ return fmt.Errorf("error filling endpointmaps, failed to walk to %q: %v", dirPath, err)
+ }
+
+ qid, _, attr, err := file.getAttr(ctx, p9.AttrMaskAll())
+ if err != nil {
+ return fmt.Errorf("failed to get file attributes of %s: %v", dirPath, err)
+ }
+
+ key := device.MultiDeviceKey{
+ Device: attr.RDev,
+ SecondaryDevice: s.connID,
+ Inode: qid.Path,
+ }
+
+ s.endpoints.keyMap[key] = ep
+ }
+ return nil
+}
+
+// fillPathMap populates paths for endpoints from dirents in direntMap
+// before save.
+func (s *session) fillPathMap() error {
+ unlock := s.endpoints.lock()
+ defer unlock()
+
+ for ep, dir := range s.endpoints.direntMap {
+ mountRoot := dir.MountRoot()
+ defer mountRoot.DecRef()
+ dirPath, _ := dir.FullName(mountRoot)
+ if dirPath == "" {
+ return fmt.Errorf("error getting path from dirent")
+ }
+ s.endpoints.pathMap[ep] = dirPath
+ }
+ return nil
+}
+
+// restoreEndpointMaps recreates and fills the key and dirent maps.
+func (s *session) restoreEndpointMaps(ctx context.Context) error {
+ // When restoring, only need to create the keyMap because the dirent and path
+ // maps got stored through the save.
+ s.endpoints.keyMap = make(map[device.MultiDeviceKey]transport.BoundEndpoint)
+ if err := s.fillKeyMap(ctx); err != nil {
+ return fmt.Errorf("failed to insert sockets into endpoint map: %v", err)
+ }
+
+ // Re-create pathMap because it can no longer be trusted as socket paths can
+ // change while process continues to run. Empty pathMap will be re-filled upon
+ // next save.
+ s.endpoints.pathMap = make(map[transport.BoundEndpoint]string)
+ return 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..68fbf3417
--- /dev/null
+++ b/pkg/sentry/fs/gofer/session_state.go
@@ -0,0 +1,115 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package gofer
+
+import (
+ "fmt"
+
+ "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/unet"
+)
+
+// beforeSave is invoked by stateify.
+func (s *session) beforeSave() {
+ if s.endpoints != nil {
+ if err := s.fillPathMap(); err != nil {
+ panic("failed to save paths to endpoint map before saving" + err.Error())
+ }
+ }
+}
+
+// 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.DataString)
+ 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))
+ }
+
+ // Check if endpointMaps exist when uds sockets are enabled
+ // (only pathmap will actualy have been saved).
+ 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.
+ 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(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))
+ }
+
+ // If private unix sockets are enabled, create and fill the session's endpoint
+ // maps.
+ if opts.privateunixsocket {
+ // TODO(b/38173783): Context is not plumbed to save/restore.
+ ctx := &dummyClockContext{context.Background()}
+
+ if err = s.restoreEndpointMaps(ctx); err != nil {
+ panic("failed to restore endpoint maps: " + err.Error())
+ }
+ }
+
+}
diff --git a/pkg/sentry/fs/gofer/socket.go b/pkg/sentry/fs/gofer/socket.go
new file mode 100644
index 000000000..cbd5b9a84
--- /dev/null
+++ b/pkg/sentry/fs/gofer/socket.go
@@ -0,0 +1,141 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package gofer
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.googlesource.com/gvisor/pkg/p9"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/host"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport"
+ "gvisor.googlesource.com/gvisor/pkg/syserr"
+ "gvisor.googlesource.com/gvisor/pkg/waiter"
+)
+
+// BoundEndpoint returns a gofer-backed transport.BoundEndpoint.
+func (i *inodeOperations) BoundEndpoint(inode *fs.Inode, path string) transport.BoundEndpoint {
+ if !fs.IsSocket(i.fileState.sattr) {
+ return nil
+ }
+
+ if i.session().endpoints != nil {
+ unlock := i.session().endpoints.lock()
+ defer unlock()
+ 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 transport.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 transport.SockType) (p9.ConnectFlags, bool) {
+ switch t {
+ case transport.SockStream:
+ return p9.StreamSocket, true
+ case transport.SockSeqpacket:
+ return p9.SeqpacketSocket, true
+ case transport.SockDgram:
+ return p9.DgramSocket, true
+ }
+ return 0, false
+}
+
+// BidirectionalConnect implements ConnectableEndpoint.BidirectionalConnect.
+func (e *endpoint) BidirectionalConnect(ce transport.ConnectingEndpoint, returnConnect func(transport.Receiver, transport.ConnectedEndpoint)) *syserr.Error {
+ cf, ok := unixSockToP9(ce.Type())
+ if !ok {
+ return syserr.ErrConnectionRefused
+ }
+
+ // No lock ordering required as only the ConnectingEndpoint has a mutex.
+ ce.Lock()
+
+ // Check connecting state.
+ if ce.Connected() {
+ ce.Unlock()
+ return syserr.ErrAlreadyConnected
+ }
+ if ce.Listening() {
+ ce.Unlock()
+ return syserr.ErrInvalidEndpointState
+ }
+
+ hostFile, err := e.file.Connect(cf)
+ if err != nil {
+ ce.Unlock()
+ return syserr.ErrConnectionRefused
+ }
+
+ c, serr := host.NewConnectedEndpoint(hostFile, ce.WaiterQueue(), e.path)
+ if serr != nil {
+ ce.Unlock()
+ log.Warningf("Gofer returned invalid host socket for BidirectionalConnect; file %+v flags %+v: %v", e.file, cf, serr)
+ return serr
+ }
+
+ returnConnect(c, c)
+ ce.Unlock()
+ c.Init()
+
+ return nil
+}
+
+// UnidirectionalConnect implements
+// transport.BoundEndpoint.UnidirectionalConnect.
+func (e *endpoint) UnidirectionalConnect() (transport.ConnectedEndpoint, *syserr.Error) {
+ hostFile, err := e.file.Connect(p9.DgramSocket)
+ if err != nil {
+ return nil, syserr.ErrConnectionRefused
+ }
+
+ c, serr := host.NewConnectedEndpoint(hostFile, &waiter.Queue{}, e.path)
+ if serr != nil {
+ log.Warningf("Gofer returned invalid host socket for UnidirectionalConnect; file %+v: %v", e.file, serr)
+ return nil, serr
+ }
+ c.Init()
+
+ // We don't need the receiver.
+ c.CloseRecv()
+ c.Release()
+
+ return c, nil
+}
+
+// Release implements transport.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..d0e1096ce
--- /dev/null
+++ b/pkg/sentry/fs/gofer/util.go
@@ -0,0 +1,60 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package 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
+ }
+}