From 15ff2893d9bf896cc33e880983cb800cd71c946b Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Mon, 17 Aug 2020 15:33:19 -0700 Subject: Extend integration test to test sequence of FUSE operation Original FUSE integration test has limited capabilities. To test more situations, the new integration test framework introduces a protocol to communicate between testing thread and the FUSE server. In summary, this change includes: 1. Remove CompareResult() and break SetExpected() into SetServerResponse() and GetServerActualRequest(). We no longer set up an expected request because we want to retrieve the actual FUSE request made to the FUSE server and check in the testing thread. 2. Declare a serial buffer data structure to save the received requests and expected responses sequentially. The data structure contains a cursor to indicate the progress of accessing. This change makes sequential SetServerResponse() and GetServerActualRequest() possible. 3. Replace 2 single directional pipes with 1 bi-directional socketpair. A protocol which starts with FuseTestCmd is used between the testing thread and the FUSE server to provide various functionality. Fixes #3405 --- pkg/sentry/fsimpl/fuse/dev.go | 8 ++++++++ 1 file changed, 8 insertions(+) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index e522ff9a0..0efd2d90d 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -307,6 +307,14 @@ func (fd *DeviceFD) writeLocked(ctx context.Context, src usermem.IOSequence, opt // Readiness implements vfs.FileDescriptionImpl.Readiness. func (fd *DeviceFD) Readiness(mask waiter.EventMask) waiter.EventMask { + fd.mu.Lock() + defer fd.mu.Unlock() + return fd.readinessLocked(mask) +} + +// readinessLocked implements checking the readiness of the fuse device while +// locked with DeviceFD.mu. +func (fd *DeviceFD) readinessLocked(mask waiter.EventMask) waiter.EventMask { var ready waiter.EventMask ready |= waiter.EventOut // FD is always writable if !fd.queue.Empty() { -- cgit v1.2.3 From d6ee3ae6d797b7f66092971f395ef63f8a430c78 Mon Sep 17 00:00:00 2001 From: Andrei Vagin Date: Thu, 13 Aug 2020 22:23:40 +0000 Subject: Implement FUSE_LOOKUP Fixes #3231 Co-authored-by: Boyuan He --- pkg/abi/linux/fuse.go | 56 ++++++++++++++++++++++++ pkg/sentry/fsimpl/fuse/fusefs.go | 80 +++++++++++++++++++++++++++++++--- pkg/sentry/fsimpl/kernfs/filesystem.go | 2 +- pkg/sentry/fsimpl/kernfs/kernfs.go | 6 +-- 4 files changed, 133 insertions(+), 11 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index 7e30483ee..346a9e6fc 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -14,12 +14,17 @@ package linux +import "gvisor.dev/gvisor/tools/go_marshal/marshal" + // +marshal type FUSEOpcode uint32 // +marshal type FUSEOpID uint64 +// FUSE_ROOT_ID is the id of root inode. +const FUSE_ROOT_ID = 1 + // Opcodes for FUSE operations. Analogous to the opcodes in include/linux/fuse.h. const ( FUSE_LOOKUP FUSEOpcode = 1 @@ -301,3 +306,54 @@ type FUSEGetAttrOut struct { // Attr contains the metadata returned from the FUSE server Attr FUSEAttr } + +// FUSEEntryOut is the reply sent by the daemon to the kernel +// for FUSE_MKNOD, FUSE_MKDIR, FUSE_SYMLINK, FUSE_LINK and +// FUSE_LOOKUP. +// +// +marshal +type FUSEEntryOut struct { + // NodeID is the ID for current inode. + NodeID uint64 + + // Generation is the generation number of inode. + // Used to identify an inode that have different ID at different time. + Generation uint64 + + // EntryValid indicates timeout for an entry. + EntryValid uint64 + + // AttrValid indicates timeout for an entry's attributes. + AttrValid uint64 + + // EntryValidNsec indicates timeout for an entry in nanosecond. + EntryValidNSec uint32 + + // AttrValidNsec indicates timeout for an entry's attributes in nanosecond. + AttrValidNSec uint32 + + // Attr contains the attributes of an entry. + Attr FUSEAttr +} + +// FUSELookupIn is the request sent by the kernel to the daemon +// to look up a file name. +// +// Dynamically-sized objects cannot be marshalled. +type FUSELookupIn struct { + marshal.StubMarshallable + + // Name is a file name to be looked up. + Name string +} + +// MarshalUnsafe serializes r.name to the dst buffer. +func (r *FUSELookupIn) MarshalUnsafe(buf []byte) { + copy(buf, []byte(r.Name)) +} + +// SizeBytes is the size of the memory representation of FUSELookupIn. +// 1 extra byte for null-terminated string. +func (r *FUSELookupIn) SizeBytes() int { + return len(r.Name) + 1 +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 810819ae4..cee5acb3f 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -26,6 +26,7 @@ import ( "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/tools/go_marshal/marshal" ) // Name is the default filesystem name. @@ -165,7 +166,7 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt } // root is the fusefs root directory. - root := fs.newInode(creds, fsopts.rootMode) + root := fs.newRootInode(creds, fsopts.rootMode) return fs.VFSFilesystem(), root.VFSDentry(), nil } @@ -205,14 +206,28 @@ type inode struct { kernfs.InodeNotSymlink kernfs.OrderedChildren - locks vfs.FileLocks - + NodeID uint64 dentry kernfs.Dentry + locks vfs.FileLocks + + // the owning filesystem. fs is immutable. + fs *filesystem } -func (fs *filesystem) newInode(creds *auth.Credentials, mode linux.FileMode) *kernfs.Dentry { - i := &inode{} - i.InodeAttrs.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.ModeDirectory|0755) +func (fs *filesystem) newRootInode(creds *auth.Credentials, mode linux.FileMode) *kernfs.Dentry { + i := &inode{fs: fs} + i.InodeAttrs.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, 1, linux.ModeDirectory|0755) + i.OrderedChildren.Init(kernfs.OrderedChildrenOptions{}) + i.dentry.Init(i) + i.NodeID = 1 + + return &i.dentry +} + +func (fs *filesystem) newInode(nodeID uint64, attr linux.FUSEAttr) *kernfs.Dentry { + i := &inode{fs: fs, NodeID: nodeID} + creds := auth.Credentials{EffectiveKGID: auth.KGID(attr.UID), EffectiveKUID: auth.KUID(attr.UID)} + i.InodeAttrs.Init(&creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.FileMode(attr.Mode)) i.OrderedChildren.Init(kernfs.OrderedChildrenOptions{}) i.EnableLeakCheck() i.dentry.Init(i) @@ -231,6 +246,57 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr return fd.VFSFileDescription(), nil } +// Lookup implements kernfs.Inode.Lookup. +func (i *inode) Lookup(ctx context.Context, name string) (*vfs.Dentry, error) { + in := linux.FUSELookupIn{Name: name} + return i.newEntry(ctx, name, 0, linux.FUSE_LOOKUP, &in) +} + +// IterDirents implements Inode.IterDirents. +func (inode) IterDirents(ctx context.Context, callback vfs.IterDirentsCallback, offset, relOffset int64) (int64, error) { + return offset, nil +} + +// Valid implements Inode.Valid. +func (inode) Valid(ctx context.Context) bool { + return true +} + +// newEntry calls FUSE server for entry creation and allocates corresponding entry according to response. +// Shared by FUSE_MKNOD, FUSE_MKDIR, FUSE_SYMLINK, FUSE_LINK and FUSE_LOOKUP. +func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMode, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*vfs.Dentry, error) { + kernelTask := kernel.TaskFromContext(ctx) + if kernelTask == nil { + log.Warningf("fusefs.Inode.newEntry: couldn't get kernel task from context", i.NodeID) + return nil, syserror.EINVAL + } + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, opcode, payload) + if err != nil { + return nil, err + } + res, err := i.fs.conn.Call(kernelTask, req) + if err != nil { + return nil, err + } + if err := res.Error(); err != nil { + return nil, err + } + out := linux.FUSEEntryOut{} + if err := res.UnmarshalPayload(&out); err != nil { + return nil, err + } + if opcode != linux.FUSE_LOOKUP && ((out.Attr.Mode&linux.S_IFMT)^uint32(fileType) != 0 || out.NodeID == 0 || out.NodeID == linux.FUSE_ROOT_ID) { + return nil, syserror.EIO + } + child := i.fs.newInode(out.NodeID, out.Attr) + if opcode == linux.FUSE_LOOKUP { + i.dentry.InsertChildLocked(name, child) + } else { + i.dentry.InsertChild(name, child) + } + return child.VFSDentry(), nil +} + // statFromFUSEAttr makes attributes from linux.FUSEAttr to linux.Statx. The // opts.Sync attribute is ignored since the synchronization is handled by the // FUSE server. @@ -299,7 +365,7 @@ func (i *inode) Stat(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOptio // finally be translated into vfs.FilesystemImpl.StatAt() (see // pkg/sentry/syscalls/linux/vfs2/stat.go), resulting in the same flow // as stat(2). Thus GetAttrFlags and Fh variable will never be used in VFS2. - req, err := conn.NewRequest(creds, uint32(task.ThreadID()), i.Ino(), linux.FUSE_GETATTR, &in) + req, err := conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_GETATTR, &in) if err != nil { return linux.Statx{}, err } diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go index d7d3e8f48..904203070 100644 --- a/pkg/sentry/fsimpl/kernfs/filesystem.go +++ b/pkg/sentry/fsimpl/kernfs/filesystem.go @@ -140,7 +140,7 @@ func (fs *Filesystem) revalidateChildLocked(ctx context.Context, vfsObj *vfs.Vir } // Reference on childVFSD dropped by a corresponding Valid. child = childVFSD.Impl().(*Dentry) - parent.insertChildLocked(name, child) + parent.InsertChildLocked(name, child) } return child, nil } diff --git a/pkg/sentry/fsimpl/kernfs/kernfs.go b/pkg/sentry/fsimpl/kernfs/kernfs.go index 88fcd54aa..67a0347fe 100644 --- a/pkg/sentry/fsimpl/kernfs/kernfs.go +++ b/pkg/sentry/fsimpl/kernfs/kernfs.go @@ -246,15 +246,15 @@ func (d *Dentry) OnZeroWatches(context.Context) {} // Precondition: d must represent a directory inode. func (d *Dentry) InsertChild(name string, child *Dentry) { d.dirMu.Lock() - d.insertChildLocked(name, child) + d.InsertChildLocked(name, child) d.dirMu.Unlock() } -// insertChildLocked is equivalent to InsertChild, with additional +// InsertChildLocked is equivalent to InsertChild, with additional // preconditions. // // Precondition: d.dirMu must be locked. -func (d *Dentry) insertChildLocked(name string, child *Dentry) { +func (d *Dentry) InsertChildLocked(name string, child *Dentry) { if !d.isDir() { panic(fmt.Sprintf("InsertChild called on non-directory Dentry: %+v.", d)) } -- cgit v1.2.3 From 32044f94e9dfbb88c17d07b235b8ed5b07d2ff18 Mon Sep 17 00:00:00 2001 From: Boyuan He Date: Tue, 18 Aug 2020 01:46:39 +0000 Subject: Implement FUSE_OPEN/OPENDIR Fixes #3174 --- pkg/abi/linux/fuse.go | 38 +++++++++++ pkg/sentry/fsimpl/fuse/BUILD | 1 + pkg/sentry/fsimpl/fuse/connection.go | 16 ++++- pkg/sentry/fsimpl/fuse/file.go | 96 +++++++++++++++++++++++++++ pkg/sentry/fsimpl/fuse/fusefs.go | 97 +++++++++++++++++++++++++-- test/fuse/BUILD | 5 ++ test/fuse/linux/BUILD | 13 ++++ test/fuse/linux/fuse_base.cc | 50 +++++++------- test/fuse/linux/fuse_base.h | 9 +++ test/fuse/linux/open_test.cc | 124 +++++++++++++++++++++++++++++++++++ test/util/BUILD | 1 + test/util/fuse_util.cc | 59 +++++++++++++++++ test/util/fuse_util.h | 4 ++ 13 files changed, 481 insertions(+), 32 deletions(-) create mode 100644 pkg/sentry/fsimpl/fuse/file.go create mode 100644 test/fuse/linux/open_test.cc create mode 100644 test/util/fuse_util.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index 346a9e6fc..e09715ecd 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -357,3 +357,41 @@ func (r *FUSELookupIn) MarshalUnsafe(buf []byte) { func (r *FUSELookupIn) SizeBytes() int { return len(r.Name) + 1 } + +// MAX_NON_LFS indicates the maximum offset without large file support. +const MAX_NON_LFS = ((1 << 31) - 1) + +// flags returned by OPEN request. +const ( + // FOPEN_DIRECT_IO indicates bypassing page cache for this opened file. + FOPEN_DIRECT_IO = 1 << 0 + // FOPEN_KEEP_CACHE avoids invalidate of data cache on open. + FOPEN_KEEP_CACHE = 1 << 1 + // FOPEN_NONSEEKABLE indicates the file cannot be seeked. + FOPEN_NONSEEKABLE = 1 << 2 +) + +// FUSEOpenIn is the request sent by the kernel to the daemon, +// to negotiate flags and get file handle. +// +// +marshal +type FUSEOpenIn struct { + // Flags of this open request. + Flags uint32 + + _ uint32 +} + +// FUSEOpenOut is the reply sent by the daemon to the kernel +// for FUSEOpenIn. +// +// +marshal +type FUSEOpenOut struct { + // Fh is the file handler for opened file. + Fh uint64 + + // OpenFlag for the opened file. + OpenFlag uint32 + + _ uint32 +} diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index 999c16bfd..75a2e96ff 100644 --- a/pkg/sentry/fsimpl/fuse/BUILD +++ b/pkg/sentry/fsimpl/fuse/BUILD @@ -31,6 +31,7 @@ go_library( srcs = [ "connection.go", "dev.go", + "file.go", "fusefs.go", "init.go", "inode_refs.go", diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index eb1d1a2b7..9cfd67158 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -78,8 +78,13 @@ type Response struct { type connection struct { fd *DeviceFD + // mu protect access to struct memebers. + mu sync.Mutex + + // attributeVersion is the version of connection's attributes. + attributeVersion uint64 + // The following FUSE_INIT flags are currently unsupported by this implementation: - // - FUSE_ATOMIC_O_TRUNC: requires open(..., O_TRUNC) // - FUSE_EXPORT_SUPPORT // - FUSE_HANDLE_KILLPRIV // - FUSE_POSIX_LOCKS: requires POSIX locks @@ -113,6 +118,11 @@ type connection struct { // TODO(gvisor.dev/issue/3185): abort all queued requests. aborted bool + // atomicOTrunc is true when FUSE does not send a separate SETATTR request + // before open with O_TRUNC flag. + // Negotiated and only set in INIT. + atomicOTrunc bool + // connInitError if FUSE_INIT encountered error (major version mismatch). // Only set in INIT. connInitError bool @@ -189,6 +199,10 @@ type connection struct { // dontMask if filestestem does not apply umask to creation modes. // Negotiated in INIT. dontMask bool + + // noOpen if FUSE server doesn't support open operation. + // This flag only influence performance, not correctness of the program. + noOpen bool } // newFUSEConnection creates a FUSE connection to fd. diff --git a/pkg/sentry/fsimpl/fuse/file.go b/pkg/sentry/fsimpl/fuse/file.go new file mode 100644 index 000000000..ab60ab714 --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/file.go @@ -0,0 +1,96 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fuse + +import ( + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/usermem" +) + +// fileDescription implements vfs.FileDescriptionImpl for fuse. +type fileDescription struct { + vfsfd vfs.FileDescription + vfs.FileDescriptionDefaultImpl + vfs.DentryMetadataFileDescriptionImpl + vfs.NoLockFD + + // the file handle used in userspace. + Fh uint64 + + // Nonseekable is indicate cannot perform seek on a file. + Nonseekable bool + + // DirectIO suggest fuse to use direct io operation. + DirectIO bool + + // OpenFlag is the flag returned by open. + OpenFlag uint32 +} + +func (fd *fileDescription) dentry() *kernfs.Dentry { + return fd.vfsfd.Dentry().Impl().(*kernfs.Dentry) +} + +func (fd *fileDescription) inode() *inode { + return fd.dentry().Inode().(*inode) +} + +func (fd *fileDescription) filesystem() *vfs.Filesystem { + return fd.vfsfd.VirtualDentry().Mount().Filesystem() +} + +// Release implements vfs.FileDescriptionImpl.Release. +func (fd *fileDescription) Release(ctx context.Context) {} + +// PRead implements vfs.FileDescriptionImpl.PRead. +func (fd *fileDescription) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) { + return 0, nil +} + +// Read implements vfs.FileDescriptionImpl.Read. +func (fd *fileDescription) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { + return 0, nil +} + +// PWrite implements vfs.FileDescriptionImpl.PWrite. +func (fd *fileDescription) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) { + return 0, nil +} + +// Write implements vfs.FileDescriptionImpl.Write. +func (fd *fileDescription) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { + return 0, nil +} + +// Seek implements vfs.FileDescriptionImpl.Seek. +func (fd *fileDescription) Seek(ctx context.Context, offset int64, whence int32) (int64, error) { + return 0, nil +} + +// Stat implements FileDescriptionImpl.Stat. +// Stat implements vfs.FileDescriptionImpl.Stat. +func (fd *fileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linux.Statx, error) { + fs := fd.filesystem() + inode := fd.inode() + return inode.Stat(ctx, fs, opts) +} + +// SetStat implements FileDescriptionImpl.SetStat. +func (fd *fileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions) error { + return nil +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index cee5acb3f..b5f05b80b 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -17,6 +17,7 @@ package fuse import ( "strconv" + "sync/atomic" "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" @@ -212,6 +213,18 @@ type inode struct { // the owning filesystem. fs is immutable. fs *filesystem + + // size of the file. + size uint64 + + // attributeVersion is the version of inode's attributes. + attributeVersion uint64 + + // attributeTime is the remaining vaild time of attributes. + attributeTime uint64 + + // version of the inode. + version uint64 } func (fs *filesystem) newRootInode(creds *auth.Credentials, mode linux.FileMode) *kernfs.Dentry { @@ -237,13 +250,87 @@ func (fs *filesystem) newInode(nodeID uint64, attr linux.FUSEAttr) *kernfs.Dentr // Open implements kernfs.Inode.Open. func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) { - fd, err := kernfs.NewGenericDirectoryFD(rp.Mount(), vfsd, &i.OrderedChildren, &i.locks, &opts, kernfs.GenericDirectoryFDOptions{ - SeekEnd: kernfs.SeekEndStaticEntries, - }) - if err != nil { + isDir := i.InodeAttrs.Mode().IsDir() + // return error if specified to open directory but inode is not a directory. + if !isDir && opts.Mode.IsDir() { + return nil, syserror.ENOTDIR + } + if opts.Flags&linux.O_LARGEFILE == 0 && atomic.LoadUint64(&i.size) > linux.MAX_NON_LFS { + return nil, syserror.EOVERFLOW + } + + // FOPEN_KEEP_CACHE is the defualt flag for noOpen. + fd := fileDescription{OpenFlag: linux.FOPEN_KEEP_CACHE} + // Only send open request when FUSE server support open or is opening a directory. + if !i.fs.conn.noOpen || isDir { + kernelTask := kernel.TaskFromContext(ctx) + if kernelTask == nil { + log.Warningf("fusefs.Inode.Open: couldn't get kernel task from context") + return nil, syserror.EINVAL + } + + var opcode linux.FUSEOpcode + if isDir { + opcode = linux.FUSE_OPENDIR + } else { + opcode = linux.FUSE_OPEN + } + in := linux.FUSEOpenIn{Flags: opts.Flags & ^uint32(linux.O_CREAT|linux.O_EXCL|linux.O_NOCTTY)} + if !i.fs.conn.atomicOTrunc { + in.Flags &= ^uint32(linux.O_TRUNC) + } + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, opcode, &in) + if err != nil { + return nil, err + } + + res, err := i.fs.conn.Call(kernelTask, req) + if err != nil { + return nil, err + } + if err := res.Error(); err == syserror.ENOSYS && !isDir { + i.fs.conn.noOpen = true + } else if err != nil { + return nil, err + } else { + out := linux.FUSEOpenOut{} + if err := res.UnmarshalPayload(&out); err != nil { + return nil, err + } + fd.OpenFlag = out.OpenFlag + fd.Fh = out.Fh + } + } + + if isDir { + fd.OpenFlag &= ^uint32(linux.FOPEN_DIRECT_IO) + } + + // TODO(gvisor.dev/issue/3234): invalidate mmap after implemented it for FUSE Inode + fd.DirectIO = fd.OpenFlag&linux.FOPEN_DIRECT_IO != 0 + fdOptions := &vfs.FileDescriptionOptions{} + if fd.OpenFlag&linux.FOPEN_NONSEEKABLE != 0 { + fdOptions.DenyPRead = true + fdOptions.DenyPWrite = true + fd.Nonseekable = true + } + + // If we don't send SETATTR before open (which is indicated by atomicOTrunc) + // and O_TRUNC is set, update the inode's version number and clean existing data + // by setting the file size to 0. + if i.fs.conn.atomicOTrunc && opts.Flags&linux.O_TRUNC != 0 { + i.fs.conn.mu.Lock() + i.fs.conn.attributeVersion++ + i.attributeVersion = i.fs.conn.attributeVersion + atomic.StoreUint64(&i.size, 0) + i.fs.conn.mu.Unlock() + i.attributeTime = 0 + } + + if err := fd.vfsfd.Init(&fd, opts.Flags, rp.Mount(), vfsd, fdOptions); err != nil { return nil, err } - return fd.VFSFileDescription(), nil + return &fd.vfsfd, nil } // Lookup implements kernfs.Inode.Lookup. diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 385920e17..209030ff1 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -6,3 +6,8 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:stat_test", ) + +syscall_test( + fuse = "True", + test = "//test/fuse/linux:open_test", +) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index e4a614e11..1d989a2f4 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -18,6 +18,19 @@ cc_binary( ], ) +cc_binary( + name = "open_test", + testonly = 1, + srcs = ["open_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_library( name = "fuse_base", testonly = 1, diff --git a/test/fuse/linux/fuse_base.cc b/test/fuse/linux/fuse_base.cc index a9fe1044e..e734100b1 100644 --- a/test/fuse/linux/fuse_base.cc +++ b/test/fuse/linux/fuse_base.cc @@ -117,6 +117,16 @@ uint32_t FuseTest::GetServerTotalReceivedBytes() { static_cast(FuseTestCmd::kGetTotalReceivedBytes)); } +// Sends the `kSkipRequest` command to the FUSE server, which would skip +// current stored request data. +void FuseTest::SkipServerActualRequest() { + uint32_t cmd = static_cast(FuseTestCmd::kSkipRequest); + EXPECT_THAT(RetryEINTR(write)(sock_[0], &cmd, sizeof(cmd)), + SyscallSucceedsWithValue(sizeof(cmd))); + + WaitServerComplete(); +} + // Sends the `kSetInodeLookup` command, expected mode, and the path of the // inode to create under the mount point. void FuseTest::SetServerInodeLookup(const std::string& path, mode_t mode) { @@ -284,6 +294,9 @@ void FuseTest::ServerHandleCommand() { case FuseTestCmd::kGetNumUnsentResponses: ServerSendData(static_cast(responses_.RemainingBlocks())); break; + case FuseTestCmd::kSkipRequest: + ServerSkipReceivedRequest(); + break; default: FAIL() << "Unknown FuseTestCmd " << cmd; break; @@ -314,32 +327,7 @@ void FuseTest::ServerReceiveInodeLookup() { .len = out_len, .error = 0, }; - struct fuse_entry_out out_payload = { - .nodeid = nodeid_, - .generation = 0, - .entry_valid = 0, - .attr_valid = 0, - .entry_valid_nsec = 0, - .attr_valid_nsec = 0, - .attr = - (struct fuse_attr){ - .ino = nodeid_, - .size = 512, - .blocks = 4, - .atime = 0, - .mtime = 0, - .ctime = 0, - .atimensec = 0, - .mtimensec = 0, - .ctimensec = 0, - .mode = mode, - .nlink = 2, - .uid = 1234, - .gid = 4321, - .rdev = 12, - .blksize = 4096, - }, - }; + struct fuse_entry_out out_payload = DefaultEntryOut(mode, nodeid_); // Since this is only used in test, nodeid_ is simply increased by 1 to // comply with the unqiueness of different path. ++nodeid_; @@ -363,6 +351,15 @@ void FuseTest::ServerSendReceivedRequest() { SyscallSucceedsWithValue(mem_block.len)); } +// Skip the request pointed by current cursor. +void FuseTest::ServerSkipReceivedRequest() { + if (requests_.End()) { + FAIL() << "No more received request."; + return; + } + requests_.Next(); +} + // Handles FUSE request. Reads request from /dev/fuse, checks if it has the // same opcode as expected, and responds with the saved fake FUSE response. // The FUSE request is copied to the serial buffer and can be retrieved one- @@ -390,6 +387,7 @@ void FuseTest::ServerProcessFuseRequest() { requests_.AddMemBlock(in_header->opcode, buf.data(), len); + if (in_header->opcode == FUSE_RELEASE) return; // Check if there is a corresponding response. if (responses_.End()) { GTEST_NONFATAL_FAILURE_("No more FUSE response is expected"); diff --git a/test/fuse/linux/fuse_base.h b/test/fuse/linux/fuse_base.h index a21b4bb8d..2474b763f 100644 --- a/test/fuse/linux/fuse_base.h +++ b/test/fuse/linux/fuse_base.h @@ -42,6 +42,7 @@ enum class FuseTestCmd { kGetNumUnconsumedRequests, kGetNumUnsentResponses, kGetTotalReceivedBytes, + kSkipRequest, }; // Holds the information of a memory block in a serial buffer. @@ -158,6 +159,10 @@ class FuseTest : public ::testing::Test { // bytes from /dev/fuse. uint32_t GetServerTotalReceivedBytes(); + // Called by the testing thread to ask the FUSE server to skip stored + // request data. + void SkipServerActualRequest(); + protected: TempPath mount_point_; @@ -211,6 +216,10 @@ class FuseTest : public ::testing::Test { // Sends a uint32_t data via socket. inline void ServerSendData(uint32_t data); + // The FUSE server side's corresponding code of `SkipServerActualRequest()`. + // Handles `kSkipRequest` command. Skip the request pointed by current cursor. + void ServerSkipReceivedRequest(); + // Handles FUSE request sent to /dev/fuse by its saved responses. void ServerProcessFuseRequest(); diff --git a/test/fuse/linux/open_test.cc b/test/fuse/linux/open_test.cc new file mode 100644 index 000000000..ed0641587 --- /dev/null +++ b/test/fuse/linux/open_test.cc @@ -0,0 +1,124 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class OpenTest : public FuseTest { + protected: + const std::string test_file_ = "test_file"; + const mode_t regular_file_ = S_IFREG | S_IRWXU | S_IRWXG | S_IRWXO; + + struct fuse_open_out out_payload_ = { + .fh = 1, + .open_flags = O_RDWR, + }; +}; + +TEST_F(OpenTest, RegularFile) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + SetServerInodeLookup(test_file_, regular_file_); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_open_out), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload_); + SetServerResponse(FUSE_OPEN, iov_out); + FileDescriptor fd = + ASSERT_NO_ERRNO_AND_VALUE(Open(test_file_path.c_str(), O_RDWR)); + + struct fuse_in_header in_header; + struct fuse_open_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_OPEN); + EXPECT_EQ(in_payload.flags, O_RDWR); + EXPECT_THAT(fcntl(fd.get(), F_GETFL), SyscallSucceedsWithValue(O_RDWR)); +} + +TEST_F(OpenTest, SetNoOpen) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + SetServerInodeLookup(test_file_, regular_file_); + + // ENOSYS indicates open is not implemented. + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_open_out), + .error = -ENOSYS, + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload_); + SetServerResponse(FUSE_OPEN, iov_out); + ASSERT_NO_ERRNO_AND_VALUE(Open(test_file_path.c_str(), O_RDWR)); + SkipServerActualRequest(); + + // check open doesn't send new request. + uint32_t recieved_before = GetServerTotalReceivedBytes(); + ASSERT_NO_ERRNO_AND_VALUE(Open(test_file_path.c_str(), O_RDWR)); + EXPECT_EQ(GetServerTotalReceivedBytes(), recieved_before); +} + +TEST_F(OpenTest, OpenFail) { + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_open_out), + .error = -ENOENT, + }; + + auto iov_out = FuseGenerateIovecs(out_header, out_payload_); + SetServerResponse(FUSE_OPENDIR, iov_out); + ASSERT_THAT(open(mount_point_.path().c_str(), O_RDWR), + SyscallFailsWithErrno(ENOENT)); + + struct fuse_in_header in_header; + struct fuse_open_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_OPENDIR); + EXPECT_EQ(in_payload.flags, O_RDWR); +} + +TEST_F(OpenTest, DirectoryFlagOnRegularFile) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + + SetServerInodeLookup(test_file_, regular_file_); + ASSERT_THAT(open(test_file_path.c_str(), O_RDWR | O_DIRECTORY), + SyscallFailsWithErrno(ENOTDIR)); +} + +} // namespace + +} // namespace testing +} // namespace gvisor diff --git a/test/util/BUILD b/test/util/BUILD index b0c2c2a5a..fc5fb3a8d 100644 --- a/test/util/BUILD +++ b/test/util/BUILD @@ -48,6 +48,7 @@ cc_library( cc_library( name = "fuse_util", testonly = 1, + srcs = ["fuse_util.cc"], hdrs = ["fuse_util.h"], ) diff --git a/test/util/fuse_util.cc b/test/util/fuse_util.cc new file mode 100644 index 000000000..5b10a9e45 --- /dev/null +++ b/test/util/fuse_util.cc @@ -0,0 +1,59 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "test/util/fuse_util.h" + +#include +#include + +#include + +namespace gvisor { +namespace testing { + +// Create response body with specified mode and nodeID. +fuse_entry_out DefaultEntryOut(mode_t mode, uint64_t node_id) { + const int time_sec = 1595436289; + const int time_nsec = 134150844; + struct fuse_entry_out default_entry_out = { + .nodeid = node_id, + .generation = 0, + .entry_valid = time_sec, + .attr_valid = time_sec, + .entry_valid_nsec = time_nsec, + .attr_valid_nsec = time_nsec, + .attr = + (struct fuse_attr){ + .ino = node_id, + .size = 512, + .blocks = 4, + .atime = time_sec, + .mtime = time_sec, + .ctime = time_sec, + .atimensec = time_nsec, + .mtimensec = time_nsec, + .ctimensec = time_nsec, + .mode = mode, + .nlink = 2, + .uid = 1234, + .gid = 4321, + .rdev = 12, + .blksize = 4096, + }, + }; + return default_entry_out; +}; + +} // namespace testing +} // namespace gvisor diff --git a/test/util/fuse_util.h b/test/util/fuse_util.h index 5f5182b96..1f1bf64a4 100644 --- a/test/util/fuse_util.h +++ b/test/util/fuse_util.h @@ -15,6 +15,7 @@ #ifndef GVISOR_TEST_UTIL_FUSE_UTIL_H_ #define GVISOR_TEST_UTIL_FUSE_UTIL_H_ +#include #include #include @@ -62,6 +63,9 @@ std::vector FuseGenerateIovecs(T &first, Types &...args) { return first_iovec; } +// Return a fuse_entry_out FUSE server response body. +fuse_entry_out DefaultEntryOut(mode_t mode, uint64_t nodeId); + } // namespace testing } // namespace gvisor #endif // GVISOR_TEST_UTIL_FUSE_UTIL_H_ -- cgit v1.2.3 From 947088e10a15b5236f2af3206f67f27245ef2770 Mon Sep 17 00:00:00 2001 From: Boyuan He Date: Tue, 18 Aug 2020 20:59:28 +0000 Subject: Implement FUSE_RELEASE/RELEASEDIR Fixes #3314 --- pkg/abi/linux/fuse.go | 18 ++++++++++ pkg/sentry/fsimpl/fuse/dev.go | 6 ++++ pkg/sentry/fsimpl/fuse/file.go | 31 +++++++++++++++-- test/fuse/BUILD | 5 +++ test/fuse/linux/BUILD | 13 ++++++++ test/fuse/linux/fuse_base.cc | 3 +- test/fuse/linux/fuse_base.h | 6 ++-- test/fuse/linux/open_test.cc | 4 +++ test/fuse/linux/release_test.cc | 74 +++++++++++++++++++++++++++++++++++++++++ 9 files changed, 154 insertions(+), 6 deletions(-) create mode 100644 test/fuse/linux/release_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index e09715ecd..a1b2a2abf 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -395,3 +395,21 @@ type FUSEOpenOut struct { _ uint32 } + +// FUSEReleaseIn is the request sent by the kernel to the daemon +// when there is no more reference to a file. +// +// +marshal +type FUSEReleaseIn struct { + // Fh is the file handler for the file to be released. + Fh uint64 + + // Flags of the file. + Flags uint32 + + // ReleaseFlags of this release request. + ReleaseFlags uint32 + + // LockOwner is the id of the lock owner if there is one. + LockOwner uint64 +} diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index 0efd2d90d..e2de8e097 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -168,6 +168,9 @@ func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts // We're done with this request. fd.queue.Remove(req) + if req.hdr.Opcode == linux.FUSE_RELEASE { + fd.numActiveRequests -= 1 + } // Restart the read as this request was invalid. log.Warningf("fuse.DeviceFD.Read: request found was too large. Restarting read.") @@ -184,6 +187,9 @@ func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts if readCursor >= req.hdr.Len { // Fully done with this req, remove it from the queue. fd.queue.Remove(req) + if req.hdr.Opcode == linux.FUSE_RELEASE { + fd.numActiveRequests -= 1 + } break } } diff --git a/pkg/sentry/fsimpl/fuse/file.go b/pkg/sentry/fsimpl/fuse/file.go index ab60ab714..01d20caf6 100644 --- a/pkg/sentry/fsimpl/fuse/file.go +++ b/pkg/sentry/fsimpl/fuse/file.go @@ -18,6 +18,8 @@ import ( "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/usermem" ) @@ -54,8 +56,34 @@ func (fd *fileDescription) filesystem() *vfs.Filesystem { return fd.vfsfd.VirtualDentry().Mount().Filesystem() } +func (fd *fileDescription) statusFlags() uint32 { + return fd.vfsfd.StatusFlags() +} + // Release implements vfs.FileDescriptionImpl.Release. -func (fd *fileDescription) Release(ctx context.Context) {} +func (fd *fileDescription) Release(ctx context.Context) { + // no need to release if FUSE server doesn't implement Open. + conn := fd.inode().fs.conn + if conn.noOpen { + return + } + + in := linux.FUSEReleaseIn{ + Fh: fd.Fh, + Flags: fd.statusFlags(), + } + // TODO(gvisor.dev/issue/3245): add logic when we support file lock owner. + var opcode linux.FUSEOpcode + if fd.inode().Mode().IsDir() { + opcode = linux.FUSE_RELEASEDIR + } else { + opcode = linux.FUSE_RELEASE + } + kernelTask := kernel.TaskFromContext(ctx) + // ignoring errors and FUSE server reply is analogous to Linux's behavior. + req, _ := conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), fd.inode().NodeID, opcode, &in) + conn.CallAsync(kernelTask, req) +} // PRead implements vfs.FileDescriptionImpl.PRead. func (fd *fileDescription) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) { @@ -82,7 +110,6 @@ func (fd *fileDescription) Seek(ctx context.Context, offset int64, whence int32) return 0, nil } -// Stat implements FileDescriptionImpl.Stat. // Stat implements vfs.FileDescriptionImpl.Stat. func (fd *fileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linux.Statx, error) { fs := fd.filesystem() diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 209030ff1..53cbadb3c 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -11,3 +11,8 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:open_test", ) + +syscall_test( + fuse = "True", + test = "//test/fuse/linux:release_test", +) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index 1d989a2f4..abee1a33c 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -31,6 +31,19 @@ cc_binary( ], ) +cc_binary( + name = "release_test", + testonly = 1, + srcs = ["release_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_library( name = "fuse_base", testonly = 1, diff --git a/test/fuse/linux/fuse_base.cc b/test/fuse/linux/fuse_base.cc index e734100b1..98b4e1466 100644 --- a/test/fuse/linux/fuse_base.cc +++ b/test/fuse/linux/fuse_base.cc @@ -387,7 +387,8 @@ void FuseTest::ServerProcessFuseRequest() { requests_.AddMemBlock(in_header->opcode, buf.data(), len); - if (in_header->opcode == FUSE_RELEASE) return; + if (in_header->opcode == FUSE_RELEASE || in_header->opcode == FUSE_RELEASEDIR) + return; // Check if there is a corresponding response. if (responses_.End()) { GTEST_NONFATAL_FAILURE_("No more FUSE response is expected"); diff --git a/test/fuse/linux/fuse_base.h b/test/fuse/linux/fuse_base.h index 2474b763f..ff4c4499d 100644 --- a/test/fuse/linux/fuse_base.h +++ b/test/fuse/linux/fuse_base.h @@ -166,13 +166,13 @@ class FuseTest : public ::testing::Test { protected: TempPath mount_point_; + // Unmounts the mountpoint of the FUSE server. + void UnmountFuse(); + private: // Opens /dev/fuse and inherit the file descriptor for the FUSE server. void MountFuse(); - // Unmounts the mountpoint of the FUSE server. - void UnmountFuse(); - // Creates a socketpair for communication and forks FUSE server. void SetUpFuseServer(); diff --git a/test/fuse/linux/open_test.cc b/test/fuse/linux/open_test.cc index ed0641587..4b0c4a805 100644 --- a/test/fuse/linux/open_test.cc +++ b/test/fuse/linux/open_test.cc @@ -33,6 +33,10 @@ namespace testing { namespace { class OpenTest : public FuseTest { + // OpenTest doesn't care the release request when close a fd, + // so doesn't check leftover requests when tearing down. + void TearDown() { UnmountFuse(); } + protected: const std::string test_file_ = "test_file"; const mode_t regular_file_ = S_IFREG | S_IRWXU | S_IRWXG | S_IRWXO; diff --git a/test/fuse/linux/release_test.cc b/test/fuse/linux/release_test.cc new file mode 100644 index 000000000..b5adb0870 --- /dev/null +++ b/test/fuse/linux/release_test.cc @@ -0,0 +1,74 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class ReleaseTest : public FuseTest { + protected: + const std::string test_file_ = "test_file"; +}; + +TEST_F(ReleaseTest, RegularFile) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + SetServerInodeLookup(test_file_, S_IFREG | S_IRWXU | S_IRWXG | S_IRWXO); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_open_out), + }; + struct fuse_open_out out_payload = { + .fh = 1, + .open_flags = O_RDWR, + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_OPEN, iov_out); + FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(Open(test_file_path, O_RDWR)); + SkipServerActualRequest(); + ASSERT_THAT(close(fd.release()), SyscallSucceeds()); + + struct fuse_in_header in_header; + struct fuse_release_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_RELEASE); + EXPECT_EQ(in_payload.flags, O_RDWR); + EXPECT_EQ(in_payload.fh, 1); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From b53e10f391929e4ad9345a823a0cf33bcfedf413 Mon Sep 17 00:00:00 2001 From: Boyuan He Date: Tue, 18 Aug 2020 21:51:06 +0000 Subject: Implement FUSE_MKNOD Fixes #3492 --- pkg/abi/linux/fuse.go | 43 ++++++++++++++++ pkg/sentry/fsimpl/fuse/fusefs.go | 13 +++++ test/fuse/BUILD | 5 ++ test/fuse/linux/BUILD | 14 +++++ test/fuse/linux/mknod_test.cc | 107 +++++++++++++++++++++++++++++++++++++++ 5 files changed, 182 insertions(+) create mode 100644 test/fuse/linux/mknod_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index a1b2a2abf..97d960096 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -413,3 +413,46 @@ type FUSEReleaseIn struct { // LockOwner is the id of the lock owner if there is one. LockOwner uint64 } + +// FUSEMknodMeta contains all the static fields of FUSEMknodIn, +// which is used for FUSE_MKNOD. +// +// +marshal +type FUSEMknodMeta struct { + // Mode of the inode to create. + Mode uint32 + + // Rdev encodes device major and minor information. + Rdev uint32 + + // Umask is the current file mode creation mask. + Umask uint32 + + _ uint32 +} + +// FUSEMknodIn contains all the arguments sent by the kernel +// to the daemon, to create a new file node. +// +// Dynamically-sized objects cannot be marshalled. +type FUSEMknodIn struct { + marshal.StubMarshallable + + // MknodMeta contains mode, rdev and umash field for FUSE_MKNODS. + MknodMeta FUSEMknodMeta + + // Name is the name of the node to create. + Name string +} + +// MarshalUnsafe serializes r.MknodMeta and r.Name to the dst buffer. +func (r *FUSEMknodIn) MarshalUnsafe(buf []byte) { + r.MknodMeta.MarshalUnsafe(buf[:r.MknodMeta.SizeBytes()]) + copy(buf[r.MknodMeta.SizeBytes():], r.Name) +} + +// SizeBytes is the size of the memory representation of FUSEMknodIn. +// 1 extra byte for null-terminated string. +func (r *FUSEMknodIn) SizeBytes() int { + return r.MknodMeta.SizeBytes() + len(r.Name) + 1 +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index b5f05b80b..5cef0b94f 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -349,6 +349,19 @@ func (inode) Valid(ctx context.Context) bool { return true } +// NewNode implements kernfs.Inode.NewNode. +func (i *inode) NewNode(ctx context.Context, name string, opts vfs.MknodOptions) (*vfs.Dentry, error) { + in := linux.FUSEMknodIn{ + MknodMeta: linux.FUSEMknodMeta{ + Mode: uint32(opts.Mode), + Rdev: linux.MakeDeviceID(uint16(opts.DevMajor), opts.DevMinor), + Umask: uint32(kernel.TaskFromContext(ctx).FSContext().Umask()), + }, + Name: name, + } + return i.newEntry(ctx, name, opts.Mode.FileType(), linux.FUSE_MKNOD, &in) +} + // newEntry calls FUSE server for entry creation and allocates corresponding entry according to response. // Shared by FUSE_MKNOD, FUSE_MKDIR, FUSE_SYMLINK, FUSE_LINK and FUSE_LOOKUP. func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMode, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*vfs.Dentry, error) { diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 53cbadb3c..ff2948eb3 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -16,3 +16,8 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:release_test", ) + +syscall_test( + fuse = "True", + test = "//test/fuse/linux:mknod_test", +) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index abee1a33c..c0f917606 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -44,6 +44,20 @@ cc_binary( ], ) +cc_binary( + name = "mknod_test", + testonly = 1, + srcs = ["mknod_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:temp_umask", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_library( name = "fuse_base", testonly = 1, diff --git a/test/fuse/linux/mknod_test.cc b/test/fuse/linux/mknod_test.cc new file mode 100644 index 000000000..74c74d76b --- /dev/null +++ b/test/fuse/linux/mknod_test.cc @@ -0,0 +1,107 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/temp_umask.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class MknodTest : public FuseTest { + protected: + const std::string test_file_ = "test_file"; + const mode_t perms_ = S_IRWXU | S_IRWXG | S_IRWXO; +}; + +TEST_F(MknodTest, RegularFile) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + const mode_t new_umask = 0077; + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_entry_out), + }; + struct fuse_entry_out out_payload = DefaultEntryOut(S_IFREG | perms_, 5); + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_MKNOD, iov_out); + TempUmask mask(new_umask); + ASSERT_THAT(mknod(test_file_path.c_str(), perms_, 0), SyscallSucceeds()); + + struct fuse_in_header in_header; + struct fuse_mknod_in in_payload; + std::vector actual_file(test_file_.length() + 1); + auto iov_in = FuseGenerateIovecs(in_header, in_payload, actual_file); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, + sizeof(in_header) + sizeof(in_payload) + test_file_.length() + 1); + EXPECT_EQ(in_header.opcode, FUSE_MKNOD); + EXPECT_EQ(in_payload.mode & 0777, perms_ & ~new_umask); + EXPECT_EQ(in_payload.umask, new_umask); + EXPECT_EQ(in_payload.rdev, 0); + EXPECT_EQ(std::string(actual_file.data()), test_file_); +} + +TEST_F(MknodTest, FileTypeError) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_entry_out), + }; + // server return directory instead of regular file should cause an error. + struct fuse_entry_out out_payload = DefaultEntryOut(S_IFDIR | perms_, 5); + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_MKNOD, iov_out); + ASSERT_THAT(mknod(test_file_path.c_str(), perms_, 0), + SyscallFailsWithErrno(EIO)); + SkipServerActualRequest(); +} + +TEST_F(MknodTest, NodeIDError) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_entry_out), + }; + struct fuse_entry_out out_payload = + DefaultEntryOut(S_IFREG | perms_, FUSE_ROOT_ID); + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_MKNOD, iov_out); + ASSERT_THAT(mknod(test_file_path.c_str(), perms_, 0), + SyscallFailsWithErrno(EIO)); + SkipServerActualRequest(); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From b50c03b5715905ebd82b1006c1bb2e2d4eb9334d Mon Sep 17 00:00:00 2001 From: Boyuan He Date: Tue, 18 Aug 2020 23:09:34 +0000 Subject: Implement FUSE_SYMLINK Fixes #3452 --- pkg/abi/linux/fuse.go | 27 ++++++++++++ pkg/sentry/fsimpl/fuse/fusefs.go | 9 ++++ test/fuse/BUILD | 5 +++ test/fuse/linux/BUILD | 13 ++++++ test/fuse/linux/symlink_test.cc | 88 ++++++++++++++++++++++++++++++++++++++++ 5 files changed, 142 insertions(+) create mode 100644 test/fuse/linux/symlink_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index 97d960096..ea5a7fd43 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -456,3 +456,30 @@ func (r *FUSEMknodIn) MarshalUnsafe(buf []byte) { func (r *FUSEMknodIn) SizeBytes() int { return r.MknodMeta.SizeBytes() + len(r.Name) + 1 } + +// FUSESymLinkIn is the request sent by the kernel to the daemon, +// to create a symbolic link. +// +// Dynamically-sized objects cannot be marshalled. +type FUSESymLinkIn struct { + marshal.StubMarshallable + + // Name of symlink to create. + Name string + + // Target of the symlink. + Target string +} + +// MarshalUnsafe serializes r.Name and r.Target to the dst buffer. +// Left null-termination at end of r.Name and r.Target. +func (r *FUSESymLinkIn) MarshalUnsafe(buf []byte) { + copy(buf, r.Name) + copy(buf[len(r.Name)+1:], r.Target) +} + +// SizeBytes is the size of the memory representation of FUSESymLinkIn. +// 2 extra bytes for null-terminated string. +func (r *FUSESymLinkIn) SizeBytes() int { + return len(r.Name) + len(r.Target) + 2 +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 5cef0b94f..0021e2933 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -362,6 +362,15 @@ func (i *inode) NewNode(ctx context.Context, name string, opts vfs.MknodOptions) return i.newEntry(ctx, name, opts.Mode.FileType(), linux.FUSE_MKNOD, &in) } +// NewSymlink implements kernfs.Inode.NewSymlink. +func (i *inode) NewSymlink(ctx context.Context, name, target string) (*vfs.Dentry, error) { + in := linux.FUSESymLinkIn{ + Name: name, + Target: target, + } + return i.newEntry(ctx, name, linux.S_IFLNK, linux.FUSE_SYMLINK, &in) +} + // newEntry calls FUSE server for entry creation and allocates corresponding entry according to response. // Shared by FUSE_MKNOD, FUSE_MKDIR, FUSE_SYMLINK, FUSE_LINK and FUSE_LOOKUP. func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMode, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*vfs.Dentry, error) { diff --git a/test/fuse/BUILD b/test/fuse/BUILD index ff2948eb3..2f91fe2c7 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -21,3 +21,8 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:mknod_test", ) + +syscall_test( + fuse = "True", + test = "//test/fuse/linux:symlink_test", +) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index c0f917606..df42857f6 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -58,6 +58,19 @@ cc_binary( ], ) +cc_binary( + name = "symlink_test", + testonly = 1, + srcs = ["symlink_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_library( name = "fuse_base", testonly = 1, diff --git a/test/fuse/linux/symlink_test.cc b/test/fuse/linux/symlink_test.cc new file mode 100644 index 000000000..2c3a52987 --- /dev/null +++ b/test/fuse/linux/symlink_test.cc @@ -0,0 +1,88 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class SymlinkTest : public FuseTest { + protected: + const std::string target_file_ = "target_file_"; + const std::string symlink_ = "symlink_"; + const mode_t perms_ = S_IRWXU | S_IRWXG | S_IRWXO; +}; + +TEST_F(SymlinkTest, CreateSymLink) { + const std::string symlink_path = + JoinPath(mount_point_.path().c_str(), symlink_); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_entry_out), + }; + struct fuse_entry_out out_payload = DefaultEntryOut(S_IFLNK | perms_, 5); + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SYMLINK, iov_out); + ASSERT_THAT(symlink(target_file_.c_str(), symlink_path.c_str()), + SyscallSucceeds()); + + struct fuse_in_header in_header; + std::vector actual_target_file(target_file_.length() + 1); + std::vector actual_symlink(symlink_.length() + 1); + auto iov_in = + FuseGenerateIovecs(in_header, actual_symlink, actual_target_file); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, + sizeof(in_header) + symlink_.length() + target_file_.length() + 2); + EXPECT_EQ(in_header.opcode, FUSE_SYMLINK); + EXPECT_EQ(std::string(actual_target_file.data()), target_file_); + EXPECT_EQ(std::string(actual_symlink.data()), symlink_); +} + +TEST_F(SymlinkTest, FileTypeError) { + const std::string symlink_path = + JoinPath(mount_point_.path().c_str(), symlink_); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_entry_out), + }; + struct fuse_entry_out out_payload = DefaultEntryOut(S_IFREG | perms_, 5); + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SYMLINK, iov_out); + ASSERT_THAT(symlink(target_file_.c_str(), symlink_path.c_str()), + SyscallFailsWithErrno(EIO)); + SkipServerActualRequest(); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From 733d013f979a2107fd866ff3c05249c3e8ba5102 Mon Sep 17 00:00:00 2001 From: Boyuan He Date: Tue, 18 Aug 2020 23:50:22 +0000 Subject: Implement FUSE_READLINK Fixes #3316 --- pkg/abi/linux/fuse.go | 11 ++++ pkg/sentry/fsimpl/fuse/fusefs.go | 30 ++++++++++ pkg/sentry/fsimpl/kernfs/filesystem.go | 2 +- pkg/sentry/fsimpl/kernfs/inode_impl_util.go | 2 +- pkg/sentry/fsimpl/kernfs/kernfs.go | 2 +- pkg/sentry/fsimpl/kernfs/symlink.go | 2 +- pkg/sentry/fsimpl/proc/task_fds.go | 2 +- pkg/sentry/fsimpl/proc/task_files.go | 6 +- pkg/sentry/fsimpl/proc/tasks_files.go | 12 ++-- test/fuse/BUILD | 5 ++ test/fuse/linux/BUILD | 13 +++++ test/fuse/linux/readlink_test.cc | 85 +++++++++++++++++++++++++++++ 12 files changed, 158 insertions(+), 14 deletions(-) create mode 100644 test/fuse/linux/readlink_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index ea5a7fd43..5de1433d7 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -483,3 +483,14 @@ func (r *FUSESymLinkIn) MarshalUnsafe(buf []byte) { func (r *FUSESymLinkIn) SizeBytes() int { return len(r.Name) + len(r.Target) + 2 } + +// FUSEEmptyIn is used by operations without request body. +type FUSEEmptyIn struct{ marshal.StubMarshallable } + +// MarshalUnsafe do nothing for marshal. +func (r *FUSEEmptyIn) MarshalUnsafe(buf []byte) {} + +// SizeBytes is 0 for empty request. +func (r *FUSEEmptyIn) SizeBytes() int { + return 0 +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 0021e2933..8db337a2e 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -225,6 +225,9 @@ type inode struct { // version of the inode. version uint64 + + // link is result of following a symbolic link. + link string } func (fs *filesystem) newRootInode(creds *auth.Credentials, mode linux.FileMode) *kernfs.Dentry { @@ -406,6 +409,33 @@ func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMo return child.VFSDentry(), nil } +// Readlink implements kernfs.Inode.Readlink. +func (i *inode) Readlink(ctx context.Context, mnt *vfs.Mount) (string, error) { + if i.Mode().FileType()&linux.S_IFLNK == 0 { + return "", syserror.EINVAL + } + if i.link == "" { + kernelTask := kernel.TaskFromContext(ctx) + if kernelTask == nil { + log.Warningf("fusefs.Inode.Readlink: couldn't get kernel task from context") + return "", syserror.EINVAL + } + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, linux.FUSE_READLINK, &linux.FUSEEmptyIn{}) + if err != nil { + return "", err + } + res, err := i.fs.conn.Call(kernelTask, req) + if err != nil { + return "", err + } + i.link = string(res.data[res.hdr.SizeBytes():]) + if !mnt.Options().ReadOnly { + i.attributeTime = 0 + } + } + return i.link, nil +} + // statFromFUSEAttr makes attributes from linux.FUSEAttr to linux.Statx. The // opts.Sync attribute is ignored since the synchronization is handled by the // FUSE server. diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go index 904203070..7aaf1146d 100644 --- a/pkg/sentry/fsimpl/kernfs/filesystem.go +++ b/pkg/sentry/fsimpl/kernfs/filesystem.go @@ -548,7 +548,7 @@ func (fs *Filesystem) ReadlinkAt(ctx context.Context, rp *vfs.ResolvingPath) (st if !d.Impl().(*Dentry).isSymlink() { return "", syserror.EINVAL } - return inode.Readlink(ctx) + return inode.Readlink(ctx, rp.Mount()) } // RenameAt implements vfs.FilesystemImpl.RenameAt. diff --git a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go index 74408e322..53758a4b8 100644 --- a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go +++ b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go @@ -172,7 +172,7 @@ func (InodeNoDynamicLookup) Valid(ctx context.Context) bool { type InodeNotSymlink struct{} // Readlink implements Inode.Readlink. -func (InodeNotSymlink) Readlink(context.Context) (string, error) { +func (InodeNotSymlink) Readlink(context.Context, *vfs.Mount) (string, error) { return "", syserror.EINVAL } diff --git a/pkg/sentry/fsimpl/kernfs/kernfs.go b/pkg/sentry/fsimpl/kernfs/kernfs.go index 67a0347fe..f656e2a8b 100644 --- a/pkg/sentry/fsimpl/kernfs/kernfs.go +++ b/pkg/sentry/fsimpl/kernfs/kernfs.go @@ -437,7 +437,7 @@ type inodeDynamicLookup interface { type inodeSymlink interface { // Readlink returns the target of a symbolic link. If an inode is not a // symlink, the implementation should return EINVAL. - Readlink(ctx context.Context) (string, error) + Readlink(ctx context.Context, mnt *vfs.Mount) (string, error) // Getlink returns the target of a symbolic link, as used by path // resolution: diff --git a/pkg/sentry/fsimpl/kernfs/symlink.go b/pkg/sentry/fsimpl/kernfs/symlink.go index 64731a3e4..a9812fcef 100644 --- a/pkg/sentry/fsimpl/kernfs/symlink.go +++ b/pkg/sentry/fsimpl/kernfs/symlink.go @@ -52,7 +52,7 @@ func (s *StaticSymlink) Init(creds *auth.Credentials, devMajor uint32, devMinor } // Readlink implements Inode. -func (s *StaticSymlink) Readlink(_ context.Context) (string, error) { +func (s *StaticSymlink) Readlink(_ context.Context, _ *vfs.Mount) (string, error) { return s.target, nil } diff --git a/pkg/sentry/fsimpl/proc/task_fds.go b/pkg/sentry/fsimpl/proc/task_fds.go index 94ec2ff69..47d744a57 100644 --- a/pkg/sentry/fsimpl/proc/task_fds.go +++ b/pkg/sentry/fsimpl/proc/task_fds.go @@ -214,7 +214,7 @@ func (fs *filesystem) newFDSymlink(task *kernel.Task, fd int32, ino uint64) *ker return d } -func (s *fdSymlink) Readlink(ctx context.Context) (string, error) { +func (s *fdSymlink) Readlink(ctx context.Context, _ *vfs.Mount) (string, error) { file, _ := getTaskFD(s.task, s.fd) if file == nil { return "", syserror.ENOENT diff --git a/pkg/sentry/fsimpl/proc/task_files.go b/pkg/sentry/fsimpl/proc/task_files.go index ce87b0d47..87726d8f7 100644 --- a/pkg/sentry/fsimpl/proc/task_files.go +++ b/pkg/sentry/fsimpl/proc/task_files.go @@ -668,7 +668,7 @@ func (fs *filesystem) newExeSymlink(task *kernel.Task, ino uint64) *kernfs.Dentr } // Readlink implements kernfs.Inode. -func (s *exeSymlink) Readlink(ctx context.Context) (string, error) { +func (s *exeSymlink) Readlink(ctx context.Context, _ *vfs.Mount) (string, error) { if !kernel.ContextCanTrace(ctx, s.task, false) { return "", syserror.EACCES } @@ -808,11 +808,11 @@ func (fs *filesystem) newNamespaceSymlink(task *kernel.Task, ino uint64, ns stri } // Readlink implements Inode. -func (s *namespaceSymlink) Readlink(ctx context.Context) (string, error) { +func (s *namespaceSymlink) Readlink(ctx context.Context, mnt *vfs.Mount) (string, error) { if err := checkTaskState(s.task); err != nil { return "", err } - return s.StaticSymlink.Readlink(ctx) + return s.StaticSymlink.Readlink(ctx, mnt) } // Getlink implements Inode.Getlink. diff --git a/pkg/sentry/fsimpl/proc/tasks_files.go b/pkg/sentry/fsimpl/proc/tasks_files.go index 8c41729e4..68c541046 100644 --- a/pkg/sentry/fsimpl/proc/tasks_files.go +++ b/pkg/sentry/fsimpl/proc/tasks_files.go @@ -51,7 +51,7 @@ func (fs *filesystem) newSelfSymlink(creds *auth.Credentials, ino uint64, pidns return d } -func (s *selfSymlink) Readlink(ctx context.Context) (string, error) { +func (s *selfSymlink) Readlink(ctx context.Context, _ *vfs.Mount) (string, error) { t := kernel.TaskFromContext(ctx) if t == nil { // Who is reading this link? @@ -64,8 +64,8 @@ func (s *selfSymlink) Readlink(ctx context.Context) (string, error) { return strconv.FormatUint(uint64(tgid), 10), nil } -func (s *selfSymlink) Getlink(ctx context.Context, _ *vfs.Mount) (vfs.VirtualDentry, string, error) { - target, err := s.Readlink(ctx) +func (s *selfSymlink) Getlink(ctx context.Context, mnt *vfs.Mount) (vfs.VirtualDentry, string, error) { + target, err := s.Readlink(ctx, mnt) return vfs.VirtualDentry{}, target, err } @@ -94,7 +94,7 @@ func (fs *filesystem) newThreadSelfSymlink(creds *auth.Credentials, ino uint64, return d } -func (s *threadSelfSymlink) Readlink(ctx context.Context) (string, error) { +func (s *threadSelfSymlink) Readlink(ctx context.Context, _ *vfs.Mount) (string, error) { t := kernel.TaskFromContext(ctx) if t == nil { // Who is reading this link? @@ -108,8 +108,8 @@ func (s *threadSelfSymlink) Readlink(ctx context.Context) (string, error) { return fmt.Sprintf("%d/task/%d", tgid, tid), nil } -func (s *threadSelfSymlink) Getlink(ctx context.Context, _ *vfs.Mount) (vfs.VirtualDentry, string, error) { - target, err := s.Readlink(ctx) +func (s *threadSelfSymlink) Getlink(ctx context.Context, mnt *vfs.Mount) (vfs.VirtualDentry, string, error) { + target, err := s.Readlink(ctx, mnt) return vfs.VirtualDentry{}, target, err } diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 2f91fe2c7..c2bdcf1ba 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -26,3 +26,8 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:symlink_test", ) + +syscall_test( + fuse = "True", + test = "//test/fuse/linux:readlink_test", +) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index df42857f6..d3e8ca148 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -71,6 +71,19 @@ cc_binary( ], ) +cc_binary( + name = "readlink_test", + testonly = 1, + srcs = ["readlink_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_library( name = "fuse_base", testonly = 1, diff --git a/test/fuse/linux/readlink_test.cc b/test/fuse/linux/readlink_test.cc new file mode 100644 index 000000000..2cba8fc23 --- /dev/null +++ b/test/fuse/linux/readlink_test.cc @@ -0,0 +1,85 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class ReadlinkTest : public FuseTest { + protected: + const std::string test_file_ = "test_file_"; + const mode_t perms_ = S_IRWXU | S_IRWXG | S_IRWXO; +}; + +TEST_F(ReadlinkTest, ReadSymLink) { + const std::string symlink_path = + JoinPath(mount_point_.path().c_str(), test_file_); + SetServerInodeLookup(test_file_, S_IFLNK | perms_); + + struct fuse_out_header out_header = { + .len = static_cast(sizeof(struct fuse_out_header)) + + static_cast(test_file_.length()) + 1, + }; + std::string link = test_file_; + auto iov_out = FuseGenerateIovecs(out_header, link); + SetServerResponse(FUSE_READLINK, iov_out); + const std::string actual_link = + ASSERT_NO_ERRNO_AND_VALUE(ReadLink(symlink_path)); + + struct fuse_in_header in_header; + auto iov_in = FuseGenerateIovecs(in_header); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, sizeof(in_header)); + EXPECT_EQ(in_header.opcode, FUSE_READLINK); + EXPECT_EQ(0, memcmp(actual_link.c_str(), link.data(), link.size())); + + // next readlink should have link cached, so shouldn't have new request to + // server. + uint32_t recieved_before = GetServerTotalReceivedBytes(); + ASSERT_NO_ERRNO(ReadLink(symlink_path)); + EXPECT_EQ(GetServerTotalReceivedBytes(), recieved_before); +} + +TEST_F(ReadlinkTest, NotSymlink) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + SetServerInodeLookup(test_file_, S_IFREG | perms_); + + std::vector buf(PATH_MAX + 1); + ASSERT_THAT(readlink(test_file_path.c_str(), buf.data(), PATH_MAX), + SyscallFailsWithErrno(EINVAL)); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From 4d26c9929de31cdfe3551d4b8be90a07f98fed55 Mon Sep 17 00:00:00 2001 From: Boyuan He Date: Tue, 18 Aug 2020 22:45:47 +0000 Subject: Implement FUSE_MKDIR Fixes #3392 --- pkg/abi/linux/fuse.go | 38 ++++++++++++++++ pkg/sentry/fsimpl/fuse/BUILD | 1 + pkg/sentry/fsimpl/fuse/directory.go | 51 +++++++++++++++++++++ pkg/sentry/fsimpl/fuse/fusefs.go | 27 ++++++++++-- test/fuse/BUILD | 5 +++ test/fuse/linux/BUILD | 14 ++++++ test/fuse/linux/mkdir_test.cc | 88 +++++++++++++++++++++++++++++++++++++ 7 files changed, 221 insertions(+), 3 deletions(-) create mode 100644 pkg/sentry/fsimpl/fuse/directory.go create mode 100644 test/fuse/linux/mkdir_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index 5de1433d7..4ef0ab9a7 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -494,3 +494,41 @@ func (r *FUSEEmptyIn) MarshalUnsafe(buf []byte) {} func (r *FUSEEmptyIn) SizeBytes() int { return 0 } + +// FUSEMkdirMeta contains all the static fields of FUSEMkdirIn, +// which is used for FUSE_MKDIR. +// +// +marshal +type FUSEMkdirMeta struct { + // Mode of the directory of create. + Mode uint32 + + // Umask is the user file creation mask. + Umask uint32 +} + +// FUSEMkdirIn contains all the arguments sent by the kernel +// to the daemon, to create a new directory. +// +// Dynamically-sized objects cannot be marshalled. +type FUSEMkdirIn struct { + marshal.StubMarshallable + + // MkdirMeta contains Mode and Umask of the directory to create. + MkdirMeta FUSEMkdirMeta + + // Name of the directory to create. + Name string +} + +// MarshalUnsafe serializes r.MkdirMeta and r.Name to the dst buffer. +func (r *FUSEMkdirIn) MarshalUnsafe(buf []byte) { + r.MkdirMeta.MarshalUnsafe(buf[:r.MkdirMeta.SizeBytes()]) + copy(buf[r.MkdirMeta.SizeBytes():], r.Name) +} + +// SizeBytes is the size of the memory representation of FUSEMkdirIn. +// 1 extra byte for null-terminated Name string. +func (r *FUSEMkdirIn) SizeBytes() int { + return r.MkdirMeta.SizeBytes() + len(r.Name) + 1 +} diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index 75a2e96ff..b9dcee96a 100644 --- a/pkg/sentry/fsimpl/fuse/BUILD +++ b/pkg/sentry/fsimpl/fuse/BUILD @@ -31,6 +31,7 @@ go_library( srcs = [ "connection.go", "dev.go", + "directory.go", "file.go", "fusefs.go", "init.go", diff --git a/pkg/sentry/fsimpl/fuse/directory.go b/pkg/sentry/fsimpl/fuse/directory.go new file mode 100644 index 000000000..44d41712a --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/directory.go @@ -0,0 +1,51 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fuse + +import ( + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/usermem" +) + +type directoryFD struct { + fileDescription +} + +// Allocate implements directoryFD.Allocate. +func (directoryFD) Allocate(ctx context.Context, mode, offset, length uint64) error { + return syserror.EISDIR +} + +// PRead implements FileDescriptionImpl.PRead. +func (directoryFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) { + return 0, syserror.EISDIR +} + +// Read implements FileDescriptionImpl.Read. +func (directoryFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { + return 0, syserror.EISDIR +} + +// PWrite implements FileDescriptionImpl.PWrite. +func (directoryFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) { + return 0, syserror.EISDIR +} + +// Write implements FileDescriptionImpl.Write. +func (directoryFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { + return 0, syserror.EISDIR +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 8db337a2e..4dc8ef993 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -262,8 +262,17 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr return nil, syserror.EOVERFLOW } - // FOPEN_KEEP_CACHE is the defualt flag for noOpen. - fd := fileDescription{OpenFlag: linux.FOPEN_KEEP_CACHE} + var fd *fileDescription + var fdImpl vfs.FileDescriptionImpl + if isDir { + directoryFD := &directoryFD{} + fd = &(directoryFD.fileDescription) + fdImpl = directoryFD + } else { + // FOPEN_KEEP_CACHE is the defualt flag for noOpen. + fd = &fileDescription{OpenFlag: linux.FOPEN_KEEP_CACHE} + fdImpl = fd + } // Only send open request when FUSE server support open or is opening a directory. if !i.fs.conn.noOpen || isDir { kernelTask := kernel.TaskFromContext(ctx) @@ -330,7 +339,7 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr i.attributeTime = 0 } - if err := fd.vfsfd.Init(&fd, opts.Flags, rp.Mount(), vfsd, fdOptions); err != nil { + if err := fd.vfsfd.Init(fdImpl, opts.Flags, rp.Mount(), vfsd, fdOptions); err != nil { return nil, err } return &fd.vfsfd, nil @@ -374,6 +383,18 @@ func (i *inode) NewSymlink(ctx context.Context, name, target string) (*vfs.Dentr return i.newEntry(ctx, name, linux.S_IFLNK, linux.FUSE_SYMLINK, &in) } +// NewDir implements kernfs.Inode.NewDir. +func (i *inode) NewDir(ctx context.Context, name string, opts vfs.MkdirOptions) (*vfs.Dentry, error) { + in := linux.FUSEMkdirIn{ + MkdirMeta: linux.FUSEMkdirMeta{ + Mode: uint32(opts.Mode), + Umask: uint32(kernel.TaskFromContext(ctx).FSContext().Umask()), + }, + Name: name, + } + return i.newEntry(ctx, name, linux.S_IFDIR, linux.FUSE_MKDIR, &in) +} + // newEntry calls FUSE server for entry creation and allocates corresponding entry according to response. // Shared by FUSE_MKNOD, FUSE_MKDIR, FUSE_SYMLINK, FUSE_LINK and FUSE_LOOKUP. func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMode, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*vfs.Dentry, error) { diff --git a/test/fuse/BUILD b/test/fuse/BUILD index c2bdcf1ba..8bde81e3c 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -31,3 +31,8 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:readlink_test", ) + +syscall_test( + fuse = "True", + test = "//test/fuse/linux:mkdir_test", +) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index d3e8ca148..298ea11f8 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -84,6 +84,20 @@ cc_binary( ], ) +cc_binary( + name = "mkdir_test", + testonly = 1, + srcs = ["mkdir_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:temp_umask", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_library( name = "fuse_base", testonly = 1, diff --git a/test/fuse/linux/mkdir_test.cc b/test/fuse/linux/mkdir_test.cc new file mode 100644 index 000000000..9647cb93f --- /dev/null +++ b/test/fuse/linux/mkdir_test.cc @@ -0,0 +1,88 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/temp_umask.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class MkdirTest : public FuseTest { + protected: + const std::string test_dir_ = "test_dir"; + const mode_t perms_ = S_IRWXU | S_IRWXG | S_IRWXO; +}; + +TEST_F(MkdirTest, CreateDir) { + const std::string test_dir_path_ = + JoinPath(mount_point_.path().c_str(), test_dir_); + const mode_t new_umask = 0077; + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_entry_out), + }; + struct fuse_entry_out out_payload = DefaultEntryOut(S_IFDIR | perms_, 5); + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_MKDIR, iov_out); + TempUmask mask(new_umask); + ASSERT_THAT(mkdir(test_dir_path_.c_str(), 0777), SyscallSucceeds()); + + struct fuse_in_header in_header; + struct fuse_mkdir_in in_payload; + std::vector actual_dir(test_dir_.length() + 1); + auto iov_in = FuseGenerateIovecs(in_header, in_payload, actual_dir); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, + sizeof(in_header) + sizeof(in_payload) + test_dir_.length() + 1); + EXPECT_EQ(in_header.opcode, FUSE_MKDIR); + EXPECT_EQ(in_payload.mode & 0777, perms_ & ~new_umask); + EXPECT_EQ(in_payload.umask, new_umask); + EXPECT_EQ(std::string(actual_dir.data()), test_dir_); +} + +TEST_F(MkdirTest, FileTypeError) { + const std::string test_dir_path_ = + JoinPath(mount_point_.path().c_str(), test_dir_); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_entry_out), + }; + struct fuse_entry_out out_payload = DefaultEntryOut(S_IFREG | perms_, 5); + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_MKDIR, iov_out); + ASSERT_THAT(mkdir(test_dir_path_.c_str(), 0777), SyscallFailsWithErrno(EIO)); + SkipServerActualRequest(); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From 713400d6b0f2eef6a368bc9cb76a15ee3334d1e5 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Wed, 15 Jul 2020 18:32:51 +0000 Subject: Implement FUSE_READ Fixes #3206 --- pkg/abi/linux/fuse.go | 37 ++++ pkg/sentry/fsimpl/fuse/BUILD | 3 + pkg/sentry/fsimpl/fuse/connection.go | 6 +- pkg/sentry/fsimpl/fuse/dev.go | 4 +- pkg/sentry/fsimpl/fuse/fusefs.go | 130 +++++++++-- pkg/sentry/fsimpl/fuse/init.go | 3 +- pkg/sentry/fsimpl/fuse/read_write.go | 152 +++++++++++++ pkg/sentry/fsimpl/fuse/regular_file.go | 125 +++++++++++ test/fuse/BUILD | 5 + test/fuse/linux/BUILD | 13 ++ test/fuse/linux/fuse_base.cc | 17 +- test/fuse/linux/fuse_base.h | 13 +- test/fuse/linux/read_test.cc | 390 +++++++++++++++++++++++++++++++++ 13 files changed, 862 insertions(+), 36 deletions(-) create mode 100644 pkg/sentry/fsimpl/fuse/read_write.go create mode 100644 pkg/sentry/fsimpl/fuse/regular_file.go create mode 100644 test/fuse/linux/read_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index 4ef0ab9a7..0ece7b756 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -184,6 +184,13 @@ const ( FUSE_KERNEL_MINOR_VERSION = 31 ) +// Constants relevant to FUSE operations. +const ( + FUSE_NAME_MAX = 1024 + FUSE_PAGE_SIZE = 4096 + FUSE_DIRENT_ALIGN = 8 +) + // FUSEInitIn is the request sent by the kernel to the daemon, // to negotiate the version and flags. // @@ -392,6 +399,36 @@ type FUSEOpenOut struct { // OpenFlag for the opened file. OpenFlag uint32 +} + +// FUSE_READ flags, consistent with the ones in include/uapi/linux/fuse.h. +const ( + FUSE_READ_LOCKOWNER = 1 << 1 +) + +// FUSEReadIn is the request sent by the kernel to the daemon +// for FUSE_READ. +// +// +marshal +type FUSEReadIn struct { + // Fh is the file handle in userspace. + Fh uint64 + + // Offset is the read offset. + Offset uint64 + + // Size is the number of bytes to read. + Size uint32 + + // ReadFlags for this FUSE_READ request. + // Currently only contains FUSE_READ_LOCKOWNER. + ReadFlags uint32 + + // LockOwner is the id of the lock owner if there is one. + LockOwner uint64 + + // Flags for the underlying file. + Flags uint32 _ uint32 } diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index b9dcee96a..241d50c3d 100644 --- a/pkg/sentry/fsimpl/fuse/BUILD +++ b/pkg/sentry/fsimpl/fuse/BUILD @@ -36,7 +36,9 @@ go_library( "fusefs.go", "init.go", "inode_refs.go", + "read_write.go", "register.go", + "regular_file.go", "request_list.go", ], visibility = ["//pkg/sentry:internal"], @@ -46,6 +48,7 @@ go_library( "//pkg/log", "//pkg/marshal", "//pkg/refs", + "//pkg/safemem", "//pkg/sentry/fsimpl/devtmpfs", "//pkg/sentry/fsimpl/kernfs", "//pkg/sentry/kernel", diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 9cfd67158..236165652 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -161,6 +161,7 @@ type connection struct { bgLock sync.Mutex // maxRead is the maximum size of a read buffer in in bytes. + // Initialized from a fuse fs parameter. maxRead uint32 // maxWrite is the maximum size of a write buffer in bytes. @@ -206,7 +207,7 @@ type connection struct { } // newFUSEConnection creates a FUSE connection to fd. -func newFUSEConnection(_ context.Context, fd *vfs.FileDescription, maxInFlightRequests uint64) (*connection, error) { +func newFUSEConnection(_ context.Context, fd *vfs.FileDescription, opts *filesystemOptions) (*connection, error) { // Mark the device as ready so it can be used. /dev/fuse can only be used if the FD was used to // mount a FUSE filesystem. fuseFD := fd.Impl().(*DeviceFD) @@ -216,13 +217,14 @@ func newFUSEConnection(_ context.Context, fd *vfs.FileDescription, maxInFlightRe hdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) fuseFD.writeBuf = make([]byte, hdrLen) fuseFD.completions = make(map[linux.FUSEOpID]*futureResponse) - fuseFD.fullQueueCh = make(chan struct{}, maxInFlightRequests) + fuseFD.fullQueueCh = make(chan struct{}, opts.maxActiveRequests) fuseFD.writeCursor = 0 return &connection{ fd: fuseFD, maxBackground: fuseDefaultMaxBackground, congestionThreshold: fuseDefaultCongestionThreshold, + maxRead: opts.maxRead, maxPages: fuseDefaultMaxPagesPerReq, initializedChan: make(chan struct{}), connected: true, diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index e2de8e097..fd3592e32 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -401,10 +401,12 @@ func (fd *DeviceFD) sendError(ctx context.Context, errno int32, req *Request) er // receiver is going to be waiting on the future channel. This is to be used by: // FUSE_INIT. func (fd *DeviceFD) noReceiverAction(ctx context.Context, r *Response) error { - if r.opcode == linux.FUSE_INIT { + switch r.opcode { + case linux.FUSE_INIT: creds := auth.CredentialsFromContext(ctx) rootUserNs := kernel.KernelFromContext(ctx).RootUserNamespace() return fd.fs.conn.InitRecv(r, creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, rootUserNs)) + // TODO(gvisor.dev/issue/3247): support async read: correctly process the response using information from r.options. } return nil diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 4dc8ef993..65e22ba4d 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -16,6 +16,7 @@ package fuse import ( + "math" "strconv" "sync/atomic" @@ -58,6 +59,11 @@ type filesystemOptions struct { // exist at any time. Any further requests will block when trying to // Call the server. maxActiveRequests uint64 + + // maxRead is the max number of bytes to read, + // specified as "max_read" in fs parameters. + // If not specified by user, use math.MaxUint32 as default value. + maxRead uint32 } // filesystem implements vfs.FilesystemImpl. @@ -144,6 +150,21 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt // Set the maxInFlightRequests option. fsopts.maxActiveRequests = maxActiveRequestsDefault + if maxReadStr, ok := mopts["max_read"]; ok { + delete(mopts, "max_read") + maxRead, err := strconv.ParseUint(maxReadStr, 10, 32) + if err != nil { + log.Warningf("%s.GetFilesystem: invalid max_read: max_read=%s", fsType.Name(), maxReadStr) + return nil, nil, syserror.EINVAL + } + if maxRead < fuseMinMaxRead { + maxRead = fuseMinMaxRead + } + fsopts.maxRead = uint32(maxRead) + } else { + fsopts.maxRead = math.MaxUint32 + } + // Check for unparsed options. if len(mopts) != 0 { log.Warningf("%s.GetFilesystem: unknown options: %v", fsType.Name(), mopts) @@ -179,7 +200,7 @@ func NewFUSEFilesystem(ctx context.Context, devMinor uint32, opts *filesystemOpt opts: opts, } - conn, err := newFUSEConnection(ctx, device, opts.maxActiveRequests) + conn, err := newFUSEConnection(ctx, device, opts) if err != nil { log.Warningf("fuse.NewFUSEFilesystem: NewFUSEConnection failed with error: %v", err) return nil, syserror.EINVAL @@ -244,6 +265,7 @@ func (fs *filesystem) newInode(nodeID uint64, attr linux.FUSEAttr) *kernfs.Dentr i := &inode{fs: fs, NodeID: nodeID} creds := auth.Credentials{EffectiveKGID: auth.KGID(attr.UID), EffectiveKUID: auth.KUID(attr.UID)} i.InodeAttrs.Init(&creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.FileMode(attr.Mode)) + atomic.StoreUint64(&i.size, attr.Size) i.OrderedChildren.Init(kernfs.OrderedChildrenOptions{}) i.EnableLeakCheck() i.dentry.Init(i) @@ -269,10 +291,13 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr fd = &(directoryFD.fileDescription) fdImpl = directoryFD } else { - // FOPEN_KEEP_CACHE is the defualt flag for noOpen. - fd = &fileDescription{OpenFlag: linux.FOPEN_KEEP_CACHE} - fdImpl = fd + regularFD := ®ularFileFD{} + fd = &(regularFD.fileDescription) + fdImpl = regularFD } + // FOPEN_KEEP_CACHE is the defualt flag for noOpen. + fd.OpenFlag = linux.FOPEN_KEEP_CACHE + // Only send open request when FUSE server support open or is opening a directory. if !i.fs.conn.noOpen || isDir { kernelTask := kernel.TaskFromContext(ctx) @@ -281,21 +306,25 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr return nil, syserror.EINVAL } + // Build the request. var opcode linux.FUSEOpcode if isDir { opcode = linux.FUSE_OPENDIR } else { opcode = linux.FUSE_OPEN } + in := linux.FUSEOpenIn{Flags: opts.Flags & ^uint32(linux.O_CREAT|linux.O_EXCL|linux.O_NOCTTY)} if !i.fs.conn.atomicOTrunc { in.Flags &= ^uint32(linux.O_TRUNC) } + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, opcode, &in) if err != nil { return nil, err } + // Send the request and receive the reply. res, err := i.fs.conn.Call(kernelTask, req) if err != nil { return nil, err @@ -309,15 +338,17 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr if err := res.UnmarshalPayload(&out); err != nil { return nil, err } + + // Process the reply. fd.OpenFlag = out.OpenFlag + if isDir { + fd.OpenFlag &= ^uint32(linux.FOPEN_DIRECT_IO) + } + fd.Fh = out.Fh } } - if isDir { - fd.OpenFlag &= ^uint32(linux.FOPEN_DIRECT_IO) - } - // TODO(gvisor.dev/issue/3234): invalidate mmap after implemented it for FUSE Inode fd.DirectIO = fd.OpenFlag&linux.FOPEN_DIRECT_IO != 0 fdOptions := &vfs.FileDescriptionOptions{} @@ -457,6 +488,16 @@ func (i *inode) Readlink(ctx context.Context, mnt *vfs.Mount) (string, error) { return i.link, nil } +// getFUSEAttr returns a linux.FUSEAttr of this inode stored in local cache. +// TODO(gvisor.dev/issue/3679): Add support for other fields. +func (i *inode) getFUSEAttr() linux.FUSEAttr { + return linux.FUSEAttr{ + Ino: i.Ino(), + Size: atomic.LoadUint64(&i.size), + Mode: uint32(i.Mode()), + } +} + // statFromFUSEAttr makes attributes from linux.FUSEAttr to linux.Statx. The // opts.Sync attribute is ignored since the synchronization is handled by the // FUSE server. @@ -510,47 +551,90 @@ func statFromFUSEAttr(attr linux.FUSEAttr, mask, devMinor uint32) linux.Statx { return stat } -// Stat implements kernfs.Inode.Stat. -func (i *inode) Stat(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) { - fusefs := fs.Impl().(*filesystem) - conn := fusefs.conn - task, creds := kernel.TaskFromContext(ctx), auth.CredentialsFromContext(ctx) +// getAttr gets the attribute of this inode by issuing a FUSE_GETATTR request +// or read from local cache. +// It updates the corresponding attributes if necessary. +func (i *inode) getAttr(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOptions) (linux.FUSEAttr, error) { + attributeVersion := atomic.LoadUint64(&i.fs.conn.attributeVersion) + + // TODO(gvisor.dev/issue/3679): send the request only if + // - invalid local cache for fields specified in the opts.Mask + // - forced update + // - i.attributeTime expired + // If local cache is still valid, return local cache. + // Currently we always send a request, + // and we always set the metadata with the new result, + // unless attributeVersion has changed. + + task := kernel.TaskFromContext(ctx) if task == nil { log.Warningf("couldn't get kernel task from context") - return linux.Statx{}, syserror.EINVAL + return linux.FUSEAttr{}, syserror.EINVAL } + creds := auth.CredentialsFromContext(ctx) + var in linux.FUSEGetAttrIn // We don't set any attribute in the request, because in VFS2 fstat(2) will // finally be translated into vfs.FilesystemImpl.StatAt() (see // pkg/sentry/syscalls/linux/vfs2/stat.go), resulting in the same flow // as stat(2). Thus GetAttrFlags and Fh variable will never be used in VFS2. - req, err := conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_GETATTR, &in) + req, err := i.fs.conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_GETATTR, &in) if err != nil { - return linux.Statx{}, err + return linux.FUSEAttr{}, err } - res, err := conn.Call(task, req) + res, err := i.fs.conn.Call(task, req) if err != nil { - return linux.Statx{}, err + return linux.FUSEAttr{}, err } if err := res.Error(); err != nil { - return linux.Statx{}, err + return linux.FUSEAttr{}, err } var out linux.FUSEGetAttrOut if err := res.UnmarshalPayload(&out); err != nil { - return linux.Statx{}, err + return linux.FUSEAttr{}, err } - // Set all metadata into kernfs.InodeAttrs. + // Local version is newer, return the local one. + // Skip the update. + if attributeVersion != 0 && atomic.LoadUint64(&i.attributeVersion) > attributeVersion { + return i.getFUSEAttr(), nil + } + + // Set the metadata of kernfs.InodeAttrs. if err := i.SetStat(ctx, fs, creds, vfs.SetStatOptions{ - Stat: statFromFUSEAttr(out.Attr, linux.STATX_ALL, fusefs.devMinor), + Stat: statFromFUSEAttr(out.Attr, linux.STATX_ALL, i.fs.devMinor), }); err != nil { + return linux.FUSEAttr{}, err + } + + // Set the size if no error (after SetStat() check). + atomic.StoreUint64(&i.size, out.Attr.Size) + + return out.Attr, nil +} + +// reviseAttr attempts to update the attributes for internal purposes +// by calling getAttr with a pre-specified mask. +// Used by read, write, lseek. +func (i *inode) reviseAttr(ctx context.Context) error { + // Never need atime for internal purposes. + _, err := i.getAttr(ctx, i.fs.VFSFilesystem(), vfs.StatOptions{ + Mask: linux.STATX_BASIC_STATS &^ linux.STATX_ATIME, + }) + return err +} + +// Stat implements kernfs.Inode.Stat. +func (i *inode) Stat(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) { + attr, err := i.getAttr(ctx, fs, opts) + if err != nil { return linux.Statx{}, err } - return statFromFUSEAttr(out.Attr, opts.Mask, fusefs.devMinor), nil + return statFromFUSEAttr(attr, opts.Mask, i.fs.devMinor), nil } // DecRef implements kernfs.Inode. diff --git a/pkg/sentry/fsimpl/fuse/init.go b/pkg/sentry/fsimpl/fuse/init.go index 779c2bd3f..2ff2542b6 100644 --- a/pkg/sentry/fsimpl/fuse/init.go +++ b/pkg/sentry/fsimpl/fuse/init.go @@ -29,9 +29,10 @@ const ( // Follow the same behavior as unix fuse implementation. fuseMaxTimeGranNs = 1000000000 - // Minimum value for MaxWrite. + // Minimum value for MaxWrite and MaxRead. // Follow the same behavior as unix fuse implementation. fuseMinMaxWrite = 4096 + fuseMinMaxRead = 4096 // Temporary default value for max readahead, 128kb. fuseDefaultMaxReadahead = 131072 diff --git a/pkg/sentry/fsimpl/fuse/read_write.go b/pkg/sentry/fsimpl/fuse/read_write.go new file mode 100644 index 000000000..4ef8531dc --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/read_write.go @@ -0,0 +1,152 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fuse + +import ( + "io" + "sync/atomic" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/log" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/usermem" +) + +// ReadInPages sends FUSE_READ requests for the size after round it up to +// a multiple of page size, blocks on it for reply, processes the reply +// and returns the payload (or joined payloads) as a byte slice. +// This is used for the general purpose reading. +// We do not support direct IO (which read the exact number of bytes) +// at this moment. +func (fs *filesystem) ReadInPages(ctx context.Context, fd *regularFileFD, off uint64, size uint32) ([][]byte, uint32, error) { + attributeVersion := atomic.LoadUint64(&fs.conn.attributeVersion) + + t := kernel.TaskFromContext(ctx) + if t == nil { + log.Warningf("fusefs.Read: couldn't get kernel task from context") + return nil, 0, syserror.EINVAL + } + + // Round up to a multiple of page size. + readSize, _ := usermem.PageRoundUp(uint64(size)) + + // One request cannnot exceed either maxRead or maxPages. + maxPages := fs.conn.maxRead >> usermem.PageShift + if maxPages > uint32(fs.conn.maxPages) { + maxPages = uint32(fs.conn.maxPages) + } + + var outs [][]byte + var sizeRead uint32 + + // readSize is a multiple of usermem.PageSize. + // Always request bytes as a multiple of pages. + pagesRead, pagesToRead := uint32(0), uint32(readSize>>usermem.PageShift) + + // Reuse the same struct for unmarshalling to avoid unnecessary memory allocation. + in := linux.FUSEReadIn{ + Fh: fd.Fh, + LockOwner: 0, // TODO(gvisor.dev/issue/3245): file lock + ReadFlags: 0, // TODO(gvisor.dev/issue/3245): |= linux.FUSE_READ_LOCKOWNER + Flags: fd.statusFlags(), + } + + // This loop is intended for fragmented read where the bytes to read is + // larger than either the maxPages or maxRead. + // For the majority of reads with normal size, this loop should only + // execute once. + for pagesRead < pagesToRead { + pagesCanRead := pagesToRead - pagesRead + if pagesCanRead > maxPages { + pagesCanRead = maxPages + } + + in.Offset = off + (uint64(pagesRead) << usermem.PageShift) + in.Size = pagesCanRead << usermem.PageShift + + req, err := fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(t.ThreadID()), fd.inode().NodeID, linux.FUSE_READ, &in) + if err != nil { + return nil, 0, err + } + + // TODO(gvisor.dev/issue/3247): support async read. + + res, err := fs.conn.Call(t, req) + if err != nil { + return nil, 0, err + } + if err := res.Error(); err != nil { + return nil, 0, err + } + + // Not enough bytes in response, + // either we reached EOF, + // or the FUSE server sends back a response + // that cannot even fit the hdr. + if len(res.data) <= res.hdr.SizeBytes() { + // We treat both case as EOF here for now + // since there is no reliable way to detect + // the over-short hdr case. + break + } + + // Directly using the slice to avoid extra copy. + out := res.data[res.hdr.SizeBytes():] + + outs = append(outs, out) + sizeRead += uint32(len(out)) + + pagesRead += pagesCanRead + } + + defer fs.ReadCallback(ctx, fd, off, size, sizeRead, attributeVersion) + + // No bytes returned: offset >= EOF. + if len(outs) == 0 { + return nil, 0, io.EOF + } + + return outs, sizeRead, nil +} + +// ReadCallback updates several information after receiving a read response. +// Due to readahead, sizeRead can be larger than size. +func (fs *filesystem) ReadCallback(ctx context.Context, fd *regularFileFD, off uint64, size uint32, sizeRead uint32, attributeVersion uint64) { + // TODO(gvisor.dev/issue/3247): support async read. + // If this is called by an async read, correctly process it. + // May need to update the signature. + + i := fd.inode() + // TODO(gvisor.dev/issue/1193): Invalidate or update atime. + + // Reached EOF. + if sizeRead < size { + // TODO(gvisor.dev/issue/3630): If we have writeback cache, then we need to fill this hole. + // Might need to update the buf to be returned from the Read(). + + // Update existing size. + newSize := off + uint64(sizeRead) + fs.conn.mu.Lock() + if attributeVersion == i.attributeVersion && newSize < atomic.LoadUint64(&i.size) { + fs.conn.attributeVersion++ + i.attributeVersion = i.fs.conn.attributeVersion + atomic.StoreUint64(&i.size, newSize) + } + fs.conn.mu.Unlock() + } +} diff --git a/pkg/sentry/fsimpl/fuse/regular_file.go b/pkg/sentry/fsimpl/fuse/regular_file.go new file mode 100644 index 000000000..37ce4e268 --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/regular_file.go @@ -0,0 +1,125 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fuse + +import ( + "io" + "math" + "sync" + "sync/atomic" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/usermem" +) + +type regularFileFD struct { + fileDescription + + // off is the file offset. + off int64 + // offMu protects off. + offMu sync.Mutex +} + +// PRead implements vfs.FileDescriptionImpl.PRead. +func (fd *regularFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) { + if offset < 0 { + return 0, syserror.EINVAL + } + + // Check that flags are supported. + // + // TODO(gvisor.dev/issue/2601): Support select preadv2 flags. + if opts.Flags&^linux.RWF_HIPRI != 0 { + return 0, syserror.EOPNOTSUPP + } + + size := dst.NumBytes() + if size == 0 { + // Early return if count is 0. + return 0, nil + } else if size > math.MaxUint32 { + // FUSE only supports uint32 for size. + // Overflow. + return 0, syserror.EINVAL + } + + // TODO(gvisor.dev/issue/3678): Add direct IO support. + + inode := fd.inode() + + // Reading beyond EOF, update file size if outdated. + if uint64(offset+size) > atomic.LoadUint64(&inode.size) { + if err := inode.reviseAttr(ctx); err != nil { + return 0, err + } + // If the offset after update is still too large, return error. + if uint64(offset) >= atomic.LoadUint64(&inode.size) { + return 0, io.EOF + } + } + + // Truncate the read with updated file size. + fileSize := atomic.LoadUint64(&inode.size) + if uint64(offset+size) > fileSize { + size = int64(fileSize) - offset + } + + buffers, n, err := inode.fs.ReadInPages(ctx, fd, uint64(offset), uint32(size)) + if err != nil { + return 0, err + } + + // TODO(gvisor.dev/issue/3237): support indirect IO (e.g. caching), + // store the bytes that were read ahead. + + // Update the number of bytes to copy for short read. + if n < uint32(size) { + size = int64(n) + } + + // Copy the bytes read to the dst. + // This loop is intended for fragmented reads. + // For the majority of reads, this loop only execute once. + var copied int64 + for _, buffer := range buffers { + toCopy := int64(len(buffer)) + if copied+toCopy > size { + toCopy = size - copied + } + cp, err := dst.DropFirst64(copied).CopyOut(ctx, buffer[:toCopy]) + if err != nil { + return 0, err + } + if int64(cp) != toCopy { + return 0, syserror.EIO + } + copied += toCopy + } + + return copied, nil +} + +// Read implements vfs.FileDescriptionImpl.Read. +func (fd *regularFileFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { + fd.offMu.Lock() + n, err := fd.PRead(ctx, dst, fd.off, opts) + fd.off += n + fd.offMu.Unlock() + return n, err +} diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 8bde81e3c..cae51ce49 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -36,3 +36,8 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:mkdir_test", ) + +syscall_test( + fuse = "True", + test = "//test/fuse/linux:read_test", +) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index 298ea11f8..8afd28f16 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -112,3 +112,16 @@ cc_library( "@com_google_absl//absl/strings:str_format", ], ) + +cc_binary( + name = "read_test", + testonly = 1, + srcs = ["read_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:test_main", + "//test/util:test_util", + ], +) \ No newline at end of file diff --git a/test/fuse/linux/fuse_base.cc b/test/fuse/linux/fuse_base.cc index 98b4e1466..e3c6b585c 100644 --- a/test/fuse/linux/fuse_base.cc +++ b/test/fuse/linux/fuse_base.cc @@ -129,7 +129,8 @@ void FuseTest::SkipServerActualRequest() { // Sends the `kSetInodeLookup` command, expected mode, and the path of the // inode to create under the mount point. -void FuseTest::SetServerInodeLookup(const std::string& path, mode_t mode) { +void FuseTest::SetServerInodeLookup(const std::string& path, mode_t mode, + uint64_t size) { uint32_t cmd = static_cast(FuseTestCmd::kSetInodeLookup); EXPECT_THAT(RetryEINTR(write)(sock_[0], &cmd, sizeof(cmd)), SyscallSucceedsWithValue(sizeof(cmd))); @@ -137,6 +138,9 @@ void FuseTest::SetServerInodeLookup(const std::string& path, mode_t mode) { EXPECT_THAT(RetryEINTR(write)(sock_[0], &mode, sizeof(mode)), SyscallSucceedsWithValue(sizeof(mode))); + EXPECT_THAT(RetryEINTR(write)(sock_[0], &size, sizeof(size)), + SyscallSucceedsWithValue(sizeof(size))); + // Pad 1 byte for null-terminate c-string. EXPECT_THAT(RetryEINTR(write)(sock_[0], path.c_str(), path.size() + 1), SyscallSucceedsWithValue(path.size() + 1)); @@ -144,10 +148,10 @@ void FuseTest::SetServerInodeLookup(const std::string& path, mode_t mode) { WaitServerComplete(); } -void FuseTest::MountFuse() { +void FuseTest::MountFuse(const char* mountOpts) { EXPECT_THAT(dev_fd_ = open("/dev/fuse", O_RDWR), SyscallSucceeds()); - std::string mount_opts = absl::StrFormat("fd=%d,%s", dev_fd_, kMountOpts); + std::string mount_opts = absl::StrFormat("fd=%d,%s", dev_fd_, mountOpts); mount_point_ = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); EXPECT_THAT(mount("fuse", mount_point_.path().c_str(), "fuse", MS_NODEV | MS_NOSUID, mount_opts.c_str()), @@ -311,11 +315,15 @@ void FuseTest::ServerHandleCommand() { // request with this specific path comes in. void FuseTest::ServerReceiveInodeLookup() { mode_t mode; + uint64_t size; std::vector buf(FUSE_MIN_READ_BUFFER); EXPECT_THAT(RetryEINTR(read)(sock_[1], &mode, sizeof(mode)), SyscallSucceedsWithValue(sizeof(mode))); + EXPECT_THAT(RetryEINTR(read)(sock_[1], &size, sizeof(size)), + SyscallSucceedsWithValue(sizeof(size))); + EXPECT_THAT(RetryEINTR(read)(sock_[1], buf.data(), buf.size()), SyscallSucceeds()); @@ -332,6 +340,9 @@ void FuseTest::ServerReceiveInodeLookup() { // comply with the unqiueness of different path. ++nodeid_; + // Set the size. + out_payload.attr.size = size; + memcpy(buf.data(), &out_header, sizeof(out_header)); memcpy(buf.data() + sizeof(out_header), &out_payload, sizeof(out_payload)); lookups_.AddMemBlock(FUSE_LOOKUP, buf.data(), out_len); diff --git a/test/fuse/linux/fuse_base.h b/test/fuse/linux/fuse_base.h index ff4c4499d..452748d6d 100644 --- a/test/fuse/linux/fuse_base.h +++ b/test/fuse/linux/fuse_base.h @@ -137,7 +137,8 @@ class FuseTest : public ::testing::Test { // path, pretending there is an inode and avoid ENOENT when testing. If mode // is not given, it creates a regular file with mode 0600. void SetServerInodeLookup(const std::string& path, - mode_t mode = S_IFREG | S_IRUSR | S_IWUSR); + mode_t mode = S_IFREG | S_IRUSR | S_IWUSR, + uint64_t size = 512); // Called by the testing thread to ask the FUSE server for its next received // FUSE request. Be sure to use the corresponding struct of iovec to receive @@ -166,16 +167,16 @@ class FuseTest : public ::testing::Test { protected: TempPath mount_point_; - // Unmounts the mountpoint of the FUSE server. - void UnmountFuse(); - - private: // Opens /dev/fuse and inherit the file descriptor for the FUSE server. - void MountFuse(); + void MountFuse(const char* mountOpts = kMountOpts); // Creates a socketpair for communication and forks FUSE server. void SetUpFuseServer(); + // Unmounts the mountpoint of the FUSE server. + void UnmountFuse(); + + private: // Sends a FuseTestCmd and gets a uint32_t data from the FUSE server. inline uint32_t GetServerData(uint32_t cmd); diff --git a/test/fuse/linux/read_test.cc b/test/fuse/linux/read_test.cc new file mode 100644 index 000000000..c702651bd --- /dev/null +++ b/test/fuse/linux/read_test.cc @@ -0,0 +1,390 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class ReadTest : public FuseTest { + void SetUp() override { + FuseTest::SetUp(); + test_file_path_ = JoinPath(mount_point_.path().c_str(), test_file_); + } + + // TearDown overrides the parent's function + // to skip checking the unconsumed release request at the end. + void TearDown() override { UnmountFuse(); } + + protected: + const std::string test_file_ = "test_file"; + const mode_t test_file_mode_ = S_IFREG | S_IRWXU | S_IRWXG | S_IRWXO; + const uint64_t test_fh_ = 1; + const uint32_t open_flag_ = O_RDWR; + + std::string test_file_path_; + + PosixErrorOr OpenTestFile(const std::string &path, + uint64_t size = 512) { + SetServerInodeLookup(test_file_, test_file_mode_, size); + + struct fuse_out_header out_header_open = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_open_out), + }; + struct fuse_open_out out_payload_open = { + .fh = test_fh_, + .open_flags = open_flag_, + }; + auto iov_out_open = FuseGenerateIovecs(out_header_open, out_payload_open); + SetServerResponse(FUSE_OPEN, iov_out_open); + + auto res = Open(path.c_str(), open_flag_); + if (res.ok()) { + SkipServerActualRequest(); + } + return res; + } +}; + +class ReadTestSmallMaxRead : public ReadTest { + void SetUp() override { + MountFuse(mountOpts); + SetUpFuseServer(); + test_file_path_ = JoinPath(mount_point_.path().c_str(), test_file_); + } + + protected: + constexpr static char mountOpts[] = + "rootmode=755,user_id=0,group_id=0,max_read=4096"; + // 4096 is hard-coded as the max_read in mount options. + const int size_fragment = 4096; +}; + +TEST_F(ReadTest, ReadWhole) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Prepare for the read. + const int n_read = 5; + std::vector data(n_read); + RandomizeBuffer(data.data(), data.size()); + struct fuse_out_header out_header_read = { + .len = + static_cast(sizeof(struct fuse_out_header) + data.size()), + }; + auto iov_out_read = FuseGenerateIovecs(out_header_read, data); + SetServerResponse(FUSE_READ, iov_out_read); + + // Read the whole "file". + std::vector buf(n_read); + EXPECT_THAT(read(fd.get(), buf.data(), n_read), + SyscallSucceedsWithValue(n_read)); + + // Check the read request. + struct fuse_in_header in_header_read; + struct fuse_read_in in_payload_read; + auto iov_in = FuseGenerateIovecs(in_header_read, in_payload_read); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_header_read.len, + sizeof(in_header_read) + sizeof(in_payload_read)); + EXPECT_EQ(in_header_read.opcode, FUSE_READ); + EXPECT_EQ(in_payload_read.offset, 0); + EXPECT_EQ(buf, data); +} + +TEST_F(ReadTest, ReadPartial) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Prepare for the read. + const int n_data = 10; + std::vector data(n_data); + RandomizeBuffer(data.data(), data.size()); + // Note: due to read ahead, current read implementation will treat any + // response that is longer than requested as correct (i.e. not reach the EOF). + // Therefore, the test below should make sure the size to read does not exceed + // n_data. + struct fuse_out_header out_header_read = { + .len = + static_cast(sizeof(struct fuse_out_header) + data.size()), + }; + auto iov_out_read = FuseGenerateIovecs(out_header_read, data); + struct fuse_in_header in_header_read; + struct fuse_read_in in_payload_read; + auto iov_in = FuseGenerateIovecs(in_header_read, in_payload_read); + + std::vector buf(n_data); + + // Read 1 bytes. + SetServerResponse(FUSE_READ, iov_out_read); + EXPECT_THAT(read(fd.get(), buf.data(), 1), SyscallSucceedsWithValue(1)); + + // Check the 1-byte read request. + GetServerActualRequest(iov_in); + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_header_read.len, + sizeof(in_header_read) + sizeof(in_payload_read)); + EXPECT_EQ(in_header_read.opcode, FUSE_READ); + EXPECT_EQ(in_payload_read.offset, 0); + + // Read 3 bytes. + SetServerResponse(FUSE_READ, iov_out_read); + EXPECT_THAT(read(fd.get(), buf.data(), 3), SyscallSucceedsWithValue(3)); + + // Check the 3-byte read request. + GetServerActualRequest(iov_in); + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_payload_read.offset, 1); + + // Read 5 bytes. + SetServerResponse(FUSE_READ, iov_out_read); + EXPECT_THAT(read(fd.get(), buf.data(), 5), SyscallSucceedsWithValue(5)); + + // Check the 5-byte read request. + GetServerActualRequest(iov_in); + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_payload_read.offset, 4); +} + +TEST_F(ReadTest, PRead) { + const int file_size = 512; + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_, file_size)); + + // Prepare for the read. + const int n_read = 5; + std::vector data(n_read); + RandomizeBuffer(data.data(), data.size()); + struct fuse_out_header out_header_read = { + .len = + static_cast(sizeof(struct fuse_out_header) + data.size()), + }; + auto iov_out_read = FuseGenerateIovecs(out_header_read, data); + SetServerResponse(FUSE_READ, iov_out_read); + + // Read some bytes. + std::vector buf(n_read); + const int offset_read = file_size >> 1; + EXPECT_THAT(pread(fd.get(), buf.data(), n_read, offset_read), + SyscallSucceedsWithValue(n_read)); + + // Check the read request. + struct fuse_in_header in_header_read; + struct fuse_read_in in_payload_read; + auto iov_in = FuseGenerateIovecs(in_header_read, in_payload_read); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_header_read.len, + sizeof(in_header_read) + sizeof(in_payload_read)); + EXPECT_EQ(in_header_read.opcode, FUSE_READ); + EXPECT_EQ(in_payload_read.offset, offset_read); + EXPECT_EQ(buf, data); +} + +TEST_F(ReadTest, ReadZero) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Issue the read. + std::vector buf; + EXPECT_THAT(read(fd.get(), buf.data(), 0), SyscallSucceedsWithValue(0)); +} + +TEST_F(ReadTest, ReadShort) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Prepare for the short read. + const int n_read = 5; + std::vector data(n_read >> 1); + RandomizeBuffer(data.data(), data.size()); + struct fuse_out_header out_header_read = { + .len = + static_cast(sizeof(struct fuse_out_header) + data.size()), + }; + auto iov_out_read = FuseGenerateIovecs(out_header_read, data); + SetServerResponse(FUSE_READ, iov_out_read); + + // Read the whole "file". + std::vector buf(n_read); + EXPECT_THAT(read(fd.get(), buf.data(), n_read), + SyscallSucceedsWithValue(data.size())); + + // Check the read request. + struct fuse_in_header in_header_read; + struct fuse_read_in in_payload_read; + auto iov_in = FuseGenerateIovecs(in_header_read, in_payload_read); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_header_read.len, + sizeof(in_header_read) + sizeof(in_payload_read)); + EXPECT_EQ(in_header_read.opcode, FUSE_READ); + EXPECT_EQ(in_payload_read.offset, 0); + std::vector short_buf(buf.begin(), buf.begin() + data.size()); + EXPECT_EQ(short_buf, data); +} + +TEST_F(ReadTest, ReadShortEOF) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Prepare for the short read. + struct fuse_out_header out_header_read = { + .len = static_cast(sizeof(struct fuse_out_header)), + }; + auto iov_out_read = FuseGenerateIovecs(out_header_read); + SetServerResponse(FUSE_READ, iov_out_read); + + // Read the whole "file". + const int n_read = 10; + std::vector buf(n_read); + EXPECT_THAT(read(fd.get(), buf.data(), n_read), SyscallSucceedsWithValue(0)); + + // Check the read request. + struct fuse_in_header in_header_read; + struct fuse_read_in in_payload_read; + auto iov_in = FuseGenerateIovecs(in_header_read, in_payload_read); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_header_read.len, + sizeof(in_header_read) + sizeof(in_payload_read)); + EXPECT_EQ(in_header_read.opcode, FUSE_READ); + EXPECT_EQ(in_payload_read.offset, 0); +} + +TEST_F(ReadTestSmallMaxRead, ReadSmallMaxRead) { + const int n_fragment = 10; + const int n_read = size_fragment * n_fragment; + + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_, n_read)); + + // Prepare for the read. + std::vector data(size_fragment); + RandomizeBuffer(data.data(), data.size()); + struct fuse_out_header out_header_read = { + .len = + static_cast(sizeof(struct fuse_out_header) + data.size()), + }; + auto iov_out_read = FuseGenerateIovecs(out_header_read, data); + + for (int i = 0; i < n_fragment; ++i) { + SetServerResponse(FUSE_READ, iov_out_read); + } + + // Read the whole "file". + std::vector buf(n_read); + EXPECT_THAT(read(fd.get(), buf.data(), n_read), + SyscallSucceedsWithValue(n_read)); + + ASSERT_EQ(GetServerNumUnsentResponses(), 0); + ASSERT_EQ(GetServerNumUnconsumedRequests(), n_fragment); + + // Check each read segment. + struct fuse_in_header in_header_read; + struct fuse_read_in in_payload_read; + auto iov_in = FuseGenerateIovecs(in_header_read, in_payload_read); + + for (int i = 0; i < n_fragment; ++i) { + GetServerActualRequest(iov_in); + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_header_read.len, + sizeof(in_header_read) + sizeof(in_payload_read)); + EXPECT_EQ(in_header_read.opcode, FUSE_READ); + EXPECT_EQ(in_payload_read.offset, i * size_fragment); + EXPECT_EQ(in_payload_read.size, size_fragment); + + auto it = buf.begin() + i * size_fragment; + EXPECT_EQ(std::vector(it, it + size_fragment), data); + } +} + +TEST_F(ReadTestSmallMaxRead, ReadSmallMaxReadShort) { + const int n_fragment = 10; + const int n_read = size_fragment * n_fragment; + + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_, n_read)); + + // Prepare for the read. + std::vector data(size_fragment); + RandomizeBuffer(data.data(), data.size()); + struct fuse_out_header out_header_read = { + .len = + static_cast(sizeof(struct fuse_out_header) + data.size()), + }; + auto iov_out_read = FuseGenerateIovecs(out_header_read, data); + + for (int i = 0; i < n_fragment - 1; ++i) { + SetServerResponse(FUSE_READ, iov_out_read); + } + + // The last fragment is a short read. + std::vector half_data(data.begin(), data.begin() + (data.size() >> 1)); + struct fuse_out_header out_header_read_short = { + .len = static_cast(sizeof(struct fuse_out_header) + + half_data.size()), + }; + auto iov_out_read_short = + FuseGenerateIovecs(out_header_read_short, half_data); + SetServerResponse(FUSE_READ, iov_out_read_short); + + // Read the whole "file". + std::vector buf(n_read); + EXPECT_THAT(read(fd.get(), buf.data(), n_read), + SyscallSucceedsWithValue(n_read - (data.size() >> 1))); + + ASSERT_EQ(GetServerNumUnsentResponses(), 0); + ASSERT_EQ(GetServerNumUnconsumedRequests(), n_fragment); + + // Check each read segment. + struct fuse_in_header in_header_read; + struct fuse_read_in in_payload_read; + auto iov_in = FuseGenerateIovecs(in_header_read, in_payload_read); + + for (int i = 0; i < n_fragment; ++i) { + GetServerActualRequest(iov_in); + EXPECT_EQ(in_payload_read.fh, test_fh_); + EXPECT_EQ(in_header_read.len, + sizeof(in_header_read) + sizeof(in_payload_read)); + EXPECT_EQ(in_header_read.opcode, FUSE_READ); + EXPECT_EQ(in_payload_read.offset, i * size_fragment); + EXPECT_EQ(in_payload_read.size, size_fragment); + + auto it = buf.begin() + i * size_fragment; + if (i != n_fragment - 1) { + EXPECT_EQ(std::vector(it, it + data.size()), data); + } else { + EXPECT_EQ(std::vector(it, it + half_data.size()), half_data); + } + } +} + +} // namespace + +} // namespace testing +} // namespace gvisor \ No newline at end of file -- cgit v1.2.3 From bc07df88878f47a496f8b364b286bf0b11c0e76f Mon Sep 17 00:00:00 2001 From: Ridwan Sharif Date: Tue, 11 Aug 2020 12:13:01 -0400 Subject: Implement FUSE_RMDIR Fixes #3587 Co-authored-by: Craig Chi --- pkg/abi/linux/fuse.go | 26 ++++++++++++ pkg/sentry/fsimpl/fuse/fusefs.go | 27 +++++++++++++ pkg/sentry/fsimpl/kernfs/filesystem.go | 7 +++- test/fuse/BUILD | 6 +++ test/fuse/linux/BUILD | 14 +++++++ test/fuse/linux/rmdir_test.cc | 73 ++++++++++++++++++++++++++++++++++ 6 files changed, 152 insertions(+), 1 deletion(-) create mode 100644 test/fuse/linux/rmdir_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index 0ece7b756..c75debb8c 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -569,3 +569,29 @@ func (r *FUSEMkdirIn) MarshalUnsafe(buf []byte) { func (r *FUSEMkdirIn) SizeBytes() int { return r.MkdirMeta.SizeBytes() + len(r.Name) + 1 } + +// FUSERmDirIn is the request sent by the kernel to the daemon +// when trying to remove a directory. +// +// Dynamically-sized objects cannot be marshalled. +type FUSERmDirIn struct { + marshal.StubMarshallable + + // Name is a directory name to be looked up. + Name string +} + +// MarshalUnsafe serializes r.name to the dst buffer. +func (r *FUSERmDirIn) MarshalUnsafe(buf []byte) { + copy(buf, r.Name) +} + +// SizeBytes is the size of the memory representation of FUSERmDirIn. +func (r *FUSERmDirIn) SizeBytes() int { + return len(r.Name) + 1 +} + +// UnmarshalUnsafe deserializes r.name from the src buffer. +func (r *FUSERmDirIn) UnmarshalUnsafe(src []byte) { + r.Name = string(src) +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 65e22ba4d..c55ea927a 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -426,6 +426,33 @@ func (i *inode) NewDir(ctx context.Context, name string, opts vfs.MkdirOptions) return i.newEntry(ctx, name, linux.S_IFDIR, linux.FUSE_MKDIR, &in) } +// RmDir implements kernfs.Inode.RmDir. +func (i *inode) RmDir(ctx context.Context, name string, child *vfs.Dentry) error { + fusefs := i.fs + task, creds := kernel.TaskFromContext(ctx), auth.CredentialsFromContext(ctx) + + in := linux.FUSERmDirIn{Name: name} + req, err := fusefs.conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_RMDIR, &in) + if err != nil { + return err + } + + res, err := i.fs.conn.Call(task, req) + if err != nil { + return err + } + if err := res.Error(); err != nil { + return err + } + + // TODO(Before merging): When creating new nodes, should we add nodes to the ordered children? + // If so we'll probably need to call this. We will also need to add them with the writable flag when + // appropriate. + // return i.OrderedChildren.RmDir(ctx, name, child) + + return nil +} + // newEntry calls FUSE server for entry creation and allocates corresponding entry according to response. // Shared by FUSE_MKNOD, FUSE_MKDIR, FUSE_SYMLINK, FUSE_LINK and FUSE_LOOKUP. func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMode, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*vfs.Dentry, error) { diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go index 7aaf1146d..2823c3b1a 100644 --- a/pkg/sentry/fsimpl/kernfs/filesystem.go +++ b/pkg/sentry/fsimpl/kernfs/filesystem.go @@ -657,6 +657,10 @@ func (fs *Filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa func (fs *Filesystem) RmdirAt(ctx context.Context, rp *vfs.ResolvingPath) error { fs.mu.Lock() defer fs.mu.Unlock() + + // Store the name before walkExistingLocked as rp will be advanced past the + // name in the following call. + name := rp.Component() vfsd, inode, err := fs.walkExistingLocked(ctx, rp) fs.processDeferredDecRefsLocked(ctx) if err != nil { @@ -686,7 +690,8 @@ func (fs *Filesystem) RmdirAt(ctx context.Context, rp *vfs.ResolvingPath) error if err := virtfs.PrepareDeleteDentry(mntns, vfsd); err != nil { return err } - if err := parentDentry.inode.RmDir(ctx, rp.Component(), vfsd); err != nil { + + if err := parentDentry.inode.RmDir(ctx, name, vfsd); err != nil { virtfs.AbortDeleteDentry(vfsd) return err } diff --git a/test/fuse/BUILD b/test/fuse/BUILD index cae51ce49..30d2a871f 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -41,3 +41,9 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:read_test", ) + +syscall_test( + test = "//test/fuse/linux:rmdir_test", + vfs2 = "True", + fuse = "True", +) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index 8afd28f16..159428fce 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -98,6 +98,20 @@ cc_binary( ], ) +cc_binary( + name = "rmdir_test", + testonly = 1, + srcs = ["rmdir_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fs_util", + "//test/util:fuse_util", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_library( name = "fuse_base", testonly = 1, diff --git a/test/fuse/linux/rmdir_test.cc b/test/fuse/linux/rmdir_test.cc new file mode 100644 index 000000000..913d3f910 --- /dev/null +++ b/test/fuse/linux/rmdir_test.cc @@ -0,0 +1,73 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fs_util.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class RmDirTest : public FuseTest { + protected: + const std::string test_dir_name_ = "test_dir"; + const mode_t test_dir_mode_ = S_IFDIR | S_IRWXU | S_IRWXG | S_IRWXO; +}; + +TEST_F(RmDirTest, NormalRmDir) { + const std::string test_dir_path_ = + JoinPath(mount_point_.path().c_str(), test_dir_name_); + + SetServerInodeLookup(test_dir_name_, test_dir_mode_); + + // RmDir code. + struct fuse_out_header rmdir_header = { + .len = sizeof(struct fuse_out_header), + }; + + auto iov_out = FuseGenerateIovecs(rmdir_header); + SetServerResponse(FUSE_RMDIR, iov_out); + + ASSERT_THAT(rmdir(test_dir_path_.c_str()), SyscallSucceeds()); + + struct fuse_in_header in_header; + std::vector actual_dirname(test_dir_name_.length() + 1); + auto iov_in = FuseGenerateIovecs(in_header, actual_dirname); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, sizeof(in_header) + test_dir_name_.length() + 1); + EXPECT_EQ(in_header.opcode, FUSE_RMDIR); + EXPECT_EQ(std::string(actual_dirname.data()), test_dir_name_); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From 4a5857d644ae0e62090bbbed86852dceca79395c Mon Sep 17 00:00:00 2001 From: Ridwan Sharif Date: Mon, 27 Jul 2020 14:42:31 -0400 Subject: fuse: Implement IterDirents for directory file description Fixes #3255. This change adds support for IterDirents. You can now use `ls` in the FUSE sandbox. Co-authored-by: Craig Chi --- pkg/abi/linux/fuse.go | 131 ++++++++++++++++++- pkg/sentry/fsimpl/fuse/connection.go | 8 ++ pkg/sentry/fsimpl/fuse/directory.go | 54 ++++++++ pkg/sentry/fsimpl/fuse/file.go | 6 +- pkg/sentry/fsimpl/kernfs/kernfs.go | 2 +- pkg/sentry/vfs/file_description_impl_util.go | 2 +- test/fuse/BUILD | 6 +- test/fuse/linux/BUILD | 14 ++ test/fuse/linux/readdir_test.cc | 188 +++++++++++++++++++++++++++ 9 files changed, 403 insertions(+), 8 deletions(-) create mode 100644 test/fuse/linux/readdir_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index c75debb8c..e7b5f45de 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -14,7 +14,10 @@ package linux -import "gvisor.dev/gvisor/tools/go_marshal/marshal" +import ( + "gvisor.dev/gvisor/tools/go_marshal/marshal" + "gvisor.dev/gvisor/tools/go_marshal/primitive" +) // +marshal type FUSEOpcode uint32 @@ -186,9 +189,9 @@ const ( // Constants relevant to FUSE operations. const ( - FUSE_NAME_MAX = 1024 - FUSE_PAGE_SIZE = 4096 - FUSE_DIRENT_ALIGN = 8 + FUSE_NAME_MAX = 1024 + FUSE_PAGE_SIZE = 4096 + FUSE_DIRENT_ALIGN = 8 ) // FUSEInitIn is the request sent by the kernel to the daemon, @@ -595,3 +598,123 @@ func (r *FUSERmDirIn) SizeBytes() int { func (r *FUSERmDirIn) UnmarshalUnsafe(src []byte) { r.Name = string(src) } + +// FUSEDirents is a list of Dirents received from the FUSE daemon server. +// It is used for FUSE_READDIR. +// +// Dynamically-sized objects cannot be marshalled. +type FUSEDirents struct { + marshal.StubMarshallable + + Dirents []*FUSEDirent +} + +// FUSEDirent is a Dirent received from the FUSE daemon server. +// It is used for FUSE_READDIR. +// +// Dynamically-sized objects cannot be marshalled. +type FUSEDirent struct { + marshal.StubMarshallable + + // Meta contains all the static fields of FUSEDirent. + Meta FUSEDirentMeta + + // Name is the filename of the dirent. + Name string +} + +// FUSEDirentMeta contains all the static fields of FUSEDirent. +// It is used for FUSE_READDIR. +// +// +marshal +type FUSEDirentMeta struct { + // Inode of the dirent. + Ino uint64 + + // Offset of the dirent. + Off uint64 + + // NameLen is the length of the dirent name. + NameLen uint32 + + // Type of the dirent. + Type uint32 +} + +// MarshalUnsafe serializes FUSEDirents to the dst buffer. +func (r *FUSEDirents) MarshalUnsafe(dst []byte) { + for _, dirent := range r.Dirents { + dirent.MarshalUnsafe(dst) + dst = dst[dirent.SizeBytes():] + } +} + +// SizeBytes is the size of the memory representation of FUSEDirents. +func (r *FUSEDirents) SizeBytes() int { + var sizeBytes int + for _, dirent := range r.Dirents { + sizeBytes += dirent.SizeBytes() + } + + return sizeBytes +} + +// UnmarshalUnsafe deserializes FUSEDirents from the src buffer. +func (r *FUSEDirents) UnmarshalUnsafe(src []byte) { + for { + if len(src) <= (*FUSEDirentMeta)(nil).SizeBytes() { + break + } + + // Its unclear how many dirents there are in src. Each dirent is dynamically + // sized and so we can't make assumptions about how many dirents we can allocate. + if r.Dirents == nil { + r.Dirents = make([]*FUSEDirent, 0) + } + + // We have to allocate a struct for each dirent - there must be a better way + // to do this. Linux allocates 1 page to store all the dirents and then + // simply reads them from the page. + var dirent FUSEDirent + dirent.UnmarshalUnsafe(src) + r.Dirents = append(r.Dirents, &dirent) + + src = src[dirent.SizeBytes():] + } +} + +// MarshalUnsafe serializes FUSEDirent to the dst buffer. +func (r *FUSEDirent) MarshalUnsafe(dst []byte) { + r.Meta.MarshalUnsafe(dst) + dst = dst[r.Meta.SizeBytes():] + + name := primitive.ByteSlice(r.Name) + name.MarshalUnsafe(dst) +} + +// SizeBytes is the size of the memory representation of FUSEDirent. +func (r *FUSEDirent) SizeBytes() int { + dataSize := r.Meta.SizeBytes() + len(r.Name) + + // Each Dirent must be padded such that its size is a multiple + // of FUSE_DIRENT_ALIGN. Similar to the fuse dirent alignment + // in linux/fuse.h. + return (dataSize + (FUSE_DIRENT_ALIGN - 1)) & ^(FUSE_DIRENT_ALIGN - 1) +} + +// UnmarshalUnsafe deserializes FUSEDirent from the src buffer. +func (r *FUSEDirent) UnmarshalUnsafe(src []byte) { + r.Meta.UnmarshalUnsafe(src) + src = src[r.Meta.SizeBytes():] + + if r.Meta.NameLen > FUSE_NAME_MAX { + // The name is too long and therefore invalid. We don't + // need to unmarshal the name since it'll be thrown away. + return + } + + buf := make([]byte, r.Meta.NameLen) + name := primitive.ByteSlice(buf) + name.UnmarshalUnsafe(src[:r.Meta.NameLen]) + r.Name = string(name) +} diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 236165652..133306158 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -21,6 +21,8 @@ import ( "sync/atomic" "syscall" + "gvisor.dev/gvisor/tools/go_marshal/marshal" + "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" "gvisor.dev/gvisor/pkg/log" @@ -352,6 +354,12 @@ func (r *Response) UnmarshalPayload(m marshal.Marshallable) error { return fmt.Errorf("payload too small. Minimum data lenth required: %d, but got data length %d", wantDataLen, haveDataLen) } + // The response data is empty unless there is some payload. And so, doesn't + // need to be unmarshalled. + if r.data == nil { + return nil + } + m.UnmarshalUnsafe(r.data[hdrLen:]) return nil } diff --git a/pkg/sentry/fsimpl/fuse/directory.go b/pkg/sentry/fsimpl/fuse/directory.go index 44d41712a..8c59680e8 100644 --- a/pkg/sentry/fsimpl/fuse/directory.go +++ b/pkg/sentry/fsimpl/fuse/directory.go @@ -15,7 +15,12 @@ package fuse import ( + "sync/atomic" + + "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/usermem" @@ -49,3 +54,52 @@ func (directoryFD) PWrite(ctx context.Context, src usermem.IOSequence, offset in func (directoryFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { return 0, syserror.EISDIR } + +// IterDirents implements FileDescriptionImpl.IterDirents. +func (dir *directoryFD) IterDirents(ctx context.Context, callback vfs.IterDirentsCallback) error { + fusefs := dir.inode().fs + task, creds := kernel.TaskFromContext(ctx), auth.CredentialsFromContext(ctx) + + in := linux.FUSEReadIn{ + Fh: dir.Fh, + Offset: uint64(atomic.LoadInt64(&dir.off)), + Size: linux.FUSE_PAGE_SIZE, + Flags: dir.statusFlags(), + } + + /// TODO(gVisor.dev/issue/3404): Support FUSE_READDIRPLUS. + req, err := fusefs.conn.NewRequest(creds, uint32(task.ThreadID()), dir.inode().NodeID, linux.FUSE_READDIR, &in) + if err != nil { + return err + } + + res, err := fusefs.conn.Call(task, req) + if err != nil { + return err + } + if err := res.Error(); err != nil { + return err + } + + var out linux.FUSEDirents + if err := res.UnmarshalPayload(&out); err != nil { + return err + } + + for _, fuseDirent := range out.Dirents { + nextOff := int64(fuseDirent.Meta.Off) + 1 + dirent := vfs.Dirent{ + Name: fuseDirent.Name, + Type: uint8(fuseDirent.Meta.Type), + Ino: fuseDirent.Meta.Ino, + NextOff: nextOff, + } + + if err := callback.Handle(dirent); err != nil { + return err + } + atomic.StoreInt64(&dir.off, nextOff) + } + + return nil +} diff --git a/pkg/sentry/fsimpl/fuse/file.go b/pkg/sentry/fsimpl/fuse/file.go index 01d20caf6..186ec2362 100644 --- a/pkg/sentry/fsimpl/fuse/file.go +++ b/pkg/sentry/fsimpl/fuse/file.go @@ -42,6 +42,9 @@ type fileDescription struct { // OpenFlag is the flag returned by open. OpenFlag uint32 + + // off is the file offset. + off int64 } func (fd *fileDescription) dentry() *kernfs.Dentry { @@ -119,5 +122,6 @@ func (fd *fileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linu // SetStat implements FileDescriptionImpl.SetStat. func (fd *fileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions) error { - return nil + creds := auth.CredentialsFromContext(ctx) + return fd.inode().SetStat(ctx, fd.inode().fs.VFSFilesystem(), creds, opts) } diff --git a/pkg/sentry/fsimpl/kernfs/kernfs.go b/pkg/sentry/fsimpl/kernfs/kernfs.go index f656e2a8b..61189af25 100644 --- a/pkg/sentry/fsimpl/kernfs/kernfs.go +++ b/pkg/sentry/fsimpl/kernfs/kernfs.go @@ -425,7 +425,7 @@ type inodeDynamicLookup interface { Valid(ctx context.Context) bool // IterDirents is used to iterate over dynamically created entries. It invokes - // cb on each entry in the directory represented by the FileDescription. + // cb on each entry in the directory represented by the Inode. // 'offset' is the offset for the entire IterDirents call, which may include // results from the caller (e.g. "." and ".."). 'relOffset' is the offset // inside the entries returned by this IterDirents invocation. In other words, diff --git a/pkg/sentry/vfs/file_description_impl_util.go b/pkg/sentry/vfs/file_description_impl_util.go index 68b80a951..2b668fd89 100644 --- a/pkg/sentry/vfs/file_description_impl_util.go +++ b/pkg/sentry/vfs/file_description_impl_util.go @@ -107,7 +107,7 @@ func (FileDescriptionDefaultImpl) Write(ctx context.Context, src usermem.IOSeque // file_operations::iterate == file_operations::iterate_shared == NULL in // Linux. func (FileDescriptionDefaultImpl) IterDirents(ctx context.Context, cb IterDirentsCallback) error { - return syserror.ENOTDIR + return syserror.ENOSYS } // Seek implements FileDescriptionImpl.Seek analogously to diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 30d2a871f..a1b29aa33 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -43,7 +43,11 @@ syscall_test( ) syscall_test( + fuse = "True", test = "//test/fuse/linux:rmdir_test", - vfs2 = "True", +) + +syscall_test( fuse = "True", + test = "//test/fuse/linux:readdir_test", ) diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index 159428fce..23c9fba31 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -112,6 +112,20 @@ cc_binary( ], ) +cc_binary( + name = "readdir_test", + testonly = 1, + srcs = ["readdir_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fs_util", + "//test/util:fuse_util", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_library( name = "fuse_base", testonly = 1, diff --git a/test/fuse/linux/readdir_test.cc b/test/fuse/linux/readdir_test.cc new file mode 100644 index 000000000..17fb630ee --- /dev/null +++ b/test/fuse/linux/readdir_test.cc @@ -0,0 +1,188 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +#define FUSE_NAME_OFFSET offsetof(struct fuse_dirent, name) +#define FUSE_DIRENT_ALIGN(x) \ + (((x) + sizeof(uint64_t) - 1) & ~(sizeof(uint64_t) - 1)) +#define FUSE_DIRENT_SIZE(d) FUSE_DIRENT_ALIGN(FUSE_NAME_OFFSET + (d)->namelen) + +namespace gvisor { +namespace testing { + +namespace { + +class ReaddirTest : public FuseTest { + public: + void fill_fuse_dirent(char *buf, const char *name) { + size_t namelen = strlen(name); + size_t entlen = FUSE_NAME_OFFSET + namelen; + size_t entlen_padded = FUSE_DIRENT_ALIGN(entlen); + struct fuse_dirent *dirent; + + dirent = reinterpret_cast(buf); + dirent->namelen = namelen; + memcpy(dirent->name, name, namelen); + memset(dirent->name + namelen, 0, entlen_padded - entlen); + } + + protected: + const std::string test_dir_name_ = "test_dir"; +}; + +TEST_F(ReaddirTest, SingleEntry) { + const std::string test_dir_path = + JoinPath(mount_point_.path().c_str(), test_dir_name_); + + // We need to make sure the test dir is a directory that can be found. + mode_t expected_mode = + S_IFDIR | S_IRWXU | S_IRGRP | S_IXGRP | S_IROTH | S_IXOTH; + struct fuse_attr dir_attr = { + .ino = 1, + .size = 512, + .blocks = 4, + .mode = expected_mode, + .blksize = 4096, + }; + struct fuse_out_header stat_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + }; + + struct fuse_attr_out stat_payload = { + .attr_valid_nsec = 2, + .attr = dir_attr, + }; + + // We need to make sure the test dir is a directory that can be found. + struct fuse_out_header lookup_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_entry_out), + }; + struct fuse_entry_out lookup_payload = { + .nodeid = 1, + .entry_valid = true, + .attr_valid = true, + .attr = dir_attr, + }; + + struct fuse_out_header open_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_open_out), + }; + struct fuse_open_out open_payload = { + .fh = 1, + }; + auto iov_out = FuseGenerateIovecs(lookup_header, lookup_payload); + SetServerResponse(FUSE_LOOKUP, iov_out); + + iov_out = FuseGenerateIovecs(open_header, open_payload); + SetServerResponse(FUSE_OPENDIR, iov_out); + + iov_out = FuseGenerateIovecs(stat_header, stat_payload); + SetServerResponse(FUSE_GETATTR, iov_out); + + DIR *dir = opendir(test_dir_path.c_str()); + + // The opendir command makes three syscalls. Lookup the dir file, stat it and + // open. + // We don't need to inspect those headers in this test. + SkipServerActualRequest(); // LOOKUP. + SkipServerActualRequest(); // GETATTR. + SkipServerActualRequest(); // OPENDIR. + + // Readdir test code. + std::string dot = "."; + std::string dot_dot = ".."; + std::string test_file = "testFile"; + + // Figure out how many dirents to send over and allocate them appropriately. + // Each dirent has a dynamic name and a static metadata part. The dirent size + // is aligned to being a multiple of 8. + size_t dot_file_dirent_size = + FUSE_DIRENT_ALIGN(dot.length() + FUSE_NAME_OFFSET); + size_t dot_dot_file_dirent_size = + FUSE_DIRENT_ALIGN(dot_dot.length() + FUSE_NAME_OFFSET); + size_t test_file_dirent_size = + FUSE_DIRENT_ALIGN(test_file.length() + FUSE_NAME_OFFSET); + + // Create an appropriately sized payload. + size_t readdir_payload_size = + test_file_dirent_size + dot_file_dirent_size + dot_dot_file_dirent_size; + char readdir_payload[readdir_payload_size]; + + fill_fuse_dirent(readdir_payload, dot.c_str()); + fill_fuse_dirent(readdir_payload + dot_file_dirent_size, dot_dot.c_str()); + fill_fuse_dirent( + readdir_payload + dot_file_dirent_size + dot_dot_file_dirent_size, + test_file.c_str()); + + std::vector readdir_payload_vec(readdir_payload, + readdir_payload + readdir_payload_size); + struct fuse_out_header readdir_header = { + .len = uint32_t(sizeof(struct fuse_out_header) + sizeof(readdir_payload)), + }; + struct fuse_out_header readdir_header_break = { + .len = uint32_t(sizeof(struct fuse_out_header)), + }; + + iov_out = FuseGenerateIovecs(readdir_header, readdir_payload_vec); + SetServerResponse(FUSE_READDIR, iov_out); + + iov_out = FuseGenerateIovecs(readdir_header_break); + SetServerResponse(FUSE_READDIR, iov_out); + + struct dirent *entry; + entry = readdir(dir); + EXPECT_EQ(std::string(entry->d_name), dot); + + entry = readdir(dir); + EXPECT_EQ(std::string(entry->d_name), dot_dot); + + entry = readdir(dir); + EXPECT_EQ(std::string(entry->d_name), test_file); + + entry = readdir(dir); + EXPECT_TRUE((entry == NULL)); + + SkipServerActualRequest(); // READDIR. + SkipServerActualRequest(); // READDIR with no data. + + // Clean up. + closedir(dir); + + struct fuse_in_header in_header; + struct fuse_release_in in_payload; + + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_RELEASEDIR); +} + +} // namespace + +} // namespace testing +} // namespace gvisor \ No newline at end of file -- cgit v1.2.3 From d51ddcefdc46a1007f7345bfa2f7006bb820b157 Mon Sep 17 00:00:00 2001 From: Ridwan Sharif Date: Wed, 19 Aug 2020 20:11:59 -0400 Subject: fuse: use safe go_marshal API for FUSE Until #3698 is resolved, this change is needed to ensure we're not corrupting memory anywhere. --- pkg/abi/linux/fuse.go | 97 +++++++++++++++++++++++++++++++++++- pkg/marshal/marshal_impl_util.go | 2 +- pkg/sentry/fsimpl/fuse/connection.go | 9 ++-- 3 files changed, 103 insertions(+), 5 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index e7b5f45de..d105c5176 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -359,7 +359,12 @@ type FUSELookupIn struct { // MarshalUnsafe serializes r.name to the dst buffer. func (r *FUSELookupIn) MarshalUnsafe(buf []byte) { - copy(buf, []byte(r.Name)) + copy(buf, r.Name) +} + +// MarshalBytes serializes r.name to the dst buffer. +func (r *FUSELookupIn) MarshalBytes(buf []byte) { + copy(buf, r.Name) } // SizeBytes is the size of the memory representation of FUSELookupIn. @@ -491,6 +496,12 @@ func (r *FUSEMknodIn) MarshalUnsafe(buf []byte) { copy(buf[r.MknodMeta.SizeBytes():], r.Name) } +// MarshalBytes serializes r.MknodMeta and r.Name to the dst buffer. +func (r *FUSEMknodIn) MarshalBytes(buf []byte) { + r.MknodMeta.MarshalBytes(buf[:r.MknodMeta.SizeBytes()]) + copy(buf[r.MknodMeta.SizeBytes():], r.Name) +} + // SizeBytes is the size of the memory representation of FUSEMknodIn. // 1 extra byte for null-terminated string. func (r *FUSEMknodIn) SizeBytes() int { @@ -518,6 +529,13 @@ func (r *FUSESymLinkIn) MarshalUnsafe(buf []byte) { copy(buf[len(r.Name)+1:], r.Target) } +// MarshalBytes serializes r.Name and r.Target to the dst buffer. +// Left null-termination at end of r.Name and r.Target. +func (r *FUSESymLinkIn) MarshalBytes(buf []byte) { + copy(buf, r.Name) + copy(buf[len(r.Name)+1:], r.Target) +} + // SizeBytes is the size of the memory representation of FUSESymLinkIn. // 2 extra bytes for null-terminated string. func (r *FUSESymLinkIn) SizeBytes() int { @@ -530,6 +548,9 @@ type FUSEEmptyIn struct{ marshal.StubMarshallable } // MarshalUnsafe do nothing for marshal. func (r *FUSEEmptyIn) MarshalUnsafe(buf []byte) {} +// MarshalBytes do nothing for marshal. +func (r *FUSEEmptyIn) MarshalBytes(buf []byte) {} + // SizeBytes is 0 for empty request. func (r *FUSEEmptyIn) SizeBytes() int { return 0 @@ -567,6 +588,12 @@ func (r *FUSEMkdirIn) MarshalUnsafe(buf []byte) { copy(buf[r.MkdirMeta.SizeBytes():], r.Name) } +// MarshalBytes serializes r.MkdirMeta and r.Name to the dst buffer. +func (r *FUSEMkdirIn) MarshalBytes(buf []byte) { + r.MkdirMeta.MarshalBytes(buf[:r.MkdirMeta.SizeBytes()]) + copy(buf[r.MkdirMeta.SizeBytes():], r.Name) +} + // SizeBytes is the size of the memory representation of FUSEMkdirIn. // 1 extra byte for null-terminated Name string. func (r *FUSEMkdirIn) SizeBytes() int { @@ -589,6 +616,11 @@ func (r *FUSERmDirIn) MarshalUnsafe(buf []byte) { copy(buf, r.Name) } +// MarshalBytes serializes r.name to the dst buffer. +func (r *FUSERmDirIn) MarshalBytes(buf []byte) { + copy(buf, r.Name) +} + // SizeBytes is the size of the memory representation of FUSERmDirIn. func (r *FUSERmDirIn) SizeBytes() int { return len(r.Name) + 1 @@ -599,6 +631,11 @@ func (r *FUSERmDirIn) UnmarshalUnsafe(src []byte) { r.Name = string(src) } +// UnmarshalBytes deserializes r.name from the src buffer. +func (r *FUSERmDirIn) UnmarshalBytes(src []byte) { + r.Name = string(src) +} + // FUSEDirents is a list of Dirents received from the FUSE daemon server. // It is used for FUSE_READDIR. // @@ -649,6 +686,14 @@ func (r *FUSEDirents) MarshalUnsafe(dst []byte) { } } +// MarshalBytes serializes FUSEDirents to the dst buffer. +func (r *FUSEDirents) MarshalBytes(dst []byte) { + for _, dirent := range r.Dirents { + dirent.MarshalBytes(dst) + dst = dst[dirent.SizeBytes():] + } +} + // SizeBytes is the size of the memory representation of FUSEDirents. func (r *FUSEDirents) SizeBytes() int { var sizeBytes int @@ -683,6 +728,30 @@ func (r *FUSEDirents) UnmarshalUnsafe(src []byte) { } } +// UnmarshalBytes deserializes FUSEDirents from the src buffer. +func (r *FUSEDirents) UnmarshalBytes(src []byte) { + for { + if len(src) <= (*FUSEDirentMeta)(nil).SizeBytes() { + break + } + + // Its unclear how many dirents there are in src. Each dirent is dynamically + // sized and so we can't make assumptions about how many dirents we can allocate. + if r.Dirents == nil { + r.Dirents = make([]*FUSEDirent, 0) + } + + // We have to allocate a struct for each dirent - there must be a better way + // to do this. Linux allocates 1 page to store all the dirents and then + // simply reads them from the page. + var dirent FUSEDirent + dirent.UnmarshalBytes(src) + r.Dirents = append(r.Dirents, &dirent) + + src = src[dirent.SizeBytes():] + } +} + // MarshalUnsafe serializes FUSEDirent to the dst buffer. func (r *FUSEDirent) MarshalUnsafe(dst []byte) { r.Meta.MarshalUnsafe(dst) @@ -692,6 +761,15 @@ func (r *FUSEDirent) MarshalUnsafe(dst []byte) { name.MarshalUnsafe(dst) } +// MarshalBytes serializes FUSEDirent to the dst buffer. +func (r *FUSEDirent) MarshalBytes(dst []byte) { + r.Meta.MarshalBytes(dst) + dst = dst[r.Meta.SizeBytes():] + + name := primitive.ByteSlice(r.Name) + name.MarshalBytes(dst) +} + // SizeBytes is the size of the memory representation of FUSEDirent. func (r *FUSEDirent) SizeBytes() int { dataSize := r.Meta.SizeBytes() + len(r.Name) @@ -718,3 +796,20 @@ func (r *FUSEDirent) UnmarshalUnsafe(src []byte) { name.UnmarshalUnsafe(src[:r.Meta.NameLen]) r.Name = string(name) } + +// UnmarshalBytes deserializes FUSEDirent from the src buffer. +func (r *FUSEDirent) UnmarshalBytes(src []byte) { + r.Meta.UnmarshalBytes(src) + src = src[r.Meta.SizeBytes():] + + if r.Meta.NameLen > FUSE_NAME_MAX { + // The name is too long and therefore invalid. We don't + // need to unmarshal the name since it'll be thrown away. + return + } + + buf := make([]byte, r.Meta.NameLen) + name := primitive.ByteSlice(buf) + name.UnmarshalBytes(src[:r.Meta.NameLen]) + r.Name = string(name) +} diff --git a/pkg/marshal/marshal_impl_util.go b/pkg/marshal/marshal_impl_util.go index 69f33321e..ea75e09f2 100644 --- a/pkg/marshal/marshal_impl_util.go +++ b/pkg/marshal/marshal_impl_util.go @@ -44,7 +44,7 @@ func (StubMarshallable) MarshalBytes(dst []byte) { // UnmarshalBytes implements Marshallable.UnmarshalBytes. func (StubMarshallable) UnmarshalBytes(src []byte) { - panic("Please implement your own UnMarshalBytes function") + panic("Please implement your own UnmarshalBytes function") } // Packed implements Marshallable.Packed. diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 133306158..0e70d37ec 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -271,8 +271,10 @@ func (conn *connection) NewRequest(creds *auth.Credentials, pid uint32, ino uint } buf := make([]byte, hdr.Len) - hdr.MarshalUnsafe(buf[:hdrLen]) - payload.MarshalUnsafe(buf[hdrLen:]) + + // TODO(gVisor.dev/3698): Use the unsafe version once go_marshal is safe to use again. + hdr.MarshalBytes(buf[:hdrLen]) + payload.MarshalBytes(buf[hdrLen:]) return &Request{ id: hdr.Unique, @@ -360,7 +362,8 @@ func (r *Response) UnmarshalPayload(m marshal.Marshallable) error { return nil } - m.UnmarshalUnsafe(r.data[hdrLen:]) + // TODO(gVisor.dev/3698): Use the unsafe version once go_marshal is safe to use again. + m.UnmarshalBytes(r.data[hdrLen:]) return nil } -- cgit v1.2.3 From 21cac9dd042f3446258387378a743b8a7cd76443 Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Wed, 19 Aug 2020 22:12:15 -0700 Subject: Fix FUSE_READDIR offset issue According to readdir(3), the offset attribute in struct dirent is the offset to the next dirent instead of the offset of itself. Send the successive FUSE_READDIR requests with the offset retrieved from the last entry. Updates #3255 --- pkg/sentry/fsimpl/fuse/directory.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/directory.go b/pkg/sentry/fsimpl/fuse/directory.go index 8c59680e8..a83357129 100644 --- a/pkg/sentry/fsimpl/fuse/directory.go +++ b/pkg/sentry/fsimpl/fuse/directory.go @@ -87,7 +87,7 @@ func (dir *directoryFD) IterDirents(ctx context.Context, callback vfs.IterDirent } for _, fuseDirent := range out.Dirents { - nextOff := int64(fuseDirent.Meta.Off) + 1 + nextOff := int64(fuseDirent.Meta.Off) dirent := vfs.Dirent{ Name: fuseDirent.Name, Type: uint8(fuseDirent.Meta.Type), -- cgit v1.2.3 From 983e30c01616b40348735f894d42bbad204f6b99 Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Thu, 20 Aug 2020 10:40:41 -0700 Subject: Implementing inode.Getlink kernfs uses inode.Getlink to resolve symlink when look up paths. Updates #3452 --- pkg/sentry/fsimpl/fuse/fusefs.go | 6 ++++++ 1 file changed, 6 insertions(+) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index c55ea927a..e1bbb4b52 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -488,6 +488,12 @@ func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMo return child.VFSDentry(), nil } +// Getlink implements Inode.Getlink. +func (i *inode) Getlink(ctx context.Context, mnt *vfs.Mount) (vfs.VirtualDentry, string, error) { + path, err := i.Readlink(ctx, mnt) + return vfs.VirtualDentry{}, path, err +} + // Readlink implements kernfs.Inode.Readlink. func (i *inode) Readlink(ctx context.Context, mnt *vfs.Mount) (string, error) { if i.Mode().FileType()&linux.S_IFLNK == 0 { -- cgit v1.2.3 From 449986264f9277c4c6174fc82294fc6644923e8b Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Thu, 20 Aug 2020 12:31:52 -0700 Subject: Support multiple FUSE kernel versions of FUSE_INIT response struct The fuse_init_out struct changes in different FUSE kernel versions. A FUSE server may implement older versions of fuse_init_out, but they share common attributes from the beginning. Implement variable-length marshallable interface to support older versions of ABI. Fixes #3707 --- pkg/abi/linux/fuse.go | 58 ++++++++++++++++++++++++++++++++++++ pkg/sentry/fsimpl/fuse/connection.go | 5 ++++ pkg/sentry/fsimpl/fuse/init.go | 6 ++-- 3 files changed, 66 insertions(+), 3 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index d105c5176..ca4ee5e80 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -15,6 +15,7 @@ package linux import ( + "gvisor.dev/gvisor/pkg/usermem" "gvisor.dev/gvisor/tools/go_marshal/marshal" "gvisor.dev/gvisor/tools/go_marshal/primitive" ) @@ -262,6 +263,63 @@ type FUSEInitOut struct { _ [8]uint32 } +// FUSEInitRes is a variable-length wrapper of FUSEInitOut. The FUSE server may +// implement older version of FUSE protocol, which contains a FUSEInitOut with +// less attributes. +// +// Dynamically-sized objects cannot be marshalled. +type FUSEInitRes struct { + marshal.StubMarshallable + + // InitOut contains the response from the FUSE server. + InitOut FUSEInitOut + + // Len is the total length of bytes of the response. + Len uint32 +} + +// UnMarshalBytes deserializes src to the InitOut attribute in a FUSEInitRes. +func (r *FUSEInitRes) UnmarshalBytes(src []byte) { + out := &r.InitOut + + // Introduced before FUSE kernel version 7.13. + out.Major = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + out.Minor = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + out.MaxReadahead = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + out.Flags = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + out.MaxBackground = uint16(usermem.ByteOrder.Uint16(src[:2])) + src = src[2:] + out.CongestionThreshold = uint16(usermem.ByteOrder.Uint16(src[:2])) + src = src[2:] + out.MaxWrite = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + + // Introduced in FUSE kernel version 7.23. + if len(src) >= 4 { + out.TimeGran = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + } + // Introduced in FUSE kernel version 7.28. + if len(src) >= 2 { + out.MaxPages = uint16(usermem.ByteOrder.Uint16(src[:2])) + src = src[2:] + } + // Introduced in FUSE kernel version 7.31. + if len(src) >= 2 { + out.MapAlignment = uint16(usermem.ByteOrder.Uint16(src[:2])) + src = src[2:] + } +} + +// SizeBytes is the size of the payload of the FUSE_INIT response. +func (r *FUSEInitRes) SizeBytes() int { + return int(r.Len) +} + // FUSEGetAttrIn is the request sent by the kernel to the daemon, // to get the attribute of a inode. // diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 0e70d37ec..dc169b497 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -346,6 +346,11 @@ func (r *Response) Error() error { return error(sysErrNo) } +// DataLen returns the size of the response without the header. +func (r *Response) DataLen() uint32 { + return r.hdr.Len - uint32(r.hdr.SizeBytes()) +} + // UnmarshalPayload unmarshals the response data into m. func (r *Response) UnmarshalPayload(m marshal.Marshallable) error { hdrLen := r.hdr.SizeBytes() diff --git a/pkg/sentry/fsimpl/fuse/init.go b/pkg/sentry/fsimpl/fuse/init.go index 2ff2542b6..6384cbbdb 100644 --- a/pkg/sentry/fsimpl/fuse/init.go +++ b/pkg/sentry/fsimpl/fuse/init.go @@ -76,12 +76,12 @@ func (conn *connection) InitRecv(res *Response, hasSysAdminCap bool) error { return err } - var out linux.FUSEInitOut - if err := res.UnmarshalPayload(&out); err != nil { + initRes := linux.FUSEInitRes{Len: res.DataLen()} + if err := res.UnmarshalPayload(&initRes); err != nil { return err } - return conn.initProcessReply(&out, hasSysAdminCap) + return conn.initProcessReply(&initRes.InitOut, hasSysAdminCap) } // Process the FUSE_INIT reply from the FUSE server. -- cgit v1.2.3 From cb9a2a1ad4f568a21382e949a592b621c11b5a2c Mon Sep 17 00:00:00 2001 From: Boyuan He & Ridwan Sharif Date: Wed, 26 Aug 2020 15:26:46 -0400 Subject: fuse: add benchmarking support for FUSE This change adds the following: - Add support for containerizing syscall tests for FUSE - Mount tmpfs in the container so we can run benchmarks against it - Run the server in a background process - benchmarks for fuse syscall Co-authored-by: Ridwan Sharif --- Makefile | 4 ++ images/basic/fuse/Dockerfile | 15 +++++ pkg/sentry/fs/g3doc/fuse.md | 47 +++++++++++++ pkg/test/dockerutil/container.go | 10 +++ pkg/test/dockerutil/dockerutil.go | 20 ++++++ scripts/common_build.sh | 5 +- test/e2e/integration_test.go | 6 +- test/fuse/BUILD | 46 +++++++++++++ test/fuse/benchmark/BUILD | 91 +++++++++++++++++++++++++ test/fuse/benchmark/mkdir_benchmark.cc | 51 ++++++++++++++ test/fuse/benchmark/open_benchmark.cc | 60 +++++++++++++++++ test/fuse/benchmark/read_benchmark.cc | 57 ++++++++++++++++ test/fuse/benchmark/stat_benchmark.cc | 65 ++++++++++++++++++ test/fuse/benchmark/symlink_benchmark.cc | 60 +++++++++++++++++ test/image/BUILD | 1 + test/image/image_test.go | 52 ++++++++++++++ test/runner/BUILD | 3 + test/runner/defs.bzl | 28 ++++++++ test/runner/runner.go | 112 ++++++++++++++++++++++++++++--- 19 files changed, 717 insertions(+), 16 deletions(-) create mode 100644 images/basic/fuse/Dockerfile create mode 100644 test/fuse/benchmark/BUILD create mode 100644 test/fuse/benchmark/mkdir_benchmark.cc create mode 100644 test/fuse/benchmark/open_benchmark.cc create mode 100644 test/fuse/benchmark/read_benchmark.cc create mode 100644 test/fuse/benchmark/stat_benchmark.cc create mode 100644 test/fuse/benchmark/symlink_benchmark.cc (limited to 'pkg/sentry') diff --git a/Makefile b/Makefile index 4f18501a5..adf2189f9 100644 --- a/Makefile +++ b/Makefile @@ -341,6 +341,10 @@ RUNTIME_BIN := $(RUNTIME_DIR)/runsc RUNTIME_LOG_DIR := $(RUNTIME_DIR)/logs RUNTIME_LOGS := $(RUNTIME_LOG_DIR)/runsc.log.%TEST%.%TIMESTAMP%.%COMMAND% +ifeq (,$(RUNTIME_NAME)) +RUNTIME_NAME := $(RUNTIME) +endif + dev: ## Installs a set of local runtimes. Requires sudo. @$(call submake,refresh ARGS="--net-raw") @$(call submake,configure RUNTIME_NAME="$(RUNTIME)" ARGS="--net-raw") diff --git a/images/basic/fuse/Dockerfile b/images/basic/fuse/Dockerfile new file mode 100644 index 000000000..9e88aa2c5 --- /dev/null +++ b/images/basic/fuse/Dockerfile @@ -0,0 +1,15 @@ +FROM ubuntu:20.04 + +ENV DEBIAN_FRONTEND=noninteractive + +RUN apt-get update +RUN apt-get install -y build-essential git pkg-config fuse3 libfuse3-3 libfuse3-dev strace + +WORKDIR /fus + +RUN mkdir -pv mountpoint +RUN git clone https://github.com/libfuse/libfuse + +RUN gcc -Wall ./libfuse/example/passthrough.c `pkg-config fuse3 --cflags --libs` -o server-bin + +CMD ["bash"] diff --git a/pkg/sentry/fs/g3doc/fuse.md b/pkg/sentry/fs/g3doc/fuse.md index 2ca84dd74..496e339ce 100644 --- a/pkg/sentry/fs/g3doc/fuse.md +++ b/pkg/sentry/fs/g3doc/fuse.md @@ -254,6 +254,53 @@ I/O syscalls like `read(2)`, `write(2)` and `mmap(2)`. - `FUSE_BMAP`: Old address space API for block defrag. Probably not needed. - `FUSE_NOTIFY_REPLY`: [TODO: what does this do?] +## Benchmark FUSE + +FUSE benchmark makes FUSE syscall inside docker container to make sure required +environment conditions are met - such as having the right libraries to start a +FUSE server. + +### Setup + +To run benchmark: + +1. Make sure you have `Docker` installed. +2. Download all docker images `make load-all-images`. +3. Config `runsc` docker runtime to have VFS2 and FUSE supported. +(e.g. `make configure RUNTIME=runsc ARGS="--vfs2 --fuse ..." ...`) + +You should now have a runtime with the following options configured in +`/etc/docker/daemon.json` +``` +"runsc": { + "path": "path/to/your/runsc", + "runtimeArgs": [ + "--vfs2", + "--fuse" + ... + ] + } +``` + +### Running benchmarks +With above setup, benchmark can be run with following command +``` +bazel test --test_output=all --cache_test_results=no --test_arg=-test.bench= //path/to:target +``` +For example: if you want to run stat test +``` +bazel test --test_output=all --cache_test_results=no --test_arg=-test.bench= //test/fuse:open_benchmark_runsc_ptrace_vfs2_fuse_container +``` + +Note: +- test target need to have `vfs2_fuse_container` to run in container with `vfs2` and `fuse` enabled +- `test_output` set to `all` to view the result in terminal +- `--cache_test_results` set to `no` to avoid cached benchmark + +### Use your fuse server + +To use your own FUSE server, change the `images/basic/fuse/Dockerfile` to compile your FUSE server into the container and name it `server-bin`. + # References - [fuse(4) Linux manual page](https://www.man7.org/linux/man-pages/man4/fuse.4.html) diff --git a/pkg/test/dockerutil/container.go b/pkg/test/dockerutil/container.go index 64d17f661..727be26b2 100644 --- a/pkg/test/dockerutil/container.go +++ b/pkg/test/dockerutil/container.go @@ -136,6 +136,11 @@ func MakeNativeContainer(ctx context.Context, logger testutil.Logger) *Container } } +// Runtime returns the runtime of the container. +func (c *Container) Runtime() string { + return c.runtime +} + // AddProfile adds a profile to this container. func (c *Container) AddProfile(p Profile) { c.profiles = append(c.profiles, p) @@ -541,3 +546,8 @@ func (c *Container) CleanUp(ctx context.Context) { // Forget all mounts. c.mounts = nil } + +// CopyErr returns the error that happened during copy. +func (c *Container) CopyErr() error { + return c.copyErr +} diff --git a/pkg/test/dockerutil/dockerutil.go b/pkg/test/dockerutil/dockerutil.go index 7027df1a5..a2d7e8c85 100644 --- a/pkg/test/dockerutil/dockerutil.go +++ b/pkg/test/dockerutil/dockerutil.go @@ -121,6 +121,26 @@ func UsingVFS2() (bool, error) { return false, nil } +// UsingFUSE returns true if the 'runtime' has the fuse flag set. +func UsingFUSE() (bool, error) { + rMap, err := runtimeMap() + if err != nil { + return false, err + } + + list, ok := rMap["runtimeArgs"].([]interface{}) + if !ok { + return false, fmt.Errorf("unexpected format: %v", rMap) + } + + for _, element := range list { + if element == "--fuse" { + return true, nil + } + } + return false, nil +} + func runtimeMap() (map[string]interface{}, error) { // Read the configuration data; the file must exist. configBytes, err := ioutil.ReadFile(*config) diff --git a/scripts/common_build.sh b/scripts/common_build.sh index d4a6c4908..6874e56f9 100755 --- a/scripts/common_build.sh +++ b/scripts/common_build.sh @@ -109,8 +109,9 @@ function collect_logs() { } function find_branch_name() { - git branch --show-current \ + (git branch --show-current \ || git rev-parse HEAD \ || bazel info workspace \ - | xargs basename + | xargs basename) \ + | tr '/' '-' } diff --git a/test/e2e/integration_test.go b/test/e2e/integration_test.go index 809244bab..0c82e98d4 100644 --- a/test/e2e/integration_test.go +++ b/test/e2e/integration_test.go @@ -168,10 +168,10 @@ func TestCheckpointRestore(t *testing.T) { } // TODO(gvisor.dev/issue/3373): Remove after implementing. - if usingVFS2, err := dockerutil.UsingVFS2(); usingVFS2 { - t.Skip("CheckpointRestore not implemented in VFS2.") - } else if err != nil { + if usingVFS2, err := dockerutil.UsingVFS2(); err != nil { t.Fatalf("failed to read config for runtime %s: %v", dockerutil.Runtime(), err) + } else if usingVFS2 { + t.Skip("CheckpointRestore not implemented in VFS2.") } ctx := context.Background() diff --git a/test/fuse/BUILD b/test/fuse/BUILD index a1b29aa33..02498b3a1 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -51,3 +51,49 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:readdir_test", ) + + +syscall_test( + size = "large", + add_overlay = True, + debug = False, + setup_command = "'./server-bin mountpoint'", + test = "//test/fuse/benchmark:stat_benchmark", + use_image = "basic/fuse", +) + +syscall_test( + size = "large", + add_overlay = True, + debug = False, + setup_command = "'./server-bin mountpoint'", + test = "//test/fuse/benchmark:open_benchmark", + use_image = "basic/fuse", +) + +syscall_test( + size = "large", + add_overlay = True, + debug = False, + setup_command = "'./server-bin mountpoint'", + test = "//test/fuse/benchmark:read_benchmark", + use_image = "basic/fuse", +) + +syscall_test( + size = "large", + add_overlay = True, + debug = False, + setup_command = "'./server-bin mountpoint'", + test = "//test/fuse/benchmark:symlink_benchmark", + use_image = "basic/fuse", +) + +syscall_test( + size = "large", + add_overlay = True, + debug = False, + setup_command = "'./server-bin mountpoint'", + test = "//test/fuse/benchmark:mkdir_benchmark", + use_image = "basic/fuse", +) diff --git a/test/fuse/benchmark/BUILD b/test/fuse/benchmark/BUILD new file mode 100644 index 000000000..16369d99b --- /dev/null +++ b/test/fuse/benchmark/BUILD @@ -0,0 +1,91 @@ +load("//tools:defs.bzl", "cc_binary", "gbenchmark", "gtest") + +package( + default_visibility = ["//:sandbox"], + licenses = ["notice"], +) + +cc_binary( + name = "stat_benchmark", + testonly = 1, + srcs = [ + "stat_benchmark.cc", + ], + deps = [ + gbenchmark, + gtest, + "//test/util:fs_util", + "//test/util:temp_path", + "//test/util:test_main", + "//test/util:test_util", + "@com_google_absl//absl/strings", + ], +) + +cc_binary( + name = "open_benchmark", + testonly = 1, + srcs = [ + "open_benchmark.cc", + ], + deps = [ + gbenchmark, + gtest, + "//test/util:fs_util", + "//test/util:temp_path", + "//test/util:test_main", + "//test/util:test_util", + "@com_google_absl//absl/strings", + ], +) + +cc_binary( + name = "read_benchmark", + testonly = 1, + srcs = [ + "read_benchmark.cc", + ], + deps = [ + gbenchmark, + gtest, + "//test/util:fs_util", + "//test/util:temp_path", + "//test/util:test_main", + "//test/util:test_util", + "@com_google_absl//absl/strings", + ], +) + +cc_binary( + name = "symlink_benchmark", + testonly = 1, + srcs = [ + "symlink_benchmark.cc", + ], + deps = [ + gbenchmark, + gtest, + "//test/util:fs_util", + "//test/util:temp_path", + "//test/util:test_main", + "//test/util:test_util", + "@com_google_absl//absl/strings", + ], +) + +cc_binary( + name = "mkdir_benchmark", + testonly = 1, + srcs = [ + "mkdir_benchmark.cc", + ], + deps = [ + gbenchmark, + gtest, + "//test/util:fs_util", + "//test/util:temp_path", + "//test/util:test_main", + "//test/util:test_util", + "@com_google_absl//absl/strings", + ], +) diff --git a/test/fuse/benchmark/mkdir_benchmark.cc b/test/fuse/benchmark/mkdir_benchmark.cc new file mode 100644 index 000000000..30759603e --- /dev/null +++ b/test/fuse/benchmark/mkdir_benchmark.cc @@ -0,0 +1,51 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +#include "absl/strings/str_cat.h" +#include "benchmark/benchmark.h" +#include "gtest/gtest.h" +#include "test/util/fs_util.h" +#include "test/util/temp_path.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +void BM_Mkdir(benchmark::State& state) { + const char* fuse_prefix = getenv("TEST_FUSEPRE"); + ASSERT_NE(fuse_prefix, nullptr); + + const TempPath top_dir = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); + std::string dir_path = top_dir.path(); + + int index = 0; + for (auto t : state) { + const std::string new_dir_path = absl::StrCat(dir_path, index); + ASSERT_THAT(mkdir(new_dir_path.c_str(), 0777), SyscallSucceeds()); + index++; + } +} + +BENCHMARK(BM_Mkdir)->Range(1, 128)->UseRealTime(); + +} // namespace + +} // namespace testing +} // namespace gvisor diff --git a/test/fuse/benchmark/open_benchmark.cc b/test/fuse/benchmark/open_benchmark.cc new file mode 100644 index 000000000..11c1c1c80 --- /dev/null +++ b/test/fuse/benchmark/open_benchmark.cc @@ -0,0 +1,60 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +#include +#include +#include + +#include "benchmark/benchmark.h" +#include "gtest/gtest.h" +#include "test/util/fs_util.h" +#include "test/util/logging.h" +#include "test/util/temp_path.h" + +namespace gvisor { +namespace testing { + +namespace { + +void BM_Open(benchmark::State& state) { + const char* fuse_prefix = getenv("TEST_FUSEPRE"); + ASSERT_NE(fuse_prefix, nullptr); + + const int size = state.range(0); + std::vector cache; + for (int i = 0; i < size; i++) { + auto path = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + cache.emplace_back(std::move(path)); + } + + unsigned int seed = 1; + for (auto _ : state) { + const int chosen = rand_r(&seed) % size; + const std::string file_path = JoinPath(fuse_prefix, cache[chosen].path()); + int fd = open(file_path.c_str(), O_RDONLY); + TEST_CHECK(fd != -1); + close(fd); + } +} + +BENCHMARK(BM_Open)->Range(1, 128)->UseRealTime(); + +} // namespace + +} // namespace testing +} // namespace gvisor diff --git a/test/fuse/benchmark/read_benchmark.cc b/test/fuse/benchmark/read_benchmark.cc new file mode 100644 index 000000000..2106b7d5a --- /dev/null +++ b/test/fuse/benchmark/read_benchmark.cc @@ -0,0 +1,57 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include + +#include "benchmark/benchmark.h" +#include "gtest/gtest.h" +#include "test/util/fs_util.h" +#include "test/util/logging.h" +#include "test/util/temp_path.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +void BM_Read(benchmark::State& state) { + const char* fuse_prefix = getenv("TEST_FUSEPRE"); + ASSERT_NE(fuse_prefix, nullptr); + + const int size = state.range(0); + const std::string contents(size, 0); + auto path = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFileWith( + GetAbsoluteTestTmpdir(), contents, TempPath::kDefaultFileMode)); + FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE( + Open(JoinPath(fuse_prefix, path.path()), O_RDONLY)); + + std::vector buf(size); + for (auto _ : state) { + TEST_CHECK(PreadFd(fd.get(), buf.data(), buf.size(), 0) == size); + } + + state.SetBytesProcessed(static_cast(size) * + static_cast(state.iterations())); +} + +BENCHMARK(BM_Read)->Range(1, 1 << 26)->UseRealTime(); + +} // namespace + +} // namespace testing +} // namespace gvisor diff --git a/test/fuse/benchmark/stat_benchmark.cc b/test/fuse/benchmark/stat_benchmark.cc new file mode 100644 index 000000000..d2ab6a706 --- /dev/null +++ b/test/fuse/benchmark/stat_benchmark.cc @@ -0,0 +1,65 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +#include "gtest/gtest.h" +#include "absl/strings/str_cat.h" +#include "benchmark/benchmark.h" +#include "test/util/fs_util.h" +#include "test/util/temp_path.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +// Creates a file in a nested directory hierarchy at least `depth` directories +// deep, and stats that file multiple times. +void BM_Stat(benchmark::State& state) { + const char* fuse_prefix = getenv("TEST_FUSEPRE"); + ASSERT_NE(fuse_prefix, nullptr); + + // Create nested directories with given depth. + int depth = state.range(0); + const TempPath top_dir = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); + std::string dir_path = top_dir.path(); + + while (depth-- > 0) { + // Don't use TempPath because it will make paths too long to use. + // + // The top_dir destructor will clean up this whole tree. + dir_path = JoinPath(dir_path, absl::StrCat(depth)); + ASSERT_NO_ERRNO(Mkdir(dir_path, 0755)); + } + + // Create the file that will be stat'd. + const TempPath file = + ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFileIn(dir_path)); + std::string file_path = JoinPath(fuse_prefix, file.path()); + struct stat st; + for (auto _ : state) { + ASSERT_THAT(stat(file_path.c_str(), &st), SyscallSucceeds()); + } +} + +BENCHMARK(BM_Stat)->Range(1, 100)->UseRealTime(); + +} // namespace + +} // namespace testing +} // namespace gvisor diff --git a/test/fuse/benchmark/symlink_benchmark.cc b/test/fuse/benchmark/symlink_benchmark.cc new file mode 100644 index 000000000..363b9a976 --- /dev/null +++ b/test/fuse/benchmark/symlink_benchmark.cc @@ -0,0 +1,60 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +#include "absl/strings/str_cat.h" +#include "benchmark/benchmark.h" +#include "gtest/gtest.h" +#include "test/util/fs_util.h" +#include "test/util/temp_path.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +void BM_Symlink(benchmark::State& state) { + char* fuse_prefix = getenv("TEST_FUSEPRE"); + ASSERT_NE(fuse_prefix, nullptr); + const TempPath top_dir = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); + std::string dir_path = top_dir.path(); + + const int size = state.range(0); + std::vector cache; + for (int i = 0; i < size; i++) { + auto path = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); + cache.emplace_back(std::move(path)); + } + + int index = 0; + unsigned int seed = 1; + for (auto t : state) { + const int chosen = rand_r(&seed) % size; + const std::string symlink_path = absl::StrCat(fuse_prefix, dir_path, index); + ASSERT_THAT(symlink(cache[chosen].path().c_str(), symlink_path.c_str()), + SyscallSucceeds()); + index++; + } +} + +BENCHMARK(BM_Symlink)->Range(1, 128)->UseRealTime(); + +} // namespace + +} // namespace testing +} // namespace gvisor diff --git a/test/image/BUILD b/test/image/BUILD index e749e47d4..e270c52ac 100644 --- a/test/image/BUILD +++ b/test/image/BUILD @@ -24,6 +24,7 @@ go_test( deps = [ "//pkg/test/dockerutil", "//pkg/test/testutil", + "@com_github_docker_docker//api/types/mount:go_default_library", ], ) diff --git a/test/image/image_test.go b/test/image/image_test.go index ac6186688..6b5928ef0 100644 --- a/test/image/image_test.go +++ b/test/image/image_test.go @@ -33,6 +33,7 @@ import ( "testing" "time" + "github.com/docker/docker/api/types/mount" "gvisor.dev/gvisor/pkg/test/dockerutil" "gvisor.dev/gvisor/pkg/test/testutil" ) @@ -63,6 +64,57 @@ func TestHelloWorld(t *testing.T) { } } +// Test that the FUSE container is set up and being used properly. +func TestFUSEInContainer(t *testing.T) { + if usingFUSE, err := dockerutil.UsingFUSE(); err != nil { + t.Fatalf("failed to read config for runtime %s: %v", dockerutil.Runtime(), err) + } else if !usingFUSE { + t.Skip("FUSE not being used.") + } + + ctx := context.Background() + d := dockerutil.MakeContainer(ctx, t) + defer d.CleanUp(ctx) + + tmpDir := "/tmpDir/" + // Run the basic container. + err := d.Spawn(ctx, dockerutil.RunOpts{ + Image: "basic/fuse", + Privileged: true, + CapAdd: []string{"CAP_SYS_ADMIN"}, + + // Mount a tmpfs directory for benchmark. + Mounts: []mount.Mount{ + { + Type: mount.TypeTmpfs, + Target: tmpDir, + ReadOnly: false, + }, + }, + }, "sleep", "1000") + if err != nil { + t.Fatalf("docker spawn failed: %v", err) + } + + out, err := d.Exec(ctx, dockerutil.ExecOpts{ + Privileged: true, + }, "/bin/sh", "-c", "ls") + if err != nil { + t.Fatalf("docker exec failed: %v, message %s", err, out) + } + if !strings.Contains(out, "server-bin") { + t.Fatalf("docker didn't find server binary: got %s", out) + } + + // Run the server. + out, err = d.Exec(ctx, dockerutil.ExecOpts{ + Privileged: true, + }, "/bin/sh", "-c", "./server-bin mountpoint") + if err != nil { + t.Fatalf("docker exec failed: %v, message %s", err, out) + } +} + func runHTTPRequest(port int) error { url := fmt.Sprintf("http://localhost:%d/not-found", port) resp, err := http.Get(url) diff --git a/test/runner/BUILD b/test/runner/BUILD index 582d2946d..049c26081 100644 --- a/test/runner/BUILD +++ b/test/runner/BUILD @@ -11,11 +11,14 @@ go_binary( ], visibility = ["//:sandbox"], deps = [ + "//pkg/context", "//pkg/log", + "//pkg/test/dockerutil", "//pkg/test/testutil", "//runsc/specutils", "//test/runner/gtest", "//test/uds", + "@com_github_docker_docker//api/types/mount:go_default_library", "@com_github_opencontainers_runtime_spec//specs-go:go_default_library", "@com_github_syndtr_gocapability//capability:go_default_library", "@org_golang_x_sys//unix:go_default_library", diff --git a/test/runner/defs.bzl b/test/runner/defs.bzl index 032ebd04e..9dc955c77 100644 --- a/test/runner/defs.bzl +++ b/test/runner/defs.bzl @@ -57,6 +57,8 @@ def _syscall_test( platform, use_tmpfs, tags, + use_image = "", + setup_command = "", network = "none", file_access = "exclusive", overlay = False, @@ -79,6 +81,8 @@ def _syscall_test( name += "_fuse" if network != "none": name += "_" + network + "net" + if use_image != "": + name += "_container" # Apply all tags. if tags == None: @@ -107,6 +111,8 @@ def _syscall_test( "--platform=" + platform, "--network=" + network, "--use-tmpfs=" + str(use_tmpfs), + "--use-image=" + use_image, + "--setup-command=" + setup_command, "--file-access=" + file_access, "--overlay=" + str(overlay), "--add-uds-tree=" + str(add_uds_tree), @@ -132,6 +138,8 @@ def syscall_test( shard_count = 5, size = "small", use_tmpfs = False, + use_image = "", + setup_command = "", add_overlay = False, add_uds_tree = False, add_hostinet = False, @@ -146,6 +154,8 @@ def syscall_test( shard_count: shards for defined tests. size: the defined test size. use_tmpfs: use tmpfs in the defined tests. + use_image: use specified docker image in the defined tests. + setup_command: command to set up the docker container. Should be used when ise_image is. add_overlay: add an overlay test. add_uds_tree: add a UDS test. add_hostinet: add a hostinet test. @@ -178,8 +188,26 @@ def syscall_test( vfs2 = True, fuse = fuse, ) + + if use_image != "": + # Run the test in the container specified. + _syscall_test( + test = test, + shard_count = shard_count, + size = size, + platform = default_platform, + use_tmpfs = use_tmpfs, + use_image = use_image, + setup_command = setup_command, + add_uds_tree = add_uds_tree, + tags = platforms[default_platform] + vfs2_tags, + vfs2 = True, + fuse = True, + ) + if fuse: # Only generate *_vfs2_fuse target if fuse parameter is enabled. + # The rest of the targets don't support FUSE as of yet. return _syscall_test( diff --git a/test/runner/runner.go b/test/runner/runner.go index 22d535f8d..fe501c4c7 100644 --- a/test/runner/runner.go +++ b/test/runner/runner.go @@ -23,16 +23,20 @@ import ( "os" "os/exec" "os/signal" + "path" "path/filepath" "strings" "syscall" "testing" "time" + "github.com/docker/docker/api/types/mount" specs "github.com/opencontainers/runtime-spec/specs-go" "github.com/syndtr/gocapability/capability" "golang.org/x/sys/unix" + "gvisor.dev/gvisor/pkg/context" "gvisor.dev/gvisor/pkg/log" + "gvisor.dev/gvisor/pkg/test/dockerutil" "gvisor.dev/gvisor/pkg/test/testutil" "gvisor.dev/gvisor/runsc/specutils" "gvisor.dev/gvisor/test/runner/gtest" @@ -40,17 +44,19 @@ import ( ) var ( - debug = flag.Bool("debug", false, "enable debug logs") - strace = flag.Bool("strace", false, "enable strace logs") - platform = flag.String("platform", "ptrace", "platform to run on") - network = flag.String("network", "none", "network stack to run on (sandbox, host, none)") - useTmpfs = flag.Bool("use-tmpfs", false, "mounts tmpfs for /tmp") - fileAccess = flag.String("file-access", "exclusive", "mounts root in exclusive or shared mode") - overlay = flag.Bool("overlay", false, "wrap filesystem mounts with writable tmpfs overlay") - vfs2 = flag.Bool("vfs2", false, "enable VFS2") - fuse = flag.Bool("fuse", false, "enable FUSE") - parallel = flag.Bool("parallel", false, "run tests in parallel") - runscPath = flag.String("runsc", "", "path to runsc binary") + debug = flag.Bool("debug", false, "enable debug logs") + strace = flag.Bool("strace", false, "enable strace logs") + platform = flag.String("platform", "ptrace", "platform to run on") + network = flag.String("network", "none", "network stack to run on (sandbox, host, none)") + useTmpfs = flag.Bool("use-tmpfs", false, "mounts tmpfs for /tmp") + useImage = flag.String("use-image", "", "container image to use for test. Path relative to //images") + setupCommand = flag.String("setup-command", "", "command to run before running the test to set up container environment") + fileAccess = flag.String("file-access", "exclusive", "mounts root in exclusive or shared mode") + overlay = flag.Bool("overlay", false, "wrap filesystem mounts with writable tmpfs overlay") + vfs2 = flag.Bool("vfs2", false, "enable VFS2") + fuse = flag.Bool("fuse", false, "enable FUSE") + parallel = flag.Bool("parallel", false, "run tests in parallel") + runscPath = flag.String("runsc", "", "path to runsc binary") addUDSTree = flag.Bool("add-uds-tree", false, "expose a tree of UDS utilities for use in tests") ) @@ -313,8 +319,92 @@ func setupUDSTree(spec *specs.Spec) (cleanup func(), err error) { return cleanup, nil } +func runTestCaseInContainer(testBin string, tc gtest.TestCase, image string, t *testing.T) { + if usingFUSE, err := dockerutil.UsingFUSE(); err != nil { + t.Fatalf("failed to read config for runtime %s: %v", dockerutil.Runtime(), err) + } else if !usingFUSE { + t.Skip("FUSE not being used.") + } + + ctx := context.Background() + d := dockerutil.MakeContainer(ctx, t) + defer d.CleanUp(ctx) + + // Run the basic container. + tmpDir := "/tmpDir" + testBinDir := "/testDir" + opts := dockerutil.RunOpts{ + Image: image, + Privileged: true, + CapAdd: []string{"CAP_SYS_ADMIN"}, + + // Mount a tmpfs directory to use when benchmarking. + Mounts: []mount.Mount{ + { + Type: mount.TypeTmpfs, + Target: tmpDir, + ReadOnly: false, + }, + }, + Env: []string{ + fmt.Sprintf("TEST_TMPDIR=%s", tmpDir), + fmt.Sprintf("TEST_FUSEPRE=%s", "/fus/mountpoint"), + }, + } + wd, err := os.Getwd() + if err != nil { + t.Fatalf("Getwd run failed: %v", err) + } + + wdPathToTestBin := strings.TrimPrefix(testBin, wd) + containerTestBin := path.Join(testBinDir, path.Base(wdPathToTestBin)) + d.CopyFiles(&opts, testBinDir, wdPathToTestBin) + if err := d.CopyErr(); err != nil { + t.Fatalf("Copy failed %v", err) + } + + err = d.Spawn(ctx, opts, "sleep", "1000") + if err != nil { + t.Fatalf("docker run failed: %v", err) + } + + // Run the server setup command. + if *setupCommand != "" { + out, err := d.Exec(ctx, dockerutil.ExecOpts{ + Privileged: true, + }, "/bin/sh", "-c", *setupCommand) + if err != nil { + t.Fatalf("docker exec failed: %v with output %v", err, out) + } + } + + cmd := "chmod +x " + containerTestBin + out, err := d.Exec(ctx, dockerutil.ExecOpts{ + Privileged: true, + }, "/bin/sh", "-c", cmd) + if err != nil { + t.Fatalf("docker exec failed: %v with output %v", err, out) + } + + cmd = containerTestBin + " " + strings.Join(tc.Args(), " ") + out, err = d.Exec(ctx, dockerutil.ExecOpts{ + Privileged: true, + }, "/bin/sh", "-c", cmd) + if err != nil { + t.Fatalf("docker exec failed: %v with output %v", err, out) + } + + fmt.Print(out) + return +} + // runsTestCaseRunsc runs the test case in runsc. func runTestCaseRunsc(testBin string, tc gtest.TestCase, t *testing.T) { + if *useImage != "" { + runTestCaseInContainer(testBin, tc, *useImage, t) + return + } + // Run a new container with the test executable and filter for the // given test suite and name. spec := testutil.NewSpecWithArgs(append([]string{testBin}, tc.Args()...)...) -- cgit v1.2.3 From 1d8029022e8ca45e177e36405506d42874f125e9 Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Wed, 2 Sep 2020 13:30:56 -0700 Subject: fuse: remove unused marshalling functions This commit removes unused marshalling functions in linux abi package and moves self-defined FUSEInitRes wrapper to fuse package. Updates #3707 --- pkg/abi/linux/fuse.go | 175 ----------------------------- pkg/sentry/fsimpl/fuse/BUILD | 1 + pkg/sentry/fsimpl/fuse/init.go | 4 +- pkg/sentry/fsimpl/fuse/request_response.go | 78 +++++++++++++ 4 files changed, 81 insertions(+), 177 deletions(-) create mode 100644 pkg/sentry/fsimpl/fuse/request_response.go (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index ca4ee5e80..5616290a5 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -15,7 +15,6 @@ package linux import ( - "gvisor.dev/gvisor/pkg/usermem" "gvisor.dev/gvisor/tools/go_marshal/marshal" "gvisor.dev/gvisor/tools/go_marshal/primitive" ) @@ -263,63 +262,6 @@ type FUSEInitOut struct { _ [8]uint32 } -// FUSEInitRes is a variable-length wrapper of FUSEInitOut. The FUSE server may -// implement older version of FUSE protocol, which contains a FUSEInitOut with -// less attributes. -// -// Dynamically-sized objects cannot be marshalled. -type FUSEInitRes struct { - marshal.StubMarshallable - - // InitOut contains the response from the FUSE server. - InitOut FUSEInitOut - - // Len is the total length of bytes of the response. - Len uint32 -} - -// UnMarshalBytes deserializes src to the InitOut attribute in a FUSEInitRes. -func (r *FUSEInitRes) UnmarshalBytes(src []byte) { - out := &r.InitOut - - // Introduced before FUSE kernel version 7.13. - out.Major = uint32(usermem.ByteOrder.Uint32(src[:4])) - src = src[4:] - out.Minor = uint32(usermem.ByteOrder.Uint32(src[:4])) - src = src[4:] - out.MaxReadahead = uint32(usermem.ByteOrder.Uint32(src[:4])) - src = src[4:] - out.Flags = uint32(usermem.ByteOrder.Uint32(src[:4])) - src = src[4:] - out.MaxBackground = uint16(usermem.ByteOrder.Uint16(src[:2])) - src = src[2:] - out.CongestionThreshold = uint16(usermem.ByteOrder.Uint16(src[:2])) - src = src[2:] - out.MaxWrite = uint32(usermem.ByteOrder.Uint32(src[:4])) - src = src[4:] - - // Introduced in FUSE kernel version 7.23. - if len(src) >= 4 { - out.TimeGran = uint32(usermem.ByteOrder.Uint32(src[:4])) - src = src[4:] - } - // Introduced in FUSE kernel version 7.28. - if len(src) >= 2 { - out.MaxPages = uint16(usermem.ByteOrder.Uint16(src[:2])) - src = src[2:] - } - // Introduced in FUSE kernel version 7.31. - if len(src) >= 2 { - out.MapAlignment = uint16(usermem.ByteOrder.Uint16(src[:2])) - src = src[2:] - } -} - -// SizeBytes is the size of the payload of the FUSE_INIT response. -func (r *FUSEInitRes) SizeBytes() int { - return int(r.Len) -} - // FUSEGetAttrIn is the request sent by the kernel to the daemon, // to get the attribute of a inode. // @@ -415,11 +357,6 @@ type FUSELookupIn struct { Name string } -// MarshalUnsafe serializes r.name to the dst buffer. -func (r *FUSELookupIn) MarshalUnsafe(buf []byte) { - copy(buf, r.Name) -} - // MarshalBytes serializes r.name to the dst buffer. func (r *FUSELookupIn) MarshalBytes(buf []byte) { copy(buf, r.Name) @@ -548,12 +485,6 @@ type FUSEMknodIn struct { Name string } -// MarshalUnsafe serializes r.MknodMeta and r.Name to the dst buffer. -func (r *FUSEMknodIn) MarshalUnsafe(buf []byte) { - r.MknodMeta.MarshalUnsafe(buf[:r.MknodMeta.SizeBytes()]) - copy(buf[r.MknodMeta.SizeBytes():], r.Name) -} - // MarshalBytes serializes r.MknodMeta and r.Name to the dst buffer. func (r *FUSEMknodIn) MarshalBytes(buf []byte) { r.MknodMeta.MarshalBytes(buf[:r.MknodMeta.SizeBytes()]) @@ -580,13 +511,6 @@ type FUSESymLinkIn struct { Target string } -// MarshalUnsafe serializes r.Name and r.Target to the dst buffer. -// Left null-termination at end of r.Name and r.Target. -func (r *FUSESymLinkIn) MarshalUnsafe(buf []byte) { - copy(buf, r.Name) - copy(buf[len(r.Name)+1:], r.Target) -} - // MarshalBytes serializes r.Name and r.Target to the dst buffer. // Left null-termination at end of r.Name and r.Target. func (r *FUSESymLinkIn) MarshalBytes(buf []byte) { @@ -603,9 +527,6 @@ func (r *FUSESymLinkIn) SizeBytes() int { // FUSEEmptyIn is used by operations without request body. type FUSEEmptyIn struct{ marshal.StubMarshallable } -// MarshalUnsafe do nothing for marshal. -func (r *FUSEEmptyIn) MarshalUnsafe(buf []byte) {} - // MarshalBytes do nothing for marshal. func (r *FUSEEmptyIn) MarshalBytes(buf []byte) {} @@ -640,12 +561,6 @@ type FUSEMkdirIn struct { Name string } -// MarshalUnsafe serializes r.MkdirMeta and r.Name to the dst buffer. -func (r *FUSEMkdirIn) MarshalUnsafe(buf []byte) { - r.MkdirMeta.MarshalUnsafe(buf[:r.MkdirMeta.SizeBytes()]) - copy(buf[r.MkdirMeta.SizeBytes():], r.Name) -} - // MarshalBytes serializes r.MkdirMeta and r.Name to the dst buffer. func (r *FUSEMkdirIn) MarshalBytes(buf []byte) { r.MkdirMeta.MarshalBytes(buf[:r.MkdirMeta.SizeBytes()]) @@ -669,11 +584,6 @@ type FUSERmDirIn struct { Name string } -// MarshalUnsafe serializes r.name to the dst buffer. -func (r *FUSERmDirIn) MarshalUnsafe(buf []byte) { - copy(buf, r.Name) -} - // MarshalBytes serializes r.name to the dst buffer. func (r *FUSERmDirIn) MarshalBytes(buf []byte) { copy(buf, r.Name) @@ -684,16 +594,6 @@ func (r *FUSERmDirIn) SizeBytes() int { return len(r.Name) + 1 } -// UnmarshalUnsafe deserializes r.name from the src buffer. -func (r *FUSERmDirIn) UnmarshalUnsafe(src []byte) { - r.Name = string(src) -} - -// UnmarshalBytes deserializes r.name from the src buffer. -func (r *FUSERmDirIn) UnmarshalBytes(src []byte) { - r.Name = string(src) -} - // FUSEDirents is a list of Dirents received from the FUSE daemon server. // It is used for FUSE_READDIR. // @@ -736,22 +636,6 @@ type FUSEDirentMeta struct { Type uint32 } -// MarshalUnsafe serializes FUSEDirents to the dst buffer. -func (r *FUSEDirents) MarshalUnsafe(dst []byte) { - for _, dirent := range r.Dirents { - dirent.MarshalUnsafe(dst) - dst = dst[dirent.SizeBytes():] - } -} - -// MarshalBytes serializes FUSEDirents to the dst buffer. -func (r *FUSEDirents) MarshalBytes(dst []byte) { - for _, dirent := range r.Dirents { - dirent.MarshalBytes(dst) - dst = dst[dirent.SizeBytes():] - } -} - // SizeBytes is the size of the memory representation of FUSEDirents. func (r *FUSEDirents) SizeBytes() int { var sizeBytes int @@ -762,30 +646,6 @@ func (r *FUSEDirents) SizeBytes() int { return sizeBytes } -// UnmarshalUnsafe deserializes FUSEDirents from the src buffer. -func (r *FUSEDirents) UnmarshalUnsafe(src []byte) { - for { - if len(src) <= (*FUSEDirentMeta)(nil).SizeBytes() { - break - } - - // Its unclear how many dirents there are in src. Each dirent is dynamically - // sized and so we can't make assumptions about how many dirents we can allocate. - if r.Dirents == nil { - r.Dirents = make([]*FUSEDirent, 0) - } - - // We have to allocate a struct for each dirent - there must be a better way - // to do this. Linux allocates 1 page to store all the dirents and then - // simply reads them from the page. - var dirent FUSEDirent - dirent.UnmarshalUnsafe(src) - r.Dirents = append(r.Dirents, &dirent) - - src = src[dirent.SizeBytes():] - } -} - // UnmarshalBytes deserializes FUSEDirents from the src buffer. func (r *FUSEDirents) UnmarshalBytes(src []byte) { for { @@ -810,24 +670,6 @@ func (r *FUSEDirents) UnmarshalBytes(src []byte) { } } -// MarshalUnsafe serializes FUSEDirent to the dst buffer. -func (r *FUSEDirent) MarshalUnsafe(dst []byte) { - r.Meta.MarshalUnsafe(dst) - dst = dst[r.Meta.SizeBytes():] - - name := primitive.ByteSlice(r.Name) - name.MarshalUnsafe(dst) -} - -// MarshalBytes serializes FUSEDirent to the dst buffer. -func (r *FUSEDirent) MarshalBytes(dst []byte) { - r.Meta.MarshalBytes(dst) - dst = dst[r.Meta.SizeBytes():] - - name := primitive.ByteSlice(r.Name) - name.MarshalBytes(dst) -} - // SizeBytes is the size of the memory representation of FUSEDirent. func (r *FUSEDirent) SizeBytes() int { dataSize := r.Meta.SizeBytes() + len(r.Name) @@ -838,23 +680,6 @@ func (r *FUSEDirent) SizeBytes() int { return (dataSize + (FUSE_DIRENT_ALIGN - 1)) & ^(FUSE_DIRENT_ALIGN - 1) } -// UnmarshalUnsafe deserializes FUSEDirent from the src buffer. -func (r *FUSEDirent) UnmarshalUnsafe(src []byte) { - r.Meta.UnmarshalUnsafe(src) - src = src[r.Meta.SizeBytes():] - - if r.Meta.NameLen > FUSE_NAME_MAX { - // The name is too long and therefore invalid. We don't - // need to unmarshal the name since it'll be thrown away. - return - } - - buf := make([]byte, r.Meta.NameLen) - name := primitive.ByteSlice(buf) - name.UnmarshalUnsafe(src[:r.Meta.NameLen]) - r.Name = string(name) -} - // UnmarshalBytes deserializes FUSEDirent from the src buffer. func (r *FUSEDirent) UnmarshalBytes(src []byte) { r.Meta.UnmarshalBytes(src) diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index 241d50c3d..5085d0521 100644 --- a/pkg/sentry/fsimpl/fuse/BUILD +++ b/pkg/sentry/fsimpl/fuse/BUILD @@ -40,6 +40,7 @@ go_library( "register.go", "regular_file.go", "request_list.go", + "request_response.go", ], visibility = ["//pkg/sentry:internal"], deps = [ diff --git a/pkg/sentry/fsimpl/fuse/init.go b/pkg/sentry/fsimpl/fuse/init.go index 6384cbbdb..256b6fb65 100644 --- a/pkg/sentry/fsimpl/fuse/init.go +++ b/pkg/sentry/fsimpl/fuse/init.go @@ -76,12 +76,12 @@ func (conn *connection) InitRecv(res *Response, hasSysAdminCap bool) error { return err } - initRes := linux.FUSEInitRes{Len: res.DataLen()} + initRes := fuseInitRes{initLen: res.DataLen()} if err := res.UnmarshalPayload(&initRes); err != nil { return err } - return conn.initProcessReply(&initRes.InitOut, hasSysAdminCap) + return conn.initProcessReply(&initRes.initOut, hasSysAdminCap) } // Process the FUSE_INIT reply from the FUSE server. diff --git a/pkg/sentry/fsimpl/fuse/request_response.go b/pkg/sentry/fsimpl/fuse/request_response.go new file mode 100644 index 000000000..ae71b5e28 --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/request_response.go @@ -0,0 +1,78 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fuse + +import ( + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/usermem" + "gvisor.dev/gvisor/tools/go_marshal/marshal" +) + +// fuseInitRes is a variable-length wrapper of linux.FUSEInitOut. The FUSE +// server may implement an older version of FUSE protocol, which contains a +// linux.FUSEInitOut with less attributes. +// +// Dynamically-sized objects cannot be marshalled. +type fuseInitRes struct { + marshal.StubMarshallable + + // initOut contains the response from the FUSE server. + initOut linux.FUSEInitOut + + // initLen is the total length of bytes of the response. + initLen uint32 +} + +// UnmarshalBytes deserializes src to the initOut attribute in a fuseInitRes. +func (r *fuseInitRes) UnmarshalBytes(src []byte) { + out := &r.initOut + + // Introduced before FUSE kernel version 7.13. + out.Major = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + out.Minor = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + out.MaxReadahead = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + out.Flags = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + out.MaxBackground = uint16(usermem.ByteOrder.Uint16(src[:2])) + src = src[2:] + out.CongestionThreshold = uint16(usermem.ByteOrder.Uint16(src[:2])) + src = src[2:] + out.MaxWrite = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + + // Introduced in FUSE kernel version 7.23. + if len(src) >= 4 { + out.TimeGran = uint32(usermem.ByteOrder.Uint32(src[:4])) + src = src[4:] + } + // Introduced in FUSE kernel version 7.28. + if len(src) >= 2 { + out.MaxPages = uint16(usermem.ByteOrder.Uint16(src[:2])) + src = src[2:] + } + // Introduced in FUSE kernel version 7.31. + if len(src) >= 2 { + out.MapAlignment = uint16(usermem.ByteOrder.Uint16(src[:2])) + src = src[2:] + } +} + +// SizeBytes is the size of the payload of the FUSE_INIT response. +func (r *fuseInitRes) SizeBytes() int { + return int(r.initLen) +} -- cgit v1.2.3 From e91c026672b9022d5554e20202648d322da0fe1d Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Wed, 2 Sep 2020 13:50:31 -0700 Subject: Downgrade FUSE minor version support and clarify comments --- pkg/abi/linux/fuse.go | 53 +++++++++++++----------------- pkg/sentry/fsimpl/fuse/connection.go | 20 +++++------ pkg/sentry/fsimpl/fuse/init.go | 2 -- pkg/sentry/fsimpl/fuse/request_response.go | 5 --- 4 files changed, 31 insertions(+), 49 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index 5616290a5..ed40df564 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -151,34 +151,27 @@ type FUSEWriteIn struct { } // FUSE_INIT flags, consistent with the ones in include/uapi/linux/fuse.h. +// Our taget version is 7.23 but we have few implemented in advance. const ( - FUSE_ASYNC_READ = 1 << 0 - FUSE_POSIX_LOCKS = 1 << 1 - FUSE_FILE_OPS = 1 << 2 - FUSE_ATOMIC_O_TRUNC = 1 << 3 - FUSE_EXPORT_SUPPORT = 1 << 4 - FUSE_BIG_WRITES = 1 << 5 - FUSE_DONT_MASK = 1 << 6 - FUSE_SPLICE_WRITE = 1 << 7 - FUSE_SPLICE_MOVE = 1 << 8 - FUSE_SPLICE_READ = 1 << 9 - FUSE_FLOCK_LOCKS = 1 << 10 - FUSE_HAS_IOCTL_DIR = 1 << 11 - FUSE_AUTO_INVAL_DATA = 1 << 12 - FUSE_DO_READDIRPLUS = 1 << 13 - FUSE_READDIRPLUS_AUTO = 1 << 14 - FUSE_ASYNC_DIO = 1 << 15 - FUSE_WRITEBACK_CACHE = 1 << 16 - FUSE_NO_OPEN_SUPPORT = 1 << 17 - FUSE_PARALLEL_DIROPS = 1 << 18 - FUSE_HANDLE_KILLPRIV = 1 << 19 - FUSE_POSIX_ACL = 1 << 20 - FUSE_ABORT_ERROR = 1 << 21 - FUSE_MAX_PAGES = 1 << 22 - FUSE_CACHE_SYMLINKS = 1 << 23 - FUSE_NO_OPENDIR_SUPPORT = 1 << 24 - FUSE_EXPLICIT_INVAL_DATA = 1 << 25 - FUSE_MAP_ALIGNMENT = 1 << 26 + FUSE_ASYNC_READ = 1 << 0 + FUSE_POSIX_LOCKS = 1 << 1 + FUSE_FILE_OPS = 1 << 2 + FUSE_ATOMIC_O_TRUNC = 1 << 3 + FUSE_EXPORT_SUPPORT = 1 << 4 + FUSE_BIG_WRITES = 1 << 5 + FUSE_DONT_MASK = 1 << 6 + FUSE_SPLICE_WRITE = 1 << 7 + FUSE_SPLICE_MOVE = 1 << 8 + FUSE_SPLICE_READ = 1 << 9 + FUSE_FLOCK_LOCKS = 1 << 10 + FUSE_HAS_IOCTL_DIR = 1 << 11 + FUSE_AUTO_INVAL_DATA = 1 << 12 + FUSE_DO_READDIRPLUS = 1 << 13 + FUSE_READDIRPLUS_AUTO = 1 << 14 + FUSE_ASYNC_DIO = 1 << 15 + FUSE_WRITEBACK_CACHE = 1 << 16 + FUSE_NO_OPEN_SUPPORT = 1 << 17 + FUSE_MAX_PAGES = 1 << 22 // From FUSE 7.28 ) // currently supported FUSE protocol version numbers. @@ -214,7 +207,7 @@ type FUSEInitIn struct { } // FUSEInitOut is the reply sent by the daemon to the kernel -// for FUSEInitIn. +// for FUSEInitIn. We target FUSE 7.23; this struct supports 7.28. // // +marshal type FUSEInitOut struct { @@ -255,9 +248,7 @@ type FUSEInitOut struct { // if the value from daemon is too large. MaxPages uint16 - // MapAlignment is an unknown field and not used by this package at this moment. - // Use as a placeholder to be consistent with the FUSE protocol. - MapAlignment uint16 + _ uint16 _ [8]uint32 } diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index dc169b497..5f20483b7 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -86,16 +86,22 @@ type connection struct { // attributeVersion is the version of connection's attributes. attributeVersion uint64 + // We target FUSE 7.23. // The following FUSE_INIT flags are currently unsupported by this implementation: // - FUSE_EXPORT_SUPPORT - // - FUSE_HANDLE_KILLPRIV // - FUSE_POSIX_LOCKS: requires POSIX locks // - FUSE_FLOCK_LOCKS: requires POSIX locks // - FUSE_AUTO_INVAL_DATA: requires page caching eviction - // - FUSE_EXPLICIT_INVAL_DATA: requires page caching eviction // - FUSE_DO_READDIRPLUS/FUSE_READDIRPLUS_AUTO: requires FUSE_READDIRPLUS implementation // - FUSE_ASYNC_DIO - // - FUSE_POSIX_ACL: affects defaultPermissions, posixACL, xattr handler + // - FUSE_PARALLEL_DIROPS (7.25) + // - FUSE_HANDLE_KILLPRIV (7.26) + // - FUSE_POSIX_ACL: affects defaultPermissions, posixACL, xattr handler (7.26) + // - FUSE_ABORT_ERROR (7.27) + // - FUSE_CACHE_SYMLINKS (7.28) + // - FUSE_NO_OPENDIR_SUPPORT (7.29) + // - FUSE_EXPLICIT_INVAL_DATA: requires page caching eviction (7.30) + // - FUSE_MAP_ALIGNMENT (7.31) // initialized after receiving FUSE_INIT reply. // Until it's set, suspend sending FUSE requests. @@ -182,19 +188,11 @@ type connection struct { // Negotiated and only set in INIT. asyncRead bool - // abortErr is true if kernel need to return an unique read error after abort. - // Negotiated and only set in INIT. - abortErr bool - // writebackCache is true for write-back cache policy, // false for write-through policy. // Negotiated and only set in INIT. writebackCache bool - // cacheSymlinks if filesystem needs to cache READLINK responses in page cache. - // Negotiated and only set in INIT. - cacheSymlinks bool - // bigWrites if doing multi-page cached writes. // Negotiated and only set in INIT. bigWrites bool diff --git a/pkg/sentry/fsimpl/fuse/init.go b/pkg/sentry/fsimpl/fuse/init.go index 256b6fb65..a47309b6e 100644 --- a/pkg/sentry/fsimpl/fuse/init.go +++ b/pkg/sentry/fsimpl/fuse/init.go @@ -132,8 +132,6 @@ func (conn *connection) initProcessReply(out *linux.FUSEInitOut, hasSysAdminCap conn.bigWrites = out.Flags&linux.FUSE_BIG_WRITES != 0 conn.dontMask = out.Flags&linux.FUSE_DONT_MASK != 0 conn.writebackCache = out.Flags&linux.FUSE_WRITEBACK_CACHE != 0 - conn.cacheSymlinks = out.Flags&linux.FUSE_CACHE_SYMLINKS != 0 - conn.abortErr = out.Flags&linux.FUSE_ABORT_ERROR != 0 // TODO(gvisor.dev/issue/3195): figure out how to use TimeGran (0 < TimeGran <= fuseMaxTimeGranNs). diff --git a/pkg/sentry/fsimpl/fuse/request_response.go b/pkg/sentry/fsimpl/fuse/request_response.go index ae71b5e28..a69b21221 100644 --- a/pkg/sentry/fsimpl/fuse/request_response.go +++ b/pkg/sentry/fsimpl/fuse/request_response.go @@ -65,11 +65,6 @@ func (r *fuseInitRes) UnmarshalBytes(src []byte) { out.MaxPages = uint16(usermem.ByteOrder.Uint16(src[:2])) src = src[2:] } - // Introduced in FUSE kernel version 7.31. - if len(src) >= 2 { - out.MapAlignment = uint16(usermem.ByteOrder.Uint16(src[:2])) - src = src[2:] - } } // SizeBytes is the size of the payload of the FUSE_INIT response. -- cgit v1.2.3 From 7ed4e46a717ece014a962e4b9cacff1ff4c461f1 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Wed, 2 Sep 2020 13:55:08 -0700 Subject: FUSE device: clean up readLocked This change removes the unnecessary loop and avoids the recursive call. It also fixes minor bugs in this function. --- pkg/sentry/fsimpl/fuse/dev.go | 89 ++++++++++++++++++++++--------------------- 1 file changed, 46 insertions(+), 43 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index fd3592e32..c53a38021 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -19,7 +19,6 @@ import ( "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" - "gvisor.dev/gvisor/pkg/log" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" @@ -143,58 +142,62 @@ func (fd *DeviceFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.R } // readLocked implements the reading of the fuse device while locked with DeviceFD.mu. +// +// Preconditions: dst is large enough for any reasonable request. func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { - if fd.queue.Empty() { - return 0, syserror.ErrWouldBlock - } + var req *Request - var readCursor uint32 - var bytesRead int64 - for { - req := fd.queue.Front() - if dst.NumBytes() < int64(req.hdr.Len) { - // The request is too large. Cannot process it. All requests must be smaller than the - // negotiated size as specified by Connection.MaxWrite set as part of the FUSE_INIT - // handshake. - errno := -int32(syscall.EIO) - if req.hdr.Opcode == linux.FUSE_SETXATTR { - errno = -int32(syscall.E2BIG) - } - - // Return the error to the calling task. - if err := fd.sendError(ctx, errno, req); err != nil { - return 0, err - } + // Find the first valid request. + // For the normal case this loop only execute once. + for !fd.queue.Empty() { + req = fd.queue.Front() - // We're done with this request. - fd.queue.Remove(req) - if req.hdr.Opcode == linux.FUSE_RELEASE { - fd.numActiveRequests -= 1 - } + if int64(req.hdr.Len) <= dst.NumBytes() { + break + } - // Restart the read as this request was invalid. - log.Warningf("fuse.DeviceFD.Read: request found was too large. Restarting read.") - return fd.readLocked(ctx, dst, opts) + // The request is too large. Cannot process it. All requests must be smaller than the + // negotiated size as specified by Connection.MaxWrite set as part of the FUSE_INIT + // handshake. + errno := -int32(syscall.EIO) + if req.hdr.Opcode == linux.FUSE_SETXATTR { + errno = -int32(syscall.E2BIG) } - n, err := dst.CopyOut(ctx, req.data[readCursor:]) - if err != nil { + // Return the error to the calling task. + if err := fd.sendError(ctx, errno, req); err != nil { return 0, err } - readCursor += uint32(n) - bytesRead += int64(n) - - if readCursor >= req.hdr.Len { - // Fully done with this req, remove it from the queue. - fd.queue.Remove(req) - if req.hdr.Opcode == linux.FUSE_RELEASE { - fd.numActiveRequests -= 1 - } - break - } + + // We're done with this request. + fd.queue.Remove(req) + req = nil + } + + if req == nil { + return 0, syserror.ErrWouldBlock + } + + // We already checked the size: dst must be able to fit the whole request. + // Now we write the marshalled header, the payload, + // and the potential additional payload + // to the user memory IOSequence. + + n, err := dst.CopyOut(ctx, req.data) + if err != nil { + return 0, err + } + if n != len(req.data) { + return 0, syserror.EIO + } + + // Fully done with this req, remove it from the queue. + fd.queue.Remove(req) + if req.hdr.Opcode == linux.FUSE_RELEASE { + fd.numActiveRequests -= 1 } - return bytesRead, nil + return int64(n), nil } // PWrite implements vfs.FileDescriptionImpl.PWrite. -- cgit v1.2.3 From 18f1e1c91b05059c333197a2a6198716c12508e7 Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Thu, 3 Sep 2020 14:06:59 -0700 Subject: Implement FUSE_CREATE FUSE_CREATE is called when issuing creat(2) or open(2) with O_CREAT. It creates a new file on the FUSE filesystem. Fixes #3825 --- pkg/abi/linux/fuse.go | 42 +++++++++++++ pkg/sentry/fsimpl/fuse/fusefs.go | 18 ++++++ test/fuse/BUILD | 4 ++ test/fuse/linux/BUILD | 17 +++++- test/fuse/linux/create_test.cc | 128 +++++++++++++++++++++++++++++++++++++++ 5 files changed, 208 insertions(+), 1 deletion(-) create mode 100644 test/fuse/linux/create_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index ed40df564..adc04dbcc 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -445,6 +445,48 @@ type FUSEReleaseIn struct { LockOwner uint64 } +// FUSECreateMeta contains all the static fields of FUSECreateIn, +// which is used for FUSE_CREATE. +// +// +marshal +type FUSECreateMeta struct { + // Flags of the creating file. + Flags uint32 + + // Mode is the mode of the creating file. + Mode uint32 + + // Umask is the current file mode creation mask. + Umask uint32 + _ uint32 +} + +// FUSECreateIn contains all the arguments sent by the kernel to the daemon, to +// atomically create and open a new regular file. +// +// Dynamically-sized objects cannot be marshalled. +type FUSECreateIn struct { + marshal.StubMarshallable + + // CreateMeta contains mode, rdev and umash field for FUSE_MKNODS. + CreateMeta FUSECreateMeta + + // Name is the name of the node to create. + Name string +} + +// MarshalBytes serializes r.CreateMeta and r.Name to the dst buffer. +func (r *FUSECreateIn) MarshalBytes(buf []byte) { + r.CreateMeta.MarshalBytes(buf[:r.CreateMeta.SizeBytes()]) + copy(buf[r.CreateMeta.SizeBytes():], r.Name) +} + +// SizeBytes is the size of the memory representation of FUSECreateIn. +// 1 extra byte for null-terminated string. +func (r *FUSECreateIn) SizeBytes() int { + return r.CreateMeta.SizeBytes() + len(r.Name) + 1 +} + // FUSEMknodMeta contains all the static fields of FUSEMknodIn, // which is used for FUSE_MKNOD. // diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index e1bbb4b52..cfae9ed0d 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -392,6 +392,24 @@ func (inode) Valid(ctx context.Context) bool { return true } +// NewFile implements kernfs.Inode.NewFile. +func (i *inode) NewFile(ctx context.Context, name string, opts vfs.OpenOptions) (*vfs.Dentry, error) { + kernelTask := kernel.TaskFromContext(ctx) + if kernelTask == nil { + log.Warningf("fusefs.Inode.NewFile: couldn't get kernel task from context", i.NodeID) + return nil, syserror.EINVAL + } + in := linux.FUSECreateIn{ + CreateMeta: linux.FUSECreateMeta{ + Flags: opts.Flags, + Mode: uint32(opts.Mode) | linux.S_IFREG, + Umask: uint32(kernelTask.FSContext().Umask()), + }, + Name: name, + } + return i.newEntry(ctx, name, linux.S_IFREG, linux.FUSE_CREATE, &in) +} + // NewNode implements kernfs.Inode.NewNode. func (i *inode) NewNode(ctx context.Context, name string, opts vfs.MknodOptions) (*vfs.Dentry, error) { in := linux.FUSEMknodIn{ diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 02498b3a1..1a6b5b516 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -52,6 +52,10 @@ syscall_test( test = "//test/fuse/linux:readdir_test", ) +syscall_test( + fuse = "True", + test = "//test/fuse/linux:create_test", +) syscall_test( size = "large", diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index 23c9fba31..2bb956af9 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -152,4 +152,19 @@ cc_binary( "//test/util:test_main", "//test/util:test_util", ], -) \ No newline at end of file +) + +cc_binary( + name = "create_test", + testonly = 1, + srcs = ["create_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fs_util", + "//test/util:fuse_util", + "//test/util:temp_umask", + "//test/util:test_main", + "//test/util:test_util", + ], +) diff --git a/test/fuse/linux/create_test.cc b/test/fuse/linux/create_test.cc new file mode 100644 index 000000000..9a0219a58 --- /dev/null +++ b/test/fuse/linux/create_test.cc @@ -0,0 +1,128 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fs_util.h" +#include "test/util/fuse_util.h" +#include "test/util/temp_umask.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class CreateTest : public FuseTest { + protected: + const std::string test_file_name_ = "test_file"; + const mode_t mode = S_IFREG | S_IRWXU | S_IRWXG | S_IRWXO; +}; + +TEST_F(CreateTest, CreateFile) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_name_); + + // Ensure the file doesn't exist. + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header), + .error = -ENOENT, + }; + auto iov_out = FuseGenerateIovecs(out_header); + SetServerResponse(FUSE_LOOKUP, iov_out); + + // creat(2) is equal to open(2) with open_flags O_CREAT | O_WRONLY | O_TRUNC. + const mode_t new_mask = S_IWGRP | S_IWOTH; + const int open_flags = O_CREAT | O_WRONLY | O_TRUNC; + out_header.error = 0; + out_header.len = sizeof(struct fuse_out_header) + + sizeof(struct fuse_entry_out) + sizeof(struct fuse_open_out); + struct fuse_entry_out entry_payload = DefaultEntryOut(mode & ~new_mask, 2); + struct fuse_open_out out_payload = { + .fh = 1, + .open_flags = open_flags, + }; + iov_out = FuseGenerateIovecs(out_header, entry_payload, out_payload); + SetServerResponse(FUSE_CREATE, iov_out); + + // kernfs generates a successive FUSE_OPEN after the file is created. Linux's + // fuse kernel module will not send this FUSE_OPEN after creat(2). + out_header.len = + sizeof(struct fuse_out_header) + sizeof(struct fuse_open_out); + iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_OPEN, iov_out); + + int fd; + TempUmask mask(new_mask); + EXPECT_THAT(fd = creat(test_file_path.c_str(), mode), SyscallSucceeds()); + EXPECT_THAT(fcntl(fd, F_GETFL), + SyscallSucceedsWithValue(open_flags & O_ACCMODE)); + + struct fuse_in_header in_header; + struct fuse_create_in in_payload; + std::vector name(test_file_name_.size() + 1); + auto iov_in = FuseGenerateIovecs(in_header, in_payload, name); + + // Skip the request of FUSE_LOOKUP. + SkipServerActualRequest(); + + // Get the first FUSE_CREATE. + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload) + + test_file_name_.size() + 1); + EXPECT_EQ(in_header.opcode, FUSE_CREATE); + EXPECT_EQ(in_payload.flags, open_flags); + EXPECT_EQ(in_payload.mode, mode & ~new_mask); + EXPECT_EQ(in_payload.umask, new_mask); + EXPECT_EQ(std::string(name.data()), test_file_name_); + + // Get the successive FUSE_OPEN. + struct fuse_open_in in_payload_open; + iov_in = FuseGenerateIovecs(in_header, in_payload_open); + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload_open)); + EXPECT_EQ(in_header.opcode, FUSE_OPEN); + EXPECT_EQ(in_payload_open.flags, open_flags & O_ACCMODE); + + EXPECT_THAT(close(fd), SyscallSucceeds()); + // Skip the FUSE_RELEASE. + SkipServerActualRequest(); +} + +TEST_F(CreateTest, CreateFileAlreadyExists) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_name_); + + const int open_flags = O_CREAT | O_EXCL; + + SetServerInodeLookup(test_file_name_); + + EXPECT_THAT(open(test_file_path.c_str(), mode, open_flags), + SyscallFailsWithErrno(EEXIST)); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From 98faed55e682cf34bb713c37b063a7d1da5e8352 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Tue, 1 Sep 2020 01:49:57 +0000 Subject: Implement FUSE_WRITE This commit adds basic write(2) support for FUSE. --- pkg/abi/linux/fuse.go | 67 +++++--- pkg/sentry/fsimpl/fuse/connection.go | 4 + pkg/sentry/fsimpl/fuse/dev.go | 13 +- pkg/sentry/fsimpl/fuse/fusefs.go | 10 +- pkg/sentry/fsimpl/fuse/read_write.go | 90 ++++++++++ pkg/sentry/fsimpl/fuse/regular_file.go | 105 ++++++++++++ test/fuse/BUILD | 5 + test/fuse/linux/BUILD | 13 ++ test/fuse/linux/fuse_base.cc | 13 +- test/fuse/linux/fuse_base.h | 7 +- test/fuse/linux/write_test.cc | 303 +++++++++++++++++++++++++++++++++ 11 files changed, 592 insertions(+), 38 deletions(-) create mode 100644 test/fuse/linux/write_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index adc04dbcc..e49a92fb2 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -124,32 +124,6 @@ type FUSEHeaderOut struct { Unique FUSEOpID } -// FUSEWriteIn is the header written by a daemon when it makes a -// write request to the FUSE filesystem. -// -// +marshal -type FUSEWriteIn struct { - // Fh specifies the file handle that is being written to. - Fh uint64 - - // Offset is the offset of the write. - Offset uint64 - - // Size is the size of data being written. - Size uint32 - - // WriteFlags is the flags used during the write. - WriteFlags uint32 - - // LockOwner is the ID of the lock owner. - LockOwner uint64 - - // Flags is the flags for the request. - Flags uint32 - - _ uint32 -} - // FUSE_INIT flags, consistent with the ones in include/uapi/linux/fuse.h. // Our taget version is 7.23 but we have few implemented in advance. const ( @@ -427,6 +401,47 @@ type FUSEReadIn struct { _ uint32 } +// FUSEWriteIn is the first part of the payload of the +// request sent by the kernel to the daemon +// for FUSE_WRITE (struct for FUSE version >= 7.9). +// +// The second part of the payload is the +// binary bytes of the data to be written. +// +// +marshal +type FUSEWriteIn struct { + // Fh is the file handle in userspace. + Fh uint64 + + // Offset is the write offset. + Offset uint64 + + // Size is the number of bytes to write. + Size uint32 + + // ReadFlags for this FUSE_WRITE request. + WriteFlags uint32 + + // LockOwner is the id of the lock owner if there is one. + LockOwner uint64 + + // Flags for the underlying file. + Flags uint32 + + _ uint32 +} + +// FUSEWriteOut is the payload of the reply sent by the daemon to the kernel +// for a FUSE_WRITE request. +// +// +marshal +type FUSEWriteOut struct { + // Size is the number of bytes written. + Size uint32 + + _ uint32 +} + // FUSEReleaseIn is the request sent by the kernel to the daemon // when there is no more reference to a file. // diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 5f20483b7..b3f981459 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -64,6 +64,10 @@ type Request struct { id linux.FUSEOpID hdr *linux.FUSEHeaderIn data []byte + + // payload for this request: extra bytes to write after + // the data slice. Used by FUSE_WRITE. + payload []byte } // Response represents an actual response from the server, including the diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index c53a38021..6022593d6 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -152,7 +152,7 @@ func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts for !fd.queue.Empty() { req = fd.queue.Front() - if int64(req.hdr.Len) <= dst.NumBytes() { + if int64(req.hdr.Len)+int64(len(req.payload)) <= dst.NumBytes() { break } @@ -191,6 +191,17 @@ func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts return 0, syserror.EIO } + if req.hdr.Opcode == linux.FUSE_WRITE { + written, err := dst.DropFirst(n).CopyOut(ctx, req.payload) + if err != nil { + return 0, err + } + if written != len(req.payload) { + return 0, syserror.EIO + } + n += int(written) + } + // Fully done with this req, remove it from the queue. fd.queue.Remove(req) if req.hdr.Opcode == linux.FUSE_RELEASE { diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index cfae9ed0d..8cf13dcb6 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -18,6 +18,7 @@ package fuse import ( "math" "strconv" + "sync" "sync/atomic" "gvisor.dev/gvisor/pkg/abi/linux" @@ -228,13 +229,18 @@ type inode struct { kernfs.InodeNotSymlink kernfs.OrderedChildren - NodeID uint64 dentry kernfs.Dentry - locks vfs.FileLocks // the owning filesystem. fs is immutable. fs *filesystem + // metaDataMu protects the metadata of this inode. + metadataMu sync.Mutex + + NodeID uint64 + + locks vfs.FileLocks + // size of the file. size uint64 diff --git a/pkg/sentry/fsimpl/fuse/read_write.go b/pkg/sentry/fsimpl/fuse/read_write.go index 4ef8531dc..22a018e5e 100644 --- a/pkg/sentry/fsimpl/fuse/read_write.go +++ b/pkg/sentry/fsimpl/fuse/read_write.go @@ -150,3 +150,93 @@ func (fs *filesystem) ReadCallback(ctx context.Context, fd *regularFileFD, off u fs.conn.mu.Unlock() } } + +// Write sends FUSE_WRITE requests and return the bytes +// written according to the response. +// +// Preconditions: len(data) == size. +func (fs *filesystem) Write(ctx context.Context, fd *regularFileFD, off uint64, size uint32, data []byte) (uint32, error) { + t := kernel.TaskFromContext(ctx) + if t == nil { + log.Warningf("fusefs.Read: couldn't get kernel task from context") + return 0, syserror.EINVAL + } + + // One request cannnot exceed either maxWrite or maxPages. + maxWrite := uint32(fs.conn.maxPages) << usermem.PageShift + if maxWrite > fs.conn.maxWrite { + maxWrite = fs.conn.maxWrite + } + + // Reuse the same struct for unmarshalling to avoid unnecessary memory allocation. + in := linux.FUSEWriteIn{ + Fh: fd.Fh, + // TODO(gvisor.dev/issue/3245): file lock + LockOwner: 0, + // TODO(gvisor.dev/issue/3245): |= linux.FUSE_READ_LOCKOWNER + // TODO(gvisor.dev/issue/3237): |= linux.FUSE_WRITE_CACHE (not added yet) + WriteFlags: 0, + Flags: fd.statusFlags(), + } + + var written uint32 + + // This loop is intended for fragmented write where the bytes to write is + // larger than either the maxWrite or maxPages or when bigWrites is false. + // Unless a small value for max_write is explicitly used, this loop + // is expected to execute only once for the majority of the writes. + for written < size { + toWrite := size - written + + // Limit the write size to one page. + // Note that the bigWrites flag is obsolete, + // latest libfuse always sets it on. + if !fs.conn.bigWrites && toWrite > usermem.PageSize { + toWrite = usermem.PageSize + } + + // Limit the write size to maxWrite. + if toWrite > maxWrite { + toWrite = maxWrite + } + + in.Offset = off + uint64(written) + in.Size = toWrite + + req, err := fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(t.ThreadID()), fd.inode().NodeID, linux.FUSE_WRITE, &in) + if err != nil { + return 0, err + } + + req.payload = data[written : written+toWrite] + + // TODO(gvisor.dev/issue/3247): support async write. + + res, err := fs.conn.Call(t, req) + if err != nil { + return 0, err + } + if err := res.Error(); err != nil { + return 0, err + } + + out := linux.FUSEWriteOut{} + if err := res.UnmarshalPayload(&out); err != nil { + return 0, err + } + + // Write more than requested? EIO. + if out.Size > toWrite { + return 0, syserror.EIO + } + + written += out.Size + + // Break if short write. Not necessarily an error. + if out.Size != toWrite { + break + } + } + + return written, nil +} diff --git a/pkg/sentry/fsimpl/fuse/regular_file.go b/pkg/sentry/fsimpl/fuse/regular_file.go index 37ce4e268..193e77392 100644 --- a/pkg/sentry/fsimpl/fuse/regular_file.go +++ b/pkg/sentry/fsimpl/fuse/regular_file.go @@ -123,3 +123,108 @@ func (fd *regularFileFD) Read(ctx context.Context, dst usermem.IOSequence, opts fd.offMu.Unlock() return n, err } + +// PWrite implements vfs.FileDescriptionImpl.PWrite. +func (fd *regularFileFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) { + n, _, err := fd.pwrite(ctx, src, offset, opts) + return n, err +} + +// Write implements vfs.FileDescriptionImpl.Write. +func (fd *regularFileFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { + fd.offMu.Lock() + n, off, err := fd.pwrite(ctx, src, fd.off, opts) + fd.off = off + fd.offMu.Unlock() + return n, err +} + +// pwrite returns the number of bytes written, final offset and error. The +// final offset should be ignored by PWrite. +func (fd *regularFileFD) pwrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (written, finalOff int64, err error) { + if offset < 0 { + return 0, offset, syserror.EINVAL + } + + // Check that flags are supported. + // + // TODO(gvisor.dev/issue/2601): Support select preadv2 flags. + if opts.Flags&^linux.RWF_HIPRI != 0 { + return 0, offset, syserror.EOPNOTSUPP + } + + inode := fd.inode() + inode.metadataMu.Lock() + defer inode.metadataMu.Unlock() + + // If the file is opened with O_APPEND, update offset to file size. + // Note: since our Open() implements the interface of kernfs, + // and kernfs currently does not support O_APPEND, this will never + // be true before we switch out from kernfs. + if fd.vfsfd.StatusFlags()&linux.O_APPEND != 0 { + // Locking inode.metadataMu is sufficient for reading size + offset = int64(inode.size) + } + + srclen := src.NumBytes() + + if srclen > math.MaxUint32 { + // FUSE only supports uint32 for size. + // Overflow. + return 0, offset, syserror.EINVAL + } + if end := offset + srclen; end < offset { + // Overflow. + return 0, offset, syserror.EINVAL + } + + srclen, err = vfs.CheckLimit(ctx, offset, srclen) + if err != nil { + return 0, offset, err + } + + if srclen == 0 { + // Return before causing any side effects. + return 0, offset, nil + } + + src = src.TakeFirst64(srclen) + + // TODO(gvisor.dev/issue/3237): Add cache support: + // buffer cache. Ideally we write from src to our buffer cache first. + // The slice passed to fs.Write() should be a slice from buffer cache. + data := make([]byte, srclen) + // Reason for making a copy here: connection.Call() blocks on kerneltask, + // which in turn acquires mm.activeMu lock. Functions like CopyInTo() will + // attemp to acquire the mm.activeMu lock as well -> deadlock. + // We must finish reading from the userspace memory before + // t.Block() deactivates it. + cp, err := src.CopyIn(ctx, data) + if err != nil { + return 0, offset, err + } + if int64(cp) != srclen { + return 0, offset, syserror.EIO + } + + n, err := fd.inode().fs.Write(ctx, fd, uint64(offset), uint32(srclen), data) + if err != nil { + return 0, offset, err + } + + if n == 0 { + // We have checked srclen != 0 previously. + // If err == nil, then it's a short write and we return EIO. + return 0, offset, syserror.EIO + } + + written = int64(n) + finalOff = offset + written + + if finalOff > int64(inode.size) { + atomic.StoreUint64(&inode.size, uint64(finalOff)) + atomic.AddUint64(&inode.fs.conn.attributeVersion, 1) + } + + return +} diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 1a6b5b516..55ad98748 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -42,6 +42,11 @@ syscall_test( test = "//test/fuse/linux:read_test", ) +syscall_test( + fuse = "True", + test = "//test/fuse/linux:write_test", +) + syscall_test( fuse = "True", test = "//test/fuse/linux:rmdir_test", diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index 2bb956af9..cc5bd560e 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -154,6 +154,19 @@ cc_binary( ], ) +cc_binary( + name = "write_test", + testonly = 1, + srcs = ["write_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_binary( name = "create_test", testonly = 1, diff --git a/test/fuse/linux/fuse_base.cc b/test/fuse/linux/fuse_base.cc index e3c6b585c..a033db117 100644 --- a/test/fuse/linux/fuse_base.cc +++ b/test/fuse/linux/fuse_base.cc @@ -164,7 +164,8 @@ void FuseTest::UnmountFuse() { } // Consumes the first FUSE request and returns the corresponding PosixError. -PosixError FuseTest::ServerConsumeFuseInit() { +PosixError FuseTest::ServerConsumeFuseInit( + const struct fuse_init_out* out_payload) { std::vector buf(FUSE_MIN_READ_BUFFER); RETURN_ERROR_IF_SYSCALL_FAIL( RetryEINTR(read)(dev_fd_, buf.data(), buf.size())); @@ -176,10 +177,8 @@ PosixError FuseTest::ServerConsumeFuseInit() { }; // Returns a fake fuse_init_out with 7.0 version to avoid ECONNREFUSED // error in the initialization of FUSE connection. - struct fuse_init_out out_payload = { - .major = 7, - }; - auto iov_out = FuseGenerateIovecs(out_header, out_payload); + auto iov_out = FuseGenerateIovecs( + out_header, *const_cast(out_payload)); RETURN_ERROR_IF_SYSCALL_FAIL( RetryEINTR(writev)(dev_fd_, iov_out.data(), iov_out.size())); @@ -244,7 +243,7 @@ void FuseTest::ServerFuseLoop() { // becomes testing thread and the child thread becomes the FUSE server running // in background. These 2 threads are connected via socketpair. sock_[0] is // opened in testing thread and sock_[1] is opened in the FUSE server. -void FuseTest::SetUpFuseServer() { +void FuseTest::SetUpFuseServer(const struct fuse_init_out* payload) { ASSERT_THAT(socketpair(AF_UNIX, SOCK_STREAM, 0, sock_), SyscallSucceeds()); switch (fork()) { @@ -261,7 +260,7 @@ void FuseTest::SetUpFuseServer() { // Begin child thread, i.e. the FUSE server. ASSERT_THAT(close(sock_[0]), SyscallSucceeds()); - ServerCompleteWith(ServerConsumeFuseInit().ok()); + ServerCompleteWith(ServerConsumeFuseInit(payload).ok()); ServerFuseLoop(); _exit(0); } diff --git a/test/fuse/linux/fuse_base.h b/test/fuse/linux/fuse_base.h index 452748d6d..6ad296ca2 100644 --- a/test/fuse/linux/fuse_base.h +++ b/test/fuse/linux/fuse_base.h @@ -33,6 +33,8 @@ namespace testing { constexpr char kMountOpts[] = "rootmode=755,user_id=0,group_id=0"; +constexpr struct fuse_init_out kDefaultFUSEInitOutPayload = {.major = 7}; + // Internal commands used to communicate between testing thread and the FUSE // server. See test/fuse/README.md for further detail. enum class FuseTestCmd { @@ -171,7 +173,8 @@ class FuseTest : public ::testing::Test { void MountFuse(const char* mountOpts = kMountOpts); // Creates a socketpair for communication and forks FUSE server. - void SetUpFuseServer(); + void SetUpFuseServer( + const struct fuse_init_out* payload = &kDefaultFUSEInitOutPayload); // Unmounts the mountpoint of the FUSE server. void UnmountFuse(); @@ -194,7 +197,7 @@ class FuseTest : public ::testing::Test { // Consumes the first FUSE request when mounting FUSE. Replies with a // response with empty payload. - PosixError ServerConsumeFuseInit(); + PosixError ServerConsumeFuseInit(const struct fuse_init_out* payload); // A command switch that dispatch different FuseTestCmd to its handler. void ServerHandleCommand(); diff --git a/test/fuse/linux/write_test.cc b/test/fuse/linux/write_test.cc new file mode 100644 index 000000000..e7a1aff13 --- /dev/null +++ b/test/fuse/linux/write_test.cc @@ -0,0 +1,303 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class WriteTest : public FuseTest { + void SetUp() override { + FuseTest::SetUp(); + test_file_path_ = JoinPath(mount_point_.path().c_str(), test_file_); + } + + // TearDown overrides the parent's function + // to skip checking the unconsumed release request at the end. + void TearDown() override { UnmountFuse(); } + + protected: + const std::string test_file_ = "test_file"; + const mode_t test_file_mode_ = S_IFREG | S_IRWXU | S_IRWXG | S_IRWXO; + const uint64_t test_fh_ = 1; + const uint32_t open_flag_ = O_RDWR; + + std::string test_file_path_; + + PosixErrorOr OpenTestFile(const std::string &path, + uint64_t size = 512) { + SetServerInodeLookup(test_file_, test_file_mode_, size); + + struct fuse_out_header out_header_open = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_open_out), + }; + struct fuse_open_out out_payload_open = { + .fh = test_fh_, + .open_flags = open_flag_, + }; + auto iov_out_open = FuseGenerateIovecs(out_header_open, out_payload_open); + SetServerResponse(FUSE_OPEN, iov_out_open); + + auto res = Open(path.c_str(), open_flag_); + if (res.ok()) { + SkipServerActualRequest(); + } + return res; + } +}; + +class WriteTestSmallMaxWrite : public WriteTest { + void SetUp() override { + MountFuse(); + SetUpFuseServer(&fuse_init_payload); + test_file_path_ = JoinPath(mount_point_.path().c_str(), test_file_); + } + + protected: + const static uint32_t max_write_ = 4096; + constexpr static struct fuse_init_out fuse_init_payload = { + .major = 7, + .max_write = max_write_, + }; + + const uint32_t size_fragment = max_write_; +}; + +TEST_F(WriteTest, WriteNormal) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Prepare for the write. + const int n_write = 10; + struct fuse_out_header out_header_write = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_write_out), + }; + struct fuse_write_out out_payload_write = { + .size = n_write, + }; + auto iov_out_write = FuseGenerateIovecs(out_header_write, out_payload_write); + SetServerResponse(FUSE_WRITE, iov_out_write); + + // Issue the write. + std::vector buf(n_write); + RandomizeBuffer(buf.data(), buf.size()); + EXPECT_THAT(write(fd.get(), buf.data(), n_write), + SyscallSucceedsWithValue(n_write)); + + // Check the write request. + struct fuse_in_header in_header_write; + struct fuse_write_in in_payload_write; + std::vector payload_buf(n_write); + auto iov_in_write = + FuseGenerateIovecs(in_header_write, in_payload_write, payload_buf); + GetServerActualRequest(iov_in_write); + + EXPECT_EQ(in_payload_write.fh, test_fh_); + EXPECT_EQ(in_header_write.len, + sizeof(in_header_write) + sizeof(in_payload_write)); + EXPECT_EQ(in_header_write.opcode, FUSE_WRITE); + EXPECT_EQ(in_payload_write.offset, 0); + EXPECT_EQ(in_payload_write.size, n_write); + EXPECT_EQ(buf, payload_buf); +} + +TEST_F(WriteTest, WriteShort) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Prepare for the write. + const int n_write = 10, n_written = 5; + struct fuse_out_header out_header_write = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_write_out), + }; + struct fuse_write_out out_payload_write = { + .size = n_written, + }; + auto iov_out_write = FuseGenerateIovecs(out_header_write, out_payload_write); + SetServerResponse(FUSE_WRITE, iov_out_write); + + // Issue the write. + std::vector buf(n_write); + RandomizeBuffer(buf.data(), buf.size()); + EXPECT_THAT(write(fd.get(), buf.data(), n_write), + SyscallSucceedsWithValue(n_written)); + + // Check the write request. + struct fuse_in_header in_header_write; + struct fuse_write_in in_payload_write; + std::vector payload_buf(n_write); + auto iov_in_write = + FuseGenerateIovecs(in_header_write, in_payload_write, payload_buf); + GetServerActualRequest(iov_in_write); + + EXPECT_EQ(in_payload_write.fh, test_fh_); + EXPECT_EQ(in_header_write.len, + sizeof(in_header_write) + sizeof(in_payload_write)); + EXPECT_EQ(in_header_write.opcode, FUSE_WRITE); + EXPECT_EQ(in_payload_write.offset, 0); + EXPECT_EQ(in_payload_write.size, n_write); + EXPECT_EQ(buf, payload_buf); +} + +TEST_F(WriteTest, WriteShortZero) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Prepare for the write. + const int n_write = 10; + struct fuse_out_header out_header_write = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_write_out), + }; + struct fuse_write_out out_payload_write = { + .size = 0, + }; + auto iov_out_write = FuseGenerateIovecs(out_header_write, out_payload_write); + SetServerResponse(FUSE_WRITE, iov_out_write); + + // Issue the write. + std::vector buf(n_write); + RandomizeBuffer(buf.data(), buf.size()); + EXPECT_THAT(write(fd.get(), buf.data(), n_write), SyscallFailsWithErrno(EIO)); + + // Check the write request. + struct fuse_in_header in_header_write; + struct fuse_write_in in_payload_write; + std::vector payload_buf(n_write); + auto iov_in_write = + FuseGenerateIovecs(in_header_write, in_payload_write, payload_buf); + GetServerActualRequest(iov_in_write); + + EXPECT_EQ(in_payload_write.fh, test_fh_); + EXPECT_EQ(in_header_write.len, + sizeof(in_header_write) + sizeof(in_payload_write)); + EXPECT_EQ(in_header_write.opcode, FUSE_WRITE); + EXPECT_EQ(in_payload_write.offset, 0); + EXPECT_EQ(in_payload_write.size, n_write); + EXPECT_EQ(buf, payload_buf); +} + +TEST_F(WriteTest, WriteZero) { + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_)); + + // Issue the write. + std::vector buf(0); + EXPECT_THAT(write(fd.get(), buf.data(), 0), SyscallSucceedsWithValue(0)); +} + +TEST_F(WriteTest, PWrite) { + const int file_size = 512; + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_, file_size)); + + // Prepare for the write. + const int n_write = 10; + struct fuse_out_header out_header_write = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_write_out), + }; + struct fuse_write_out out_payload_write = { + .size = n_write, + }; + auto iov_out_write = FuseGenerateIovecs(out_header_write, out_payload_write); + SetServerResponse(FUSE_WRITE, iov_out_write); + + // Issue the write. + std::vector buf(n_write); + RandomizeBuffer(buf.data(), buf.size()); + const int offset_write = file_size >> 1; + EXPECT_THAT(pwrite(fd.get(), buf.data(), n_write, offset_write), + SyscallSucceedsWithValue(n_write)); + + // Check the write request. + struct fuse_in_header in_header_write; + struct fuse_write_in in_payload_write; + std::vector payload_buf(n_write); + auto iov_in_write = + FuseGenerateIovecs(in_header_write, in_payload_write, payload_buf); + GetServerActualRequest(iov_in_write); + + EXPECT_EQ(in_payload_write.fh, test_fh_); + EXPECT_EQ(in_header_write.len, + sizeof(in_header_write) + sizeof(in_payload_write)); + EXPECT_EQ(in_header_write.opcode, FUSE_WRITE); + EXPECT_EQ(in_payload_write.offset, offset_write); + EXPECT_EQ(in_payload_write.size, n_write); + EXPECT_EQ(buf, payload_buf); +} + +TEST_F(WriteTestSmallMaxWrite, WriteSmallMaxWrie) { + const int n_fragment = 10; + const int n_write = size_fragment * n_fragment; + + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenTestFile(test_file_path_, n_write)); + + // Prepare for the write. + struct fuse_out_header out_header_write = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_write_out), + }; + struct fuse_write_out out_payload_write = { + .size = size_fragment, + }; + auto iov_out_write = FuseGenerateIovecs(out_header_write, out_payload_write); + + for (int i = 0; i < n_fragment; ++i) { + SetServerResponse(FUSE_WRITE, iov_out_write); + } + + // Issue the write. + std::vector buf(n_write); + RandomizeBuffer(buf.data(), buf.size()); + EXPECT_THAT(write(fd.get(), buf.data(), n_write), + SyscallSucceedsWithValue(n_write)); + + ASSERT_EQ(GetServerNumUnsentResponses(), 0); + ASSERT_EQ(GetServerNumUnconsumedRequests(), n_fragment); + + // Check the write request. + struct fuse_in_header in_header_write; + struct fuse_write_in in_payload_write; + std::vector payload_buf(size_fragment); + auto iov_in_write = + FuseGenerateIovecs(in_header_write, in_payload_write, payload_buf); + + for (int i = 0; i < n_fragment; ++i) { + GetServerActualRequest(iov_in_write); + + EXPECT_EQ(in_payload_write.fh, test_fh_); + EXPECT_EQ(in_header_write.len, + sizeof(in_header_write) + sizeof(in_payload_write)); + EXPECT_EQ(in_header_write.opcode, FUSE_WRITE); + EXPECT_EQ(in_payload_write.offset, i * size_fragment); + EXPECT_EQ(in_payload_write.size, size_fragment); + + auto it = buf.begin() + i * size_fragment; + EXPECT_EQ(std::vector(it, it + size_fragment), payload_buf); + } +} + +} // namespace + +} // namespace testing +} // namespace gvisor \ No newline at end of file -- cgit v1.2.3 From 4181e8c97482a3c787c2b508e6d75a21323ba515 Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Wed, 9 Sep 2020 09:16:09 -0700 Subject: Add fh support for revise attr and fstat(2) test According to Linux 4.4's FUSE behavior, the flags and fh attributes in FUSE_GETATTR are only used in read, write, and lseek. fstat(2) doesn't use them either. Add tests to ensure the requests sent from FUSE module are consistent with Linux's. Updates #3655 --- pkg/abi/linux/fuse.go | 5 ++ pkg/sentry/fsimpl/fuse/fusefs.go | 21 +++---- pkg/sentry/fsimpl/fuse/regular_file.go | 2 +- test/fuse/linux/BUILD | 4 +- test/fuse/linux/stat_test.cc | 112 ++++++++++++++++++++++++++++++--- test/util/fuse_util.cc | 12 ++-- test/util/fuse_util.h | 5 +- 7 files changed, 133 insertions(+), 28 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index e49a92fb2..f0bef1e8e 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -227,6 +227,11 @@ type FUSEInitOut struct { _ [8]uint32 } +// FUSE_GETATTR_FH is currently the only flag of FUSEGetAttrIn.GetAttrFlags. +// If it is set, the file handle (FUSEGetAttrIn.Fh) is used to indicate the +// object instead of the node id attribute in the request header. +const FUSE_GETATTR_FH = (1 << 0) + // FUSEGetAttrIn is the request sent by the kernel to the daemon, // to get the attribute of a inode. // diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 8cf13dcb6..821048d87 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -609,9 +609,9 @@ func statFromFUSEAttr(attr linux.FUSEAttr, mask, devMinor uint32) linux.Statx { } // getAttr gets the attribute of this inode by issuing a FUSE_GETATTR request -// or read from local cache. -// It updates the corresponding attributes if necessary. -func (i *inode) getAttr(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOptions) (linux.FUSEAttr, error) { +// or read from local cache. It updates the corresponding attributes if +// necessary. +func (i *inode) getAttr(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOptions, flags uint32, fh uint64) (linux.FUSEAttr, error) { attributeVersion := atomic.LoadUint64(&i.fs.conn.attributeVersion) // TODO(gvisor.dev/issue/3679): send the request only if @@ -631,11 +631,10 @@ func (i *inode) getAttr(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOp creds := auth.CredentialsFromContext(ctx) - var in linux.FUSEGetAttrIn - // We don't set any attribute in the request, because in VFS2 fstat(2) will - // finally be translated into vfs.FilesystemImpl.StatAt() (see - // pkg/sentry/syscalls/linux/vfs2/stat.go), resulting in the same flow - // as stat(2). Thus GetAttrFlags and Fh variable will never be used in VFS2. + in := linux.FUSEGetAttrIn{ + GetAttrFlags: flags, + Fh: fh, + } req, err := i.fs.conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_GETATTR, &in) if err != nil { return linux.FUSEAttr{}, err @@ -676,17 +675,17 @@ func (i *inode) getAttr(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOp // reviseAttr attempts to update the attributes for internal purposes // by calling getAttr with a pre-specified mask. // Used by read, write, lseek. -func (i *inode) reviseAttr(ctx context.Context) error { +func (i *inode) reviseAttr(ctx context.Context, flags uint32, fh uint64) error { // Never need atime for internal purposes. _, err := i.getAttr(ctx, i.fs.VFSFilesystem(), vfs.StatOptions{ Mask: linux.STATX_BASIC_STATS &^ linux.STATX_ATIME, - }) + }, flags, fh) return err } // Stat implements kernfs.Inode.Stat. func (i *inode) Stat(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) { - attr, err := i.getAttr(ctx, fs, opts) + attr, err := i.getAttr(ctx, fs, opts, 0, 0) if err != nil { return linux.Statx{}, err } diff --git a/pkg/sentry/fsimpl/fuse/regular_file.go b/pkg/sentry/fsimpl/fuse/regular_file.go index 193e77392..5bdd096c3 100644 --- a/pkg/sentry/fsimpl/fuse/regular_file.go +++ b/pkg/sentry/fsimpl/fuse/regular_file.go @@ -65,7 +65,7 @@ func (fd *regularFileFD) PRead(ctx context.Context, dst usermem.IOSequence, offs // Reading beyond EOF, update file size if outdated. if uint64(offset+size) > atomic.LoadUint64(&inode.size) { - if err := inode.reviseAttr(ctx); err != nil { + if err := inode.reviseAttr(ctx, linux.FUSE_GETATTR_FH, fd.Fh); err != nil { return 0, err } // If the offset after update is still too large, return error. diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index eb090cbfd..7a3e52fad 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -11,7 +11,9 @@ cc_binary( srcs = ["stat_test.cc"], deps = [ gtest, - ":fuse_base", + ":fuse_fd_util", + "//test/util:cleanup", + "//test/util:fs_util", "//test/util:fuse_util", "//test/util:test_main", "//test/util:test_util", diff --git a/test/fuse/linux/stat_test.cc b/test/fuse/linux/stat_test.cc index 717fd1fac..6f032cac1 100644 --- a/test/fuse/linux/stat_test.cc +++ b/test/fuse/linux/stat_test.cc @@ -18,12 +18,15 @@ #include #include #include +#include #include #include #include "gtest/gtest.h" -#include "test/fuse/linux/fuse_base.h" +#include "test/fuse/linux/fuse_fd_util.h" +#include "test/util/cleanup.h" +#include "test/util/fs_util.h" #include "test/util/fuse_util.h" #include "test/util/test_util.h" @@ -32,19 +35,30 @@ namespace testing { namespace { -class StatTest : public FuseTest { +class StatTest : public FuseFdTest { public: + void SetUp() override { + FuseFdTest::SetUp(); + test_file_path_ = JoinPath(mount_point_.path(), test_file_); + } + + protected: bool StatsAreEqual(struct stat expected, struct stat actual) { - // device number will be dynamically allocated by kernel, we cannot know - // in advance + // Device number will be dynamically allocated by kernel, we cannot know in + // advance. actual.st_dev = expected.st_dev; return memcmp(&expected, &actual, sizeof(struct stat)) == 0; } + + const std::string test_file_ = "testfile"; + const mode_t expected_mode = S_IFREG | S_IRUSR | S_IWUSR; + const uint64_t fh = 23; + + std::string test_file_path_; }; TEST_F(StatTest, StatNormal) { // Set up fixture. - mode_t expected_mode = S_IRWXU | S_IRGRP | S_IXGRP | S_IROTH | S_IXOTH; struct fuse_attr attr = DefaultFuseAttr(expected_mode, 1); struct fuse_out_header out_header = { .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), @@ -55,7 +69,7 @@ TEST_F(StatTest, StatNormal) { auto iov_out = FuseGenerateIovecs(out_header, out_payload); SetServerResponse(FUSE_GETATTR, iov_out); - // Do integration test. + // Make syscall. struct stat stat_buf; EXPECT_THAT(stat(mount_point_.path().c_str(), &stat_buf), SyscallSucceeds()); @@ -99,7 +113,7 @@ TEST_F(StatTest, StatNotFound) { auto iov_out = FuseGenerateIovecs(out_header); SetServerResponse(FUSE_GETATTR, iov_out); - // Do integration test. + // Make syscall. struct stat stat_buf; EXPECT_THAT(stat(mount_point_.path().c_str(), &stat_buf), SyscallFailsWithErrno(ENOENT)); @@ -115,6 +129,90 @@ TEST_F(StatTest, StatNotFound) { EXPECT_EQ(in_payload.fh, 0); } +TEST_F(StatTest, FstatNormal) { + // Set up fixture. + SetServerInodeLookup(test_file_); + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenPath(test_file_path_, O_RDONLY, fh)); + auto close_fd = CloseFD(fd); + + struct fuse_attr attr = DefaultFuseAttr(expected_mode, 2); + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + }; + struct fuse_attr_out out_payload = { + .attr = attr, + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_GETATTR, iov_out); + + // Make syscall. + struct stat stat_buf; + EXPECT_THAT(fstat(fd.get(), &stat_buf), SyscallSucceeds()); + + // Check filesystem operation result. + struct stat expected_stat = { + .st_ino = attr.ino, + .st_nlink = attr.nlink, + .st_mode = expected_mode, + .st_uid = attr.uid, + .st_gid = attr.gid, + .st_rdev = attr.rdev, + .st_size = static_cast(attr.size), + .st_blksize = attr.blksize, + .st_blocks = static_cast(attr.blocks), + .st_atim = (struct timespec){.tv_sec = static_cast(attr.atime), + .tv_nsec = attr.atimensec}, + .st_mtim = (struct timespec){.tv_sec = static_cast(attr.mtime), + .tv_nsec = attr.mtimensec}, + .st_ctim = (struct timespec){.tv_sec = static_cast(attr.ctime), + .tv_nsec = attr.ctimensec}, + }; + EXPECT_TRUE(StatsAreEqual(stat_buf, expected_stat)); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_getattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.opcode, FUSE_GETATTR); + EXPECT_EQ(in_payload.getattr_flags, 0); + EXPECT_EQ(in_payload.fh, 0); +} + +TEST_F(StatTest, StatByFileHandle) { + // Set up fixture. + SetServerInodeLookup(test_file_, expected_mode, 0); + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenPath(test_file_path_, O_RDONLY, fh)); + auto close_fd = CloseFD(fd); + + struct fuse_attr attr = DefaultFuseAttr(expected_mode, 2, 0); + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + }; + struct fuse_attr_out out_payload = { + .attr = attr, + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_GETATTR, iov_out); + + // Make syscall. + std::vector buf(1); + // Since this is an empty file, it won't issue FUSE_READ. But a FUSE_GETATTR + // will be issued before read completes. + EXPECT_THAT(read(fd.get(), buf.data(), buf.size()), SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_getattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.opcode, FUSE_GETATTR); + EXPECT_EQ(in_payload.getattr_flags, FUSE_GETATTR_FH); + EXPECT_EQ(in_payload.fh, fh); +} + } // namespace } // namespace testing diff --git a/test/util/fuse_util.cc b/test/util/fuse_util.cc index 4db053335..595d0cebf 100644 --- a/test/util/fuse_util.cc +++ b/test/util/fuse_util.cc @@ -22,13 +22,13 @@ namespace gvisor { namespace testing { -// Create a default FuseAttr struct with specified mode and inode. -fuse_attr DefaultFuseAttr(mode_t mode, uint64_t inode) { +// Create a default FuseAttr struct with specified mode, inode, and size. +fuse_attr DefaultFuseAttr(mode_t mode, uint64_t inode, uint64_t size) { const int time_sec = 1595436289; const int time_nsec = 134150844; return (struct fuse_attr){ .ino = inode, - .size = 512, + .size = size, .blocks = 4, .atime = time_sec, .mtime = time_sec, @@ -45,8 +45,8 @@ fuse_attr DefaultFuseAttr(mode_t mode, uint64_t inode) { }; } -// Create response body with specified mode and nodeID. -fuse_entry_out DefaultEntryOut(mode_t mode, uint64_t node_id) { +// Create response body with specified mode, nodeID, and size. +fuse_entry_out DefaultEntryOut(mode_t mode, uint64_t node_id, uint64_t size) { struct fuse_entry_out default_entry_out = { .nodeid = node_id, .generation = 0, @@ -54,7 +54,7 @@ fuse_entry_out DefaultEntryOut(mode_t mode, uint64_t node_id) { .attr_valid = 0, .entry_valid_nsec = 0, .attr_valid_nsec = 0, - .attr = DefaultFuseAttr(mode, node_id), + .attr = DefaultFuseAttr(mode, node_id, size), }; return default_entry_out; }; diff --git a/test/util/fuse_util.h b/test/util/fuse_util.h index 6b5a8ce1f..544fe1b38 100644 --- a/test/util/fuse_util.h +++ b/test/util/fuse_util.h @@ -64,10 +64,11 @@ std::vector FuseGenerateIovecs(T &first, Types &...args) { } // Create a fuse_attr filled with the specified mode and inode. -fuse_attr DefaultFuseAttr(mode_t mode, uint64_t inode); +fuse_attr DefaultFuseAttr(mode_t mode, uint64_t inode, uint64_t size = 512); // Return a fuse_entry_out FUSE server response body. -fuse_entry_out DefaultEntryOut(mode_t mode, uint64_t nodeId); +fuse_entry_out DefaultEntryOut(mode_t mode, uint64_t node_id, + uint64_t size = 512); } // namespace testing } // namespace gvisor -- cgit v1.2.3 From bf8efe8cdf4b6af50ec89cda37342cf51c8b50b4 Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Wed, 9 Sep 2020 10:44:09 -0700 Subject: Implement FUSE_SETATTR This commit implements FUSE_SETATTR command. When a system call modifies the metadata of a regular file or a folder by chown(2), chmod(2), truncate(2), utime(2), or utimes(2), they should be translated to corresponding FUSE_SETATTR command and sent to the FUSE server. Fixes #3332 --- pkg/abi/linux/fuse.go | 67 ++++++ pkg/sentry/fsimpl/fuse/file.go | 2 +- pkg/sentry/fsimpl/fuse/fusefs.go | 83 ++++++- pkg/sentry/fsimpl/kernfs/inode_impl_util.go | 7 + test/fuse/BUILD | 5 + test/fuse/linux/BUILD | 16 ++ test/fuse/linux/setstat_test.cc | 338 ++++++++++++++++++++++++++++ 7 files changed, 516 insertions(+), 2 deletions(-) create mode 100644 test/fuse/linux/setstat_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index f0bef1e8e..fcc957bfe 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -749,3 +749,70 @@ func (r *FUSEDirent) UnmarshalBytes(src []byte) { name.UnmarshalBytes(src[:r.Meta.NameLen]) r.Name = string(name) } + +// FATTR_* consts are the attribute flags defined in include/uapi/linux/fuse.h. +// These should be or-ed together for setattr to know what has been changed. +const ( + FATTR_MODE = (1 << 0) + FATTR_UID = (1 << 1) + FATTR_GID = (1 << 2) + FATTR_SIZE = (1 << 3) + FATTR_ATIME = (1 << 4) + FATTR_MTIME = (1 << 5) + FATTR_FH = (1 << 6) + FATTR_ATIME_NOW = (1 << 7) + FATTR_MTIME_NOW = (1 << 8) + FATTR_LOCKOWNER = (1 << 9) + FATTR_CTIME = (1 << 10) +) + +// FUSESetAttrIn is the request sent by the kernel to the daemon, +// to set the attribute(s) of a file. +// +// +marshal +type FUSESetAttrIn struct { + // Valid indicates which attributes are modified by this request. + Valid uint32 + + _ uint32 + + // Fh is used to identify the file if FATTR_FH is set in Valid. + Fh uint64 + + // Size is the size that the request wants to change to. + Size uint64 + + // LockOwner is the owner of the lock that the request wants to change to. + LockOwner uint64 + + // Atime is the access time that the request wants to change to. + Atime uint64 + + // Mtime is the modification time that the request wants to change to. + Mtime uint64 + + // Ctime is the status change time that the request wants to change to. + Ctime uint64 + + // AtimeNsec is the nano second part of Atime. + AtimeNsec uint32 + + // MtimeNsec is the nano second part of Mtime. + MtimeNsec uint32 + + // CtimeNsec is the nano second part of Ctime. + CtimeNsec uint32 + + // Mode is the file mode that the request wants to change to. + Mode uint32 + + _ uint32 + + // UID is the user ID of the owner that the request wants to change to. + UID uint32 + + // GID is the group ID of the owner that the request wants to change to. + GID uint32 + + _ uint32 +} diff --git a/pkg/sentry/fsimpl/fuse/file.go b/pkg/sentry/fsimpl/fuse/file.go index 186ec2362..15c0e3f41 100644 --- a/pkg/sentry/fsimpl/fuse/file.go +++ b/pkg/sentry/fsimpl/fuse/file.go @@ -123,5 +123,5 @@ func (fd *fileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linu // SetStat implements FileDescriptionImpl.SetStat. func (fd *fileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions) error { creds := auth.CredentialsFromContext(ctx) - return fd.inode().SetStat(ctx, fd.inode().fs.VFSFilesystem(), creds, opts) + return fd.inode().setAttr(ctx, fd.inode().fs.VFSFilesystem(), creds, opts, true, fd.Fh) } diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 821048d87..572245303 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -660,7 +660,7 @@ func (i *inode) getAttr(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOp } // Set the metadata of kernfs.InodeAttrs. - if err := i.SetStat(ctx, fs, creds, vfs.SetStatOptions{ + if err := i.SetInodeStat(ctx, fs, creds, vfs.SetStatOptions{ Stat: statFromFUSEAttr(out.Attr, linux.STATX_ALL, i.fs.devMinor), }); err != nil { return linux.FUSEAttr{}, err @@ -703,3 +703,84 @@ func (i *inode) StatFS(ctx context.Context, fs *vfs.Filesystem) (linux.Statfs, e // TODO(gvisor.dev/issues/3413): Complete the implementation of statfs. return vfs.GenericStatFS(linux.FUSE_SUPER_MAGIC), nil } + +// fattrMaskFromStats converts vfs.SetStatOptions.Stat.Mask to linux stats mask +// aligned with the attribute mask defined in include/linux/fs.h. +func fattrMaskFromStats(mask uint32) uint32 { + var fuseAttrMask uint32 + maskMap := map[uint32]uint32{ + linux.STATX_MODE: linux.FATTR_MODE, + linux.STATX_UID: linux.FATTR_UID, + linux.STATX_GID: linux.FATTR_GID, + linux.STATX_SIZE: linux.FATTR_SIZE, + linux.STATX_ATIME: linux.FATTR_ATIME, + linux.STATX_MTIME: linux.FATTR_MTIME, + linux.STATX_CTIME: linux.FATTR_CTIME, + } + for statxMask, fattrMask := range maskMap { + if mask&statxMask != 0 { + fuseAttrMask |= fattrMask + } + } + return fuseAttrMask +} + +// SetStat implements kernfs.Inode.SetStat. +func (i *inode) SetStat(ctx context.Context, fs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions) error { + return i.setAttr(ctx, fs, creds, opts, false, 0) +} + +func (i *inode) setAttr(ctx context.Context, fs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions, useFh bool, fh uint64) error { + conn := i.fs.conn + task := kernel.TaskFromContext(ctx) + if task == nil { + log.Warningf("couldn't get kernel task from context") + return syserror.EINVAL + } + + // We should retain the original file type when assigning new mode. + fileType := uint16(i.Mode()) & linux.S_IFMT + fattrMask := fattrMaskFromStats(opts.Stat.Mask) + if useFh { + fattrMask |= linux.FATTR_FH + } + in := linux.FUSESetAttrIn{ + Valid: fattrMask, + Fh: fh, + Size: opts.Stat.Size, + Atime: uint64(opts.Stat.Atime.Sec), + Mtime: uint64(opts.Stat.Mtime.Sec), + Ctime: uint64(opts.Stat.Ctime.Sec), + AtimeNsec: opts.Stat.Atime.Nsec, + MtimeNsec: opts.Stat.Mtime.Nsec, + CtimeNsec: opts.Stat.Ctime.Nsec, + Mode: uint32(fileType | opts.Stat.Mode), + UID: opts.Stat.UID, + GID: opts.Stat.GID, + } + req, err := conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_SETATTR, &in) + if err != nil { + return err + } + + res, err := conn.Call(task, req) + if err != nil { + return err + } + if err := res.Error(); err != nil { + return err + } + out := linux.FUSEGetAttrOut{} + if err := res.UnmarshalPayload(&out); err != nil { + return err + } + + // Set the metadata of kernfs.InodeAttrs. + if err := i.SetInodeStat(ctx, fs, creds, vfs.SetStatOptions{ + Stat: statFromFUSEAttr(out.Attr, linux.STATX_ALL, i.fs.devMinor), + }); err != nil { + return err + } + + return nil +} diff --git a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go index 53758a4b8..c2109cf76 100644 --- a/pkg/sentry/fsimpl/kernfs/inode_impl_util.go +++ b/pkg/sentry/fsimpl/kernfs/inode_impl_util.go @@ -256,6 +256,13 @@ func (a *InodeAttrs) Stat(context.Context, *vfs.Filesystem, vfs.StatOptions) (li // SetStat implements Inode.SetStat. func (a *InodeAttrs) SetStat(ctx context.Context, fs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions) error { + return a.SetInodeStat(ctx, fs, creds, opts) +} + +// SetInodeStat sets the corresponding attributes from opts to InodeAttrs. +// This function can be used by other kernfs-based filesystem implementation to +// sets the unexported attributes into kernfs.InodeAttrs. +func (a *InodeAttrs) SetInodeStat(ctx context.Context, fs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions) error { if opts.Stat.Mask == 0 { return nil } diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 55ad98748..29b9a9d93 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -62,6 +62,11 @@ syscall_test( test = "//test/fuse/linux:create_test", ) +syscall_test( + fuse = "True", + test = "//test/fuse/linux:setstat_test", +) + syscall_test( size = "large", add_overlay = True, diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index 7a3e52fad..7ecd6d8cb 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -100,6 +100,22 @@ cc_binary( ], ) +cc_binary( + name = "setstat_test", + testonly = 1, + srcs = ["setstat_test.cc"], + deps = [ + gtest, + ":fuse_fd_util", + "//test/util:cleanup", + "//test/util:fs_util", + "//test/util:fuse_util", + "//test/util:temp_umask", + "//test/util:test_main", + "//test/util:test_util", + ], +) + cc_binary( name = "rmdir_test", testonly = 1, diff --git a/test/fuse/linux/setstat_test.cc b/test/fuse/linux/setstat_test.cc new file mode 100644 index 000000000..68301c775 --- /dev/null +++ b/test/fuse/linux/setstat_test.cc @@ -0,0 +1,338 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_fd_util.h" +#include "test/util/cleanup.h" +#include "test/util/fs_util.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class SetStatTest : public FuseFdTest { + public: + void SetUp() override { + FuseFdTest::SetUp(); + test_dir_path_ = JoinPath(mount_point_.path(), test_dir_); + test_file_path_ = JoinPath(mount_point_.path(), test_file_); + } + + protected: + const uint64_t fh = 23; + const std::string test_dir_ = "testdir"; + const std::string test_file_ = "testfile"; + const mode_t test_dir_mode_ = S_IFDIR | S_IRUSR | S_IWUSR | S_IXUSR; + const mode_t test_file_mode_ = S_IFREG | S_IRUSR | S_IWUSR | S_IXUSR; + + std::string test_dir_path_; + std::string test_file_path_; +}; + +TEST_F(SetStatTest, ChmodDir) { + // Set up fixture. + SetServerInodeLookup(test_dir_, test_dir_mode_); + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + .error = 0, + }; + mode_t set_mode = S_IRGRP | S_IWGRP | S_IXGRP; + struct fuse_attr_out out_payload = { + .attr = DefaultFuseAttr(set_mode, 2), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SETATTR, iov_out); + + // Make syscall. + EXPECT_THAT(chmod(test_dir_path_.c_str(), set_mode), SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_setattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_SETATTR); + EXPECT_EQ(in_header.uid, 0); + EXPECT_EQ(in_header.gid, 0); + EXPECT_EQ(in_payload.valid, FATTR_MODE); + EXPECT_EQ(in_payload.mode, S_IFDIR | set_mode); +} + +TEST_F(SetStatTest, ChownDir) { + // Set up fixture. + SetServerInodeLookup(test_dir_, test_dir_mode_); + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + .error = 0, + }; + struct fuse_attr_out out_payload = { + .attr = DefaultFuseAttr(test_dir_mode_, 2), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SETATTR, iov_out); + + // Make syscall. + EXPECT_THAT(chown(test_dir_path_.c_str(), 1025, 1025), SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_setattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_SETATTR); + EXPECT_EQ(in_header.uid, 0); + EXPECT_EQ(in_header.gid, 0); + EXPECT_EQ(in_payload.valid, FATTR_UID | FATTR_GID); + EXPECT_EQ(in_payload.uid, 1025); + EXPECT_EQ(in_payload.gid, 1025); +} + +TEST_F(SetStatTest, TruncateFile) { + // Set up fixture. + SetServerInodeLookup(test_file_, test_file_mode_); + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + .error = 0, + }; + struct fuse_attr_out out_payload = { + .attr = DefaultFuseAttr(S_IFREG | S_IRUSR | S_IWUSR, 2), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SETATTR, iov_out); + + // Make syscall. + EXPECT_THAT(truncate(test_file_path_.c_str(), 321), SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_setattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_SETATTR); + EXPECT_EQ(in_header.uid, 0); + EXPECT_EQ(in_header.gid, 0); + EXPECT_EQ(in_payload.valid, FATTR_SIZE); + EXPECT_EQ(in_payload.size, 321); +} + +TEST_F(SetStatTest, UtimeFile) { + // Set up fixture. + SetServerInodeLookup(test_file_, test_file_mode_); + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + .error = 0, + }; + struct fuse_attr_out out_payload = { + .attr = DefaultFuseAttr(S_IFREG | S_IRUSR | S_IWUSR, 2), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SETATTR, iov_out); + + // Make syscall. + time_t expected_atime = 1597159766, expected_mtime = 1597159765; + struct utimbuf times = { + .actime = expected_atime, + .modtime = expected_mtime, + }; + EXPECT_THAT(utime(test_file_path_.c_str(), ×), SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_setattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_SETATTR); + EXPECT_EQ(in_header.uid, 0); + EXPECT_EQ(in_header.gid, 0); + EXPECT_EQ(in_payload.valid, FATTR_ATIME | FATTR_MTIME); + EXPECT_EQ(in_payload.atime, expected_atime); + EXPECT_EQ(in_payload.mtime, expected_mtime); +} + +TEST_F(SetStatTest, UtimesFile) { + // Set up fixture. + SetServerInodeLookup(test_file_, test_file_mode_); + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + .error = 0, + }; + struct fuse_attr_out out_payload = { + .attr = DefaultFuseAttr(test_file_mode_, 2), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SETATTR, iov_out); + + // Make syscall. + struct timeval expected_times[2] = { + { + .tv_sec = 1597159766, + .tv_usec = 234945, + }, + { + .tv_sec = 1597159765, + .tv_usec = 232341, + }, + }; + EXPECT_THAT(utimes(test_file_path_.c_str(), expected_times), + SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_setattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_SETATTR); + EXPECT_EQ(in_header.uid, 0); + EXPECT_EQ(in_header.gid, 0); + EXPECT_EQ(in_payload.valid, FATTR_ATIME | FATTR_MTIME); + EXPECT_EQ(in_payload.atime, expected_times[0].tv_sec); + EXPECT_EQ(in_payload.atimensec, expected_times[0].tv_usec * 1000); + EXPECT_EQ(in_payload.mtime, expected_times[1].tv_sec); + EXPECT_EQ(in_payload.mtimensec, expected_times[1].tv_usec * 1000); +} + +TEST_F(SetStatTest, FtruncateFile) { + // Set up fixture. + SetServerInodeLookup(test_file_, test_file_mode_); + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenPath(test_file_path_, O_RDWR, fh)); + auto close_fd = CloseFD(fd); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + .error = 0, + }; + struct fuse_attr_out out_payload = { + .attr = DefaultFuseAttr(test_file_mode_, 2), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SETATTR, iov_out); + + // Make syscall. + EXPECT_THAT(ftruncate(fd.get(), 321), SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_setattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_SETATTR); + EXPECT_EQ(in_header.uid, 0); + EXPECT_EQ(in_header.gid, 0); + EXPECT_EQ(in_payload.valid, FATTR_SIZE | FATTR_FH); + EXPECT_EQ(in_payload.fh, fh); + EXPECT_EQ(in_payload.size, 321); +} + +TEST_F(SetStatTest, FchmodFile) { + // Set up fixture. + SetServerInodeLookup(test_file_, test_file_mode_); + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenPath(test_file_path_, O_RDWR, fh)); + auto close_fd = CloseFD(fd); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + .error = 0, + }; + mode_t set_mode = S_IROTH | S_IWOTH | S_IXOTH; + struct fuse_attr_out out_payload = { + .attr = DefaultFuseAttr(set_mode, 2), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SETATTR, iov_out); + + // Make syscall. + EXPECT_THAT(fchmod(fd.get(), set_mode), SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_setattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_SETATTR); + EXPECT_EQ(in_header.uid, 0); + EXPECT_EQ(in_header.gid, 0); + EXPECT_EQ(in_payload.valid, FATTR_MODE | FATTR_FH); + EXPECT_EQ(in_payload.fh, fh); + EXPECT_EQ(in_payload.mode, S_IFREG | set_mode); +} + +TEST_F(SetStatTest, FchownFile) { + // Set up fixture. + SetServerInodeLookup(test_file_, test_file_mode_); + auto fd = ASSERT_NO_ERRNO_AND_VALUE(OpenPath(test_file_path_, O_RDWR, fh)); + auto close_fd = CloseFD(fd); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header) + sizeof(struct fuse_attr_out), + .error = 0, + }; + struct fuse_attr_out out_payload = { + .attr = DefaultFuseAttr(S_IFREG | S_IRUSR | S_IWUSR | S_IXUSR, 2), + }; + auto iov_out = FuseGenerateIovecs(out_header, out_payload); + SetServerResponse(FUSE_SETATTR, iov_out); + + // Make syscall. + EXPECT_THAT(fchown(fd.get(), 1025, 1025), SyscallSucceeds()); + + // Check FUSE request. + struct fuse_in_header in_header; + struct fuse_setattr_in in_payload; + auto iov_in = FuseGenerateIovecs(in_header, in_payload); + + GetServerActualRequest(iov_in); + EXPECT_EQ(in_header.len, sizeof(in_header) + sizeof(in_payload)); + EXPECT_EQ(in_header.opcode, FUSE_SETATTR); + EXPECT_EQ(in_header.uid, 0); + EXPECT_EQ(in_header.gid, 0); + EXPECT_EQ(in_payload.valid, FATTR_UID | FATTR_GID | FATTR_FH); + EXPECT_EQ(in_payload.fh, fh); + EXPECT_EQ(in_payload.uid, 1025); + EXPECT_EQ(in_payload.gid, 1025); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From f1219ec5f17b10f18d7d5e6605ea27ecba40c409 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Thu, 3 Sep 2020 18:41:26 +0000 Subject: Refactor FUSE connection for readability and structure This change decouples the code that is weakly tied to the connection struct from connection.go, rename variables and files with more meaningful choices, adds detailed comments, explains lock orders, and adds other minor improvement to make the existing FUSE code more readable and more organized. Purpose is to avoid too much code in one file and provide better structure for the future commits. --- pkg/sentry/fsimpl/fuse/BUILD | 2 +- pkg/sentry/fsimpl/fuse/connection.go | 264 +++++---------------------- pkg/sentry/fsimpl/fuse/connection_control.go | 183 +++++++++++++++++++ pkg/sentry/fsimpl/fuse/dev_test.go | 2 +- pkg/sentry/fsimpl/fuse/fusefs.go | 25 ++- pkg/sentry/fsimpl/fuse/init.go | 165 ----------------- pkg/sentry/fsimpl/fuse/request_response.go | 150 +++++++++++++++ 7 files changed, 398 insertions(+), 393 deletions(-) create mode 100644 pkg/sentry/fsimpl/fuse/connection_control.go delete mode 100644 pkg/sentry/fsimpl/fuse/init.go (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index 5085d0521..5b60b1277 100644 --- a/pkg/sentry/fsimpl/fuse/BUILD +++ b/pkg/sentry/fsimpl/fuse/BUILD @@ -30,11 +30,11 @@ go_library( name = "fuse", srcs = [ "connection.go", + "connection_control.go", "dev.go", "directory.go", "file.go", "fusefs.go", - "init.go", "inode_refs.go", "read_write.go", "register.go", diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index b3f981459..880956c75 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -16,32 +16,18 @@ package fuse import ( "errors" - "fmt" "sync" - "sync/atomic" - "syscall" - - "gvisor.dev/gvisor/tools/go_marshal/marshal" "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" "gvisor.dev/gvisor/pkg/log" "gvisor.dev/gvisor/pkg/marshal" "gvisor.dev/gvisor/pkg/sentry/kernel" - "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/waiter" ) -// maxActiveRequestsDefault is the default setting controlling the upper bound -// on the number of active requests at any given time. -const maxActiveRequestsDefault = 10000 - -// Ordinary requests have even IDs, while interrupts IDs are odd. -// Used to increment the unique ID for each FUSE request. -var reqIDStep uint64 = 2 - const ( // fuseDefaultMaxBackground is the default value for MaxBackground. fuseDefaultMaxBackground = 12 @@ -54,33 +40,11 @@ const ( fuseDefaultMaxPagesPerReq = 32 ) -// Request represents a FUSE operation request that hasn't been sent to the -// server yet. -// -// +stateify savable -type Request struct { - requestEntry - - id linux.FUSEOpID - hdr *linux.FUSEHeaderIn - data []byte - - // payload for this request: extra bytes to write after - // the data slice. Used by FUSE_WRITE. - payload []byte -} - -// Response represents an actual response from the server, including the -// response payload. -// -// +stateify savable -type Response struct { - opcode linux.FUSEOpcode - hdr linux.FUSEHeaderOut - data []byte -} - // connection is the struct by which the sentry communicates with the FUSE server daemon. +// Lock order: +// - conn.fd.mu +// - conn.mu +// - conn.asyncMu type connection struct { fd *DeviceFD @@ -115,10 +79,6 @@ type connection struct { // initializedChan is used to block requests before initialization. initializedChan chan struct{} - // blocked when there are too many outstading backgrounds requests (NumBackground == MaxBackground). - // TODO(gvisor.dev/issue/3185): update the numBackground accordingly; use a channel to block. - blocked bool - // connected (connection established) when a new FUSE file system is created. // Set to false when: // umount, @@ -126,51 +86,52 @@ type connection struct { // device release. connected bool - // aborted via sysfs. - // TODO(gvisor.dev/issue/3185): abort all queued requests. - aborted bool - - // atomicOTrunc is true when FUSE does not send a separate SETATTR request - // before open with O_TRUNC flag. - // Negotiated and only set in INIT. - atomicOTrunc bool - // connInitError if FUSE_INIT encountered error (major version mismatch). // Only set in INIT. connInitError bool // connInitSuccess if FUSE_INIT is successful. // Only set in INIT. - // Used for destory. + // Used for destory (not yet implemented). connInitSuccess bool - // TODO(gvisor.dev/issue/3185): All the queue logic are working in progress. - - // NumberBackground is the number of requests in the background. - numBackground uint16 + // aborted via sysfs, and will send ECONNABORTED to read after disconnection (instead of ENODEV). + // Set only if abortErr is true and via fuse control fs (not yet implemented). + // TODO(gvisor.dev/issue/3525): set this to true when user aborts. + aborted bool - // congestionThreshold for NumBackground. - // Negotiated in FUSE_INIT. - congestionThreshold uint16 + // numWating is the number of requests waiting to be + // sent to FUSE device or being processed by FUSE daemon. + numWaiting uint32 - // maxBackground is the maximum number of NumBackground. - // Block connection when it is reached. - // Negotiated in FUSE_INIT. - maxBackground uint16 + // Terminology note: + // + // - `asyncNumMax` is the `MaxBackground` in the FUSE_INIT_IN struct. + // + // - `asyncCongestionThreshold` is the `CongestionThreshold` in the FUSE_INIT_IN struct. + // + // We call the "background" requests in unix term as async requests. + // The "async requests" in unix term is our async requests that expect a reply, + // i.e. `!requestOptions.noReply` - // numActiveBackground is the number of requests in background and has being marked as active. - numActiveBackground uint16 + // asyncMu protects the async request fields. + asyncMu sync.Mutex - // numWating is the number of requests waiting for completion. - numWaiting uint32 + // asyncNum is the number of async requests. + // Protected by asyncMu. + asyncNum uint16 - // TODO(gvisor.dev/issue/3185): BgQueue - // some queue for background queued requests. + // asyncCongestionThreshold the number of async requests. + // Negotiated in FUSE_INIT as "CongestionThreshold". + // TODO(gvisor.dev/issue/3529): add congestion control. + // Protected by asyncMu. + asyncCongestionThreshold uint16 - // bgLock protects: - // MaxBackground, CongestionThreshold, NumBackground, - // NumActiveBackground, BgQueue, Blocked. - bgLock sync.Mutex + // asyncNumMax is the maximum number of asyncNum. + // Connection blocks the async requests when it is reached. + // Negotiated in FUSE_INIT as "MaxBackground". + // Protected by asyncMu. + asyncNumMax uint16 // maxRead is the maximum size of a read buffer in in bytes. // Initialized from a fuse fs parameter. @@ -188,6 +149,11 @@ type connection struct { // Negotiated and only set in INIT. minor uint32 + // atomicOTrunc is true when FUSE does not send a separate SETATTR request + // before open with O_TRUNC flag. + // Negotiated and only set in INIT. + atomicOTrunc bool + // asyncRead if read pages asynchronously. // Negotiated and only set in INIT. asyncRead bool @@ -225,63 +191,13 @@ func newFUSEConnection(_ context.Context, fd *vfs.FileDescription, opts *filesys fuseFD.writeCursor = 0 return &connection{ - fd: fuseFD, - maxBackground: fuseDefaultMaxBackground, - congestionThreshold: fuseDefaultCongestionThreshold, - maxRead: opts.maxRead, - maxPages: fuseDefaultMaxPagesPerReq, - initializedChan: make(chan struct{}), - connected: true, - }, nil -} - -// SetInitialized atomically sets the connection as initialized. -func (conn *connection) SetInitialized() { - // Unblock the requests sent before INIT. - close(conn.initializedChan) - - // Close the channel first to avoid the non-atomic situation - // where conn.initialized is true but there are - // tasks being blocked on the channel. - // And it prevents the newer tasks from gaining - // unnecessary higher chance to be issued before the blocked one. - - atomic.StoreInt32(&(conn.initialized), int32(1)) -} - -// IsInitialized atomically check if the connection is initialized. -// pairs with SetInitialized(). -func (conn *connection) Initialized() bool { - return atomic.LoadInt32(&(conn.initialized)) != 0 -} - -// NewRequest creates a new request that can be sent to the FUSE server. -func (conn *connection) NewRequest(creds *auth.Credentials, pid uint32, ino uint64, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*Request, error) { - conn.fd.mu.Lock() - defer conn.fd.mu.Unlock() - conn.fd.nextOpID += linux.FUSEOpID(reqIDStep) - - hdrLen := (*linux.FUSEHeaderIn)(nil).SizeBytes() - hdr := linux.FUSEHeaderIn{ - Len: uint32(hdrLen + payload.SizeBytes()), - Opcode: opcode, - Unique: conn.fd.nextOpID, - NodeID: ino, - UID: uint32(creds.EffectiveKUID), - GID: uint32(creds.EffectiveKGID), - PID: pid, - } - - buf := make([]byte, hdr.Len) - - // TODO(gVisor.dev/3698): Use the unsafe version once go_marshal is safe to use again. - hdr.MarshalBytes(buf[:hdrLen]) - payload.MarshalBytes(buf[hdrLen:]) - - return &Request{ - id: hdr.Unique, - hdr: &hdr, - data: buf, + fd: fuseFD, + asyncNumMax: fuseDefaultMaxBackground, + asyncCongestionThreshold: fuseDefaultCongestionThreshold, + maxRead: opts.maxRead, + maxPages: fuseDefaultMaxPagesPerReq, + initializedChan: make(chan struct{}), + connected: true, }, nil } @@ -337,43 +253,6 @@ func (conn *connection) call(t *kernel.Task, r *Request) (*Response, error) { return fut.resolve(t) } -// Error returns the error of the FUSE call. -func (r *Response) Error() error { - errno := r.hdr.Error - if errno >= 0 { - return nil - } - - sysErrNo := syscall.Errno(-errno) - return error(sysErrNo) -} - -// DataLen returns the size of the response without the header. -func (r *Response) DataLen() uint32 { - return r.hdr.Len - uint32(r.hdr.SizeBytes()) -} - -// UnmarshalPayload unmarshals the response data into m. -func (r *Response) UnmarshalPayload(m marshal.Marshallable) error { - hdrLen := r.hdr.SizeBytes() - haveDataLen := r.hdr.Len - uint32(hdrLen) - wantDataLen := uint32(m.SizeBytes()) - - if haveDataLen < wantDataLen { - return fmt.Errorf("payload too small. Minimum data lenth required: %d, but got data length %d", wantDataLen, haveDataLen) - } - - // The response data is empty unless there is some payload. And so, doesn't - // need to be unmarshalled. - if r.data == nil { - return nil - } - - // TODO(gVisor.dev/3698): Use the unsafe version once go_marshal is safe to use again. - m.UnmarshalBytes(r.data[hdrLen:]) - return nil -} - // callFuture makes a request to the server and returns a future response. // Call resolve() when the response needs to be fulfilled. func (conn *connection) callFuture(t *kernel.Task, r *Request) (*futureResponse, error) { @@ -422,50 +301,3 @@ func (conn *connection) callFutureLocked(t *kernel.Task, r *Request) (*futureRes return fut, nil } - -// futureResponse represents an in-flight request, that may or may not have -// completed yet. Convert it to a resolved Response by calling Resolve, but note -// that this may block. -// -// +stateify savable -type futureResponse struct { - opcode linux.FUSEOpcode - ch chan struct{} - hdr *linux.FUSEHeaderOut - data []byte -} - -// newFutureResponse creates a future response to a FUSE request. -func newFutureResponse(opcode linux.FUSEOpcode) *futureResponse { - return &futureResponse{ - opcode: opcode, - ch: make(chan struct{}), - } -} - -// resolve blocks the task until the server responds to its corresponding request, -// then returns a resolved response. -func (f *futureResponse) resolve(t *kernel.Task) (*Response, error) { - // If there is no Task associated with this request - then we don't try to resolve - // the response. Instead, the task writing the response (proxy to the server) will - // process the response on our behalf. - if t == nil { - log.Infof("fuse.Response.resolve: Not waiting on a response from server.") - return nil, nil - } - - if err := t.Block(f.ch); err != nil { - return nil, err - } - - return f.getResponse(), nil -} - -// getResponse creates a Response from the data the futureResponse has. -func (f *futureResponse) getResponse() *Response { - return &Response{ - opcode: f.opcode, - hdr: *f.hdr, - data: f.data, - } -} diff --git a/pkg/sentry/fsimpl/fuse/connection_control.go b/pkg/sentry/fsimpl/fuse/connection_control.go new file mode 100644 index 000000000..d84d9caf2 --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/connection_control.go @@ -0,0 +1,183 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fuse + +import ( + "sync/atomic" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" +) + +// consts used by FUSE_INIT negotiation. +const ( + // fuseMaxMaxPages is the maximum value for MaxPages received in InitOut. + // Follow the same behavior as unix fuse implementation. + fuseMaxMaxPages = 256 + + // Maximum value for the time granularity for file time stamps, 1s. + // Follow the same behavior as unix fuse implementation. + fuseMaxTimeGranNs = 1000000000 + + // Minimum value for MaxWrite and MaxRead. + // Follow the same behavior as unix fuse implementation. + fuseMinMaxWrite = 4096 + fuseMinMaxRead = 4096 + + // Temporary default value for max readahead, 128kb. + fuseDefaultMaxReadahead = 131072 + + // The FUSE_INIT_IN flags sent to the daemon. + // TODO(gvisor.dev/issue/3199): complete the flags. + fuseDefaultInitFlags = linux.FUSE_MAX_PAGES +) + +// Adjustable maximums for Connection's cogestion control parameters. +// Used as the upperbound of the config values. +// Currently we do not support adjustment to them. +var ( + MaxUserBackgroundRequest uint16 = fuseDefaultMaxBackground + MaxUserCongestionThreshold uint16 = fuseDefaultCongestionThreshold +) + +// SetInitialized atomically sets the connection as initialized. +func (conn *connection) SetInitialized() { + // Unblock the requests sent before INIT. + close(conn.initializedChan) + + // Close the channel first to avoid the non-atomic situation + // where conn.initialized is true but there are + // tasks being blocked on the channel. + // And it prevents the newer tasks from gaining + // unnecessary higher chance to be issued before the blocked one. + + atomic.StoreInt32(&(conn.initialized), int32(1)) +} + +// IsInitialized atomically check if the connection is initialized. +// pairs with SetInitialized(). +func (conn *connection) Initialized() bool { + return atomic.LoadInt32(&(conn.initialized)) != 0 +} + +// InitSend sends a FUSE_INIT request. +func (conn *connection) InitSend(creds *auth.Credentials, pid uint32) error { + in := linux.FUSEInitIn{ + Major: linux.FUSE_KERNEL_VERSION, + Minor: linux.FUSE_KERNEL_MINOR_VERSION, + // TODO(gvisor.dev/issue/3196): find appropriate way to calculate this + MaxReadahead: fuseDefaultMaxReadahead, + Flags: fuseDefaultInitFlags, + } + + req, err := conn.NewRequest(creds, pid, 0, linux.FUSE_INIT, &in) + if err != nil { + return err + } + + // Since there is no task to block on and FUSE_INIT is the request + // to unblock other requests, use nil. + return conn.CallAsync(nil, req) +} + +// InitRecv receives a FUSE_INIT reply and process it. +func (conn *connection) InitRecv(res *Response, hasSysAdminCap bool) error { + if err := res.Error(); err != nil { + return err + } + + initRes := fuseInitRes{initLen: res.DataLen()} + if err := res.UnmarshalPayload(&initRes); err != nil { + return err + } + + return conn.initProcessReply(&initRes.initOut, hasSysAdminCap) +} + +// Process the FUSE_INIT reply from the FUSE server. +// It tries to acquire the conn.asyncMu lock if minor version is newer than 13. +func (conn *connection) initProcessReply(out *linux.FUSEInitOut, hasSysAdminCap bool) error { + // No matter error or not, always set initialzied. + // to unblock the blocked requests. + defer conn.SetInitialized() + + // No support for old major fuse versions. + if out.Major != linux.FUSE_KERNEL_VERSION { + conn.connInitError = true + return nil + } + + // Start processing the reply. + conn.connInitSuccess = true + conn.minor = out.Minor + + // No support for negotiating MaxWrite before minor version 5. + if out.Minor >= 5 { + conn.maxWrite = out.MaxWrite + } else { + conn.maxWrite = fuseMinMaxWrite + } + if conn.maxWrite < fuseMinMaxWrite { + conn.maxWrite = fuseMinMaxWrite + } + + // No support for the following flags before minor version 6. + if out.Minor >= 6 { + conn.asyncRead = out.Flags&linux.FUSE_ASYNC_READ != 0 + conn.bigWrites = out.Flags&linux.FUSE_BIG_WRITES != 0 + conn.dontMask = out.Flags&linux.FUSE_DONT_MASK != 0 + conn.writebackCache = out.Flags&linux.FUSE_WRITEBACK_CACHE != 0 + + // TODO(gvisor.dev/issue/3195): figure out how to use TimeGran (0 < TimeGran <= fuseMaxTimeGranNs). + + if out.Flags&linux.FUSE_MAX_PAGES != 0 { + maxPages := out.MaxPages + if maxPages < 1 { + maxPages = 1 + } + if maxPages > fuseMaxMaxPages { + maxPages = fuseMaxMaxPages + } + conn.maxPages = maxPages + } + } + + // No support for limits before minor version 13. + if out.Minor >= 13 { + conn.asyncMu.Lock() + + if out.MaxBackground > 0 { + conn.asyncNumMax = out.MaxBackground + + if !hasSysAdminCap && + conn.asyncNumMax > MaxUserBackgroundRequest { + conn.asyncNumMax = MaxUserBackgroundRequest + } + } + + if out.CongestionThreshold > 0 { + conn.asyncCongestionThreshold = out.CongestionThreshold + + if !hasSysAdminCap && + conn.asyncCongestionThreshold > MaxUserCongestionThreshold { + conn.asyncCongestionThreshold = MaxUserCongestionThreshold + } + } + + conn.asyncMu.Unlock() + } + + return nil +} diff --git a/pkg/sentry/fsimpl/fuse/dev_test.go b/pkg/sentry/fsimpl/fuse/dev_test.go index c18921dd8..59928d75e 100644 --- a/pkg/sentry/fsimpl/fuse/dev_test.go +++ b/pkg/sentry/fsimpl/fuse/dev_test.go @@ -369,7 +369,7 @@ func newTestConnection(system *testutil.System, k *kernel.Kernel, maxActiveReque fsopts := filesystemOptions{ maxActiveRequests: maxActiveRequests, } - fs, err := NewFUSEFilesystem(system.Ctx, 0, &fsopts, &fuseDev.vfsfd) + fs, err := newFUSEFilesystem(system.Ctx, 0, &fsopts, &fuseDev.vfsfd) if err != nil { return nil, nil, err } diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 572245303..30725182a 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -35,6 +35,10 @@ import ( // Name is the default filesystem name. const Name = "fuse" +// maxActiveRequestsDefault is the default setting controlling the upper bound +// on the number of active requests at any given time. +const maxActiveRequestsDefault = 10000 + // FilesystemType implements vfs.FilesystemType. type FilesystemType struct{} @@ -168,12 +172,12 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt // Check for unparsed options. if len(mopts) != 0 { - log.Warningf("%s.GetFilesystem: unknown options: %v", fsType.Name(), mopts) + log.Warningf("%s.GetFilesystem: unsupported or unknown options: %v", fsType.Name(), mopts) return nil, nil, syserror.EINVAL } // Create a new FUSE filesystem. - fs, err := NewFUSEFilesystem(ctx, devMinor, &fsopts, fuseFd) + fs, err := newFUSEFilesystem(ctx, devMinor, &fsopts, fuseFd) if err != nil { log.Warningf("%s.NewFUSEFilesystem: failed with error: %v", fsType.Name(), err) return nil, nil, err @@ -194,21 +198,22 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt return fs.VFSFilesystem(), root.VFSDentry(), nil } -// NewFUSEFilesystem creates a new FUSE filesystem. -func NewFUSEFilesystem(ctx context.Context, devMinor uint32, opts *filesystemOptions, device *vfs.FileDescription) (*filesystem, error) { - fs := &filesystem{ - devMinor: devMinor, - opts: opts, - } - +// newFUSEFilesystem creates a new FUSE filesystem. +func newFUSEFilesystem(ctx context.Context, devMinor uint32, opts *filesystemOptions, device *vfs.FileDescription) (*filesystem, error) { conn, err := newFUSEConnection(ctx, device, opts) if err != nil { log.Warningf("fuse.NewFUSEFilesystem: NewFUSEConnection failed with error: %v", err) return nil, syserror.EINVAL } - fs.conn = conn fuseFD := device.Impl().(*DeviceFD) + + fs := &filesystem{ + devMinor: devMinor, + opts: opts, + conn: conn, + } + fuseFD.fs = fs return fs, nil diff --git a/pkg/sentry/fsimpl/fuse/init.go b/pkg/sentry/fsimpl/fuse/init.go deleted file mode 100644 index a47309b6e..000000000 --- a/pkg/sentry/fsimpl/fuse/init.go +++ /dev/null @@ -1,165 +0,0 @@ -// Copyright 2020 The gVisor Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package fuse - -import ( - "gvisor.dev/gvisor/pkg/abi/linux" - "gvisor.dev/gvisor/pkg/sentry/kernel/auth" -) - -// consts used by FUSE_INIT negotiation. -const ( - // fuseMaxMaxPages is the maximum value for MaxPages received in InitOut. - // Follow the same behavior as unix fuse implementation. - fuseMaxMaxPages = 256 - - // Maximum value for the time granularity for file time stamps, 1s. - // Follow the same behavior as unix fuse implementation. - fuseMaxTimeGranNs = 1000000000 - - // Minimum value for MaxWrite and MaxRead. - // Follow the same behavior as unix fuse implementation. - fuseMinMaxWrite = 4096 - fuseMinMaxRead = 4096 - - // Temporary default value for max readahead, 128kb. - fuseDefaultMaxReadahead = 131072 - - // The FUSE_INIT_IN flags sent to the daemon. - // TODO(gvisor.dev/issue/3199): complete the flags. - fuseDefaultInitFlags = linux.FUSE_MAX_PAGES -) - -// Adjustable maximums for Connection's cogestion control parameters. -// Used as the upperbound of the config values. -// Currently we do not support adjustment to them. -var ( - MaxUserBackgroundRequest uint16 = fuseDefaultMaxBackground - MaxUserCongestionThreshold uint16 = fuseDefaultCongestionThreshold -) - -// InitSend sends a FUSE_INIT request. -func (conn *connection) InitSend(creds *auth.Credentials, pid uint32) error { - in := linux.FUSEInitIn{ - Major: linux.FUSE_KERNEL_VERSION, - Minor: linux.FUSE_KERNEL_MINOR_VERSION, - // TODO(gvisor.dev/issue/3196): find appropriate way to calculate this - MaxReadahead: fuseDefaultMaxReadahead, - Flags: fuseDefaultInitFlags, - } - - req, err := conn.NewRequest(creds, pid, 0, linux.FUSE_INIT, &in) - if err != nil { - return err - } - - // Since there is no task to block on and FUSE_INIT is the request - // to unblock other requests, use nil. - return conn.CallAsync(nil, req) -} - -// InitRecv receives a FUSE_INIT reply and process it. -func (conn *connection) InitRecv(res *Response, hasSysAdminCap bool) error { - if err := res.Error(); err != nil { - return err - } - - initRes := fuseInitRes{initLen: res.DataLen()} - if err := res.UnmarshalPayload(&initRes); err != nil { - return err - } - - return conn.initProcessReply(&initRes.initOut, hasSysAdminCap) -} - -// Process the FUSE_INIT reply from the FUSE server. -func (conn *connection) initProcessReply(out *linux.FUSEInitOut, hasSysAdminCap bool) error { - // No support for old major fuse versions. - if out.Major != linux.FUSE_KERNEL_VERSION { - conn.connInitError = true - - // Set the connection as initialized and unblock the blocked requests - // (i.e. return error for them). - conn.SetInitialized() - - return nil - } - - // Start processing the reply. - conn.connInitSuccess = true - conn.minor = out.Minor - - // No support for limits before minor version 13. - if out.Minor >= 13 { - conn.bgLock.Lock() - - if out.MaxBackground > 0 { - conn.maxBackground = out.MaxBackground - - if !hasSysAdminCap && - conn.maxBackground > MaxUserBackgroundRequest { - conn.maxBackground = MaxUserBackgroundRequest - } - } - - if out.CongestionThreshold > 0 { - conn.congestionThreshold = out.CongestionThreshold - - if !hasSysAdminCap && - conn.congestionThreshold > MaxUserCongestionThreshold { - conn.congestionThreshold = MaxUserCongestionThreshold - } - } - - conn.bgLock.Unlock() - } - - // No support for the following flags before minor version 6. - if out.Minor >= 6 { - conn.asyncRead = out.Flags&linux.FUSE_ASYNC_READ != 0 - conn.bigWrites = out.Flags&linux.FUSE_BIG_WRITES != 0 - conn.dontMask = out.Flags&linux.FUSE_DONT_MASK != 0 - conn.writebackCache = out.Flags&linux.FUSE_WRITEBACK_CACHE != 0 - - // TODO(gvisor.dev/issue/3195): figure out how to use TimeGran (0 < TimeGran <= fuseMaxTimeGranNs). - - if out.Flags&linux.FUSE_MAX_PAGES != 0 { - maxPages := out.MaxPages - if maxPages < 1 { - maxPages = 1 - } - if maxPages > fuseMaxMaxPages { - maxPages = fuseMaxMaxPages - } - conn.maxPages = maxPages - } - } - - // No support for negotiating MaxWrite before minor version 5. - if out.Minor >= 5 { - conn.maxWrite = out.MaxWrite - } else { - conn.maxWrite = fuseMinMaxWrite - } - if conn.maxWrite < fuseMinMaxWrite { - conn.maxWrite = fuseMinMaxWrite - } - - // Set connection as initialized and unblock the requests - // issued before init. - conn.SetInitialized() - - return nil -} diff --git a/pkg/sentry/fsimpl/fuse/request_response.go b/pkg/sentry/fsimpl/fuse/request_response.go index a69b21221..648eaf263 100644 --- a/pkg/sentry/fsimpl/fuse/request_response.go +++ b/pkg/sentry/fsimpl/fuse/request_response.go @@ -15,7 +15,13 @@ package fuse import ( + "fmt" + "syscall" + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/log" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/usermem" "gvisor.dev/gvisor/tools/go_marshal/marshal" ) @@ -71,3 +77,147 @@ func (r *fuseInitRes) UnmarshalBytes(src []byte) { func (r *fuseInitRes) SizeBytes() int { return int(r.initLen) } + +// Ordinary requests have even IDs, while interrupts IDs are odd. +// Used to increment the unique ID for each FUSE request. +var reqIDStep uint64 = 2 + +// Request represents a FUSE operation request that hasn't been sent to the +// server yet. +// +// +stateify savable +type Request struct { + requestEntry + + id linux.FUSEOpID + hdr *linux.FUSEHeaderIn + data []byte + + // payload for this request: extra bytes to write after + // the data slice. Used by FUSE_WRITE. + payload []byte +} + +// NewRequest creates a new request that can be sent to the FUSE server. +func (conn *connection) NewRequest(creds *auth.Credentials, pid uint32, ino uint64, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*Request, error) { + conn.fd.mu.Lock() + defer conn.fd.mu.Unlock() + conn.fd.nextOpID += linux.FUSEOpID(reqIDStep) + + hdrLen := (*linux.FUSEHeaderIn)(nil).SizeBytes() + hdr := linux.FUSEHeaderIn{ + Len: uint32(hdrLen + payload.SizeBytes()), + Opcode: opcode, + Unique: conn.fd.nextOpID, + NodeID: ino, + UID: uint32(creds.EffectiveKUID), + GID: uint32(creds.EffectiveKGID), + PID: pid, + } + + buf := make([]byte, hdr.Len) + + // TODO(gVisor.dev/3698): Use the unsafe version once go_marshal is safe to use again. + hdr.MarshalBytes(buf[:hdrLen]) + payload.MarshalBytes(buf[hdrLen:]) + + return &Request{ + id: hdr.Unique, + hdr: &hdr, + data: buf, + }, nil +} + +// futureResponse represents an in-flight request, that may or may not have +// completed yet. Convert it to a resolved Response by calling Resolve, but note +// that this may block. +// +// +stateify savable +type futureResponse struct { + opcode linux.FUSEOpcode + ch chan struct{} + hdr *linux.FUSEHeaderOut + data []byte +} + +// newFutureResponse creates a future response to a FUSE request. +func newFutureResponse(opcode linux.FUSEOpcode) *futureResponse { + return &futureResponse{ + opcode: opcode, + ch: make(chan struct{}), + } +} + +// resolve blocks the task until the server responds to its corresponding request, +// then returns a resolved response. +func (f *futureResponse) resolve(t *kernel.Task) (*Response, error) { + // If there is no Task associated with this request - then we don't try to resolve + // the response. Instead, the task writing the response (proxy to the server) will + // process the response on our behalf. + if t == nil { + log.Infof("fuse.Response.resolve: Not waiting on a response from server.") + return nil, nil + } + + if err := t.Block(f.ch); err != nil { + return nil, err + } + + return f.getResponse(), nil +} + +// getResponse creates a Response from the data the futureResponse has. +func (f *futureResponse) getResponse() *Response { + return &Response{ + opcode: f.opcode, + hdr: *f.hdr, + data: f.data, + } +} + +// Response represents an actual response from the server, including the +// response payload. +// +// +stateify savable +type Response struct { + opcode linux.FUSEOpcode + hdr linux.FUSEHeaderOut + data []byte +} + +// Error returns the error of the FUSE call. +func (r *Response) Error() error { + errno := r.hdr.Error + if errno >= 0 { + return nil + } + + sysErrNo := syscall.Errno(-errno) + return error(sysErrNo) +} + +// DataLen returns the size of the response without the header. +func (r *Response) DataLen() uint32 { + return r.hdr.Len - uint32(r.hdr.SizeBytes()) +} + +// UnmarshalPayload unmarshals the response data into m. +func (r *Response) UnmarshalPayload(m marshal.Marshallable) error { + hdrLen := r.hdr.SizeBytes() + haveDataLen := r.hdr.Len - uint32(hdrLen) + wantDataLen := uint32(m.SizeBytes()) + + if haveDataLen < wantDataLen { + return fmt.Errorf("payload too small. Minimum data lenth required: %d, but got data length %d", wantDataLen, haveDataLen) + } + + // The response data is empty unless there is some payload. And so, doesn't + // need to be unmarshalled. + if r.data == nil { + return nil + } + + // TODO(gVisor.dev/3698): Use the unsafe version once go_marshal is safe to use again. + m.UnmarshalBytes(r.data[hdrLen:]) + return nil +} -- cgit v1.2.3 From 826a685a95bec32286688035922a56faf622e87e Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Thu, 3 Sep 2020 19:16:17 +0000 Subject: Improve FUSE async/noreply call logic This change adds bookkeeping variables for the FUSE request. With them, old insecure confusing code we used to process async requests is replaced by new clear compiling ones. Future code can take advantage of them to have better control of each requests. --- pkg/sentry/fsimpl/fuse/connection.go | 62 +++++++++++++----------------- pkg/sentry/fsimpl/fuse/dev.go | 31 ++++++--------- pkg/sentry/fsimpl/fuse/request_response.go | 22 +++++++---- 3 files changed, 53 insertions(+), 62 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 880956c75..72eac4179 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -15,7 +15,6 @@ package fuse import ( - "errors" "sync" "gvisor.dev/gvisor/pkg/abi/linux" @@ -201,42 +200,40 @@ func newFUSEConnection(_ context.Context, fd *vfs.FileDescription, opts *filesys }, nil } -// Call makes a request to the server and blocks the invoking task until a -// server responds with a response. Task should never be nil. -// Requests will not be sent before the connection is initialized. -// For async tasks, use CallAsync(). -func (conn *connection) Call(t *kernel.Task, r *Request) (*Response, error) { - // Block requests sent before connection is initalized. - if !conn.Initialized() { - if err := t.Block(conn.initializedChan); err != nil { - return nil, err - } - } - - return conn.call(t, r) +// CallAsync makes an async (aka background) request. +// It's a simple wrapper around Call(). +func (conn *connection) CallAsync(t *kernel.Task, r *Request) error { + r.async = true + _, err := conn.Call(t, r) + return err } -// CallAsync makes an async (aka background) request. -// Those requests either do not expect a response (e.g. release) or -// the response should be handled by others (e.g. init). -// Return immediately unless the connection is blocked (before initialization). -// Async call example: init, release, forget, aio, interrupt. +// Call makes a request to the server. +// Block before the connection is initialized. // When the Request is FUSE_INIT, it will not be blocked before initialization. -func (conn *connection) CallAsync(t *kernel.Task, r *Request) error { +// Task should never be nil. +// +// For a sync request, it blocks the invoking task until +// a server responds with a response. +// +// For an async request (that do not expect a response immediately), +// it returns directly unless being blocked either before initialization +// or when there are too many async requests ongoing. +// +// Example for async request: +// init, readahead, write, async read/write, fuse_notify_reply, +// non-sync release, interrupt, forget. +// +// The forget request does not have a reply, +// as documented in include/uapi/linux/fuse.h:FUSE_FORGET. +func (conn *connection) Call(t *kernel.Task, r *Request) (*Response, error) { // Block requests sent before connection is initalized. if !conn.Initialized() && r.hdr.Opcode != linux.FUSE_INIT { if err := t.Block(conn.initializedChan); err != nil { - return err + return nil, err } } - // This should be the only place that invokes call() with a nil task. - _, err := conn.call(nil, r) - return err -} - -// call makes a call without blocking checks. -func (conn *connection) call(t *kernel.Task, r *Request) (*Response, error) { if !conn.connected { return nil, syserror.ENOTCONN } @@ -271,11 +268,6 @@ func (conn *connection) callFuture(t *kernel.Task, r *Request) (*futureResponse, // if there are always too many ongoing requests all the time. The // supported maxActiveRequests setting should be really high to avoid this. for conn.fd.numActiveRequests == conn.fd.fs.opts.maxActiveRequests { - if t == nil { - // Since there is no task that is waiting. We must error out. - return nil, errors.New("FUSE request queue full") - } - log.Infof("Blocking request %v from being queued. Too many active requests: %v", r.id, conn.fd.numActiveRequests) conn.fd.mu.Unlock() @@ -292,8 +284,8 @@ func (conn *connection) callFuture(t *kernel.Task, r *Request) (*futureResponse, // callFutureLocked makes a request to the server and returns a future response. func (conn *connection) callFutureLocked(t *kernel.Task, r *Request) (*futureResponse, error) { conn.fd.queue.PushBack(r) - conn.fd.numActiveRequests += 1 - fut := newFutureResponse(r.hdr.Opcode) + conn.fd.numActiveRequests++ + fut := newFutureResponse(r) conn.fd.completions[r.id] = fut // Signal the readers that there is something to read. diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index 6022593d6..e7296c189 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -367,22 +367,20 @@ func (fd *DeviceFD) Seek(ctx context.Context, offset int64, whence int32) (int64 // sendResponse sends a response to the waiting task (if any). func (fd *DeviceFD) sendResponse(ctx context.Context, fut *futureResponse) error { - // See if the running task need to perform some action before returning. - // Since we just finished writing the future, we can be sure that - // getResponse generates a populated response. - if err := fd.noReceiverAction(ctx, fut.getResponse()); err != nil { - return err - } + // Signal the task waiting on a response if any. + defer close(fut.ch) // Signal that the queue is no longer full. select { case fd.fullQueueCh <- struct{}{}: default: } - fd.numActiveRequests -= 1 + fd.numActiveRequests-- + + if fut.async { + return fd.asyncCallBack(ctx, fut.getResponse()) + } - // Signal the task waiting on a response. - close(fut.ch) return nil } @@ -404,23 +402,18 @@ func (fd *DeviceFD) sendError(ctx context.Context, errno int32, req *Request) er delete(fd.completions, respHdr.Unique) fut.hdr = &respHdr - if err := fd.sendResponse(ctx, fut); err != nil { - return err - } - - return nil + return fd.sendResponse(ctx, fut) } -// noReceiverAction has the calling kernel.Task do some action if its known that no -// receiver is going to be waiting on the future channel. This is to be used by: -// FUSE_INIT. -func (fd *DeviceFD) noReceiverAction(ctx context.Context, r *Response) error { +// asyncCallBack executes pre-defined callback function for async requests. +// Currently used by: FUSE_INIT. +func (fd *DeviceFD) asyncCallBack(ctx context.Context, r *Response) error { switch r.opcode { case linux.FUSE_INIT: creds := auth.CredentialsFromContext(ctx) rootUserNs := kernel.KernelFromContext(ctx).RootUserNamespace() return fd.fs.conn.InitRecv(r, creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, rootUserNs)) - // TODO(gvisor.dev/issue/3247): support async read: correctly process the response using information from r.options. + // TODO(gvisor.dev/issue/3247): support async read: correctly process the response. } return nil diff --git a/pkg/sentry/fsimpl/fuse/request_response.go b/pkg/sentry/fsimpl/fuse/request_response.go index 648eaf263..fd9a96197 100644 --- a/pkg/sentry/fsimpl/fuse/request_response.go +++ b/pkg/sentry/fsimpl/fuse/request_response.go @@ -19,7 +19,6 @@ import ( "syscall" "gvisor.dev/gvisor/pkg/abi/linux" - "gvisor.dev/gvisor/pkg/log" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/usermem" @@ -96,6 +95,12 @@ type Request struct { // payload for this request: extra bytes to write after // the data slice. Used by FUSE_WRITE. payload []byte + + // If this request is async. + async bool + // If we don't care its response. + // Manually set by the caller. + noReply bool } // NewRequest creates a new request that can be sent to the FUSE server. @@ -138,24 +143,25 @@ type futureResponse struct { ch chan struct{} hdr *linux.FUSEHeaderOut data []byte + + // If this request is async. + async bool } // newFutureResponse creates a future response to a FUSE request. -func newFutureResponse(opcode linux.FUSEOpcode) *futureResponse { +func newFutureResponse(req *Request) *futureResponse { return &futureResponse{ - opcode: opcode, + opcode: req.hdr.Opcode, ch: make(chan struct{}), + async: req.async, } } // resolve blocks the task until the server responds to its corresponding request, // then returns a resolved response. func (f *futureResponse) resolve(t *kernel.Task) (*Response, error) { - // If there is no Task associated with this request - then we don't try to resolve - // the response. Instead, the task writing the response (proxy to the server) will - // process the response on our behalf. - if t == nil { - log.Infof("fuse.Response.resolve: Not waiting on a response from server.") + // Return directly for async requests. + if f.async { return nil, nil } -- cgit v1.2.3 From 4edc56d3e99b161f2f3311bc22a51012bf0a90ee Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Thu, 3 Sep 2020 19:22:24 +0000 Subject: Fix FUSE_RELEASE protocol reply processing This commit fixes the potential unexpected errors of original handling of FUSE_RELEASE responses while keep the same behavior (ignoring any reply). --- pkg/sentry/fsimpl/fuse/dev.go | 9 ++++++++- pkg/sentry/fsimpl/fuse/file.go | 7 ++++++- 2 files changed, 14 insertions(+), 2 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index e7296c189..6f0b896cb 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -204,8 +204,11 @@ func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts // Fully done with this req, remove it from the queue. fd.queue.Remove(req) - if req.hdr.Opcode == linux.FUSE_RELEASE { + + // Remove noReply ones from map of requests expecting a reply. + if req.noReply { fd.numActiveRequests -= 1 + delete(fd.completions, req.hdr.Unique) } return int64(n), nil @@ -296,6 +299,10 @@ func (fd *DeviceFD) writeLocked(ctx context.Context, src usermem.IOSequence, opt fut, ok := fd.completions[hdr.Unique] if !ok { + if fut.hdr.Unique == linux.FUSE_RELEASE { + // Currently we simply discard the reply for FUSE_RELEASE. + return n + src.NumBytes(), nil + } // Server sent us a response for a request we never sent? return 0, syserror.EINVAL } diff --git a/pkg/sentry/fsimpl/fuse/file.go b/pkg/sentry/fsimpl/fuse/file.go index 15c0e3f41..b98145ba2 100644 --- a/pkg/sentry/fsimpl/fuse/file.go +++ b/pkg/sentry/fsimpl/fuse/file.go @@ -84,7 +84,12 @@ func (fd *fileDescription) Release(ctx context.Context) { } kernelTask := kernel.TaskFromContext(ctx) // ignoring errors and FUSE server reply is analogous to Linux's behavior. - req, _ := conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), fd.inode().NodeID, opcode, &in) + req, err := conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), fd.inode().NodeID, opcode, &in) + if err != nil { + // No way to invoke Call() with an errored request. + return + } + req.noReply = true conn.CallAsync(kernelTask, req) } -- cgit v1.2.3 From d459bb3372384a4d16fe0d9791847285449dc184 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Wed, 9 Sep 2020 16:22:33 -0700 Subject: Add FUSE umount support This change implements Release for the FUSE filesystem and expected behaviors of the FUSE devices. It includes several checks for aborted connection in the path for making a request and a function to abort all the ongoing FUSE requests in order. --- pkg/sentry/fsimpl/fuse/BUILD | 6 +- pkg/sentry/fsimpl/fuse/connection.go | 11 ++- pkg/sentry/fsimpl/fuse/connection_control.go | 62 +++++++++++++ pkg/sentry/fsimpl/fuse/connection_test.go | 127 +++++++++++++++++++++++++++ pkg/sentry/fsimpl/fuse/dev.go | 51 +++++++---- pkg/sentry/fsimpl/fuse/dev_test.go | 4 - pkg/sentry/fsimpl/fuse/fusefs.go | 12 +++ pkg/sentry/fsimpl/fuse/utils_test.go | 111 +++++++++++++++++++++++ pkg/syserror/syserror.go | 1 + 9 files changed, 363 insertions(+), 22 deletions(-) create mode 100644 pkg/sentry/fsimpl/fuse/connection_test.go create mode 100644 pkg/sentry/fsimpl/fuse/utils_test.go (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index 5b60b1277..045d7ab08 100644 --- a/pkg/sentry/fsimpl/fuse/BUILD +++ b/pkg/sentry/fsimpl/fuse/BUILD @@ -66,7 +66,11 @@ go_library( go_test( name = "fuse_test", size = "small", - srcs = ["dev_test.go"], + srcs = [ + "connection_test.go", + "dev_test.go", + "utils_test.go", + ], library = ":fuse", deps = [ "//pkg/abi/linux", diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 72eac4179..5960d9368 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -180,7 +180,6 @@ func newFUSEConnection(_ context.Context, fd *vfs.FileDescription, opts *filesys // Mark the device as ready so it can be used. /dev/fuse can only be used if the FD was used to // mount a FUSE filesystem. fuseFD := fd.Impl().(*DeviceFD) - fuseFD.mounted = true // Create the writeBuf for the header to be stored in. hdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) @@ -283,6 +282,16 @@ func (conn *connection) callFuture(t *kernel.Task, r *Request) (*futureResponse, // callFutureLocked makes a request to the server and returns a future response. func (conn *connection) callFutureLocked(t *kernel.Task, r *Request) (*futureResponse, error) { + // Check connected again holding conn.mu. + conn.mu.Lock() + if !conn.connected { + conn.mu.Unlock() + // we checked connected before, + // this must be due to aborted connection. + return nil, syserror.ECONNABORTED + } + conn.mu.Unlock() + conn.fd.queue.PushBack(r) conn.fd.numActiveRequests++ fut := newFutureResponse(r) diff --git a/pkg/sentry/fsimpl/fuse/connection_control.go b/pkg/sentry/fsimpl/fuse/connection_control.go index d84d9caf2..a63c66e7c 100644 --- a/pkg/sentry/fsimpl/fuse/connection_control.go +++ b/pkg/sentry/fsimpl/fuse/connection_control.go @@ -16,8 +16,10 @@ package fuse import ( "sync/atomic" + "syscall" "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" ) @@ -181,3 +183,63 @@ func (conn *connection) initProcessReply(out *linux.FUSEInitOut, hasSysAdminCap return nil } + +// Abort this FUSE connection. +// It tries to acquire conn.fd.mu, conn.lock, conn.bgLock in order. +// All possible requests waiting or blocking will be aborted. +func (conn *connection) Abort(ctx context.Context) { + conn.fd.mu.Lock() + conn.mu.Lock() + conn.asyncMu.Lock() + + if !conn.connected { + conn.asyncMu.Unlock() + conn.mu.Unlock() + conn.fd.mu.Unlock() + return + } + + conn.connected = false + + // Empty the `fd.queue` that holds the requests + // not yet read by the FUSE daemon yet. + // These are a subset of the requests in `fuse.completion` map. + for !conn.fd.queue.Empty() { + req := conn.fd.queue.Front() + conn.fd.queue.Remove(req) + } + + var terminate []linux.FUSEOpID + + // 2. Collect the requests have not been sent to FUSE daemon, + // or have not received a reply. + for unique := range conn.fd.completions { + terminate = append(terminate, unique) + } + + // Release all locks to avoid deadlock. + conn.asyncMu.Unlock() + conn.mu.Unlock() + conn.fd.mu.Unlock() + + // 1. The requets blocked before initialization. + // Will reach call() `connected` check and return. + if !conn.Initialized() { + conn.SetInitialized() + } + + // 2. Terminate the requests collected above. + // Set ECONNABORTED error. + // sendError() will remove them from `fd.completion` map. + // Will enter the path of a normally received error. + for _, toTerminate := range terminate { + conn.fd.sendError(ctx, -int32(syscall.ECONNABORTED), toTerminate) + } + + // 3. The requests not yet written to FUSE device. + // Early terminate. + // Will reach callFutureLocked() `connected` check and return. + close(conn.fd.fullQueueCh) + + // TODO(gvisor.dev/issue/3528): Forget all pending forget reqs. +} diff --git a/pkg/sentry/fsimpl/fuse/connection_test.go b/pkg/sentry/fsimpl/fuse/connection_test.go new file mode 100644 index 000000000..6aa77b36d --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/connection_test.go @@ -0,0 +1,127 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fuse + +import ( + "math/rand" + "syscall" + "testing" + + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/syserror" +) + +// TestConnectionInitBlock tests if initialization +// correctly blocks and unblocks the connection. +// Since it's unfeasible to test kernelTask.Block() in unit test, +// the code in Call() are not tested here. +func TestConnectionInitBlock(t *testing.T) { + s := setup(t) + defer s.Destroy() + + k := kernel.KernelFromContext(s.Ctx) + + conn, _, err := newTestConnection(s, k, maxActiveRequestsDefault) + if err != nil { + t.Fatalf("newTestConnection: %v", err) + } + + select { + case <-conn.initializedChan: + t.Fatalf("initializedChan should be blocking before SetInitialized") + default: + } + + conn.SetInitialized() + + select { + case <-conn.initializedChan: + default: + t.Fatalf("initializedChan should not be blocking after SetInitialized") + } +} + +func TestConnectionAbort(t *testing.T) { + s := setup(t) + defer s.Destroy() + + k := kernel.KernelFromContext(s.Ctx) + creds := auth.CredentialsFromContext(s.Ctx) + task := kernel.TaskFromContext(s.Ctx) + + const maxActiveRequest uint64 = 10 + + conn, _, err := newTestConnection(s, k, maxActiveRequest) + if err != nil { + t.Fatalf("newTestConnection: %v", err) + } + + testObj := &testPayload{ + data: rand.Uint32(), + } + + var futNormal []*futureResponse + + for i := 0; i < 2*int(maxActiveRequest); i++ { + req, err := conn.NewRequest(creds, uint32(i), uint64(i), 0, testObj) + if err != nil { + t.Fatalf("NewRequest creation failed: %v", err) + } + if i < int(maxActiveRequest) { + // Issue the requests that will not be blocked due to maxActiveRequest. + fut, err := conn.callFutureLocked(task, req) + if err != nil { + t.Fatalf("callFutureLocked failed: %v", err) + } + futNormal = append(futNormal, fut) + } else { + go func(t *testing.T) { + // The requests beyond maxActiveRequest will be blocked and receive expected errors. + _, err := conn.callFutureLocked(task, req) + if err != syserror.ECONNABORTED && err != syserror.ENOTCONN { + t.Fatalf("Incorrect error code received for blocked callFutureLocked() on aborted connection") + } + }(t) + } + } + + conn.Abort(s.Ctx) + + // Abort should unblock the initialization channel. + select { + case <-conn.initializedChan: + default: + t.Fatalf("initializedChan should not be blocking after SetInitialized") + } + + // Abort will return ECONNABORTED error to unblocked requests. + for _, fut := range futNormal { + if fut.getResponse().hdr.Error != -int32(syscall.ECONNABORTED) { + t.Fatalf("Incorrect error code received for aborted connection") + } + } + + // After abort, Call() should return directly with ENOTCONN. + req, err := conn.NewRequest(creds, 0, 0, 0, testObj) + if err != nil { + t.Fatalf("NewRequest creation failed: %v", err) + } + _, err = conn.Call(task, req) + if err != syserror.ENOTCONN { + t.Fatalf("Incorrect error code received for Call() after connection aborted") + } + +} diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index 6f0b896cb..64c3e32e2 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -55,9 +55,6 @@ type DeviceFD struct { vfs.DentryMetadataFileDescriptionImpl vfs.NoLockFD - // mounted specifies whether a FUSE filesystem was mounted using the DeviceFD. - mounted bool - // nextOpID is used to create new requests. nextOpID linux.FUSEOpID @@ -99,13 +96,15 @@ type DeviceFD struct { // Release implements vfs.FileDescriptionImpl.Release. func (fd *DeviceFD) Release(context.Context) { - fd.fs.conn.connected = false + if fd.fs != nil { + fd.fs.conn.connected = false + } } // PRead implements vfs.FileDescriptionImpl.PRead. func (fd *DeviceFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) { // Operations on /dev/fuse don't make sense until a FUSE filesystem is mounted. - if !fd.mounted { + if fd.fs == nil { return 0, syserror.EPERM } @@ -115,10 +114,16 @@ func (fd *DeviceFD) PRead(ctx context.Context, dst usermem.IOSequence, offset in // Read implements vfs.FileDescriptionImpl.Read. func (fd *DeviceFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { // Operations on /dev/fuse don't make sense until a FUSE filesystem is mounted. - if !fd.mounted { + if fd.fs == nil { return 0, syserror.EPERM } + // Return ENODEV if the filesystem is umounted. + if fd.fs.umounted { + // TODO(gvisor.dev/issue/3525): return ECONNABORTED if aborted via fuse control fs. + return 0, syserror.ENODEV + } + // We require that any Read done on this filesystem have a sane minimum // read buffer. It must have the capacity for the fixed parts of any request // header (Linux uses the request header and the FUSEWriteIn header for this @@ -165,7 +170,7 @@ func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts } // Return the error to the calling task. - if err := fd.sendError(ctx, errno, req); err != nil { + if err := fd.sendError(ctx, errno, req.hdr.Unique); err != nil { return 0, err } @@ -217,7 +222,7 @@ func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts // PWrite implements vfs.FileDescriptionImpl.PWrite. func (fd *DeviceFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) { // Operations on /dev/fuse don't make sense until a FUSE filesystem is mounted. - if !fd.mounted { + if fd.fs == nil { return 0, syserror.EPERM } @@ -234,10 +239,15 @@ func (fd *DeviceFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs. // writeLocked implements writing to the fuse device while locked with DeviceFD.mu. func (fd *DeviceFD) writeLocked(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { // Operations on /dev/fuse don't make sense until a FUSE filesystem is mounted. - if !fd.mounted { + if fd.fs == nil { return 0, syserror.EPERM } + // Return ENODEV if the filesystem is umounted. + if fd.fs.umounted { + return 0, syserror.ENODEV + } + var cn, n int64 hdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) @@ -303,7 +313,8 @@ func (fd *DeviceFD) writeLocked(ctx context.Context, src usermem.IOSequence, opt // Currently we simply discard the reply for FUSE_RELEASE. return n + src.NumBytes(), nil } - // Server sent us a response for a request we never sent? + // Server sent us a response for a request we never sent, + // or for which we already received a reply (e.g. aborted), an unlikely event. return 0, syserror.EINVAL } @@ -343,7 +354,14 @@ func (fd *DeviceFD) Readiness(mask waiter.EventMask) waiter.EventMask { // locked with DeviceFD.mu. func (fd *DeviceFD) readinessLocked(mask waiter.EventMask) waiter.EventMask { var ready waiter.EventMask - ready |= waiter.EventOut // FD is always writable + + if fd.fs.umounted { + ready |= waiter.EventErr + return ready & mask + } + + // FD is always writable. + ready |= waiter.EventOut if !fd.queue.Empty() { // Have reqs available, FD is readable. ready |= waiter.EventIn @@ -365,7 +383,7 @@ func (fd *DeviceFD) EventUnregister(e *waiter.Entry) { // Seek implements vfs.FileDescriptionImpl.Seek. func (fd *DeviceFD) Seek(ctx context.Context, offset int64, whence int32) (int64, error) { // Operations on /dev/fuse don't make sense until a FUSE filesystem is mounted. - if !fd.mounted { + if fd.fs == nil { return 0, syserror.EPERM } @@ -391,19 +409,20 @@ func (fd *DeviceFD) sendResponse(ctx context.Context, fut *futureResponse) error return nil } -// sendError sends an error response to the waiting task (if any). -func (fd *DeviceFD) sendError(ctx context.Context, errno int32, req *Request) error { +// sendError sends an error response to the waiting task (if any) by calling sendResponse(). +func (fd *DeviceFD) sendError(ctx context.Context, errno int32, unique linux.FUSEOpID) error { // Return the error to the calling task. outHdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) respHdr := linux.FUSEHeaderOut{ Len: outHdrLen, Error: errno, - Unique: req.hdr.Unique, + Unique: unique, } fut, ok := fd.completions[respHdr.Unique] if !ok { - // Server sent us a response for a request we never sent? + // A response for a request we never sent, + // or for which we already received a reply (e.g. aborted). return syserror.EINVAL } delete(fd.completions, respHdr.Unique) diff --git a/pkg/sentry/fsimpl/fuse/dev_test.go b/pkg/sentry/fsimpl/fuse/dev_test.go index 59928d75e..5abdf82d3 100644 --- a/pkg/sentry/fsimpl/fuse/dev_test.go +++ b/pkg/sentry/fsimpl/fuse/dev_test.go @@ -35,10 +35,6 @@ import ( // will simply echo the payload back with the appropriate headers. const echoTestOpcode linux.FUSEOpcode = 1000 -type testPayload struct { - data uint32 -} - // TestFUSECommunication tests that the communication layer between the Sentry and the // FUSE server daemon works as expected. func TestFUSECommunication(t *testing.T) { diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 30725182a..f4ed73c12 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -29,6 +29,7 @@ import ( "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/waiter" "gvisor.dev/gvisor/tools/go_marshal/marshal" ) @@ -82,6 +83,9 @@ type filesystem struct { // opts is the options the fusefs is initialized with. opts *filesystemOptions + + // umounted is true if filesystem.Release() has been called. + umounted bool } // Name implements vfs.FilesystemType.Name. @@ -221,6 +225,14 @@ func newFUSEFilesystem(ctx context.Context, devMinor uint32, opts *filesystemOpt // Release implements vfs.FilesystemImpl.Release. func (fs *filesystem) Release(ctx context.Context) { + fs.umounted = true + fs.conn.Abort(ctx) + + fs.conn.fd.mu.Lock() + // Notify all the waiters on this fd. + fs.conn.fd.waitQueue.Notify(waiter.EventIn) + fs.conn.fd.mu.Unlock() + fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor) fs.Filesystem.Release(ctx) } diff --git a/pkg/sentry/fsimpl/fuse/utils_test.go b/pkg/sentry/fsimpl/fuse/utils_test.go new file mode 100644 index 000000000..436322830 --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/utils_test.go @@ -0,0 +1,111 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package fuse + +import ( + "testing" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/testutil" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/usermem" + "gvisor.dev/gvisor/tools/go_marshal/marshal" +) + +func setup(t *testing.T) *testutil.System { + k, err := testutil.Boot() + if err != nil { + t.Fatalf("Error creating kernel: %v", err) + } + + ctx := k.SupervisorContext() + creds := auth.CredentialsFromContext(ctx) + + k.VFS().MustRegisterFilesystemType(Name, &FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ + AllowUserList: true, + AllowUserMount: true, + }) + + mntns, err := k.VFS().NewMountNamespace(ctx, creds, "", "tmpfs", &vfs.GetFilesystemOptions{}) + if err != nil { + t.Fatalf("NewMountNamespace(): %v", err) + } + + return testutil.NewSystem(ctx, t, k.VFS(), mntns) +} + +// newTestConnection creates a fuse connection that the sentry can communicate with +// and the FD for the server to communicate with. +func newTestConnection(system *testutil.System, k *kernel.Kernel, maxActiveRequests uint64) (*connection, *vfs.FileDescription, error) { + vfsObj := &vfs.VirtualFilesystem{} + fuseDev := &DeviceFD{} + + if err := vfsObj.Init(system.Ctx); err != nil { + return nil, nil, err + } + + vd := vfsObj.NewAnonVirtualDentry("genCountFD") + defer vd.DecRef(system.Ctx) + if err := fuseDev.vfsfd.Init(fuseDev, linux.O_RDWR|linux.O_CREAT, vd.Mount(), vd.Dentry(), &vfs.FileDescriptionOptions{}); err != nil { + return nil, nil, err + } + + fsopts := filesystemOptions{ + maxActiveRequests: maxActiveRequests, + } + fs, err := newFUSEFilesystem(system.Ctx, 0, &fsopts, &fuseDev.vfsfd) + if err != nil { + return nil, nil, err + } + + return fs.conn, &fuseDev.vfsfd, nil +} + +type testPayload struct { + marshal.StubMarshallable + data uint32 +} + +// SizeBytes implements marshal.Marshallable.SizeBytes. +func (t *testPayload) SizeBytes() int { + return 4 +} + +// MarshalBytes implements marshal.Marshallable.MarshalBytes. +func (t *testPayload) MarshalBytes(dst []byte) { + usermem.ByteOrder.PutUint32(dst[:4], t.data) +} + +// UnmarshalBytes implements marshal.Marshallable.UnmarshalBytes. +func (t *testPayload) UnmarshalBytes(src []byte) { + *t = testPayload{data: usermem.ByteOrder.Uint32(src[:4])} +} + +// Packed implements marshal.Marshallable.Packed. +func (t *testPayload) Packed() bool { + return true +} + +// MarshalUnsafe implements marshal.Marshallable.MarshalUnsafe. +func (t *testPayload) MarshalUnsafe(dst []byte) { + t.MarshalBytes(dst) +} + +// UnmarshalUnsafe implements marshal.Marshallable.UnmarshalUnsafe. +func (t *testPayload) UnmarshalUnsafe(src []byte) { + t.UnmarshalBytes(src) +} diff --git a/pkg/syserror/syserror.go b/pkg/syserror/syserror.go index fe9f50169..f516c8e46 100644 --- a/pkg/syserror/syserror.go +++ b/pkg/syserror/syserror.go @@ -33,6 +33,7 @@ var ( EBADFD = error(syscall.EBADFD) EBUSY = error(syscall.EBUSY) ECHILD = error(syscall.ECHILD) + ECONNABORTED = error(syscall.ECONNABORTED) ECONNREFUSED = error(syscall.ECONNREFUSED) ECONNRESET = error(syscall.ECONNRESET) EDEADLK = error(syscall.EDEADLK) -- cgit v1.2.3 From 70cfea23776189575807f6aae28769bdfaa1e3fb Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Wed, 9 Sep 2020 16:44:35 -0700 Subject: Fix comments of TODO issues. --- pkg/abi/linux/fuse.go | 2 +- pkg/sentry/fsimpl/fuse/request_response.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index ba3316ad6..ca304af05 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -663,7 +663,7 @@ func (r *FUSEMkdirIn) SizeBytes() int { type FUSERmDirIn struct { marshal.StubMarshallable - // Name is a directory name to be looked up. + // Name is a directory name to be removed. Name string } diff --git a/pkg/sentry/fsimpl/fuse/request_response.go b/pkg/sentry/fsimpl/fuse/request_response.go index fd9a96197..70db50f38 100644 --- a/pkg/sentry/fsimpl/fuse/request_response.go +++ b/pkg/sentry/fsimpl/fuse/request_response.go @@ -122,7 +122,7 @@ func (conn *connection) NewRequest(creds *auth.Credentials, pid uint32, ino uint buf := make([]byte, hdr.Len) - // TODO(gVisor.dev/3698): Use the unsafe version once go_marshal is safe to use again. + // TODO(gVisor.dev/issue/3698): Use the unsafe version once go_marshal is safe to use again. hdr.MarshalBytes(buf[:hdrLen]) payload.MarshalBytes(buf[hdrLen:]) @@ -223,7 +223,7 @@ func (r *Response) UnmarshalPayload(m marshal.Marshallable) error { return nil } - // TODO(gVisor.dev/3698): Use the unsafe version once go_marshal is safe to use again. + // TODO(gVisor.dev/issue/3698): Use the unsafe version once go_marshal is safe to use again. m.UnmarshalBytes(r.data[hdrLen:]) return nil } -- cgit v1.2.3 From 2051260e8286b25ab39f3c1cb4614005236cbcc6 Mon Sep 17 00:00:00 2001 From: Boyuan He Date: Wed, 9 Sep 2020 17:13:18 -0700 Subject: Implement FUSE_UNLINK Fixes #3696 --- pkg/abi/linux/fuse.go | 23 ++++++++++ pkg/sentry/fsimpl/fuse/fusefs.go | 23 ++++++++++ pkg/sentry/fsimpl/kernfs/filesystem.go | 6 ++- pkg/sentry/fsimpl/kernfs/kernfs.go | 31 +++++++++++++ test/fuse/BUILD | 5 ++ test/fuse/linux/BUILD | 14 ++++++ test/fuse/linux/unlink_test.cc | 83 ++++++++++++++++++++++++++++++++++ 7 files changed, 184 insertions(+), 1 deletion(-) create mode 100644 test/fuse/linux/unlink_test.cc (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index ca304af05..fdd22a13d 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -846,3 +846,26 @@ type FUSESetAttrIn struct { _ uint32 } + +// FUSEUnlinkIn is the request sent by the kernel to the daemon +// when trying to unlink a node. +// +// Dynamically-sized objects cannot be marshalled. +type FUSEUnlinkIn struct { + marshal.StubMarshallable + + // Name of the node to unlink. + Name string +} + +// MarshalBytes serializes r.name to the dst buffer, which should +// have size len(r.Name) + 1 and last byte set to 0. +func (r *FUSEUnlinkIn) MarshalBytes(buf []byte) { + copy(buf, r.Name) +} + +// SizeBytes is the size of the memory representation of FUSEUnlinkIn. +// 1 extra byte for null-terminated Name string. +func (r *FUSEUnlinkIn) SizeBytes() int { + return len(r.Name) + 1 +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index f4ed73c12..8e749bdad 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -455,6 +455,29 @@ func (i *inode) NewSymlink(ctx context.Context, name, target string) (*vfs.Dentr return i.newEntry(ctx, name, linux.S_IFLNK, linux.FUSE_SYMLINK, &in) } +// Unlink implements kernfs.Inode.Unlink. +func (i *inode) Unlink(ctx context.Context, name string, child *vfs.Dentry) error { + kernelTask := kernel.TaskFromContext(ctx) + if kernelTask == nil { + log.Warningf("fusefs.Inode.newEntry: couldn't get kernel task from context", i.NodeID) + return syserror.EINVAL + } + in := linux.FUSEUnlinkIn{Name: name} + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, linux.FUSE_UNLINK, &in) + if err != nil { + return err + } + res, err := i.fs.conn.Call(kernelTask, req) + if err != nil { + return err + } + // only return error, discard res. + if err := res.Error(); err != nil { + return err + } + return i.dentry.RemoveChildLocked(name, child) +} + // NewDir implements kernfs.Inode.NewDir. func (i *inode) NewDir(ctx context.Context, name string, opts vfs.MkdirOptions) (*vfs.Dentry, error) { in := linux.FUSEMkdirIn{ diff --git a/pkg/sentry/fsimpl/kernfs/filesystem.go b/pkg/sentry/fsimpl/kernfs/filesystem.go index 2823c3b1a..49f6a0f1d 100644 --- a/pkg/sentry/fsimpl/kernfs/filesystem.go +++ b/pkg/sentry/fsimpl/kernfs/filesystem.go @@ -770,6 +770,10 @@ func (fs *Filesystem) SymlinkAt(ctx context.Context, rp *vfs.ResolvingPath, targ func (fs *Filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error { fs.mu.Lock() defer fs.mu.Unlock() + + // Store the name before walkExistingLocked as rp will be advanced past the + // name in the following call. + name := rp.Component() vfsd, _, err := fs.walkExistingLocked(ctx, rp) fs.processDeferredDecRefsLocked(ctx) if err != nil { @@ -795,7 +799,7 @@ func (fs *Filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error if err := virtfs.PrepareDeleteDentry(mntns, vfsd); err != nil { return err } - if err := parentDentry.inode.Unlink(ctx, rp.Component(), vfsd); err != nil { + if err := parentDentry.inode.Unlink(ctx, name, vfsd); err != nil { virtfs.AbortDeleteDentry(vfsd) return err } diff --git a/pkg/sentry/fsimpl/kernfs/kernfs.go b/pkg/sentry/fsimpl/kernfs/kernfs.go index 61189af25..163f26ceb 100644 --- a/pkg/sentry/fsimpl/kernfs/kernfs.go +++ b/pkg/sentry/fsimpl/kernfs/kernfs.go @@ -60,6 +60,7 @@ import ( "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/sync" + "gvisor.dev/gvisor/pkg/syserror" ) // Filesystem mostly implements vfs.FilesystemImpl for a generic in-memory @@ -267,6 +268,36 @@ func (d *Dentry) InsertChildLocked(name string, child *Dentry) { d.children[name] = child } +// RemoveChild removes child from the vfs dentry cache. This does not update the +// directory inode or modify the inode to be unlinked. So calling this on its own +// isn't sufficient to remove a child from a directory. +// +// Precondition: d must represent a directory inode. +func (d *Dentry) RemoveChild(name string, child *vfs.Dentry) error { + d.dirMu.Lock() + defer d.dirMu.Unlock() + return d.RemoveChildLocked(name, child) +} + +// RemoveChildLocked is equivalent to RemoveChild, with additional +// preconditions. +// +// Precondition: d.dirMu must be locked. +func (d *Dentry) RemoveChildLocked(name string, child *vfs.Dentry) error { + if !d.isDir() { + panic(fmt.Sprintf("RemoveChild called on non-directory Dentry: %+v.", d)) + } + c, ok := d.children[name] + if !ok { + return syserror.ENOENT + } + if &c.vfsd != child { + panic(fmt.Sprintf("Dentry hashed into inode doesn't match what vfs thinks! Child: %+v, vfs: %+v", c, child)) + } + delete(d.children, name) + return nil +} + // Inode returns the dentry's inode. func (d *Dentry) Inode() Inode { return d.inode diff --git a/test/fuse/BUILD b/test/fuse/BUILD index 29b9a9d93..dacfe0175 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -62,6 +62,11 @@ syscall_test( test = "//test/fuse/linux:create_test", ) +syscall_test( + fuse = "True", + test = "//test/fuse/linux:unlink_test", +) + syscall_test( fuse = "True", test = "//test/fuse/linux:setstat_test", diff --git a/test/fuse/linux/BUILD b/test/fuse/linux/BUILD index 7ecd6d8cb..7673252ec 100644 --- a/test/fuse/linux/BUILD +++ b/test/fuse/linux/BUILD @@ -214,3 +214,17 @@ cc_binary( "//test/util:test_util", ], ) + +cc_binary( + name = "unlink_test", + testonly = 1, + srcs = ["unlink_test.cc"], + deps = [ + gtest, + ":fuse_base", + "//test/util:fuse_util", + "//test/util:temp_umask", + "//test/util:test_main", + "//test/util:test_util", + ], +) diff --git a/test/fuse/linux/unlink_test.cc b/test/fuse/linux/unlink_test.cc new file mode 100644 index 000000000..5702e9b32 --- /dev/null +++ b/test/fuse/linux/unlink_test.cc @@ -0,0 +1,83 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "gtest/gtest.h" +#include "test/fuse/linux/fuse_base.h" +#include "test/util/fuse_util.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +class UnlinkTest : public FuseTest { + protected: + const std::string test_file_ = "test_file"; +}; + +TEST_F(UnlinkTest, RegularFile) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + SetServerInodeLookup(test_file_, S_IFREG | S_IRWXU | S_IRWXG | S_IRWXO); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header), + }; + auto iov_out = FuseGenerateIovecs(out_header); + SetServerResponse(FUSE_UNLINK, iov_out); + + ASSERT_THAT(unlink(test_file_path.c_str()), SyscallSucceeds()); + struct fuse_in_header in_header; + std::vector unlinked_file(test_file_.length() + 1); + auto iov_in = FuseGenerateIovecs(in_header, unlinked_file); + GetServerActualRequest(iov_in); + + EXPECT_EQ(in_header.len, sizeof(in_header) + test_file_.length() + 1); + EXPECT_EQ(in_header.opcode, FUSE_UNLINK); + EXPECT_EQ(std::string(unlinked_file.data()), test_file_); +} + +TEST_F(UnlinkTest, NoFile) { + const std::string test_file_path = + JoinPath(mount_point_.path().c_str(), test_file_); + SetServerInodeLookup(test_file_, S_IFREG | S_IRWXU | S_IRWXG | S_IRWXO); + + struct fuse_out_header out_header = { + .len = sizeof(struct fuse_out_header), + .error = -ENOENT, + }; + auto iov_out = FuseGenerateIovecs(out_header); + SetServerResponse(FUSE_UNLINK, iov_out); + + ASSERT_THAT(unlink(test_file_path.c_str()), SyscallFailsWithErrno(ENOENT)); + SkipServerActualRequest(); +} + +} // namespace + +} // namespace testing +} // namespace gvisor -- cgit v1.2.3 From dd103527298375edd3e14600ac762c9a3103ac37 Mon Sep 17 00:00:00 2001 From: Craig Chi Date: Wed, 9 Sep 2020 16:30:25 -0700 Subject: Unexport fusefs.inode.nodeID --- pkg/sentry/fsimpl/fuse/connection.go | 2 +- pkg/sentry/fsimpl/fuse/directory.go | 4 ++-- pkg/sentry/fsimpl/fuse/file.go | 7 ++++--- pkg/sentry/fsimpl/fuse/fusefs.go | 38 ++++++++++++++++++------------------ pkg/sentry/fsimpl/fuse/read_write.go | 4 ++-- 5 files changed, 28 insertions(+), 27 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 5960d9368..fedf4d067 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -47,7 +47,7 @@ const ( type connection struct { fd *DeviceFD - // mu protect access to struct memebers. + // mu protects access to struct memebers. mu sync.Mutex // attributeVersion is the version of connection's attributes. diff --git a/pkg/sentry/fsimpl/fuse/directory.go b/pkg/sentry/fsimpl/fuse/directory.go index a83357129..798c4a6f3 100644 --- a/pkg/sentry/fsimpl/fuse/directory.go +++ b/pkg/sentry/fsimpl/fuse/directory.go @@ -67,8 +67,8 @@ func (dir *directoryFD) IterDirents(ctx context.Context, callback vfs.IterDirent Flags: dir.statusFlags(), } - /// TODO(gVisor.dev/issue/3404): Support FUSE_READDIRPLUS. - req, err := fusefs.conn.NewRequest(creds, uint32(task.ThreadID()), dir.inode().NodeID, linux.FUSE_READDIR, &in) + // TODO(gVisor.dev/issue/3404): Support FUSE_READDIRPLUS. + req, err := fusefs.conn.NewRequest(creds, uint32(task.ThreadID()), dir.inode().nodeID, linux.FUSE_READDIR, &in) if err != nil { return err } diff --git a/pkg/sentry/fsimpl/fuse/file.go b/pkg/sentry/fsimpl/fuse/file.go index b98145ba2..991efcda4 100644 --- a/pkg/sentry/fsimpl/fuse/file.go +++ b/pkg/sentry/fsimpl/fuse/file.go @@ -84,7 +84,7 @@ func (fd *fileDescription) Release(ctx context.Context) { } kernelTask := kernel.TaskFromContext(ctx) // ignoring errors and FUSE server reply is analogous to Linux's behavior. - req, err := conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), fd.inode().NodeID, opcode, &in) + req, err := conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), fd.inode().nodeID, opcode, &in) if err != nil { // No way to invoke Call() with an errored request. return @@ -125,8 +125,9 @@ func (fd *fileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linu return inode.Stat(ctx, fs, opts) } -// SetStat implements FileDescriptionImpl.SetStat. +// SetStat implements vfs.FileDescriptionImpl.SetStat. func (fd *fileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions) error { + fs := fd.filesystem() creds := auth.CredentialsFromContext(ctx) - return fd.inode().setAttr(ctx, fd.inode().fs.VFSFilesystem(), creds, opts, true, fd.Fh) + return fd.inode().setAttr(ctx, fs, creds, opts, true, fd.Fh) } diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 8e749bdad..402dabe5a 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -254,7 +254,7 @@ type inode struct { // metaDataMu protects the metadata of this inode. metadataMu sync.Mutex - NodeID uint64 + nodeID uint64 locks vfs.FileLocks @@ -279,13 +279,13 @@ func (fs *filesystem) newRootInode(creds *auth.Credentials, mode linux.FileMode) i.InodeAttrs.Init(creds, linux.UNNAMED_MAJOR, fs.devMinor, 1, linux.ModeDirectory|0755) i.OrderedChildren.Init(kernfs.OrderedChildrenOptions{}) i.dentry.Init(i) - i.NodeID = 1 + i.nodeID = 1 return &i.dentry } func (fs *filesystem) newInode(nodeID uint64, attr linux.FUSEAttr) *kernfs.Dentry { - i := &inode{fs: fs, NodeID: nodeID} + i := &inode{fs: fs, nodeID: nodeID} creds := auth.Credentials{EffectiveKGID: auth.KGID(attr.UID), EffectiveKUID: auth.KUID(attr.UID)} i.InodeAttrs.Init(&creds, linux.UNNAMED_MAJOR, fs.devMinor, fs.NextIno(), linux.FileMode(attr.Mode)) atomic.StoreUint64(&i.size, attr.Size) @@ -342,7 +342,7 @@ func (i *inode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentr in.Flags &= ^uint32(linux.O_TRUNC) } - req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, opcode, &in) + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.nodeID, opcode, &in) if err != nil { return nil, err } @@ -405,13 +405,13 @@ func (i *inode) Lookup(ctx context.Context, name string) (*vfs.Dentry, error) { return i.newEntry(ctx, name, 0, linux.FUSE_LOOKUP, &in) } -// IterDirents implements Inode.IterDirents. -func (inode) IterDirents(ctx context.Context, callback vfs.IterDirentsCallback, offset, relOffset int64) (int64, error) { +// IterDirents implements kernfs.Inode.IterDirents. +func (*inode) IterDirents(ctx context.Context, callback vfs.IterDirentsCallback, offset, relOffset int64) (int64, error) { return offset, nil } -// Valid implements Inode.Valid. -func (inode) Valid(ctx context.Context) bool { +// Valid implements kernfs.Inode.Valid. +func (*inode) Valid(ctx context.Context) bool { return true } @@ -419,7 +419,7 @@ func (inode) Valid(ctx context.Context) bool { func (i *inode) NewFile(ctx context.Context, name string, opts vfs.OpenOptions) (*vfs.Dentry, error) { kernelTask := kernel.TaskFromContext(ctx) if kernelTask == nil { - log.Warningf("fusefs.Inode.NewFile: couldn't get kernel task from context", i.NodeID) + log.Warningf("fusefs.Inode.NewFile: couldn't get kernel task from context", i.nodeID) return nil, syserror.EINVAL } in := linux.FUSECreateIn{ @@ -459,11 +459,11 @@ func (i *inode) NewSymlink(ctx context.Context, name, target string) (*vfs.Dentr func (i *inode) Unlink(ctx context.Context, name string, child *vfs.Dentry) error { kernelTask := kernel.TaskFromContext(ctx) if kernelTask == nil { - log.Warningf("fusefs.Inode.newEntry: couldn't get kernel task from context", i.NodeID) + log.Warningf("fusefs.Inode.newEntry: couldn't get kernel task from context", i.nodeID) return syserror.EINVAL } in := linux.FUSEUnlinkIn{Name: name} - req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, linux.FUSE_UNLINK, &in) + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.nodeID, linux.FUSE_UNLINK, &in) if err != nil { return err } @@ -496,7 +496,7 @@ func (i *inode) RmDir(ctx context.Context, name string, child *vfs.Dentry) error task, creds := kernel.TaskFromContext(ctx), auth.CredentialsFromContext(ctx) in := linux.FUSERmDirIn{Name: name} - req, err := fusefs.conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_RMDIR, &in) + req, err := fusefs.conn.NewRequest(creds, uint32(task.ThreadID()), i.nodeID, linux.FUSE_RMDIR, &in) if err != nil { return err } @@ -522,10 +522,10 @@ func (i *inode) RmDir(ctx context.Context, name string, child *vfs.Dentry) error func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMode, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*vfs.Dentry, error) { kernelTask := kernel.TaskFromContext(ctx) if kernelTask == nil { - log.Warningf("fusefs.Inode.newEntry: couldn't get kernel task from context", i.NodeID) + log.Warningf("fusefs.Inode.newEntry: couldn't get kernel task from context", i.nodeID) return nil, syserror.EINVAL } - req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, opcode, payload) + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.nodeID, opcode, payload) if err != nil { return nil, err } @@ -552,7 +552,7 @@ func (i *inode) newEntry(ctx context.Context, name string, fileType linux.FileMo return child.VFSDentry(), nil } -// Getlink implements Inode.Getlink. +// Getlink implements kernfs.Inode.Getlink. func (i *inode) Getlink(ctx context.Context, mnt *vfs.Mount) (vfs.VirtualDentry, string, error) { path, err := i.Readlink(ctx, mnt) return vfs.VirtualDentry{}, path, err @@ -563,13 +563,13 @@ func (i *inode) Readlink(ctx context.Context, mnt *vfs.Mount) (string, error) { if i.Mode().FileType()&linux.S_IFLNK == 0 { return "", syserror.EINVAL } - if i.link == "" { + if len(i.link) == 0 { kernelTask := kernel.TaskFromContext(ctx) if kernelTask == nil { log.Warningf("fusefs.Inode.Readlink: couldn't get kernel task from context") return "", syserror.EINVAL } - req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.NodeID, linux.FUSE_READLINK, &linux.FUSEEmptyIn{}) + req, err := i.fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(kernelTask.ThreadID()), i.nodeID, linux.FUSE_READLINK, &linux.FUSEEmptyIn{}) if err != nil { return "", err } @@ -675,7 +675,7 @@ func (i *inode) getAttr(ctx context.Context, fs *vfs.Filesystem, opts vfs.StatOp GetAttrFlags: flags, Fh: fh, } - req, err := i.fs.conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_GETATTR, &in) + req, err := i.fs.conn.NewRequest(creds, uint32(task.ThreadID()), i.nodeID, linux.FUSE_GETATTR, &in) if err != nil { return linux.FUSEAttr{}, err } @@ -798,7 +798,7 @@ func (i *inode) setAttr(ctx context.Context, fs *vfs.Filesystem, creds *auth.Cre UID: opts.Stat.UID, GID: opts.Stat.GID, } - req, err := conn.NewRequest(creds, uint32(task.ThreadID()), i.NodeID, linux.FUSE_SETATTR, &in) + req, err := conn.NewRequest(creds, uint32(task.ThreadID()), i.nodeID, linux.FUSE_SETATTR, &in) if err != nil { return err } diff --git a/pkg/sentry/fsimpl/fuse/read_write.go b/pkg/sentry/fsimpl/fuse/read_write.go index 22a018e5e..625d1547f 100644 --- a/pkg/sentry/fsimpl/fuse/read_write.go +++ b/pkg/sentry/fsimpl/fuse/read_write.go @@ -79,7 +79,7 @@ func (fs *filesystem) ReadInPages(ctx context.Context, fd *regularFileFD, off ui in.Offset = off + (uint64(pagesRead) << usermem.PageShift) in.Size = pagesCanRead << usermem.PageShift - req, err := fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(t.ThreadID()), fd.inode().NodeID, linux.FUSE_READ, &in) + req, err := fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(t.ThreadID()), fd.inode().nodeID, linux.FUSE_READ, &in) if err != nil { return nil, 0, err } @@ -203,7 +203,7 @@ func (fs *filesystem) Write(ctx context.Context, fd *regularFileFD, off uint64, in.Offset = off + uint64(written) in.Size = toWrite - req, err := fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(t.ThreadID()), fd.inode().NodeID, linux.FUSE_WRITE, &in) + req, err := fs.conn.NewRequest(auth.CredentialsFromContext(ctx), uint32(t.ThreadID()), fd.inode().nodeID, linux.FUSE_WRITE, &in) if err != nil { return 0, err } -- cgit v1.2.3 From c6e749fb32efc38fa51e2997f622e2279b1b935e Mon Sep 17 00:00:00 2001 From: Andrei Vagin Date: Mon, 14 Sep 2020 10:56:25 -0700 Subject: fs/fuse: Move the 'marshal' and 'primitive' packages to the 'pkg' directory. --- pkg/abi/linux/fuse.go | 4 ++-- pkg/sentry/fsimpl/fuse/connection.go | 1 - pkg/sentry/fsimpl/fuse/fusefs.go | 2 +- pkg/sentry/fsimpl/fuse/request_response.go | 2 +- pkg/sentry/fsimpl/fuse/utils_test.go | 2 +- 5 files changed, 5 insertions(+), 6 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/abi/linux/fuse.go b/pkg/abi/linux/fuse.go index c2357a2ee..d91c97a64 100644 --- a/pkg/abi/linux/fuse.go +++ b/pkg/abi/linux/fuse.go @@ -15,8 +15,8 @@ package linux import ( - "gvisor.dev/gvisor/tools/go_marshal/marshal" - "gvisor.dev/gvisor/tools/go_marshal/primitive" + "gvisor.dev/gvisor/pkg/marshal" + "gvisor.dev/gvisor/pkg/marshal/primitive" ) // +marshal diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index fedf4d067..a7402c149 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -20,7 +20,6 @@ import ( "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" "gvisor.dev/gvisor/pkg/log" - "gvisor.dev/gvisor/pkg/marshal" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/syserror" diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 402dabe5a..86922a96a 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -24,13 +24,13 @@ import ( "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" "gvisor.dev/gvisor/pkg/log" + "gvisor.dev/gvisor/pkg/marshal" "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/waiter" - "gvisor.dev/gvisor/tools/go_marshal/marshal" ) // Name is the default filesystem name. diff --git a/pkg/sentry/fsimpl/fuse/request_response.go b/pkg/sentry/fsimpl/fuse/request_response.go index 70db50f38..7fa00569b 100644 --- a/pkg/sentry/fsimpl/fuse/request_response.go +++ b/pkg/sentry/fsimpl/fuse/request_response.go @@ -19,10 +19,10 @@ import ( "syscall" "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/marshal" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/usermem" - "gvisor.dev/gvisor/tools/go_marshal/marshal" ) // fuseInitRes is a variable-length wrapper of linux.FUSEInitOut. The FUSE diff --git a/pkg/sentry/fsimpl/fuse/utils_test.go b/pkg/sentry/fsimpl/fuse/utils_test.go index 436322830..ec2d2c7bc 100644 --- a/pkg/sentry/fsimpl/fuse/utils_test.go +++ b/pkg/sentry/fsimpl/fuse/utils_test.go @@ -18,12 +18,12 @@ import ( "testing" "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/marshal" "gvisor.dev/gvisor/pkg/sentry/fsimpl/testutil" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/usermem" - "gvisor.dev/gvisor/tools/go_marshal/marshal" ) func setup(t *testing.T) *testutil.System { -- cgit v1.2.3 From 093b0ab6c57d83407704e31005c5c446cf51a5f1 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Mon, 14 Sep 2020 21:30:51 +0000 Subject: Fix FUSE go unit test merge conflict mistake --- pkg/sentry/fsimpl/fuse/dev_test.go | 101 ----------------------------------- pkg/sentry/fsimpl/fuse/utils_test.go | 21 ++++++++ 2 files changed, 21 insertions(+), 101 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/dev_test.go b/pkg/sentry/fsimpl/fuse/dev_test.go index 5abdf82d3..5986133e9 100644 --- a/pkg/sentry/fsimpl/fuse/dev_test.go +++ b/pkg/sentry/fsimpl/fuse/dev_test.go @@ -16,12 +16,10 @@ package fuse import ( "fmt" - "io" "math/rand" "testing" "gvisor.dev/gvisor/pkg/abi/linux" - "gvisor.dev/gvisor/pkg/marshal" "gvisor.dev/gvisor/pkg/sentry/fsimpl/testutil" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" @@ -323,102 +321,3 @@ func fuseServerRun(t *testing.T, s *testutil.System, k *kernel.Kernel, fd *vfs.F } } } - -func setup(t *testing.T) *testutil.System { - k, err := testutil.Boot() - if err != nil { - t.Fatalf("Error creating kernel: %v", err) - } - - ctx := k.SupervisorContext() - creds := auth.CredentialsFromContext(ctx) - - k.VFS().MustRegisterFilesystemType(Name, &FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ - AllowUserList: true, - AllowUserMount: true, - }) - - mntns, err := k.VFS().NewMountNamespace(ctx, creds, "", "tmpfs", &vfs.MountOptions{}) - if err != nil { - t.Fatalf("NewMountNamespace(): %v", err) - } - - return testutil.NewSystem(ctx, t, k.VFS(), mntns) -} - -// newTestConnection creates a fuse connection that the sentry can communicate with -// and the FD for the server to communicate with. -func newTestConnection(system *testutil.System, k *kernel.Kernel, maxActiveRequests uint64) (*connection, *vfs.FileDescription, error) { - vfsObj := &vfs.VirtualFilesystem{} - fuseDev := &DeviceFD{} - - if err := vfsObj.Init(system.Ctx); err != nil { - return nil, nil, err - } - - vd := vfsObj.NewAnonVirtualDentry("genCountFD") - defer vd.DecRef(system.Ctx) - if err := fuseDev.vfsfd.Init(fuseDev, linux.O_RDWR|linux.O_CREAT, vd.Mount(), vd.Dentry(), &vfs.FileDescriptionOptions{}); err != nil { - return nil, nil, err - } - - fsopts := filesystemOptions{ - maxActiveRequests: maxActiveRequests, - } - fs, err := newFUSEFilesystem(system.Ctx, 0, &fsopts, &fuseDev.vfsfd) - if err != nil { - return nil, nil, err - } - - return fs.conn, &fuseDev.vfsfd, nil -} - -// SizeBytes implements marshal.Marshallable.SizeBytes. -func (t *testPayload) SizeBytes() int { - return 4 -} - -// MarshalBytes implements marshal.Marshallable.MarshalBytes. -func (t *testPayload) MarshalBytes(dst []byte) { - usermem.ByteOrder.PutUint32(dst[:4], t.data) -} - -// UnmarshalBytes implements marshal.Marshallable.UnmarshalBytes. -func (t *testPayload) UnmarshalBytes(src []byte) { - *t = testPayload{data: usermem.ByteOrder.Uint32(src[:4])} -} - -// Packed implements marshal.Marshallable.Packed. -func (t *testPayload) Packed() bool { - return true -} - -// MarshalUnsafe implements marshal.Marshallable.MarshalUnsafe. -func (t *testPayload) MarshalUnsafe(dst []byte) { - t.MarshalBytes(dst) -} - -// UnmarshalUnsafe implements marshal.Marshallable.UnmarshalUnsafe. -func (t *testPayload) UnmarshalUnsafe(src []byte) { - t.UnmarshalBytes(src) -} - -// CopyOutN implements marshal.Marshallable.CopyOutN. -func (t *testPayload) CopyOutN(cc marshal.CopyContext, addr usermem.Addr, limit int) (int, error) { - panic("not implemented") -} - -// CopyOut implements marshal.Marshallable.CopyOut. -func (t *testPayload) CopyOut(cc marshal.CopyContext, addr usermem.Addr) (int, error) { - panic("not implemented") -} - -// CopyIn implements marshal.Marshallable.CopyIn. -func (t *testPayload) CopyIn(cc marshal.CopyContext, addr usermem.Addr) (int, error) { - panic("not implemented") -} - -// WriteTo implements io.WriterTo.WriteTo. -func (t *testPayload) WriteTo(w io.Writer) (int64, error) { - panic("not implemented") -} diff --git a/pkg/sentry/fsimpl/fuse/utils_test.go b/pkg/sentry/fsimpl/fuse/utils_test.go index ec2d2c7bc..db1521f13 100644 --- a/pkg/sentry/fsimpl/fuse/utils_test.go +++ b/pkg/sentry/fsimpl/fuse/utils_test.go @@ -15,6 +15,7 @@ package fuse import ( + "io" "testing" "gvisor.dev/gvisor/pkg/abi/linux" @@ -109,3 +110,23 @@ func (t *testPayload) MarshalUnsafe(dst []byte) { func (t *testPayload) UnmarshalUnsafe(src []byte) { t.UnmarshalBytes(src) } + +// CopyOutN implements marshal.Marshallable.CopyOutN. +func (t *testPayload) CopyOutN(task marshal.CopyContext, addr usermem.Addr, limit int) (int, error) { + panic("not implemented") +} + +// CopyOut implements marshal.Marshallable.CopyOut. +func (t *testPayload) CopyOut(task marshal.CopyContext, addr usermem.Addr) (int, error) { + panic("not implemented") +} + +// CopyIn implements marshal.Marshallable.CopyIn. +func (t *testPayload) CopyIn(task marshal.CopyContext, addr usermem.Addr) (int, error) { + panic("not implemented") +} + +// WriteTo implements io.WriterTo.WriteTo. +func (t *testPayload) WriteTo(w io.Writer) (int64, error) { + panic("not implemented") +} -- cgit v1.2.3 From 113928754c26ea3d4d7d387bae459ce447774d30 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Mon, 14 Sep 2020 21:54:18 +0000 Subject: Fix FUSE unit test after vfs interface change --- pkg/sentry/fsimpl/fuse/utils_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/utils_test.go b/pkg/sentry/fsimpl/fuse/utils_test.go index db1521f13..e1d9e3365 100644 --- a/pkg/sentry/fsimpl/fuse/utils_test.go +++ b/pkg/sentry/fsimpl/fuse/utils_test.go @@ -41,7 +41,7 @@ func setup(t *testing.T) *testutil.System { AllowUserMount: true, }) - mntns, err := k.VFS().NewMountNamespace(ctx, creds, "", "tmpfs", &vfs.GetFilesystemOptions{}) + mntns, err := k.VFS().NewMountNamespace(ctx, creds, "", "tmpfs", &vfs.MountOptions{}) if err != nil { t.Fatalf("NewMountNamespace(): %v", err) } -- cgit v1.2.3 From f148242abab1ff043428036ccaf464de98268863 Mon Sep 17 00:00:00 2001 From: Andrei Vagin Date: Mon, 14 Sep 2020 15:32:46 -0700 Subject: Revert "fuse: add benchmarking support for FUSE" test/fuse/benchmark/read_benchmark.cc:34: Failure Expected: (fuse_prefix) != (nullptr), actual: NULL vs (nullptr) external/com_google_benchmark/src/benchmark_runner.cc:120: RunInThread: Check `st.iterations() >= st.max_iterations' failed. Benchmark returned before State::KeepRunning() returned false! --- FAIL: Benchmarks_BM_Read/262144/real_time (0.29s) runner.go:502: test "Benchmarks.BM_Read/262144/real_time" failed with error exit status 134, want nil FAIL --- Makefile | 4 -- images/basic/fuse/Dockerfile | 15 ----- pkg/sentry/fs/g3doc/fuse.md | 47 ------------- pkg/test/dockerutil/container.go | 10 --- pkg/test/dockerutil/dockerutil.go | 20 ------ scripts/common_build.sh | 5 +- test/e2e/integration_test.go | 6 +- test/fuse/BUILD | 45 ------------- test/fuse/benchmark/BUILD | 91 ------------------------- test/fuse/benchmark/mkdir_benchmark.cc | 51 -------------- test/fuse/benchmark/open_benchmark.cc | 60 ----------------- test/fuse/benchmark/read_benchmark.cc | 57 ---------------- test/fuse/benchmark/stat_benchmark.cc | 65 ------------------ test/fuse/benchmark/symlink_benchmark.cc | 60 ----------------- test/image/BUILD | 1 - test/image/image_test.go | 52 -------------- test/runner/BUILD | 3 - test/runner/defs.bzl | 28 -------- test/runner/runner.go | 112 +++---------------------------- 19 files changed, 16 insertions(+), 716 deletions(-) delete mode 100644 images/basic/fuse/Dockerfile delete mode 100644 test/fuse/benchmark/BUILD delete mode 100644 test/fuse/benchmark/mkdir_benchmark.cc delete mode 100644 test/fuse/benchmark/open_benchmark.cc delete mode 100644 test/fuse/benchmark/read_benchmark.cc delete mode 100644 test/fuse/benchmark/stat_benchmark.cc delete mode 100644 test/fuse/benchmark/symlink_benchmark.cc (limited to 'pkg/sentry') diff --git a/Makefile b/Makefile index adf2189f9..4f18501a5 100644 --- a/Makefile +++ b/Makefile @@ -341,10 +341,6 @@ RUNTIME_BIN := $(RUNTIME_DIR)/runsc RUNTIME_LOG_DIR := $(RUNTIME_DIR)/logs RUNTIME_LOGS := $(RUNTIME_LOG_DIR)/runsc.log.%TEST%.%TIMESTAMP%.%COMMAND% -ifeq (,$(RUNTIME_NAME)) -RUNTIME_NAME := $(RUNTIME) -endif - dev: ## Installs a set of local runtimes. Requires sudo. @$(call submake,refresh ARGS="--net-raw") @$(call submake,configure RUNTIME_NAME="$(RUNTIME)" ARGS="--net-raw") diff --git a/images/basic/fuse/Dockerfile b/images/basic/fuse/Dockerfile deleted file mode 100644 index 9e88aa2c5..000000000 --- a/images/basic/fuse/Dockerfile +++ /dev/null @@ -1,15 +0,0 @@ -FROM ubuntu:20.04 - -ENV DEBIAN_FRONTEND=noninteractive - -RUN apt-get update -RUN apt-get install -y build-essential git pkg-config fuse3 libfuse3-3 libfuse3-dev strace - -WORKDIR /fus - -RUN mkdir -pv mountpoint -RUN git clone https://github.com/libfuse/libfuse - -RUN gcc -Wall ./libfuse/example/passthrough.c `pkg-config fuse3 --cflags --libs` -o server-bin - -CMD ["bash"] diff --git a/pkg/sentry/fs/g3doc/fuse.md b/pkg/sentry/fs/g3doc/fuse.md index 496e339ce..2ca84dd74 100644 --- a/pkg/sentry/fs/g3doc/fuse.md +++ b/pkg/sentry/fs/g3doc/fuse.md @@ -254,53 +254,6 @@ I/O syscalls like `read(2)`, `write(2)` and `mmap(2)`. - `FUSE_BMAP`: Old address space API for block defrag. Probably not needed. - `FUSE_NOTIFY_REPLY`: [TODO: what does this do?] -## Benchmark FUSE - -FUSE benchmark makes FUSE syscall inside docker container to make sure required -environment conditions are met - such as having the right libraries to start a -FUSE server. - -### Setup - -To run benchmark: - -1. Make sure you have `Docker` installed. -2. Download all docker images `make load-all-images`. -3. Config `runsc` docker runtime to have VFS2 and FUSE supported. -(e.g. `make configure RUNTIME=runsc ARGS="--vfs2 --fuse ..." ...`) - -You should now have a runtime with the following options configured in -`/etc/docker/daemon.json` -``` -"runsc": { - "path": "path/to/your/runsc", - "runtimeArgs": [ - "--vfs2", - "--fuse" - ... - ] - } -``` - -### Running benchmarks -With above setup, benchmark can be run with following command -``` -bazel test --test_output=all --cache_test_results=no --test_arg=-test.bench= //path/to:target -``` -For example: if you want to run stat test -``` -bazel test --test_output=all --cache_test_results=no --test_arg=-test.bench= //test/fuse:open_benchmark_runsc_ptrace_vfs2_fuse_container -``` - -Note: -- test target need to have `vfs2_fuse_container` to run in container with `vfs2` and `fuse` enabled -- `test_output` set to `all` to view the result in terminal -- `--cache_test_results` set to `no` to avoid cached benchmark - -### Use your fuse server - -To use your own FUSE server, change the `images/basic/fuse/Dockerfile` to compile your FUSE server into the container and name it `server-bin`. - # References - [fuse(4) Linux manual page](https://www.man7.org/linux/man-pages/man4/fuse.4.html) diff --git a/pkg/test/dockerutil/container.go b/pkg/test/dockerutil/container.go index 727be26b2..64d17f661 100644 --- a/pkg/test/dockerutil/container.go +++ b/pkg/test/dockerutil/container.go @@ -136,11 +136,6 @@ func MakeNativeContainer(ctx context.Context, logger testutil.Logger) *Container } } -// Runtime returns the runtime of the container. -func (c *Container) Runtime() string { - return c.runtime -} - // AddProfile adds a profile to this container. func (c *Container) AddProfile(p Profile) { c.profiles = append(c.profiles, p) @@ -546,8 +541,3 @@ func (c *Container) CleanUp(ctx context.Context) { // Forget all mounts. c.mounts = nil } - -// CopyErr returns the error that happened during copy. -func (c *Container) CopyErr() error { - return c.copyErr -} diff --git a/pkg/test/dockerutil/dockerutil.go b/pkg/test/dockerutil/dockerutil.go index a2d7e8c85..7027df1a5 100644 --- a/pkg/test/dockerutil/dockerutil.go +++ b/pkg/test/dockerutil/dockerutil.go @@ -121,26 +121,6 @@ func UsingVFS2() (bool, error) { return false, nil } -// UsingFUSE returns true if the 'runtime' has the fuse flag set. -func UsingFUSE() (bool, error) { - rMap, err := runtimeMap() - if err != nil { - return false, err - } - - list, ok := rMap["runtimeArgs"].([]interface{}) - if !ok { - return false, fmt.Errorf("unexpected format: %v", rMap) - } - - for _, element := range list { - if element == "--fuse" { - return true, nil - } - } - return false, nil -} - func runtimeMap() (map[string]interface{}, error) { // Read the configuration data; the file must exist. configBytes, err := ioutil.ReadFile(*config) diff --git a/scripts/common_build.sh b/scripts/common_build.sh index 6874e56f9..d4a6c4908 100755 --- a/scripts/common_build.sh +++ b/scripts/common_build.sh @@ -109,9 +109,8 @@ function collect_logs() { } function find_branch_name() { - (git branch --show-current \ + git branch --show-current \ || git rev-parse HEAD \ || bazel info workspace \ - | xargs basename) \ - | tr '/' '-' + | xargs basename } diff --git a/test/e2e/integration_test.go b/test/e2e/integration_test.go index 0c82e98d4..809244bab 100644 --- a/test/e2e/integration_test.go +++ b/test/e2e/integration_test.go @@ -168,10 +168,10 @@ func TestCheckpointRestore(t *testing.T) { } // TODO(gvisor.dev/issue/3373): Remove after implementing. - if usingVFS2, err := dockerutil.UsingVFS2(); err != nil { - t.Fatalf("failed to read config for runtime %s: %v", dockerutil.Runtime(), err) - } else if usingVFS2 { + if usingVFS2, err := dockerutil.UsingVFS2(); usingVFS2 { t.Skip("CheckpointRestore not implemented in VFS2.") + } else if err != nil { + t.Fatalf("failed to read config for runtime %s: %v", dockerutil.Runtime(), err) } ctx := context.Background() diff --git a/test/fuse/BUILD b/test/fuse/BUILD index dacfe0175..8e31fdd41 100644 --- a/test/fuse/BUILD +++ b/test/fuse/BUILD @@ -71,48 +71,3 @@ syscall_test( fuse = "True", test = "//test/fuse/linux:setstat_test", ) - -syscall_test( - size = "large", - add_overlay = True, - debug = False, - setup_command = "'./server-bin mountpoint'", - test = "//test/fuse/benchmark:stat_benchmark", - use_image = "basic/fuse", -) - -syscall_test( - size = "large", - add_overlay = True, - debug = False, - setup_command = "'./server-bin mountpoint'", - test = "//test/fuse/benchmark:open_benchmark", - use_image = "basic/fuse", -) - -syscall_test( - size = "large", - add_overlay = True, - debug = False, - setup_command = "'./server-bin mountpoint'", - test = "//test/fuse/benchmark:read_benchmark", - use_image = "basic/fuse", -) - -syscall_test( - size = "large", - add_overlay = True, - debug = False, - setup_command = "'./server-bin mountpoint'", - test = "//test/fuse/benchmark:symlink_benchmark", - use_image = "basic/fuse", -) - -syscall_test( - size = "large", - add_overlay = True, - debug = False, - setup_command = "'./server-bin mountpoint'", - test = "//test/fuse/benchmark:mkdir_benchmark", - use_image = "basic/fuse", -) diff --git a/test/fuse/benchmark/BUILD b/test/fuse/benchmark/BUILD deleted file mode 100644 index 16369d99b..000000000 --- a/test/fuse/benchmark/BUILD +++ /dev/null @@ -1,91 +0,0 @@ -load("//tools:defs.bzl", "cc_binary", "gbenchmark", "gtest") - -package( - default_visibility = ["//:sandbox"], - licenses = ["notice"], -) - -cc_binary( - name = "stat_benchmark", - testonly = 1, - srcs = [ - "stat_benchmark.cc", - ], - deps = [ - gbenchmark, - gtest, - "//test/util:fs_util", - "//test/util:temp_path", - "//test/util:test_main", - "//test/util:test_util", - "@com_google_absl//absl/strings", - ], -) - -cc_binary( - name = "open_benchmark", - testonly = 1, - srcs = [ - "open_benchmark.cc", - ], - deps = [ - gbenchmark, - gtest, - "//test/util:fs_util", - "//test/util:temp_path", - "//test/util:test_main", - "//test/util:test_util", - "@com_google_absl//absl/strings", - ], -) - -cc_binary( - name = "read_benchmark", - testonly = 1, - srcs = [ - "read_benchmark.cc", - ], - deps = [ - gbenchmark, - gtest, - "//test/util:fs_util", - "//test/util:temp_path", - "//test/util:test_main", - "//test/util:test_util", - "@com_google_absl//absl/strings", - ], -) - -cc_binary( - name = "symlink_benchmark", - testonly = 1, - srcs = [ - "symlink_benchmark.cc", - ], - deps = [ - gbenchmark, - gtest, - "//test/util:fs_util", - "//test/util:temp_path", - "//test/util:test_main", - "//test/util:test_util", - "@com_google_absl//absl/strings", - ], -) - -cc_binary( - name = "mkdir_benchmark", - testonly = 1, - srcs = [ - "mkdir_benchmark.cc", - ], - deps = [ - gbenchmark, - gtest, - "//test/util:fs_util", - "//test/util:temp_path", - "//test/util:test_main", - "//test/util:test_util", - "@com_google_absl//absl/strings", - ], -) diff --git a/test/fuse/benchmark/mkdir_benchmark.cc b/test/fuse/benchmark/mkdir_benchmark.cc deleted file mode 100644 index 30759603e..000000000 --- a/test/fuse/benchmark/mkdir_benchmark.cc +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2020 The gVisor Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include - -#include "absl/strings/str_cat.h" -#include "benchmark/benchmark.h" -#include "gtest/gtest.h" -#include "test/util/fs_util.h" -#include "test/util/temp_path.h" -#include "test/util/test_util.h" - -namespace gvisor { -namespace testing { - -namespace { - -void BM_Mkdir(benchmark::State& state) { - const char* fuse_prefix = getenv("TEST_FUSEPRE"); - ASSERT_NE(fuse_prefix, nullptr); - - const TempPath top_dir = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); - std::string dir_path = top_dir.path(); - - int index = 0; - for (auto t : state) { - const std::string new_dir_path = absl::StrCat(dir_path, index); - ASSERT_THAT(mkdir(new_dir_path.c_str(), 0777), SyscallSucceeds()); - index++; - } -} - -BENCHMARK(BM_Mkdir)->Range(1, 128)->UseRealTime(); - -} // namespace - -} // namespace testing -} // namespace gvisor diff --git a/test/fuse/benchmark/open_benchmark.cc b/test/fuse/benchmark/open_benchmark.cc deleted file mode 100644 index 11c1c1c80..000000000 --- a/test/fuse/benchmark/open_benchmark.cc +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright 2020 The gVisor Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include - -#include -#include -#include - -#include "benchmark/benchmark.h" -#include "gtest/gtest.h" -#include "test/util/fs_util.h" -#include "test/util/logging.h" -#include "test/util/temp_path.h" - -namespace gvisor { -namespace testing { - -namespace { - -void BM_Open(benchmark::State& state) { - const char* fuse_prefix = getenv("TEST_FUSEPRE"); - ASSERT_NE(fuse_prefix, nullptr); - - const int size = state.range(0); - std::vector cache; - for (int i = 0; i < size; i++) { - auto path = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); - cache.emplace_back(std::move(path)); - } - - unsigned int seed = 1; - for (auto _ : state) { - const int chosen = rand_r(&seed) % size; - const std::string file_path = JoinPath(fuse_prefix, cache[chosen].path()); - int fd = open(file_path.c_str(), O_RDONLY); - TEST_CHECK(fd != -1); - close(fd); - } -} - -BENCHMARK(BM_Open)->Range(1, 128)->UseRealTime(); - -} // namespace - -} // namespace testing -} // namespace gvisor diff --git a/test/fuse/benchmark/read_benchmark.cc b/test/fuse/benchmark/read_benchmark.cc deleted file mode 100644 index 2106b7d5a..000000000 --- a/test/fuse/benchmark/read_benchmark.cc +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright 2020 The gVisor Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include -#include - -#include "benchmark/benchmark.h" -#include "gtest/gtest.h" -#include "test/util/fs_util.h" -#include "test/util/logging.h" -#include "test/util/temp_path.h" -#include "test/util/test_util.h" - -namespace gvisor { -namespace testing { - -namespace { - -void BM_Read(benchmark::State& state) { - const char* fuse_prefix = getenv("TEST_FUSEPRE"); - ASSERT_NE(fuse_prefix, nullptr); - - const int size = state.range(0); - const std::string contents(size, 0); - auto path = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFileWith( - GetAbsoluteTestTmpdir(), contents, TempPath::kDefaultFileMode)); - FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE( - Open(JoinPath(fuse_prefix, path.path()), O_RDONLY)); - - std::vector buf(size); - for (auto _ : state) { - TEST_CHECK(PreadFd(fd.get(), buf.data(), buf.size(), 0) == size); - } - - state.SetBytesProcessed(static_cast(size) * - static_cast(state.iterations())); -} - -BENCHMARK(BM_Read)->Range(1, 1 << 26)->UseRealTime(); - -} // namespace - -} // namespace testing -} // namespace gvisor diff --git a/test/fuse/benchmark/stat_benchmark.cc b/test/fuse/benchmark/stat_benchmark.cc deleted file mode 100644 index d2ab6a706..000000000 --- a/test/fuse/benchmark/stat_benchmark.cc +++ /dev/null @@ -1,65 +0,0 @@ -// Copyright 2020 The gVisor Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include - -#include "gtest/gtest.h" -#include "absl/strings/str_cat.h" -#include "benchmark/benchmark.h" -#include "test/util/fs_util.h" -#include "test/util/temp_path.h" -#include "test/util/test_util.h" - -namespace gvisor { -namespace testing { - -namespace { - -// Creates a file in a nested directory hierarchy at least `depth` directories -// deep, and stats that file multiple times. -void BM_Stat(benchmark::State& state) { - const char* fuse_prefix = getenv("TEST_FUSEPRE"); - ASSERT_NE(fuse_prefix, nullptr); - - // Create nested directories with given depth. - int depth = state.range(0); - const TempPath top_dir = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); - std::string dir_path = top_dir.path(); - - while (depth-- > 0) { - // Don't use TempPath because it will make paths too long to use. - // - // The top_dir destructor will clean up this whole tree. - dir_path = JoinPath(dir_path, absl::StrCat(depth)); - ASSERT_NO_ERRNO(Mkdir(dir_path, 0755)); - } - - // Create the file that will be stat'd. - const TempPath file = - ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFileIn(dir_path)); - std::string file_path = JoinPath(fuse_prefix, file.path()); - struct stat st; - for (auto _ : state) { - ASSERT_THAT(stat(file_path.c_str(), &st), SyscallSucceeds()); - } -} - -BENCHMARK(BM_Stat)->Range(1, 100)->UseRealTime(); - -} // namespace - -} // namespace testing -} // namespace gvisor diff --git a/test/fuse/benchmark/symlink_benchmark.cc b/test/fuse/benchmark/symlink_benchmark.cc deleted file mode 100644 index 363b9a976..000000000 --- a/test/fuse/benchmark/symlink_benchmark.cc +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright 2020 The gVisor Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include - -#include "absl/strings/str_cat.h" -#include "benchmark/benchmark.h" -#include "gtest/gtest.h" -#include "test/util/fs_util.h" -#include "test/util/temp_path.h" -#include "test/util/test_util.h" - -namespace gvisor { -namespace testing { - -namespace { - -void BM_Symlink(benchmark::State& state) { - char* fuse_prefix = getenv("TEST_FUSEPRE"); - ASSERT_NE(fuse_prefix, nullptr); - const TempPath top_dir = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateDir()); - std::string dir_path = top_dir.path(); - - const int size = state.range(0); - std::vector cache; - for (int i = 0; i < size; i++) { - auto path = ASSERT_NO_ERRNO_AND_VALUE(TempPath::CreateFile()); - cache.emplace_back(std::move(path)); - } - - int index = 0; - unsigned int seed = 1; - for (auto t : state) { - const int chosen = rand_r(&seed) % size; - const std::string symlink_path = absl::StrCat(fuse_prefix, dir_path, index); - ASSERT_THAT(symlink(cache[chosen].path().c_str(), symlink_path.c_str()), - SyscallSucceeds()); - index++; - } -} - -BENCHMARK(BM_Symlink)->Range(1, 128)->UseRealTime(); - -} // namespace - -} // namespace testing -} // namespace gvisor diff --git a/test/image/BUILD b/test/image/BUILD index e270c52ac..e749e47d4 100644 --- a/test/image/BUILD +++ b/test/image/BUILD @@ -24,7 +24,6 @@ go_test( deps = [ "//pkg/test/dockerutil", "//pkg/test/testutil", - "@com_github_docker_docker//api/types/mount:go_default_library", ], ) diff --git a/test/image/image_test.go b/test/image/image_test.go index 6b5928ef0..ac6186688 100644 --- a/test/image/image_test.go +++ b/test/image/image_test.go @@ -33,7 +33,6 @@ import ( "testing" "time" - "github.com/docker/docker/api/types/mount" "gvisor.dev/gvisor/pkg/test/dockerutil" "gvisor.dev/gvisor/pkg/test/testutil" ) @@ -64,57 +63,6 @@ func TestHelloWorld(t *testing.T) { } } -// Test that the FUSE container is set up and being used properly. -func TestFUSEInContainer(t *testing.T) { - if usingFUSE, err := dockerutil.UsingFUSE(); err != nil { - t.Fatalf("failed to read config for runtime %s: %v", dockerutil.Runtime(), err) - } else if !usingFUSE { - t.Skip("FUSE not being used.") - } - - ctx := context.Background() - d := dockerutil.MakeContainer(ctx, t) - defer d.CleanUp(ctx) - - tmpDir := "/tmpDir/" - // Run the basic container. - err := d.Spawn(ctx, dockerutil.RunOpts{ - Image: "basic/fuse", - Privileged: true, - CapAdd: []string{"CAP_SYS_ADMIN"}, - - // Mount a tmpfs directory for benchmark. - Mounts: []mount.Mount{ - { - Type: mount.TypeTmpfs, - Target: tmpDir, - ReadOnly: false, - }, - }, - }, "sleep", "1000") - if err != nil { - t.Fatalf("docker spawn failed: %v", err) - } - - out, err := d.Exec(ctx, dockerutil.ExecOpts{ - Privileged: true, - }, "/bin/sh", "-c", "ls") - if err != nil { - t.Fatalf("docker exec failed: %v, message %s", err, out) - } - if !strings.Contains(out, "server-bin") { - t.Fatalf("docker didn't find server binary: got %s", out) - } - - // Run the server. - out, err = d.Exec(ctx, dockerutil.ExecOpts{ - Privileged: true, - }, "/bin/sh", "-c", "./server-bin mountpoint") - if err != nil { - t.Fatalf("docker exec failed: %v, message %s", err, out) - } -} - func runHTTPRequest(port int) error { url := fmt.Sprintf("http://localhost:%d/not-found", port) resp, err := http.Get(url) diff --git a/test/runner/BUILD b/test/runner/BUILD index 049c26081..582d2946d 100644 --- a/test/runner/BUILD +++ b/test/runner/BUILD @@ -11,14 +11,11 @@ go_binary( ], visibility = ["//:sandbox"], deps = [ - "//pkg/context", "//pkg/log", - "//pkg/test/dockerutil", "//pkg/test/testutil", "//runsc/specutils", "//test/runner/gtest", "//test/uds", - "@com_github_docker_docker//api/types/mount:go_default_library", "@com_github_opencontainers_runtime_spec//specs-go:go_default_library", "@com_github_syndtr_gocapability//capability:go_default_library", "@org_golang_x_sys//unix:go_default_library", diff --git a/test/runner/defs.bzl b/test/runner/defs.bzl index 9dc955c77..032ebd04e 100644 --- a/test/runner/defs.bzl +++ b/test/runner/defs.bzl @@ -57,8 +57,6 @@ def _syscall_test( platform, use_tmpfs, tags, - use_image = "", - setup_command = "", network = "none", file_access = "exclusive", overlay = False, @@ -81,8 +79,6 @@ def _syscall_test( name += "_fuse" if network != "none": name += "_" + network + "net" - if use_image != "": - name += "_container" # Apply all tags. if tags == None: @@ -111,8 +107,6 @@ def _syscall_test( "--platform=" + platform, "--network=" + network, "--use-tmpfs=" + str(use_tmpfs), - "--use-image=" + use_image, - "--setup-command=" + setup_command, "--file-access=" + file_access, "--overlay=" + str(overlay), "--add-uds-tree=" + str(add_uds_tree), @@ -138,8 +132,6 @@ def syscall_test( shard_count = 5, size = "small", use_tmpfs = False, - use_image = "", - setup_command = "", add_overlay = False, add_uds_tree = False, add_hostinet = False, @@ -154,8 +146,6 @@ def syscall_test( shard_count: shards for defined tests. size: the defined test size. use_tmpfs: use tmpfs in the defined tests. - use_image: use specified docker image in the defined tests. - setup_command: command to set up the docker container. Should be used when ise_image is. add_overlay: add an overlay test. add_uds_tree: add a UDS test. add_hostinet: add a hostinet test. @@ -188,26 +178,8 @@ def syscall_test( vfs2 = True, fuse = fuse, ) - - if use_image != "": - # Run the test in the container specified. - _syscall_test( - test = test, - shard_count = shard_count, - size = size, - platform = default_platform, - use_tmpfs = use_tmpfs, - use_image = use_image, - setup_command = setup_command, - add_uds_tree = add_uds_tree, - tags = platforms[default_platform] + vfs2_tags, - vfs2 = True, - fuse = True, - ) - if fuse: # Only generate *_vfs2_fuse target if fuse parameter is enabled. - # The rest of the targets don't support FUSE as of yet. return _syscall_test( diff --git a/test/runner/runner.go b/test/runner/runner.go index fe501c4c7..22d535f8d 100644 --- a/test/runner/runner.go +++ b/test/runner/runner.go @@ -23,20 +23,16 @@ import ( "os" "os/exec" "os/signal" - "path" "path/filepath" "strings" "syscall" "testing" "time" - "github.com/docker/docker/api/types/mount" specs "github.com/opencontainers/runtime-spec/specs-go" "github.com/syndtr/gocapability/capability" "golang.org/x/sys/unix" - "gvisor.dev/gvisor/pkg/context" "gvisor.dev/gvisor/pkg/log" - "gvisor.dev/gvisor/pkg/test/dockerutil" "gvisor.dev/gvisor/pkg/test/testutil" "gvisor.dev/gvisor/runsc/specutils" "gvisor.dev/gvisor/test/runner/gtest" @@ -44,19 +40,17 @@ import ( ) var ( - debug = flag.Bool("debug", false, "enable debug logs") - strace = flag.Bool("strace", false, "enable strace logs") - platform = flag.String("platform", "ptrace", "platform to run on") - network = flag.String("network", "none", "network stack to run on (sandbox, host, none)") - useTmpfs = flag.Bool("use-tmpfs", false, "mounts tmpfs for /tmp") - useImage = flag.String("use-image", "", "container image to use for test. Path relative to //images") - setupCommand = flag.String("setup-command", "", "command to run before running the test to set up container environment") - fileAccess = flag.String("file-access", "exclusive", "mounts root in exclusive or shared mode") - overlay = flag.Bool("overlay", false, "wrap filesystem mounts with writable tmpfs overlay") - vfs2 = flag.Bool("vfs2", false, "enable VFS2") - fuse = flag.Bool("fuse", false, "enable FUSE") - parallel = flag.Bool("parallel", false, "run tests in parallel") - runscPath = flag.String("runsc", "", "path to runsc binary") + debug = flag.Bool("debug", false, "enable debug logs") + strace = flag.Bool("strace", false, "enable strace logs") + platform = flag.String("platform", "ptrace", "platform to run on") + network = flag.String("network", "none", "network stack to run on (sandbox, host, none)") + useTmpfs = flag.Bool("use-tmpfs", false, "mounts tmpfs for /tmp") + fileAccess = flag.String("file-access", "exclusive", "mounts root in exclusive or shared mode") + overlay = flag.Bool("overlay", false, "wrap filesystem mounts with writable tmpfs overlay") + vfs2 = flag.Bool("vfs2", false, "enable VFS2") + fuse = flag.Bool("fuse", false, "enable FUSE") + parallel = flag.Bool("parallel", false, "run tests in parallel") + runscPath = flag.String("runsc", "", "path to runsc binary") addUDSTree = flag.Bool("add-uds-tree", false, "expose a tree of UDS utilities for use in tests") ) @@ -319,92 +313,8 @@ func setupUDSTree(spec *specs.Spec) (cleanup func(), err error) { return cleanup, nil } -func runTestCaseInContainer(testBin string, tc gtest.TestCase, image string, t *testing.T) { - if usingFUSE, err := dockerutil.UsingFUSE(); err != nil { - t.Fatalf("failed to read config for runtime %s: %v", dockerutil.Runtime(), err) - } else if !usingFUSE { - t.Skip("FUSE not being used.") - } - - ctx := context.Background() - d := dockerutil.MakeContainer(ctx, t) - defer d.CleanUp(ctx) - - // Run the basic container. - tmpDir := "/tmpDir" - testBinDir := "/testDir" - opts := dockerutil.RunOpts{ - Image: image, - Privileged: true, - CapAdd: []string{"CAP_SYS_ADMIN"}, - - // Mount a tmpfs directory to use when benchmarking. - Mounts: []mount.Mount{ - { - Type: mount.TypeTmpfs, - Target: tmpDir, - ReadOnly: false, - }, - }, - Env: []string{ - fmt.Sprintf("TEST_TMPDIR=%s", tmpDir), - fmt.Sprintf("TEST_FUSEPRE=%s", "/fus/mountpoint"), - }, - } - wd, err := os.Getwd() - if err != nil { - t.Fatalf("Getwd run failed: %v", err) - } - - wdPathToTestBin := strings.TrimPrefix(testBin, wd) - containerTestBin := path.Join(testBinDir, path.Base(wdPathToTestBin)) - d.CopyFiles(&opts, testBinDir, wdPathToTestBin) - if err := d.CopyErr(); err != nil { - t.Fatalf("Copy failed %v", err) - } - - err = d.Spawn(ctx, opts, "sleep", "1000") - if err != nil { - t.Fatalf("docker run failed: %v", err) - } - - // Run the server setup command. - if *setupCommand != "" { - out, err := d.Exec(ctx, dockerutil.ExecOpts{ - Privileged: true, - }, "/bin/sh", "-c", *setupCommand) - if err != nil { - t.Fatalf("docker exec failed: %v with output %v", err, out) - } - } - - cmd := "chmod +x " + containerTestBin - out, err := d.Exec(ctx, dockerutil.ExecOpts{ - Privileged: true, - }, "/bin/sh", "-c", cmd) - if err != nil { - t.Fatalf("docker exec failed: %v with output %v", err, out) - } - - cmd = containerTestBin + " " + strings.Join(tc.Args(), " ") - out, err = d.Exec(ctx, dockerutil.ExecOpts{ - Privileged: true, - }, "/bin/sh", "-c", cmd) - if err != nil { - t.Fatalf("docker exec failed: %v with output %v", err, out) - } - - fmt.Print(out) - return -} - // runsTestCaseRunsc runs the test case in runsc. func runTestCaseRunsc(testBin string, tc gtest.TestCase, t *testing.T) { - if *useImage != "" { - runTestCaseInContainer(testBin, tc, *useImage, t) - return - } - // Run a new container with the test executable and filter for the // given test suite and name. spec := testutil.NewSpecWithArgs(append([]string{testBin}, tc.Args()...)...) -- cgit v1.2.3 From 92a020c798fe237fb7038f3da71f24c5a88bb744 Mon Sep 17 00:00:00 2001 From: Andrei Vagin Date: Tue, 15 Sep 2020 13:31:40 -0700 Subject: fuse: fix a compile time error readdir_test.cc:134:24: error: variable length arrays are a C99 feature [-Werror,-Wvla-extension] char readdir_payload[readdir_payload_size]; --- pkg/sentry/vfs/file_description_impl_util.go | 2 +- test/fuse/linux/readdir_test.cc | 9 ++++----- 2 files changed, 5 insertions(+), 6 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/vfs/file_description_impl_util.go b/pkg/sentry/vfs/file_description_impl_util.go index 2b668fd89..68b80a951 100644 --- a/pkg/sentry/vfs/file_description_impl_util.go +++ b/pkg/sentry/vfs/file_description_impl_util.go @@ -107,7 +107,7 @@ func (FileDescriptionDefaultImpl) Write(ctx context.Context, src usermem.IOSeque // file_operations::iterate == file_operations::iterate_shared == NULL in // Linux. func (FileDescriptionDefaultImpl) IterDirents(ctx context.Context, cb IterDirentsCallback) error { - return syserror.ENOSYS + return syserror.ENOTDIR } // Seek implements FileDescriptionImpl.Seek analogously to diff --git a/test/fuse/linux/readdir_test.cc b/test/fuse/linux/readdir_test.cc index 17fb630ee..ed11e2ec9 100644 --- a/test/fuse/linux/readdir_test.cc +++ b/test/fuse/linux/readdir_test.cc @@ -131,7 +131,8 @@ TEST_F(ReaddirTest, SingleEntry) { // Create an appropriately sized payload. size_t readdir_payload_size = test_file_dirent_size + dot_file_dirent_size + dot_dot_file_dirent_size; - char readdir_payload[readdir_payload_size]; + std::vector readdir_payload_vec(readdir_payload_size); + char *readdir_payload = readdir_payload_vec.data(); fill_fuse_dirent(readdir_payload, dot.c_str()); fill_fuse_dirent(readdir_payload + dot_file_dirent_size, dot_dot.c_str()); @@ -139,10 +140,8 @@ TEST_F(ReaddirTest, SingleEntry) { readdir_payload + dot_file_dirent_size + dot_dot_file_dirent_size, test_file.c_str()); - std::vector readdir_payload_vec(readdir_payload, - readdir_payload + readdir_payload_size); struct fuse_out_header readdir_header = { - .len = uint32_t(sizeof(struct fuse_out_header) + sizeof(readdir_payload)), + .len = uint32_t(sizeof(struct fuse_out_header) + readdir_payload_size), }; struct fuse_out_header readdir_header_break = { .len = uint32_t(sizeof(struct fuse_out_header)), @@ -185,4 +184,4 @@ TEST_F(ReaddirTest, SingleEntry) { } // namespace } // namespace testing -} // namespace gvisor \ No newline at end of file +} // namespace gvisor -- cgit v1.2.3 From 3ea925a423ed2c195498fac63b200b8a1a302f4c Mon Sep 17 00:00:00 2001 From: Andrei Vagin Date: Tue, 15 Sep 2020 13:51:18 -0700 Subject: fuse: don't pass lock by value copylocks: directory.go:34:7: Allocate passes lock by value: fuse/fuse.directoryFD contains fuse/fuse.fileDescription contains pkg/sentry/vfs/vfs.FileDescription contains pkg/sync/sync.Mutex --- pkg/sentry/fsimpl/fuse/directory.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/directory.go b/pkg/sentry/fsimpl/fuse/directory.go index 798c4a6f3..ee77dbaa3 100644 --- a/pkg/sentry/fsimpl/fuse/directory.go +++ b/pkg/sentry/fsimpl/fuse/directory.go @@ -31,27 +31,27 @@ type directoryFD struct { } // Allocate implements directoryFD.Allocate. -func (directoryFD) Allocate(ctx context.Context, mode, offset, length uint64) error { +func (*directoryFD) Allocate(ctx context.Context, mode, offset, length uint64) error { return syserror.EISDIR } // PRead implements FileDescriptionImpl.PRead. -func (directoryFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) { +func (*directoryFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) { return 0, syserror.EISDIR } // Read implements FileDescriptionImpl.Read. -func (directoryFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { +func (*directoryFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { return 0, syserror.EISDIR } // PWrite implements FileDescriptionImpl.PWrite. -func (directoryFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) { +func (*directoryFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) { return 0, syserror.EISDIR } // Write implements FileDescriptionImpl.Write. -func (directoryFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { +func (*directoryFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { return 0, syserror.EISDIR } -- cgit v1.2.3 From 96fb1e60c355b330eaf42db3a14a828befe73db9 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Mon, 14 Sep 2020 23:16:56 +0000 Subject: Fix FUSE connection control lock ordering and race in unit test --- pkg/sentry/fsimpl/fuse/connection_control.go | 7 +++++-- pkg/sentry/fsimpl/fuse/connection_test.go | 28 +++++++++------------------- pkg/sentry/fsimpl/fuse/dev.go | 4 ++++ 3 files changed, 18 insertions(+), 21 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/connection_control.go b/pkg/sentry/fsimpl/fuse/connection_control.go index a63c66e7c..50cf4a761 100644 --- a/pkg/sentry/fsimpl/fuse/connection_control.go +++ b/pkg/sentry/fsimpl/fuse/connection_control.go @@ -95,6 +95,8 @@ func (conn *connection) InitSend(creds *auth.Credentials, pid uint32) error { } // InitRecv receives a FUSE_INIT reply and process it. +// +// Preconditions: conn.asyncMu must not be held if minor verion is newer than 13. func (conn *connection) InitRecv(res *Response, hasSysAdminCap bool) error { if err := res.Error(); err != nil { return err @@ -189,6 +191,8 @@ func (conn *connection) initProcessReply(out *linux.FUSEInitOut, hasSysAdminCap // All possible requests waiting or blocking will be aborted. func (conn *connection) Abort(ctx context.Context) { conn.fd.mu.Lock() + defer conn.fd.mu.Unlock() + conn.mu.Lock() conn.asyncMu.Lock() @@ -217,10 +221,9 @@ func (conn *connection) Abort(ctx context.Context) { terminate = append(terminate, unique) } - // Release all locks to avoid deadlock. + // Release locks to avoid deadlock. conn.asyncMu.Unlock() conn.mu.Unlock() - conn.fd.mu.Unlock() // 1. The requets blocked before initialization. // Will reach call() `connected` check and return. diff --git a/pkg/sentry/fsimpl/fuse/connection_test.go b/pkg/sentry/fsimpl/fuse/connection_test.go index 6aa77b36d..91d16c1cf 100644 --- a/pkg/sentry/fsimpl/fuse/connection_test.go +++ b/pkg/sentry/fsimpl/fuse/connection_test.go @@ -62,9 +62,9 @@ func TestConnectionAbort(t *testing.T) { creds := auth.CredentialsFromContext(s.Ctx) task := kernel.TaskFromContext(s.Ctx) - const maxActiveRequest uint64 = 10 + const numRequests uint64 = 256 - conn, _, err := newTestConnection(s, k, maxActiveRequest) + conn, _, err := newTestConnection(s, k, numRequests) if err != nil { t.Fatalf("newTestConnection: %v", err) } @@ -75,32 +75,22 @@ func TestConnectionAbort(t *testing.T) { var futNormal []*futureResponse - for i := 0; i < 2*int(maxActiveRequest); i++ { + for i := 0; i < int(numRequests); i++ { req, err := conn.NewRequest(creds, uint32(i), uint64(i), 0, testObj) if err != nil { t.Fatalf("NewRequest creation failed: %v", err) } - if i < int(maxActiveRequest) { - // Issue the requests that will not be blocked due to maxActiveRequest. - fut, err := conn.callFutureLocked(task, req) - if err != nil { - t.Fatalf("callFutureLocked failed: %v", err) - } - futNormal = append(futNormal, fut) - } else { - go func(t *testing.T) { - // The requests beyond maxActiveRequest will be blocked and receive expected errors. - _, err := conn.callFutureLocked(task, req) - if err != syserror.ECONNABORTED && err != syserror.ENOTCONN { - t.Fatalf("Incorrect error code received for blocked callFutureLocked() on aborted connection") - } - }(t) + fut, err := conn.callFutureLocked(task, req) + if err != nil { + t.Fatalf("callFutureLocked failed: %v", err) } + futNormal = append(futNormal, fut) } conn.Abort(s.Ctx) // Abort should unblock the initialization channel. + // Note: no test requests are actually blocked on `conn.initializedChan`. select { case <-conn.initializedChan: default: @@ -110,7 +100,7 @@ func TestConnectionAbort(t *testing.T) { // Abort will return ECONNABORTED error to unblocked requests. for _, fut := range futNormal { if fut.getResponse().hdr.Error != -int32(syscall.ECONNABORTED) { - t.Fatalf("Incorrect error code received for aborted connection") + t.Fatalf("Incorrect error code received for aborted connection: %v", fut.getResponse().hdr.Error) } } diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index 64c3e32e2..a19544fe9 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -391,6 +391,8 @@ func (fd *DeviceFD) Seek(ctx context.Context, offset int64, whence int32) (int64 } // sendResponse sends a response to the waiting task (if any). +// +// Preconditions: fd.mu must be held. func (fd *DeviceFD) sendResponse(ctx context.Context, fut *futureResponse) error { // Signal the task waiting on a response if any. defer close(fut.ch) @@ -410,6 +412,8 @@ func (fd *DeviceFD) sendResponse(ctx context.Context, fut *futureResponse) error } // sendError sends an error response to the waiting task (if any) by calling sendResponse(). +// +// Preconditions: fd.mu must be held. func (fd *DeviceFD) sendError(ctx context.Context, errno int32, unique linux.FUSEOpID) error { // Return the error to the calling task. outHdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) -- cgit v1.2.3 From 5c080f07deea6ae18ec026b394d9ea5d76ab4566 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Wed, 16 Sep 2020 01:38:22 +0000 Subject: Update fuse.md design doc with design details --- pkg/sentry/fs/g3doc/fuse.md | 47 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 47 insertions(+) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fs/g3doc/fuse.md b/pkg/sentry/fs/g3doc/fuse.md index 2ca84dd74..b102c9b34 100644 --- a/pkg/sentry/fs/g3doc/fuse.md +++ b/pkg/sentry/fs/g3doc/fuse.md @@ -95,6 +95,53 @@ ops can be implemented in parallel. - Implement the remaining FUSE ops and decide if we can omit rarely used operations like ioctl. +### Design Details + +#### Lifecycle for a FUSE Request + +- User invokes a syscall +- Sentry prepares corresponding request + - If FUSE device is available + - Write the request in binary + - If FUSE device is full + - Kernel task blocked until available +- Sentry notifies the readers of fuse device that it's ready for read +- FUSE daemon reads the request and processes it +- Sentry waits until a reply is written to the FUSE device + - but returns directly for async requests +- FUSE daemon writes to the fuse device +- Sentry processes the reply + - For sync requests, unblock blocked kernel task + - For async requests, execute pre-specified callback if any +- Sentry returns the syscall to the user + +#### Channels and Queues for Requests in Different Stages + +`connection.initializedChan` + +- a channel that the requests issued before connection initialization + blocks on. + +`fd.queue` + +- a queue of requests that haven’t been read by the FUSE daemon yet. + +`fd.completions` + +- a map of the requests that have been prepared but + not yet received a response, including the ones on the + `fd.queue`. + +`fd.waitQueue` + +- a queue of waiters that is waiting for the fuse device fd to be available, + such as the FUSE daemon. + +`fd.fullQueueCh` + +- a channel that the kernel task will be blocked on when the fd is not + available. + # Appendix ## FUSE Protocol -- cgit v1.2.3 From 70cf503b4c3653df8253438a8873a5d3ebb688e3 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Wed, 16 Sep 2020 17:39:55 +0000 Subject: fuse: fix FUSE_RELEASE reply handling fix #3963 --- pkg/sentry/fsimpl/fuse/connection.go | 2 +- pkg/sentry/fsimpl/fuse/dev.go | 4 ---- pkg/sentry/fsimpl/fuse/file.go | 2 +- 3 files changed, 2 insertions(+), 6 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index a7402c149..dbc5e1954 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -110,7 +110,7 @@ type connection struct { // // We call the "background" requests in unix term as async requests. // The "async requests" in unix term is our async requests that expect a reply, - // i.e. `!requestOptions.noReply` + // i.e. `!request.noReply` // asyncMu protects the async request fields. asyncMu sync.Mutex diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index a19544fe9..5539466ff 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -309,10 +309,6 @@ func (fd *DeviceFD) writeLocked(ctx context.Context, src usermem.IOSequence, opt fut, ok := fd.completions[hdr.Unique] if !ok { - if fut.hdr.Unique == linux.FUSE_RELEASE { - // Currently we simply discard the reply for FUSE_RELEASE. - return n + src.NumBytes(), nil - } // Server sent us a response for a request we never sent, // or for which we already received a reply (e.g. aborted), an unlikely event. return 0, syserror.EINVAL diff --git a/pkg/sentry/fsimpl/fuse/file.go b/pkg/sentry/fsimpl/fuse/file.go index 991efcda4..83f2816b7 100644 --- a/pkg/sentry/fsimpl/fuse/file.go +++ b/pkg/sentry/fsimpl/fuse/file.go @@ -89,7 +89,7 @@ func (fd *fileDescription) Release(ctx context.Context) { // No way to invoke Call() with an errored request. return } - req.noReply = true + // The reply will be ignored since no callback is defined in asyncCallBack(). conn.CallAsync(kernelTask, req) } -- cgit v1.2.3 From c4c302a27e5811ce04bb904686e657c9ed830a44 Mon Sep 17 00:00:00 2001 From: Jinmou Li Date: Wed, 16 Sep 2020 17:44:06 +0000 Subject: fuse: fix data race in fusefs Release() fix #3956 --- pkg/sentry/fsimpl/fuse/connection_control.go | 5 ++--- pkg/sentry/fsimpl/fuse/fusefs.go | 5 +++-- 2 files changed, 5 insertions(+), 5 deletions(-) (limited to 'pkg/sentry') diff --git a/pkg/sentry/fsimpl/fuse/connection_control.go b/pkg/sentry/fsimpl/fuse/connection_control.go index 50cf4a761..bfde78559 100644 --- a/pkg/sentry/fsimpl/fuse/connection_control.go +++ b/pkg/sentry/fsimpl/fuse/connection_control.go @@ -189,10 +189,9 @@ func (conn *connection) initProcessReply(out *linux.FUSEInitOut, hasSysAdminCap // Abort this FUSE connection. // It tries to acquire conn.fd.mu, conn.lock, conn.bgLock in order. // All possible requests waiting or blocking will be aborted. +// +// Preconditions: conn.fd.mu is locked. func (conn *connection) Abort(ctx context.Context) { - conn.fd.mu.Lock() - defer conn.fd.mu.Unlock() - conn.mu.Lock() conn.asyncMu.Lock() diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index 86922a96a..f1ffd2343 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -225,12 +225,13 @@ func newFUSEFilesystem(ctx context.Context, devMinor uint32, opts *filesystemOpt // Release implements vfs.FilesystemImpl.Release. func (fs *filesystem) Release(ctx context.Context) { + fs.conn.fd.mu.Lock() + fs.umounted = true fs.conn.Abort(ctx) - - fs.conn.fd.mu.Lock() // Notify all the waiters on this fd. fs.conn.fd.waitQueue.Notify(waiter.EventIn) + fs.conn.fd.mu.Unlock() fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor) -- cgit v1.2.3