summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--pkg/sentry/fsimpl/pipefs/BUILD20
-rw-r--r--pkg/sentry/fsimpl/pipefs/pipefs.go148
-rw-r--r--pkg/sentry/fsimpl/tmpfs/filesystem.go2
-rw-r--r--pkg/sentry/fsimpl/tmpfs/named_pipe.go23
-rw-r--r--pkg/sentry/fsimpl/tmpfs/tmpfs.go2
-rw-r--r--pkg/sentry/kernel/BUILD1
-rw-r--r--pkg/sentry/kernel/kernel.go30
-rw-r--r--pkg/sentry/kernel/pipe/vfs.go162
-rw-r--r--pkg/sentry/syscalls/linux/sys_pipe.go14
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/BUILD3
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/fd.go17
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/linux64_override_amd64.go4
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/pipe.go63
-rw-r--r--pkg/sentry/syscalls/linux/vfs2/read_write.go8
-rw-r--r--pkg/sentry/vfs/vfs.go2
-rw-r--r--test/syscalls/linux/pipe.cc2
16 files changed, 389 insertions, 112 deletions
diff --git a/pkg/sentry/fsimpl/pipefs/BUILD b/pkg/sentry/fsimpl/pipefs/BUILD
new file mode 100644
index 000000000..0d411606f
--- /dev/null
+++ b/pkg/sentry/fsimpl/pipefs/BUILD
@@ -0,0 +1,20 @@
+load("//tools:defs.bzl", "go_library")
+
+licenses(["notice"])
+
+go_library(
+ name = "pipefs",
+ srcs = ["pipefs.go"],
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/context",
+ "//pkg/sentry/fsimpl/kernfs",
+ "//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/pipe",
+ "//pkg/sentry/kernel/time",
+ "//pkg/sentry/vfs",
+ "//pkg/syserror",
+ "//pkg/usermem",
+ ],
+)
diff --git a/pkg/sentry/fsimpl/pipefs/pipefs.go b/pkg/sentry/fsimpl/pipefs/pipefs.go
new file mode 100644
index 000000000..faf3179bc
--- /dev/null
+++ b/pkg/sentry/fsimpl/pipefs/pipefs.go
@@ -0,0 +1,148 @@
+// 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 pipefs provides the filesystem implementation backing
+// Kernel.PipeMount.
+package pipefs
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
+ ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+)
+
+type filesystemType struct{}
+
+// Name implements vfs.FilesystemType.Name.
+func (filesystemType) Name() string {
+ return "pipefs"
+}
+
+// GetFilesystem implements vfs.FilesystemType.GetFilesystem.
+func (filesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
+ panic("pipefs.filesystemType.GetFilesystem should never be called")
+}
+
+// filesystem implements vfs.FilesystemImpl.
+type filesystem struct {
+ kernfs.Filesystem
+
+ // TODO(gvisor.dev/issue/1193):
+ //
+ // - kernfs does not provide a way to implement statfs, from which we
+ // should indicate PIPEFS_MAGIC.
+ //
+ // - kernfs does not provide a way to override names for
+ // vfs.FilesystemImpl.PrependPath(); pipefs inodes should use synthetic
+ // name fmt.Sprintf("pipe:[%d]", inode.ino).
+}
+
+// NewFilesystem sets up and returns a new vfs.Filesystem implemented by
+// pipefs.
+func NewFilesystem(vfsObj *vfs.VirtualFilesystem) *vfs.Filesystem {
+ fs := &filesystem{}
+ fs.Init(vfsObj, filesystemType{})
+ return fs.VFSFilesystem()
+}
+
+// inode implements kernfs.Inode.
+type inode struct {
+ kernfs.InodeNotDirectory
+ kernfs.InodeNotSymlink
+ kernfs.InodeNoopRefCount
+
+ pipe *pipe.VFSPipe
+
+ ino uint64
+ uid auth.KUID
+ gid auth.KGID
+ // We use the creation timestamp for all of atime, mtime, and ctime.
+ ctime ktime.Time
+}
+
+func newInode(ctx context.Context, fs *kernfs.Filesystem) *inode {
+ creds := auth.CredentialsFromContext(ctx)
+ return &inode{
+ pipe: pipe.NewVFSPipe(false /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize),
+ ino: fs.NextIno(),
+ uid: creds.EffectiveKUID,
+ gid: creds.EffectiveKGID,
+ ctime: ktime.NowFromContext(ctx),
+ }
+}
+
+const pipeMode = 0600 | linux.S_IFIFO
+
+// CheckPermissions implements kernfs.Inode.CheckPermissions.
+func (i *inode) CheckPermissions(ctx context.Context, creds *auth.Credentials, ats vfs.AccessTypes) error {
+ return vfs.GenericCheckPermissions(creds, ats, pipeMode, i.uid, i.gid)
+}
+
+// Mode implements kernfs.Inode.Mode.
+func (i *inode) Mode() linux.FileMode {
+ return pipeMode
+}
+
+// Stat implements kernfs.Inode.Stat.
+func (i *inode) Stat(vfsfs *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) {
+ ts := linux.NsecToStatxTimestamp(i.ctime.Nanoseconds())
+ return linux.Statx{
+ Mask: linux.STATX_TYPE | linux.STATX_MODE | linux.STATX_NLINK | linux.STATX_UID | linux.STATX_GID | linux.STATX_ATIME | linux.STATX_MTIME | linux.STATX_CTIME | linux.STATX_INO | linux.STATX_SIZE | linux.STATX_BLOCKS,
+ Blksize: usermem.PageSize,
+ Nlink: 1,
+ UID: uint32(i.uid),
+ GID: uint32(i.gid),
+ Mode: pipeMode,
+ Ino: i.ino,
+ Size: 0,
+ Blocks: 0,
+ Atime: ts,
+ Ctime: ts,
+ Mtime: ts,
+ // TODO(gvisor.dev/issue/1197): Device number.
+ }, nil
+}
+
+// SetStat implements kernfs.Inode.SetStat.
+func (i *inode) SetStat(ctx context.Context, vfsfs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions) error {
+ if opts.Stat.Mask == 0 {
+ return nil
+ }
+ return syserror.EPERM
+}
+
+// Open implements kernfs.Inode.Open.
+func (i *inode) Open(rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
+ // FIXME(b/38173783): kernfs does not plumb Context here.
+ return i.pipe.Open(context.Background(), rp.Mount(), vfsd, opts.Flags)
+}
+
+// NewConnectedPipeFDs returns a pair of FileDescriptions representing the read
+// and write ends of a newly-created pipe, as for pipe(2) and pipe2(2).
+//
+// Preconditions: mnt.Filesystem() must have been returned by NewFilesystem().
+func NewConnectedPipeFDs(ctx context.Context, mnt *vfs.Mount, flags uint32) (*vfs.FileDescription, *vfs.FileDescription) {
+ fs := mnt.Filesystem().Impl().(*kernfs.Filesystem)
+ inode := newInode(ctx, fs)
+ var d kernfs.Dentry
+ d.Init(inode)
+ defer d.DecRef()
+ return inode.pipe.ReaderWriterPair(mnt, d.VFSDentry(), flags)
+}
diff --git a/pkg/sentry/fsimpl/tmpfs/filesystem.go b/pkg/sentry/fsimpl/tmpfs/filesystem.go
index f4d50d64f..660f5a29b 100644
--- a/pkg/sentry/fsimpl/tmpfs/filesystem.go
+++ b/pkg/sentry/fsimpl/tmpfs/filesystem.go
@@ -392,7 +392,7 @@ func (d *dentry) open(ctx context.Context, rp *vfs.ResolvingPath, opts *vfs.Open
// Can't open symlinks without O_PATH (which is unimplemented).
return nil, syserror.ELOOP
case *namedPipe:
- return newNamedPipeFD(ctx, impl, rp, &d.vfsd, opts.Flags)
+ return impl.pipe.Open(ctx, rp.Mount(), &d.vfsd, opts.Flags)
case *deviceFile:
return rp.VirtualFilesystem().OpenDeviceSpecialFile(ctx, rp.Mount(), &d.vfsd, impl.kind, impl.major, impl.minor, opts)
case *socketFile:
diff --git a/pkg/sentry/fsimpl/tmpfs/named_pipe.go b/pkg/sentry/fsimpl/tmpfs/named_pipe.go
index 2c5c739df..8d77b3fa8 100644
--- a/pkg/sentry/fsimpl/tmpfs/named_pipe.go
+++ b/pkg/sentry/fsimpl/tmpfs/named_pipe.go
@@ -16,10 +16,8 @@ package tmpfs
import (
"gvisor.dev/gvisor/pkg/abi/linux"
- "gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
- "gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/usermem"
)
@@ -33,27 +31,8 @@ type namedPipe struct {
// * fs.mu must be locked.
// * rp.Mount().CheckBeginWrite() has been called successfully.
func (fs *filesystem) newNamedPipe(creds *auth.Credentials, mode linux.FileMode) *inode {
- file := &namedPipe{pipe: pipe.NewVFSPipe(pipe.DefaultPipeSize, usermem.PageSize)}
+ file := &namedPipe{pipe: pipe.NewVFSPipe(true /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize)}
file.inode.init(file, fs, creds, linux.S_IFIFO|mode)
file.inode.nlink = 1 // Only the parent has a link.
return &file.inode
}
-
-// namedPipeFD implements vfs.FileDescriptionImpl. Methods are implemented
-// entirely via struct embedding.
-type namedPipeFD struct {
- fileDescription
-
- *pipe.VFSPipeFD
-}
-
-func newNamedPipeFD(ctx context.Context, np *namedPipe, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, flags uint32) (*vfs.FileDescription, error) {
- var err error
- var fd namedPipeFD
- fd.VFSPipeFD, err = np.pipe.NewVFSPipeFD(ctx, vfsd, &fd.vfsfd, flags)
- if err != nil {
- return nil, err
- }
- fd.vfsfd.Init(&fd, flags, rp.Mount(), vfsd, &vfs.FileDescriptionOptions{})
- return &fd.vfsfd, nil
-}
diff --git a/pkg/sentry/fsimpl/tmpfs/tmpfs.go b/pkg/sentry/fsimpl/tmpfs/tmpfs.go
index 9fa8637d5..a59b24d45 100644
--- a/pkg/sentry/fsimpl/tmpfs/tmpfs.go
+++ b/pkg/sentry/fsimpl/tmpfs/tmpfs.go
@@ -357,6 +357,7 @@ func (i *inode) setStat(ctx context.Context, creds *auth.Credentials, stat *linu
return err
}
i.mu.Lock()
+ defer i.mu.Unlock()
var (
needsMtimeBump bool
needsCtimeBump bool
@@ -427,7 +428,6 @@ func (i *inode) setStat(ctx context.Context, creds *auth.Credentials, stat *linu
atomic.StoreInt64(&i.ctime, now)
}
- i.mu.Unlock()
return nil
}
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index e0ff58d8c..e47af66d6 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -170,6 +170,7 @@ go_library(
"//pkg/sentry/fs/timerfd",
"//pkg/sentry/fsbridge",
"//pkg/sentry/fsimpl/kernfs",
+ "//pkg/sentry/fsimpl/pipefs",
"//pkg/sentry/fsimpl/sockfs",
"//pkg/sentry/hostcpu",
"//pkg/sentry/inet",
diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go
index de8a95854..fef60e636 100644
--- a/pkg/sentry/kernel/kernel.go
+++ b/pkg/sentry/kernel/kernel.go
@@ -50,6 +50,7 @@ import (
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/fs/timerfd"
"gvisor.dev/gvisor/pkg/sentry/fsbridge"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/pipefs"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/sockfs"
"gvisor.dev/gvisor/pkg/sentry/hostcpu"
"gvisor.dev/gvisor/pkg/sentry/inet"
@@ -254,6 +255,10 @@ type Kernel struct {
// VFS keeps the filesystem state used across the kernel.
vfs vfs.VirtualFilesystem
+ // pipeMount is the Mount used for pipes created by the pipe() and pipe2()
+ // syscalls (as opposed to named pipes created by mknod()).
+ pipeMount *vfs.Mount
+
// If set to true, report address space activation waits as if the task is in
// external wait so that the watchdog doesn't report the task stuck.
SleepForAddressSpaceActivation bool
@@ -354,19 +359,29 @@ func (k *Kernel) Init(args InitKernelArgs) error {
k.monotonicClock = &timekeeperClock{tk: args.Timekeeper, c: sentrytime.Monotonic}
k.futexes = futex.NewManager()
k.netlinkPorts = port.New()
+
if VFS2Enabled {
if err := k.vfs.Init(); err != nil {
return fmt.Errorf("failed to initialize VFS: %v", err)
}
- fs := sockfs.NewFilesystem(&k.vfs)
- // NewDisconnectedMount will take an additional reference on fs.
- defer fs.DecRef()
- sm, err := k.vfs.NewDisconnectedMount(fs, nil, &vfs.MountOptions{})
+
+ pipeFilesystem := pipefs.NewFilesystem(&k.vfs)
+ defer pipeFilesystem.DecRef()
+ pipeMount, err := k.vfs.NewDisconnectedMount(pipeFilesystem, nil, &vfs.MountOptions{})
+ if err != nil {
+ return fmt.Errorf("failed to create pipefs mount: %v", err)
+ }
+ k.pipeMount = pipeMount
+
+ socketFilesystem := sockfs.NewFilesystem(&k.vfs)
+ defer socketFilesystem.DecRef()
+ socketMount, err := k.vfs.NewDisconnectedMount(socketFilesystem, nil, &vfs.MountOptions{})
if err != nil {
return fmt.Errorf("failed to initialize socket mount: %v", err)
}
- k.socketMount = sm
+ k.socketMount = socketMount
}
+
return nil
}
@@ -1613,3 +1628,8 @@ func (k *Kernel) EmitUnimplementedEvent(ctx context.Context) {
func (k *Kernel) VFS() *vfs.VirtualFilesystem {
return &k.vfs
}
+
+// PipeMount returns the pipefs mount.
+func (k *Kernel) PipeMount() *vfs.Mount {
+ return k.pipeMount
+}
diff --git a/pkg/sentry/kernel/pipe/vfs.go b/pkg/sentry/kernel/pipe/vfs.go
index a5675bd70..b54f08a30 100644
--- a/pkg/sentry/kernel/pipe/vfs.go
+++ b/pkg/sentry/kernel/pipe/vfs.go
@@ -49,38 +49,42 @@ type VFSPipe struct {
}
// NewVFSPipe returns an initialized VFSPipe.
-func NewVFSPipe(sizeBytes, atomicIOBytes int64) *VFSPipe {
+func NewVFSPipe(isNamed bool, sizeBytes, atomicIOBytes int64) *VFSPipe {
var vp VFSPipe
- initPipe(&vp.pipe, true /* isNamed */, sizeBytes, atomicIOBytes)
+ initPipe(&vp.pipe, isNamed, sizeBytes, atomicIOBytes)
return &vp
}
-// NewVFSPipeFD opens a named pipe. Named pipes have special blocking semantics
-// during open:
+// ReaderWriterPair returns read-only and write-only FDs for vp.
//
-// "Normally, opening the FIFO blocks until the other end is opened also. A
-// process can open a FIFO in nonblocking mode. In this case, opening for
-// read-only will succeed even if no-one has opened on the write side yet,
-// opening for write-only will fail with ENXIO (no such device or address)
-// unless the other end has already been opened. Under Linux, opening a FIFO
-// for read and write will succeed both in blocking and nonblocking mode. POSIX
-// leaves this behavior undefined. This can be used to open a FIFO for writing
-// while there are no readers available." - fifo(7)
-func (vp *VFSPipe) NewVFSPipeFD(ctx context.Context, vfsd *vfs.Dentry, vfsfd *vfs.FileDescription, flags uint32) (*VFSPipeFD, error) {
+// Preconditions: statusFlags should not contain an open access mode.
+func (vp *VFSPipe) ReaderWriterPair(mnt *vfs.Mount, vfsd *vfs.Dentry, statusFlags uint32) (*vfs.FileDescription, *vfs.FileDescription) {
+ return vp.newFD(mnt, vfsd, linux.O_RDONLY|statusFlags), vp.newFD(mnt, vfsd, linux.O_WRONLY|statusFlags)
+}
+
+// Open opens the pipe represented by vp.
+func (vp *VFSPipe) Open(ctx context.Context, mnt *vfs.Mount, vfsd *vfs.Dentry, statusFlags uint32) (*vfs.FileDescription, error) {
vp.mu.Lock()
defer vp.mu.Unlock()
- readable := vfs.MayReadFileWithOpenFlags(flags)
- writable := vfs.MayWriteFileWithOpenFlags(flags)
+ readable := vfs.MayReadFileWithOpenFlags(statusFlags)
+ writable := vfs.MayWriteFileWithOpenFlags(statusFlags)
if !readable && !writable {
return nil, syserror.EINVAL
}
- vfd, err := vp.open(vfsd, vfsfd, flags)
- if err != nil {
- return nil, err
- }
+ fd := vp.newFD(mnt, vfsd, statusFlags)
+ // Named pipes have special blocking semantics during open:
+ //
+ // "Normally, opening the FIFO blocks until the other end is opened also. A
+ // process can open a FIFO in nonblocking mode. In this case, opening for
+ // read-only will succeed even if no-one has opened on the write side yet,
+ // opening for write-only will fail with ENXIO (no such device or address)
+ // unless the other end has already been opened. Under Linux, opening a
+ // FIFO for read and write will succeed both in blocking and nonblocking
+ // mode. POSIX leaves this behavior undefined. This can be used to open a
+ // FIFO for writing while there are no readers available." - fifo(7)
switch {
case readable && writable:
// Pipes opened for read-write always succeed without blocking.
@@ -89,23 +93,26 @@ func (vp *VFSPipe) NewVFSPipeFD(ctx context.Context, vfsd *vfs.Dentry, vfsfd *vf
case readable:
newHandleLocked(&vp.rWakeup)
- // If this pipe is being opened as nonblocking and there's no
+ // If this pipe is being opened as blocking and there's no
// writer, we have to wait for a writer to open the other end.
- if flags&linux.O_NONBLOCK == 0 && !vp.pipe.HasWriters() && !waitFor(&vp.mu, &vp.wWakeup, ctx) {
+ if vp.pipe.isNamed && statusFlags&linux.O_NONBLOCK == 0 && !vp.pipe.HasWriters() && !waitFor(&vp.mu, &vp.wWakeup, ctx) {
+ fd.DecRef()
return nil, syserror.EINTR
}
case writable:
newHandleLocked(&vp.wWakeup)
- if !vp.pipe.HasReaders() {
- // Nonblocking, write-only opens fail with ENXIO when
- // the read side isn't open yet.
- if flags&linux.O_NONBLOCK != 0 {
+ if vp.pipe.isNamed && !vp.pipe.HasReaders() {
+ // Non-blocking, write-only opens fail with ENXIO when the read
+ // side isn't open yet.
+ if statusFlags&linux.O_NONBLOCK != 0 {
+ fd.DecRef()
return nil, syserror.ENXIO
}
// Wait for a reader to open the other end.
if !waitFor(&vp.mu, &vp.rWakeup, ctx) {
+ fd.DecRef()
return nil, syserror.EINTR
}
}
@@ -114,96 +121,93 @@ func (vp *VFSPipe) NewVFSPipeFD(ctx context.Context, vfsd *vfs.Dentry, vfsfd *vf
panic("invalid pipe flags: must be readable, writable, or both")
}
- return vfd, nil
+ return fd, nil
}
// Preconditions: vp.mu must be held.
-func (vp *VFSPipe) open(vfsd *vfs.Dentry, vfsfd *vfs.FileDescription, flags uint32) (*VFSPipeFD, error) {
- var fd VFSPipeFD
- fd.flags = flags
- fd.readable = vfs.MayReadFileWithOpenFlags(flags)
- fd.writable = vfs.MayWriteFileWithOpenFlags(flags)
- fd.vfsfd = vfsfd
- fd.pipe = &vp.pipe
+func (vp *VFSPipe) newFD(mnt *vfs.Mount, vfsd *vfs.Dentry, statusFlags uint32) *vfs.FileDescription {
+ fd := &VFSPipeFD{
+ pipe: &vp.pipe,
+ }
+ fd.vfsfd.Init(fd, statusFlags, mnt, vfsd, &vfs.FileDescriptionOptions{
+ DenyPRead: true,
+ DenyPWrite: true,
+ UseDentryMetadata: true,
+ })
switch {
- case fd.readable && fd.writable:
+ case fd.vfsfd.IsReadable() && fd.vfsfd.IsWritable():
vp.pipe.rOpen()
vp.pipe.wOpen()
- case fd.readable:
+ case fd.vfsfd.IsReadable():
vp.pipe.rOpen()
- case fd.writable:
+ case fd.vfsfd.IsWritable():
vp.pipe.wOpen()
default:
panic("invalid pipe flags: must be readable, writable, or both")
}
- return &fd, nil
+ return &fd.vfsfd
}
-// VFSPipeFD implements a subset of vfs.FileDescriptionImpl for pipes. It is
-// expected that filesystesm will use this in a struct implementing
-// vfs.FileDescriptionImpl.
+// VFSPipeFD implements vfs.FileDescriptionImpl for pipes.
type VFSPipeFD struct {
- pipe *Pipe
- flags uint32
- readable bool
- writable bool
- vfsfd *vfs.FileDescription
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+ vfs.DentryMetadataFileDescriptionImpl
+
+ pipe *Pipe
}
// Release implements vfs.FileDescriptionImpl.Release.
func (fd *VFSPipeFD) Release() {
var event waiter.EventMask
- if fd.readable {
+ if fd.vfsfd.IsReadable() {
fd.pipe.rClose()
- event |= waiter.EventIn
+ event |= waiter.EventOut
}
- if fd.writable {
+ if fd.vfsfd.IsWritable() {
fd.pipe.wClose()
- event |= waiter.EventOut
+ event |= waiter.EventIn | waiter.EventHUp
}
if event == 0 {
panic("invalid pipe flags: must be readable, writable, or both")
}
- if fd.writable {
- fd.vfsfd.VirtualDentry().Mount().EndWrite()
- }
-
fd.pipe.Notify(event)
}
-// OnClose implements vfs.FileDescriptionImpl.OnClose.
-func (fd *VFSPipeFD) OnClose(_ context.Context) error {
- return nil
+// Readiness implements waiter.Waitable.Readiness.
+func (fd *VFSPipeFD) Readiness(mask waiter.EventMask) waiter.EventMask {
+ switch {
+ case fd.vfsfd.IsReadable() && fd.vfsfd.IsWritable():
+ return fd.pipe.rwReadiness()
+ case fd.vfsfd.IsReadable():
+ return fd.pipe.rReadiness()
+ case fd.vfsfd.IsWritable():
+ return fd.pipe.wReadiness()
+ default:
+ panic("pipe FD is neither readable nor writable")
+ }
}
-// PRead implements vfs.FileDescriptionImpl.PRead.
-func (fd *VFSPipeFD) PRead(_ context.Context, _ usermem.IOSequence, _ int64, _ vfs.ReadOptions) (int64, error) {
- return 0, syserror.ESPIPE
+// EventRegister implements waiter.Waitable.EventRegister.
+func (fd *VFSPipeFD) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ fd.pipe.EventRegister(e, mask)
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (fd *VFSPipeFD) EventUnregister(e *waiter.Entry) {
+ fd.pipe.EventUnregister(e)
}
// Read implements vfs.FileDescriptionImpl.Read.
func (fd *VFSPipeFD) Read(ctx context.Context, dst usermem.IOSequence, _ vfs.ReadOptions) (int64, error) {
- if !fd.readable {
- return 0, syserror.EINVAL
- }
-
return fd.pipe.Read(ctx, dst)
}
-// PWrite implements vfs.FileDescriptionImpl.PWrite.
-func (fd *VFSPipeFD) PWrite(_ context.Context, _ usermem.IOSequence, _ int64, _ vfs.WriteOptions) (int64, error) {
- return 0, syserror.ESPIPE
-}
-
// Write implements vfs.FileDescriptionImpl.Write.
func (fd *VFSPipeFD) Write(ctx context.Context, src usermem.IOSequence, _ vfs.WriteOptions) (int64, error) {
- if !fd.writable {
- return 0, syserror.EINVAL
- }
-
return fd.pipe.Write(ctx, src)
}
@@ -211,3 +215,17 @@ func (fd *VFSPipeFD) Write(ctx context.Context, src usermem.IOSequence, _ vfs.Wr
func (fd *VFSPipeFD) Ioctl(ctx context.Context, uio usermem.IO, args arch.SyscallArguments) (uintptr, error) {
return fd.pipe.Ioctl(ctx, uio, args)
}
+
+// PipeSize implements fcntl(F_GETPIPE_SZ).
+func (fd *VFSPipeFD) PipeSize() int64 {
+ // Inline Pipe.FifoSize() rather than calling it with nil Context and
+ // fs.File and ignoring the returned error (which is always nil).
+ fd.pipe.mu.Lock()
+ defer fd.pipe.mu.Unlock()
+ return fd.pipe.max
+}
+
+// SetPipeSize implements fcntl(F_SETPIPE_SZ).
+func (fd *VFSPipeFD) SetPipeSize(size int64) (int64, error) {
+ return fd.pipe.SetFifoSize(size)
+}
diff --git a/pkg/sentry/syscalls/linux/sys_pipe.go b/pkg/sentry/syscalls/linux/sys_pipe.go
index 798344042..43c510930 100644
--- a/pkg/sentry/syscalls/linux/sys_pipe.go
+++ b/pkg/sentry/syscalls/linux/sys_pipe.go
@@ -24,6 +24,8 @@ import (
"gvisor.dev/gvisor/pkg/usermem"
)
+// LINT.IfChange
+
// pipe2 implements the actual system call with flags.
func pipe2(t *kernel.Task, addr usermem.Addr, flags uint) (uintptr, error) {
if flags&^(linux.O_NONBLOCK|linux.O_CLOEXEC) != 0 {
@@ -45,10 +47,12 @@ func pipe2(t *kernel.Task, addr usermem.Addr, flags uint) (uintptr, error) {
}
if _, err := t.CopyOut(addr, fds); err != nil {
- // The files are not closed in this case, the exact semantics
- // of this error case are not well defined, but they could have
- // already been observed by user space.
- return 0, syserror.EFAULT
+ for _, fd := range fds {
+ if file, _ := t.FDTable().Remove(fd); file != nil {
+ file.DecRef()
+ }
+ }
+ return 0, err
}
return 0, nil
}
@@ -69,3 +73,5 @@ func Pipe2(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall
n, err := pipe2(t, addr, flags)
return n, nil, err
}
+
+// LINT.ThenChange(vfs2/pipe.go)
diff --git a/pkg/sentry/syscalls/linux/vfs2/BUILD b/pkg/sentry/syscalls/linux/vfs2/BUILD
index b32abfe59..6ff2d84d2 100644
--- a/pkg/sentry/syscalls/linux/vfs2/BUILD
+++ b/pkg/sentry/syscalls/linux/vfs2/BUILD
@@ -18,6 +18,7 @@ go_library(
"linux64_override_arm64.go",
"mmap.go",
"path.go",
+ "pipe.go",
"poll.go",
"read_write.go",
"setstat.go",
@@ -39,8 +40,10 @@ go_library(
"//pkg/gohacks",
"//pkg/sentry/arch",
"//pkg/sentry/fsbridge",
+ "//pkg/sentry/fsimpl/pipefs",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/pipe",
"//pkg/sentry/kernel/time",
"//pkg/sentry/limits",
"//pkg/sentry/loader",
diff --git a/pkg/sentry/syscalls/linux/vfs2/fd.go b/pkg/sentry/syscalls/linux/vfs2/fd.go
index 3afcea665..8181d80f4 100644
--- a/pkg/sentry/syscalls/linux/vfs2/fd.go
+++ b/pkg/sentry/syscalls/linux/vfs2/fd.go
@@ -18,6 +18,7 @@ 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"
slinux "gvisor.dev/gvisor/pkg/sentry/syscalls/linux"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -140,6 +141,22 @@ func Fcntl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.Syscall
return uintptr(file.StatusFlags()), nil, nil
case linux.F_SETFL:
return 0, nil, file.SetStatusFlags(t, t.Credentials(), args[2].Uint())
+ case linux.F_SETPIPE_SZ:
+ pipefile, ok := file.Impl().(*pipe.VFSPipeFD)
+ if !ok {
+ return 0, nil, syserror.EBADF
+ }
+ n, err := pipefile.SetPipeSize(int64(args[2].Int()))
+ if err != nil {
+ return 0, nil, err
+ }
+ return uintptr(n), nil, nil
+ case linux.F_GETPIPE_SZ:
+ pipefile, ok := file.Impl().(*pipe.VFSPipeFD)
+ if !ok {
+ return 0, nil, syserror.EBADF
+ }
+ return uintptr(pipefile.PipeSize()), nil, nil
default:
// TODO(gvisor.dev/issue/1623): Everything else is not yet supported.
return 0, nil, syserror.EINVAL
diff --git a/pkg/sentry/syscalls/linux/vfs2/linux64_override_amd64.go b/pkg/sentry/syscalls/linux/vfs2/linux64_override_amd64.go
index 645e0bcb8..21eb98444 100644
--- a/pkg/sentry/syscalls/linux/vfs2/linux64_override_amd64.go
+++ b/pkg/sentry/syscalls/linux/vfs2/linux64_override_amd64.go
@@ -39,7 +39,7 @@ func Override(table map[uintptr]kernel.Syscall) {
table[19] = syscalls.Supported("readv", Readv)
table[20] = syscalls.Supported("writev", Writev)
table[21] = syscalls.Supported("access", Access)
- delete(table, 22) // pipe
+ table[22] = syscalls.Supported("pipe", Pipe)
table[23] = syscalls.Supported("select", Select)
table[32] = syscalls.Supported("dup", Dup)
table[33] = syscalls.Supported("dup2", Dup2)
@@ -151,7 +151,7 @@ func Override(table map[uintptr]kernel.Syscall) {
delete(table, 290) // eventfd2
table[291] = syscalls.Supported("epoll_create1", EpollCreate1)
table[292] = syscalls.Supported("dup3", Dup3)
- delete(table, 293) // pipe2
+ table[293] = syscalls.Supported("pipe2", Pipe2)
delete(table, 294) // inotify_init1
table[295] = syscalls.Supported("preadv", Preadv)
table[296] = syscalls.Supported("pwritev", Pwritev)
diff --git a/pkg/sentry/syscalls/linux/vfs2/pipe.go b/pkg/sentry/syscalls/linux/vfs2/pipe.go
new file mode 100644
index 000000000..4a01e4209
--- /dev/null
+++ b/pkg/sentry/syscalls/linux/vfs2/pipe.go
@@ -0,0 +1,63 @@
+// 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/fsimpl/pipefs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+)
+
+// Pipe implements Linux syscall pipe(2).
+func Pipe(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
+ addr := args[0].Pointer()
+ return 0, nil, pipe2(t, addr, 0)
+}
+
+// Pipe2 implements Linux syscall pipe2(2).
+func Pipe2(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
+ addr := args[0].Pointer()
+ flags := args[1].Int()
+ return 0, nil, pipe2(t, addr, flags)
+}
+
+func pipe2(t *kernel.Task, addr usermem.Addr, flags int32) error {
+ if flags&^(linux.O_NONBLOCK|linux.O_CLOEXEC) != 0 {
+ return syserror.EINVAL
+ }
+ r, w := pipefs.NewConnectedPipeFDs(t, t.Kernel().PipeMount(), uint32(flags&linux.O_NONBLOCK))
+ defer r.DecRef()
+ defer w.DecRef()
+
+ fds, err := t.NewFDsVFS2(0, []*vfs.FileDescription{r, w}, kernel.FDFlags{
+ CloseOnExec: flags&linux.O_CLOEXEC != 0,
+ })
+ if err != nil {
+ return err
+ }
+ if _, err := t.CopyOut(addr, fds); err != nil {
+ for _, fd := range fds {
+ if _, file := t.FDTable().Remove(fd); file != nil {
+ file.DecRef()
+ }
+ }
+ return err
+ }
+ return nil
+}
diff --git a/pkg/sentry/syscalls/linux/vfs2/read_write.go b/pkg/sentry/syscalls/linux/vfs2/read_write.go
index 898b190fd..6c6998f45 100644
--- a/pkg/sentry/syscalls/linux/vfs2/read_write.go
+++ b/pkg/sentry/syscalls/linux/vfs2/read_write.go
@@ -103,7 +103,7 @@ func read(t *kernel.Task, file *vfs.FileDescription, dst usermem.IOSequence, opt
// Issue the request and break out if it completes with anything other than
// "would block".
- n, err := file.Read(t, dst, opts)
+ n, err = file.Read(t, dst, opts)
total += n
if err != syserror.ErrWouldBlock {
break
@@ -248,7 +248,7 @@ func pread(t *kernel.Task, file *vfs.FileDescription, dst usermem.IOSequence, of
// Issue the request and break out if it completes with anything other than
// "would block".
- n, err := file.PRead(t, dst, offset+total, opts)
+ n, err = file.PRead(t, dst, offset+total, opts)
total += n
if err != syserror.ErrWouldBlock {
break
@@ -335,7 +335,7 @@ func write(t *kernel.Task, file *vfs.FileDescription, src usermem.IOSequence, op
// Issue the request and break out if it completes with anything other than
// "would block".
- n, err := file.Write(t, src, opts)
+ n, err = file.Write(t, src, opts)
total += n
if err != syserror.ErrWouldBlock {
break
@@ -480,7 +480,7 @@ func pwrite(t *kernel.Task, file *vfs.FileDescription, src usermem.IOSequence, o
// Issue the request and break out if it completes with anything other than
// "would block".
- n, err := file.PWrite(t, src, offset+total, opts)
+ n, err = file.PWrite(t, src, offset+total, opts)
total += n
if err != syserror.ErrWouldBlock {
break
diff --git a/pkg/sentry/vfs/vfs.go b/pkg/sentry/vfs/vfs.go
index 053c6e1d1..cb5bbd781 100644
--- a/pkg/sentry/vfs/vfs.go
+++ b/pkg/sentry/vfs/vfs.go
@@ -335,7 +335,7 @@ func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentia
rp := vfs.getResolvingPath(creds, pop)
for {
err := rp.mount.fs.impl.MknodAt(ctx, rp, *opts)
- if err != nil {
+ if err == nil {
vfs.putResolvingPath(rp)
return nil
}
diff --git a/test/syscalls/linux/pipe.cc b/test/syscalls/linux/pipe.cc
index d8e19e910..67228b66b 100644
--- a/test/syscalls/linux/pipe.cc
+++ b/test/syscalls/linux/pipe.cc
@@ -265,6 +265,8 @@ TEST_P(PipeTest, OffsetCalls) {
SyscallFailsWithErrno(ESPIPE));
struct iovec iov;
+ iov.iov_base = &buf;
+ iov.iov_len = sizeof(buf);
EXPECT_THAT(preadv(wfd_.get(), &iov, 1, 0), SyscallFailsWithErrno(ESPIPE));
EXPECT_THAT(pwritev(rfd_.get(), &iov, 1, 0), SyscallFailsWithErrno(ESPIPE));
}