From d02b74a5dcfed4bfc8f2f8e545bca4d2afabb296 Mon Sep 17 00:00:00 2001
From: Googler <noreply@google.com>
Date: Fri, 27 Apr 2018 10:37:02 -0700
Subject: Check in gVisor.

PiperOrigin-RevId: 194583126
Change-Id: Ica1d8821a90f74e7e745962d71801c598c652463
---
 pkg/sentry/fs/tmpfs/BUILD           |  64 +++++
 pkg/sentry/fs/tmpfs/device.go       |  20 ++
 pkg/sentry/fs/tmpfs/file_regular.go |  56 ++++
 pkg/sentry/fs/tmpfs/file_test.go    |  73 ++++++
 pkg/sentry/fs/tmpfs/fs.go           | 131 ++++++++++
 pkg/sentry/fs/tmpfs/inode_file.go   | 492 ++++++++++++++++++++++++++++++++++++
 pkg/sentry/fs/tmpfs/tmpfs.go        | 204 +++++++++++++++
 7 files changed, 1040 insertions(+)
 create mode 100644 pkg/sentry/fs/tmpfs/BUILD
 create mode 100644 pkg/sentry/fs/tmpfs/device.go
 create mode 100644 pkg/sentry/fs/tmpfs/file_regular.go
 create mode 100644 pkg/sentry/fs/tmpfs/file_test.go
 create mode 100644 pkg/sentry/fs/tmpfs/fs.go
 create mode 100644 pkg/sentry/fs/tmpfs/inode_file.go
 create mode 100644 pkg/sentry/fs/tmpfs/tmpfs.go

(limited to 'pkg/sentry/fs/tmpfs')

