summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/fs
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/fs')
-rw-r--r--pkg/sentry/fs/BUILD4
-rw-r--r--pkg/sentry/fs/ashmem/area.go3
-rw-r--r--pkg/sentry/fs/binder/binder.go3
-rw-r--r--pkg/sentry/fs/dev/full.go3
-rw-r--r--pkg/sentry/fs/dev/null.go4
-rw-r--r--pkg/sentry/fs/dev/random.go1
-rw-r--r--pkg/sentry/fs/fdpipe/pipe.go1
-rw-r--r--pkg/sentry/fs/file.go139
-rw-r--r--pkg/sentry/fs/file_operations.go47
-rw-r--r--pkg/sentry/fs/file_overlay.go79
-rw-r--r--pkg/sentry/fs/file_test.go24
-rw-r--r--pkg/sentry/fs/filetest/filetest.go1
-rw-r--r--pkg/sentry/fs/fsutil/file.go16
-rw-r--r--pkg/sentry/fs/fsutil/inode.go3
-rw-r--r--pkg/sentry/fs/gofer/file.go5
-rw-r--r--pkg/sentry/fs/host/file.go1
-rw-r--r--pkg/sentry/fs/inotify.go10
-rw-r--r--pkg/sentry/fs/proc/exec_args.go3
-rw-r--r--pkg/sentry/fs/proc/rpcinet_proc.go3
-rw-r--r--pkg/sentry/fs/proc/seqfile/seqfile.go1
-rw-r--r--pkg/sentry/fs/proc/sys.go3
-rw-r--r--pkg/sentry/fs/proc/sys_net.go6
-rw-r--r--pkg/sentry/fs/proc/task.go10
-rw-r--r--pkg/sentry/fs/proc/uid_gid_map.go1
-rw-r--r--pkg/sentry/fs/proc/uptime.go5
-rw-r--r--pkg/sentry/fs/ramfs/socket.go7
-rw-r--r--pkg/sentry/fs/ramfs/symlink.go7
-rw-r--r--pkg/sentry/fs/splice.go187
-rw-r--r--pkg/sentry/fs/timerfd/timerfd.go5
-rw-r--r--pkg/sentry/fs/tmpfs/file_regular.go3
-rw-r--r--pkg/sentry/fs/tty/dir.go3
-rw-r--r--pkg/sentry/fs/tty/master.go3
-rw-r--r--pkg/sentry/fs/tty/slave.go3
33 files changed, 461 insertions, 133 deletions
diff --git a/pkg/sentry/fs/BUILD b/pkg/sentry/fs/BUILD
index 1fd9e30f6..142a00840 100644
--- a/pkg/sentry/fs/BUILD
+++ b/pkg/sentry/fs/BUILD
@@ -40,6 +40,7 @@ go_library(
"restore.go",
"save.go",
"seek.go",
+ "splice.go",
"sync.go",
],
importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/fs",
@@ -51,6 +52,7 @@ go_library(
"//pkg/metric",
"//pkg/p9",
"//pkg/refs",
+ "//pkg/secio",
"//pkg/sentry/arch",
"//pkg/sentry/context",
"//pkg/sentry/device",
@@ -66,7 +68,6 @@ go_library(
"//pkg/sentry/usermem",
"//pkg/state",
"//pkg/syserror",
- "//pkg/tcpip",
"//pkg/waiter",
],
)
@@ -122,7 +123,6 @@ go_test(
srcs = [
"dirent_cache_test.go",
"dirent_refs_test.go",
- "file_test.go",
"mount_test.go",
"path_test.go",
],
diff --git a/pkg/sentry/fs/ashmem/area.go b/pkg/sentry/fs/ashmem/area.go
index b53746519..b4b0cc08b 100644
--- a/pkg/sentry/fs/ashmem/area.go
+++ b/pkg/sentry/fs/ashmem/area.go
@@ -42,11 +42,12 @@ const (
//
// +stateify savable
type Area struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileNoFsync `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
ad *Device
diff --git a/pkg/sentry/fs/binder/binder.go b/pkg/sentry/fs/binder/binder.go
index a992253e6..c78f1fc40 100644
--- a/pkg/sentry/fs/binder/binder.go
+++ b/pkg/sentry/fs/binder/binder.go
@@ -86,10 +86,11 @@ func (bd *Device) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags)
//
// +stateify savable
type Proc struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileNoFsync `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
bd *Device
task *kernel.Task
diff --git a/pkg/sentry/fs/dev/full.go b/pkg/sentry/fs/dev/full.go
index 17d68b5c4..8f6c6da2d 100644
--- a/pkg/sentry/fs/dev/full.go
+++ b/pkg/sentry/fs/dev/full.go
@@ -60,6 +60,7 @@ func (f *fullDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.Fi
// +stateify savable
type fullFileOperations struct {
+ waiter.AlwaysReady `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
@@ -68,8 +69,8 @@ type fullFileOperations struct {
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
readZeros `state:"nosave"`
- waiter.AlwaysReady `state:"nosave"`
}
var _ fs.FileOperations = (*fullFileOperations)(nil)
diff --git a/pkg/sentry/fs/dev/null.go b/pkg/sentry/fs/dev/null.go
index ee13183c8..3f1accef8 100644
--- a/pkg/sentry/fs/dev/null.go
+++ b/pkg/sentry/fs/dev/null.go
@@ -64,6 +64,7 @@ type nullFileOperations struct {
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRead `state:"nosave"`
@@ -104,14 +105,15 @@ func (zd *zeroDevice) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.F
type zeroFileOperations struct {
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNoopWrite `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
- readZeros `state:"nosave"`
waiter.AlwaysReady `state:"nosave"`
+ readZeros `state:"nosave"`
}
var _ fs.FileOperations = (*zeroFileOperations)(nil)
diff --git a/pkg/sentry/fs/dev/random.go b/pkg/sentry/fs/dev/random.go
index b0a412382..e5a01a906 100644
--- a/pkg/sentry/fs/dev/random.go
+++ b/pkg/sentry/fs/dev/random.go
@@ -61,6 +61,7 @@ type randomFileOperations struct {
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go
index 95e66ea8d..4ef7ea08a 100644
--- a/pkg/sentry/fs/fdpipe/pipe.go
+++ b/pkg/sentry/fs/fdpipe/pipe.go
@@ -43,6 +43,7 @@ type pipeOperations struct {
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
waiter.Queue `state:"nosave"`
diff --git a/pkg/sentry/fs/file.go b/pkg/sentry/fs/file.go
index 62b35dabc..8f1baca23 100644
--- a/pkg/sentry/fs/file.go
+++ b/pkg/sentry/fs/file.go
@@ -21,7 +21,6 @@ import (
"time"
"gvisor.googlesource.com/gvisor/pkg/amutex"
- "gvisor.googlesource.com/gvisor/pkg/log"
"gvisor.googlesource.com/gvisor/pkg/metric"
"gvisor.googlesource.com/gvisor/pkg/refs"
"gvisor.googlesource.com/gvisor/pkg/sentry/context"
@@ -35,8 +34,13 @@ import (
)
var (
- // RecordWaitTime controls writing metrics for filesystem reads. Enabling this comes at a small
- // CPU cost due to performing two monotonic clock reads per read call.
+ // RecordWaitTime controls writing metrics for filesystem reads.
+ // Enabling this comes at a small CPU cost due to performing two
+ // monotonic clock reads per read call.
+ //
+ // Note that this is only performed in the direct read path, and may
+ // not be consistently applied for other forms of reads, such as
+ // splice.
RecordWaitTime = false
reads = metric.MustCreateNewUint64Metric("/fs/reads", false /* sync */, "Number of file reads.")
@@ -306,14 +310,28 @@ func (f *File) Writev(ctx context.Context, src usermem.IOSequence) (int64, error
return 0, syserror.ErrInterrupted
}
- offset, err := f.checkWriteLocked(ctx, &src, f.offset)
- if err != nil {
+ // Handle append mode.
+ if f.Flags().Append {
+ if err := f.offsetForAppend(ctx, &f.offset); err != nil {
+ f.mu.Unlock()
+ return 0, err
+ }
+ }
+
+ // Enforce file limits.
+ limit, ok := f.checkLimit(ctx, f.offset)
+ switch {
+ case ok && limit == 0:
f.mu.Unlock()
- return 0, err
+ return 0, syserror.ErrExceedsFileSizeLimit
+ case ok:
+ src = src.TakeFirst64(limit)
}
- n, err := f.FileOperations.Write(ctx, f, src, offset)
+
+ // We must hold the lock during the write.
+ n, err := f.FileOperations.Write(ctx, f, src, f.offset)
if n >= 0 {
- atomic.StoreInt64(&f.offset, offset+n)
+ atomic.StoreInt64(&f.offset, f.offset+n)
}
f.mu.Unlock()
return n, err
@@ -325,51 +343,67 @@ func (f *File) Writev(ctx context.Context, src usermem.IOSequence) (int64, error
//
// Otherwise same as Writev.
func (f *File) Pwritev(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) {
- if !f.mu.Lock(ctx) {
- return 0, syserror.ErrInterrupted
+ // "POSIX requires that opening a file with the O_APPEND flag should
+ // have no effect on the location at which pwrite() writes data.
+ // However, on Linux, if a file is opened with O_APPEND, pwrite()
+ // appends data to the end of the file, regardless of the value of
+ // offset."
+ if f.Flags().Append {
+ if !f.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer f.mu.Unlock()
+ if err := f.offsetForAppend(ctx, &offset); err != nil {
+ f.mu.Unlock()
+ return 0, err
+ }
}
- offset, err := f.checkWriteLocked(ctx, &src, offset)
- if err != nil {
- f.mu.Unlock()
- return 0, err
+ // Enforce file limits.
+ limit, ok := f.checkLimit(ctx, offset)
+ switch {
+ case ok && limit == 0:
+ return 0, syserror.ErrExceedsFileSizeLimit
+ case ok:
+ src = src.TakeFirst64(limit)
}
- n, err := f.FileOperations.Write(ctx, f, src, offset)
- f.mu.Unlock()
- return n, err
+
+ return f.FileOperations.Write(ctx, f, src, offset)
}
-// checkWriteLocked returns the offset to write at or an error if the write
-// would not succeed. May update src to fit a write operation into a file
-// size limit.
-func (f *File) checkWriteLocked(ctx context.Context, src *usermem.IOSequence, offset int64) (int64, error) {
- // Handle append only files. Note that this is still racy for network
- // filesystems.
- if f.Flags().Append {
- uattr, err := f.Dirent.Inode.UnstableAttr(ctx)
- if err != nil {
- // This is an odd error, most likely it is evidence
- // that something is terribly wrong with the filesystem.
- // Return a generic EIO error.
- log.Warningf("Failed to check write of inode %#v: %v", f.Dirent.Inode.StableAttr, err)
- return offset, syserror.EIO
- }
- offset = uattr.Size
+// offsetForAppend sets the given offset to the end of the file.
+//
+// Precondition: the underlying file mutex should be held.
+func (f *File) offsetForAppend(ctx context.Context, offset *int64) error {
+ uattr, err := f.Dirent.Inode.UnstableAttr(ctx)
+ if err != nil {
+ // This is an odd error, we treat it as evidence that
+ // something is terribly wrong with the filesystem.
+ return syserror.EIO
}
- // Is this a regular file?
+ // Update the offset.
+ *offset = uattr.Size
+
+ return nil
+}
+
+// checkLimit checks the offset that the write will be performed at. The
+// returned boolean indicates that the write must be limited. The returned
+// integer indicates the new maximum write length.
+func (f *File) checkLimit(ctx context.Context, offset int64) (int64, bool) {
if IsRegular(f.Dirent.Inode.StableAttr) {
// Enforce size limits.
fileSizeLimit := limits.FromContext(ctx).Get(limits.FileSize).Cur
if fileSizeLimit <= math.MaxInt64 {
if offset >= int64(fileSizeLimit) {
- return offset, syserror.ErrExceedsFileSizeLimit
+ return 0, true
}
- *src = src.TakeFirst64(int64(fileSizeLimit) - offset)
+ return int64(fileSizeLimit) - offset, true
}
}
- return offset, nil
+ return 0, false
}
// Fsync calls f.FileOperations.Fsync with f as the File.
@@ -466,8 +500,13 @@ func (f *File) Async(newAsync func() FileAsync) FileAsync {
return f.async
}
-// FileReader implements io.Reader and io.ReaderAt.
-type FileReader struct {
+// lockedReader implements io.Reader and io.ReaderAt.
+//
+// Note this reads the underlying file using the file operations directly. It
+// is the responsibility of the caller to ensure that locks are appropriately
+// held and offsets updated if required. This should be used only by internal
+// functions that perform these operations and checks at other times.
+type lockedReader struct {
// Ctx is the context for the file reader.
Ctx context.Context
@@ -476,19 +515,21 @@ type FileReader struct {
}
// Read implements io.Reader.Read.
-func (r *FileReader) Read(buf []byte) (int, error) {
- n, err := r.File.Readv(r.Ctx, usermem.BytesIOSequence(buf))
+func (r *lockedReader) Read(buf []byte) (int, error) {
+ n, err := r.File.FileOperations.Read(r.Ctx, r.File, usermem.BytesIOSequence(buf), r.File.offset)
return int(n), err
}
// ReadAt implements io.Reader.ReadAt.
-func (r *FileReader) ReadAt(buf []byte, offset int64) (int, error) {
- n, err := r.File.Preadv(r.Ctx, usermem.BytesIOSequence(buf), offset)
+func (r *lockedReader) ReadAt(buf []byte, offset int64) (int, error) {
+ n, err := r.File.FileOperations.Read(r.Ctx, r.File, usermem.BytesIOSequence(buf), offset)
return int(n), err
}
-// FileWriter implements io.Writer and io.WriterAt.
-type FileWriter struct {
+// lockedWriter implements io.Writer and io.WriterAt.
+//
+// The same constraints as lockedReader apply; see above.
+type lockedWriter struct {
// Ctx is the context for the file writer.
Ctx context.Context
@@ -497,13 +538,13 @@ type FileWriter struct {
}
// Write implements io.Writer.Write.
-func (w *FileWriter) Write(buf []byte) (int, error) {
- n, err := w.File.Writev(w.Ctx, usermem.BytesIOSequence(buf))
+func (w *lockedWriter) Write(buf []byte) (int, error) {
+ n, err := w.File.FileOperations.Write(w.Ctx, w.File, usermem.BytesIOSequence(buf), w.File.offset)
return int(n), err
}
// WriteAt implements io.Writer.WriteAt.
-func (w *FileWriter) WriteAt(buf []byte, offset int64) (int, error) {
- n, err := w.File.Pwritev(w.Ctx, usermem.BytesIOSequence(buf), offset)
+func (w *lockedWriter) WriteAt(buf []byte, offset int64) (int, error) {
+ n, err := w.File.FileOperations.Write(w.Ctx, w.File, usermem.BytesIOSequence(buf), offset)
return int(n), err
}
diff --git a/pkg/sentry/fs/file_operations.go b/pkg/sentry/fs/file_operations.go
index ab0acb6eb..0f2dfa273 100644
--- a/pkg/sentry/fs/file_operations.go
+++ b/pkg/sentry/fs/file_operations.go
@@ -22,6 +22,38 @@ import (
"gvisor.googlesource.com/gvisor/pkg/waiter"
)
+// SpliceOpts define how a splice works.
+type SpliceOpts struct {
+ // Length is the length of the splice operation.
+ Length int64
+
+ // SrcOffset indicates whether the existing source file offset should
+ // be used. If this is true, then the Start value below is used.
+ //
+ // When passed to FileOperations object, this should always be true as
+ // the offset will be provided by a layer above, unless the object in
+ // question is a pipe or socket. This value can be relied upon for such
+ // an indicator.
+ SrcOffset bool
+
+ // SrcStart is the start of the source file. This is used only if
+ // SrcOffset is false.
+ SrcStart int64
+
+ // Dup indicates that the contents should not be consumed from the
+ // source (e.g. in the case of a socket or a pipe), but duplicated.
+ Dup bool
+
+ // DstOffset indicates that the destination file offset should be used.
+ //
+ // See SrcOffset for additional information.
+ DstOffset bool
+
+ // DstStart is the start of the destination file. This is used only if
+ // DstOffset is false.
+ DstStart int64
+}
+
// FileOperations are operations on a File that diverge per file system.
//
// Operations that take a *File may use only the following interfaces:
@@ -67,6 +99,15 @@ type FileOperations interface {
// Read must not be called if !FileFlags.Read.
Read(ctx context.Context, file *File, dst usermem.IOSequence, offset int64) (int64, error)
+ // WriteTo is a variant of read that takes another file as a
+ // destination. For a splice (copy or move from one file to another),
+ // first a WriteTo on the source is attempted, followed by a ReadFrom
+ // on the destination, following by a buffered copy with standard Read
+ // and Write operations.
+ //
+ // The same preconditions as Read apply.
+ WriteTo(ctx context.Context, file *File, dst *File, opts SpliceOpts) (int64, error)
+
// Write writes src to file at offset and returns the number of bytes
// written which must be greater than or equal to 0. Like Read, file
// systems that do not support writing at an offset (i.e. pipefs, sockfs)
@@ -81,6 +122,12 @@ type FileOperations interface {
// Write must not be called if !FileFlags.Write.
Write(ctx context.Context, file *File, src usermem.IOSequence, offset int64) (int64, error)
+ // ReadFrom is a variant of write that takes a another file as a
+ // source. See WriteTo for details regarding how this is called.
+ //
+ // The same preconditions as Write apply; FileFlags.Write must be set.
+ ReadFrom(ctx context.Context, file *File, src *File, opts SpliceOpts) (int64, error)
+
// Fsync writes buffered modifications of file and/or flushes in-flight
// operations to backing storage based on syncType. The range to sync is
// [start, end]. The end is inclusive so that the last byte of a maximally
diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go
index 948ce9c6f..273de1e14 100644
--- a/pkg/sentry/fs/file_overlay.go
+++ b/pkg/sentry/fs/file_overlay.go
@@ -17,7 +17,6 @@ package fs
import (
"sync"
- "gvisor.googlesource.com/gvisor/pkg/log"
"gvisor.googlesource.com/gvisor/pkg/refs"
"gvisor.googlesource.com/gvisor/pkg/sentry/arch"
"gvisor.googlesource.com/gvisor/pkg/sentry/context"
@@ -222,31 +221,50 @@ func (f *overlayFileOperations) IterateDir(ctx context.Context, dirCtx *DirCtx,
return offset + n, err
}
-// Read implements FileOperations.Read.
-func (f *overlayFileOperations) Read(ctx context.Context, file *File, dst usermem.IOSequence, offset int64) (int64, error) {
- o := file.Dirent.Inode.overlay
+// onTop performs the given operation on the top-most available layer.
+func (f *overlayFileOperations) onTop(ctx context.Context, file *File, fn func(*File, FileOperations) error) error {
+ file.Dirent.Inode.overlay.copyMu.RLock()
+ defer file.Dirent.Inode.overlay.copyMu.RUnlock()
- o.copyMu.RLock()
- defer o.copyMu.RUnlock()
+ // Only lower layer is available.
+ if file.Dirent.Inode.overlay.upper == nil {
+ return fn(f.lower, f.lower.FileOperations)
+ }
- if o.upper != nil {
- // We may need to acquire an open file handle to read from if
- // copy up has occurred. Otherwise we risk reading from the
- // wrong source.
- f.upperMu.Lock()
- if f.upper == nil {
- var err error
- f.upper, err = overlayFile(ctx, o.upper, file.Flags())
- if err != nil {
- f.upperMu.Unlock()
- log.Warningf("failed to acquire handle with flags %v: %v", file.Flags(), err)
- return 0, syserror.EIO
- }
+ f.upperMu.Lock()
+ if f.upper == nil {
+ upper, err := overlayFile(ctx, file.Dirent.Inode.overlay.upper, file.Flags())
+ if err != nil {
+ // Something very wrong; return a generic filesystem
+ // error to avoid propagating internals.
+ f.upperMu.Unlock()
+ return syserror.EIO
}
- f.upperMu.Unlock()
- return f.upper.FileOperations.Read(ctx, f.upper, dst, offset)
+
+ // Save upper file.
+ f.upper = upper
}
- return f.lower.FileOperations.Read(ctx, f.lower, dst, offset)
+ f.upperMu.Unlock()
+
+ return fn(f.upper, f.upper.FileOperations)
+}
+
+// Read implements FileOperations.Read.
+func (f *overlayFileOperations) Read(ctx context.Context, file *File, dst usermem.IOSequence, offset int64) (n int64, err error) {
+ err = f.onTop(ctx, file, func(file *File, ops FileOperations) error {
+ n, err = ops.Read(ctx, file, dst, offset)
+ return err // Will overwrite itself.
+ })
+ return
+}
+
+// WriteTo implements FileOperations.WriteTo.
+func (f *overlayFileOperations) WriteTo(ctx context.Context, file *File, dst *File, opts SpliceOpts) (n int64, err error) {
+ err = f.onTop(ctx, file, func(file *File, ops FileOperations) error {
+ n, err = ops.WriteTo(ctx, file, dst, opts)
+ return err // Will overwrite itself.
+ })
+ return
}
// Write implements FileOperations.Write.
@@ -257,15 +275,20 @@ func (f *overlayFileOperations) Write(ctx context.Context, file *File, src userm
return f.upper.FileOperations.Write(ctx, f.upper, src, offset)
}
+// ReadFrom implements FileOperations.ReadFrom.
+func (f *overlayFileOperations) ReadFrom(ctx context.Context, file *File, src *File, opts SpliceOpts) (n int64, err error) {
+ // See above; f.upper must be non-nil.
+ return f.upper.FileOperations.ReadFrom(ctx, f.upper, src, opts)
+}
+
// Fsync implements FileOperations.Fsync.
-func (f *overlayFileOperations) Fsync(ctx context.Context, file *File, start, end int64, syncType SyncType) error {
- var err error
+func (f *overlayFileOperations) Fsync(ctx context.Context, file *File, start, end int64, syncType SyncType) (err error) {
f.upperMu.Lock()
if f.upper != nil {
err = f.upper.FileOperations.Fsync(ctx, f.upper, start, end, syncType)
}
f.upperMu.Unlock()
- if f.lower != nil {
+ if err == nil && f.lower != nil {
// N.B. Fsync on the lower filesystem can cause writes of file
// attributes (i.e. access time) despite the fact that we must
// treat the lower filesystem as read-only.
@@ -277,15 +300,14 @@ func (f *overlayFileOperations) Fsync(ctx context.Context, file *File, start, en
}
// Flush implements FileOperations.Flush.
-func (f *overlayFileOperations) Flush(ctx context.Context, file *File) error {
+func (f *overlayFileOperations) Flush(ctx context.Context, file *File) (err error) {
// Flush whatever handles we have.
- var err error
f.upperMu.Lock()
if f.upper != nil {
err = f.upper.FileOperations.Flush(ctx, f.upper)
}
f.upperMu.Unlock()
- if f.lower != nil {
+ if err == nil && f.lower != nil {
err = f.lower.FileOperations.Flush(ctx, f.lower)
}
return err
@@ -329,6 +351,7 @@ func (*overlayFileOperations) ConfigureMMap(ctx context.Context, file *File, opt
if !o.isMappableLocked() {
return syserror.ENODEV
}
+
// FIXME(jamieliu): This is a copy/paste of fsutil.GenericConfigureMMap,
// which we can't use because the overlay implementation is in package fs,
// so depending on fs/fsutil would create a circular dependency. Move
diff --git a/pkg/sentry/fs/file_test.go b/pkg/sentry/fs/file_test.go
deleted file mode 100644
index d867a0257..000000000
--- a/pkg/sentry/fs/file_test.go
+++ /dev/null
@@ -1,24 +0,0 @@
-// 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 fs
-
-import "io"
-
-var (
- _ = io.Reader(&FileReader{})
- _ = io.ReaderAt(&FileReader{})
- _ = io.Writer(&FileWriter{})
- _ = io.WriterAt(&FileWriter{})
-)
diff --git a/pkg/sentry/fs/filetest/filetest.go b/pkg/sentry/fs/filetest/filetest.go
index f6b827800..c0b1b088d 100644
--- a/pkg/sentry/fs/filetest/filetest.go
+++ b/pkg/sentry/fs/filetest/filetest.go
@@ -38,6 +38,7 @@ type TestFileOperations struct {
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
waiter.AlwaysReady `state:"nosave"`
}
diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go
index e355d8594..9381963d0 100644
--- a/pkg/sentry/fs/fsutil/file.go
+++ b/pkg/sentry/fs/fsutil/file.go
@@ -223,6 +223,20 @@ func (FileNoIoctl) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallAr
return 0, syserror.ENOTTY
}
+// FileNoSplice implements fs.FileOperations.ReadFrom and
+// fs.FileOperations.WriteTo for files that don't support splice.
+type FileNoSplice struct{}
+
+// WriteTo implements fs.FileOperations.WriteTo.
+func (FileNoSplice) WriteTo(context.Context, *fs.File, *fs.File, fs.SpliceOpts) (int64, error) {
+ return 0, syserror.ENOSYS
+}
+
+// ReadFrom implements fs.FileOperations.ReadFrom.
+func (FileNoSplice) ReadFrom(context.Context, *fs.File, *fs.File, fs.SpliceOpts) (int64, error) {
+ return 0, syserror.ENOSYS
+}
+
// DirFileOperations implements most of fs.FileOperations for directories,
// except for Readdir and UnstableAttr which the embedding type must implement.
type DirFileOperations struct {
@@ -233,6 +247,7 @@ type DirFileOperations struct {
FileNoopFlush
FileNoopFsync
FileNoopRelease
+ FileNoSplice
}
// Read implements fs.FileOperations.Read
@@ -303,6 +318,7 @@ type NoReadWriteFile struct {
FileNoWrite `state:"nosave"`
FileNotDirReaddir `state:"nosave"`
FileUseInodeUnstableAttr `state:"nosave"`
+ FileNoSplice `state:"nosave"`
}
var _ fs.FileOperations = (*NoReadWriteFile)(nil)
diff --git a/pkg/sentry/fs/fsutil/inode.go b/pkg/sentry/fs/fsutil/inode.go
index a22b6ce9c..925887335 100644
--- a/pkg/sentry/fs/fsutil/inode.go
+++ b/pkg/sentry/fs/fsutil/inode.go
@@ -250,16 +250,17 @@ func (i *InodeSimpleExtendedAttributes) Listxattr(_ *fs.Inode) (map[string]struc
//
// +stateify savable
type staticFile struct {
- waiter.AlwaysReady `state:"nosave"`
FileGenericSeek `state:"nosave"`
FileNoIoctl `state:"nosave"`
FileNoMMap `state:"nosave"`
+ FileNoSplice `state:"nosave"`
FileNoopFsync `state:"nosave"`
FileNoopFlush `state:"nosave"`
FileNoopRelease `state:"nosave"`
FileNoopWrite `state:"nosave"`
FileNotDirReaddir `state:"nosave"`
FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
FileStaticContentReader
}
diff --git a/pkg/sentry/fs/gofer/file.go b/pkg/sentry/fs/gofer/file.go
index bc2be546e..fb4f50113 100644
--- a/pkg/sentry/fs/gofer/file.go
+++ b/pkg/sentry/fs/gofer/file.go
@@ -46,8 +46,9 @@ var (
//
// +stateify savable
type fileOperations struct {
- fsutil.FileNoIoctl `state:"nosave"`
- waiter.AlwaysReady `state:"nosave"`
+ fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosplice"`
+ waiter.AlwaysReady `state:"nosave"`
// inodeOperations is the inodeOperations backing the file. It is protected
// by a reference held by File.Dirent.Inode which is stable until
diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go
index 82e2ae3b9..ad0a3ec85 100644
--- a/pkg/sentry/fs/host/file.go
+++ b/pkg/sentry/fs/host/file.go
@@ -37,6 +37,7 @@ import (
// +stateify savable
type fileOperations struct {
fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosplice"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
diff --git a/pkg/sentry/fs/inotify.go b/pkg/sentry/fs/inotify.go
index 2652582c3..7dfd31020 100644
--- a/pkg/sentry/fs/inotify.go
+++ b/pkg/sentry/fs/inotify.go
@@ -171,11 +171,21 @@ func (i *Inotify) Read(ctx context.Context, _ *File, dst usermem.IOSequence, _ i
return writeLen, nil
}
+// WriteTo implements FileOperations.WriteTo.
+func (*Inotify) WriteTo(context.Context, *File, *File, SpliceOpts) (int64, error) {
+ return 0, syserror.ENOSYS
+}
+
// Fsync implements FileOperations.Fsync.
func (*Inotify) Fsync(context.Context, *File, int64, int64, SyncType) error {
return syserror.EINVAL
}
+// ReadFrom implements FileOperations.ReadFrom.
+func (*Inotify) ReadFrom(context.Context, *File, *File, SpliceOpts) (int64, error) {
+ return 0, syserror.ENOSYS
+}
+
// Flush implements FileOperations.Flush.
func (*Inotify) Flush(context.Context, *File) error {
return nil
diff --git a/pkg/sentry/fs/proc/exec_args.go b/pkg/sentry/fs/proc/exec_args.go
index d49dad685..cb28f6bc3 100644
--- a/pkg/sentry/fs/proc/exec_args.go
+++ b/pkg/sentry/fs/proc/exec_args.go
@@ -77,16 +77,17 @@ func (i *execArgInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.
// +stateify savable
type execArgFile struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopWrite `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
// arg is the type of exec argument this file contains.
arg execArgType
diff --git a/pkg/sentry/fs/proc/rpcinet_proc.go b/pkg/sentry/fs/proc/rpcinet_proc.go
index db53686f6..e36c0bfa6 100644
--- a/pkg/sentry/fs/proc/rpcinet_proc.go
+++ b/pkg/sentry/fs/proc/rpcinet_proc.go
@@ -60,15 +60,16 @@ func (i *rpcInetInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.
// rpcInetFile implements fs.FileOperations as RPCs.
type rpcInetFile struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
inode *rpcInetInode
}
diff --git a/pkg/sentry/fs/proc/seqfile/seqfile.go b/pkg/sentry/fs/proc/seqfile/seqfile.go
index 6b0ae9e60..8364d86ed 100644
--- a/pkg/sentry/fs/proc/seqfile/seqfile.go
+++ b/pkg/sentry/fs/proc/seqfile/seqfile.go
@@ -187,6 +187,7 @@ type seqFileOperations struct {
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go
index b889ed625..59846af4f 100644
--- a/pkg/sentry/fs/proc/sys.go
+++ b/pkg/sentry/fs/proc/sys.go
@@ -134,7 +134,6 @@ var _ fs.InodeOperations = (*hostname)(nil)
// +stateify savable
type hostnameFile struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
fsutil.FileNoSeek `state:"nosave"`
@@ -143,7 +142,9 @@ type hostnameFile struct {
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileNoWrite `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
}
// Read implements fs.FileOperations.Read.
diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go
index e49794a48..dbf1a987c 100644
--- a/pkg/sentry/fs/proc/sys_net.go
+++ b/pkg/sentry/fs/proc/sys_net.go
@@ -85,15 +85,16 @@ func (m *tcpMemInode) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.F
// +stateify savable
type tcpMemFile struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
tcpMemInode *tcpMemInode
}
@@ -198,15 +199,16 @@ func (s *tcpSack) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileF
// +stateify savable
type tcpSackFile struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
tcpSack *tcpSack
diff --git a/pkg/sentry/fs/proc/task.go b/pkg/sentry/fs/proc/task.go
index 66d76d194..494b195cd 100644
--- a/pkg/sentry/fs/proc/task.go
+++ b/pkg/sentry/fs/proc/task.go
@@ -672,16 +672,17 @@ func (c *comm) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlag
// +stateify savable
type commFile struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoWrite `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
- fsutil.FileNoWrite `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
t *kernel.Task
}
@@ -728,16 +729,17 @@ func (a *auxvec) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl
// +stateify savable
type auxvecFile struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoWrite `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
- fsutil.FileNoWrite `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
t *kernel.Task
}
diff --git a/pkg/sentry/fs/proc/uid_gid_map.go b/pkg/sentry/fs/proc/uid_gid_map.go
index 5df3cee13..a14b1b45f 100644
--- a/pkg/sentry/fs/proc/uid_gid_map.go
+++ b/pkg/sentry/fs/proc/uid_gid_map.go
@@ -85,6 +85,7 @@ type idMapFileOperations struct {
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
diff --git a/pkg/sentry/fs/proc/uptime.go b/pkg/sentry/fs/proc/uptime.go
index 1ddf9fafa..35c3851e1 100644
--- a/pkg/sentry/fs/proc/uptime.go
+++ b/pkg/sentry/fs/proc/uptime.go
@@ -54,16 +54,17 @@ func (u *uptime) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl
// +stateify savable
type uptimeFile struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoWrite `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
- fsutil.FileNoWrite `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
startTime ktime.Time
}
diff --git a/pkg/sentry/fs/ramfs/socket.go b/pkg/sentry/fs/ramfs/socket.go
index a7cb1bb86..7d8bca70e 100644
--- a/pkg/sentry/fs/ramfs/socket.go
+++ b/pkg/sentry/fs/ramfs/socket.go
@@ -70,13 +70,14 @@ func (s *Socket) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFl
type socketFileOperations struct {
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoRead `state:"nosave"`
+ fsutil.FileNoSeek `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoWrite `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
- fsutil.FileNoRead `state:"nosave"`
- fsutil.FileNoSeek `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
- fsutil.FileNoWrite `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
waiter.AlwaysReady `state:"nosave"`
}
diff --git a/pkg/sentry/fs/ramfs/symlink.go b/pkg/sentry/fs/ramfs/symlink.go
index dd2585b02..21c246169 100644
--- a/pkg/sentry/fs/ramfs/symlink.go
+++ b/pkg/sentry/fs/ramfs/symlink.go
@@ -91,13 +91,14 @@ func (s *Symlink) GetFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileF
type symlinkFileOperations struct {
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoRead `state:"nosave"`
+ fsutil.FileNoSeek `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoWrite `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoopFsync `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
- fsutil.FileNoRead `state:"nosave"`
- fsutil.FileNoSeek `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
- fsutil.FileNoWrite `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
waiter.AlwaysReady `state:"nosave"`
}
diff --git a/pkg/sentry/fs/splice.go b/pkg/sentry/fs/splice.go
new file mode 100644
index 000000000..65937f44d
--- /dev/null
+++ b/pkg/sentry/fs/splice.go
@@ -0,0 +1,187 @@
+// 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 fs
+
+import (
+ "io"
+ "sync/atomic"
+
+ "gvisor.googlesource.com/gvisor/pkg/secio"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/syserror"
+)
+
+// Splice moves data to this file, directly from another.
+//
+// Offsets are updated only if DstOffset and SrcOffset are set.
+func Splice(ctx context.Context, dst *File, src *File, opts SpliceOpts) (int64, error) {
+ // Verify basic file flag permissions.
+ if !dst.Flags().Write || !src.Flags().Read {
+ return 0, syserror.EBADF
+ }
+
+ // Check whether or not the objects being sliced are stream-oriented
+ // (i.e. pipes or sockets). If yes, we elide checks and offset locks.
+ srcPipe := IsPipe(src.Dirent.Inode.StableAttr) || IsSocket(src.Dirent.Inode.StableAttr)
+ dstPipe := IsPipe(dst.Dirent.Inode.StableAttr) || IsSocket(dst.Dirent.Inode.StableAttr)
+
+ if !dstPipe && !opts.DstOffset && !srcPipe && !opts.SrcOffset {
+ switch {
+ case dst.UniqueID < src.UniqueID:
+ // Acquire dst first.
+ if !dst.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer dst.mu.Unlock()
+ if !src.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer src.mu.Unlock()
+ case dst.UniqueID > src.UniqueID:
+ // Acquire src first.
+ if !src.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer src.mu.Unlock()
+ if !dst.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer dst.mu.Unlock()
+ case dst.UniqueID == src.UniqueID:
+ // Acquire only one lock; it's the same file. This is a
+ // bit of a edge case, but presumably it's possible.
+ if !dst.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer dst.mu.Unlock()
+ }
+ // Use both offsets (locked).
+ opts.DstStart = dst.offset
+ opts.SrcStart = src.offset
+ } else if !dstPipe && !opts.DstOffset {
+ // Acquire only dst.
+ if !dst.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer dst.mu.Unlock()
+ opts.DstStart = dst.offset // Safe: locked.
+ } else if !srcPipe && !opts.SrcOffset {
+ // Acquire only src.
+ if !src.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer src.mu.Unlock()
+ opts.SrcStart = src.offset // Safe: locked.
+ }
+
+ // Check append-only mode and the limit.
+ if !dstPipe {
+ if dst.Flags().Append {
+ if opts.DstOffset {
+ // We need to acquire the lock.
+ if !dst.mu.Lock(ctx) {
+ return 0, syserror.ErrInterrupted
+ }
+ defer dst.mu.Unlock()
+ }
+ // Figure out the appropriate offset to use.
+ if err := dst.offsetForAppend(ctx, &opts.DstStart); err != nil {
+ return 0, err
+ }
+ }
+
+ // Enforce file limits.
+ limit, ok := dst.checkLimit(ctx, opts.DstStart)
+ switch {
+ case ok && limit == 0:
+ return 0, syserror.ErrExceedsFileSizeLimit
+ case ok && limit < opts.Length:
+ opts.Length = limit // Cap the write.
+ }
+ }
+
+ // Attempt to do a WriteTo; this is likely the most efficient.
+ //
+ // The underlying implementation may be able to donate buffers.
+ newOpts := SpliceOpts{
+ Length: opts.Length,
+ SrcStart: opts.SrcStart,
+ SrcOffset: !srcPipe,
+ Dup: opts.Dup,
+ DstStart: opts.DstStart,
+ DstOffset: !dstPipe,
+ }
+ n, err := src.FileOperations.WriteTo(ctx, src, dst, newOpts)
+ if n == 0 && err != nil {
+ // Attempt as a ReadFrom. If a WriteTo, a ReadFrom may also
+ // be more efficient than a copy if buffers are cached or readily
+ // available. (It's unlikely that they can actually be donate
+ n, err = dst.FileOperations.ReadFrom(ctx, dst, src, newOpts)
+ }
+ if n == 0 && err != nil {
+ // If we've failed up to here, and at least one of the sources
+ // is a pipe or socket, then we can't properly support dup.
+ // Return an error indicating that this operation is not
+ // supported.
+ if (srcPipe || dstPipe) && newOpts.Dup {
+ return 0, syserror.EINVAL
+ }
+
+ // We failed to splice the files. But that's fine; we just fall
+ // back to a slow path in this case. This copies without doing
+ // any mode changes, so should still be more efficient.
+ var (
+ r io.Reader
+ w io.Writer
+ )
+ fw := &lockedWriter{
+ Ctx: ctx,
+ File: dst,
+ }
+ if newOpts.DstOffset {
+ // Use the provided offset.
+ w = secio.NewOffsetWriter(fw, newOpts.DstStart)
+ } else {
+ // Writes will proceed with no offset.
+ w = fw
+ }
+ fr := &lockedReader{
+ Ctx: ctx,
+ File: src,
+ }
+ if newOpts.SrcOffset {
+ // Limit to the given offset and length.
+ r = io.NewSectionReader(fr, opts.SrcStart, opts.Length)
+ } else {
+ // Limit just to the given length.
+ r = &io.LimitedReader{fr, opts.Length}
+ }
+
+ // Copy between the two.
+ n, err = io.Copy(w, r)
+ }
+
+ // Update offsets, if required.
+ if n > 0 {
+ if !dstPipe && !opts.DstOffset {
+ atomic.StoreInt64(&dst.offset, dst.offset+n)
+ }
+ if !srcPipe && !opts.SrcOffset {
+ atomic.StoreInt64(&src.offset, src.offset+n)
+ }
+ }
+
+ return n, err
+}
diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go
index 749961f51..bce5f091d 100644
--- a/pkg/sentry/fs/timerfd/timerfd.go
+++ b/pkg/sentry/fs/timerfd/timerfd.go
@@ -36,9 +36,10 @@ type TimerOperations struct {
fsutil.FileZeroSeek `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileNoFsync `state:"nosave"`
- fsutil.FileNoopFlush `state:"nosave"`
- fsutil.FileNoMMap `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
events waiter.Queue `state:"zerovalue"`
diff --git a/pkg/sentry/fs/tmpfs/file_regular.go b/pkg/sentry/fs/tmpfs/file_regular.go
index 1ef256511..d1c163879 100644
--- a/pkg/sentry/fs/tmpfs/file_regular.go
+++ b/pkg/sentry/fs/tmpfs/file_regular.go
@@ -28,14 +28,15 @@ import (
//
// +stateify savable
type regularFileOperations struct {
- waiter.AlwaysReady `state:"nosave"`
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,
diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go
index 8dc40e1f2..2603354c4 100644
--- a/pkg/sentry/fs/tty/dir.go
+++ b/pkg/sentry/fs/tty/dir.go
@@ -286,14 +286,15 @@ func (d *dirInodeOperations) masterClose(t *Terminal) {
//
// +stateify savable
type dirFileOperations struct {
- waiter.AlwaysReady `state:"nosave"`
fsutil.FileNoopRelease `state:"nosave"`
fsutil.FileGenericSeek `state:"nosave"`
fsutil.FileNoFsync `state:"nosave"`
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+ waiter.AlwaysReady `state:"nosave"`
// di is the inode operations.
di *dirInodeOperations
diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go
index 45e167e5f..afdf44cd1 100644
--- a/pkg/sentry/fs/tty/master.go
+++ b/pkg/sentry/fs/tty/master.go
@@ -98,8 +98,9 @@ type masterFileOperations struct {
fsutil.FilePipeSeek `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileNoFsync `state:"nosave"`
- fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
// d is the containing dir.
diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go
index 0ae57a02c..2abf32e57 100644
--- a/pkg/sentry/fs/tty/slave.go
+++ b/pkg/sentry/fs/tty/slave.go
@@ -87,8 +87,9 @@ type slaveFileOperations struct {
fsutil.FilePipeSeek `state:"nosave"`
fsutil.FileNotDirReaddir `state:"nosave"`
fsutil.FileNoFsync `state:"nosave"`
- fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
// si is the inode operations.