Commit 7e1f11bc authored by Kirill Smelkov's avatar Kirill Smelkov

X review feedback on newapi

- add nodefs package overview; in particular describe what inode is so
  that there is no confusion about its meaning.

- Draftly implement Lookup / Forget. They work not under global
  rawBridge.mu and still there should be no race of Lookup / Forget due to
  careful locking of inode -> (inode, ichild) + retrying.
  Add description of Forget semantic and what happens when we receive
  forget for a directory for which children are not forgotten yet.

  ( it is too late here now and I did not checked the implementation with
    a fresh head. I thought that it is better to release current state
    for discussion as I likely won't be able to work on newapi for at
    least another week )

- use atomics in DefaultNode setInode/inode; see Lookup and
  corresponding description nearby DefaultNode.setInode for why it is
  needed.

- inode.{lookupCount,nodeID} are now protected not by global
  rawBridge.mu, but instead by inode.mu .

- change Node operation to return Nodes, not Inode. In particulare
  Node.Lookup should now return Node. Inodes are internal index of
  nodefs VFS (see package description) and we should not load filesystem
  implementations to think about them where we can. Also it makes a
  more closed interface when filesystem works in terms it nodes
  completely. Also this way we offload filesystems for caring about
  tricky details of how to create inode for a hardlinked entry (see
  Lookup for details which handles about it)

- Remove Node.Inode -> nodefs.InodeOf(Node). this way there will be no
  possibility to override Node.Inode and we can specify InodeOf semantic
  exactly in API docs.

- unlockNodes: sort is not needed

