Commit 83630d42 authored by Aaron Jacobs's avatar Aaron Jacobs

Refactored the way the user obtains contexts and replies to ops.

The new interface gives much much cleaner documentation for package
fuseops, which is what we mostly look at. In particular, there isn't a
ton of stuttering in the method listings.

The change also simplifies the implementation of package fuseops, which
was becoming super unwieldy.
parents a4ffa71f 5c332f76
......@@ -26,11 +26,14 @@ import (
"golang.org/x/net/context"
"github.com/jacobsa/fuse/fuseops"
"github.com/jacobsa/fuse/internal/buffer"
"github.com/jacobsa/fuse/internal/fusekernel"
)
type contextKeyType uint64
const contextKey contextKeyType = 0
// Ask the Linux kernel for larger read requests.
//
// As of 2015-03-26, the behavior in the kernel is:
......@@ -82,6 +85,14 @@ type Connection struct {
cancelFuncs map[uint64]func()
}
// State that is maintained for each in-flight op. This is stuffed into the
// context that the user uses to reply to the op.
type opState struct {
inMsg *buffer.InMessage
op interface{}
opID uint32 // For logging
}
// Create a connection wrapping the supplied file descriptor connected to the
// kernel. You must eventually call c.close().
//
......@@ -113,15 +124,16 @@ func newConnection(
// Do the work necessary to cause the mount process to complete.
func (c *Connection) Init() (err error) {
// Read the init op.
op, err := c.ReadOp()
ctx, op, err := c.ReadOp()
if err != nil {
err = fmt.Errorf("Reading init op: %v", err)
return
}
initOp, ok := op.(*fuseops.InternalInitOp)
initOp, ok := op.(*initOp)
if !ok {
err = fmt.Errorf("Expected *fuseops.InternalInitOp, got %T", op)
c.Reply(ctx, syscall.EPROTO)
err = fmt.Errorf("Expected *initOp, got %T", op)
return
}
......@@ -132,7 +144,7 @@ func (c *Connection) Init() (err error) {
}
if initOp.Kernel.LT(min) {
initOp.Respond(syscall.EPROTO)
c.Reply(ctx, syscall.EPROTO)
err = fmt.Errorf("Version too old: %v", initOp.Kernel)
return
}
......@@ -152,8 +164,8 @@ func (c *Connection) Init() (err error) {
initOp.MaxReadahead = maxReadahead
initOp.MaxWrite = buffer.MaxWriteSize
initOp.Flags = fusekernel.InitBigWrites
initOp.Respond(nil)
c.Reply(ctx, nil)
return
}
......@@ -372,14 +384,18 @@ func (c *Connection) writeMessage(msg []byte) (err error) {
return
}
// Read the next op from the kernel process. Return io.EOF if the kernel has
// closed the connection.
// Read the next op from the kernel process, returning the op and a context
// that should be used for work related to the op. Return io.EOF if the kernel
// has closed the connection.
//
// If err != nil, the user is responsible for later calling c.Reply with the
// returned context.
//
// This function delivers ops in exactly the order they are received from
// /dev/fuse. It must not be called multiple times concurrently.
//
// LOCKS_EXCLUDED(c.mu)
func (c *Connection) ReadOp() (op fuseops.Op, err error) {
func (c *Connection) ReadOp() (ctx context.Context, op interface{}, err error) {
// Keep going until we find a request we know how to convert.
for {
// Read the next message from the kernel.
......@@ -389,87 +405,88 @@ func (c *Connection) ReadOp() (op fuseops.Op, err error) {
return
}
// Choose an ID for this operation for the purposes of logging.
// Convert the message to an op.
op, err = convertInMessage(m, c.protocol)
if err != nil {
err = fmt.Errorf("convertInMessage: %v", err)
return
}
// Choose an ID for this operation for the purposes of logging, and log it.
opID := c.nextOpID
c.nextOpID++
// Set up op dependencies.
opCtx := c.beginOp(m.Header().Opcode, m.Header().Unique)
c.debugLog(opID, 1, "<- %#v", op)
var debugLogForOp func(int, string, ...interface{})
if c.debugLogger != nil {
debugLogForOp = func(calldepth int, format string, v ...interface{}) {
c.debugLog(opID, calldepth+1, format, v...)
}
// Special case: handle interrupt requests inline.
if interruptOp, ok := op.(*interruptOp); ok {
c.handleInterrupt(interruptOp.FuseID)
continue
}
sendReply := func(
op fuseops.Op,
fuseID uint64,
replyMsg []byte,
opErr error) (err error) {
// Make sure we destroy the message, as required by readMessage.
defer c.destroyInMessage(m)
// Clean up state for this op.
c.finishOp(m.Header().Opcode, m.Header().Unique)
// Debug logging
if c.debugLogger != nil {
if opErr == nil {
op.Logf("-> OK: %s", op.DebugString())
} else {
op.Logf("-> error: %v", opErr)
}
}
// Set up a context that remembers information about this op.
ctx = c.beginOp(m.Header().Opcode, m.Header().Unique)
ctx = context.WithValue(ctx, contextKey, opState{m, op, opID})
// Error logging
if opErr != nil && c.errorLogger != nil {
c.errorLogger.Printf("(%s) error: %v", op.ShortDesc(), opErr)
}
// Special case: responding to statfs is required to make mounting work on
// OS X. We don't currently expose the capability for the file system to
// intercept this.
if _, ok := op.(*statFSOp); ok {
c.Reply(ctx, nil)
continue
}
// Send the reply to the kernel.
err = c.writeMessage(replyMsg)
if err != nil {
err = fmt.Errorf("writeMessage: %v", err)
return
}
// Return the op to the user.
return
}
}
return
}
// Reply to an op previously read using ReadOp, with the supplied error (or nil
// if successful). The context must be the context returned by ReadOp.
//
// LOCKS_EXCLUDED(c.mu)
func (c *Connection) Reply(ctx context.Context, opErr error) {
// Extract the state we stuffed in earlier.
state, ok := ctx.Value(contextKey).(opState)
if !ok {
panic(fmt.Sprintf("Reply called with invalid context: %#v", ctx))
}
// Convert the message to an Op.
op, err = fuseops.Convert(
opCtx,
m,
c.protocol,
debugLogForOp,
c.errorLogger,
sendReply)
op := state.op
m := state.inMsg
opID := state.opID
if err != nil {
err = fmt.Errorf("fuseops.Convert: %v", err)
return
}
// Make sure we destroy the message when we're done.
defer c.destroyInMessage(m)
// Log the receipt of the operation.
c.debugLog(opID, 1, "<- %v", op.ShortDesc())
// Clean up state for this op.
c.finishOp(m.Header().Opcode, m.Header().Unique)
// Special case: responding to statfs is required to make mounting work on
// OS X. We don't currently expose the capability for the file system to
// intercept this.
if _, ok := op.(*fuseops.InternalStatFSOp); ok {
op.Respond(nil)
continue
// Debug logging
if c.debugLogger != nil {
if opErr == nil {
c.debugLog(opID, 1, "-> OK: %#v", op)
} else {
c.debugLog(opID, 1, "-> error: %v", opErr)
}
}
// Special case: handle interrupt requests.
if interruptOp, ok := op.(*fuseops.InternalInterruptOp); ok {
c.handleInterrupt(interruptOp.FuseID)
continue
// Error logging
if opErr != nil && c.errorLogger != nil {
c.errorLogger.Printf("(%#v) error: %v", op, opErr)
}
// Send the reply to the kernel.
replyMsg := kernelResponse(m.Header().Unique, op, opErr, c.protocol)
if replyMsg != nil {
if err := c.writeMessage(replyMsg); err != nil {
if c.errorLogger != nil {
c.errorLogger.Printf("writeMessage: %v", err)
}
return
}
return
}
}
......
This diff is collapsed.
......@@ -18,7 +18,7 @@ import "syscall"
const (
// Errors corresponding to kernel error numbers. These may be treated
// specially by fuseops.Op.Respond methods.
// specially by Connection.Reply.
EEXIST = syscall.EEXIST
EINVAL = syscall.EINVAL
EIO = syscall.EIO
......
// Copyright 2015 Google Inc. All Rights Reserved.
//
// 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 fuseops
import (
"fmt"
"log"
"reflect"
"strings"
"syscall"
"github.com/jacobsa/fuse/internal/buffer"
"github.com/jacobsa/reqtrace"
"golang.org/x/net/context"
)
// An interface that all ops inside which commonOp is embedded must
// implement.
type internalOp interface {
Op
// Create a response message for the kernel, leaving the leading
// fusekernel.OutHeader untouched.
//
// Special case: a zero return value means that the kernel is not expecting a
// response.
kernelResponse() (b buffer.OutMessage)
}
// A function that sends a reply message back to the kernel for the request
// with the given fuse unique ID. The error argument is for informational
// purposes only; the error to hand to the kernel is encoded in the message.
type replyFunc func(Op, uint64, []byte, error) error
// A helper for embedding common behavior.
type commonOp struct {
// The context exposed to the user.
ctx context.Context
// The op in which this struct is embedded.
op internalOp
// The fuse unique ID of this request, as assigned by the kernel.
fuseID uint64
// A function that can be used to send a reply to the kernel.
sendReply replyFunc
// A function that can be used to log debug information about the op. The
// first argument is a call depth.
//
// May be nil.
debugLog func(int, string, ...interface{})
// A logger to be used for logging exceptional errors.
//
// May be nil.
errorLogger *log.Logger
}
func (o *commonOp) ShortDesc() (desc string) {
v := reflect.ValueOf(o.op)
opName := v.Type().String()
// Attempt to better handle the usual case: a string that looks like
// "*fuseops.GetInodeAttributesOp".
const prefix = "*fuseops."
const suffix = "Op"
if strings.HasPrefix(opName, prefix) && strings.HasSuffix(opName, suffix) {
opName = opName[len(prefix) : len(opName)-len(suffix)]
}
desc = opName
// Include the inode number to which the op applies, if possible.
if f := v.Elem().FieldByName("Inode"); f.IsValid() {
desc = fmt.Sprintf("%s(inode=%v)", desc, f.Interface())
}
return
}
func (o *commonOp) DebugString() string {
// By default, defer to ShortDesc.
return o.op.ShortDesc()
}
func (o *commonOp) init(
ctx context.Context,
op internalOp,
fuseID uint64,
sendReply replyFunc,
debugLog func(int, string, ...interface{}),
errorLogger *log.Logger) {
// Initialize basic fields.
o.ctx = ctx
o.op = op
o.fuseID = fuseID
o.sendReply = sendReply
o.debugLog = debugLog
o.errorLogger = errorLogger
// Set up a trace span for this op.
var reportForTrace reqtrace.ReportFunc
o.ctx, reportForTrace = reqtrace.StartSpan(o.ctx, o.op.ShortDesc())
// When the op is finished, report to both reqtrace and the connection.
prevSendReply := o.sendReply
o.sendReply = func(op Op, fuseID uint64, msg []byte, opErr error) (err error) {
reportForTrace(opErr)
err = prevSendReply(op, fuseID, msg, opErr)
return
}
}
func (o *commonOp) Context() context.Context {
return o.ctx
}
func (o *commonOp) Logf(format string, v ...interface{}) {
if o.debugLog == nil {
return
}
const calldepth = 2
o.debugLog(calldepth, format, v...)
}
func (o *commonOp) Respond(err error) {
// If successful, we ask the op for an appopriate response to the kernel, and
// it is responsible for leaving room for the fusekernel.OutHeader struct.
// Otherwise, create our own.
var b buffer.OutMessage
if err == nil {
b = o.op.kernelResponse()
} else {
b = buffer.NewOutMessage(0)
}
// Fill in the header if a reply is needed.
msg := b.Bytes()
if msg != nil {
h := b.OutHeader()
h.Unique = o.fuseID
h.Len = uint32(len(msg))
if err != nil {
// If the user gave us a syscall.Errno, use that value in the reply.
// Otherwise use the generic EIO.
if errno, ok := err.(syscall.Errno); ok {
h.Error = -int32(errno)
} else {
h.Error = -int32(syscall.EIO)
}
}
}
// Reply.
replyErr := o.sendReply(o.op, o.fuseID, msg, err)
if replyErr != nil && o.errorLogger != nil {
o.errorLogger.Printf("Error from sendReply: %v", replyErr)
}
}
......@@ -12,7 +12,6 @@
// See the License for the specific language governing permissions and
// limitations under the License.
// Package fuseops contains implementations of the fuse.Op interface that may
// be returned by fuse.Connection.ReadOp. See documentation in that package for
// more.
// Package fuseops contains ops that may be returned by fuse.Connection.ReadOp.
// See documentation in that package for more.
package fuseops
This diff is collapsed.
......@@ -18,6 +18,8 @@ import (
"io"
"sync"
"golang.org/x/net/context"
"github.com/jacobsa/fuse"
"github.com/jacobsa/fuse/fuseops"
)
......@@ -27,32 +29,32 @@ import (
// loop" that switches on op types, instead receiving typed method calls
// directly.
//
// The FileSystem implementation should not call Op.Respond, instead returning
// the error with which the caller should respond.
// The FileSystem implementation should not call Connection.Reply, instead
// returning the error with which the caller should respond.
//
// See NotImplementedFileSystem for a convenient way to embed default
// implementations for methods you don't care about.
type FileSystem interface {
LookUpInode(*fuseops.LookUpInodeOp) error
GetInodeAttributes(*fuseops.GetInodeAttributesOp) error
SetInodeAttributes(*fuseops.SetInodeAttributesOp) error
ForgetInode(*fuseops.ForgetInodeOp) error
MkDir(*fuseops.MkDirOp) error
CreateFile(*fuseops.CreateFileOp) error
CreateSymlink(*fuseops.CreateSymlinkOp) error
Rename(*fuseops.RenameOp) error
RmDir(*fuseops.RmDirOp) error
Unlink(*fuseops.UnlinkOp) error
OpenDir(*fuseops.OpenDirOp) error
ReadDir(*fuseops.ReadDirOp) error
ReleaseDirHandle(*fuseops.ReleaseDirHandleOp) error
OpenFile(*fuseops.OpenFileOp) error
ReadFile(*fuseops.ReadFileOp) error
WriteFile(*fuseops.WriteFileOp) error
SyncFile(*fuseops.SyncFileOp) error
FlushFile(*fuseops.FlushFileOp) error
ReleaseFileHandle(*fuseops.ReleaseFileHandleOp) error
ReadSymlink(*fuseops.ReadSymlinkOp) error
LookUpInode(context.Context, *fuseops.LookUpInodeOp) error
GetInodeAttributes(context.Context, *fuseops.GetInodeAttributesOp) error
SetInodeAttributes(context.Context, *fuseops.SetInodeAttributesOp) error
ForgetInode(context.Context, *fuseops.ForgetInodeOp) error
MkDir(context.Context, *fuseops.MkDirOp) error
CreateFile(context.Context, *fuseops.CreateFileOp) error
CreateSymlink(context.Context, *fuseops.CreateSymlinkOp) error
Rename(context.Context, *fuseops.RenameOp) error
RmDir(context.Context, *fuseops.RmDirOp) error
Unlink(context.Context, *fuseops.UnlinkOp) error
OpenDir(context.Context, *fuseops.OpenDirOp) error
ReadDir(context.Context, *fuseops.ReadDirOp) error
ReleaseDirHandle(context.Context, *fuseops.ReleaseDirHandleOp) error
OpenFile(context.Context, *fuseops.OpenFileOp) error
ReadFile(context.Context, *fuseops.ReadFileOp) error
WriteFile(context.Context, *fuseops.WriteFileOp) error
SyncFile(context.Context, *fuseops.SyncFileOp) error
FlushFile(context.Context, *fuseops.FlushFileOp) error
ReleaseFileHandle(context.Context, *fuseops.ReleaseFileHandleOp) error
ReadSymlink(context.Context, *fuseops.ReadSymlinkOp) error
// Regard all inodes (including the root inode) as having their lookup counts
// decremented to zero, and clean up any resources associated with the file
......@@ -91,7 +93,7 @@ func (s *fileSystemServer) ServeOps(c *fuse.Connection) {
}()
for {
op, err := c.ReadOp()
ctx, op, err := c.ReadOp()
if err == io.EOF {
break
}
......@@ -101,11 +103,14 @@ func (s *fileSystemServer) ServeOps(c *fuse.Connection) {
}
s.opsInFlight.Add(1)
go s.handleOp(op)
go s.handleOp(c, ctx, op)
}
}
func (s *fileSystemServer) handleOp(op fuseops.Op) {
func (s *fileSystemServer) handleOp(
c *fuse.Connection,
ctx context.Context,
op interface{}) {
defer s.opsInFlight.Done()
// Dispatch to the appropriate method.
......@@ -115,65 +120,65 @@ func (s *fileSystemServer) handleOp(op fuseops.Op) {
err = fuse.ENOSYS
case *fuseops.LookUpInodeOp:
err = s.fs.LookUpInode(typed)
err = s.fs.LookUpInode(ctx, typed)
case *fuseops.GetInodeAttributesOp:
err = s.fs.GetInodeAttributes(typed)
err = s.fs.GetInodeAttributes(ctx, typed)
case *fuseops.SetInodeAttributesOp:
err = s.fs.SetInodeAttributes(typed)
err = s.fs.SetInodeAttributes(ctx, typed)
case *fuseops.ForgetInodeOp:
err = s.fs.ForgetInode(typed)
err = s.fs.ForgetInode(ctx, typed)
case *fuseops.MkDirOp:
err = s.fs.MkDir(typed)
err = s.fs.MkDir(ctx, typed)
case *fuseops.CreateFileOp:
err = s.fs.CreateFile(typed)
err = s.fs.CreateFile(ctx, typed)
case *fuseops.CreateSymlinkOp:
err = s.fs.CreateSymlink(typed)
err = s.fs.CreateSymlink(ctx, typed)
case *fuseops.RenameOp:
err = s.fs.Rename(typed)
err = s.fs.Rename(ctx, typed)
case *fuseops.RmDirOp:
err = s.fs.RmDir(typed)
err = s.fs.RmDir(ctx, typed)
case *fuseops.UnlinkOp:
err = s.fs.Unlink(typed)
err = s.fs.Unlink(ctx, typed)
case *fuseops.OpenDirOp:
err = s.fs.OpenDir(typed)
err = s.fs.OpenDir(ctx, typed)
case *fuseops.ReadDirOp:
err = s.fs.ReadDir(typed)
err = s.fs.ReadDir(ctx, typed)
case *fuseops.ReleaseDirHandleOp:
err = s.fs.ReleaseDirHandle(typed)
err = s.fs.ReleaseDirHandle(ctx, typed)
case *fuseops.OpenFileOp:
err = s.fs.OpenFile(typed)
err = s.fs.OpenFile(ctx, typed)
case *fuseops.ReadFileOp:
err = s.fs.ReadFile(typed)
err = s.fs.ReadFile(ctx, typed)
case *fuseops.WriteFileOp:
err = s.fs.WriteFile(typed)
err = s.fs.WriteFile(ctx, typed)
case *fuseops.SyncFileOp:
err = s.fs.SyncFile(typed)
err = s.fs.SyncFile(ctx, typed)
case *fuseops.FlushFileOp:
err = s.fs.FlushFile(typed)
err = s.fs.FlushFile(ctx, typed)
case *fuseops.ReleaseFileHandleOp:
err = s.fs.ReleaseFileHandle(typed)
err = s.fs.ReleaseFileHandle(ctx, typed)
case *fuseops.ReadSymlinkOp:
err = s.fs.ReadSymlink(typed)
err = s.fs.ReadSymlink(ctx, typed)
}
op.Respond(err)
c.Reply(ctx, err)
}
......@@ -17,6 +17,7 @@ package fuseutil
import (
"github.com/jacobsa/fuse"
"github.com/jacobsa/fuse/fuseops"
"golang.org/x/net/context"
)
// A FileSystem that responds to all ops with fuse.ENOSYS. Embed this in your
......@@ -29,120 +30,140 @@ type NotImplementedFileSystem struct {
var _ FileSystem = &NotImplementedFileSystem{}
func (fs *NotImplementedFileSystem) LookUpInode(
ctx context.Context,
op *fuseops.LookUpInodeOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) GetInodeAttributes(
ctx context.Context,
op *fuseops.GetInodeAttributesOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) SetInodeAttributes(
ctx context.Context,
op *fuseops.SetInodeAttributesOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) ForgetInode(
ctx context.Context,
op *fuseops.ForgetInodeOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) MkDir(
ctx context.Context,
op *fuseops.MkDirOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) CreateFile(
ctx context.Context,
op *fuseops.CreateFileOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) CreateSymlink(
ctx context.Context,
op *fuseops.CreateSymlinkOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) Rename(
ctx context.Context,
op *fuseops.RenameOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) RmDir(
ctx context.Context,
op *fuseops.RmDirOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) Unlink(
ctx context.Context,
op *fuseops.UnlinkOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) OpenDir(
ctx context.Context,
op *fuseops.OpenDirOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) ReadDir(
ctx context.Context,
op *fuseops.ReadDirOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) ReleaseDirHandle(
ctx context.Context,
op *fuseops.ReleaseDirHandleOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) OpenFile(
ctx context.Context,
op *fuseops.OpenFileOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) ReadFile(
ctx context.Context,
op *fuseops.ReadFileOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) WriteFile(
ctx context.Context,
op *fuseops.WriteFileOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) SyncFile(
ctx context.Context,
op *fuseops.SyncFileOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) FlushFile(
ctx context.Context,
op *fuseops.FlushFileOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) ReleaseFileHandle(
ctx context.Context,
op *fuseops.ReleaseFileHandleOp) (err error) {
err = fuse.ENOSYS
return
}
func (fs *NotImplementedFileSystem) ReadSymlink(
ctx context.Context,
op *fuseops.ReadSymlinkOp) (err error) {
err = fuse.ENOSYS
return
......
// Copyright 2015 Google Inc. All Rights Reserved.
//
// 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 (
"github.com/jacobsa/fuse/fuseops"
"github.com/jacobsa/fuse/internal/fusekernel"
)
// A sentinel used for unknown ops. The user is expected to respond with a
// non-nil error.
type unknownOp struct {
opCode uint32
inode fuseops.InodeID
}
// Required in order to mount on OS X.
type statFSOp struct {
}
// Causes us to cancel the associated context.
type interruptOp struct {
FuseID uint64
}
// Required in order to mount on Linux and OS X.
type initOp struct {
// In
Kernel fusekernel.Protocol
// Out
Library fusekernel.Protocol
MaxReadahead uint32
Flags fusekernel.InitFlags
MaxWrite uint32
}
......@@ -19,6 +19,8 @@ import (
"os"
"time"
"golang.org/x/net/context"
"github.com/jacobsa/fuse"
"github.com/jacobsa/fuse/fuseops"
"github.com/jacobsa/fuse/fuseutil"
......@@ -240,6 +242,7 @@ func (fs *cachingFS) SetMtime(mtime time.Time) {
// LOCKS_EXCLUDED(fs.mu)
func (fs *cachingFS) LookUpInode(
ctx context.Context,
op *fuseops.LookUpInodeOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -294,6 +297,7 @@ func (fs *cachingFS) LookUpInode(
// LOCKS_EXCLUDED(fs.mu)
func (fs *cachingFS) GetInodeAttributes(
ctx context.Context,
op *fuseops.GetInodeAttributesOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -323,11 +327,13 @@ func (fs *cachingFS) GetInodeAttributes(
}
func (fs *cachingFS) OpenDir(
ctx context.Context,
op *fuseops.OpenDirOp) (err error) {
return
}
func (fs *cachingFS) OpenFile(
ctx context.Context,
op *fuseops.OpenFileOp) (err error) {
return
}
......@@ -19,6 +19,8 @@ import (
"os"
"sync"
"golang.org/x/net/context"
"github.com/jacobsa/fuse"
"github.com/jacobsa/fuse/fuseops"
"github.com/jacobsa/fuse/fuseutil"
......@@ -93,6 +95,7 @@ func (fs *flushFS) barAttributes() fuseops.InodeAttributes {
////////////////////////////////////////////////////////////////////////
func (fs *flushFS) LookUpInode(
ctx context.Context,
op *fuseops.LookUpInodeOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -126,6 +129,7 @@ func (fs *flushFS) LookUpInode(
}
func (fs *flushFS) GetInodeAttributes(
ctx context.Context,
op *fuseops.GetInodeAttributesOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -150,6 +154,7 @@ func (fs *flushFS) GetInodeAttributes(
}
func (fs *flushFS) OpenFile(
ctx context.Context,
op *fuseops.OpenFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -164,6 +169,7 @@ func (fs *flushFS) OpenFile(
}
func (fs *flushFS) ReadFile(
ctx context.Context,
op *fuseops.ReadFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -181,6 +187,7 @@ func (fs *flushFS) ReadFile(
}
func (fs *flushFS) WriteFile(
ctx context.Context,
op *fuseops.WriteFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -204,6 +211,7 @@ func (fs *flushFS) WriteFile(
}
func (fs *flushFS) SyncFile(
ctx context.Context,
op *fuseops.SyncFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -213,6 +221,7 @@ func (fs *flushFS) SyncFile(
}
func (fs *flushFS) FlushFile(
ctx context.Context,
op *fuseops.FlushFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -222,6 +231,7 @@ func (fs *flushFS) FlushFile(
}
func (fs *flushFS) OpenDir(
ctx context.Context,
op *fuseops.OpenDirOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -240,6 +250,7 @@ func (fs *flushFS) OpenDir(
}
func (fs *flushFS) ReadDir(
ctx context.Context,
op *fuseops.ReadDirOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......
......@@ -18,6 +18,8 @@ import (
"fmt"
"os"
"golang.org/x/net/context"
"github.com/jacobsa/fuse"
"github.com/jacobsa/fuse/fuseops"
"github.com/jacobsa/fuse/fuseutil"
......@@ -223,6 +225,7 @@ func (fs *fsImpl) findInodeByID(id fuseops.InodeID) (in *inode) {
////////////////////////////////////////////////////////////////////////
func (fs *fsImpl) LookUpInode(
ctx context.Context,
op *fuseops.LookUpInodeOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -258,6 +261,7 @@ func (fs *fsImpl) LookUpInode(
}
func (fs *fsImpl) GetInodeAttributes(
ctx context.Context,
op *fuseops.GetInodeAttributesOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -272,6 +276,7 @@ func (fs *fsImpl) GetInodeAttributes(
}
func (fs *fsImpl) ForgetInode(
ctx context.Context,
op *fuseops.ForgetInodeOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -284,6 +289,7 @@ func (fs *fsImpl) ForgetInode(
}
func (fs *fsImpl) MkDir(
ctx context.Context,
op *fuseops.MkDirOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -315,6 +321,7 @@ func (fs *fsImpl) MkDir(
}
func (fs *fsImpl) CreateFile(
ctx context.Context,
op *fuseops.CreateFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -346,6 +353,7 @@ func (fs *fsImpl) CreateFile(
}
func (fs *fsImpl) OpenFile(
ctx context.Context,
op *fuseops.OpenFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -357,6 +365,7 @@ func (fs *fsImpl) OpenFile(
}
func (fs *fsImpl) OpenDir(
ctx context.Context,
op *fuseops.OpenDirOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......
......@@ -19,6 +19,8 @@ import (
"os"
"strings"
"golang.org/x/net/context"
"github.com/jacobsa/fuse"
"github.com/jacobsa/fuse/fuseops"
"github.com/jacobsa/fuse/fuseutil"
......@@ -147,7 +149,9 @@ func (fs *helloFS) patchAttributes(
attr.Crtime = now
}
func (fs *helloFS) LookUpInode(op *fuseops.LookUpInodeOp) (err error) {
func (fs *helloFS) LookUpInode(
ctx context.Context,
op *fuseops.LookUpInodeOp) (err error) {
// Find the info for the parent.
parentInfo, ok := gInodeInfo[op.Parent]
if !ok {
......@@ -172,6 +176,7 @@ func (fs *helloFS) LookUpInode(op *fuseops.LookUpInodeOp) (err error) {
}
func (fs *helloFS) GetInodeAttributes(
ctx context.Context,
op *fuseops.GetInodeAttributesOp) (err error) {
// Find the info for this inode.
info, ok := gInodeInfo[op.Inode]
......@@ -190,12 +195,14 @@ func (fs *helloFS) GetInodeAttributes(
}
func (fs *helloFS) OpenDir(
ctx context.Context,
op *fuseops.OpenDirOp) (err error) {
// Allow opening any directory.
return
}
func (fs *helloFS) ReadDir(
ctx context.Context,
op *fuseops.ReadDirOp) (err error) {
// Find the info for this inode.
info, ok := gInodeInfo[op.Inode]
......@@ -232,12 +239,14 @@ func (fs *helloFS) ReadDir(
}
func (fs *helloFS) OpenFile(
ctx context.Context,
op *fuseops.OpenFileOp) (err error) {
// Allow opening any file.
return
}
func (fs *helloFS) ReadFile(
ctx context.Context,
op *fuseops.ReadFileOp) (err error) {
// Let io.ReaderAt deal with the semantics.
reader := strings.NewReader("Hello, world!")
......
......@@ -19,6 +19,8 @@ import (
"os"
"sync"
"golang.org/x/net/context"
"github.com/jacobsa/fuse"
"github.com/jacobsa/fuse/fuseops"
"github.com/jacobsa/fuse/fuseutil"
......@@ -78,6 +80,7 @@ func (fs *InterruptFS) WaitForReadInFlight() {
////////////////////////////////////////////////////////////////////////
func (fs *InterruptFS) LookUpInode(
ctx context.Context,
op *fuseops.LookUpInodeOp) (err error) {
// We support only one parent.
if op.Parent != fuseops.RootInodeID {
......@@ -99,6 +102,7 @@ func (fs *InterruptFS) LookUpInode(
}
func (fs *InterruptFS) GetInodeAttributes(
ctx context.Context,
op *fuseops.GetInodeAttributesOp) (err error) {
switch op.Inode {
case fuseops.RootInodeID:
......@@ -116,11 +120,13 @@ func (fs *InterruptFS) GetInodeAttributes(
}
func (fs *InterruptFS) OpenFile(
ctx context.Context,
op *fuseops.OpenFileOp) (err error) {
return
}
func (fs *InterruptFS) ReadFile(
ctx context.Context,
op *fuseops.ReadFileOp) (err error) {
// Signal that a read has been received.
fs.mu.Lock()
......@@ -129,7 +135,7 @@ func (fs *InterruptFS) ReadFile(
fs.mu.Unlock()
// Wait for cancellation.
done := op.Context().Done()
done := ctx.Done()
if done == nil {
panic("Expected non-nil channel.")
}
......@@ -137,7 +143,7 @@ func (fs *InterruptFS) ReadFile(
<-done
// Return the context's error.
err = op.Context().Err()
err = ctx.Err()
return
}
......@@ -20,6 +20,8 @@ import (
"os"
"time"
"golang.org/x/net/context"
"github.com/jacobsa/fuse"
"github.com/jacobsa/fuse/fuseops"
"github.com/jacobsa/fuse/fuseutil"
......@@ -190,6 +192,7 @@ func (fs *memFS) deallocateInode(id fuseops.InodeID) {
////////////////////////////////////////////////////////////////////////
func (fs *memFS) LookUpInode(
ctx context.Context,
op *fuseops.LookUpInodeOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -220,6 +223,7 @@ func (fs *memFS) LookUpInode(
}
func (fs *memFS) GetInodeAttributes(
ctx context.Context,
op *fuseops.GetInodeAttributesOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -238,6 +242,7 @@ func (fs *memFS) GetInodeAttributes(
}
func (fs *memFS) SetInodeAttributes(
ctx context.Context,
op *fuseops.SetInodeAttributesOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -259,6 +264,7 @@ func (fs *memFS) SetInodeAttributes(
}
func (fs *memFS) MkDir(
ctx context.Context,
op *fuseops.MkDirOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -301,6 +307,7 @@ func (fs *memFS) MkDir(
}
func (fs *memFS) CreateFile(
ctx context.Context,
op *fuseops.CreateFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -350,6 +357,7 @@ func (fs *memFS) CreateFile(
}
func (fs *memFS) CreateSymlink(
ctx context.Context,
op *fuseops.CreateSymlinkOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -400,6 +408,7 @@ func (fs *memFS) CreateSymlink(
}
func (fs *memFS) Rename(
ctx context.Context,
op *fuseops.RenameOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -440,6 +449,7 @@ func (fs *memFS) Rename(
}
func (fs *memFS) RmDir(
ctx context.Context,
op *fuseops.RmDirOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -473,6 +483,7 @@ func (fs *memFS) RmDir(
}
func (fs *memFS) Unlink(
ctx context.Context,
op *fuseops.UnlinkOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -500,6 +511,7 @@ func (fs *memFS) Unlink(
}
func (fs *memFS) OpenDir(
ctx context.Context,
op *fuseops.OpenDirOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -517,6 +529,7 @@ func (fs *memFS) OpenDir(
}
func (fs *memFS) ReadDir(
ctx context.Context,
op *fuseops.ReadDirOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -531,6 +544,7 @@ func (fs *memFS) ReadDir(
}
func (fs *memFS) OpenFile(
ctx context.Context,
op *fuseops.OpenFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -548,6 +562,7 @@ func (fs *memFS) OpenFile(
}
func (fs *memFS) ReadFile(
ctx context.Context,
op *fuseops.ReadFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -569,6 +584,7 @@ func (fs *memFS) ReadFile(
}
func (fs *memFS) WriteFile(
ctx context.Context,
op *fuseops.WriteFileOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......@@ -583,6 +599,7 @@ func (fs *memFS) WriteFile(
}
func (fs *memFS) ReadSymlink(
ctx context.Context,
op *fuseops.ReadSymlinkOp) (err error) {
fs.mu.Lock()
defer fs.mu.Unlock()
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment