From 492229d0176c1af2ab4ea4cf91bf211e940b5b12 Mon Sep 17 00:00:00 2001 From: Jamie Liu Date: Tue, 4 Feb 2020 11:28:36 -0800 Subject: VFS2 gofer client Updates #1198 Opening host pipes (by spinning in fdpipe) and host sockets is not yet complete, and will be done in a future CL. Major differences from VFS1 gofer client (sentry/fs/gofer), with varying levels of backportability: - "Cache policies" are replaced by InteropMode, which control the behavior of timestamps in addition to caching. Under InteropModeExclusive (analogous to cacheAll) and InteropModeWritethrough (analogous to cacheAllWritethrough), client timestamps are *not* written back to the server (it is not possible in 9P or Linux for clients to set ctime, so writing back client-authoritative timestamps results in incoherence between atime/mtime and ctime). Under InteropModeShared (analogous to cacheRemoteRevalidating), client timestamps are not used at all (remote filesystem clocks are authoritative). cacheNone is translated to InteropModeShared + new option filesystemOptions.specialRegularFiles. - Under InteropModeShared, "unstable attribute" reloading for permission checks, lookup, and revalidation are fused, which is feasible in VFS2 since gofer.filesystem controls path resolution. This results in a ~33% reduction in RPCs for filesystem operations compared to cacheRemoteRevalidating. For example, consider stat("/foo/bar/baz") where "/foo/bar/baz" fails revalidation, resulting in the instantiation of a new dentry: VFS1 RPCs: getattr("/") // fs.MountNamespace.FindLink() => fs.Inode.CheckPermission() => gofer.inodeOperations.check() => gofer.inodeOperations.UnstableAttr() walkgetattr("/", "foo") = fid1 // fs.Dirent.walk() => gofer.session.Revalidate() => gofer.cachePolicy.Revalidate() clunk(fid1) getattr("/foo") // CheckPermission walkgetattr("/foo", "bar") = fid2 // Revalidate clunk(fid2) getattr("/foo/bar") // CheckPermission walkgetattr("/foo/bar", "baz") = fid3 // Revalidate clunk(fid3) walkgetattr("/foo/bar", "baz") = fid4 // fs.Dirent.walk() => gofer.inodeOperations.Lookup getattr("/foo/bar/baz") // linux.stat() => gofer.inodeOperations.UnstableAttr() VFS2 RPCs: getattr("/") // gofer.filesystem.walkExistingLocked() walkgetattr("/", "foo") = fid1 // gofer.filesystem.stepExistingLocked() clunk(fid1) // No getattr: walkgetattr already updated metadata for permission check walkgetattr("/foo", "bar") = fid2 clunk(fid2) walkgetattr("/foo/bar", "baz") = fid3 // No clunk: fid3 used for new gofer.dentry // No getattr: walkgetattr already updated metadata for stat() - gofer.filesystem.unlinkAt() does not require instantiation of a dentry that represents the file to be deleted. Updates #898. - gofer.regularFileFD.OnClose() skips Tflushf for regular files under InteropModeExclusive, as it's nonsensical to request a remote file flush without flushing locally-buffered writes to that remote file first. - Symlink targets are cached when InteropModeShared is not in effect. - p9.QID.Path (which is already required to be unique for each file within a server, and is accordingly already synthesized from device/inode numbers in all known gofers) is used as-is for inode numbers, rather than being mapped along with attr.RDev in the client to yet another synthetic inode number. - Relevant parts of fsutil.CachingInodeOperations are inlined directly into gofer package code. This avoids having to duplicate part of its functionality in fsutil.HostMappable. PiperOrigin-RevId: 293190213 --- pkg/sentry/fsimpl/gofer/gofer.go | 1147 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 1147 insertions(+) create mode 100644 pkg/sentry/fsimpl/gofer/gofer.go (limited to 'pkg/sentry/fsimpl/gofer/gofer.go') diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go new file mode 100644 index 000000000..d0552bd99 --- /dev/null +++ b/pkg/sentry/fsimpl/gofer/gofer.go @@ -0,0 +1,1147 @@ +// Copyright 2019 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package gofer provides a filesystem implementation that is backed by a 9p +// server, interchangably referred to as "gofers" throughout this package. +// +// Lock order: +// regularFileFD/directoryFD.mu +// filesystem.renameMu +// dentry.dirMu +// filesystem.syncMu +// dentry.metadataMu +// *** "memmap.Mappable locks" below this point +// dentry.mapsMu +// *** "memmap.Mappable locks taken by Translate" below this point +// dentry.handleMu +// dentry.dataMu +// +// Locking dentry.dirMu in multiple dentries requires holding +// filesystem.renameMu for writing. +package gofer + +import ( + "fmt" + "strconv" + "sync" + "sync/atomic" + "syscall" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/log" + "gvisor.dev/gvisor/pkg/p9" + "gvisor.dev/gvisor/pkg/sentry/fs/fsutil" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/sentry/memmap" + "gvisor.dev/gvisor/pkg/sentry/pgalloc" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/unet" + "gvisor.dev/gvisor/pkg/usermem" +) + +// FilesystemType implements vfs.FilesystemType. +type FilesystemType struct{} + +// filesystem implements vfs.FilesystemImpl. +type filesystem struct { + vfsfs vfs.Filesystem + + // mfp is used to allocate memory that caches regular file contents. mfp is + // immutable. + mfp pgalloc.MemoryFileProvider + + // Immutable options. + opts filesystemOptions + + // client is the client used by this filesystem. client is immutable. + client *p9.Client + + // uid and gid are the effective KUID and KGID of the filesystem's creator, + // and are used as the owner and group for files that don't specify one. + // uid and gid are immutable. + uid auth.KUID + gid auth.KGID + + // renameMu serves two purposes: + // + // - It synchronizes path resolution with renaming initiated by this + // client. + // + // - It is held by path resolution to ensure that reachable dentries remain + // valid. A dentry is reachable by path resolution if it has a non-zero + // reference count (such that it is usable as vfs.ResolvingPath.Start() or + // is reachable from its children), or if it is a child dentry (such that + // it is reachable from its parent). + renameMu sync.RWMutex + + // cachedDentries contains all dentries with 0 references. (Due to race + // conditions, it may also contain dentries with non-zero references.) + // cachedDentriesLen is the number of dentries in cachedDentries. These + // fields are protected by renameMu. + cachedDentries dentryList + cachedDentriesLen uint64 + + // dentries contains all dentries in this filesystem. specialFileFDs + // contains all open specialFileFDs. These fields are protected by syncMu. + syncMu sync.Mutex + dentries map[*dentry]struct{} + specialFileFDs map[*specialFileFD]struct{} +} + +type filesystemOptions struct { + // "Standard" 9P options. + fd int + aname string + interop InteropMode // derived from the "cache" mount option + msize uint32 + version string + + // maxCachedDentries is the maximum number of dentries with 0 references + // retained by the client. + maxCachedDentries uint64 + + // If forcePageCache is true, host FDs may not be used for application + // memory mappings even if available; instead, the client must perform its + // own caching of regular file pages. This is primarily useful for testing. + forcePageCache bool + + // If limitHostFDTranslation is true, apply maxFillRange() constraints to + // host FD mappings returned by dentry.(memmap.Mappable).Translate(). This + // makes memory accounting behavior more consistent between cases where + // host FDs are / are not available, but may increase the frequency of + // sentry-handled page faults on files for which a host FD is available. + limitHostFDTranslation bool + + // If overlayfsStaleRead is true, O_RDONLY host FDs provided by the remote + // filesystem may not be coherent with writable host FDs opened later, so + // mappings of the former must be replaced by mappings of the latter. This + // is usually only the case when the remote filesystem is an overlayfs + // mount on Linux < 4.19. + overlayfsStaleRead bool + + // If regularFilesUseSpecialFileFD is true, application FDs representing + // regular files will use distinct file handles for each FD, in the same + // way that application FDs representing "special files" such as sockets + // do. Note that this disables client caching and mmap for regular files. + regularFilesUseSpecialFileFD bool +} + +// InteropMode controls the client's interaction with other remote filesystem +// users. +type InteropMode uint32 + +const ( + // InteropModeExclusive is appropriate when the filesystem client is the + // only user of the remote filesystem. + // + // - The client may cache arbitrary filesystem state (file data, metadata, + // filesystem structure, etc.). + // + // - Client changes to filesystem state may be sent to the remote + // filesystem asynchronously, except when server permission checks are + // necessary. + // + // - File timestamps are based on client clocks. This ensures that users of + // the client observe timestamps that are coherent with their own clocks + // and consistent with Linux's semantics. However, since it is not always + // possible for clients to set arbitrary atimes and mtimes, and never + // possible for clients to set arbitrary ctimes, file timestamp changes are + // stored in the client only and never sent to the remote filesystem. + InteropModeExclusive InteropMode = iota + + // InteropModeWritethrough is appropriate when there are read-only users of + // the remote filesystem that expect to observe changes made by the + // filesystem client. + // + // - The client may cache arbitrary filesystem state. + // + // - Client changes to filesystem state must be sent to the remote + // filesystem synchronously. + // + // - File timestamps are based on client clocks. As a corollary, access + // timestamp changes from other remote filesystem users will not be visible + // to the client. + InteropModeWritethrough + + // InteropModeShared is appropriate when there are users of the remote + // filesystem that may mutate its state other than the client. + // + // - The client must verify cached filesystem state before using it. + // + // - Client changes to filesystem state must be sent to the remote + // filesystem synchronously. + // + // - File timestamps are based on server clocks. This is necessary to + // ensure that timestamp changes are synchronized between remote filesystem + // users. + // + // Note that the correctness of InteropModeShared depends on the server + // correctly implementing 9P fids (i.e. each fid immutably represents a + // single filesystem object), even in the presence of remote filesystem + // mutations from other users. If this is violated, the behavior of the + // client is undefined. + InteropModeShared +) + +// GetFilesystem implements vfs.FilesystemType.GetFilesystem. +func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) { + mfp := pgalloc.MemoryFileProviderFromContext(ctx) + if mfp == nil { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: context does not provide a pgalloc.MemoryFileProvider") + return nil, nil, syserror.EINVAL + } + + mopts := vfs.GenericParseMountOptions(opts.Data) + var fsopts filesystemOptions + + // Check that the transport is "fd". + trans, ok := mopts["trans"] + if !ok { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: transport must be specified as 'trans=fd'") + return nil, nil, syserror.EINVAL + } + delete(mopts, "trans") + if trans != "fd" { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: unsupported transport: trans=%s", trans) + return nil, nil, syserror.EINVAL + } + + // Check that read and write FDs are provided and identical. + rfdstr, ok := mopts["rfdno"] + if !ok { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: read FD must be specified as 'rfdno=") + return nil, nil, syserror.EINVAL + } + delete(mopts, "rfdno") + rfd, err := strconv.Atoi(rfdstr) + if err != nil { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: invalid read FD: rfdno=%s", rfdstr) + return nil, nil, syserror.EINVAL + } + wfdstr, ok := mopts["wfdno"] + if !ok { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: write FD must be specified as 'wfdno=") + return nil, nil, syserror.EINVAL + } + delete(mopts, "wfdno") + wfd, err := strconv.Atoi(wfdstr) + if err != nil { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: invalid write FD: wfdno=%s", wfdstr) + return nil, nil, syserror.EINVAL + } + if rfd != wfd { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: read FD (%d) and write FD (%d) must be equal", rfd, wfd) + return nil, nil, syserror.EINVAL + } + fsopts.fd = rfd + + // Get the attach name. + fsopts.aname = "/" + if aname, ok := mopts["aname"]; ok { + delete(mopts, "aname") + fsopts.aname = aname + } + + // Parse the cache policy. For historical reasons, this defaults to the + // least generally-applicable option, InteropModeExclusive. + fsopts.interop = InteropModeExclusive + if cache, ok := mopts["cache"]; ok { + delete(mopts, "cache") + switch cache { + case "fscache": + fsopts.interop = InteropModeExclusive + case "fscache_writethrough": + fsopts.interop = InteropModeWritethrough + case "none": + fsopts.regularFilesUseSpecialFileFD = true + fallthrough + case "remote_revalidating": + fsopts.interop = InteropModeShared + default: + ctx.Warningf("gofer.FilesystemType.GetFilesystem: invalid cache policy: cache=%s", cache) + return nil, nil, syserror.EINVAL + } + } + + // Parse the 9P message size. + fsopts.msize = 1024 * 1024 // 1M, tested to give good enough performance up to 64M + if msizestr, ok := mopts["msize"]; ok { + delete(mopts, "msize") + msize, err := strconv.ParseUint(msizestr, 10, 32) + if err != nil { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: invalid message size: msize=%s", msizestr) + return nil, nil, syserror.EINVAL + } + fsopts.msize = uint32(msize) + } + + // Parse the 9P protocol version. + fsopts.version = p9.HighestVersionString() + if version, ok := mopts["version"]; ok { + delete(mopts, "version") + fsopts.version = version + } + + // Parse the dentry cache limit. + fsopts.maxCachedDentries = 1000 + if str, ok := mopts["dentry_cache_limit"]; ok { + delete(mopts, "dentry_cache_limit") + maxCachedDentries, err := strconv.ParseUint(str, 10, 64) + if err != nil { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: invalid dentry cache limit: dentry_cache_limit=%s", str) + return nil, nil, syserror.EINVAL + } + fsopts.maxCachedDentries = maxCachedDentries + } + + // Handle simple flags. + if _, ok := mopts["force_page_cache"]; ok { + delete(mopts, "force_page_cache") + fsopts.forcePageCache = true + } + if _, ok := mopts["limit_host_fd_translation"]; ok { + delete(mopts, "limit_host_fd_translation") + fsopts.limitHostFDTranslation = true + } + if _, ok := mopts["overlayfs_stale_read"]; ok { + delete(mopts, "overlayfs_stale_read") + fsopts.overlayfsStaleRead = true + } + // fsopts.regularFilesUseSpecialFileFD can only be enabled by specifying + // "cache=none". + + // Check for unparsed options. + if len(mopts) != 0 { + ctx.Warningf("gofer.FilesystemType.GetFilesystem: unknown options: %v", mopts) + return nil, nil, syserror.EINVAL + } + + // Establish a connection with the server. + conn, err := unet.NewSocket(fsopts.fd) + if err != nil { + return nil, nil, err + } + + // Perform version negotiation with the server. + ctx.UninterruptibleSleepStart(false) + client, err := p9.NewClient(conn, fsopts.msize, fsopts.version) + ctx.UninterruptibleSleepFinish(false) + if err != nil { + conn.Close() + return nil, nil, err + } + // Ownership of conn has been transferred to client. + + // Perform attach to obtain the filesystem root. + ctx.UninterruptibleSleepStart(false) + attached, err := client.Attach(fsopts.aname) + ctx.UninterruptibleSleepFinish(false) + if err != nil { + client.Close() + return nil, nil, err + } + attachFile := p9file{attached} + qid, attrMask, attr, err := attachFile.getAttr(ctx, dentryAttrMask()) + if err != nil { + attachFile.close(ctx) + client.Close() + return nil, nil, err + } + + // Construct the filesystem object. + fs := &filesystem{ + mfp: mfp, + opts: fsopts, + uid: creds.EffectiveKUID, + gid: creds.EffectiveKGID, + client: client, + dentries: make(map[*dentry]struct{}), + specialFileFDs: make(map[*specialFileFD]struct{}), + } + fs.vfsfs.Init(vfsObj, fs) + + // Construct the root dentry. + root, err := fs.newDentry(ctx, attachFile, qid, attrMask, &attr) + if err != nil { + attachFile.close(ctx) + fs.vfsfs.DecRef() + return nil, nil, err + } + // Set the root's reference count to 2. One reference is returned to the + // caller, and the other is deliberately leaked to prevent the root from + // being "cached" and subsequently evicted. Its resources will still be + // cleaned up by fs.Release(). + root.refs = 2 + + return &fs.vfsfs, &root.vfsd, nil +} + +// Release implements vfs.FilesystemImpl.Release. +func (fs *filesystem) Release() { + ctx := context.Background() + mf := fs.mfp.MemoryFile() + + fs.syncMu.Lock() + for d := range fs.dentries { + d.handleMu.Lock() + d.dataMu.Lock() + if d.handleWritable { + // Write dirty cached data to the remote file. + if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, fs.mfp.MemoryFile(), d.handle.writeFromBlocksAt); err != nil { + log.Warningf("gofer.filesystem.Release: failed to flush dentry: %v", err) + } + // TODO(jamieliu): Do we need to flushf/fsync d? + } + // Discard cached pages. + d.cache.DropAll(mf) + d.dirty.RemoveAll() + d.dataMu.Unlock() + // Close the host fd if one exists. + if d.handle.fd >= 0 { + syscall.Close(int(d.handle.fd)) + d.handle.fd = -1 + } + d.handleMu.Unlock() + } + // There can't be any specialFileFDs still using fs, since each such + // FileDescription would hold a reference on a Mount holding a reference on + // fs. + fs.syncMu.Unlock() + + // Close the connection to the server. This implicitly clunks all fids. + fs.client.Close() +} + +// dentry implements vfs.DentryImpl. +type dentry struct { + vfsd vfs.Dentry + + // refs is the reference count. Each dentry holds a reference on its + // parent, even if disowned. refs is accessed using atomic memory + // operations. + refs int64 + + // fs is the owning filesystem. fs is immutable. + fs *filesystem + + // We don't support hard links, so each dentry maps 1:1 to an inode. + + // file is the unopened p9.File that backs this dentry. file is immutable. + file p9file + + // If deleted is non-zero, the file represented by this dentry has been + // deleted. deleted is accessed using atomic memory operations. + deleted uint32 + + // If cached is true, dentryEntry links dentry into + // filesystem.cachedDentries. cached and dentryEntry are protected by + // filesystem.renameMu. + cached bool + dentryEntry + + dirMu sync.Mutex + + // If this dentry represents a directory, and InteropModeShared is not in + // effect, negativeChildren is a set of child names in this directory that + // are known not to exist. negativeChildren is protected by dirMu. + negativeChildren map[string]struct{} + + // If this dentry represents a directory, InteropModeShared is not in + // effect, and dirents is not nil, it is a cache of all entries in the + // directory, in the order they were returned by the server. dirents is + // protected by dirMu. + dirents []vfs.Dirent + + // Cached metadata; protected by metadataMu and accessed using atomic + // memory operations unless otherwise specified. + metadataMu sync.Mutex + ino uint64 // immutable + mode uint32 // type is immutable, perms are mutable + uid uint32 // auth.KUID, but stored as raw uint32 for sync/atomic + gid uint32 // auth.KGID, but ... + blockSize uint32 // 0 if unknown + // Timestamps, all nsecs from the Unix epoch. + atime int64 + mtime int64 + ctime int64 + btime int64 + // File size, protected by both metadataMu and dataMu (i.e. both must be + // locked to mutate it). + size uint64 + + mapsMu sync.Mutex + + // If this dentry represents a regular file, mappings tracks mappings of + // the file into memmap.MappingSpaces. mappings is protected by mapsMu. + mappings memmap.MappingSet + + // If this dentry represents a regular file or directory: + // + // - handle is the I/O handle used by all regularFileFDs/directoryFDs + // representing this dentry. + // + // - handleReadable is true if handle is readable. + // + // - handleWritable is true if handle is writable. + // + // Invariants: + // + // - If handleReadable == handleWritable == false, then handle.file == nil + // (i.e. there is no open handle). Conversely, if handleReadable || + // handleWritable == true, then handle.file != nil (i.e. there is an open + // handle). + // + // - handleReadable and handleWritable cannot transition from true to false + // (i.e. handles may not be downgraded). + // + // These fields are protected by handleMu. + handleMu sync.RWMutex + handle handle + handleReadable bool + handleWritable bool + + dataMu sync.RWMutex + + // If this dentry represents a regular file that is client-cached, cache + // maps offsets into the cached file to offsets into + // filesystem.mfp.MemoryFile() that store the file's data. cache is + // protected by dataMu. + cache fsutil.FileRangeSet + + // If this dentry represents a regular file that is client-cached, dirty + // tracks dirty segments in cache. dirty is protected by dataMu. + dirty fsutil.DirtySet + + // pf implements platform.File for mappings of handle.fd. + pf dentryPlatformFile + + // If this dentry represents a symbolic link, InteropModeShared is not in + // effect, and haveTarget is true, target is the symlink target. haveTarget + // and target are protected by dataMu. + haveTarget bool + target string +} + +// dentryAttrMask returns a p9.AttrMask enabling all attributes used by the +// gofer client. +func dentryAttrMask() p9.AttrMask { + return p9.AttrMask{ + Mode: true, + UID: true, + GID: true, + ATime: true, + MTime: true, + CTime: true, + Size: true, + BTime: true, + } +} + +// newDentry creates a new dentry representing the given file. The dentry +// initially has no references, but is not cached; it is the caller's +// responsibility to set the dentry's reference count and/or call +// dentry.checkCachingLocked() as appropriate. +func (fs *filesystem) newDentry(ctx context.Context, file p9file, qid p9.QID, mask p9.AttrMask, attr *p9.Attr) (*dentry, error) { + if !mask.Mode { + ctx.Warningf("can't create gofer.dentry without file type") + return nil, syserror.EIO + } + if attr.Mode.FileType() == p9.ModeRegular && !mask.Size { + ctx.Warningf("can't create regular file gofer.dentry without file size") + return nil, syserror.EIO + } + + d := &dentry{ + fs: fs, + file: file, + ino: qid.Path, + mode: uint32(attr.Mode), + uid: uint32(fs.uid), + gid: uint32(fs.gid), + blockSize: usermem.PageSize, + handle: handle{ + fd: -1, + }, + } + d.pf.dentry = d + if mask.UID { + d.uid = uint32(attr.UID) + } + if mask.GID { + d.gid = uint32(attr.GID) + } + if mask.Size { + d.size = attr.Size + } + if attr.BlockSize != 0 { + d.blockSize = uint32(attr.BlockSize) + } + if mask.ATime { + d.atime = dentryTimestampFromP9(attr.ATimeSeconds, attr.ATimeNanoSeconds) + } + if mask.MTime { + d.mtime = dentryTimestampFromP9(attr.MTimeSeconds, attr.MTimeNanoSeconds) + } + if mask.CTime { + d.ctime = dentryTimestampFromP9(attr.CTimeSeconds, attr.CTimeNanoSeconds) + } + if mask.BTime { + d.btime = dentryTimestampFromP9(attr.BTimeSeconds, attr.BTimeNanoSeconds) + } + d.vfsd.Init(d) + + fs.syncMu.Lock() + fs.dentries[d] = struct{}{} + fs.syncMu.Unlock() + return d, nil +} + +// updateFromP9Attrs is called to update d's metadata after an update from the +// remote filesystem. +func (d *dentry) updateFromP9Attrs(mask p9.AttrMask, attr *p9.Attr) { + d.metadataMu.Lock() + if mask.Mode { + if got, want := uint32(attr.Mode.FileType()), d.fileType(); got != want { + d.metadataMu.Unlock() + panic(fmt.Sprintf("gofer.dentry file type changed from %#o to %#o", want, got)) + } + atomic.StoreUint32(&d.mode, uint32(attr.Mode)) + } + if mask.UID { + atomic.StoreUint32(&d.uid, uint32(attr.UID)) + } + if mask.GID { + atomic.StoreUint32(&d.gid, uint32(attr.GID)) + } + // There is no P9_GETATTR_* bit for I/O block size. + if attr.BlockSize != 0 { + atomic.StoreUint32(&d.blockSize, uint32(attr.BlockSize)) + } + if mask.ATime { + atomic.StoreInt64(&d.atime, dentryTimestampFromP9(attr.ATimeSeconds, attr.ATimeNanoSeconds)) + } + if mask.MTime { + atomic.StoreInt64(&d.mtime, dentryTimestampFromP9(attr.MTimeSeconds, attr.MTimeNanoSeconds)) + } + if mask.CTime { + atomic.StoreInt64(&d.ctime, dentryTimestampFromP9(attr.CTimeSeconds, attr.CTimeNanoSeconds)) + } + if mask.BTime { + atomic.StoreInt64(&d.btime, dentryTimestampFromP9(attr.BTimeSeconds, attr.BTimeNanoSeconds)) + } + if mask.Size { + d.dataMu.Lock() + atomic.StoreUint64(&d.size, attr.Size) + d.dataMu.Unlock() + } + d.metadataMu.Unlock() +} + +func (d *dentry) updateFromGetattr(ctx context.Context) error { + // Use d.handle.file, which represents a 9P fid that has been opened, in + // preference to d.file, which represents a 9P fid that has not. This may + // be significantly more efficient in some implementations. + var ( + file p9file + handleMuRLocked bool + ) + d.handleMu.RLock() + if !d.handle.file.isNil() { + file = d.handle.file + handleMuRLocked = true + } else { + file = d.file + d.handleMu.RUnlock() + } + _, attrMask, attr, err := file.getAttr(ctx, dentryAttrMask()) + if handleMuRLocked { + d.handleMu.RUnlock() + } + if err != nil { + return err + } + d.updateFromP9Attrs(attrMask, &attr) + return nil +} + +func (d *dentry) fileType() uint32 { + return atomic.LoadUint32(&d.mode) & linux.S_IFMT +} + +func (d *dentry) statTo(stat *linux.Statx) { + stat.Mask = linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_NLINK | linux.STATX_UID | linux.STATX_GID | linux.STATX_ATIME | linux.STATX_MTIME | linux.STATX_CTIME | linux.STATX_INO | linux.STATX_SIZE | linux.STATX_BLOCKS | linux.STATX_BTIME + stat.Blksize = atomic.LoadUint32(&d.blockSize) + stat.Nlink = 1 + if d.isDir() { + stat.Nlink = 2 + } + stat.UID = atomic.LoadUint32(&d.uid) + stat.GID = atomic.LoadUint32(&d.gid) + stat.Mode = uint16(atomic.LoadUint32(&d.mode)) + stat.Ino = d.ino + stat.Size = atomic.LoadUint64(&d.size) + // This is consistent with regularFileFD.Seek(), which treats regular files + // as having no holes. + stat.Blocks = (stat.Size + 511) / 512 + stat.Atime = statxTimestampFromDentry(atomic.LoadInt64(&d.atime)) + stat.Btime = statxTimestampFromDentry(atomic.LoadInt64(&d.btime)) + stat.Ctime = statxTimestampFromDentry(atomic.LoadInt64(&d.ctime)) + stat.Mtime = statxTimestampFromDentry(atomic.LoadInt64(&d.mtime)) + // TODO(jamieliu): device number +} + +func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, stat *linux.Statx, mnt *vfs.Mount) error { + if stat.Mask == 0 { + return nil + } + if stat.Mask&^(linux.STATX_MODE|linux.STATX_UID|linux.STATX_GID|linux.STATX_ATIME|linux.STATX_MTIME|linux.STATX_SIZE) != 0 { + return syserror.EPERM + } + if err := vfs.CheckSetStat(creds, stat, uint16(atomic.LoadUint32(&d.mode))&^linux.S_IFMT, auth.KUID(atomic.LoadUint32(&d.uid)), auth.KGID(atomic.LoadUint32(&d.gid))); err != nil { + return err + } + if err := mnt.CheckBeginWrite(); err != nil { + return err + } + defer mnt.EndWrite() + setLocalAtime := false + setLocalMtime := false + if d.fs.opts.interop != InteropModeShared { + // Timestamp updates will be handled locally. + setLocalAtime = stat.Mask&linux.STATX_ATIME != 0 + setLocalMtime = stat.Mask&linux.STATX_MTIME != 0 + stat.Mask &^= linux.STATX_ATIME | linux.STATX_MTIME + if !setLocalMtime && (stat.Mask&linux.STATX_SIZE != 0) { + // Truncate updates mtime. + setLocalMtime = true + stat.Mtime.Nsec = linux.UTIME_NOW + } + } + d.metadataMu.Lock() + defer d.metadataMu.Unlock() + if stat.Mask != 0 { + if err := d.file.setAttr(ctx, p9.SetAttrMask{ + Permissions: stat.Mask&linux.STATX_MODE != 0, + UID: stat.Mask&linux.STATX_UID != 0, + GID: stat.Mask&linux.STATX_GID != 0, + Size: stat.Mask&linux.STATX_SIZE != 0, + ATime: stat.Mask&linux.STATX_ATIME != 0, + MTime: stat.Mask&linux.STATX_MTIME != 0, + ATimeNotSystemTime: stat.Atime.Nsec != linux.UTIME_NOW, + MTimeNotSystemTime: stat.Mtime.Nsec != linux.UTIME_NOW, + }, p9.SetAttr{ + Permissions: p9.FileMode(stat.Mode), + UID: p9.UID(stat.UID), + GID: p9.GID(stat.GID), + Size: stat.Size, + ATimeSeconds: uint64(stat.Atime.Sec), + ATimeNanoSeconds: uint64(stat.Atime.Nsec), + MTimeSeconds: uint64(stat.Mtime.Sec), + MTimeNanoSeconds: uint64(stat.Mtime.Nsec), + }); err != nil { + return err + } + } + if d.fs.opts.interop == InteropModeShared { + // There's no point to updating d's metadata in this case since it'll + // be overwritten by revalidation before the next time it's used + // anyway. (InteropModeShared inhibits client caching of regular file + // data, so there's no cache to truncate either.) + return nil + } + now, haveNow := nowFromContext(ctx) + if !haveNow { + ctx.Warningf("gofer.dentry.setStat: current time not available") + } + if stat.Mask&linux.STATX_MODE != 0 { + atomic.StoreUint32(&d.mode, d.fileType()|uint32(stat.Mode)) + } + if stat.Mask&linux.STATX_UID != 0 { + atomic.StoreUint32(&d.uid, stat.UID) + } + if stat.Mask&linux.STATX_GID != 0 { + atomic.StoreUint32(&d.gid, stat.GID) + } + if setLocalAtime { + if stat.Atime.Nsec == linux.UTIME_NOW { + if haveNow { + atomic.StoreInt64(&d.atime, now) + } + } else { + atomic.StoreInt64(&d.atime, dentryTimestampFromStatx(stat.Atime)) + } + } + if setLocalMtime { + if stat.Mtime.Nsec == linux.UTIME_NOW { + if haveNow { + atomic.StoreInt64(&d.mtime, now) + } + } else { + atomic.StoreInt64(&d.mtime, dentryTimestampFromStatx(stat.Mtime)) + } + } + if haveNow { + atomic.StoreInt64(&d.ctime, now) + } + if stat.Mask&linux.STATX_SIZE != 0 { + d.dataMu.Lock() + oldSize := d.size + d.size = stat.Size + // d.dataMu must be unlocked to lock d.mapsMu and invalidate mappings + // below. This allows concurrent calls to Read/Translate/etc. These + // functions synchronize with truncation by refusing to use cache + // contents beyond the new d.size. (We are still holding d.metadataMu, + // so we can't race with Write or another truncate.) + d.dataMu.Unlock() + if d.size < oldSize { + oldpgend := pageRoundUp(oldSize) + newpgend := pageRoundUp(d.size) + if oldpgend != newpgend { + d.mapsMu.Lock() + d.mappings.Invalidate(memmap.MappableRange{newpgend, oldpgend}, memmap.InvalidateOpts{ + // Compare Linux's mm/truncate.c:truncate_setsize() => + // truncate_pagecache() => + // mm/memory.c:unmap_mapping_range(evencows=1). + InvalidatePrivate: true, + }) + d.mapsMu.Unlock() + } + // We are now guaranteed that there are no translations of + // truncated pages, and can remove them from the cache. Since + // truncated pages have been removed from the remote file, they + // should be dropped without being written back. + d.dataMu.Lock() + d.cache.Truncate(d.size, d.fs.mfp.MemoryFile()) + d.dirty.KeepClean(memmap.MappableRange{d.size, oldpgend}) + d.dataMu.Unlock() + } + } + return nil +} + +func (d *dentry) checkPermissions(creds *auth.Credentials, ats vfs.AccessTypes, isDir bool) error { + return vfs.GenericCheckPermissions(creds, ats, isDir, uint16(atomic.LoadUint32(&d.mode))&0777, auth.KUID(atomic.LoadUint32(&d.uid)), auth.KGID(atomic.LoadUint32(&d.gid))) +} + +// IncRef implements vfs.DentryImpl.IncRef. +func (d *dentry) IncRef() { + // d.refs may be 0 if d.fs.renameMu is locked, which serializes against + // d.checkCachingLocked(). + atomic.AddInt64(&d.refs, 1) +} + +// TryIncRef implements vfs.DentryImpl.TryIncRef. +func (d *dentry) TryIncRef() bool { + for { + refs := atomic.LoadInt64(&d.refs) + if refs == 0 { + return false + } + if atomic.CompareAndSwapInt64(&d.refs, refs, refs+1) { + return true + } + } +} + +// DecRef implements vfs.DentryImpl.DecRef. +func (d *dentry) DecRef() { + if refs := atomic.AddInt64(&d.refs, -1); refs == 0 { + d.fs.renameMu.Lock() + d.checkCachingLocked() + d.fs.renameMu.Unlock() + } else if refs < 0 { + panic("gofer.dentry.DecRef() called without holding a reference") + } +} + +// checkCachingLocked should be called after d's reference count becomes 0 or it +// becomes disowned. +// +// Preconditions: d.fs.renameMu must be locked for writing. +func (d *dentry) checkCachingLocked() { + // Dentries with a non-zero reference count must be retained. (The only way + // to obtain a reference on a dentry with zero references is via path + // resolution, which requires renameMu, so if d.refs is zero then it will + // remain zero while we hold renameMu for writing.) + if atomic.LoadInt64(&d.refs) != 0 { + if d.cached { + d.fs.cachedDentries.Remove(d) + d.fs.cachedDentriesLen-- + d.cached = false + } + return + } + // Non-child dentries with zero references are no longer reachable by path + // resolution and should be dropped immediately. + if d.vfsd.Parent() == nil || d.vfsd.IsDisowned() { + if d.cached { + d.fs.cachedDentries.Remove(d) + d.fs.cachedDentriesLen-- + d.cached = false + } + d.destroyLocked() + return + } + // If d is already cached, just move it to the front of the LRU. + if d.cached { + d.fs.cachedDentries.Remove(d) + d.fs.cachedDentries.PushFront(d) + return + } + // Cache the dentry, then evict the least recently used cached dentry if + // the cache becomes over-full. + d.fs.cachedDentries.PushFront(d) + d.fs.cachedDentriesLen++ + d.cached = true + if d.fs.cachedDentriesLen > d.fs.opts.maxCachedDentries { + victim := d.fs.cachedDentries.Back() + d.fs.cachedDentries.Remove(victim) + d.fs.cachedDentriesLen-- + victim.cached = false + // victim.refs may have become non-zero from an earlier path + // resolution since it was inserted into fs.cachedDentries; see + // dentry.incRefLocked(). Either way, we brought + // fs.cachedDentriesLen back down to fs.opts.maxCachedDentries, so + // we don't loop. + if atomic.LoadInt64(&victim.refs) == 0 { + if victimParentVFSD := victim.vfsd.Parent(); victimParentVFSD != nil { + victimParent := victimParentVFSD.Impl().(*dentry) + victimParent.dirMu.Lock() + if !victim.vfsd.IsDisowned() { + // victim can't be a mount point (in any mount + // namespace), since VFS holds references on mount + // points. + d.fs.vfsfs.VirtualFilesystem().ForceDeleteDentry(&victim.vfsd) + // We're only deleting the dentry, not the file it + // represents, so we don't need to update + // victimParent.dirents etc. + } + victimParent.dirMu.Unlock() + } + victim.destroyLocked() + } + } +} + +// Preconditions: d.fs.renameMu must be locked for writing. d.refs == 0. d is +// not a child dentry. +func (d *dentry) destroyLocked() { + ctx := context.Background() + d.handleMu.Lock() + if !d.handle.file.isNil() { + mf := d.fs.mfp.MemoryFile() + d.dataMu.Lock() + // Write dirty pages back to the remote filesystem. + if d.handleWritable { + if err := fsutil.SyncDirtyAll(ctx, &d.cache, &d.dirty, d.size, mf, d.handle.writeFromBlocksAt); err != nil { + log.Warningf("gofer.dentry.DecRef: failed to write dirty data back: %v", err) + } + } + // Discard cached data. + d.cache.DropAll(mf) + d.dirty.RemoveAll() + d.dataMu.Unlock() + // Clunk open fids and close open host FDs. + d.handle.close(ctx) + } + d.handleMu.Unlock() + d.file.close(ctx) + // Remove d from the set of all dentries. + d.fs.syncMu.Lock() + delete(d.fs.dentries, d) + d.fs.syncMu.Unlock() + // Drop the reference held by d on its parent. + if parentVFSD := d.vfsd.Parent(); parentVFSD != nil { + parent := parentVFSD.Impl().(*dentry) + // This is parent.DecRef() without recursive locking of d.fs.renameMu. + if refs := atomic.AddInt64(&parent.refs, -1); refs == 0 { + parent.checkCachingLocked() + } else if refs < 0 { + panic("gofer.dentry.DecRef() called without holding a reference") + } + } +} + +func (d *dentry) isDeleted() bool { + return atomic.LoadUint32(&d.deleted) != 0 +} + +func (d *dentry) setDeleted() { + atomic.StoreUint32(&d.deleted, 1) +} + +func (d *dentry) listxattr(ctx context.Context) ([]string, error) { + return nil, syserror.ENOTSUP +} + +func (d *dentry) getxattr(ctx context.Context, name string) (string, error) { + // TODO(jamieliu): add vfs.GetxattrOptions.Size + return d.file.getXattr(ctx, name, linux.XATTR_SIZE_MAX) +} + +func (d *dentry) setxattr(ctx context.Context, opts *vfs.SetxattrOptions) error { + return d.file.setXattr(ctx, opts.Name, opts.Value, opts.Flags) +} + +func (d *dentry) removexattr(ctx context.Context, name string) error { + return syserror.ENOTSUP +} + +// Preconditions: d.isRegularFile() || d.isDirectory(). +func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool) error { + // O_TRUNC unconditionally requires us to obtain a new handle (opened with + // O_TRUNC). + if !trunc { + d.handleMu.RLock() + if (!read || d.handleReadable) && (!write || d.handleWritable) { + // The current handle is sufficient. + d.handleMu.RUnlock() + return nil + } + d.handleMu.RUnlock() + } + + haveOldFD := false + d.handleMu.Lock() + if (read && !d.handleReadable) || (write && !d.handleWritable) || trunc { + // Get a new handle. + wantReadable := d.handleReadable || read + wantWritable := d.handleWritable || write + h, err := openHandle(ctx, d.file, wantReadable, wantWritable, trunc) + if err != nil { + d.handleMu.Unlock() + return err + } + if !d.handle.file.isNil() { + // Check that old and new handles are compatible: If the old handle + // includes a host file descriptor but the new one does not, or + // vice versa, old and new memory mappings may be incoherent. + haveOldFD = d.handle.fd >= 0 + haveNewFD := h.fd >= 0 + if haveOldFD != haveNewFD { + d.handleMu.Unlock() + ctx.Warningf("gofer.dentry.ensureSharedHandle: can't change host FD availability from %v to %v across dentry handle upgrade", haveOldFD, haveNewFD) + h.close(ctx) + return syserror.EIO + } + if haveOldFD { + // We may have raced with callers of d.pf.FD() that are now + // using the old file descriptor, preventing us from safely + // closing it. We could handle this by invalidating existing + // memmap.Translations, but this is expensive. Instead, use + // dup2() to make the old file descriptor refer to the new file + // description, then close the new file descriptor (which is no + // longer needed). Racing callers may use the old or new file + // description, but this doesn't matter since they refer to the + // same file (unless d.fs.opts.overlayfsStaleRead is true, + // which we handle separately). + if err := syscall.Dup2(int(h.fd), int(d.handle.fd)); err != nil { + d.handleMu.Unlock() + ctx.Warningf("gofer.dentry.ensureSharedHandle: failed to dup fd %d to fd %d: %v", h.fd, d.handle.fd, err) + h.close(ctx) + return err + } + syscall.Close(int(h.fd)) + h.fd = d.handle.fd + if d.fs.opts.overlayfsStaleRead { + // Replace sentry mappings of the old FD with mappings of + // the new FD, since the two are not necessarily coherent. + if err := d.pf.hostFileMapper.RegenerateMappings(int(h.fd)); err != nil { + d.handleMu.Unlock() + ctx.Warningf("gofer.dentry.ensureSharedHandle: failed to replace sentry mappings of old FD with mappings of new FD: %v", err) + h.close(ctx) + return err + } + } + // Clunk the old fid before making the new handle visible (by + // unlocking d.handleMu). + d.handle.file.close(ctx) + } + } + // Switch to the new handle. + d.handle = h + d.handleReadable = wantReadable + d.handleWritable = wantWritable + } + d.handleMu.Unlock() + + if d.fs.opts.overlayfsStaleRead && haveOldFD { + // Invalidate application mappings that may be using the old FD; they + // will be replaced with mappings using the new FD after future calls + // to d.Translate(). This requires holding d.mapsMu, which precedes + // d.handleMu in the lock order. + d.mapsMu.Lock() + d.mappings.InvalidateAll(memmap.InvalidateOpts{}) + d.mapsMu.Unlock() + } + + return nil +} + +// fileDescription is embedded by gofer implementations of +// vfs.FileDescriptionImpl. +type fileDescription struct { + vfsfd vfs.FileDescription + vfs.FileDescriptionDefaultImpl +} + +func (fd *fileDescription) filesystem() *filesystem { + return fd.vfsfd.Mount().Filesystem().Impl().(*filesystem) +} + +func (fd *fileDescription) dentry() *dentry { + return fd.vfsfd.Dentry().Impl().(*dentry) +} + +// Stat implements vfs.FileDescriptionImpl.Stat. +func (fd *fileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linux.Statx, error) { + d := fd.dentry() + if d.fs.opts.interop == InteropModeShared && opts.Mask&(linux.STATX_MODE|linux.STATX_UID|linux.STATX_GID|linux.STATX_ATIME|linux.STATX_MTIME|linux.STATX_CTIME|linux.STATX_SIZE|linux.STATX_BLOCKS|linux.STATX_BTIME) != 0 && opts.Sync != linux.AT_STATX_DONT_SYNC { + // TODO(jamieliu): Use specialFileFD.handle.file for the getattr if + // available? + if err := d.updateFromGetattr(ctx); err != nil { + return linux.Statx{}, err + } + } + var stat linux.Statx + d.statTo(&stat) + return stat, nil +} + +// SetStat implements vfs.FileDescriptionImpl.SetStat. +func (fd *fileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions) error { + return fd.dentry().setStat(ctx, auth.CredentialsFromContext(ctx), &opts.Stat, fd.vfsfd.Mount()) +} + +// Listxattr implements vfs.FileDescriptionImpl.Listxattr. +func (fd *fileDescription) Listxattr(ctx context.Context) ([]string, error) { + return fd.dentry().listxattr(ctx) +} + +// Getxattr implements vfs.FileDescriptionImpl.Getxattr. +func (fd *fileDescription) Getxattr(ctx context.Context, name string) (string, error) { + return fd.dentry().getxattr(ctx, name) +} + +// Setxattr implements vfs.FileDescriptionImpl.Setxattr. +func (fd *fileDescription) Setxattr(ctx context.Context, opts vfs.SetxattrOptions) error { + return fd.dentry().setxattr(ctx, &opts) +} + +// Removexattr implements vfs.FileDescriptionImpl.Removexattr. +func (fd *fileDescription) Removexattr(ctx context.Context, name string) error { + return fd.dentry().removexattr(ctx, name) +} -- cgit v1.2.3