summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fs/tmpfs
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2019-06-02 06:44:55 +0000
committergVisor bot <gvisor-bot@google.com>2019-06-02 06:44:55 +0000
commitceb0d792f328d1fc0692197d8856a43c3936a571 (patch)
tree83155f302eff44a78bcc30a3a08f4efe59a79379 /pkg/sentry/fs/tmpfs
parentdeb7ecf1e46862d54f4b102f2d163cfbcfc37f3b (diff)
parent216da0b733dbed9aad9b2ab92ac75bcb906fd7ee (diff)
Merge 216da0b7 (automated)
Diffstat (limited to 'pkg/sentry/fs/tmpfs')
-rw-r--r--pkg/sentry/fs/tmpfs/device.go20
-rw-r--r--pkg/sentry/fs/tmpfs/file_regular.go60
-rw-r--r--pkg/sentry/fs/tmpfs/fs.go136
-rw-r--r--pkg/sentry/fs/tmpfs/inode_file.go681
-rw-r--r--pkg/sentry/fs/tmpfs/tmpfs.go348
-rwxr-xr-xpkg/sentry/fs/tmpfs/tmpfs_state_autogen.go108
6 files changed, 1353 insertions, 0 deletions
diff --git a/pkg/sentry/fs/tmpfs/device.go b/pkg/sentry/fs/tmpfs/device.go
new file mode 100644
index 000000000..179c3a46f
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/device.go
@@ -0,0 +1,20 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package tmpfs
+
+import "gvisor.googlesource.com/gvisor/pkg/sentry/device"
+
+// tmpfsDevice is the kernel tmpfs device.
+var tmpfsDevice = device.NewAnonDevice()
diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go
new file mode 100644
index 000000000..d1c163879
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/file_regular.go
@@ -0,0 +1,60 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package tmpfs
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/memmap"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
+ "gvisor.googlesource.com/gvisor/pkg/waiter"
+)
+
+// regularFileOperations implements fs.FileOperations for a regular
+// tmpfs file.
+//
+// +stateify savable
+type regularFileOperations struct {
+ fsutil.FileNoopRelease `state:"nosave"`
+ fsutil.FileGenericSeek `state:"nosave"`
+ fsutil.FileNotDirReaddir `state:"nosave"`
+ fsutil.FileNoopFsync `state:"nosave"`
+ fsutil.FileNoopFlush `state:"nosave"`
+ fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
+
+ // iops is the InodeOperations of a regular tmpfs file. It is
+ // guaranteed to be the same as file.Dirent.Inode.InodeOperations,
+ // see operations that take fs.File below.
+ iops *fileInodeOperations
+}
+
+// Read implements fs.FileOperations.Read.
+func (r *regularFileOperations) Read(ctx context.Context, file *fs.File, dst usermem.IOSequence, offset int64) (int64, error) {
+ return r.iops.read(ctx, file, dst, offset)
+}
+
+// Write implements fs.FileOperations.Write.
+func (r *regularFileOperations) Write(ctx context.Context, file *fs.File, src usermem.IOSequence, offset int64) (int64, error) {
+ return r.iops.write(ctx, src, offset)
+}
+
+// ConfigureMMap implements fs.FileOperations.ConfigureMMap.
+func (r *regularFileOperations) ConfigureMMap(ctx context.Context, file *fs.File, opts *memmap.MMapOpts) error {
+ return fsutil.GenericConfigureMMap(file, r.iops, opts)
+}
diff --git a/pkg/sentry/fs/tmpfs/fs.go b/pkg/sentry/fs/tmpfs/fs.go
new file mode 100644
index 000000000..b7c29a4d1
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/fs.go
@@ -0,0 +1,136 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package tmpfs
+
+import (
+ "fmt"
+ "strconv"
+
+ "gvisor.googlesource.com/gvisor/pkg/abi/linux"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth"
+)
+
+const (
+ // Set initial permissions for the root directory.
+ modeKey = "mode"
+
+ // UID for the root directory.
+ rootUIDKey = "uid"
+
+ // GID for the root directory.
+ rootGIDKey = "gid"
+
+ // TODO(edahlgren/mpratt): support a tmpfs size limit.
+ // size = "size"
+
+ // Permissions that exceed modeMask will be rejected.
+ modeMask = 01777
+
+ // Default permissions are read/write/execute.
+ defaultMode = 0777
+)
+
+// Filesystem is a tmpfs.
+//
+// +stateify savable
+type Filesystem struct{}
+
+var _ fs.Filesystem = (*Filesystem)(nil)
+
+func init() {
+ fs.RegisterFilesystem(&Filesystem{})
+}
+
+// FilesystemName is the name underwhich the filesystem is registered.
+// Name matches mm/shmem.c:shmem_fs_type.name.
+const FilesystemName = "tmpfs"
+
+// Name is the name of the file system.
+func (*Filesystem) Name() string {
+ return FilesystemName
+}
+
+// AllowUserMount allows users to mount(2) this file system.
+func (*Filesystem) AllowUserMount() bool {
+ return true
+}
+
+// AllowUserList allows this filesystem to be listed in /proc/filesystems.
+func (*Filesystem) AllowUserList() bool {
+ return true
+}
+
+// Flags returns that there is nothing special about this file system.
+//
+// In Linux, tmpfs returns FS_USERNS_MOUNT, see mm/shmem.c.
+func (*Filesystem) Flags() fs.FilesystemFlags {
+ return 0
+}
+
+// Mount returns a tmpfs root that can be positioned in the vfs.
+func (f *Filesystem) Mount(ctx context.Context, device string, flags fs.MountSourceFlags, data string, _ interface{}) (*fs.Inode, error) {
+ // device is always ignored.
+
+ // Parse generic comma-separated key=value options, this file system expects them.
+ options := fs.GenericMountSourceOptions(data)
+
+ // Parse the root directory permissions.
+ perms := fs.FilePermsFromMode(defaultMode)
+ if m, ok := options[modeKey]; ok {
+ i, err := strconv.ParseUint(m, 8, 32)
+ if err != nil {
+ return nil, fmt.Errorf("mode value not parsable 'mode=%s': %v", m, err)
+ }
+ if i&^modeMask != 0 {
+ return nil, fmt.Errorf("invalid mode %q: must be less than %o", m, modeMask)
+ }
+ perms = fs.FilePermsFromMode(linux.FileMode(i))
+ delete(options, modeKey)
+ }
+
+ creds := auth.CredentialsFromContext(ctx)
+ owner := fs.FileOwnerFromContext(ctx)
+ if uidstr, ok := options[rootUIDKey]; ok {
+ uid, err := strconv.ParseInt(uidstr, 10, 32)
+ if err != nil {
+ return nil, fmt.Errorf("uid value not parsable 'uid=%d': %v", uid, err)
+ }
+ owner.UID = creds.UserNamespace.MapToKUID(auth.UID(uid))
+ delete(options, rootUIDKey)
+ }
+
+ if gidstr, ok := options[rootGIDKey]; ok {
+ gid, err := strconv.ParseInt(gidstr, 10, 32)
+ if err != nil {
+ return nil, fmt.Errorf("gid value not parsable 'gid=%d': %v", gid, err)
+ }
+ owner.GID = creds.UserNamespace.MapToKGID(auth.GID(gid))
+ delete(options, rootGIDKey)
+ }
+
+ // Fail if the caller passed us more options than we can parse. They may be
+ // expecting us to set something we can't set.
+ if len(options) > 0 {
+ return nil, fmt.Errorf("unsupported mount options: %v", options)
+ }
+
+ // Construct a mount which will cache dirents.
+ msrc := fs.NewCachingMountSource(f, flags)
+
+ // Construct the tmpfs root.
+ return NewDir(ctx, nil, owner, perms, msrc), nil
+}
diff --git a/pkg/sentry/fs/tmpfs/inode_file.go b/pkg/sentry/fs/tmpfs/inode_file.go
new file mode 100644
index 000000000..3fe659543
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/inode_file.go
@@ -0,0 +1,681 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package tmpfs
+
+import (
+ "fmt"
+ "io"
+ "sync"
+ "time"
+
+ "gvisor.googlesource.com/gvisor/pkg/abi/linux"
+ "gvisor.googlesource.com/gvisor/pkg/metric"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
+ ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/memmap"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/safemem"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usage"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
+ "gvisor.googlesource.com/gvisor/pkg/syserror"
+)
+
+var (
+ opensRO = metric.MustCreateNewUint64Metric("/in_memory_file/opens_ro", false /* sync */, "Number of times an in-memory file was opened in read-only mode.")
+ opensW = metric.MustCreateNewUint64Metric("/in_memory_file/opens_w", false /* sync */, "Number of times an in-memory file was opened in write mode.")
+ reads = metric.MustCreateNewUint64Metric("/in_memory_file/reads", false /* sync */, "Number of in-memory file reads.")
+ readWait = metric.MustCreateNewUint64Metric("/in_memory_file/read_wait", false /* sync */, "Time waiting on in-memory file reads, in nanoseconds.")
+)
+
+// fileInodeOperations implements fs.InodeOperations for a regular tmpfs file.
+// These files are backed by pages allocated from a platform.Memory, and may be
+// directly mapped.
+//
+// Lock order: attrMu -> mapsMu -> dataMu.
+//
+// +stateify savable
+type fileInodeOperations struct {
+ fsutil.InodeGenericChecker `state:"nosave"`
+ fsutil.InodeNoopWriteOut `state:"nosave"`
+ fsutil.InodeNotDirectory `state:"nosave"`
+ fsutil.InodeNotSocket `state:"nosave"`
+ fsutil.InodeNotSymlink `state:"nosave"`
+
+ fsutil.InodeSimpleExtendedAttributes
+
+ // kernel is used to allocate memory that stores the file's contents.
+ kernel *kernel.Kernel
+
+ // memUsage is the default memory usage that will be reported by this file.
+ memUsage usage.MemoryKind
+
+ attrMu sync.Mutex `state:"nosave"`
+
+ // attr contains the unstable metadata for the file.
+ //
+ // attr is protected by attrMu. attr.Size is protected by both attrMu
+ // and dataMu; reading it requires locking either mutex, while mutating
+ // it requires locking both.
+ attr fs.UnstableAttr
+
+ mapsMu sync.Mutex `state:"nosave"`
+
+ // mappings tracks mappings of the file into memmap.MappingSpaces.
+ //
+ // mappings is protected by mapsMu.
+ mappings memmap.MappingSet
+
+ // writableMappingPages tracks how many pages of virtual memory are mapped
+ // as potentially writable from this file. If a page has multiple mappings,
+ // each mapping is counted separately.
+ //
+ // This counter is susceptible to overflow as we can potentially count
+ // mappings from many VMAs. We count pages rather than bytes to slightly
+ // mitigate this.
+ //
+ // Protected by mapsMu.
+ writableMappingPages uint64
+
+ dataMu sync.RWMutex `state:"nosave"`
+
+ // data maps offsets into the file to offsets into platform.Memory() that
+ // store the file's data.
+ //
+ // data is protected by dataMu.
+ data fsutil.FileRangeSet
+
+ // seals represents file seals on this inode.
+ //
+ // Protected by dataMu.
+ seals uint32
+}
+
+var _ fs.InodeOperations = (*fileInodeOperations)(nil)
+
+// NewInMemoryFile returns a new file backed by Kernel.MemoryFile().
+func NewInMemoryFile(ctx context.Context, usage usage.MemoryKind, uattr fs.UnstableAttr) fs.InodeOperations {
+ return &fileInodeOperations{
+ attr: uattr,
+ kernel: kernel.KernelFromContext(ctx),
+ memUsage: usage,
+ seals: linux.F_SEAL_SEAL,
+ }
+}
+
+// NewMemfdInode creates a new inode backing a memfd. Memory used by the memfd
+// is backed by platform memory.
+func NewMemfdInode(ctx context.Context, allowSeals bool) *fs.Inode {
+ // Per Linux, mm/shmem.c:__shmem_file_setup(), memfd inodes are set up with
+ // S_IRWXUGO.
+ perms := fs.PermMask{Read: true, Write: true, Execute: true}
+ iops := NewInMemoryFile(ctx, usage.Tmpfs, fs.UnstableAttr{
+ Owner: fs.FileOwnerFromContext(ctx),
+ Perms: fs.FilePermissions{User: perms, Group: perms, Other: perms}}).(*fileInodeOperations)
+ if allowSeals {
+ iops.seals = 0
+ }
+ return fs.NewInode(iops, fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}), fs.StableAttr{
+ Type: fs.RegularFile,
+ DeviceID: tmpfsDevice.DeviceID(),
+ InodeID: tmpfsDevice.NextIno(),
+ BlockSize: usermem.PageSize,
+ })
+}
+
+// Release implements fs.InodeOperations.Release.
+func (f *fileInodeOperations) Release(context.Context) {
+ f.dataMu.Lock()
+ defer f.dataMu.Unlock()
+ f.data.DropAll(f.kernel.MemoryFile())
+}
+
+// Mappable implements fs.InodeOperations.Mappable.
+func (f *fileInodeOperations) Mappable(*fs.Inode) memmap.Mappable {
+ return f
+}
+
+// Rename implements fs.InodeOperations.Rename.
+func (*fileInodeOperations) Rename(ctx context.Context, inode *fs.Inode, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string, replacement bool) error {
+ return rename(ctx, oldParent, oldName, newParent, newName, replacement)
+}
+
+// GetFile implements fs.InodeOperations.GetFile.
+func (f *fileInodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) {
+ if flags.Write {
+ opensW.Increment()
+ } else if flags.Read {
+ opensRO.Increment()
+ }
+ flags.Pread = true
+ flags.Pwrite = true
+ return fs.NewFile(ctx, d, flags, &regularFileOperations{iops: f}), nil
+}
+
+// UnstableAttr returns unstable attributes of this tmpfs file.
+func (f *fileInodeOperations) UnstableAttr(ctx context.Context, inode *fs.Inode) (fs.UnstableAttr, error) {
+ f.attrMu.Lock()
+ f.dataMu.RLock()
+ attr := f.attr
+ attr.Usage = int64(f.data.Span())
+ f.dataMu.RUnlock()
+ f.attrMu.Unlock()
+ return attr, nil
+}
+
+// Check implements fs.InodeOperations.Check.
+func (f *fileInodeOperations) Check(ctx context.Context, inode *fs.Inode, p fs.PermMask) bool {
+ return fs.ContextCanAccessFile(ctx, inode, p)
+}
+
+// SetPermissions implements fs.InodeOperations.SetPermissions.
+func (f *fileInodeOperations) SetPermissions(ctx context.Context, _ *fs.Inode, p fs.FilePermissions) bool {
+ f.attrMu.Lock()
+ f.attr.SetPermissions(ctx, p)
+ f.attrMu.Unlock()
+ return true
+}
+
+// SetTimestamps implements fs.InodeOperations.SetTimestamps.
+func (f *fileInodeOperations) SetTimestamps(ctx context.Context, _ *fs.Inode, ts fs.TimeSpec) error {
+ f.attrMu.Lock()
+ f.attr.SetTimestamps(ctx, ts)
+ f.attrMu.Unlock()
+ return nil
+}
+
+// SetOwner implements fs.InodeOperations.SetOwner.
+func (f *fileInodeOperations) SetOwner(ctx context.Context, _ *fs.Inode, owner fs.FileOwner) error {
+ f.attrMu.Lock()
+ f.attr.SetOwner(ctx, owner)
+ f.attrMu.Unlock()
+ return nil
+}
+
+// Truncate implements fs.InodeOperations.Truncate.
+func (f *fileInodeOperations) Truncate(ctx context.Context, _ *fs.Inode, size int64) error {
+ f.attrMu.Lock()
+ defer f.attrMu.Unlock()
+
+ f.dataMu.Lock()
+ oldSize := f.attr.Size
+
+ // Check if current seals allow truncation.
+ switch {
+ case size > oldSize && f.seals&linux.F_SEAL_GROW != 0: // Grow sealed
+ fallthrough
+ case oldSize > size && f.seals&linux.F_SEAL_SHRINK != 0: // Shrink sealed
+ f.dataMu.Unlock()
+ return syserror.EPERM
+ }
+
+ if oldSize != size {
+ f.attr.Size = size
+ // Update mtime and ctime.
+ now := ktime.NowFromContext(ctx)
+ f.attr.ModificationTime = now
+ f.attr.StatusChangeTime = now
+ }
+ f.dataMu.Unlock()
+
+ // Nothing left to do unless shrinking the file.
+ if oldSize <= size {
+ return nil
+ }
+
+ oldpgend := fs.OffsetPageEnd(oldSize)
+ newpgend := fs.OffsetPageEnd(size)
+
+ // Invalidate past translations of truncated pages.
+ if newpgend != oldpgend {
+ f.mapsMu.Lock()
+ f.mappings.Invalidate(memmap.MappableRange{newpgend, oldpgend}, memmap.InvalidateOpts{
+ // Compare Linux's mm/shmem.c:shmem_setattr() =>
+ // mm/memory.c:unmap_mapping_range(evencows=1).
+ InvalidatePrivate: true,
+ })
+ f.mapsMu.Unlock()
+ }
+
+ // We are now guaranteed that there are no translations of truncated pages,
+ // and can remove them.
+ f.dataMu.Lock()
+ defer f.dataMu.Unlock()
+ f.data.Truncate(uint64(size), f.kernel.MemoryFile())
+
+ return nil
+}
+
+// Allocate implements fs.InodeOperations.Allocate.
+func (f *fileInodeOperations) Allocate(ctx context.Context, _ *fs.Inode, offset, length int64) error {
+ newSize := offset + length
+
+ f.attrMu.Lock()
+ defer f.attrMu.Unlock()
+ f.dataMu.Lock()
+ defer f.dataMu.Unlock()
+
+ if newSize <= f.attr.Size {
+ return nil
+ }
+
+ // Check if current seals allow growth.
+ if f.seals&linux.F_SEAL_GROW != 0 {
+ return syserror.EPERM
+ }
+
+ f.attr.Size = newSize
+
+ now := ktime.NowFromContext(ctx)
+ f.attr.ModificationTime = now
+ f.attr.StatusChangeTime = now
+
+ return nil
+}
+
+// AddLink implements fs.InodeOperations.AddLink.
+func (f *fileInodeOperations) AddLink() {
+ f.attrMu.Lock()
+ f.attr.Links++
+ f.attrMu.Unlock()
+}
+
+// DropLink implements fs.InodeOperations.DropLink.
+func (f *fileInodeOperations) DropLink() {
+ f.attrMu.Lock()
+ f.attr.Links--
+ f.attrMu.Unlock()
+}
+
+// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange.
+func (f *fileInodeOperations) NotifyStatusChange(ctx context.Context) {
+ f.attrMu.Lock()
+ f.attr.StatusChangeTime = ktime.NowFromContext(ctx)
+ f.attrMu.Unlock()
+}
+
+// IsVirtual implements fs.InodeOperations.IsVirtual.
+func (*fileInodeOperations) IsVirtual() bool {
+ return true
+}
+
+// StatFS implements fs.InodeOperations.StatFS.
+func (*fileInodeOperations) StatFS(context.Context) (fs.Info, error) {
+ return fsInfo, nil
+}
+
+func (f *fileInodeOperations) read(ctx context.Context, file *fs.File, dst usermem.IOSequence, offset int64) (int64, error) {
+ var start time.Time
+ if fs.RecordWaitTime {
+ start = time.Now()
+ }
+ reads.Increment()
+ // Zero length reads for tmpfs are no-ops.
+ if dst.NumBytes() == 0 {
+ fs.IncrementWait(readWait, start)
+ return 0, nil
+ }
+
+ // Have we reached EOF? We check for this again in
+ // fileReadWriter.ReadToBlocks to avoid holding f.attrMu (which would
+ // serialize reads) or f.dataMu (which would violate lock ordering), but
+ // check here first (before calling into MM) since reading at EOF is
+ // common: getting a return value of 0 from a read syscall is the only way
+ // to detect EOF.
+ //
+ // TODO(jamieliu): Separate out f.attr.Size and use atomics instead of
+ // f.dataMu.
+ f.dataMu.RLock()
+ size := f.attr.Size
+ f.dataMu.RUnlock()
+ if offset >= size {
+ fs.IncrementWait(readWait, start)
+ return 0, io.EOF
+ }
+
+ n, err := dst.CopyOutFrom(ctx, &fileReadWriter{f, offset})
+ if !file.Dirent.Inode.MountSource.Flags.NoAtime {
+ // Compare Linux's mm/filemap.c:do_generic_file_read() => file_accessed().
+ f.attrMu.Lock()
+ f.attr.AccessTime = ktime.NowFromContext(ctx)
+ f.attrMu.Unlock()
+ }
+ fs.IncrementWait(readWait, start)
+ return n, err
+}
+
+func (f *fileInodeOperations) write(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) {
+ // Zero length writes for tmpfs are no-ops.
+ if src.NumBytes() == 0 {
+ return 0, nil
+ }
+
+ f.attrMu.Lock()
+ defer f.attrMu.Unlock()
+ // Compare Linux's mm/filemap.c:__generic_file_write_iter() => file_update_time().
+ now := ktime.NowFromContext(ctx)
+ f.attr.ModificationTime = now
+ f.attr.StatusChangeTime = now
+ return src.CopyInTo(ctx, &fileReadWriter{f, offset})
+}
+
+type fileReadWriter struct {
+ f *fileInodeOperations
+ offset int64
+}
+
+// ReadToBlocks implements safemem.Reader.ReadToBlocks.
+func (rw *fileReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error) {
+ rw.f.dataMu.RLock()
+ defer rw.f.dataMu.RUnlock()
+
+ // Compute the range to read.
+ if rw.offset >= rw.f.attr.Size {
+ return 0, io.EOF
+ }
+ end := fs.ReadEndOffset(rw.offset, int64(dsts.NumBytes()), rw.f.attr.Size)
+ if end == rw.offset { // dsts.NumBytes() == 0?
+ return 0, nil
+ }
+
+ mf := rw.f.kernel.MemoryFile()
+ var done uint64
+ seg, gap := rw.f.data.Find(uint64(rw.offset))
+ for rw.offset < end {
+ mr := memmap.MappableRange{uint64(rw.offset), uint64(end)}
+ switch {
+ case seg.Ok():
+ // Get internal mappings.
+ ims, err := mf.MapInternal(seg.FileRangeOf(seg.Range().Intersect(mr)), usermem.Read)
+ if err != nil {
+ return done, err
+ }
+
+ // Copy from internal mappings.
+ n, err := safemem.CopySeq(dsts, ims)
+ done += n
+ rw.offset += int64(n)
+ dsts = dsts.DropFirst64(n)
+ if err != nil {
+ return done, err
+ }
+
+ // Continue.
+ seg, gap = seg.NextNonEmpty()
+
+ case gap.Ok():
+ // Tmpfs holes are zero-filled.
+ gapmr := gap.Range().Intersect(mr)
+ dst := dsts.TakeFirst64(gapmr.Length())
+ n, err := safemem.ZeroSeq(dst)
+ done += n
+ rw.offset += int64(n)
+ dsts = dsts.DropFirst64(n)
+ if err != nil {
+ return done, err
+ }
+
+ // Continue.
+ seg, gap = gap.NextSegment(), fsutil.FileRangeGapIterator{}
+
+ default:
+ break
+ }
+ }
+ return done, nil
+}
+
+// WriteFromBlocks implements safemem.Writer.WriteFromBlocks.
+func (rw *fileReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) {
+ rw.f.dataMu.Lock()
+ defer rw.f.dataMu.Unlock()
+
+ // Compute the range to write.
+ end := fs.WriteEndOffset(rw.offset, int64(srcs.NumBytes()))
+ if end == rw.offset { // srcs.NumBytes() == 0?
+ return 0, nil
+ }
+
+ // Check if seals prevent either file growth or all writes.
+ switch {
+ case rw.f.seals&linux.F_SEAL_WRITE != 0: // Write sealed
+ return 0, syserror.EPERM
+ case end > rw.f.attr.Size && rw.f.seals&linux.F_SEAL_GROW != 0: // Grow sealed
+ // When growth is sealed, Linux effectively allows writes which would
+ // normally grow the file to partially succeed up to the current EOF,
+ // rounded down to the page boundary before the EOF.
+ //
+ // This happens because writes (and thus the growth check) for tmpfs
+ // files proceed page-by-page on Linux, and the final write to the page
+ // containing EOF fails, resulting in a partial write up to the start of
+ // that page.
+ //
+ // To emulate this behaviour, artifically truncate the write to the
+ // start of the page containing the current EOF.
+ //
+ // See Linux, mm/filemap.c:generic_perform_write() and
+ // mm/shmem.c:shmem_write_begin().
+ if pgstart := int64(usermem.Addr(rw.f.attr.Size).RoundDown()); end > pgstart {
+ end = pgstart
+ }
+ if end <= rw.offset {
+ // Truncation would result in no data being written.
+ return 0, syserror.EPERM
+ }
+ }
+
+ defer func() {
+ // If the write ends beyond the file's previous size, it causes the
+ // file to grow.
+ if rw.offset > rw.f.attr.Size {
+ rw.f.attr.Size = rw.offset
+ }
+ }()
+
+ mf := rw.f.kernel.MemoryFile()
+ // Page-aligned mr for when we need to allocate memory. RoundUp can't
+ // overflow since end is an int64.
+ pgstartaddr := usermem.Addr(rw.offset).RoundDown()
+ pgendaddr, _ := usermem.Addr(end).RoundUp()
+ pgMR := memmap.MappableRange{uint64(pgstartaddr), uint64(pgendaddr)}
+
+ var done uint64
+ seg, gap := rw.f.data.Find(uint64(rw.offset))
+ for rw.offset < end {
+ mr := memmap.MappableRange{uint64(rw.offset), uint64(end)}
+ switch {
+ case seg.Ok():
+ // Get internal mappings.
+ ims, err := mf.MapInternal(seg.FileRangeOf(seg.Range().Intersect(mr)), usermem.Write)
+ if err != nil {
+ return done, err
+ }
+
+ // Copy to internal mappings.
+ n, err := safemem.CopySeq(ims, srcs)
+ done += n
+ rw.offset += int64(n)
+ srcs = srcs.DropFirst64(n)
+ if err != nil {
+ return done, err
+ }
+
+ // Continue.
+ seg, gap = seg.NextNonEmpty()
+
+ case gap.Ok():
+ // Allocate memory for the write.
+ gapMR := gap.Range().Intersect(pgMR)
+ fr, err := mf.Allocate(gapMR.Length(), rw.f.memUsage)
+ if err != nil {
+ return done, err
+ }
+
+ // Write to that memory as usual.
+ seg, gap = rw.f.data.Insert(gap, gapMR, fr.Start), fsutil.FileRangeGapIterator{}
+
+ default:
+ break
+ }
+ }
+ return done, nil
+}
+
+// AddMapping implements memmap.Mappable.AddMapping.
+func (f *fileInodeOperations) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) error {
+ f.mapsMu.Lock()
+ defer f.mapsMu.Unlock()
+
+ f.dataMu.RLock()
+ defer f.dataMu.RUnlock()
+
+ // Reject writable mapping if F_SEAL_WRITE is set.
+ if f.seals&linux.F_SEAL_WRITE != 0 && writable {
+ return syserror.EPERM
+ }
+
+ f.mappings.AddMapping(ms, ar, offset, writable)
+ if writable {
+ pagesBefore := f.writableMappingPages
+
+ // ar is guaranteed to be page aligned per memmap.Mappable.
+ f.writableMappingPages += uint64(ar.Length() / usermem.PageSize)
+
+ if f.writableMappingPages < pagesBefore {
+ panic(fmt.Sprintf("Overflow while mapping potentially writable pages pointing to a tmpfs file. Before %v, after %v", pagesBefore, f.writableMappingPages))
+ }
+ }
+
+ return nil
+}
+
+// RemoveMapping implements memmap.Mappable.RemoveMapping.
+func (f *fileInodeOperations) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64, writable bool) {
+ f.mapsMu.Lock()
+ defer f.mapsMu.Unlock()
+
+ f.mappings.RemoveMapping(ms, ar, offset, writable)
+
+ if writable {
+ pagesBefore := f.writableMappingPages
+
+ // ar is guaranteed to be page aligned per memmap.Mappable.
+ f.writableMappingPages -= uint64(ar.Length() / usermem.PageSize)
+
+ if f.writableMappingPages > pagesBefore {
+ panic(fmt.Sprintf("Underflow while unmapping potentially writable pages pointing to a tmpfs file. Before %v, after %v", pagesBefore, f.writableMappingPages))
+ }
+ }
+}
+
+// CopyMapping implements memmap.Mappable.CopyMapping.
+func (f *fileInodeOperations) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR usermem.AddrRange, offset uint64, writable bool) error {
+ return f.AddMapping(ctx, ms, dstAR, offset, writable)
+}
+
+// Translate implements memmap.Mappable.Translate.
+func (f *fileInodeOperations) Translate(ctx context.Context, required, optional memmap.MappableRange, at usermem.AccessType) ([]memmap.Translation, error) {
+ f.dataMu.Lock()
+ defer f.dataMu.Unlock()
+
+ // Constrain translations to f.attr.Size (rounded up) to prevent
+ // translation to pages that may be concurrently truncated.
+ pgend := fs.OffsetPageEnd(f.attr.Size)
+ var beyondEOF bool
+ if required.End > pgend {
+ if required.Start >= pgend {
+ return nil, &memmap.BusError{io.EOF}
+ }
+ beyondEOF = true
+ required.End = pgend
+ }
+ if optional.End > pgend {
+ optional.End = pgend
+ }
+
+ mf := f.kernel.MemoryFile()
+ cerr := f.data.Fill(ctx, required, optional, mf, f.memUsage, func(_ context.Context, dsts safemem.BlockSeq, _ uint64) (uint64, error) {
+ // Newly-allocated pages are zeroed, so we don't need to do anything.
+ return dsts.NumBytes(), nil
+ })
+
+ var ts []memmap.Translation
+ var translatedEnd uint64
+ for seg := f.data.FindSegment(required.Start); seg.Ok() && seg.Start() < required.End; seg, _ = seg.NextNonEmpty() {
+ segMR := seg.Range().Intersect(optional)
+ ts = append(ts, memmap.Translation{
+ Source: segMR,
+ File: mf,
+ Offset: seg.FileRangeOf(segMR).Start,
+ Perms: usermem.AnyAccess,
+ })
+ translatedEnd = segMR.End
+ }
+
+ // Don't return the error returned by f.data.Fill if it occurred outside of
+ // required.
+ if translatedEnd < required.End && cerr != nil {
+ return ts, &memmap.BusError{cerr}
+ }
+ if beyondEOF {
+ return ts, &memmap.BusError{io.EOF}
+ }
+ return ts, nil
+}
+
+// InvalidateUnsavable implements memmap.Mappable.InvalidateUnsavable.
+func (f *fileInodeOperations) InvalidateUnsavable(ctx context.Context) error {
+ return nil
+}
+
+// GetSeals returns the current set of seals on a memfd inode.
+func GetSeals(inode *fs.Inode) (uint32, error) {
+ if f, ok := inode.InodeOperations.(*fileInodeOperations); ok {
+ f.dataMu.RLock()
+ defer f.dataMu.RUnlock()
+ return f.seals, nil
+ }
+ // Not a memfd inode.
+ return 0, syserror.EINVAL
+}
+
+// AddSeals adds new file seals to a memfd inode.
+func AddSeals(inode *fs.Inode, val uint32) error {
+ if f, ok := inode.InodeOperations.(*fileInodeOperations); ok {
+ f.mapsMu.Lock()
+ defer f.mapsMu.Unlock()
+ f.dataMu.Lock()
+ defer f.dataMu.Unlock()
+
+ if f.seals&linux.F_SEAL_SEAL != 0 {
+ // Seal applied which prevents addition of any new seals.
+ return syserror.EPERM
+ }
+
+ // F_SEAL_WRITE can only be added if there are no active writable maps.
+ if f.seals&linux.F_SEAL_WRITE == 0 && val&linux.F_SEAL_WRITE != 0 {
+ if f.writableMappingPages > 0 {
+ return syserror.EBUSY
+ }
+ }
+
+ // Seals can only be added, never removed.
+ f.seals |= val
+ return nil
+ }
+ // Not a memfd inode.
+ return syserror.EINVAL
+}
diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go
new file mode 100644
index 000000000..263d10cfe
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/tmpfs.go
@@ -0,0 +1,348 @@
+// Copyright 2018 The gVisor Authors.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Package tmpfs is a filesystem implementation backed by memory.
+package tmpfs
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/abi/linux"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/pipe"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix/transport"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usage"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
+ "gvisor.googlesource.com/gvisor/pkg/syserror"
+)
+
+var fsInfo = fs.Info{
+ Type: linux.TMPFS_MAGIC,
+
+ // TODO(b/29637826): allow configuring a tmpfs size and enforce it.
+ TotalBlocks: 0,
+ FreeBlocks: 0,
+}
+
+// rename implements fs.InodeOperations.Rename for tmpfs nodes.
+func rename(ctx context.Context, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string, replacement bool) error {
+ op, ok := oldParent.InodeOperations.(*Dir)
+ if !ok {
+ return syserror.EXDEV
+ }
+ np, ok := newParent.InodeOperations.(*Dir)
+ if !ok {
+ return syserror.EXDEV
+ }
+ return ramfs.Rename(ctx, op.ramfsDir, oldName, np.ramfsDir, newName, replacement)
+}
+
+// Dir is a directory.
+//
+// +stateify savable
+type Dir struct {
+ fsutil.InodeGenericChecker `state:"nosave"`
+ fsutil.InodeIsDirTruncate `state:"nosave"`
+ fsutil.InodeNoopRelease `state:"nosave"`
+ fsutil.InodeNoopWriteOut `state:"nosave"`
+ fsutil.InodeNotMappable `state:"nosave"`
+ fsutil.InodeNotSocket `state:"nosave"`
+ fsutil.InodeNotSymlink `state:"nosave"`
+ fsutil.InodeVirtual `state:"nosave"`
+
+ // Ideally this would be embedded, so that we "inherit" all of the
+ // InodeOperations implemented by ramfs.Dir for free.
+ //
+ // However, ramfs.dirFileOperations stores a pointer to a ramfs.Dir,
+ // and our save/restore package does not allow saving a pointer to an
+ // embedded field elsewhere.
+ //
+ // Thus, we must make the ramfs.Dir is a field, and we delegate all the
+ // InodeOperation methods to it.
+ ramfsDir *ramfs.Dir
+
+ // kernel is used to allocate memory as storage for tmpfs Files.
+ kernel *kernel.Kernel
+}
+
+var _ fs.InodeOperations = (*Dir)(nil)
+
+// NewDir returns a new directory.
+func NewDir(ctx context.Context, contents map[string]*fs.Inode, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode {
+ d := &Dir{
+ ramfsDir: ramfs.NewDir(ctx, contents, owner, perms),
+ kernel: kernel.KernelFromContext(ctx),
+ }
+
+ // Manually set the CreateOps.
+ d.ramfsDir.CreateOps = d.newCreateOps()
+
+ return fs.NewInode(d, msrc, fs.StableAttr{
+ DeviceID: tmpfsDevice.DeviceID(),
+ InodeID: tmpfsDevice.NextIno(),
+ BlockSize: usermem.PageSize,
+ Type: fs.Directory,
+ })
+}
+
+// afterLoad is invoked by stateify.
+func (d *Dir) afterLoad() {
+ // Per NewDir, manually set the CreateOps.
+ d.ramfsDir.CreateOps = d.newCreateOps()
+}
+
+// GetFile implements fs.InodeOperations.GetFile.
+func (d *Dir) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags) (*fs.File, error) {
+ return d.ramfsDir.GetFile(ctx, dirent, flags)
+}
+
+// AddLink implements fs.InodeOperations.AddLink.
+func (d *Dir) AddLink() {
+ d.ramfsDir.AddLink()
+}
+
+// DropLink implements fs.InodeOperations.DropLink.
+func (d *Dir) DropLink() {
+ d.ramfsDir.DropLink()
+}
+
+// Bind implements fs.InodeOperations.Bind.
+func (d *Dir) Bind(ctx context.Context, dir *fs.Inode, name string, ep transport.BoundEndpoint, perms fs.FilePermissions) (*fs.Dirent, error) {
+ return d.ramfsDir.Bind(ctx, dir, name, ep, perms)
+}
+
+// Create implements fs.InodeOperations.Create.
+func (d *Dir) Create(ctx context.Context, dir *fs.Inode, name string, flags fs.FileFlags, perms fs.FilePermissions) (*fs.File, error) {
+ return d.ramfsDir.Create(ctx, dir, name, flags, perms)
+}
+
+// CreateLink implements fs.InodeOperations.CreateLink.
+func (d *Dir) CreateLink(ctx context.Context, dir *fs.Inode, oldname, newname string) error {
+ return d.ramfsDir.CreateLink(ctx, dir, oldname, newname)
+}
+
+// CreateHardLink implements fs.InodeOperations.CreateHardLink.
+func (d *Dir) CreateHardLink(ctx context.Context, dir *fs.Inode, target *fs.Inode, name string) error {
+ return d.ramfsDir.CreateHardLink(ctx, dir, target, name)
+}
+
+// CreateDirectory implements fs.InodeOperations.CreateDirectory.
+func (d *Dir) CreateDirectory(ctx context.Context, dir *fs.Inode, name string, perms fs.FilePermissions) error {
+ return d.ramfsDir.CreateDirectory(ctx, dir, name, perms)
+}
+
+// CreateFifo implements fs.InodeOperations.CreateFifo.
+func (d *Dir) CreateFifo(ctx context.Context, dir *fs.Inode, name string, perms fs.FilePermissions) error {
+ return d.ramfsDir.CreateFifo(ctx, dir, name, perms)
+}
+
+// Getxattr implements fs.InodeOperations.Getxattr.
+func (d *Dir) Getxattr(i *fs.Inode, name string) (string, error) {
+ return d.ramfsDir.Getxattr(i, name)
+}
+
+// Setxattr implements fs.InodeOperations.Setxattr.
+func (d *Dir) Setxattr(i *fs.Inode, name, value string) error {
+ return d.ramfsDir.Setxattr(i, name, value)
+}
+
+// Listxattr implements fs.InodeOperations.Listxattr.
+func (d *Dir) Listxattr(i *fs.Inode) (map[string]struct{}, error) {
+ return d.ramfsDir.Listxattr(i)
+}
+
+// Lookup implements fs.InodeOperations.Lookup.
+func (d *Dir) Lookup(ctx context.Context, i *fs.Inode, p string) (*fs.Dirent, error) {
+ return d.ramfsDir.Lookup(ctx, i, p)
+}
+
+// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange.
+func (d *Dir) NotifyStatusChange(ctx context.Context) {
+ d.ramfsDir.NotifyStatusChange(ctx)
+}
+
+// Remove implements fs.InodeOperations.Remove.
+func (d *Dir) Remove(ctx context.Context, i *fs.Inode, name string) error {
+ return d.ramfsDir.Remove(ctx, i, name)
+}
+
+// RemoveDirectory implements fs.InodeOperations.RemoveDirectory.
+func (d *Dir) RemoveDirectory(ctx context.Context, i *fs.Inode, name string) error {
+ return d.ramfsDir.RemoveDirectory(ctx, i, name)
+}
+
+// UnstableAttr implements fs.InodeOperations.UnstableAttr.
+func (d *Dir) UnstableAttr(ctx context.Context, i *fs.Inode) (fs.UnstableAttr, error) {
+ return d.ramfsDir.UnstableAttr(ctx, i)
+}
+
+// SetPermissions implements fs.InodeOperations.SetPermissions.
+func (d *Dir) SetPermissions(ctx context.Context, i *fs.Inode, p fs.FilePermissions) bool {
+ return d.ramfsDir.SetPermissions(ctx, i, p)
+}
+
+// SetOwner implements fs.InodeOperations.SetOwner.
+func (d *Dir) SetOwner(ctx context.Context, i *fs.Inode, owner fs.FileOwner) error {
+ return d.ramfsDir.SetOwner(ctx, i, owner)
+}
+
+// SetTimestamps implements fs.InodeOperations.SetTimestamps.
+func (d *Dir) SetTimestamps(ctx context.Context, i *fs.Inode, ts fs.TimeSpec) error {
+ return d.ramfsDir.SetTimestamps(ctx, i, ts)
+}
+
+// newCreateOps builds the custom CreateOps for this Dir.
+func (d *Dir) newCreateOps() *ramfs.CreateOps {
+ return &ramfs.CreateOps{
+ NewDir: func(ctx context.Context, dir *fs.Inode, perms fs.FilePermissions) (*fs.Inode, error) {
+ return NewDir(ctx, nil, fs.FileOwnerFromContext(ctx), perms, dir.MountSource), nil
+ },
+ NewFile: func(ctx context.Context, dir *fs.Inode, perms fs.FilePermissions) (*fs.Inode, error) {
+ uattr := fs.WithCurrentTime(ctx, fs.UnstableAttr{
+ Owner: fs.FileOwnerFromContext(ctx),
+ Perms: perms,
+ // Always start unlinked.
+ Links: 0,
+ })
+ iops := NewInMemoryFile(ctx, usage.Tmpfs, uattr)
+ return fs.NewInode(iops, dir.MountSource, fs.StableAttr{
+ DeviceID: tmpfsDevice.DeviceID(),
+ InodeID: tmpfsDevice.NextIno(),
+ BlockSize: usermem.PageSize,
+ Type: fs.RegularFile,
+ }), nil
+ },
+ NewSymlink: func(ctx context.Context, dir *fs.Inode, target string) (*fs.Inode, error) {
+ return NewSymlink(ctx, target, fs.FileOwnerFromContext(ctx), dir.MountSource), nil
+ },
+ NewBoundEndpoint: func(ctx context.Context, dir *fs.Inode, socket transport.BoundEndpoint, perms fs.FilePermissions) (*fs.Inode, error) {
+ return NewSocket(ctx, socket, fs.FileOwnerFromContext(ctx), perms, dir.MountSource), nil
+ },
+ NewFifo: func(ctx context.Context, dir *fs.Inode, perms fs.FilePermissions) (*fs.Inode, error) {
+ return NewFifo(ctx, fs.FileOwnerFromContext(ctx), perms, dir.MountSource), nil
+ },
+ }
+}
+
+// Rename implements fs.InodeOperations.Rename.
+func (d *Dir) Rename(ctx context.Context, inode *fs.Inode, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string, replacement bool) error {
+ return rename(ctx, oldParent, oldName, newParent, newName, replacement)
+}
+
+// StatFS implements fs.InodeOperations.StatFS.
+func (*Dir) StatFS(context.Context) (fs.Info, error) {
+ return fsInfo, nil
+}
+
+// Allocate implements fs.InodeOperations.Allocate.
+func (d *Dir) Allocate(ctx context.Context, node *fs.Inode, offset, length int64) error {
+ return d.ramfsDir.Allocate(ctx, node, offset, length)
+}
+
+// Symlink is a symlink.
+//
+// +stateify savable
+type Symlink struct {
+ ramfs.Symlink
+}
+
+// NewSymlink returns a new symlink with the provided permissions.
+func NewSymlink(ctx context.Context, target string, owner fs.FileOwner, msrc *fs.MountSource) *fs.Inode {
+ s := &Symlink{Symlink: *ramfs.NewSymlink(ctx, owner, target)}
+ return fs.NewInode(s, msrc, fs.StableAttr{
+ DeviceID: tmpfsDevice.DeviceID(),
+ InodeID: tmpfsDevice.NextIno(),
+ BlockSize: usermem.PageSize,
+ Type: fs.Symlink,
+ })
+}
+
+// Rename implements fs.InodeOperations.Rename.
+func (s *Symlink) Rename(ctx context.Context, inode *fs.Inode, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string, replacement bool) error {
+ return rename(ctx, oldParent, oldName, newParent, newName, replacement)
+}
+
+// StatFS returns the tmpfs info.
+func (s *Symlink) StatFS(context.Context) (fs.Info, error) {
+ return fsInfo, nil
+}
+
+// Socket is a socket.
+//
+// +stateify savable
+type Socket struct {
+ ramfs.Socket
+ fsutil.InodeNotTruncatable `state:"nosave"`
+ fsutil.InodeNotAllocatable `state:"nosave"`
+}
+
+// NewSocket returns a new socket with the provided permissions.
+func NewSocket(ctx context.Context, socket transport.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode {
+ s := &Socket{Socket: *ramfs.NewSocket(ctx, socket, owner, perms)}
+ return fs.NewInode(s, msrc, fs.StableAttr{
+ DeviceID: tmpfsDevice.DeviceID(),
+ InodeID: tmpfsDevice.NextIno(),
+ BlockSize: usermem.PageSize,
+ Type: fs.Socket,
+ })
+}
+
+// Rename implements fs.InodeOperations.Rename.
+func (s *Socket) Rename(ctx context.Context, inode *fs.Inode, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string, replacement bool) error {
+ return rename(ctx, oldParent, oldName, newParent, newName, replacement)
+}
+
+// StatFS returns the tmpfs info.
+func (s *Socket) StatFS(context.Context) (fs.Info, error) {
+ return fsInfo, nil
+}
+
+// Fifo is a tmpfs named pipe.
+//
+// +stateify savable
+type Fifo struct {
+ fs.InodeOperations
+}
+
+// NewFifo creates a new named pipe.
+func NewFifo(ctx context.Context, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode {
+ // First create a pipe.
+ p := pipe.NewPipe(ctx, true /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize)
+
+ // Build pipe InodeOperations.
+ iops := pipe.NewInodeOperations(ctx, perms, p)
+
+ // Wrap the iops with our Fifo.
+ fifoIops := &Fifo{iops}
+
+ // Build a new Inode.
+ return fs.NewInode(fifoIops, msrc, fs.StableAttr{
+ DeviceID: tmpfsDevice.DeviceID(),
+ InodeID: tmpfsDevice.NextIno(),
+ BlockSize: usermem.PageSize,
+ Type: fs.Pipe,
+ })
+}
+
+// Rename implements fs.InodeOperations.Rename.
+func (f *Fifo) Rename(ctx context.Context, inode *fs.Inode, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string, replacement bool) error {
+ return rename(ctx, oldParent, oldName, newParent, newName, replacement)
+}
+
+// StatFS returns the tmpfs info.
+func (*Fifo) StatFS(context.Context) (fs.Info, error) {
+ return fsInfo, nil
+}
diff --git a/pkg/sentry/fs/tmpfs/tmpfs_state_autogen.go b/pkg/sentry/fs/tmpfs/tmpfs_state_autogen.go
new file mode 100755
index 000000000..0fe2e2e93
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/tmpfs_state_autogen.go
@@ -0,0 +1,108 @@
+// automatically generated by stateify.
+
+package tmpfs
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/state"
+)
+
+func (x *regularFileOperations) beforeSave() {}
+func (x *regularFileOperations) save(m state.Map) {
+ x.beforeSave()
+ m.Save("iops", &x.iops)
+}
+
+func (x *regularFileOperations) afterLoad() {}
+func (x *regularFileOperations) load(m state.Map) {
+ m.Load("iops", &x.iops)
+}
+
+func (x *Filesystem) beforeSave() {}
+func (x *Filesystem) save(m state.Map) {
+ x.beforeSave()
+}
+
+func (x *Filesystem) afterLoad() {}
+func (x *Filesystem) load(m state.Map) {
+}
+
+func (x *fileInodeOperations) beforeSave() {}
+func (x *fileInodeOperations) save(m state.Map) {
+ x.beforeSave()
+ m.Save("InodeSimpleExtendedAttributes", &x.InodeSimpleExtendedAttributes)
+ m.Save("kernel", &x.kernel)
+ m.Save("memUsage", &x.memUsage)
+ m.Save("attr", &x.attr)
+ m.Save("mappings", &x.mappings)
+ m.Save("writableMappingPages", &x.writableMappingPages)
+ m.Save("data", &x.data)
+ m.Save("seals", &x.seals)
+}
+
+func (x *fileInodeOperations) afterLoad() {}
+func (x *fileInodeOperations) load(m state.Map) {
+ m.Load("InodeSimpleExtendedAttributes", &x.InodeSimpleExtendedAttributes)
+ m.Load("kernel", &x.kernel)
+ m.Load("memUsage", &x.memUsage)
+ m.Load("attr", &x.attr)
+ m.Load("mappings", &x.mappings)
+ m.Load("writableMappingPages", &x.writableMappingPages)
+ m.Load("data", &x.data)
+ m.Load("seals", &x.seals)
+}
+
+func (x *Dir) beforeSave() {}
+func (x *Dir) save(m state.Map) {
+ x.beforeSave()
+ m.Save("ramfsDir", &x.ramfsDir)
+ m.Save("kernel", &x.kernel)
+}
+
+func (x *Dir) load(m state.Map) {
+ m.Load("ramfsDir", &x.ramfsDir)
+ m.Load("kernel", &x.kernel)
+ m.AfterLoad(x.afterLoad)
+}
+
+func (x *Symlink) beforeSave() {}
+func (x *Symlink) save(m state.Map) {
+ x.beforeSave()
+ m.Save("Symlink", &x.Symlink)
+}
+
+func (x *Symlink) afterLoad() {}
+func (x *Symlink) load(m state.Map) {
+ m.Load("Symlink", &x.Symlink)
+}
+
+func (x *Socket) beforeSave() {}
+func (x *Socket) save(m state.Map) {
+ x.beforeSave()
+ m.Save("Socket", &x.Socket)
+}
+
+func (x *Socket) afterLoad() {}
+func (x *Socket) load(m state.Map) {
+ m.Load("Socket", &x.Socket)
+}
+
+func (x *Fifo) beforeSave() {}
+func (x *Fifo) save(m state.Map) {
+ x.beforeSave()
+ m.Save("InodeOperations", &x.InodeOperations)
+}
+
+func (x *Fifo) afterLoad() {}
+func (x *Fifo) load(m state.Map) {
+ m.Load("InodeOperations", &x.InodeOperations)
+}
+
+func init() {
+ state.Register("tmpfs.regularFileOperations", (*regularFileOperations)(nil), state.Fns{Save: (*regularFileOperations).save, Load: (*regularFileOperations).load})
+ state.Register("tmpfs.Filesystem", (*Filesystem)(nil), state.Fns{Save: (*Filesystem).save, Load: (*Filesystem).load})
+ state.Register("tmpfs.fileInodeOperations", (*fileInodeOperations)(nil), state.Fns{Save: (*fileInodeOperations).save, Load: (*fileInodeOperations).load})
+ state.Register("tmpfs.Dir", (*Dir)(nil), state.Fns{Save: (*Dir).save, Load: (*Dir).load})
+ state.Register("tmpfs.Symlink", (*Symlink)(nil), state.Fns{Save: (*Symlink).save, Load: (*Symlink).load})
+ state.Register("tmpfs.Socket", (*Socket)(nil), state.Fns{Save: (*Socket).save, Load: (*Socket).load})
+ state.Register("tmpfs.Fifo", (*Fifo)(nil), state.Fns{Save: (*Fifo).save, Load: (*Fifo).load})
+}