Support pipes and sockets in VFS2 gofer fs.

Named pipes and sockets can be represented in two ways in gofer fs:
1. As a file on the remote filesystem. In this case, all file operations are
   passed through 9p.
2. As a synthetic file that is internal to the sandbox. In this case, the
   dentry stores an endpoint or VFSPipe for sockets and pipes respectively,
   which replaces interactions with the remote fs through the gofer.
In gofer.filesystem.MknodAt, we attempt to call mknod(2) through 9p,
and if it fails, fall back to the synthetic version.

Updates #1200.

PiperOrigin-RevId: 308828161
This commit is contained in:
Dean Deng 2020-04-28 08:32:20 -07:00 committed by gVisor bot
parent 316394ee89
commit f3ca5ca82a
14 changed files with 359 additions and 77 deletions

View File

@ -47,6 +47,8 @@ func (i *inodeOperations) BoundEndpoint(inode *fs.Inode, path string) transport.
return &endpoint{inode, i.fileState.file.file, path}
}
// LINT.IfChange
// endpoint is a Gofer-backed transport.BoundEndpoint.
//
// An endpoint's lifetime is the time between when InodeOperations.BoundEndpoint()
@ -146,3 +148,5 @@ func (e *endpoint) Release() {
func (e *endpoint) Passcred() bool {
return false
}
// LINT.ThenChange(../../fsimpl/gofer/socket.go)

View File

@ -485,7 +485,7 @@ func (fs *filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath) (transport.BoundEndpoint, error) {
func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
_, _, err := fs.walk(rp, false)
if err != nil {
return nil, err

View File

@ -38,6 +38,7 @@ go_library(
"p9file.go",
"pagemath.go",
"regular_file.go",
"socket.go",
"special_file.go",
"symlink.go",
"time.go",
@ -52,18 +53,25 @@ go_library(
"//pkg/p9",
"//pkg/safemem",
"//pkg/sentry/fs/fsutil",
"//pkg/sentry/fsimpl/host",
"//pkg/sentry/hostfd",
"//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/kernel/pipe",
"//pkg/sentry/kernel/time",
"//pkg/sentry/memmap",
"//pkg/sentry/pgalloc",
"//pkg/sentry/platform",
"//pkg/sentry/socket/control",
"//pkg/sentry/socket/unix",
"//pkg/sentry/socket/unix/transport",
"//pkg/sentry/usage",
"//pkg/sentry/vfs",
"//pkg/syserr",
"//pkg/syserror",
"//pkg/unet",
"//pkg/usermem",
"//pkg/waiter",
],
)

View File

@ -15,6 +15,7 @@
package gofer
import (
"fmt"
"sync"
"sync/atomic"
@ -22,6 +23,8 @@ import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
@ -59,28 +62,52 @@ func (d *dentry) cacheNegativeLookupLocked(name string) {
d.children[name] = nil
}
// createSyntheticDirectory creates a synthetic directory with the given name
type createSyntheticOpts struct {
name string
mode linux.FileMode
kuid auth.KUID
kgid auth.KGID
// The endpoint for a synthetic socket. endpoint should be nil if the file
// being created is not a socket.
endpoint transport.BoundEndpoint
// pipe should be nil if the file being created is not a pipe.
pipe *pipe.VFSPipe
}
// createSyntheticChildLocked creates a synthetic file with the given name
// in d.
//
// Preconditions: d.dirMu must be locked. d.isDir(). d does not already contain
// a child with the given name.
func (d *dentry) createSyntheticDirectoryLocked(name string, mode linux.FileMode, kuid auth.KUID, kgid auth.KGID) {
func (d *dentry) createSyntheticChildLocked(opts *createSyntheticOpts) {
d2 := &dentry{
refs: 1, // held by d
fs: d.fs,
mode: uint32(mode) | linux.S_IFDIR,
uid: uint32(kuid),
gid: uint32(kgid),
mode: uint32(opts.mode),
uid: uint32(opts.kuid),
gid: uint32(opts.kgid),
blockSize: usermem.PageSize, // arbitrary
handle: handle{
fd: -1,
},
nlink: uint32(2),
}
switch opts.mode.FileType() {
case linux.S_IFDIR:
// Nothing else needs to be done.
case linux.S_IFSOCK:
d2.endpoint = opts.endpoint
case linux.S_IFIFO:
d2.pipe = opts.pipe
default:
panic(fmt.Sprintf("failed to create synthetic file of unrecognized type: %v", opts.mode.FileType()))
}
d2.pf.dentry = d2
d2.vfsd.Init(d2)
d.cacheNewChildLocked(d2, name)
d.cacheNewChildLocked(d2, opts.name)
d.syntheticChildren++
}

View File

@ -22,9 +22,11 @@ import (
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/pipe"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
)
// Sync implements vfs.FilesystemImpl.Sync.
@ -652,7 +654,12 @@ func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
return err
}
ctx.Infof("Failed to create remote directory %q: %v; falling back to synthetic directory", name, err)
parent.createSyntheticDirectoryLocked(name, opts.Mode, creds.EffectiveKUID, creds.EffectiveKGID)
parent.createSyntheticChildLocked(&createSyntheticOpts{
name: name,
mode: linux.S_IFDIR | opts.Mode,
kuid: creds.EffectiveKUID,
kgid: creds.EffectiveKGID,
})
}
if fs.opts.interop != InteropModeShared {
parent.incLinks()
@ -663,7 +670,12 @@ func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
// Can't create non-synthetic files in synthetic directories.
return syserror.EPERM
}
parent.createSyntheticDirectoryLocked(name, opts.Mode, creds.EffectiveKUID, creds.EffectiveKGID)
parent.createSyntheticChildLocked(&createSyntheticOpts{
name: name,
mode: linux.S_IFDIR | opts.Mode,
kuid: creds.EffectiveKUID,
kgid: creds.EffectiveKGID,
})
parent.incLinks()
return nil
})
@ -674,6 +686,28 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, name string) error {
creds := rp.Credentials()
_, err := parent.file.mknod(ctx, name, (p9.FileMode)(opts.Mode), opts.DevMajor, opts.DevMinor, (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
if err == syserror.EPERM {
switch opts.Mode.FileType() {
case linux.S_IFSOCK:
parent.createSyntheticChildLocked(&createSyntheticOpts{
name: name,
mode: opts.Mode,
kuid: creds.EffectiveKUID,
kgid: creds.EffectiveKGID,
endpoint: opts.Endpoint,
})
return nil
case linux.S_IFIFO:
parent.createSyntheticChildLocked(&createSyntheticOpts{
name: name,
mode: opts.Mode,
kuid: creds.EffectiveKUID,
kgid: creds.EffectiveKGID,
pipe: pipe.NewVFSPipe(true /* isNamed */, pipe.DefaultPipeSize, usermem.PageSize),
})
return nil
}
}
return err
}, nil)
}
@ -756,9 +790,9 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
return nil, err
}
mnt := rp.Mount()
filetype := d.fileType()
switch {
case filetype == linux.S_IFREG && !d.fs.opts.regularFilesUseSpecialFileFD:
switch d.fileType() {
case linux.S_IFREG:
if !d.fs.opts.regularFilesUseSpecialFileFD {
if err := d.ensureSharedHandle(ctx, ats&vfs.MayRead != 0, ats&vfs.MayWrite != 0, opts.Flags&linux.O_TRUNC != 0); err != nil {
return nil, err
}
@ -769,7 +803,8 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
return nil, err
}
return &fd.vfsfd, nil
case filetype == linux.S_IFDIR:
}
case linux.S_IFDIR:
// Can't open directories with O_CREAT.
if opts.Flags&linux.O_CREAT != 0 {
return nil, syserror.EISDIR
@ -791,10 +826,25 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
return nil, err
}
return &fd.vfsfd, nil
case filetype == linux.S_IFLNK:
case linux.S_IFLNK:
// Can't open symlinks without O_PATH (which is unimplemented).
return nil, syserror.ELOOP
default:
case linux.S_IFSOCK:
if d.isSynthetic() {
return nil, syserror.ENXIO
}
case linux.S_IFIFO:
if d.isSynthetic() {
return d.pipe.Open(ctx, mnt, &d.vfsd, opts.Flags)
}
}
return d.openSpecialFileLocked(ctx, mnt, opts)
}
func (d *dentry) openSpecialFileLocked(ctx context.Context, mnt *vfs.Mount, opts *vfs.OpenOptions) (*vfs.FileDescription, error) {
ats := vfs.AccessTypesForOpenFlags(opts)
// Treat as a special file. This is done for non-synthetic pipes as well as
// regular files when d.fs.opts.regularFilesUseSpecialFileFD is true.
if opts.Flags&linux.O_DIRECT != 0 {
return nil, syserror.EINVAL
}
@ -810,7 +860,6 @@ func (d *dentry) openLocked(ctx context.Context, rp *vfs.ResolvingPath, opts *vf
return nil, err
}
return &fd.vfsfd, nil
}
}
// Preconditions: d.fs.renameMu must be locked. d.dirMu must be locked.
@ -1196,15 +1245,28 @@ func (fs *filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath) (transport.BoundEndpoint, error) {
func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
var ds *[]*dentry
fs.renameMu.RLock()
defer fs.renameMuRUnlockAndCheckCaching(&ds)
_, err := fs.resolveLocked(ctx, rp, &ds)
d, err := fs.resolveLocked(ctx, rp, &ds)
if err != nil {
return nil, err
}
// TODO(gvisor.dev/issue/1476): Implement BoundEndpointAt.
if err := d.checkPermissions(rp.Credentials(), vfs.MayWrite); err != nil {
return nil, err
}
if d.isSocket() {
if !d.isSynthetic() {
d.IncRef()
return &endpoint{
dentry: d,
file: d.file.file,
path: opts.Addr,
}, nil
}
return d.endpoint, nil
}
return nil, syserror.ECONNREFUSED
}

View File

@ -46,9 +46,11 @@ import (
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
"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/memmap"
"gvisor.dev/gvisor/pkg/sentry/pgalloc"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/unet"
@ -585,6 +587,14 @@ type dentry struct {
// and target are protected by dataMu.
haveTarget bool
target string
// If this dentry represents a synthetic socket file, endpoint is the
// transport endpoint bound to this file.
endpoint transport.BoundEndpoint
// If this dentry represents a synthetic named pipe, pipe is the pipe
// endpoint bound to this file.
pipe *pipe.VFSPipe
}
// dentryAttrMask returns a p9.AttrMask enabling all attributes used by the
@ -791,11 +801,22 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, stat *lin
setLocalAtime = stat.Mask&linux.STATX_ATIME != 0
setLocalMtime = stat.Mask&linux.STATX_MTIME != 0
stat.Mask &^= linux.STATX_ATIME | linux.STATX_MTIME
if !setLocalMtime && (stat.Mask&linux.STATX_SIZE != 0) {
// Prepare for truncate.
if stat.Mask&linux.STATX_SIZE != 0 {
switch d.mode & linux.S_IFMT {
case linux.S_IFREG:
if !setLocalMtime {
// Truncate updates mtime.
setLocalMtime = true
stat.Mtime.Nsec = linux.UTIME_NOW
}
case linux.S_IFDIR:
return syserror.EISDIR
default:
return syserror.EINVAL
}
}
}
d.metadataMu.Lock()
defer d.metadataMu.Unlock()
@ -1049,6 +1070,7 @@ func (d *dentry) destroyLocked() {
d.handle.close(ctx)
}
d.handleMu.Unlock()
if !d.file.isNil() {
d.file.close(ctx)
d.file = p9file{}
@ -1134,7 +1156,7 @@ func (d *dentry) removexattr(ctx context.Context, creds *auth.Credentials, name
return d.file.removeXattr(ctx, name)
}
// Preconditions: !d.file.isNil(). d.isRegularFile() || d.isDirectory().
// Preconditions: !d.isSynthetic(). d.isRegularFile() || d.isDirectory().
func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool) error {
// O_TRUNC unconditionally requires us to obtain a new handle (opened with
// O_TRUNC).

View File

@ -0,0 +1,141 @@
// 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 gofer
import (
"syscall"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/host"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/syserr"
"gvisor.dev/gvisor/pkg/waiter"
)
func (d *dentry) isSocket() bool {
return d.fileType() == linux.S_IFSOCK
}
// endpoint is a Gofer-backed transport.BoundEndpoint.
//
// An endpoint's lifetime is the time between when filesystem.BoundEndpointAt()
// is called and either BoundEndpoint.BidirectionalConnect or
// BoundEndpoint.UnidirectionalConnect is called.
type endpoint struct {
// dentry is the filesystem dentry which produced this endpoint.
dentry *dentry
// file is the p9 file that contains a single unopened fid.
file p9.File
// path is the sentry path where this endpoint is bound.
path string
}
func sockTypeToP9(t linux.SockType) (p9.ConnectFlags, bool) {
switch t {
case linux.SOCK_STREAM:
return p9.StreamSocket, true
case linux.SOCK_SEQPACKET:
return p9.SeqpacketSocket, true
case linux.SOCK_DGRAM:
return p9.DgramSocket, true
}
return 0, false
}
// BidirectionalConnect implements ConnectableEndpoint.BidirectionalConnect.
func (e *endpoint) BidirectionalConnect(ctx context.Context, ce transport.ConnectingEndpoint, returnConnect func(transport.Receiver, transport.ConnectedEndpoint)) *syserr.Error {
cf, ok := sockTypeToP9(ce.Type())
if !ok {
return syserr.ErrConnectionRefused
}
// No lock ordering required as only the ConnectingEndpoint has a mutex.
ce.Lock()
// Check connecting state.
if ce.Connected() {
ce.Unlock()
return syserr.ErrAlreadyConnected
}
if ce.Listening() {
ce.Unlock()
return syserr.ErrInvalidEndpointState
}
c, err := e.newConnectedEndpoint(ctx, cf, ce.WaiterQueue())
if err != nil {
ce.Unlock()
return err
}
returnConnect(c, c)
ce.Unlock()
c.Init()
return nil
}
// UnidirectionalConnect implements
// transport.BoundEndpoint.UnidirectionalConnect.
func (e *endpoint) UnidirectionalConnect(ctx context.Context) (transport.ConnectedEndpoint, *syserr.Error) {
c, err := e.newConnectedEndpoint(ctx, p9.DgramSocket, &waiter.Queue{})
if err != nil {
return nil, err
}
c.Init()
// We don't need the receiver.
c.CloseRecv()
c.Release()
return c, nil
}
func (e *endpoint) newConnectedEndpoint(ctx context.Context, flags p9.ConnectFlags, queue *waiter.Queue) (*host.SCMConnectedEndpoint, *syserr.Error) {
hostFile, err := e.file.Connect(flags)
if err != nil {
return nil, syserr.ErrConnectionRefused
}
// Dup the fd so that the new endpoint can manage its lifetime.
hostFD, err := syscall.Dup(hostFile.FD())
if err != nil {
log.Warningf("Could not dup host socket fd %d: %v", hostFile.FD(), err)
return nil, syserr.FromError(err)
}
// After duplicating, we no longer need hostFile.
hostFile.Close()
c, serr := host.NewSCMEndpoint(ctx, hostFD, queue, e.path)
if serr != nil {
log.Warningf("Gofer returned invalid host socket for BidirectionalConnect; file %+v flags %+v: %v", e.file, flags, serr)
return nil, serr
}
return c, nil
}
// Release implements transport.BoundEndpoint.Release.
func (e *endpoint) Release() {
e.dentry.DecRef()
}
// Passcred implements transport.BoundEndpoint.Passcred.
func (e *endpoint) Passcred() bool {
return false
}

View File

@ -766,7 +766,7 @@ func (fs *Filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
func (fs *Filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath) (transport.BoundEndpoint, error) {
func (fs *Filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
fs.mu.RLock()
_, _, err := fs.walkExistingLocked(ctx, rp)
fs.mu.RUnlock()

View File

@ -697,7 +697,7 @@ func (fs *filesystem) UnlinkAt(ctx context.Context, rp *vfs.ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath) (transport.BoundEndpoint, error) {
func (fs *filesystem) BoundEndpointAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.BoundEndpointOptions) (transport.BoundEndpoint, error) {
fs.mu.RLock()
defer fs.mu.RUnlock()
d, err := resolveLocked(rp)

View File

@ -373,7 +373,7 @@ func extractEndpoint(t *kernel.Task, sockaddr []byte) (transport.BoundEndpoint,
Path: p,
FollowFinalSymlink: true,
}
ep, e := t.Kernel().VFS().BoundEndpointAt(t, t.Credentials(), &pop)
ep, e := t.Kernel().VFS().BoundEndpointAt(t, t.Credentials(), &pop, &vfs.BoundEndpointOptions{path})
root.DecRef()
if relPath {
start.DecRef()

View File

@ -237,7 +237,7 @@ func (fs *anonFilesystem) UnlinkAt(ctx context.Context, rp *ResolvingPath) error
}
// BoundEndpointAt implements FilesystemImpl.BoundEndpointAt.
func (fs *anonFilesystem) BoundEndpointAt(ctx context.Context, rp *ResolvingPath) (transport.BoundEndpoint, error) {
func (fs *anonFilesystem) BoundEndpointAt(ctx context.Context, rp *ResolvingPath, opts BoundEndpointOptions) (transport.BoundEndpoint, error) {
if !rp.Final() {
return nil, syserror.ENOTDIR
}

View File

@ -495,7 +495,7 @@ type FilesystemImpl interface {
// BoundEndpointAt returns the Unix socket endpoint bound at the path rp.
//
// - If a non-socket file exists at rp, then BoundEndpointAt returns ECONNREFUSED.
BoundEndpointAt(ctx context.Context, rp *ResolvingPath) (transport.BoundEndpoint, error)
BoundEndpointAt(ctx context.Context, rp *ResolvingPath, opts BoundEndpointOptions) (transport.BoundEndpoint, error)
// PrependPath prepends a path from vd to vd.Mount().Root() to b.
//

View File

@ -151,6 +151,24 @@ type SetStatOptions struct {
Stat linux.Statx
}
// BoundEndpointOptions contains options to VirtualFilesystem.BoundEndpointAt()
// and FilesystemImpl.BoundEndpointAt().
type BoundEndpointOptions struct {
// Addr is the path of the file whose socket endpoint is being retrieved.
// It is generally irrelevant: most endpoints are stored at a dentry that
// was created through a bind syscall, so the path can be stored on creation.
// However, if the endpoint was created in FilesystemImpl.BoundEndpointAt(),
// then we may not know what the original bind address was.
//
// For example, if connect(2) is called with address "foo" which corresponds
// a remote named socket in goferfs, we need to generate an endpoint wrapping
// that file. In this case, we can use Addr to set the endpoint address to
// "foo". Note that Addr is only a best-effort attempt--we still do not know
// the exact address that was used on the remote fs to bind the socket (it
// may have been "foo", "./foo", etc.).
Addr string
}
// GetxattrOptions contains options to VirtualFilesystem.GetxattrAt(),
// FilesystemImpl.GetxattrAt(), FileDescription.Getxattr(), and
// FileDescriptionImpl.Getxattr().

View File

@ -351,33 +351,6 @@ func (vfs *VirtualFilesystem) MknodAt(ctx context.Context, creds *auth.Credentia
}
}
// BoundEndpointAt gets the bound endpoint at the given path, if one exists.
func (vfs *VirtualFilesystem) BoundEndpointAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation) (transport.BoundEndpoint, error) {
if !pop.Path.Begin.Ok() {
if pop.Path.Absolute {
return nil, syserror.ECONNREFUSED
}
return nil, syserror.ENOENT
}
rp := vfs.getResolvingPath(creds, pop)
for {
bep, err := rp.mount.fs.impl.BoundEndpointAt(ctx, rp)
if err == nil {
vfs.putResolvingPath(rp)
return bep, nil
}
if checkInvariants {
if rp.canHandleError(err) && rp.Done() {
panic(fmt.Sprintf("%T.BoundEndpointAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
if !rp.handleError(err) {
vfs.putResolvingPath(rp)
return nil, err
}
}
}
// OpenAt returns a FileDescription providing access to the file at the given
// path. A reference is taken on the returned FileDescription.
func (vfs *VirtualFilesystem) OpenAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *OpenOptions) (*FileDescription, error) {
@ -675,6 +648,33 @@ func (vfs *VirtualFilesystem) UnlinkAt(ctx context.Context, creds *auth.Credenti
}
}
// BoundEndpointAt gets the bound endpoint at the given path, if one exists.
func (vfs *VirtualFilesystem) BoundEndpointAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, opts *BoundEndpointOptions) (transport.BoundEndpoint, error) {
if !pop.Path.Begin.Ok() {
if pop.Path.Absolute {
return nil, syserror.ECONNREFUSED
}
return nil, syserror.ENOENT
}
rp := vfs.getResolvingPath(creds, pop)
for {
bep, err := rp.mount.fs.impl.BoundEndpointAt(ctx, rp, *opts)
if err == nil {
vfs.putResolvingPath(rp)
return bep, nil
}
if checkInvariants {
if rp.canHandleError(err) && rp.Done() {
panic(fmt.Sprintf("%T.BoundEndpointAt() consumed all path components and returned %v", rp.mount.fs.impl, err))
}
}
if !rp.handleError(err) {
vfs.putResolvingPath(rp)
return nil, err
}
}
}
// ListxattrAt returns all extended attribute names for the file at the given
// path.
func (vfs *VirtualFilesystem) ListxattrAt(ctx context.Context, creds *auth.Credentials, pop *PathOperation, size uint64) ([]string, error) {