diff options
Diffstat (limited to 'pkg/sentry/fsimpl')
-rw-r--r-- | pkg/sentry/fsimpl/fuse/BUILD | 41 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/fuse/connection.go | 255 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/fuse/dev.go | 289 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/fuse/dev_test.go | 429 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/fuse/fusefs.go | 52 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/fuse/register.go | 42 |
6 files changed, 1079 insertions, 29 deletions
diff --git a/pkg/sentry/fsimpl/fuse/BUILD b/pkg/sentry/fsimpl/fuse/BUILD index 737007748..67649e811 100644 --- a/pkg/sentry/fsimpl/fuse/BUILD +++ b/pkg/sentry/fsimpl/fuse/BUILD @@ -1,12 +1,28 @@ -load("//tools:defs.bzl", "go_library") +load("//tools:defs.bzl", "go_library", "go_test") +load("//tools/go_generics:defs.bzl", "go_template_instance") licenses(["notice"]) +go_template_instance( + name = "request_list", + out = "request_list.go", + package = "fuse", + prefix = "request", + template = "//pkg/ilist:generic_list", + types = { + "Element": "*Request", + "Linker": "*Request", + }, +) + go_library( name = "fuse", srcs = [ + "connection.go", "dev.go", "fusefs.go", + "register.go", + "request_list.go", ], visibility = ["//pkg/sentry:internal"], deps = [ @@ -18,7 +34,30 @@ go_library( "//pkg/sentry/kernel", "//pkg/sentry/kernel/auth", "//pkg/sentry/vfs", + "//pkg/sync", + "//pkg/syserror", + "//pkg/usermem", + "//pkg/waiter", + "//tools/go_marshal/marshal", + "@org_golang_x_sys//unix:go_default_library", + ], +) + +go_test( + name = "dev_test", + size = "small", + srcs = ["dev_test.go"], + library = ":fuse", + deps = [ + "//pkg/abi/linux", + "//pkg/sentry/fsimpl/testutil", + "//pkg/sentry/fsimpl/tmpfs", + "//pkg/sentry/kernel", + "//pkg/sentry/kernel/auth", + "//pkg/sentry/vfs", "//pkg/syserror", "//pkg/usermem", + "//pkg/waiter", + "//tools/go_marshal/marshal", ], ) diff --git a/pkg/sentry/fsimpl/fuse/connection.go b/pkg/sentry/fsimpl/fuse/connection.go new file mode 100644 index 000000000..f330da0bd --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/connection.go @@ -0,0 +1,255 @@ +// 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 fuse + +import ( + "errors" + "fmt" + "syscall" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/log" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/waiter" + "gvisor.dev/gvisor/tools/go_marshal/marshal" +) + +// MaxActiveRequestsDefault is the default setting controlling the upper bound +// on the number of active requests at any given time. +const MaxActiveRequestsDefault = 10000 + +var ( + // Ordinary requests have even IDs, while interrupts IDs are odd. + InitReqBit uint64 = 1 + ReqIDStep uint64 = 2 +) + +// Request represents a FUSE operation request that hasn't been sent to the +// server yet. +// +// +stateify savable +type Request struct { + requestEntry + + id linux.FUSEOpID + hdr *linux.FUSEHeaderIn + data []byte +} + +// Response represents an actual response from the server, including the +// response payload. +// +// +stateify savable +type Response struct { + opcode linux.FUSEOpcode + hdr linux.FUSEHeaderOut + data []byte +} + +// Connection is the struct by which the sentry communicates with the FUSE server daemon. +type Connection struct { + fd *DeviceFD + + // MaxWrite is the daemon's maximum size of a write buffer. + // This is negotiated during FUSE_INIT. + MaxWrite uint32 +} + +// NewFUSEConnection creates a FUSE connection to fd +func NewFUSEConnection(_ context.Context, fd *vfs.FileDescription, maxInFlightRequests uint64) (*Connection, error) { + // Mark the device as ready so it can be used. /dev/fuse can only be used if the FD was used to + // mount a FUSE filesystem. + fuseFD := fd.Impl().(*DeviceFD) + fuseFD.mounted = true + + // Create the writeBuf for the header to be stored in. + hdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) + fuseFD.writeBuf = make([]byte, hdrLen) + fuseFD.completions = make(map[linux.FUSEOpID]*futureResponse) + fuseFD.fullQueueCh = make(chan struct{}, maxInFlightRequests) + fuseFD.writeCursor = 0 + + return &Connection{ + fd: fuseFD, + }, nil +} + +// NewRequest creates a new request that can be sent to the FUSE server. +func (conn *Connection) NewRequest(creds *auth.Credentials, pid uint32, ino uint64, opcode linux.FUSEOpcode, payload marshal.Marshallable) (*Request, error) { + conn.fd.mu.Lock() + defer conn.fd.mu.Unlock() + conn.fd.nextOpID += linux.FUSEOpID(ReqIDStep) + + hdrLen := (*linux.FUSEHeaderIn)(nil).SizeBytes() + hdr := linux.FUSEHeaderIn{ + Len: uint32(hdrLen + payload.SizeBytes()), + Opcode: opcode, + Unique: conn.fd.nextOpID, + NodeID: ino, + UID: uint32(creds.EffectiveKUID), + GID: uint32(creds.EffectiveKGID), + PID: pid, + } + + buf := make([]byte, hdr.Len) + hdr.MarshalUnsafe(buf[:hdrLen]) + payload.MarshalUnsafe(buf[hdrLen:]) + + return &Request{ + id: hdr.Unique, + hdr: &hdr, + data: buf, + }, nil +} + +// Call makes a request to the server and blocks the invoking task until a +// server responds with a response. +// NOTE: If no task is provided then the Call will simply enqueue the request +// and return a nil response. No blocking will happen in this case. Instead, +// this is used to signify that the processing of this request will happen by +// the kernel.Task that writes the response. See FUSE_INIT for such an +// invocation. +func (conn *Connection) Call(t *kernel.Task, r *Request) (*Response, error) { + fut, err := conn.callFuture(t, r) + if err != nil { + return nil, err + } + + return fut.resolve(t) +} + +// Error returns the error of the FUSE call. +func (r *Response) Error() error { + errno := r.hdr.Error + if errno >= 0 { + return nil + } + + sysErrNo := syscall.Errno(-errno) + return error(sysErrNo) +} + +// UnmarshalPayload unmarshals the response data into m. +func (r *Response) UnmarshalPayload(m marshal.Marshallable) error { + hdrLen := r.hdr.SizeBytes() + haveDataLen := r.hdr.Len - uint32(hdrLen) + wantDataLen := uint32(m.SizeBytes()) + + if haveDataLen < wantDataLen { + return fmt.Errorf("payload too small. Minimum data lenth required: %d, but got data length %d", wantDataLen, haveDataLen) + } + + m.UnmarshalUnsafe(r.data[hdrLen:]) + return nil +} + +// callFuture makes a request to the server and returns a future response. +// Call resolve() when the response needs to be fulfilled. +func (conn *Connection) callFuture(t *kernel.Task, r *Request) (*futureResponse, error) { + conn.fd.mu.Lock() + defer conn.fd.mu.Unlock() + + // Is the queue full? + // + // We must busy wait here until the request can be queued. We don't + // block on the fd.fullQueueCh with a lock - so after being signalled, + // before we acquire the lock, it is possible that a barging task enters + // and queues a request. As a result, upon acquiring the lock we must + // again check if the room is available. + // + // This can potentially starve a request forever but this can only happen + // if there are always too many ongoing requests all the time. The + // supported maxActiveRequests setting should be really high to avoid this. + for conn.fd.numActiveRequests == conn.fd.fs.opts.maxActiveRequests { + if t == nil { + // Since there is no task that is waiting. We must error out. + return nil, errors.New("FUSE request queue full") + } + + log.Infof("Blocking request %v from being queued. Too many active requests: %v", + r.id, conn.fd.numActiveRequests) + conn.fd.mu.Unlock() + err := t.Block(conn.fd.fullQueueCh) + conn.fd.mu.Lock() + if err != nil { + return nil, err + } + } + + return conn.callFutureLocked(t, r) +} + +// callFutureLocked makes a request to the server and returns a future response. +func (conn *Connection) callFutureLocked(t *kernel.Task, r *Request) (*futureResponse, error) { + conn.fd.queue.PushBack(r) + conn.fd.numActiveRequests += 1 + fut := newFutureResponse(r.hdr.Opcode) + conn.fd.completions[r.id] = fut + + // Signal the readers that there is something to read. + conn.fd.waitQueue.Notify(waiter.EventIn) + + return fut, nil +} + +// futureResponse represents an in-flight request, that may or may not have +// completed yet. Convert it to a resolved Response by calling Resolve, but note +// that this may block. +// +// +stateify savable +type futureResponse struct { + opcode linux.FUSEOpcode + ch chan struct{} + hdr *linux.FUSEHeaderOut + data []byte +} + +// newFutureResponse creates a future response to a FUSE request. +func newFutureResponse(opcode linux.FUSEOpcode) *futureResponse { + return &futureResponse{ + opcode: opcode, + ch: make(chan struct{}), + } +} + +// resolve blocks the task until the server responds to its corresponding request, +// then returns a resolved response. +func (f *futureResponse) resolve(t *kernel.Task) (*Response, error) { + // If there is no Task associated with this request - then we don't try to resolve + // the response. Instead, the task writing the response (proxy to the server) will + // process the response on our behalf. + if t == nil { + log.Infof("fuse.Response.resolve: Not waiting on a response from server.") + return nil, nil + } + + if err := t.Block(f.ch); err != nil { + return nil, err + } + + return f.getResponse(), nil +} + +// getResponse creates a Response from the data the futureResponse has. +func (f *futureResponse) getResponse() *Response { + return &Response{ + opcode: f.opcode, + hdr: *f.hdr, + data: f.data, + } +} diff --git a/pkg/sentry/fsimpl/fuse/dev.go b/pkg/sentry/fsimpl/fuse/dev.go index c9e12a94f..f3443ac71 100644 --- a/pkg/sentry/fsimpl/fuse/dev.go +++ b/pkg/sentry/fsimpl/fuse/dev.go @@ -15,13 +15,17 @@ package fuse import ( + "syscall" + "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" - "gvisor.dev/gvisor/pkg/sentry/fsimpl/devtmpfs" + "gvisor.dev/gvisor/pkg/log" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/usermem" + "gvisor.dev/gvisor/pkg/waiter" ) const fuseDevMinor = 229 @@ -54,9 +58,43 @@ type DeviceFD struct { // mounted specifies whether a FUSE filesystem was mounted using the DeviceFD. mounted bool - // TODO(gvisor.dev/issue/2987): Add all the data structures needed to enqueue - // and deque requests, control synchronization and establish communication - // between the FUSE kernel module and the /dev/fuse character device. + // nextOpID is used to create new requests. + nextOpID linux.FUSEOpID + + // queue is the list of requests that need to be processed by the FUSE server. + queue requestList + + // numActiveRequests is the number of requests made by the Sentry that has + // yet to be responded to. + numActiveRequests uint64 + + // completions is used to map a request to its response. A Writer will use this + // to notify the caller of a completed response. + completions map[linux.FUSEOpID]*futureResponse + + writeCursor uint32 + + // writeBuf is the memory buffer used to copy in the FUSE out header from + // userspace. + writeBuf []byte + + // writeCursorFR current FR being copied from server. + writeCursorFR *futureResponse + + // mu protects all the queues, maps, buffers and cursors and nextOpID. + mu sync.Mutex + + // waitQueue is used to notify interested parties when the device becomes + // readable or writable. + waitQueue waiter.Queue + + // fullQueueCh is a channel used to synchronize the readers with the writers. + // Writers (inbound requests to the filesystem) block if there are too many + // unprocessed in-flight requests. + fullQueueCh chan struct{} + + // fs is the FUSE filesystem that this FD is being used for. + fs *filesystem } // Release implements vfs.FileDescriptionImpl.Release. @@ -79,7 +117,75 @@ func (fd *DeviceFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.R return 0, syserror.EPERM } - return 0, syserror.ENOSYS + // We require that any Read done on this filesystem have a sane minimum + // read buffer. It must have the capacity for the fixed parts of any request + // header (Linux uses the request header and the FUSEWriteIn header for this + // calculation) + the negotiated MaxWrite room for the data. + minBuffSize := linux.FUSE_MIN_READ_BUFFER + inHdrLen := uint32((*linux.FUSEHeaderIn)(nil).SizeBytes()) + writeHdrLen := uint32((*linux.FUSEWriteIn)(nil).SizeBytes()) + negotiatedMinBuffSize := inHdrLen + writeHdrLen + fd.fs.conn.MaxWrite + if minBuffSize < negotiatedMinBuffSize { + minBuffSize = negotiatedMinBuffSize + } + + // If the read buffer is too small, error out. + if dst.NumBytes() < int64(minBuffSize) { + return 0, syserror.EINVAL + } + + fd.mu.Lock() + defer fd.mu.Unlock() + return fd.readLocked(ctx, dst, opts) +} + +// readLocked implements the reading of the fuse device while locked with DeviceFD.mu. +func (fd *DeviceFD) readLocked(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) { + if fd.queue.Empty() { + return 0, syserror.ErrWouldBlock + } + + var readCursor uint32 + var bytesRead int64 + for { + req := fd.queue.Front() + if dst.NumBytes() < int64(req.hdr.Len) { + // The request is too large. Cannot process it. All requests must be smaller than the + // negotiated size as specified by Connection.MaxWrite set as part of the FUSE_INIT + // handshake. + errno := -int32(syscall.EIO) + if req.hdr.Opcode == linux.FUSE_SETXATTR { + errno = -int32(syscall.E2BIG) + } + + // Return the error to the calling task. + if err := fd.sendError(ctx, errno, req); err != nil { + return 0, err + } + + // We're done with this request. + fd.queue.Remove(req) + + // Restart the read as this request was invalid. + log.Warningf("fuse.DeviceFD.Read: request found was too large. Restarting read.") + return fd.readLocked(ctx, dst, opts) + } + + n, err := dst.CopyOut(ctx, req.data[readCursor:]) + if err != nil { + return 0, err + } + readCursor += uint32(n) + bytesRead += int64(n) + + if readCursor >= req.hdr.Len { + // Fully done with this req, remove it from the queue. + fd.queue.Remove(req) + break + } + } + + return bytesRead, nil } // PWrite implements vfs.FileDescriptionImpl.PWrite. @@ -94,12 +200,128 @@ func (fd *DeviceFD) PWrite(ctx context.Context, src usermem.IOSequence, offset i // Write implements vfs.FileDescriptionImpl.Write. func (fd *DeviceFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { + fd.mu.Lock() + defer fd.mu.Unlock() + return fd.writeLocked(ctx, src, opts) +} + +// writeLocked implements writing to the fuse device while locked with DeviceFD.mu. +func (fd *DeviceFD) writeLocked(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) { // Operations on /dev/fuse don't make sense until a FUSE filesystem is mounted. if !fd.mounted { return 0, syserror.EPERM } - return 0, syserror.ENOSYS + var cn, n int64 + hdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) + + for src.NumBytes() > 0 { + if fd.writeCursorFR != nil { + // Already have common header, and we're now copying the payload. + wantBytes := fd.writeCursorFR.hdr.Len + + // Note that the FR data doesn't have the header. Copy it over if its necessary. + if fd.writeCursorFR.data == nil { + fd.writeCursorFR.data = make([]byte, wantBytes) + } + + bytesCopied, err := src.CopyIn(ctx, fd.writeCursorFR.data[fd.writeCursor:wantBytes]) + if err != nil { + return 0, err + } + src = src.DropFirst(bytesCopied) + + cn = int64(bytesCopied) + n += cn + fd.writeCursor += uint32(cn) + if fd.writeCursor == wantBytes { + // Done reading this full response. Clean up and unblock the + // initiator. + break + } + + // Check if we have more data in src. + continue + } + + // Assert that the header isn't read into the writeBuf yet. + if fd.writeCursor >= hdrLen { + return 0, syserror.EINVAL + } + + // We don't have the full common response header yet. + wantBytes := hdrLen - fd.writeCursor + bytesCopied, err := src.CopyIn(ctx, fd.writeBuf[fd.writeCursor:wantBytes]) + if err != nil { + return 0, err + } + src = src.DropFirst(bytesCopied) + + cn = int64(bytesCopied) + n += cn + fd.writeCursor += uint32(cn) + if fd.writeCursor == hdrLen { + // Have full header in the writeBuf. Use it to fetch the actual futureResponse + // from the device's completions map. + var hdr linux.FUSEHeaderOut + hdr.UnmarshalBytes(fd.writeBuf) + + // We have the header now and so the writeBuf has served its purpose. + // We could reset it manually here but instead of doing that, at the + // end of the write, the writeCursor will be set to 0 thereby allowing + // the next request to overwrite whats in the buffer, + + fut, ok := fd.completions[hdr.Unique] + if !ok { + // Server sent us a response for a request we never sent? + return 0, syserror.EINVAL + } + + delete(fd.completions, hdr.Unique) + + // Copy over the header into the future response. The rest of the payload + // will be copied over to the FR's data in the next iteration. + fut.hdr = &hdr + fd.writeCursorFR = fut + + // Next iteration will now try read the complete request, if src has + // any data remaining. Otherwise we're done. + } + } + + if fd.writeCursorFR != nil { + if err := fd.sendResponse(ctx, fd.writeCursorFR); err != nil { + return 0, err + } + + // Ready the device for the next request. + fd.writeCursorFR = nil + fd.writeCursor = 0 + } + + return n, nil +} + +// Readiness implements vfs.FileDescriptionImpl.Readiness. +func (fd *DeviceFD) Readiness(mask waiter.EventMask) waiter.EventMask { + var ready waiter.EventMask + ready |= waiter.EventOut // FD is always writable + if !fd.queue.Empty() { + // Have reqs available, FD is readable. + ready |= waiter.EventIn + } + + return ready & mask +} + +// EventRegister implements waiter.Waitable.EventRegister. +func (fd *DeviceFD) EventRegister(e *waiter.Entry, mask waiter.EventMask) { + fd.waitQueue.EventRegister(e, mask) +} + +// EventUnregister implements waiter.Waitable.EventUnregister. +func (fd *DeviceFD) EventUnregister(e *waiter.Entry) { + fd.waitQueue.EventUnregister(e) } // Seek implements vfs.FileDescriptionImpl.Seek. @@ -112,22 +334,61 @@ func (fd *DeviceFD) Seek(ctx context.Context, offset int64, whence int32) (int64 return 0, syserror.ENOSYS } -// Register registers the FUSE device with vfsObj. -func Register(vfsObj *vfs.VirtualFilesystem) error { - if err := vfsObj.RegisterDevice(vfs.CharDevice, linux.MISC_MAJOR, fuseDevMinor, fuseDevice{}, &vfs.RegisterDeviceOptions{ - GroupName: "misc", - }); err != nil { +// sendResponse sends a response to the waiting task (if any). +func (fd *DeviceFD) sendResponse(ctx context.Context, fut *futureResponse) error { + // See if the running task need to perform some action before returning. + // Since we just finished writing the future, we can be sure that + // getResponse generates a populated response. + if err := fd.noReceiverAction(ctx, fut.getResponse()); err != nil { return err } + // Signal that the queue is no longer full. + select { + case fd.fullQueueCh <- struct{}{}: + default: + } + fd.numActiveRequests -= 1 + + // Signal the task waiting on a response. + close(fut.ch) return nil } -// CreateDevtmpfsFile creates a device special file in devtmpfs. -func CreateDevtmpfsFile(ctx context.Context, dev *devtmpfs.Accessor) error { - if err := dev.CreateDeviceFile(ctx, "fuse", vfs.CharDevice, linux.MISC_MAJOR, fuseDevMinor, 0666 /* mode */); err != nil { +// sendError sends an error response to the waiting task (if any). +func (fd *DeviceFD) sendError(ctx context.Context, errno int32, req *Request) error { + // Return the error to the calling task. + outHdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) + respHdr := linux.FUSEHeaderOut{ + Len: outHdrLen, + Error: errno, + Unique: req.hdr.Unique, + } + + fut, ok := fd.completions[respHdr.Unique] + if !ok { + // Server sent us a response for a request we never sent? + return syserror.EINVAL + } + delete(fd.completions, respHdr.Unique) + + fut.hdr = &respHdr + if err := fd.sendResponse(ctx, fut); err != nil { return err } return nil } + +// noReceiverAction has the calling kernel.Task do some action if its known that no +// receiver is going to be waiting on the future channel. This is to be used by: +// FUSE_INIT. +func (fd *DeviceFD) noReceiverAction(ctx context.Context, r *Response) error { + if r.opcode == linux.FUSE_INIT { + // TODO: process init response here. + // Maybe get the creds from the context? + // creds := auth.CredentialsFromContext(ctx) + } + + return nil +} diff --git a/pkg/sentry/fsimpl/fuse/dev_test.go b/pkg/sentry/fsimpl/fuse/dev_test.go new file mode 100644 index 000000000..fcd77832a --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/dev_test.go @@ -0,0 +1,429 @@ +// 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 fuse + +import ( + "fmt" + "io" + "math/rand" + "testing" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/testutil" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/tmpfs" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/usermem" + "gvisor.dev/gvisor/pkg/waiter" + "gvisor.dev/gvisor/tools/go_marshal/marshal" +) + +// echoTestOpcode is the Opcode used during testing. The server used in tests +// will simply echo the payload back with the appropriate headers. +const echoTestOpcode linux.FUSEOpcode = 1000 + +type testPayload struct { + data uint32 +} + +// TestFUSECommunication tests that the communication layer between the Sentry and the +// FUSE server daemon works as expected. +func TestFUSECommunication(t *testing.T) { + s := setup(t) + defer s.Destroy() + + k := kernel.KernelFromContext(s.Ctx) + creds := auth.CredentialsFromContext(s.Ctx) + + // Create test cases with different number of concurrent clients and servers. + testCases := []struct { + Name string + NumClients int + NumServers int + MaxActiveRequests uint64 + }{ + { + Name: "SingleClientSingleServer", + NumClients: 1, + NumServers: 1, + MaxActiveRequests: MaxActiveRequestsDefault, + }, + { + Name: "SingleClientMultipleServers", + NumClients: 1, + NumServers: 10, + MaxActiveRequests: MaxActiveRequestsDefault, + }, + { + Name: "MultipleClientsSingleServer", + NumClients: 10, + NumServers: 1, + MaxActiveRequests: MaxActiveRequestsDefault, + }, + { + Name: "MultipleClientsMultipleServers", + NumClients: 10, + NumServers: 10, + MaxActiveRequests: MaxActiveRequestsDefault, + }, + { + Name: "RequestCapacityFull", + NumClients: 10, + NumServers: 1, + MaxActiveRequests: 1, + }, + { + Name: "RequestCapacityContinuouslyFull", + NumClients: 100, + NumServers: 2, + MaxActiveRequests: 2, + }, + } + + for _, testCase := range testCases { + t.Run(testCase.Name, func(t *testing.T) { + conn, fd, err := newTestConnection(s, k, testCase.MaxActiveRequests) + if err != nil { + t.Fatalf("newTestConnection: %v", err) + } + + clientsDone := make([]chan struct{}, testCase.NumClients) + serversDone := make([]chan struct{}, testCase.NumServers) + serversKill := make([]chan struct{}, testCase.NumServers) + + // FUSE clients. + for i := 0; i < testCase.NumClients; i++ { + clientsDone[i] = make(chan struct{}) + go func(i int) { + fuseClientRun(t, s, k, conn, creds, uint32(i), uint64(i), clientsDone[i]) + }(i) + } + + // FUSE servers. + for j := 0; j < testCase.NumServers; j++ { + serversDone[j] = make(chan struct{}) + serversKill[j] = make(chan struct{}, 1) // The kill command shouldn't block. + go func(j int) { + fuseServerRun(t, s, k, fd, serversDone[j], serversKill[j]) + }(j) + } + + // Tear down. + // + // Make sure all the clients are done. + for i := 0; i < testCase.NumClients; i++ { + <-clientsDone[i] + } + + // Kill any server that is potentially waiting. + for j := 0; j < testCase.NumServers; j++ { + serversKill[j] <- struct{}{} + } + + // Make sure all the servers are done. + for j := 0; j < testCase.NumServers; j++ { + <-serversDone[j] + } + }) + } +} + +// CallTest makes a request to the server and blocks the invoking +// goroutine until a server responds with a response. Doesn't block +// a kernel.Task. Analogous to Connection.Call but used for testing. +func CallTest(conn *Connection, t *kernel.Task, r *Request, i uint32) (*Response, error) { + conn.fd.mu.Lock() + + // Wait until we're certain that a new request can be processed. + for conn.fd.numActiveRequests == conn.fd.fs.opts.maxActiveRequests { + conn.fd.mu.Unlock() + select { + case <-conn.fd.fullQueueCh: + } + conn.fd.mu.Lock() + } + + fut, err := conn.callFutureLocked(t, r) // No task given. + conn.fd.mu.Unlock() + + if err != nil { + return nil, err + } + + // Resolve the response. + // + // Block without a task. + select { + case <-fut.ch: + } + + // A response is ready. Resolve and return it. + return fut.getResponse(), nil +} + +// ReadTest is analogous to vfs.FileDescription.Read and reads from the FUSE +// device. However, it does so by - not blocking the task that is calling - and +// instead just waits on a channel. The behaviour is essentially the same as +// DeviceFD.Read except it guarantees that the task is not blocked. +func ReadTest(serverTask *kernel.Task, fd *vfs.FileDescription, inIOseq usermem.IOSequence, killServer chan struct{}) (int64, bool, error) { + var err error + var n, total int64 + + dev := fd.Impl().(*DeviceFD) + + // Register for notifications. + w, ch := waiter.NewChannelEntry(nil) + dev.EventRegister(&w, waiter.EventIn) + for { + // Issue the request and break out if it completes with anything other than + // "would block". + n, err = dev.Read(serverTask, inIOseq, vfs.ReadOptions{}) + total += n + if err != syserror.ErrWouldBlock { + break + } + + // Wait for a notification that we should retry. + // Emulate the blocking for when no requests are available + select { + case <-ch: + case <-killServer: + // Server killed by the main program. + return 0, true, nil + } + } + + dev.EventUnregister(&w) + return total, false, err +} + +// fuseClientRun emulates all the actions of a normal FUSE request. It creates +// a header, a payload, calls the server, waits for the response, and processes +// the response. +func fuseClientRun(t *testing.T, s *testutil.System, k *kernel.Kernel, conn *Connection, creds *auth.Credentials, pid uint32, inode uint64, clientDone chan struct{}) { + defer func() { clientDone <- struct{}{} }() + + tc := k.NewThreadGroup(nil, k.RootPIDNamespace(), kernel.NewSignalHandlers(), linux.SIGCHLD, k.GlobalInit().Limits()) + clientTask, err := testutil.CreateTask(s.Ctx, fmt.Sprintf("fuse-client-%v", pid), tc, s.MntNs, s.Root, s.Root) + if err != nil { + t.Fatal(err) + } + testObj := &testPayload{ + data: rand.Uint32(), + } + + req, err := conn.NewRequest(creds, pid, inode, echoTestOpcode, testObj) + if err != nil { + t.Fatalf("NewRequest creation failed: %v", err) + } + + // Queue up a request. + // Analogous to Call except it doesn't block on the task. + resp, err := CallTest(conn, clientTask, req, pid) + if err != nil { + t.Fatalf("CallTaskNonBlock failed: %v", err) + } + + if err = resp.Error(); err != nil { + t.Fatalf("Server responded with an error: %v", err) + } + + var respTestPayload testPayload + if err := resp.UnmarshalPayload(&respTestPayload); err != nil { + t.Fatalf("Unmarshalling payload error: %v", err) + } + + if resp.hdr.Unique != req.hdr.Unique { + t.Fatalf("got response for another request. Expected response for req %v but got response for req %v", + req.hdr.Unique, resp.hdr.Unique) + } + + if respTestPayload.data != testObj.data { + t.Fatalf("read incorrect data. Data expected: %v, but got %v", testObj.data, respTestPayload.data) + } + +} + +// fuseServerRun creates a task and emulates all the actions of a simple FUSE server +// that simply reads a request and echos the same struct back as a response using the +// appropriate headers. +func fuseServerRun(t *testing.T, s *testutil.System, k *kernel.Kernel, fd *vfs.FileDescription, serverDone, killServer chan struct{}) { + defer func() { serverDone <- struct{}{} }() + + // Create the tasks that the server will be using. + tc := k.NewThreadGroup(nil, k.RootPIDNamespace(), kernel.NewSignalHandlers(), linux.SIGCHLD, k.GlobalInit().Limits()) + var readPayload testPayload + + serverTask, err := testutil.CreateTask(s.Ctx, "fuse-server", tc, s.MntNs, s.Root, s.Root) + if err != nil { + t.Fatal(err) + } + + // Read the request. + for { + inHdrLen := uint32((*linux.FUSEHeaderIn)(nil).SizeBytes()) + payloadLen := uint32(readPayload.SizeBytes()) + + // The raed buffer must meet some certain size criteria. + buffSize := inHdrLen + payloadLen + if buffSize < linux.FUSE_MIN_READ_BUFFER { + buffSize = linux.FUSE_MIN_READ_BUFFER + } + inBuf := make([]byte, buffSize) + inIOseq := usermem.BytesIOSequence(inBuf) + + n, serverKilled, err := ReadTest(serverTask, fd, inIOseq, killServer) + if err != nil { + t.Fatalf("Read failed :%v", err) + } + + // Server should shut down. No new requests are going to be made. + if serverKilled { + break + } + + if n <= 0 { + t.Fatalf("Read read no bytes") + } + + var readFUSEHeaderIn linux.FUSEHeaderIn + readFUSEHeaderIn.UnmarshalUnsafe(inBuf[:inHdrLen]) + readPayload.UnmarshalUnsafe(inBuf[inHdrLen : inHdrLen+payloadLen]) + + if readFUSEHeaderIn.Opcode != echoTestOpcode { + t.Fatalf("read incorrect data. Header: %v, Payload: %v", readFUSEHeaderIn, readPayload) + } + + // Write the response. + outHdrLen := uint32((*linux.FUSEHeaderOut)(nil).SizeBytes()) + outBuf := make([]byte, outHdrLen+payloadLen) + outHeader := linux.FUSEHeaderOut{ + Len: outHdrLen + payloadLen, + Error: 0, + Unique: readFUSEHeaderIn.Unique, + } + + // Echo the payload back. + outHeader.MarshalUnsafe(outBuf[:outHdrLen]) + readPayload.MarshalUnsafe(outBuf[outHdrLen:]) + outIOseq := usermem.BytesIOSequence(outBuf) + + n, err = fd.Write(s.Ctx, outIOseq, vfs.WriteOptions{}) + if err != nil { + t.Fatalf("Write failed :%v", err) + } + } +} + +func setup(t *testing.T) *testutil.System { + k, err := testutil.Boot() + if err != nil { + t.Fatalf("Error creating kernel: %v", err) + } + + ctx := k.SupervisorContext() + creds := auth.CredentialsFromContext(ctx) + + k.VFS().MustRegisterFilesystemType(Name, &FilesystemType{}, &vfs.RegisterFilesystemTypeOptions{ + AllowUserList: true, + AllowUserMount: true, + }) + + mntns, err := k.VFS().NewMountNamespace(ctx, creds, "", tmpfs.Name, &vfs.GetFilesystemOptions{}) + if err != nil { + t.Fatalf("NewMountNamespace(): %v", err) + } + + return testutil.NewSystem(ctx, t, k.VFS(), mntns) +} + +// newTestConnection creates a fuse connection that the sentry can communicate with +// and the FD for the server to communicate with. +func newTestConnection(system *testutil.System, k *kernel.Kernel, maxActiveRequests uint64) (*Connection, *vfs.FileDescription, error) { + vfsObj := &vfs.VirtualFilesystem{} + fuseDev := &DeviceFD{} + + if err := vfsObj.Init(); err != nil { + return nil, nil, err + } + + vd := vfsObj.NewAnonVirtualDentry("genCountFD") + defer vd.DecRef() + if err := fuseDev.vfsfd.Init(fuseDev, linux.O_RDWR|linux.O_CREAT, vd.Mount(), vd.Dentry(), &vfs.FileDescriptionOptions{}); err != nil { + return nil, nil, err + } + + fsopts := filesystemOptions{ + maxActiveRequests: maxActiveRequests, + } + fs, err := NewFUSEFilesystem(system.Ctx, 0, &fsopts, &fuseDev.vfsfd) + if err != nil { + return nil, nil, err + } + + return fs.conn, &fuseDev.vfsfd, nil +} + +// SizeBytes implements marshal.Marshallable.SizeBytes. +func (t *testPayload) SizeBytes() int { + return 4 +} + +// MarshalBytes implements marshal.Marshallable.MarshalBytes. +func (t *testPayload) MarshalBytes(dst []byte) { + usermem.ByteOrder.PutUint32(dst[:4], t.data) +} + +// UnmarshalBytes implements marshal.Marshallable.UnmarshalBytes. +func (t *testPayload) UnmarshalBytes(src []byte) { + *t = testPayload{data: usermem.ByteOrder.Uint32(src[:4])} +} + +// Packed implements marshal.Marshallable.Packed. +func (t *testPayload) Packed() bool { + return true +} + +// MarshalUnsafe implements marshal.Marshallable.MarshalUnsafe. +func (t *testPayload) MarshalUnsafe(dst []byte) { + t.MarshalBytes(dst) +} + +// UnmarshalUnsafe implements marshal.Marshallable.UnmarshalUnsafe. +func (t *testPayload) UnmarshalUnsafe(src []byte) { + t.UnmarshalBytes(src) +} + +// CopyOutN implements marshal.Marshallable.CopyOutN. +func (t *testPayload) CopyOutN(task marshal.Task, addr usermem.Addr, limit int) (int, error) { + panic("not implemented") +} + +// CopyOut implements marshal.Marshallable.CopyOut. +func (t *testPayload) CopyOut(task marshal.Task, addr usermem.Addr) (int, error) { + panic("not implemented") +} + +// CopyIn implements marshal.Marshallable.CopyIn. +func (t *testPayload) CopyIn(task marshal.Task, addr usermem.Addr) (int, error) { + panic("not implemented") +} + +// WriteTo implements io.WriterTo.WriteTo. +func (t *testPayload) WriteTo(w io.Writer) (int64, error) { + panic("not implemented") +} diff --git a/pkg/sentry/fsimpl/fuse/fusefs.go b/pkg/sentry/fsimpl/fuse/fusefs.go index f7775fb9b..911b6f7cb 100644 --- a/pkg/sentry/fsimpl/fuse/fusefs.go +++ b/pkg/sentry/fsimpl/fuse/fusefs.go @@ -51,6 +51,11 @@ type filesystemOptions struct { // rootMode specifies the the file mode of the filesystem's root. rootMode linux.FileMode + + // maxActiveRequests specifies the maximum number of active requests that can + // exist at any time. Any further requests will block when trying to + // Call the server. + maxActiveRequests uint64 } // filesystem implements vfs.FilesystemImpl. @@ -58,12 +63,12 @@ type filesystem struct { kernfs.Filesystem devMinor uint32 - // fuseFD is the FD returned when opening /dev/fuse. It is used for communication - // between the FUSE server daemon and the sentry fusefs. - fuseFD *DeviceFD + // conn is used for communication between the FUSE server + // daemon and the sentry fusefs. + conn *Connection // opts is the options the fusefs is initialized with. - opts filesystemOptions + opts *filesystemOptions } // Name implements vfs.FilesystemType.Name. @@ -100,7 +105,7 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt fuseFd := kernelTask.GetFileVFS2(int32(deviceDescriptor)) // Parse and set all the other supported FUSE mount options. - // TODO: Expand the supported mount options. + // TODO(gVisor.dev/issue/3229): Expand the supported mount options. if userIDStr, ok := mopts["user_id"]; ok { delete(mopts, "user_id") userID, err := strconv.ParseUint(userIDStr, 10, 32) @@ -134,21 +139,20 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt } fsopts.rootMode = rootMode + // Set the maxInFlightRequests option. + fsopts.maxActiveRequests = MaxActiveRequestsDefault + // Check for unparsed options. if len(mopts) != 0 { log.Warningf("%s.GetFilesystem: unknown options: %v", fsType.Name(), mopts) return nil, nil, syserror.EINVAL } - // Mark the device as ready so it can be used. /dev/fuse can only be used if the FD was used to - // mount a FUSE filesystem. - fuseFD := fuseFd.Impl().(*DeviceFD) - fuseFD.mounted = true - - fs := &filesystem{ - devMinor: devMinor, - fuseFD: fuseFD, - opts: fsopts, + // Create a new FUSE filesystem. + fs, err := NewFUSEFilesystem(ctx, devMinor, &fsopts, fuseFd) + if err != nil { + log.Warningf("%s.NewFUSEFilesystem: failed with error: %v", fsType.Name(), err) + return nil, nil, err } fs.VFSFilesystem().Init(vfsObj, &fsType, fs) @@ -162,6 +166,26 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt return fs.VFSFilesystem(), root.VFSDentry(), nil } +// NewFUSEFilesystem creates a new FUSE filesystem. +func NewFUSEFilesystem(ctx context.Context, devMinor uint32, opts *filesystemOptions, device *vfs.FileDescription) (*filesystem, error) { + fs := &filesystem{ + devMinor: devMinor, + opts: opts, + } + + conn, err := NewFUSEConnection(ctx, device, opts.maxActiveRequests) + if err != nil { + log.Warningf("fuse.NewFUSEFilesystem: NewFUSEConnection failed with error: %v", err) + return nil, syserror.EINVAL + } + + fs.conn = conn + fuseFD := device.Impl().(*DeviceFD) + fuseFD.fs = fs + + return fs, nil +} + // Release implements vfs.FilesystemImpl.Release. func (fs *filesystem) Release() { fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor) diff --git a/pkg/sentry/fsimpl/fuse/register.go b/pkg/sentry/fsimpl/fuse/register.go new file mode 100644 index 000000000..b5b581152 --- /dev/null +++ b/pkg/sentry/fsimpl/fuse/register.go @@ -0,0 +1,42 @@ +// 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 fuse + +import ( + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/devtmpfs" + "gvisor.dev/gvisor/pkg/sentry/vfs" +) + +// Register registers the FUSE device with vfsObj. +func Register(vfsObj *vfs.VirtualFilesystem) error { + if err := vfsObj.RegisterDevice(vfs.CharDevice, linux.MISC_MAJOR, fuseDevMinor, fuseDevice{}, &vfs.RegisterDeviceOptions{ + GroupName: "misc", + }); err != nil { + return err + } + + return nil +} + +// CreateDevtmpfsFile creates a device special file in devtmpfs. +func CreateDevtmpfsFile(ctx context.Context, dev *devtmpfs.Accessor) error { + if err := dev.CreateDeviceFile(ctx, "fuse", vfs.CharDevice, linux.MISC_MAJOR, fuseDevMinor, 0666 /* mode */); err != nil { + return err + } + + return nil +} |