diff options
author | Jamie Liu <jamieliu@google.com> | 2020-05-26 21:42:07 -0700 |
---|---|---|
committer | gVisor bot <gvisor-bot@google.com> | 2020-05-26 21:43:26 -0700 |
commit | af3121a52383fb60579d769994be5d91bd788015 (patch) | |
tree | 7ab26ef04f70f2bfa25433c262cfd0564a083af1 | |
parent | 8e2c5d951ea04f47ad1d37bab196a99fc56f5521 (diff) |
Implement splice(2) and tee(2) for VFS2.
Updates #138
PiperOrigin-RevId: 313326354
-rw-r--r-- | pkg/buffer/safemem.go | 82 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/tmpfs/regular_file.go | 2 | ||||
-rw-r--r-- | pkg/sentry/kernel/pipe/BUILD | 2 | ||||
-rw-r--r-- | pkg/sentry/kernel/pipe/pipe.go | 6 | ||||
-rw-r--r-- | pkg/sentry/kernel/pipe/pipe_unsafe.go | 35 | ||||
-rw-r--r-- | pkg/sentry/kernel/pipe/vfs.go | 219 | ||||
-rw-r--r-- | pkg/sentry/syscalls/linux/vfs2/BUILD | 1 | ||||
-rw-r--r-- | pkg/sentry/syscalls/linux/vfs2/splice.go | 286 | ||||
-rw-r--r-- | pkg/sentry/syscalls/linux/vfs2/vfs2.go | 4 | ||||
-rw-r--r-- | pkg/sentry/vfs/file_description.go | 5 | ||||
-rw-r--r-- | test/syscalls/linux/splice.cc | 49 |
11 files changed, 648 insertions, 43 deletions
diff --git a/pkg/buffer/safemem.go b/pkg/buffer/safemem.go index 0e5b86344..b789e56e9 100644 --- a/pkg/buffer/safemem.go +++ b/pkg/buffer/safemem.go @@ -28,12 +28,11 @@ func (b *buffer) ReadBlock() safemem.Block { return safemem.BlockFromSafeSlice(b.ReadSlice()) } -// WriteFromBlocks implements safemem.Writer.WriteFromBlocks. -// -// This will advance the write index. -func (v *View) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) { - need := int(srcs.NumBytes()) - if need == 0 { +// WriteFromSafememReader writes up to count bytes from r to v and advances the +// write index by the number of bytes written. It calls r.ReadToBlocks() at +// most once. +func (v *View) WriteFromSafememReader(r safemem.Reader, count uint64) (uint64, error) { + if count == 0 { return 0, nil } @@ -50,32 +49,33 @@ func (v *View) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) { } // Does the last block have sufficient capacity alone? - if l := firstBuf.WriteSize(); l >= need { - dst = safemem.BlockSeqOf(firstBuf.WriteBlock()) + if l := uint64(firstBuf.WriteSize()); l >= count { + dst = safemem.BlockSeqOf(firstBuf.WriteBlock().TakeFirst64(count)) } else { // Append blocks until sufficient. - need -= l + count -= l blocks = append(blocks, firstBuf.WriteBlock()) - for need > 0 { + for count > 0 { emptyBuf := bufferPool.Get().(*buffer) v.data.PushBack(emptyBuf) - need -= emptyBuf.WriteSize() - blocks = append(blocks, emptyBuf.WriteBlock()) + block := emptyBuf.WriteBlock().TakeFirst64(count) + count -= uint64(block.Len()) + blocks = append(blocks, block) } dst = safemem.BlockSeqFromSlice(blocks) } - // Perform the copy. - n, err := safemem.CopySeq(dst, srcs) + // Perform I/O. + n, err := r.ReadToBlocks(dst) v.size += int64(n) // Update all indices. - for left := int(n); left > 0; firstBuf = firstBuf.Next() { - if l := firstBuf.WriteSize(); left >= l { + for left := n; left > 0; firstBuf = firstBuf.Next() { + if l := firstBuf.WriteSize(); left >= uint64(l) { firstBuf.WriteMove(l) // Whole block. - left -= l + left -= uint64(l) } else { - firstBuf.WriteMove(left) // Partial block. + firstBuf.WriteMove(int(left)) // Partial block. left = 0 } } @@ -83,14 +83,16 @@ func (v *View) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) { return n, err } -// ReadToBlocks implements safemem.Reader.ReadToBlocks. -// -// This will not advance the read index; the caller should follow -// this call with a call to TrimFront in order to remove the read -// data from the buffer. This is done to support pipe sematics. -func (v *View) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error) { - need := int(dsts.NumBytes()) - if need == 0 { +// WriteFromBlocks implements safemem.Writer.WriteFromBlocks. It advances the +// write index by the number of bytes written. +func (v *View) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) { + return v.WriteFromSafememReader(&safemem.BlockSeqReader{srcs}, srcs.NumBytes()) +} + +// ReadToSafememWriter reads up to count bytes from v to w. It does not advance +// the read index. It calls w.WriteFromBlocks() at most once. +func (v *View) ReadToSafememWriter(w safemem.Writer, count uint64) (uint64, error) { + if count == 0 { return 0, nil } @@ -105,25 +107,27 @@ func (v *View) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error) { } // Is all the data in a single block? - if l := firstBuf.ReadSize(); l >= need { - src = safemem.BlockSeqOf(firstBuf.ReadBlock()) + if l := uint64(firstBuf.ReadSize()); l >= count { + src = safemem.BlockSeqOf(firstBuf.ReadBlock().TakeFirst64(count)) } else { // Build a list of all the buffers. - need -= l + count -= l blocks = append(blocks, firstBuf.ReadBlock()) - for buf := firstBuf.Next(); buf != nil && need > 0; buf = buf.Next() { - need -= buf.ReadSize() - blocks = append(blocks, buf.ReadBlock()) + for buf := firstBuf.Next(); buf != nil && count > 0; buf = buf.Next() { + block := buf.ReadBlock().TakeFirst64(count) + count -= uint64(block.Len()) + blocks = append(blocks, block) } src = safemem.BlockSeqFromSlice(blocks) } - // Perform the copy. - n, err := safemem.CopySeq(dsts, src) - - // See above: we would normally advance the read index here, but we - // don't do that in order to support pipe semantics. We rely on a - // separate call to TrimFront() in this case. + // Perform I/O. As documented, we don't advance the read index. + return w.WriteFromBlocks(src) +} - return n, err +// ReadToBlocks implements safemem.Reader.ReadToBlocks. It does not advance the +// read index by the number of bytes read, such that it's only safe to call if +// the caller guarantees that ReadToBlocks will only be called once. +func (v *View) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error) { + return v.ReadToSafememWriter(&safemem.BlockSeqWriter{dsts}, dsts.NumBytes()) } diff --git a/pkg/sentry/fsimpl/tmpfs/regular_file.go b/pkg/sentry/fsimpl/tmpfs/regular_file.go index 3f433d666..fee174375 100644 --- a/pkg/sentry/fsimpl/tmpfs/regular_file.go +++ b/pkg/sentry/fsimpl/tmpfs/regular_file.go @@ -312,7 +312,7 @@ func (fd *regularFileFD) PWrite(ctx context.Context, src usermem.IOSequence, off f := fd.inode().impl.(*regularFile) if end := offset + srclen; end < offset { // Overflow. - return 0, syserror.EFBIG + return 0, syserror.EINVAL } var err error diff --git a/pkg/sentry/kernel/pipe/BUILD b/pkg/sentry/kernel/pipe/BUILD index f29dc0472..7bfa9075a 100644 --- a/pkg/sentry/kernel/pipe/BUILD +++ b/pkg/sentry/kernel/pipe/BUILD @@ -8,6 +8,7 @@ go_library( "device.go", "node.go", "pipe.go", + "pipe_unsafe.go", "pipe_util.go", "reader.go", "reader_writer.go", @@ -20,6 +21,7 @@ go_library( "//pkg/amutex", "//pkg/buffer", "//pkg/context", + "//pkg/safemem", "//pkg/sentry/arch", "//pkg/sentry/device", "//pkg/sentry/fs", diff --git a/pkg/sentry/kernel/pipe/pipe.go b/pkg/sentry/kernel/pipe/pipe.go index 62c8691f1..79645d7d2 100644 --- a/pkg/sentry/kernel/pipe/pipe.go +++ b/pkg/sentry/kernel/pipe/pipe.go @@ -207,7 +207,10 @@ func (p *Pipe) read(ctx context.Context, ops readOps) (int64, error) { p.mu.Lock() defer p.mu.Unlock() + return p.readLocked(ctx, ops) +} +func (p *Pipe) readLocked(ctx context.Context, ops readOps) (int64, error) { // Is the pipe empty? if p.view.Size() == 0 { if !p.HasWriters() { @@ -246,7 +249,10 @@ type writeOps struct { func (p *Pipe) write(ctx context.Context, ops writeOps) (int64, error) { p.mu.Lock() defer p.mu.Unlock() + return p.writeLocked(ctx, ops) +} +func (p *Pipe) writeLocked(ctx context.Context, ops writeOps) (int64, error) { // Can't write to a pipe with no readers. if !p.HasReaders() { return 0, syscall.EPIPE diff --git a/pkg/sentry/kernel/pipe/pipe_unsafe.go b/pkg/sentry/kernel/pipe/pipe_unsafe.go new file mode 100644 index 000000000..dd60cba24 --- /dev/null +++ b/pkg/sentry/kernel/pipe/pipe_unsafe.go @@ -0,0 +1,35 @@ +// 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 pipe + +import ( + "unsafe" +) + +// lockTwoPipes locks both x.mu and y.mu in an order that is guaranteed to be +// consistent for both lockTwoPipes(x, y) and lockTwoPipes(y, x), such that +// concurrent calls cannot deadlock. +// +// Preconditions: x != y. +func lockTwoPipes(x, y *Pipe) { + // Lock the two pipes in order of increasing address. + if uintptr(unsafe.Pointer(x)) < uintptr(unsafe.Pointer(y)) { + x.mu.Lock() + y.mu.Lock() + } else { + y.mu.Lock() + x.mu.Lock() + } +} diff --git a/pkg/sentry/kernel/pipe/vfs.go b/pkg/sentry/kernel/pipe/vfs.go index b54f08a30..2602bed72 100644 --- a/pkg/sentry/kernel/pipe/vfs.go +++ b/pkg/sentry/kernel/pipe/vfs.go @@ -16,7 +16,9 @@ package pipe import ( "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/buffer" "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/safemem" "gvisor.dev/gvisor/pkg/sentry/arch" "gvisor.dev/gvisor/pkg/sentry/vfs" "gvisor.dev/gvisor/pkg/sync" @@ -150,7 +152,9 @@ func (vp *VFSPipe) newFD(mnt *vfs.Mount, vfsd *vfs.Dentry, statusFlags uint32) * return &fd.vfsfd } -// VFSPipeFD implements vfs.FileDescriptionImpl for pipes. +// VFSPipeFD implements vfs.FileDescriptionImpl for pipes. It also implements +// non-atomic usermem.IO methods, allowing it to be passed as usermem.IO to +// other FileDescriptions for splice(2) and tee(2). type VFSPipeFD struct { vfsfd vfs.FileDescription vfs.FileDescriptionDefaultImpl @@ -229,3 +233,216 @@ func (fd *VFSPipeFD) PipeSize() int64 { func (fd *VFSPipeFD) SetPipeSize(size int64) (int64, error) { return fd.pipe.SetFifoSize(size) } + +// IOSequence returns a useremm.IOSequence that reads up to count bytes from, +// or writes up to count bytes to, fd. +func (fd *VFSPipeFD) IOSequence(count int64) usermem.IOSequence { + return usermem.IOSequence{ + IO: fd, + Addrs: usermem.AddrRangeSeqOf(usermem.AddrRange{0, usermem.Addr(count)}), + } +} + +// CopyIn implements usermem.IO.CopyIn. +func (fd *VFSPipeFD) CopyIn(ctx context.Context, addr usermem.Addr, dst []byte, opts usermem.IOOpts) (int, error) { + origCount := int64(len(dst)) + n, err := fd.pipe.read(ctx, readOps{ + left: func() int64 { + return int64(len(dst)) + }, + limit: func(l int64) { + dst = dst[:l] + }, + read: func(view *buffer.View) (int64, error) { + n, err := view.ReadAt(dst, 0) + view.TrimFront(int64(n)) + return int64(n), err + }, + }) + if n > 0 { + fd.pipe.Notify(waiter.EventOut) + } + if err == nil && n != origCount { + return int(n), syserror.ErrWouldBlock + } + return int(n), err +} + +// CopyOut implements usermem.IO.CopyOut. +func (fd *VFSPipeFD) CopyOut(ctx context.Context, addr usermem.Addr, src []byte, opts usermem.IOOpts) (int, error) { + origCount := int64(len(src)) + n, err := fd.pipe.write(ctx, writeOps{ + left: func() int64 { + return int64(len(src)) + }, + limit: func(l int64) { + src = src[:l] + }, + write: func(view *buffer.View) (int64, error) { + view.Append(src) + return int64(len(src)), nil + }, + }) + if n > 0 { + fd.pipe.Notify(waiter.EventIn) + } + if err == nil && n != origCount { + return int(n), syserror.ErrWouldBlock + } + return int(n), err +} + +// ZeroOut implements usermem.IO.ZeroOut. +func (fd *VFSPipeFD) ZeroOut(ctx context.Context, addr usermem.Addr, toZero int64, opts usermem.IOOpts) (int64, error) { + origCount := toZero + n, err := fd.pipe.write(ctx, writeOps{ + left: func() int64 { + return toZero + }, + limit: func(l int64) { + toZero = l + }, + write: func(view *buffer.View) (int64, error) { + view.Grow(view.Size()+toZero, true /* zero */) + return toZero, nil + }, + }) + if n > 0 { + fd.pipe.Notify(waiter.EventIn) + } + if err == nil && n != origCount { + return n, syserror.ErrWouldBlock + } + return n, err +} + +// CopyInTo implements usermem.IO.CopyInTo. +func (fd *VFSPipeFD) CopyInTo(ctx context.Context, ars usermem.AddrRangeSeq, dst safemem.Writer, opts usermem.IOOpts) (int64, error) { + count := ars.NumBytes() + if count == 0 { + return 0, nil + } + origCount := count + n, err := fd.pipe.read(ctx, readOps{ + left: func() int64 { + return count + }, + limit: func(l int64) { + count = l + }, + read: func(view *buffer.View) (int64, error) { + n, err := view.ReadToSafememWriter(dst, uint64(count)) + view.TrimFront(int64(n)) + return int64(n), err + }, + }) + if n > 0 { + fd.pipe.Notify(waiter.EventOut) + } + if err == nil && n != origCount { + return n, syserror.ErrWouldBlock + } + return n, err +} + +// CopyOutFrom implements usermem.IO.CopyOutFrom. +func (fd *VFSPipeFD) CopyOutFrom(ctx context.Context, ars usermem.AddrRangeSeq, src safemem.Reader, opts usermem.IOOpts) (int64, error) { + count := ars.NumBytes() + if count == 0 { + return 0, nil + } + origCount := count + n, err := fd.pipe.write(ctx, writeOps{ + left: func() int64 { + return count + }, + limit: func(l int64) { + count = l + }, + write: func(view *buffer.View) (int64, error) { + n, err := view.WriteFromSafememReader(src, uint64(count)) + return int64(n), err + }, + }) + if n > 0 { + fd.pipe.Notify(waiter.EventIn) + } + if err == nil && n != origCount { + return n, syserror.ErrWouldBlock + } + return n, err +} + +// SwapUint32 implements usermem.IO.SwapUint32. +func (fd *VFSPipeFD) SwapUint32(ctx context.Context, addr usermem.Addr, new uint32, opts usermem.IOOpts) (uint32, error) { + // How did a pipe get passed as the virtual address space to futex(2)? + panic("VFSPipeFD.SwapUint32 called unexpectedly") +} + +// CompareAndSwapUint32 implements usermem.IO.CompareAndSwapUint32. +func (fd *VFSPipeFD) CompareAndSwapUint32(ctx context.Context, addr usermem.Addr, old, new uint32, opts usermem.IOOpts) (uint32, error) { + panic("VFSPipeFD.CompareAndSwapUint32 called unexpectedly") +} + +// LoadUint32 implements usermem.IO.LoadUint32. +func (fd *VFSPipeFD) LoadUint32(ctx context.Context, addr usermem.Addr, opts usermem.IOOpts) (uint32, error) { + panic("VFSPipeFD.LoadUint32 called unexpectedly") +} + +// Splice reads up to count bytes from src and writes them to dst. It returns +// the number of bytes moved. +// +// Preconditions: count > 0. +func Splice(ctx context.Context, dst, src *VFSPipeFD, count int64) (int64, error) { + return spliceOrTee(ctx, dst, src, count, true /* removeFromSrc */) +} + +// Tee reads up to count bytes from src and writes them to dst, without +// removing the read bytes from src. It returns the number of bytes copied. +// +// Preconditions: count > 0. +func Tee(ctx context.Context, dst, src *VFSPipeFD, count int64) (int64, error) { + return spliceOrTee(ctx, dst, src, count, false /* removeFromSrc */) +} + +// Preconditions: count > 0. +func spliceOrTee(ctx context.Context, dst, src *VFSPipeFD, count int64, removeFromSrc bool) (int64, error) { + if dst.pipe == src.pipe { + return 0, syserror.EINVAL + } + + lockTwoPipes(dst.pipe, src.pipe) + defer dst.pipe.mu.Unlock() + defer src.pipe.mu.Unlock() + + n, err := dst.pipe.writeLocked(ctx, writeOps{ + left: func() int64 { + return count + }, + limit: func(l int64) { + count = l + }, + write: func(dstView *buffer.View) (int64, error) { + return src.pipe.readLocked(ctx, readOps{ + left: func() int64 { + return count + }, + limit: func(l int64) { + count = l + }, + read: func(srcView *buffer.View) (int64, error) { + n, err := srcView.ReadToSafememWriter(dstView, uint64(count)) + if n > 0 && removeFromSrc { + srcView.TrimFront(int64(n)) + } + return int64(n), err + }, + }) + }, + }) + if n > 0 { + dst.pipe.Notify(waiter.EventIn) + src.pipe.Notify(waiter.EventOut) + } + return n, err +} diff --git a/pkg/sentry/syscalls/linux/vfs2/BUILD b/pkg/sentry/syscalls/linux/vfs2/BUILD index f882ef840..d56927ff5 100644 --- a/pkg/sentry/syscalls/linux/vfs2/BUILD +++ b/pkg/sentry/syscalls/linux/vfs2/BUILD @@ -22,6 +22,7 @@ go_library( "setstat.go", "signal.go", "socket.go", + "splice.go", "stat.go", "stat_amd64.go", "stat_arm64.go", diff --git a/pkg/sentry/syscalls/linux/vfs2/splice.go b/pkg/sentry/syscalls/linux/vfs2/splice.go new file mode 100644 index 000000000..8f3c22a02 --- /dev/null +++ b/pkg/sentry/syscalls/linux/vfs2/splice.go @@ -0,0 +1,286 @@ +// Copyright 2020 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package vfs2 + +import ( + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/sentry/arch" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/pipe" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/waiter" +) + +// Splice implements Linux syscall splice(2). +func Splice(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { + inFD := args[0].Int() + inOffsetPtr := args[1].Pointer() + outFD := args[2].Int() + outOffsetPtr := args[3].Pointer() + count := int64(args[4].SizeT()) + flags := args[5].Int() + + if count == 0 { + return 0, nil, nil + } + if count > int64(kernel.MAX_RW_COUNT) { + count = int64(kernel.MAX_RW_COUNT) + } + + // Check for invalid flags. + if flags&^(linux.SPLICE_F_MOVE|linux.SPLICE_F_NONBLOCK|linux.SPLICE_F_MORE|linux.SPLICE_F_GIFT) != 0 { + return 0, nil, syserror.EINVAL + } + + // Get file descriptions. + inFile := t.GetFileVFS2(inFD) + if inFile == nil { + return 0, nil, syserror.EBADF + } + defer inFile.DecRef() + outFile := t.GetFileVFS2(outFD) + if outFile == nil { + return 0, nil, syserror.EBADF + } + defer outFile.DecRef() + + // Check that both files support the required directionality. + if !inFile.IsReadable() || !outFile.IsWritable() { + return 0, nil, syserror.EBADF + } + + // The operation is non-blocking if anything is non-blocking. + // + // N.B. This is a rather simplistic heuristic that avoids some + // poor edge case behavior since the exact semantics here are + // underspecified and vary between versions of Linux itself. + nonBlock := ((inFile.StatusFlags()|outFile.StatusFlags())&linux.O_NONBLOCK != 0) || (flags&linux.SPLICE_F_NONBLOCK != 0) + + // At least one file description must represent a pipe. + inPipeFD, inIsPipe := inFile.Impl().(*pipe.VFSPipeFD) + outPipeFD, outIsPipe := outFile.Impl().(*pipe.VFSPipeFD) + if !inIsPipe && !outIsPipe { + return 0, nil, syserror.EINVAL + } + + // Copy in offsets. + inOffset := int64(-1) + if inOffsetPtr != 0 { + if inIsPipe { + return 0, nil, syserror.ESPIPE + } + if inFile.Options().DenyPRead { + return 0, nil, syserror.EINVAL + } + if _, err := t.CopyIn(inOffsetPtr, &inOffset); err != nil { + return 0, nil, err + } + if inOffset < 0 { + return 0, nil, syserror.EINVAL + } + } + outOffset := int64(-1) + if outOffsetPtr != 0 { + if outIsPipe { + return 0, nil, syserror.ESPIPE + } + if outFile.Options().DenyPWrite { + return 0, nil, syserror.EINVAL + } + if _, err := t.CopyIn(outOffsetPtr, &outOffset); err != nil { + return 0, nil, err + } + if outOffset < 0 { + return 0, nil, syserror.EINVAL + } + } + + // Move data. + var ( + n int64 + err error + inCh chan struct{} + outCh chan struct{} + ) + for { + // If both input and output are pipes, delegate to the pipe + // implementation. Otherwise, exactly one end is a pipe, which we + // ensure is consistently ordered after the non-pipe FD's locks by + // passing the pipe FD as usermem.IO to the non-pipe end. + switch { + case inIsPipe && outIsPipe: + n, err = pipe.Splice(t, outPipeFD, inPipeFD, count) + case inIsPipe: + if outOffset != -1 { + n, err = outFile.PWrite(t, inPipeFD.IOSequence(count), outOffset, vfs.WriteOptions{}) + outOffset += n + } else { + n, err = outFile.Write(t, inPipeFD.IOSequence(count), vfs.WriteOptions{}) + } + case outIsPipe: + if inOffset != -1 { + n, err = inFile.PRead(t, outPipeFD.IOSequence(count), inOffset, vfs.ReadOptions{}) + inOffset += n + } else { + n, err = inFile.Read(t, outPipeFD.IOSequence(count), vfs.ReadOptions{}) + } + } + if n != 0 || err != syserror.ErrWouldBlock || nonBlock { + break + } + + // Note that the blocking behavior here is a bit different than the + // normal pattern. Because we need to have both data to read and data + // to write simultaneously, we actually explicitly block on both of + // these cases in turn before returning to the splice operation. + if inFile.Readiness(eventMaskRead)&eventMaskRead == 0 { + if inCh == nil { + inCh = make(chan struct{}, 1) + inW, _ := waiter.NewChannelEntry(inCh) + inFile.EventRegister(&inW, eventMaskRead) + defer inFile.EventUnregister(&inW) + continue // Need to refresh readiness. + } + if err = t.Block(inCh); err != nil { + break + } + } + if outFile.Readiness(eventMaskWrite)&eventMaskWrite == 0 { + if outCh == nil { + outCh = make(chan struct{}, 1) + outW, _ := waiter.NewChannelEntry(outCh) + outFile.EventRegister(&outW, eventMaskWrite) + defer outFile.EventUnregister(&outW) + continue // Need to refresh readiness. + } + if err = t.Block(outCh); err != nil { + break + } + } + } + + // Copy updated offsets out. + if inOffsetPtr != 0 { + if _, err := t.CopyOut(inOffsetPtr, &inOffset); err != nil { + return 0, nil, err + } + } + if outOffsetPtr != 0 { + if _, err := t.CopyOut(outOffsetPtr, &outOffset); err != nil { + return 0, nil, err + } + } + + if n == 0 { + return 0, nil, err + } + return uintptr(n), nil, nil +} + +// Tee implements Linux syscall tee(2). +func Tee(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) { + inFD := args[0].Int() + outFD := args[1].Int() + count := int64(args[2].SizeT()) + flags := args[3].Int() + + if count == 0 { + return 0, nil, nil + } + if count > int64(kernel.MAX_RW_COUNT) { + count = int64(kernel.MAX_RW_COUNT) + } + + // Check for invalid flags. + if flags&^(linux.SPLICE_F_MOVE|linux.SPLICE_F_NONBLOCK|linux.SPLICE_F_MORE|linux.SPLICE_F_GIFT) != 0 { + return 0, nil, syserror.EINVAL + } + + // Get file descriptions. + inFile := t.GetFileVFS2(inFD) + if inFile == nil { + return 0, nil, syserror.EBADF + } + defer inFile.DecRef() + outFile := t.GetFileVFS2(outFD) + if outFile == nil { + return 0, nil, syserror.EBADF + } + defer outFile.DecRef() + + // Check that both files support the required directionality. + if !inFile.IsReadable() || !outFile.IsWritable() { + return 0, nil, syserror.EBADF + } + + // The operation is non-blocking if anything is non-blocking. + // + // N.B. This is a rather simplistic heuristic that avoids some + // poor edge case behavior since the exact semantics here are + // underspecified and vary between versions of Linux itself. + nonBlock := ((inFile.StatusFlags()|outFile.StatusFlags())&linux.O_NONBLOCK != 0) || (flags&linux.SPLICE_F_NONBLOCK != 0) + + // Both file descriptions must represent pipes. + inPipeFD, inIsPipe := inFile.Impl().(*pipe.VFSPipeFD) + outPipeFD, outIsPipe := outFile.Impl().(*pipe.VFSPipeFD) + if !inIsPipe || !outIsPipe { + return 0, nil, syserror.EINVAL + } + + // Copy data. + var ( + inCh chan struct{} + outCh chan struct{} + ) + for { + n, err := pipe.Tee(t, outPipeFD, inPipeFD, count) + if n != 0 { + return uintptr(n), nil, nil + } + if err != syserror.ErrWouldBlock || nonBlock { + return 0, nil, err + } + + // Note that the blocking behavior here is a bit different than the + // normal pattern. Because we need to have both data to read and data + // to write simultaneously, we actually explicitly block on both of + // these cases in turn before returning to the tee operation. + if inFile.Readiness(eventMaskRead)&eventMaskRead == 0 { + if inCh == nil { + inCh = make(chan struct{}, 1) + inW, _ := waiter.NewChannelEntry(inCh) + inFile.EventRegister(&inW, eventMaskRead) + defer inFile.EventUnregister(&inW) + continue // Need to refresh readiness. + } + if err := t.Block(inCh); err != nil { + return 0, nil, err + } + } + if outFile.Readiness(eventMaskWrite)&eventMaskWrite == 0 { + if outCh == nil { + outCh = make(chan struct{}, 1) + outW, _ := waiter.NewChannelEntry(outCh) + outFile.EventRegister(&outW, eventMaskWrite) + defer outFile.EventUnregister(&outW) + continue // Need to refresh readiness. + } + if err := t.Block(outCh); err != nil { + return 0, nil, err + } + } + } +} diff --git a/pkg/sentry/syscalls/linux/vfs2/vfs2.go b/pkg/sentry/syscalls/linux/vfs2/vfs2.go index a332d01bd..083fdcf82 100644 --- a/pkg/sentry/syscalls/linux/vfs2/vfs2.go +++ b/pkg/sentry/syscalls/linux/vfs2/vfs2.go @@ -134,8 +134,8 @@ func Override() { s.Table[269] = syscalls.Supported("faccessat", Faccessat) s.Table[270] = syscalls.Supported("pselect", Pselect) s.Table[271] = syscalls.Supported("ppoll", Ppoll) - delete(s.Table, 275) // splice - delete(s.Table, 276) // tee + s.Table[275] = syscalls.Supported("splice", Splice) + s.Table[276] = syscalls.Supported("tee", Tee) s.Table[277] = syscalls.Supported("sync_file_range", SyncFileRange) s.Table[280] = syscalls.Supported("utimensat", Utimensat) s.Table[281] = syscalls.Supported("epoll_pwait", EpollPwait) diff --git a/pkg/sentry/vfs/file_description.go b/pkg/sentry/vfs/file_description.go index cfabd936c..bb294563d 100644 --- a/pkg/sentry/vfs/file_description.go +++ b/pkg/sentry/vfs/file_description.go @@ -210,6 +210,11 @@ func (fd *FileDescription) VirtualDentry() VirtualDentry { return fd.vd } +// Options returns the options passed to fd.Init(). +func (fd *FileDescription) Options() FileDescriptionOptions { + return fd.opts +} + // StatusFlags returns file description status flags, as for fcntl(F_GETFL). func (fd *FileDescription) StatusFlags() uint32 { return atomic.LoadUint32(&fd.statusFlags) diff --git a/test/syscalls/linux/splice.cc b/test/syscalls/linux/splice.cc index f103e2e56..08fc4b1b7 100644 --- a/test/syscalls/linux/splice.cc +++ b/test/syscalls/linux/splice.cc @@ -430,6 +430,55 @@ TEST(SpliceTest, TwoPipes) { EXPECT_EQ(memcmp(rbuf.data(), buf.data(), kPageSize), 0); } +TEST(SpliceTest, TwoPipesCircular) { + // This test deadlocks the sentry on VFS1 because VFS1 splice ordering is + // based on fs.File.UniqueID, which does not prevent circular ordering between + // e.g. inode-level locks taken by fs.FileOperations. + SKIP_IF(IsRunningWithVFS1()); + + // Create two pipes. + int fds[2]; + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor first_rfd(fds[0]); + const FileDescriptor first_wfd(fds[1]); + ASSERT_THAT(pipe(fds), SyscallSucceeds()); + const FileDescriptor second_rfd(fds[0]); + const FileDescriptor second_wfd(fds[1]); + + // On Linux, each pipe is normally limited to + // include/linux/pipe_fs_i.h:PIPE_DEF_BUFFERS buffers worth of data. + constexpr size_t PIPE_DEF_BUFFERS = 16; + + // Write some data to each pipe. Below we splice 1 byte at a time between + // pipes, which very quickly causes each byte to be stored in a separate + // buffer, so we must ensure that the total amount of data in the system is <= + // PIPE_DEF_BUFFERS bytes. + std::vector<char> buf(PIPE_DEF_BUFFERS / 2); + RandomizeBuffer(buf.data(), buf.size()); + ASSERT_THAT(write(first_wfd.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(buf.size())); + ASSERT_THAT(write(second_wfd.get(), buf.data(), buf.size()), + SyscallSucceedsWithValue(buf.size())); + + // Have another thread splice from the second pipe to the first, while we + // splice from the first to the second. The test passes if this does not + // deadlock. + const int kIterations = 1000; + DisableSave ds; + ScopedThread t([&]() { + for (int i = 0; i < kIterations; i++) { + ASSERT_THAT( + splice(second_rfd.get(), nullptr, first_wfd.get(), nullptr, 1, 0), + SyscallSucceedsWithValue(1)); + } + }); + for (int i = 0; i < kIterations; i++) { + ASSERT_THAT( + splice(first_rfd.get(), nullptr, second_wfd.get(), nullptr, 1, 0), + SyscallSucceedsWithValue(1)); + } +} + TEST(SpliceTest, Blocking) { // Create two new pipes. int first[2], second[2]; |