From fc1196ee65a53004e3e8b97ec5d57404552b90b8 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/sentry/fsimpl/fuse/connection.go | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) (limited to 'pkg/sentry/fsimpl/fuse/connection.go') diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 6df2728ab..7d3c30116 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. -- cgit v1.2.3 From 74e229c56ceb488a61a1b42d8f7da2d58c3c5418 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/fsimpl/fuse/connection.go') 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 2d9350d57..a6ee6100d 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"], @@ -45,6 +47,7 @@ go_library( "//pkg/context", "//pkg/log", "//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 7d3c30116..a6525249d 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 eccdd440899113c229f4abea53c03364d7f9875c 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 | 9 +- 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(+), 9 deletions(-) create mode 100644 test/fuse/linux/readdir_test.cc (limited to 'pkg/sentry/fsimpl/fuse/connection.go') 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 a6525249d..f1a5c2ecb 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" @@ -29,7 +31,6 @@ import ( "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/waiter" - "gvisor.dev/gvisor/tools/go_marshal/marshal" ) // maxActiveRequestsDefault is the default setting controlling the upper bound @@ -352,6 +353,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 0f2a8b08f13cc39d924c328bc7a9bab73c1c2328 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/sentry/fsimpl/fuse/connection.go | 9 ++- tools/go_marshal/marshal/marshal_impl_util.go | 2 +- 3 files changed, 103 insertions(+), 5 deletions(-) (limited to 'pkg/sentry/fsimpl/fuse/connection.go') 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/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index f1a5c2ecb..f7d1a5c52 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -270,8 +270,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, @@ -359,7 +361,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 } diff --git a/tools/go_marshal/marshal/marshal_impl_util.go b/tools/go_marshal/marshal/marshal_impl_util.go index 89c7d3575..1724652f7 100644 --- a/tools/go_marshal/marshal/marshal_impl_util.go +++ b/tools/go_marshal/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. -- cgit v1.2.3 From 3bd85840c8f0364083c88d65c2bc1f968069b04e 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/fsimpl/fuse/connection.go') 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 f7d1a5c52..0e91bb18e 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -345,6 +345,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 8554fda1b86e1ae430f155c41a820e6ce6632057 Mon Sep 17 00:00:00 2001 From: jinmouil <67118279+jinmouil@users.noreply.github.com> 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/fsimpl/fuse/connection.go') 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 0e91bb18e..c6e064f70 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -85,16 +85,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. @@ -181,19 +187,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 77e3d54bae3197856535ea71ae4841e3360a1a28 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/fsimpl/fuse/connection.go') 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 c6e064f70..8dd86afad 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -63,6 +63,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 e67e30bdce9e7ed910c1cb136b4c82223b1d7f8c 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/fsimpl/fuse/connection.go') diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index 86dc6a7a4..23660a708 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 8dd86afad..9d72b927f 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -16,31 +16,17 @@ 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/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 @@ -53,33 +39,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 @@ -114,10 +78,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, @@ -125,51 +85,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. @@ -187,6 +148,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 @@ -224,63 +190,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 } @@ -336,43 +252,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) { @@ -421,50 +300,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 aedc2fa39..936683454 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 84acc461cda70348a70992aac984c0e6e2b1621f 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/fsimpl/fuse/connection.go') diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 9d72b927f..122b7d92f 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" @@ -200,42 +199,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 } @@ -270,11 +267,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() @@ -291,8 +283,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 33d324792fecf11f8088d14bffb704805a3df38d Mon Sep 17 00:00:00 2001 From: jinmouil <67118279+jinmouil@users.noreply.github.com> 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/fsimpl/fuse/connection.go') diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index 23660a708..77025772c 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 122b7d92f..6009cdf97 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -179,7 +179,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()) @@ -282,6 +281,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 936683454..3da8b469b 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 b8bee78d0c99a0c9423061b3dda10b9ef8c9719d 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/fsimpl/fuse/connection.go') diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go index 6009cdf97..a7402c149 100644 --- a/pkg/sentry/fsimpl/fuse/connection.go +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -46,7 +46,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