- lock/unlock Nodes: avoid duplicates (e.g. there can be duplicates if dir/a and
  dir/b are hardlinks to the same file. If we don't avoid duplicates lockNodes will deadlock)

- made some other edits, part of them not complete...
parent 9d7cb89b
......@@ -2,6 +2,48 @@
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
// Package nodefs provides infrastructure to build tree-organized filesystems.
//
// A tree-organized filesystem is similar to UNIX or Plan 9 filesystem: it
// consists of nodes with each node being either a file or a directory. Files
// are located at tree leafs. A directory node can have other nodes as its
// children and refer to each child by name unique through the directory.
// There can be several paths leading from tree root to a particular node,
// known as hard-linking, for example
//
// root
// / \
// dir1 dir2
// \ /
// file
//
// A /-separated string path describes location of a node in the tree. For example
//
// /dir1/file
//
// describes path root → dir1 → file.
//
// Each node is associated with integer ID uniquely identifying the node
// throughout filesystem. The tree-level structure of any filesystem is
// expressed through index-nodes (also known as "inode", see Inode) which
// describe parent/child relation in between nodes and node-ID association.
//
// A particular filesystem should provide nodes with filesystem operations
// implemented as defined by Node interface. When filesystem is mounted, its
// root Node is associated with root of the tree, and the tree is further build
// lazily when nodefs infrastructure needs to lookup children of nodes to
// process client requests. For every new node, the filesystem infrastructure
// automatically builds new index node and links it in the filesystem tree.
// InodeOf can be used to get particular Inode associated with a Node.
//
// XXX ^^^ inodes cleaned on cache clean (FORGET).
//
// XXX describe how to mount.
//
// XXX node example with Lookup.
//
// XXX describe how to pre-add nodes to tree.
//
package nodefs
import (
......@@ -11,6 +53,15 @@ import (
"github.com/hanwen/go-fuse/fuse"
)
// InodeOf returns index-node associated with filesystem node.
//
// The identity of the Inode does not change over the lifetime of
// the node object.
func InodeOf(node Node) *Inode {
return node.inode()
}
/*
NOSUBMIT: how to structure?
......@@ -19,23 +70,27 @@ NOSUBMIT: how to structure?
- one giant interface?
- use raw types as args rather than mimicking Golang signatures?
Every Node implementation must directly or indirectly embed DefaultNode.
*/
type Node interface {
// setInode links the Inode to a Node.
setInode(*Inode)
// Inode must return a non-nil associated inode structure. The
// identity of the Inode may not change over the lifetime of
// the object.
Inode() *Inode
// Lookup finds a child Inode. If a new Inode must be created,
// the inode does not have to be added to the tree.
Lookup(ctx context.Context, name string, out *fuse.EntryOut) (*Inode, fuse.Status)
// setInode and inode are used by nodefs internally to link Inode to a Node.
//
// When a new Node instance is created, e.g. on Lookup, it has nil Inode.
// Nodefs infrastructure will notice this and associate created Node with new Inode.
//
// See InodeOf for public API to retrieve an inode from Node.
inode() *Inode
setInode(*Inode) (set bool)
// Lookup should find a direct child of the node by child name.
//
// VFS makes sure to call Lookup only once for particular (node, name)
// pair.
Lookup(ctx context.Context, name string, out *fuse.EntryOut) (Node, fuse.Status)
Open(ctx context.Context, flags uint32) (fh File, fuseFlags uint32, code fuse.Status)
Create(ctx context.Context, name string, flags uint32, mode uint32) (inode *Inode, fh File, fuseFlags uint32, code fuse.Status)
Create(ctx context.Context, name string, flags uint32, mode uint32) (node Node, fh File, fuseFlags uint32, code fuse.Status)
Read(ctx context.Context, f File, dest []byte, off int64) (fuse.ReadResult, fuse.Status)
......
......@@ -29,9 +29,15 @@ type rawBridge struct {
options Options
root *Inode
  • gitlab helpfully collapsed this diff, so I'm seeing this only now.

  • @hanwen, appologize for the inconvenience. I've tried to avoid it by including ?expand_all_diffs=1 in the URL, but I agree it is not helpful, when by default, some information is hidden.

Please register or sign in to reply
// we maintain index
//
// ID -> inode
//
// so that we can serve FUSE operational requests where nodes are specified by
// their ID.
mu sync.Mutex
nodes []mapEntry
free []uint64
nodes []mapEntry // ID -> Inode; some inodes may be here in "forgotten" state not yet removed
free []uint64 // free ID
files []fileEntry
freeFiles []uint64
......@@ -70,65 +76,308 @@ func NewNodeFS(root Node, opts *Options) fuse.RawFileSystem {
return bridge
}
// XXX kill?
func (b *rawBridge) inode(id uint64, fh uint64) (*Inode, fileEntry) {
b.mu.Lock()
defer b.mu.Unlock()
// b.mu.Lock()
// defer b.mu.Unlock() // XXX not good to return with released lock
  • why? I don't see much alternative: we have to call into user-supplied Operations, which we can't do under lock.

  • @hanwen, this edit was a draft - my idea wa that rawBridge should be called under lock already so that upon return callers have the guaranty that things don't change under them so that they can do other checks/edits consistently. If you look at Lookup and Forget handlers they were reworked (IIRC) not to use b.inode() at all. Once I get to work on Forget/Lookup again I'll try to clean everything up.

    P.S. you are commenting on the first version of my review. saying just in case that is not missed.

Please register or sign in to reply
return b.nodes[id].inode, b.files[fh]
}
func (b *rawBridge) Lookup(header *fuse.InHeader, name string, out *fuse.EntryOut) (status fuse.Status) {
parent, _ := b.inode(header.NodeId, 0)
// Lookup & Forget
//
// Lookup checks inode.children and potentially calls inode.node.Lookup;
// increments ichild.lookupCount. Forget decrements inode.lookupCount and, if
// that reaches 0, unregisters inode from its parents.
//
// To avoid races, whenever we need to update group of inodes (for example
// inode and its parents) we take locks on all inodes in the group.
//
// Before getting to take locks on a group, we start from locking only one
// inode - the inode in question itself. This is required becuase we need to
// first inspect the inode to decide what to do next and what is the inode
// group to change.
//
// Going from only one inode locked to locked inode group requires to unlock
// the inode in the middle. This creates possibility that at the time when
// inode group will be locked, the inode itself was already changed. If the
// inode was indeed changed, we restart the whole procedure.
//
// Inode.changeCounter is used to detect if inode was changed or not - it is
// incremented every time the inode is modified.
//
// rawBridge.mu (the lock for ID -> inode index) is locked either:
//
// - on its own, or
// - always after locking some inodes.
//
// it is never locked the other way (e.g. rawBridge.mu -> inode).
//
//
// Note on FUSE FORGET semantic: Forget(inodeID) means filesystem client
// (usually the kernel) evicts inode from its cache. It can happen that the
// inode is directory and it children inodes are still kept in client cache
// with nlookup > 0. If we see such a forgotten directory, we don't unlink it
// from the tree until all of its children are also forgotten.
child, code := parent.node.Lookup(context.TODO(), name, out)
if !code.Ok() {
if b.options.NegativeTimeout != nil {
out.SetEntryTimeout(*b.options.NegativeTimeout)
func (b *rawBridge) Forget(nodeid, nlookup uint64) {
b.mu.Lock()
inode := b.nodes[nodeid].inode
b.mu.Unlock()
// XXX if inode == nil || already forgoten -> log (client bug)
b.forgetInode(inode, nlookup)
}
func (b *rawBridge) forgetInode(inode *Inode, nlookup uint64) {
// lock inode, extract parents, unlock inode, relock(inode, parents),
// verify that inode had not changed (and retry if it changed), and
// perform unlinking.
retry:
for {
inode.mu.Lock()
if nlookup != 0 {
inode.lookupCount -= nlookup // XXX log if it goes < 0
inode.changeCounter++
}
return code
if inode.lookupCount != 0 {
inode.mu.Unlock()
return
}
// don't perform unlinking if inode was already forgotten.
//
// Examples when inode could be already forgotten:
//
// - if Forget, in between inode -> inode + parents relock, was
// interrupted by Lookup + another Forget -> we could retry here
// with the inode already forgotten.
//
// - if inode is directory and its forgetting was delayed due
// to alive children, the forget could be triggered by last
// forgotten children, which could be simultaneous with new
// Lookup and again Forget.
if inode.nodeID == 0 {
inode.mu.Unlock()
return
}
if len(inode.children) != 0 {
// client forgot about directory, but not about its children.
// delay directory inode forget to until children are alive.
inode.mu.Unlock()
return
}
// relock inode + parents.
// If inode stays the same - perform the forget; retry otherwise.
nlookup = 0 // make sure not to change lookupCount again, if we'll be retrying
lockv := []*Inode{inode}
for p := range inode.parents {
lockv = append(lockv, p.parent)
}
inodeVer := inode.changeCounter
inode.mu.Unlock()
lockNodes(lockv...)
if inodeVer != inode.changeCounter {
unlockNodes(lockv...)
continue retry
}
// we have locks on inode and .parents; let's unlink and forget
// inode. Also check if for unlinked parents forget was delayed,
// and if so and we are last child - do the forget on parent.
delayedForget := map[*Inode]struct{}{}
for p := range inode.parents {
iparent := p.parent
delete(iparent.children, p.name)
iparent.changeCounter++
// if parent was a directory with delayed forget and we
// were the last of its children - now is the time to
// do forget on the parent.
if iparent.lookupCount == 0 && len(iparent.children) == 0 {
delayedForget[iparent] = struct{}{}
}
}
nodeID := inode.nodeID
inode.nodeID = 0 // forgotten
inode.parents = map[parentData]struct{}{} // clear
inode.changeCounter++
unlockNodes(lockv...)
// release nodeID. In the meantime, while we were not holding
// b.mu locked and inode was forgotten, the ID could be already
// reused. So check if it is not the case.
b.mu.Lock()
  • why like this? We hold b.mu in l.209. Couldn't we update b.nodes there?

  • @hanwen, I'm not sure I understand. By l.209 did you mean

    func (b *rawBridge) Forget(nodeid, nlookup uint64) {
            b.mu.Lock()
            inode := b.nodes[nodeid].inode
            b.mu.Unlock()
    
            // XXX if inode == nil || already forgoten -> log (client bug)
            b.forgetInode(inode, nlookup)
    }

    ? (Asking because on l.209 above we hold only inode and iparent locks, not b.mu)

    If so we can't update b.nodes there, because we don't yet know if inode.lookup count will reach zero after -= nlookup, or the inode will stay alive and thus must stay with kept nodeID.

    Or did I misunderstood something? Could you please clarify?

  • I mean, something like

    func (n *Node) forget( ) { n.mu.Lock() n.lookupCount -=nlookup .. n.mu.Unlock()

        lockNodes(..)
         // remove parent/child pointers    

    if n.lookupCount == 0 { b.mu.Lock() n.NodeId = 0 b.nodes[nodeid].node = null b.free = append(b.free, nodeid) b.mu.Unlock() } unlockNodes()

    then there is no "in the meantime" ?

  • @hanwen, I see now. Probably I did it that way because initially I tried to not lock b under inodes at all. It was only after implementing Lookup

                    // now either:
                    //
                    // - iparent & ichild are locked, or
                    // - iparent is locked and ichild was just created anew.
                    //
                    // (what was locked is covered by lockv)
                    //
                    // -> perform `iparent <-> ichild` linking.
                    iparent.setEntry(name, ichild)
                    ichild.lookupCount++ // XXX ichild.changeCounter++ is in setEntry
    
                    // if ichild was created anew - we have to register ID for it.
                    // retrieve child generation while b is locked along the way.
                    b.mu.Lock()
                    if ichildNew {
                            b.registerInode(ichild)
                    }
                    childID := ichild.nodeID
                    childGen := b.nodes[childID].generation
                    b.mu.Unlock()
    
                    // release iparent + (ichild)? locks and we are done
                    unlockNodes(lockv...)
    
                    ...

    where it turned that we still have to lock b from under inode group also locked, but I was in a hurry to "release" my feedback - even in draft state - on that time.

    Offhand what you propose (keeping inode.nodeID under bridge.mu and then there is no "in the meantime") looks reasonable.

    Thanks for bringing this up,
    I will try to have a closer look at everything, including this, on the next newapi iteration on my side.

    Kirill

Please register or sign in to reply
if b.nodes[nodeID].inode == inode {
b.free = append(b.free, nodeID)
b.nodes[nodeID].inode = nil
}
b.mu.Unlock()
// execute delayed forget on parents
for iparent := range delayedForget {
b.forgetInode(iparent, 0)
}
// done
return
}
}
func (b *rawBridge) Lookup(header *fuse.InHeader, name string, out *fuse.EntryOut) (status fuse.Status) {
b.mu.Lock()
defer b.mu.Unlock()
pentry := b.nodes[header.NodeId]
b.mu.Unlock()
lockNodes(parent, child)
parent.addLookup(name, child)
unlockNodes(parent, child)
iparent := pentry.inode
if child.nodeID == 0 {
b.registerInode(child)
if iparent == nil {
return fuse.ENOENT // FIXME errcode=ok?
}
out.NodeId = child.nodeID
out.Generation = b.nodes[child.nodeID].generation
retry:
for {
iparent.mu.Lock()
if iparent.nodeID == 0 {
iparent.mu.Unlock()
return fuse.ENOENT // forgotten XXX errcode=ok?
}
if b.options.AttrTimeout != nil {
out.SetAttrTimeout(*b.options.AttrTimeout)
}
if b.options.EntryTimeout != nil {
out.SetEntryTimeout(*b.options.EntryTimeout)
}
if iparent.mode&fuse.S_IFDIR == 0 {
iparent.mu.Unlock()
return fuse.ENOTDIR
}
return fuse.OK
// if parent already has inode for this name - we can use it
  • no - the FS must also have the option to change the child here. See https://github.com/hanwen/go-fuse/issues/211

  • @hanwen, thanks for the link. Such sitution should be handled via sending FUSE invalidation messages. Though I agree that if there is no way to get the invalidation from backend (and convey it to kernel) there should be an option to always call the Lookup. This should be optional though and explicitly requested either by MountOptions globally, or by particular node.

  • that is what LookupKnownChildren is for, I think it is a bit of a wart, which I want to fix with the new API. If the kernel calls LOOKUP on us, it's our job to relay that to the user code, and they can decide to return a known node or not.

  • @hanwen, both behaviours might be needed. For example in my current filesystem I precreate nodes as child of / and in /.Lookup it is convenient to handle calls with knowing nodes were not yet created. In general for a filesystem that cares to properly invalidate FUSE client about its changes, it is better to rely on receiving LOOKUP only for unknown nodes.

Please register or sign in to reply
ichild := iparent.children[name]
if ichild != nil {
// relock iparent + ichild; retry if iparent changed
iparentVer := iparent.changeCounter
iparent.mu.Unlock()
lockNodes(iparent, ichild)
if iparentVer != iparent.changeCounter {
unlockNodes(iparent, ichild)
continue retry
}
}
// iparent & ichild are locked; ichild might be nil.
lockv := []*Inode{iparent, ichild}
ichildNew := false
if ichild == nil {
// this goroutine becomes reponsible to call node.Lookup
// TODO place lookup-in-progress inode and do Lookup without iparent lock
child, code := iparent.node.Lookup(context.TODO(), name, out)
  • this is calling user code under lock, which is dangerous. In particular, the few unittests that exist for this new package now all hang because they use Inode.GetPath to get a filename, which locks parent nodes.

    Is there a way that we can avoid holding locks when we call into user code?

  • @hanwen, yes this is known badness here and the above TODO place lookup-in-progress inode and do Lookup without iparent lock is exactly about this. I just did not have time to do it and did not want to delay posting my changes for at least another week (or maybe more - the time when I'll be able to work on newapi again).

    Usually in such sitution what I do is: add ready chan to inode, the first one who initialtes the lookup puts this inode in non-ready state into children, and calls Lookup not from under lock. Other users, who do concurrent lookups, find the inode, and before actually using it, wait on inode.ready. It should be more or less straightforward to do.

Please register or sign in to reply
if !code.Ok() {
iparent.mu.Unlock()
if b.options.NegativeTimeout != nil {
out.SetEntryTimeout(*b.options.NegativeTimeout)
}
return code
}
ichild := newInode(child, out.Attr.Mode) // XXX + bridge
ichildNew = true
ok := child.setInode(ichild)
if !ok {
// someone concurrently going to this node (via
// different path), already set node -> inode
// assocation. This can happen e.g. if for
//
// root
// / \
// dir1 dir2
// \ /
// file
//
// dir1.Lookup("file") and dir2.Lookup("file") are executed simultaneously.
//
// we have to use that inode.
ichild = child.inode()
// relock to iparent + ichild and retry if iparent changes.
iparentVer := iparent.changeCounter
iparent.mu.Unlock()
lockNodes(iparent, ichild)
if iparentVer != iparent.changeCounter {
unlockNodes(iparent, ichild)
continue retry
}
ichildNew = false
lockv[1] = ichild
}
}
// now either:
//
// - iparent & ichild are locked, or
// - iparent is locked and ichild was just created anew.
//
// (what was locked is covered by lockv)
//
// -> perform `iparent <-> ichild` linking.
iparent.setEntry(name, ichild)
ichild.lookupCount++ // XXX ichild.changeCounter++ is in setEntry
// if ichild was created anew - we have to register ID for it.
// retrieve child generation while b is locked along the way.
b.mu.Lock()
if ichildNew {
b.registerInode(ichild)
}
childID := ichild.nodeID
childGen := b.nodes[childID].generation
b.mu.Unlock()
// release iparent + (ichild)? locks and we are done
unlockNodes(lockv...)
out.NodeId = childID
out.Generation = childGen
if b.options.AttrTimeout != nil {
out.SetAttrTimeout(*b.options.AttrTimeout)
}
if b.options.EntryTimeout != nil {
out.SetEntryTimeout(*b.options.EntryTimeout)
}
return fuse.OK
}
}
func (b *rawBridge) registerInode(child *Inode) {
// registerInode allocates new inode ID and registers inode with that ID to
// inode ID index.
//
// must be called with inode and b locked.
func (b *rawBridge) registerInode(inode *Inode) {
if l := len(b.free); l > 0 {
last := b.free[l-1]
b.free = b.free[:l-1]
child.nodeID = last
b.nodes[last].inode = child
inode.nodeID = last
b.nodes[last].inode = inode
b.nodes[last].generation++
} else {
last := len(b.nodes)
b.nodes = append(b.nodes, mapEntry{
inode: child,
inode: inode,
})
child.nodeID = uint64(last)
inode.nodeID = uint64(last)
}
inode.changeCounter++
}
func (b *rawBridge) Create(input *fuse.CreateIn, name string, out *fuse.CreateOut) (code fuse.Status) {
panic("TODO - similarly to Lookup")
/*
ctx := context.TODO()
parent, _ := b.inode(input.NodeId, 0)
child, f, flags, code := parent.node.Create(ctx, name, input.Flags, input.Mode)
......@@ -163,8 +412,11 @@ func (b *rawBridge) Create(input *fuse.CreateIn, name string, out *fuse.CreateOu
f.GetAttr(ctx, &out.Attr)
return fuse.OK
*/
}
// XXX move -> near Lookup, so that the interaction between Forget/Lookup is easier to oversee
/*
func (b *rawBridge) Forget(nodeid, nlookup uint64) {
b.mu.Lock()
defer b.mu.Unlock()
......@@ -179,6 +431,7 @@ func (b *rawBridge) Forget(nodeid, nlookup uint64) {
}
}
*/
func (b *rawBridge) SetDebug(debug bool) {}
......
......@@ -6,22 +6,47 @@ package nodefs
import (
"context"
"sync/atomic"
"time"
"unsafe"
"github.com/hanwen/go-fuse/fuse"
)
// DefaultNode must be embedded in a Node implementation.
// DefaultNode provides common base Node functionality.
//
// It must be embedded in any Node implementation.
type DefaultNode struct {
inode *Inode
}
func (dn *DefaultNode) setInode(n *Inode) {
dn.inode = n
}
func (dn *DefaultNode) Inode() *Inode {
return dn.inode
inode_ *Inode
}
// set/retrieve inode.
//
// node -> inode association, can be simultaneously tried to be set, if for e.g.
Please register or sign in to reply
//
// root
// / \
// dir1 dir2
// \ /
// file
//
// dir1.Lookup("file") and dir2.Lookup("file") are executed simultaneously.
//
// We use atomics so that only one set can win and rawBridge.Lookup cares to
// cancel inode that loosed.
//
// To read node.inode atomic.LoadPointer is used, however it is not expensive
// since it translates to regular MOVQ on amd64.
func (dn *DefaultNode) setInode(inode *Inode) bool {
return atomic.CompareAndSwapPointer(
(*unsafe.Pointer)(unsafe.Pointer(&dn.inode_)),
nil, unsafe.Pointer(inode))
}
func (dn *DefaultNode) inode() *Inode {
return (*Inode)(atomic.LoadPointer(
(*unsafe.Pointer)(unsafe.Pointer(&dn.inode_))))
}
func (n *DefaultNode) Read(ctx context.Context, f File, dest []byte, off int64) (fuse.ReadResult, fuse.Status) {
......
......@@ -34,38 +34,93 @@ type Inode struct {
// Following data is mutable.
// Protected by bridge.mu
lookupCount uint64
nodeID uint64
// mu protects the following mutable fields. When locking
// multiple Inodes, locks must be acquired using
// lockNodes/unlockNodes
mu sync.Mutex
// incremented every time the 'children' or 'parents' field is changed.
// changeCounter increments every time the below mutable state
// (lookupCount, nodeID, children, parents) is modified.
//
// This is used in places where we have to relock inode into inode
// group lock, and after locking the group we have to check if inode
// did not changed, and if it changed - retry the operation.
changeCounter uint32
lookupCount uint64
// ID of the inode; 0 if inode was forgotten.
// forgotten inodes are unlinked from parent and children, but could be
// still not yet removed from bridge.nodes .
nodeID uint64
children map[string]*Inode
parents map[parentData]struct{}
}
// newInode creates creates new inode pointing to node.
//
// node -> inode association is NOT set.
// the inode is _not_ yet has
func newInode(node Node, mode uint32) *Inode {
inode := &Inode{
mode: mode ^ 07777,
node: node,
//bridge: n.bridge,
parents: make(map[parentData]struct{}),
}
if mode&fuse.S_IFDIR != 0 {
inode.children = make(map[string]*Inode)
}
//node.setInode(ch)
return inode
}
// sortNodes rearranges inode group in consistent order.
//
// The nodes are ordered by their in-RAM address, which gives consistency
// property: for any A and B inodes, sortNodes will either always order A < B,
// or always order A > B.
//
// See lockNodes where this property is used to avoid deadlock when taking
// locks on inode group.
func sortNodes(ns []*Inode) {
sort.Slice(ns, func(i, j int) bool {
return uintptr(unsafe.Pointer(ns[i])) < uintptr(unsafe.Pointer(ns[j]))
})
}
// lockNodes locks group of inodes.
//
// It always lock the inodes in the same order - to avoid deadlocks.
// It also avoids locking an inode more than once, if it was specified multiple times.
// An example when an inode might be given multiple times is if dir/a and dir/b
// are hardlinked to the same inode and the caller needs to take locks on dir children.
//
// It is valid to give nil nodes - those are simply ignored.
  • you sure? It looks like it would crash.

  • @hanwen, I had not tested it but it should not crash: initilly nprev is nil and nodes, after sorting, will contain nil ones in the beginning (we order them by address and nil has 0 address). Then the check if n != nprev will not allow to call lock on nil node. Maybe we should add a comment in the function itself describing this.

Please register or sign in to reply
func lockNodes(ns ...*Inode) {
sortNodes(ns)
var nprev *Inode
for _, n := range ns {
n.mu.Lock()
if n != nprev {
n.mu.Lock()
nprev = n
}
}
}
// unlockNodes releases locks taken by lockNodes.
func unlockNodes(ns ...*Inode) {
// we don't need to unlock in the same order that was used in lockNodes.
// however it still helps to have nodes sorted to avoid duplicates.
sortNodes(ns)
var nprev *Inode
for _, n := range ns {
n.mu.Unlock()
if n != nprev {
n.mu.Unlock()
nprev = n
}
}
}
......@@ -74,9 +129,14 @@ func unlockNodes(ns ...*Inode) {
// kernel has no way of reviving forgotten nodes by its own
// initiative.
func (n *Inode) Forgotten() bool {
/*
n.bridge.mu.Lock()
defer n.bridge.mu.Unlock()
return n.lookupCount == 0
*/
n.mu.Lock()
defer n.mu.Unlock()
return n.nodeID == 0
}
// Node returns the Node object implementing the file system operations.
......@@ -155,14 +215,24 @@ func (n *Inode) FindChildByOpaqueID(name string, opaqueID uint64) *Inode {
return nil
}
func (n *Inode) addLookup(name string, child *Inode) {
child.lookupCount++
child.parents[parentData{name, n}] = struct{}{}
n.children[name] = child
child.changeCounter++
n.changeCounter++
// setEntry does `iparent[name] = ichild` linking.
//
// setEntry must not be called simultaneously for any of iparent or ichild.
// This, for example could be satisfied if both iparent and ichild are locked,
// but it could be also valid if only iparent is locked and ichild was just
// created and only one goroutine keeps referencing it.
//
// XXX also ichild.lookupCount++ ?
func (iparent *Inode) setEntry(name string, ichild *Inode) {
// ichild.lookupCount++
ichild.parents[parentData{name, iparent}] = struct{}{}
iparent.children[name] = ichild
ichild.changeCounter++
iparent.changeCounter++
}
// XXX kill
/*
func (n *Inode) clearParents() {
for {
lockme := []*Inode{n}
......@@ -191,7 +261,10 @@ func (n *Inode) clearParents() {
}
}
}
*/
// XXX kill
/*
func (n *Inode) clearChildren() {
if n.mode != fuse.S_IFDIR {
return
......@@ -226,12 +299,18 @@ func (n *Inode) clearChildren() {
}
}
// XXX not right - we cannot fully clear our children, because they can
// be also children of another directory.
  • is this possible for directories?

  • Traditionally they forbid creating hardlinks for directories to e.g. (if i understand correctly) avoid deadlocks in simple lookup schemes where every inode on the path is locked. However it is possible to both prepare ext4 image with such hardlinks and a FUSE filesystem server that will create such hardlinks. I think we should not misbehave on such cases, especially since handling them properly is not a problem for us. See also about delayed forgets + below.

Please register or sign in to reply
//
// XXX also not right - the kernel can send FORGET(idir) but keep
  • are you sure? Would that not break libfuse, because then you can't discover the full path anymore.

  • What I did:

    • I studied fs/fuse/ in Linux to see when it sends forget. What I saw was essentially that a forget is queued whenever pagecache wants to evict a node:

    https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/tree/fs/fuse/inode.c?id=cb5b020a8d38#n134 https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/tree/fs/fuse/inode.c?id=cb5b020a8d38#n815

    • I did not studied how Linux evicts inodes from pagecache. But offhand to me it seems, at least theoreticlly, possible that after a file inode was looked up, opened and being used, under memory pressure its directory inode could be evicted. I admit that kernel implementation might want to keep the directory inode, but please see below

    • Even though in practice FUSE is not specified and strongly tied to Linux VFS semantics, it is still more correct to think about FUSE as about network protocol in the first place. From this point of view we (filesystem server) should not blindly trust our client (usually kernel) to not misbehave and to be bug free. This is especially true since there are several FUSE clients out there (Linux, MacOS, FreeBSD, ...) and that event just for Linux for years they are having trouble to document what VFS semantics are: https://lwn.net/Articles/779747.

    • Even for libfuse there is a history when kernel was sending something that was causing abort in the library, see e.g. https://git.kernel.org/linus/50322fe7d4, ...

    With all those arguments, should we trust the kernel (our client) not to misbehave? Is it impossible on the protocol level to send FORGET for idir without sending FORGET for ifile that was under it?

    To me the answer is: we should not blindly trust client and we should not misbehave/crash if client sends something unexpected. That's why in my implementation FORGET(idir) is delayed untill all its children are gone.

Please register or sign in to reply
// references to children inodes.
for _, ch := range lockme {
if ch != n {
ch.clearChildren()
}
}
}
*/
// NewPersistentInode returns an Inode with a LookupCount == 1, ie. the
// node will only get garbage collected if the kernel issues a forget
......
......@@ -37,11 +37,11 @@ type loopbackNode struct {
}
func (n *loopbackNode) path() string {
path := n.Inode().Path(nil)
path := InodeOf(n).Path(nil)
return filepath.Join(n.rootNode.root, path)
}
func (n *loopbackNode) Lookup(ctx context.Context, name string, out *fuse.EntryOut) (*Inode, fuse.Status) {
func (n *loopbackNode) Lookup(ctx context.Context, name string, out *fuse.EntryOut) (Node, fuse.Status) {
p := filepath.Join(n.path(), name)
st := syscall.Stat_t{}
......@@ -52,17 +52,18 @@ func (n *loopbackNode) Lookup(ctx context.Context, name string, out *fuse.EntryO
out.Attr.FromStat(&st)
ch := n.Inode().FindChildByOpaqueID(name, out.Attr.Ino)
ch := InodeOf(n).FindChildByOpaqueID(name, out.Attr.Ino)
if ch != nil {
return ch, fuse.OK
return ch.Node(), fuse.OK
}
node := &loopbackNode{rootNode: n.rootNode}
ch = n.Inode().NewInode(node, out.Attr.Mode, out.Attr.Ino)
return ch, fuse.OK
return node, fuse.OK
// ch = n.Inode().NewInode(node, out.Attr.Mode, out.Attr.Ino)
// return ch, fuse.OK
}
func (n *loopbackNode) Create(ctx context.Context, name string, flags uint32, mode uint32) (inode *Inode, fh File, fuseFlags uint32, code fuse.Status) {
func (n *loopbackNode) Create(ctx context.Context, name string, flags uint32, mode uint32) (node Node, fh File, fuseFlags uint32, code fuse.Status) {
p := filepath.Join(n.path(), name)
f, err := os.OpenFile(p, int(flags)|os.O_CREATE, os.FileMode(mode))
......@@ -76,9 +77,9 @@ func (n *loopbackNode) Create(ctx context.Context, name string, flags uint32, mo
return nil, nil, 0, fuse.ToStatus(err)
}
node := &loopbackNode{rootNode: n.rootNode}
ch := n.Inode().NewInode(node, st.Mode, st.Ino)
return ch, NewLoopbackFile(f), 0, fuse.OK
node = &loopbackNode{rootNode: n.rootNode}
// ch := n.Inode().NewInode(node, st.Mode, st.Ino)
return node, NewLoopbackFile(f), 0, fuse.OK
}
func (n *loopbackNode) Open(ctx context.Context, flags uint32) (fh File, fuseFlags uint32, code fuse.Status) {
......
......@@ -6,10 +6,10 @@ package nodefs
import (
"bytes"
"io"
"io/ioutil"
"os"
"path/filepath"
"runtime"
"testing"
"time"
......
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