Commit 03d762c0 authored by Han-Wen Nienhuys's avatar Han-Wen Nienhuys

nodefs: get rid of InodeLink

Instead, define InodeEmbedder, an interface that can only be satisfied
by a struct that embeds Inode.
parent fed41d2f
......@@ -28,16 +28,23 @@
// expressed through index-nodes (also known as "inode", see Inode) which
// describe parent/child relation in between nodes and node-ID association.
//
// All error reporting must use the syscall.Errno type. The value 0
// (`OK`) should be used to indicate success. The method names are
// inspired on the system call names, so we have Listxattr rather than
// ListXAttr.
// The filesystem nodes are struct that embed the Inode type, so they
// comply with the InodeEmbedder interface. They should be
// initialized by calling NewInode or NewPersistentInode before being
// manipulated further, eg.
//
//
// type myNode struct {
// Inode
// }
//
// func (n *myNode) Lookup(ctx context.Context, name string, ... ) (*Inode, syscall.Errno) {
// child := myNode{}
// return n.NewInode(ctx, &myNode{}, NodeAttr{Mode: syscall.S_IFDIR}), 0
// }
//
// A filesystem should provide nodes that embed InodeEmbed, so the
// node comply with the InodeLink interface. When filesystem is
// mounted, its root InodeEmbed is associated with root of the tree,
// and the tree is further built lazily when nodefs infrastructure
// needs to lookup children of nodes to process client requests.
// On mounting, the root InodeEmbedder is associated with root of the
// tree.
//
// The kernel can evict inode data to free up memory. It does so by
// issuing FORGET calls. When a node has no children, and no kernel
......@@ -55,6 +62,12 @@
// systems should implement Lookuper instead. The loopback file
// system created by `NewLoopbackRoot` provides a straightforward
// example.
//
// All error reporting must use the syscall.Errno type. The value 0
// (`OK`) should be used to indicate success. The method names are
// inspired on the system call names, so we have Listxattr rather than
// ListXAttr.
//
package nodefs
......@@ -66,26 +79,21 @@ import (
"github.com/hanwen/go-fuse/fuse"
)
// InodeLink provides the machinery to connect user defined methods to
// an Inode (a node in the filesystem tree).
// InodeEmbedder is an interface for structs that embed Inode.
//
// In general, if InodeLink does not implement specific filesystem
// methods, the filesystem will react as if it is a read-only
// filesystem with a predefined tree structure. See zipfs_test.go for
// an example. A example is in zip_test.go
type InodeLink interface {
// In general, if an InodeEmbedder does not implement specific
// filesystem methods, the filesystem will react as if it is a
// read-only filesystem with a predefined tree structure. See
// zipfs_test.go for an example. A example is in zip_test.go
type InodeEmbedder interface {
// populateInode and inode are used by nodefs internally to
// link Inode to a Node.
//
// See Inode() for the public API to retrieve an inode from Node.
inode() *Inode
init(ops InodeLink, attr NodeAttr, bridge *rawBridge, persistent bool)
embed() *Inode
// Inode returns the *Inode associated with this Operations
// instance. The identity of the Inode does not change over
// the lifetime of the node object. Inode() is provided by
// OperationStubs, and should not be reimplemented.
Inode() *Inode
// EmbeddedInode returns a pointer to the embedded inode.
EmbeddedInode() *Inode
}
// Statfs implements statistics for the filesystem that holds this
......@@ -119,8 +127,7 @@ type Setattrer interface {
Setattr(ctx context.Context, f FileHandle, in *fuse.SetAttrIn, out *fuse.AttrOut) syscall.Errno
}
// OnAdd is called once this Operations object is attached to
// an Inode.
// OnAdd is called when this InodeEmbedder is initialized.
type OnAdder interface {
OnAdd(ctx context.Context)
}
......@@ -196,7 +203,7 @@ type Flusher interface {
}
// This is called to before the file handle is forgotten. The
// kernel ingores the return value of this method,
// kernel ignores the return value of this method,
// so any cleanup that requires specific synchronization or
// could fail with I/O errors should happen in Flush instead.
// The default implementation forwards to the FileHandle.
......@@ -308,7 +315,7 @@ type Mknoder interface {
// Link is similar to Lookup, but must create a new link to an existing Inode.
// Default is to return EROFS.
type Linker interface {
Link(ctx context.Context, target InodeLink, name string, out *fuse.EntryOut) (node *Inode, errno syscall.Errno)
Link(ctx context.Context, target InodeEmbedder, name string, out *fuse.EntryOut) (node *Inode, errno syscall.Errno)
}
// Symlink is similar to Lookup, but must create a new symbolic link.
......@@ -343,7 +350,7 @@ type Rmdirer interface {
// OK.
// Default is to return EROFS.
type Renamer interface {
Rename(ctx context.Context, name string, newParent InodeLink, newName string, flags uint32) syscall.Errno
Rename(ctx context.Context, name string, newParent InodeEmbedder, newName string, flags uint32) syscall.Errno
}
// FileHandle is a resource identifier for opened files. FileHandles
......
......@@ -49,7 +49,7 @@ type rawBridge struct {
}
// newInode creates creates new inode pointing to ops.
func (b *rawBridge) newInode(ctx context.Context, ops InodeLink, id NodeAttr, persistent bool) *Inode {
func (b *rawBridge) newInode(ctx context.Context, ops InodeEmbedder, id NodeAttr, persistent bool) *Inode {
b.mu.Lock()
defer b.mu.Unlock()
......@@ -58,8 +58,8 @@ func (b *rawBridge) newInode(ctx context.Context, ops InodeLink, id NodeAttr, pe
}
// This ops already was populated. Just return it.
if ops.inode().bridge != nil {
return ops.inode()
if ops.embed().bridge != nil {
return ops.embed()
}
if id.Ino == 0 {
......@@ -94,12 +94,12 @@ func (b *rawBridge) newInode(ctx context.Context, ops InodeLink, id NodeAttr, pe
id.Mode = fuse.S_IFREG
}
b.nodes[id.Ino] = ops.inode()
ops.init(ops, id, b, persistent)
b.nodes[id.Ino] = ops.embed()
initInode(ops.embed(), ops, id, b, persistent)
if oa, ok := ops.(OnAdder); ok {
oa.OnAdd(ctx)
}
return ops.inode()
return ops.embed()
}
// addNewChild inserts the child into the tree. Returns file handle if file != nil.
......@@ -141,7 +141,7 @@ func (b *rawBridge) setAttrTimeout(out *fuse.AttrOut) {
// NewNodeFS creates a node based filesystem based on an Operations
// instance for the root.
func NewNodeFS(root InodeLink, opts *Options) fuse.RawFileSystem {
func NewNodeFS(root InodeEmbedder, opts *Options) fuse.RawFileSystem {
bridge := &rawBridge{
automaticIno: opts.FirstAutomaticIno,
}
......@@ -161,7 +161,7 @@ func NewNodeFS(root InodeLink, opts *Options) fuse.RawFileSystem {
bridge.options.AttrTimeout = &oneSec
}
root.init(root,
initInode(root.embed(), root,
NodeAttr{
Ino: 1,
Mode: fuse.S_IFDIR,
......@@ -169,7 +169,7 @@ func NewNodeFS(root InodeLink, opts *Options) fuse.RawFileSystem {
bridge,
false,
)
bridge.root = root.inode()
bridge.root = root.embed()
bridge.root.lookupCount = 1
bridge.nodes = map[uint64]*Inode{
1: bridge.root,
......
......@@ -19,7 +19,7 @@ import (
)
type keepCacheFile struct {
InodeEmbed
Inode
keepCache bool
mu sync.Mutex
......@@ -66,7 +66,7 @@ var _ = (Opener)((*keepCacheFile)(nil))
var _ = (Getattrer)((*keepCacheFile)(nil))
type keepCacheRoot struct {
InodeEmbed
Inode
keep, nokeep *keepCacheFile
}
......@@ -74,7 +74,7 @@ type keepCacheRoot struct {
var _ = (OnAdder)((*keepCacheRoot)(nil))
func (r *keepCacheRoot) OnAdd(ctx context.Context) {
i := r.Inode()
i := &r.Inode
r.keep = &keepCacheFile{
keepCache: true,
......@@ -119,7 +119,7 @@ func TestKeepCache(t *testing.T) {
t.Errorf("keep read 2 got %q want read 1 %q", c2, c1)
}
if s := root.keep.Inode().NotifyContent(0, 100); s != OK {
if s := root.keep.Inode.NotifyContent(0, 100); s != OK {
t.Errorf("NotifyContent: %v", s)
}
......
......@@ -3,38 +3,3 @@
// license that can be found in the LICENSE file.
package nodefs
import (
"github.com/hanwen/go-fuse/fuse"
)
// InodeEmbed embeds the Inode into a filesystem node. It is the only
// type that implements the InodeLink interface, and hence, it must be
// part of any implementation of Operations.
type InodeEmbed struct {
inode_ Inode
}
var _ = (InodeLink)((*InodeEmbed)(nil))
func (n *InodeEmbed) inode() *Inode {
return &n.inode_
}
func (n *InodeEmbed) init(ops InodeLink, attr NodeAttr, bridge *rawBridge, persistent bool) {
n.inode_ = Inode{
ops: ops,
nodeAttr: attr,
bridge: bridge,
persistent: persistent,
parents: make(map[parentData]struct{}),
}
if attr.Mode == fuse.S_IFDIR {
n.inode_.children = make(map[string]*Inode)
}
}
// Inode returns the Inode for this Operations
func (n *InodeEmbed) Inode() *Inode {
return &n.inode_
}
......@@ -17,11 +17,11 @@ import (
)
type dioRoot struct {
InodeEmbed
Inode
}
func (r *dioRoot) OnAdd(ctx context.Context) {
r.Inode().AddChild("file", r.Inode().NewInode(ctx, &dioFile{}, NodeAttr{}), false)
r.Inode.AddChild("file", r.Inode.NewInode(ctx, &dioFile{}, NodeAttr{}), false)
}
// A file handle that pretends that every hole/data starts at
......@@ -44,7 +44,7 @@ func (fh *dioFH) Read(ctx context.Context, data []byte, off int64) (fuse.ReadRes
// overrides Open so it can return a dioFH file handle
type dioFile struct {
InodeEmbed
Inode
}
var _ = (Opener)((*dioFile)(nil))
......
......@@ -13,6 +13,8 @@ import (
"sync"
"syscall"
"unsafe"
"github.com/hanwen/go-fuse/fuse"
)
type parentData struct {
......@@ -50,10 +52,14 @@ func (i *NodeAttr) Reserved() bool {
// Operations instances, which is the extension interface for file
// systems. One can create fully-formed trees of Inodes ahead of time
// by creating "persistent" Inodes.
//
// The Inode struct contains a lock, so it should not be
// copied. Inodes should be obtained by calling Inode.NewInode() or
// Inode.NewPersistentInode().
type Inode struct {
nodeAttr NodeAttr
ops InodeLink
ops InodeEmbedder
bridge *rawBridge
// Following data is mutable.
......@@ -90,6 +96,25 @@ type Inode struct {
parents map[parentData]struct{}
}
func (n *Inode) embed() *Inode {
return n
}
func (n *Inode) EmbeddedInode() *Inode {
return n
}
func initInode(n *Inode, ops InodeEmbedder, attr NodeAttr, bridge *rawBridge, persistent bool) {
n.ops = ops
n.nodeAttr = attr
n.bridge = bridge
n.persistent = persistent
n.parents = make(map[parentData]struct{})
if attr.Mode == fuse.S_IFDIR {
n.children = make(map[string]*Inode)
}
}
// NodeAttr returns the (Ino, Gen) tuple for this node.
func (n *Inode) NodeAttr() NodeAttr {
return n.nodeAttr
......@@ -202,7 +227,7 @@ func (n *Inode) Forgotten() bool {
// Operations returns the object implementing the file system
// operations.
func (n *Inode) Operations() InodeLink {
func (n *Inode) Operations() InodeEmbedder {
return n.ops
}
......@@ -261,7 +286,7 @@ func (iparent *Inode) setEntry(name string, ichild *Inode) {
// NewPersistentInode returns an Inode whose lifetime is not in
// control of the kernel.
func (n *Inode) NewPersistentInode(ctx context.Context, node InodeLink, id NodeAttr) *Inode {
func (n *Inode) NewPersistentInode(ctx context.Context, node InodeEmbedder, id NodeAttr) *Inode {
return n.newInode(ctx, node, id, true)
}
......@@ -272,16 +297,16 @@ func (n *Inode) ForgetPersistent() {
n.removeRef(0, true)
}
// NewInode returns an inode for the given InodeLink. The mode should
// be standard mode argument (eg. S_IFDIR). The inode number in id.Ino
// argument is used to implement hard-links. If it is given, and
// another node with the same ID is known, that will node will be
// NewInode returns an inode for the given InodeEmbedder. The mode
// should be standard mode argument (eg. S_IFDIR). The inode number in
// id.Ino argument is used to implement hard-links. If it is given,
// and another node with the same ID is known, that will node will be
// returned, and the passed-in `node` is ignored.
func (n *Inode) NewInode(ctx context.Context, ops InodeLink, id NodeAttr) *Inode {
func (n *Inode) NewInode(ctx context.Context, node InodeEmbedder, id NodeAttr) *Inode {
return n.newInode(ctx, ops, id, false)
}
func (n *Inode) newInode(ctx context.Context, ops InodeLink, id NodeAttr, persistent bool) *Inode {
func (n *Inode) newInode(ctx context.Context, ops InodeEmbedder, id NodeAttr, persistent bool) *Inode {
return n.bridge.newInode(ctx, ops, id, persistent)
}
......
......@@ -17,7 +17,7 @@ import (
)
type interruptRoot struct {
InodeEmbed
Inode
child interruptOps
}
......@@ -27,7 +27,7 @@ func (r *interruptRoot) Lookup(ctx context.Context, name string, out *fuse.Entry
if name != "file" {
return nil, syscall.ENOENT
}
ch := r.Inode().NewInode(ctx, &r.child, NodeAttr{
ch := r.Inode.NewInode(ctx, &r.child, NodeAttr{
Ino: 2,
Gen: 1})
......@@ -35,7 +35,7 @@ func (r *interruptRoot) Lookup(ctx context.Context, name string, out *fuse.Entry
}
type interruptOps struct {
InodeEmbed
Inode
interrupted bool
}
......
......@@ -20,6 +20,11 @@ type loopbackRoot struct {
rootDev uint64
}
type loopbackNode struct {
Inode
}
var _ = (Statfser)((*loopbackNode)(nil))
var _ = (Statfser)((*loopbackNode)(nil))
var _ = (Getattrer)((*loopbackNode)(nil))
var _ = (Getxattrer)((*loopbackNode)(nil))
......@@ -61,16 +66,12 @@ func (n *loopbackRoot) Getattr(ctx context.Context, out *fuse.AttrOut) syscall.E
return OK
}
type loopbackNode struct {
InodeEmbed
}
func (n *loopbackNode) root() *loopbackRoot {
return n.Inode().Root().Operations().(*loopbackRoot)
return n.Root().Operations().(*loopbackRoot)
}
func (n *loopbackNode) path() string {
path := n.Inode().Path(nil)
path := n.Path(nil)
return filepath.Join(n.root().rootPath, path)
}
......@@ -85,7 +86,7 @@ func (n *loopbackNode) Lookup(ctx context.Context, name string, out *fuse.EntryO
out.Attr.FromStat(&st)
node := &loopbackNode{}
ch := n.inode().NewInode(ctx, node, n.root().idFromStat(&st))
ch := n.NewInode(ctx, node, n.root().idFromStat(&st))
return ch, 0
}
......@@ -104,7 +105,7 @@ func (n *loopbackNode) Mknod(ctx context.Context, name string, mode, rdev uint32
out.Attr.FromStat(&st)
node := &loopbackNode{}
ch := n.inode().NewInode(ctx, node, n.root().idFromStat(&st))
ch := n.NewInode(ctx, node, n.root().idFromStat(&st))
return ch, 0
}
......@@ -124,7 +125,7 @@ func (n *loopbackNode) Mkdir(ctx context.Context, name string, mode uint32, out
out.Attr.FromStat(&st)
node := &loopbackNode{}
ch := n.inode().NewInode(ctx, node, n.root().idFromStat(&st))
ch := n.NewInode(ctx, node, n.root().idFromStat(&st))
return ch, 0
}
......@@ -141,14 +142,14 @@ func (n *loopbackNode) Unlink(ctx context.Context, name string) syscall.Errno {
return ToErrno(err)
}
func toLoopbackNode(op InodeLink) *loopbackNode {
func toLoopbackNode(op InodeEmbedder) *loopbackNode {
if r, ok := op.(*loopbackRoot); ok {
return &r.loopbackNode
}
return op.(*loopbackNode)
}
func (n *loopbackNode) Rename(ctx context.Context, name string, newParent InodeLink, newName string, flags uint32) syscall.Errno {
func (n *loopbackNode) Rename(ctx context.Context, name string, newParent InodeEmbedder, newName string, flags uint32) syscall.Errno {
newParentLoopback := toLoopbackNode(newParent)
if flags&RENAME_EXCHANGE != 0 {
return n.renameExchange(name, newParentLoopback, newName)
......@@ -195,7 +196,7 @@ func (n *loopbackNode) Create(ctx context.Context, name string, flags uint32, mo
}
node := &loopbackNode{}
ch := n.inode().NewInode(ctx, node, n.root().idFromStat(&st))
ch := n.NewInode(ctx, node, n.root().idFromStat(&st))
lf := NewLoopbackFile(fd)
return ch, lf, 0, 0
}
......@@ -212,13 +213,13 @@ func (n *loopbackNode) Symlink(ctx context.Context, target, name string, out *fu
return nil, ToErrno(err)
}
node := &loopbackNode{}
ch := n.inode().NewInode(ctx, node, n.root().idFromStat(&st))
ch := n.NewInode(ctx, node, n.root().idFromStat(&st))
out.Attr.FromStat(&st)
return ch, 0
}
func (n *loopbackNode) Link(ctx context.Context, target InodeLink, name string, out *fuse.EntryOut) (*Inode, syscall.Errno) {
func (n *loopbackNode) Link(ctx context.Context, target InodeEmbedder, name string, out *fuse.EntryOut) (*Inode, syscall.Errno) {
p := filepath.Join(n.path(), name)
targetNode := toLoopbackNode(target)
......@@ -232,7 +233,7 @@ func (n *loopbackNode) Link(ctx context.Context, target InodeLink, name string,
return nil, ToErrno(err)
}
node := &loopbackNode{}
ch := n.inode().NewInode(ctx, node, n.root().idFromStat(&st))
ch := n.NewInode(ctx, node, n.root().idFromStat(&st))
out.Attr.FromStat(&st)
return ch, 0
......@@ -292,7 +293,7 @@ func (n *loopbackNode) Getattr(ctx context.Context, out *fuse.AttrOut) syscall.E
// NewLoopback returns a root node for a loopback file system whose
// root is at the given root.
func NewLoopbackRoot(root string) (InodeLink, error) {
func NewLoopbackRoot(root string) (InodeEmbedder, error) {
var st syscall.Stat_t
err := syscall.Stat(root, &st)
if err != nil {
......
......@@ -49,7 +49,7 @@ func (n *loopbackNode) renameExchange(name string, newparent *loopbackNode, newN
}
// Double check that nodes didn't change from under us.
inode := n.Inode()
inode := &n.Inode
if inode.Root() != inode && inode.NodeAttr().Ino != n.root().idFromStat(&st).Ino {
return syscall.EBUSY
}
......@@ -57,7 +57,7 @@ func (n *loopbackNode) renameExchange(name string, newparent *loopbackNode, newN
return ToErrno(err)
}
newinode := newparent.Inode()
newinode := &newparent.Inode
if newinode.Root() != newinode && newinode.NodeAttr().Ino != n.root().idFromStat(&st).Ino {
return syscall.EBUSY
}
......
......@@ -14,7 +14,7 @@ import (
// requests. This is a convenience wrapper around NewNodeFS and
// fuse.NewServer. If nil is given as options, default settings are
// applied, which are 1 second entry and attribute timeout.
func Mount(dir string, root InodeLink, options *Options) (*fuse.Server, error) {
func Mount(dir string, root InodeEmbedder, options *Options) (*fuse.Server, error) {
if options == nil {
oneSec := time.Second
options = &Options{
......
......@@ -28,7 +28,7 @@ type testCase struct {
origDir string
mntDir string
loopback InodeLink
loopback InodeEmbedder
rawFS fuse.RawFileSystem
server *fuse.Server
}
......@@ -437,7 +437,7 @@ func TestNotifyEntry(t *testing.T) {
t.Fatalf("got after %#v, want %#v", after, st)
}
if errno := tc.loopback.Inode().NotifyEntry("file"); errno != 0 {
if errno := tc.loopback.EmbeddedInode().NotifyEntry("file"); errno != 0 {
t.Errorf("notify failed: %v", errno)
}
......
......@@ -106,7 +106,7 @@ func TestZipFS(t *testing.T) {
// zipFile is a file read from a zip archive.
type zipFile struct {
InodeEmbed
Inode
file *zip.File
mu sync.Mutex
......@@ -158,7 +158,7 @@ func (zf *zipFile) Read(ctx context.Context, f FileHandle, dest []byte, off int6
// zipRoot is the root of the Zip filesystem. Its only functionality
// is populating the filesystem.
type zipRoot struct {
InodeEmbed
Inode
r *zip.Reader
}
......@@ -173,14 +173,14 @@ func (zr *zipRoot) OnAdd(ctx context.Context) {
for _, f := range zr.r.File {
dir, base := filepath.Split(f.Name)
p := zr.Inode()
p := &zr.Inode
for _, component := range strings.Split(dir, "/") {
if len(component) == 0 {
continue
}
ch := p.GetChild(component)
if ch == nil {
ch = p.NewPersistentInode(ctx, &InodeEmbed{},
ch = p.NewPersistentInode(ctx, &Inode{},
NodeAttr{Mode: fuse.S_IFDIR})
p.AddChild(component, ch, true)
}
......
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