diff --git a/pkg/sentry/fs/tmpfs/BUILD b/pkg/sentry/fs/tmpfs/BUILD
new file mode 100644
index 000000000..be4e695d3
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/BUILD
@@ -0,0 +1,64 @@
+package(licenses = ["notice"])  # Apache 2.0
+
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+load("//tools/go_stateify:defs.bzl", "go_stateify")
+
+go_stateify(
+    name = "tmpfs_state",
+    srcs = [
+        "file_regular.go",
+        "fs.go",
+        "inode_file.go",
+        "tmpfs.go",
+    ],
+    out = "tmpfs_state.go",
+    package = "tmpfs",
+)
+
+go_library(
+    name = "tmpfs",
+    srcs = [
+        "device.go",
+        "file_regular.go",
+        "fs.go",
+        "inode_file.go",
+        "tmpfs.go",
+        "tmpfs_state.go",
+    ],
+    importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs/tmpfs",
+    visibility = ["//pkg/sentry:internal"],
+    deps = [
+        "//pkg/abi/linux",
+        "//pkg/log",
+        "//pkg/sentry/context",
+        "//pkg/sentry/device",
+        "//pkg/sentry/fs",
+        "//pkg/sentry/fs/fsutil",
+        "//pkg/sentry/fs/ramfs",
+        "//pkg/sentry/kernel/auth",
+        "//pkg/sentry/kernel/pipe",
+        "//pkg/sentry/memmap",
+        "//pkg/sentry/platform",
+        "//pkg/sentry/safemem",
+        "//pkg/sentry/usage",
+        "//pkg/sentry/usermem",
+        "//pkg/state",
+        "//pkg/tcpip/transport/unix",
+        "//pkg/waiter",
+    ],
+)
+
+go_test(
+    name = "tmpfs_test",
+    size = "small",
+    srcs = ["file_test.go"],
+    embed = [":tmpfs"],
+    deps = [
+        "//pkg/sentry/context",
+        "//pkg/sentry/context/contexttest",
+        "//pkg/sentry/fs",
+        "//pkg/sentry/platform",
+        "//pkg/sentry/usage",
+        "//pkg/sentry/usermem",
+    ],
+)
diff --git a/pkg/sentry/fs/tmpfs/device.go b/pkg/sentry/fs/tmpfs/device.go
new file mode 100644
index 000000000..e588b3440
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/device.go
@@ -0,0 +1,20 @@
+// Copyright 2018 Google Inc.
+//
+// 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..9811d90bc
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/file_regular.go
@@ -0,0 +1,56 @@
+// Copyright 2018 Google Inc.
+//
+// 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.
+type regularFileOperations struct {
+	waiter.AlwaysReady   `state:"nosave"`
+	fsutil.NoopRelease   `state:"nosave"`
+	fsutil.GenericSeek   `state:"nosave"`
+	fsutil.NotDirReaddir `state:"nosave"`
+	fsutil.NoopFsync     `state:"nosave"`
+	fsutil.NoopFlush     `state:"nosave"`
+	fsutil.NoIoctl       `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, 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/file_test.go b/pkg/sentry/fs/tmpfs/file_test.go
new file mode 100644
index 000000000..f064eb1ac
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/file_test.go
@@ -0,0 +1,73 @@
+// Copyright 2018 Google Inc.
+//
+// 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 (
+	"bytes"
+	"testing"
+
+	"gvisor.googlesource.com/gvisor/pkg/sentry/context"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/context/contexttest"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/platform"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/usage"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
+)
+
+func newFileInode(ctx context.Context) *fs.Inode {
+	m := fs.NewCachingMountSource(&Filesystem{}, fs.MountSourceFlags{})
+	iops := NewInMemoryFile(ctx, usage.Tmpfs, fs.WithCurrentTime(ctx, fs.UnstableAttr{}), platform.FromContext(ctx))
+	return fs.NewInode(iops, m, fs.StableAttr{
+		DeviceID:  tmpfsDevice.DeviceID(),
+		InodeID:   tmpfsDevice.NextIno(),
+		BlockSize: usermem.PageSize,
+		Type:      fs.RegularFile,
+	})
+}
+
+func newFile(ctx context.Context) *fs.File {
+	inode := newFileInode(ctx)
+	f, _ := inode.GetFile(ctx, fs.NewDirent(inode, "stub"), fs.FileFlags{Read: true, Write: true})
+	return f
+}
+
+// Allocate once, write twice.
+func TestGrow(t *testing.T) {
+	ctx := contexttest.Context(t)
+	f := newFile(ctx)
+	defer f.DecRef()
+
+	abuf := bytes.Repeat([]byte{'a'}, 68)
+	n, err := f.Pwritev(ctx, usermem.BytesIOSequence(abuf), 0)
+	if n != int64(len(abuf)) || err != nil {
+		t.Fatalf("DeprecatedPwritev got (%d, %v) want (%d, nil)", n, err, len(abuf))
+	}
+
+	bbuf := bytes.Repeat([]byte{'b'}, 856)
+	n, err = f.Pwritev(ctx, usermem.BytesIOSequence(bbuf), 68)
+	if n != int64(len(bbuf)) || err != nil {
+		t.Fatalf("DeprecatedPwritev got (%d, %v) want (%d, nil)", n, err, len(bbuf))
+	}
+
+	rbuf := make([]byte, len(abuf)+len(bbuf))
+	n, err = f.Preadv(ctx, usermem.BytesIOSequence(rbuf), 0)
+	if n != int64(len(rbuf)) || err != nil {
+		t.Fatalf("DeprecatedPreadv got (%d, %v) want (%d, nil)", n, err, len(rbuf))
+	}
+
+	if want := append(abuf, bbuf...); !bytes.Equal(rbuf, want) {
+		t.Fatalf("Read %v, want %v", rbuf, want)
+	}
+}
diff --git a/pkg/sentry/fs/tmpfs/fs.go b/pkg/sentry/fs/tmpfs/fs.go
new file mode 100644
index 000000000..639a19b0d
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/fs.go
@@ -0,0 +1,131 @@
+// Copyright 2018 Google Inc.
+//
+// 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"
+	"regexp"
+	"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"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/platform"
+)
+
+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: support a tmpfs size limit.
+	// size = "size"
+
+	// default permissions are read/write/execute.
+	defaultMode = 0777
+)
+
+// modeRegexp is the expected format of the mode option.
+var modeRegexp = regexp.MustCompile("0[0-7][0-7][0-7]")
+
+// Filesystem is a tmpfs.
+type Filesystem struct{}
+
+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
+}
+
+// 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) (*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 {
+		if !modeRegexp.MatchString(m) {
+			return nil, fmt.Errorf("unsupported mode value: 'mode=%s'", m)
+		}
+		// It's basically impossible that we error out at this point,
+		// maybe we should panic.
+		i, err := strconv.ParseUint(m, 8, 32)
+		if err != nil {
+			return nil, fmt.Errorf("mode value not parsable 'mode=%s': %v", m, err)
+		}
+		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, platform.FromContext(ctx)), 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..66bc934ae
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/inode_file.go
@@ -0,0 +1,492 @@
+// Copyright 2018 Google Inc.
+//
+// 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 (
+	"io"
+	"sync"
+
+	"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/platform"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/safemem"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/usage"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
+)
+
+// fileInodeOperations implements fs.InodeOperations for a regular tmpfs file.
+// These files are backed by FrameRegions allocated from a platform.Memory,
+// and may be directly mapped.
+//
+// The tmpfs file memory is backed by FrameRegions, each of which is reference
+// counted. frames maintains a single reference on each of the FrameRegions.
+// Since these contain the contents of the file, the reference may only be
+// decremented once this file is both deleted and all handles to the file have
+// been closed.
+//
+// Mappable users may also call IncRefOn/DecRefOn, generally to indicate that
+// they plan to use MapInto to map the file into an AddressSpace. These calls
+// include an InvalidatorRegion associated with that reference. When the
+// referenced portion of the file is removed (with Truncate), the associated
+// InvalidatorRegion is invalidated.
+type fileInodeOperations struct {
+	fsutil.DeprecatedFileOperations `state:"nosave"`
+	fsutil.InodeNotDirectory        `state:"nosave"`
+	fsutil.InodeNotSocket           `state:"nosave"`
+	fsutil.InodeNotSymlink          `state:"nosave"`
+	fsutil.NoopWriteOut             `state:"nosave"`
+
+	// platform is used to allocate memory that stores the file's contents.
+	platform platform.Platform
+
+	// 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.Unstable.Size is protected by both
+	// attrMu and dataMu; reading it requires locking either mutex, while
+	// mutating it requires locking both.
+	attr fsutil.InMemoryAttributes
+
+	mapsMu sync.Mutex `state:"nosave"`
+
+	// mappings tracks mappings of the file into memmap.MappingSpaces.
+	//
+	// mappings is protected by mapsMu.
+	mappings memmap.MappingSet
+
+	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
+}
+
+// NewInMemoryFile returns a new file backed by p.Memory().
+func NewInMemoryFile(ctx context.Context, usage usage.MemoryKind, uattr fs.UnstableAttr, p platform.Platform) fs.InodeOperations {
+	return &fileInodeOperations{
+		attr: fsutil.InMemoryAttributes{
+			Unstable: uattr,
+		},
+		platform: p,
+		memUsage: usage,
+	}
+}
+
+// Release implements fs.InodeOperations.Release.
+func (f *fileInodeOperations) Release(context.Context) {
+	f.dataMu.Lock()
+	defer f.dataMu.Unlock()
+	f.data.DropAll(f.platform.Memory())
+}
+
+// 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, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error {
+	return rename(ctx, oldParent, oldName, newParent, newName)
+}
+
+// GetFile implements fs.InodeOperations.GetFile.
+func (f *fileInodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) {
+	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()
+	defer f.attrMu.Unlock()
+	f.dataMu.RLock()
+	defer f.dataMu.RUnlock()
+	attr := f.attr.Unstable
+	attr.Usage = int64(f.data.Span())
+	return attr, nil
+}
+
+// Getxattr implements fs.InodeOperations.Getxattr.
+func (f *fileInodeOperations) Getxattr(inode *fs.Inode, name string) ([]byte, error) {
+	f.attrMu.Lock()
+	defer f.attrMu.Unlock()
+	return f.attr.Getxattr(name)
+}
+
+// Setxattr implements fs.InodeOperations.Setxattr.
+func (f *fileInodeOperations) Setxattr(inode *fs.Inode, name string, value []byte) error {
+	f.attrMu.Lock()
+	defer f.attrMu.Unlock()
+	return f.attr.Setxattr(name, value)
+}
+
+// Listxattr implements fs.InodeOperations.Listxattr.
+func (f *fileInodeOperations) Listxattr(inode *fs.Inode) (map[string]struct{}, error) {
+	f.attrMu.Lock()
+	defer f.attrMu.Unlock()
+	return f.attr.Listxattr()
+}
+
+// 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, inode *fs.Inode, p fs.FilePermissions) bool {
+	f.attrMu.Lock()
+	defer f.attrMu.Unlock()
+	return f.attr.SetPermissions(ctx, p)
+}
+
+// SetTimestamps implements fs.InodeOperations.SetTimestamps.
+func (f *fileInodeOperations) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error {
+	f.attrMu.Lock()
+	defer f.attrMu.Unlock()
+	return f.attr.SetTimestamps(ctx, ts)
+}
+
+// SetOwner implements fs.InodeOperations.SetOwner.
+func (f *fileInodeOperations) SetOwner(ctx context.Context, inode *fs.Inode, owner fs.FileOwner) error {
+	f.attrMu.Lock()
+	defer f.attrMu.Unlock()
+	return f.attr.SetOwner(ctx, owner)
+}
+
+// Truncate implements fs.InodeOperations.Truncate.
+func (f *fileInodeOperations) Truncate(ctx context.Context, inode *fs.Inode, size int64) error {
+	f.attrMu.Lock()
+	defer f.attrMu.Unlock()
+
+	f.dataMu.Lock()
+	oldSize := f.attr.Unstable.Size
+	if oldSize != size {
+		f.attr.Unstable.Size = size
+		f.attr.TouchModificationTime(ctx)
+	}
+	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.platform.Memory())
+
+	return nil
+}
+
+// AddLink implements fs.InodeOperations.AddLink.
+func (f *fileInodeOperations) AddLink() {
+	f.attrMu.Lock()
+	f.attr.Unstable.Links++
+	f.attrMu.Unlock()
+}
+
+// DropLink implements fs.InodeOperations.DropLink.
+func (f *fileInodeOperations) DropLink() {
+	f.attrMu.Lock()
+	f.attr.Unstable.Links--
+	f.attrMu.Unlock()
+}
+
+// NotifyStatusChange implements fs.InodeOperations.NotifyStatusChange.
+func (f *fileInodeOperations) NotifyStatusChange(ctx context.Context) {
+	f.attrMu.Lock()
+	f.attr.TouchStatusChangeTime(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, dst usermem.IOSequence, offset int64) (int64, error) {
+	// Zero length reads for tmpfs are no-ops.
+	if dst.NumBytes() == 0 {
+		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: Separate out f.attr.Size and use atomics instead of
+	// f.dataMu.
+	f.dataMu.RLock()
+	size := f.attr.Unstable.Size
+	f.dataMu.RUnlock()
+	if offset >= size {
+		return 0, io.EOF
+	}
+
+	n, err := dst.CopyOutFrom(ctx, &fileReadWriter{f, offset})
+	// Compare Linux's mm/filemap.c:do_generic_file_read() => file_accessed().
+	f.attrMu.Lock()
+	f.attr.TouchAccessTime(ctx)
+	f.attrMu.Unlock()
+	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().
+	f.attr.TouchModificationTime(ctx)
+	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.Unstable.Size {
+		return 0, io.EOF
+	}
+	end := fs.ReadEndOffset(rw.offset, int64(dsts.NumBytes()), rw.f.attr.Unstable.Size)
+	if end == rw.offset { // dsts.NumBytes() == 0?
+		return 0, nil
+	}
+
+	mem := rw.f.platform.Memory()
+	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 := mem.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
+	}
+
+	defer func() {
+		// If the write ends beyond the file's previous size, it causes the
+		// file to grow.
+		if rw.offset > rw.f.attr.Unstable.Size {
+			rw.f.attr.Unstable.Size = rw.offset
+		}
+	}()
+
+	mem := rw.f.platform.Memory()
+	// 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 := mem.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 := mem.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) error {
+	f.mapsMu.Lock()
+	defer f.mapsMu.Unlock()
+	f.mappings.AddMapping(ms, ar, offset)
+	return nil
+}
+
+// RemoveMapping implements memmap.Mappable.RemoveMapping.
+func (f *fileInodeOperations) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64) {
+	f.mapsMu.Lock()
+	defer f.mapsMu.Unlock()
+	f.mappings.RemoveMapping(ms, ar, offset)
+}
+
+// CopyMapping implements memmap.Mappable.CopyMapping.
+func (f *fileInodeOperations) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR usermem.AddrRange, offset uint64) error {
+	return f.AddMapping(ctx, ms, dstAR, offset)
+}
+
+// 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.Unstable.Size (rounded up) to prevent
+	// translation to pages that may be concurrently truncated.
+	pgend := fs.OffsetPageEnd(f.attr.Unstable.Size)
+	var buserr error
+	if required.End > pgend {
+		buserr = &memmap.BusError{io.EOF}
+		required.End = pgend
+	}
+	if optional.End > pgend {
+		optional.End = pgend
+	}
+
+	mem := f.platform.Memory()
+	cerr := f.data.Fill(ctx, required, optional, mem, 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:   mem,
+			Offset: seg.FileRangeOf(segMR).Start,
+		})
+		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, cerr
+	}
+	return ts, buserr
+}
+
+// InvalidateUnsavable implements memmap.Mappable.InvalidateUnsavable.
+func (f *fileInodeOperations) InvalidateUnsavable(ctx context.Context) error {
+	return nil
+}
diff --git a/pkg/sentry/fs/tmpfs/tmpfs.go b/pkg/sentry/fs/tmpfs/tmpfs.go
new file mode 100644
index 000000000..1cc7ae491
--- /dev/null
+++ b/pkg/sentry/fs/tmpfs/tmpfs.go
@@ -0,0 +1,204 @@
+// Copyright 2018 Google Inc.
+//
+// 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/ramfs"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/kernel/pipe"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/platform"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/usage"
+	"gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
+	"gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix"
+)
+
+var fsInfo = fs.Info{
+	Type: linux.TMPFS_MAGIC,
+
+	// TODO: 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) error {
+	op, ok := oldParent.InodeOperations.(*Dir)
+	if !ok {
+		return ramfs.ErrCrossDevice
+	}
+	np, ok := newParent.InodeOperations.(*Dir)
+	if !ok {
+		return ramfs.ErrCrossDevice
+	}
+	return ramfs.Rename(ctx, &op.Dir, oldName, &np.Dir, newName)
+}
+
+// Dir is a directory.
+type Dir struct {
+	ramfs.Dir
+
+	// platform is used to allocate storage for tmpfs Files.
+	platform platform.Platform
+}
+
+// NewDir returns a new directory.
+func NewDir(ctx context.Context, contents map[string]*fs.Inode, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource, platform platform.Platform) *fs.Inode {
+	d := &Dir{platform: platform}
+	d.InitDir(ctx, contents, owner, perms)
+
+	// Manually set the CreateOps.
+	d.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.Dir.CreateOps = d.newCreateOps()
+}
+
+// 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, d.platform), 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, d.platform)
+			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 unix.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, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error {
+	return rename(ctx, oldParent, oldName, newParent, newName)
+}
+
+// StatFS implments fs.InodeOperations.StatFS.
+func (*Dir) StatFS(context.Context) (fs.Info, error) {
+	return fsInfo, nil
+}
+
+// Symlink is a symlink.
+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{}
+	s.InitSymlink(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, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error {
+	return rename(ctx, oldParent, oldName, newParent, newName)
+}
+
+// StatFS returns the tmpfs info.
+func (s *Symlink) StatFS(context.Context) (fs.Info, error) {
+	return fsInfo, nil
+}
+
+// Socket is a socket.
+type Socket struct {
+	ramfs.Socket
+}
+
+// NewSocket returns a new socket with the provided permissions.
+func NewSocket(ctx context.Context, socket unix.BoundEndpoint, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode {
+	s := &Socket{}
+	s.InitSocket(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, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error {
+	return rename(ctx, oldParent, oldName, newParent, newName)
+}
+
+// StatFS returns the tmpfs info.
+func (s *Socket) StatFS(context.Context) (fs.Info, error) {
+	return fsInfo, nil
+}
+
+// Fifo is a tmpfs named pipe.
+type Fifo struct {
+	ramfs.Entry
+}
+
+// NewFifo creates a new named pipe.
+func NewFifo(ctx context.Context, owner fs.FileOwner, perms fs.FilePermissions, msrc *fs.MountSource) *fs.Inode {
+	f := &Fifo{}
+	f.InitEntry(ctx, owner, perms)
+	iops := pipe.NewInodeOperations(f, pipe.NewPipe(ctx, true /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize))
+	return fs.NewInode(iops, 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, oldParent *fs.Inode, oldName string, newParent *fs.Inode, newName string) error {
+	return rename(ctx, oldParent, oldName, newParent, newName)
+}
+
+// StatFS returns the tmpfs info.
+func (*Fifo) StatFS(context.Context) (fs.Info, error) {
+	return fsInfo, nil
+}
-- 
cgit v1.2.3