Commit 1d0096e5 authored by Jakob Unterwurzacher's avatar Jakob Unterwurzacher Committed by Han-Wen Nienhuys

fs: split inode number tracking and kernel nodeid tracking

Using the inode numbers as the nodeid causes problems
when the fs reuses inode numbers. This is the case with
any overlay filesystem that is backed by ext4 like the
loopback example or gocryptfs.

We already had the expSleep() and re-add hack,

	7090b027 fs: wait out apparent inode type change
	68f70527 fs: addNewChild(): handle concurrent FORGETs

to mitigate some of the problems (at the risk of deadlocking
forever), but I could not find a way to work around the following
case uncovered by fsstress:

The kernel expects a fresh nodeid from MKDIR (see #372 for a
debug log).

This is now guaranteed by passing the O_EXCL to addNewChild().
However, this also means that the hard link detection must
happen in addNewChild() as opposed to newInodeUnlocked()
before. The expSleep and re-add hacks are no longer needed
and have been dropped.

This survived 24 hours (42587 iterations) of fsstress testing.
Tested was the loopback example running on top of ext4 on
Linux 5.8.10.

Fixes https://github.com/hanwen/go-fuse/issues/372 .

v2: Rename inoMap -> stableAttrs, nodeidMap -> kernelNodeIds
    acc. to feedback from Han-Wen.
v3: initialize Inode.nodeId early
v4: drop leftover file stash.txt

Change-Id: Ibb36a886f15d48727daa10b9717ea88e45a6b8af
parent 6a785da2
...@@ -7,7 +7,6 @@ package fs ...@@ -7,7 +7,6 @@ package fs
import ( import (
"context" "context"
"log" "log"
"math/rand"
"sync" "sync"
"syscall" "syscall"
"time" "time"
...@@ -61,10 +60,28 @@ type rawBridge struct { ...@@ -61,10 +60,28 @@ type rawBridge struct {
// mu protects the following data. Locks for inodes must be // mu protects the following data. Locks for inodes must be
// taken before rawBridge.mu // taken before rawBridge.mu
mu sync.Mutex mu sync.Mutex
nodes map[uint64]*Inode
// stableAttrs is used to detect already-known nodes and hard links by
// looking at:
// 1) file type ......... StableAttr.Mode
// 2) inode number ...... StableAttr.Ino
// 3) generation number . StableAttr.Gen
stableAttrs map[StableAttr]*Inode
automaticIno uint64 automaticIno uint64
// The *Node ID* is an arbitrary uint64 identifier chosen by the FUSE library.
// It is used the identify *nodes* (files/directories/symlinks/...) in the
// communication between the FUSE library and the Linux kernel.
//
// The kernelNodeIds map translates between the NodeID and the corresponding
// go-fuse Inode object.
//
// A simple incrementing counter is used as the NodeID (see `nextNodeID`).
kernelNodeIds map[uint64]*Inode
// nextNodeID is the next free NodeID. Increment after copying the value.
nextNodeId uint64
files []*fileEntry files []*fileEntry
freeFiles []uint32 freeFiles []uint32
} }
...@@ -83,55 +100,26 @@ func (b *rawBridge) newInodeUnlocked(ops InodeEmbedder, id StableAttr, persisten ...@@ -83,55 +100,26 @@ func (b *rawBridge) newInodeUnlocked(ops InodeEmbedder, id StableAttr, persisten
return ops.embed() return ops.embed()
} }
// Only the file type bits matter
id.Mode = id.Mode & syscall.S_IFMT
if id.Mode == 0 {
id.Mode = fuse.S_IFREG
}
if id.Ino == 0 { if id.Ino == 0 {
// Find free inode number.
for { for {
id.Ino = b.automaticIno id.Ino = b.automaticIno
b.automaticIno++ b.automaticIno++
_, ok := b.nodes[id.Ino] _, ok := b.stableAttrs[id]
if !ok { if !ok {
break break
} }
} }
} }
// Only the file type bits matter initInode(ops.embed(), ops, id, b, persistent, b.nextNodeId)
id.Mode = id.Mode & syscall.S_IFMT b.nextNodeId++
if id.Mode == 0 {
id.Mode = fuse.S_IFREG
}
// the same node can be looked up through 2 paths in parallel, eg.
//
// root
// / \
// dir1 dir2
// \ /
// file
//
// dir1.Lookup("file") and dir2.Lookup("file") are executed
// simultaneously. The matching StableAttrs ensure that we return the
// same node.
var t time.Duration
t0 := time.Now()
for i := 1; true; i++ {
old := b.nodes[id.Ino]
if old == nil {
break
}
if old.stableAttr == id {
return old
}
b.mu.Unlock()
t = expSleep(t)
if i%5000 == 0 {
b.logf("blocked for %.0f seconds waiting for FORGET on i%d", time.Since(t0).Seconds(), id.Ino)
}
b.mu.Lock()
}
b.nodes[id.Ino] = ops.embed()
initInode(ops.embed(), ops, id, b, persistent)
return ops.embed() return ops.embed()
} }
...@@ -141,21 +129,6 @@ func (b *rawBridge) logf(format string, args ...interface{}) { ...@@ -141,21 +129,6 @@ func (b *rawBridge) logf(format string, args ...interface{}) {
} }
} }
// expSleep sleeps for time `t` and returns an exponentially increasing value
// for the next sleep time, capped at 1 ms.
func expSleep(t time.Duration) time.Duration {
if t == 0 {
return time.Microsecond
}
time.Sleep(t)
// Next sleep is between t and 2*t
t += time.Duration(rand.Int63n(int64(t)))
if t >= time.Millisecond {
return time.Millisecond
}
return t
}
func (b *rawBridge) newInode(ctx context.Context, ops InodeEmbedder, id StableAttr, persistent bool) *Inode { func (b *rawBridge) newInode(ctx context.Context, ops InodeEmbedder, id StableAttr, persistent bool) *Inode {
ch := b.newInodeUnlocked(ops, id, persistent) ch := b.newInodeUnlocked(ops, id, persistent)
if ch != ops.embed() { if ch != ops.embed() {
...@@ -169,34 +142,81 @@ func (b *rawBridge) newInode(ctx context.Context, ops InodeEmbedder, id StableAt ...@@ -169,34 +142,81 @@ func (b *rawBridge) newInode(ctx context.Context, ops InodeEmbedder, id StableAt
} }
// addNewChild inserts the child into the tree. Returns file handle if file != nil. // addNewChild inserts the child into the tree. Returns file handle if file != nil.
func (b *rawBridge) addNewChild(parent *Inode, name string, child *Inode, file FileHandle, fileFlags uint32, out *fuse.EntryOut) uint32 { // Unless fileFlags has the syscall.O_EXCL bit set, child.stableAttr will be used
// to find an already-known node. If one is found, `child` is ignored and the
// already-known one is used. The node that was actually used is returned.
func (b *rawBridge) addNewChild(parent *Inode, name string, child *Inode, file FileHandle, fileFlags uint32, out *fuse.EntryOut) (selected *Inode, fh uint32) {
if name == "." || name == ".." { if name == "." || name == ".." {
log.Panicf("BUG: tried to add virtual entry %q to the actual tree", name) log.Panicf("BUG: tried to add virtual entry %q to the actual tree", name)
} }
lockNodes(parent, child)
parent.setEntry(name, child)
b.mu.Lock()
// Due to concurrent FORGETs, lookupCount may have dropped to zero. // the same node can be looked up through 2 paths in parallel, eg.
// This means it MAY have been deleted from nodes[] already. Add it back. //
if child.lookupCount == 0 { // root
b.nodes[child.stableAttr.Ino] = child // / \
// dir1 dir2
// \ /
// file
//
// dir1.Lookup("file") and dir2.Lookup("file") are executed
// simultaneously. The matching StableAttrs ensure that we return the
// same node.
orig := child
id := child.stableAttr
if id.Mode & ^(uint32(syscall.S_IFMT)) != 0 {
log.Panicf("%#v", id)
} }
for {
lockNodes(parent, child)
b.mu.Lock()
if fileFlags&syscall.O_EXCL != 0 {
// must create a new node - don't look for existing nodes
break
}
old := b.stableAttrs[id]
if old == nil {
if child == orig {
// no pre-existing node under this inode number
break
} else {
// old inode disappeared while we were looping here. Go back to
// original child.
b.mu.Unlock()
unlockNodes(parent, child)
child = orig
continue
}
}
if old == child {
// we now have the right inode locked
break
}
// found a different existing node
b.mu.Unlock()
unlockNodes(parent, child)
child = old
}
child.lookupCount++ child.lookupCount++
child.changeCounter++ child.changeCounter++
var fh uint32 b.kernelNodeIds[child.nodeId] = child
// Any node that might be there is overwritten - it is obsolete now
b.stableAttrs[id] = child
if file != nil { if file != nil {
fh = b.registerFile(child, file, fileFlags) fh = b.registerFile(child, file, fileFlags)
} }
out.NodeId = child.stableAttr.Ino parent.setEntry(name, child)
out.NodeId = child.nodeId
out.Generation = child.stableAttr.Gen out.Generation = child.stableAttr.Gen
out.Attr.Ino = child.stableAttr.Ino out.Attr.Ino = child.stableAttr.Ino
b.mu.Unlock() b.mu.Unlock()
unlockNodes(parent, child) unlockNodes(parent, child)
return fh
return child, fh
} }
func (b *rawBridge) setEntryOutTimeout(out *fuse.EntryOut) { func (b *rawBridge) setEntryOutTimeout(out *fuse.EntryOut) {
...@@ -237,6 +257,8 @@ func NewNodeFS(root InodeEmbedder, opts *Options) fuse.RawFileSystem { ...@@ -237,6 +257,8 @@ func NewNodeFS(root InodeEmbedder, opts *Options) fuse.RawFileSystem {
bridge := &rawBridge{ bridge := &rawBridge{
automaticIno: opts.FirstAutomaticIno, automaticIno: opts.FirstAutomaticIno,
server: opts.ServerCallbacks, server: opts.ServerCallbacks,
nextNodeId: 2, // the root node has nodeid 1
stableAttrs: make(map[StableAttr]*Inode),
} }
if bridge.automaticIno == 1 { if bridge.automaticIno == 1 {
bridge.automaticIno++ bridge.automaticIno++
...@@ -256,15 +278,16 @@ func NewNodeFS(root InodeEmbedder, opts *Options) fuse.RawFileSystem { ...@@ -256,15 +278,16 @@ func NewNodeFS(root InodeEmbedder, opts *Options) fuse.RawFileSystem {
initInode(root.embed(), root, initInode(root.embed(), root,
StableAttr{ StableAttr{
Ino: 1, Ino: root.embed().StableAttr().Ino,
Mode: fuse.S_IFDIR, Mode: fuse.S_IFDIR,
}, },
bridge, bridge,
false, false,
1,
) )
bridge.root = root.embed() bridge.root = root.embed()
bridge.root.lookupCount = 1 bridge.root.lookupCount = 1
bridge.nodes = map[uint64]*Inode{ bridge.kernelNodeIds = map[uint64]*Inode{
1: bridge.root, 1: bridge.root,
} }
...@@ -287,7 +310,7 @@ func (b *rawBridge) String() string { ...@@ -287,7 +310,7 @@ func (b *rawBridge) String() string {
func (b *rawBridge) inode(id uint64, fh uint64) (*Inode, *fileEntry) { func (b *rawBridge) inode(id uint64, fh uint64) (*Inode, *fileEntry) {
b.mu.Lock() b.mu.Lock()
defer b.mu.Unlock() defer b.mu.Unlock()
n, f := b.nodes[id], b.files[fh] n, f := b.kernelNodeIds[id], b.files[fh]
if n == nil { if n == nil {
log.Panicf("unknown node %d", id) log.Panicf("unknown node %d", id)
} }
...@@ -306,8 +329,8 @@ func (b *rawBridge) Lookup(cancel <-chan struct{}, header *fuse.InHeader, name s ...@@ -306,8 +329,8 @@ func (b *rawBridge) Lookup(cancel <-chan struct{}, header *fuse.InHeader, name s
return errnoToStatus(errno) return errnoToStatus(errno)
} }
child, _ = b.addNewChild(parent, name, child, nil, 0, out)
child.setEntryOut(out) child.setEntryOut(out)
b.addNewChild(parent, name, child, nil, 0, out)
b.setEntryOutTimeout(out) b.setEntryOutTimeout(out)
return fuse.OK return fuse.OK
} }
...@@ -382,8 +405,8 @@ func (b *rawBridge) Mkdir(cancel <-chan struct{}, input *fuse.MkdirIn, name stri ...@@ -382,8 +405,8 @@ func (b *rawBridge) Mkdir(cancel <-chan struct{}, input *fuse.MkdirIn, name stri
log.Panicf("Mkdir: mode must be S_IFDIR (%o), got %o", fuse.S_IFDIR, out.Attr.Mode) log.Panicf("Mkdir: mode must be S_IFDIR (%o), got %o", fuse.S_IFDIR, out.Attr.Mode)
} }
child, _ = b.addNewChild(parent, name, child, nil, syscall.O_EXCL, out)
child.setEntryOut(out) child.setEntryOut(out)
b.addNewChild(parent, name, child, nil, 0, out)
b.setEntryOutTimeout(out) b.setEntryOutTimeout(out)
return fuse.OK return fuse.OK
} }
...@@ -401,8 +424,8 @@ func (b *rawBridge) Mknod(cancel <-chan struct{}, input *fuse.MknodIn, name stri ...@@ -401,8 +424,8 @@ func (b *rawBridge) Mknod(cancel <-chan struct{}, input *fuse.MknodIn, name stri
return errnoToStatus(errno) return errnoToStatus(errno)
} }
child, _ = b.addNewChild(parent, name, child, nil, syscall.O_EXCL, out)
child.setEntryOut(out) child.setEntryOut(out)
b.addNewChild(parent, name, child, nil, 0, out)
b.setEntryOutTimeout(out) b.setEntryOutTimeout(out)
return fuse.OK return fuse.OK
} }
...@@ -428,8 +451,9 @@ func (b *rawBridge) Create(cancel <-chan struct{}, input *fuse.CreateIn, name st ...@@ -428,8 +451,9 @@ func (b *rawBridge) Create(cancel <-chan struct{}, input *fuse.CreateIn, name st
return errnoToStatus(errno) return errnoToStatus(errno)
} }
out.Fh = uint64(b.addNewChild(parent, name, child, f, input.Flags|syscall.O_CREAT, &out.EntryOut)) child, fh := b.addNewChild(parent, name, child, f, input.Flags|syscall.O_CREAT|syscall.O_EXCL, &out.EntryOut)
out.Fh = uint64(fh)
out.OpenFlags = flags out.OpenFlags = flags
child.setEntryOut(&out.EntryOut) child.setEntryOut(&out.EntryOut)
...@@ -540,8 +564,8 @@ func (b *rawBridge) Link(cancel <-chan struct{}, input *fuse.LinkIn, name string ...@@ -540,8 +564,8 @@ func (b *rawBridge) Link(cancel <-chan struct{}, input *fuse.LinkIn, name string
return errnoToStatus(errno) return errnoToStatus(errno)
} }
child, _ = b.addNewChild(parent, name, child, nil, 0, out)
child.setEntryOut(out) child.setEntryOut(out)
b.addNewChild(parent, name, child, nil, 0, out)
b.setEntryOutTimeout(out) b.setEntryOutTimeout(out)
return fuse.OK return fuse.OK
} }
...@@ -557,7 +581,7 @@ func (b *rawBridge) Symlink(cancel <-chan struct{}, header *fuse.InHeader, targe ...@@ -557,7 +581,7 @@ func (b *rawBridge) Symlink(cancel <-chan struct{}, header *fuse.InHeader, targe
return errnoToStatus(status) return errnoToStatus(status)
} }
b.addNewChild(parent, name, child, nil, 0, out) child, _ = b.addNewChild(parent, name, child, nil, syscall.O_EXCL, out)
child.setEntryOut(out) child.setEntryOut(out)
b.setEntryOutTimeout(out) b.setEntryOutTimeout(out)
return fuse.OK return fuse.OK
...@@ -769,7 +793,7 @@ func (b *rawBridge) releaseFileEntry(nid uint64, fh uint64) (*Inode, *fileEntry) ...@@ -769,7 +793,7 @@ func (b *rawBridge) releaseFileEntry(nid uint64, fh uint64) (*Inode, *fileEntry)
b.mu.Lock() b.mu.Lock()
defer b.mu.Unlock() defer b.mu.Unlock()
n := b.nodes[nid] n := b.kernelNodeIds[nid]
var entry *fileEntry var entry *fileEntry
if fh > 0 { if fh > 0 {
last := len(n.openFiles) - 1 last := len(n.openFiles) - 1
...@@ -994,7 +1018,7 @@ func (b *rawBridge) ReadDirPlus(cancel <-chan struct{}, input *fuse.ReadIn, out ...@@ -994,7 +1018,7 @@ func (b *rawBridge) ReadDirPlus(cancel <-chan struct{}, input *fuse.ReadIn, out
entryOut.SetEntryTimeout(*b.options.NegativeTimeout) entryOut.SetEntryTimeout(*b.options.NegativeTimeout)
} }
} else { } else {
b.addNewChild(n, e.Name, child, nil, 0, entryOut) child, _ = b.addNewChild(n, e.Name, child, nil, 0, entryOut)
child.setEntryOut(entryOut) child.setEntryOut(entryOut)
b.setEntryOutTimeout(entryOut) b.setEntryOutTimeout(entryOut)
if e.Mode&syscall.S_IFMT != child.stableAttr.Mode&syscall.S_IFMT { if e.Mode&syscall.S_IFMT != child.stableAttr.Mode&syscall.S_IFMT {
......
...@@ -112,7 +112,7 @@ func TestForget(t *testing.T) { ...@@ -112,7 +112,7 @@ func TestForget(t *testing.T) {
bridge := rawFS.(*rawBridge) bridge := rawFS.(*rawBridge)
bridge.mu.Lock() bridge.mu.Lock()
l := len(bridge.nodes) l := len(bridge.kernelNodeIds)
bridge.mu.Unlock() bridge.mu.Unlock()
if l != 1 { if l != 1 {
t.Fatalf("got %d live nodes, want 1", l) t.Fatalf("got %d live nodes, want 1", l)
......
...@@ -63,6 +63,11 @@ type Inode struct { ...@@ -63,6 +63,11 @@ type Inode struct {
ops InodeEmbedder ops InodeEmbedder
bridge *rawBridge bridge *rawBridge
// The *Node ID* is an arbitrary uint64 identifier chosen by the FUSE library.
// It is used the identify *nodes* (files/directories/symlinks/...) in the
// communication between the FUSE library and the Linux kernel.
nodeId uint64
// Following data is mutable. // Following data is mutable.
// file handles. // file handles.
...@@ -115,11 +120,12 @@ func (n *Inode) EmbeddedInode() *Inode { ...@@ -115,11 +120,12 @@ func (n *Inode) EmbeddedInode() *Inode {
return n return n
} }
func initInode(n *Inode, ops InodeEmbedder, attr StableAttr, bridge *rawBridge, persistent bool) { func initInode(n *Inode, ops InodeEmbedder, attr StableAttr, bridge *rawBridge, persistent bool, nodeId uint64) {
n.ops = ops n.ops = ops
n.stableAttr = attr n.stableAttr = attr
n.bridge = bridge n.bridge = bridge
n.persistent = persistent n.persistent = persistent
n.nodeId = nodeId
n.parents = make(map[parentData]struct{}) n.parents = make(map[parentData]struct{})
if attr.Mode == fuse.S_IFDIR { if attr.Mode == fuse.S_IFDIR {
n.children = make(map[string]*Inode) n.children = make(map[string]*Inode)
...@@ -128,7 +134,7 @@ func initInode(n *Inode, ops InodeEmbedder, attr StableAttr, bridge *rawBridge, ...@@ -128,7 +134,7 @@ func initInode(n *Inode, ops InodeEmbedder, attr StableAttr, bridge *rawBridge,
// Set node ID and mode in EntryOut // Set node ID and mode in EntryOut
func (n *Inode) setEntryOut(out *fuse.EntryOut) { func (n *Inode) setEntryOut(out *fuse.EntryOut) {
out.NodeId = n.stableAttr.Ino out.NodeId = n.nodeId
out.Ino = n.stableAttr.Ino out.Ino = n.stableAttr.Ino
out.Mode = (out.Attr.Mode & 07777) | n.stableAttr.Mode out.Mode = (out.Attr.Mode & 07777) | n.stableAttr.Mode
} }
...@@ -329,7 +335,16 @@ func (n *Inode) Path(root *Inode) string { ...@@ -329,7 +335,16 @@ func (n *Inode) Path(root *Inode) string {
// but it could be also valid if only iparent is locked and ichild was just // but it could be also valid if only iparent is locked and ichild was just
// created and only one goroutine keeps referencing it. // created and only one goroutine keeps referencing it.
func (iparent *Inode) setEntry(name string, ichild *Inode) { func (iparent *Inode) setEntry(name string, ichild *Inode) {
ichild.parents[parentData{name, iparent}] = struct{}{} newParent := parentData{name, iparent}
if ichild.stableAttr.Mode == syscall.S_IFDIR {
// Directories cannot have more than one parent. Clear the map.
// This special-case is neccessary because ichild may still have a
// parent that was forgotten (i.e. removed from bridge.inoMap).
for i := range ichild.parents {
delete(ichild.parents, i)
}
}
ichild.parents[newParent] = struct{}{}
iparent.children[name] = ichild iparent.children[name] = ichild
ichild.changeCounter++ ichild.changeCounter++
iparent.changeCounter++ iparent.changeCounter++
...@@ -360,8 +375,8 @@ func (n *Inode) ForgetPersistent() { ...@@ -360,8 +375,8 @@ func (n *Inode) ForgetPersistent() {
// NewInode returns an inode for the given InodeEmbedder. The mode // NewInode returns an inode for the given InodeEmbedder. The mode
// should be standard mode argument (eg. S_IFDIR). The inode number in // 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, // 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 // and another node with the same ID is known, the new inode may be
// returned, and the passed-in `node` is ignored. // ignored, and the old one used instead.
func (n *Inode) NewInode(ctx context.Context, node InodeEmbedder, id StableAttr) *Inode { func (n *Inode) NewInode(ctx context.Context, node InodeEmbedder, id StableAttr) *Inode {
return n.newInode(ctx, node, id, false) return n.newInode(ctx, node, id, false)
} }
...@@ -381,9 +396,8 @@ func (n *Inode) removeRef(nlookup uint64, dropPersistence bool) (forgotten bool, ...@@ -381,9 +396,8 @@ func (n *Inode) removeRef(nlookup uint64, dropPersistence bool) (forgotten bool,
if nlookup > 0 && dropPersistence { if nlookup > 0 && dropPersistence {
log.Panic("only one allowed") log.Panic("only one allowed")
} else if nlookup > n.lookupCount { } else if nlookup > n.lookupCount {
log.Panicf("i%d lookupCount underflow: lookupCount=%d, decrement=%d", n.stableAttr.Ino, n.lookupCount, nlookup) log.Panicf("n%d lookupCount underflow: lookupCount=%d, decrement=%d", n.nodeId, n.lookupCount, nlookup)
} else if nlookup > 0 { } else if nlookup > 0 {
n.lookupCount -= nlookup n.lookupCount -= nlookup
n.changeCounter++ n.changeCounter++
} else if dropPersistence && n.persistent { } else if dropPersistence && n.persistent {
...@@ -391,13 +405,22 @@ func (n *Inode) removeRef(nlookup uint64, dropPersistence bool) (forgotten bool, ...@@ -391,13 +405,22 @@ func (n *Inode) removeRef(nlookup uint64, dropPersistence bool) (forgotten bool,
n.changeCounter++ n.changeCounter++
} }
n.bridge.mu.Lock()
if n.lookupCount == 0 {
forgotten = true
// Dropping the node from inoMap guarantees that no new references to this node are
// handed out to the kernel, hence we can also safely delete it from nodeidMap.
delete(n.bridge.stableAttrs, n.stableAttr)
delete(n.bridge.kernelNodeIds, n.nodeId)
}
n.bridge.mu.Unlock()
retry: retry:
for { for {
lockme = append(lockme[:0], n) lockme = append(lockme[:0], n)
parents = parents[:0] parents = parents[:0]
nChange := n.changeCounter nChange := n.changeCounter
live = n.lookupCount > 0 || len(n.children) > 0 || n.persistent live = n.lookupCount > 0 || len(n.children) > 0 || n.persistent
forgotten = n.lookupCount == 0
for p := range n.parents { for p := range n.parents {
parents = append(parents, p) parents = append(parents, p)
lockme = append(lockme, p.parent) lockme = append(lockme, p.parent)
...@@ -417,6 +440,10 @@ retry: ...@@ -417,6 +440,10 @@ retry:
} }
for _, p := range parents { for _, p := range parents {
if p.parent.children[p.name] != n {
// another node has replaced us already
continue
}
delete(p.parent.children, p.name) delete(p.parent.children, p.name)
p.parent.changeCounter++ p.parent.changeCounter++
} }
...@@ -424,13 +451,9 @@ retry: ...@@ -424,13 +451,9 @@ retry:
n.changeCounter++ n.changeCounter++
if n.lookupCount != 0 { if n.lookupCount != 0 {
panic("lookupCount changed") log.Panicf("i%d %p lookupCount changed: %d", n.nodeId, n, n.lookupCount)
} }
n.bridge.mu.Lock()
delete(n.bridge.nodes, n.stableAttr.Ino)
n.bridge.mu.Unlock()
unlockNodes(lockme...) unlockNodes(lockme...)
break break
} }
...@@ -714,7 +737,7 @@ retry: ...@@ -714,7 +737,7 @@ retry:
// tuple should be invalidated. On next access, a LOOKUP operation // tuple should be invalidated. On next access, a LOOKUP operation
// will be started. // will be started.
func (n *Inode) NotifyEntry(name string) syscall.Errno { func (n *Inode) NotifyEntry(name string) syscall.Errno {
status := n.bridge.server.EntryNotify(n.stableAttr.Ino, name) status := n.bridge.server.EntryNotify(n.nodeId, name)
return syscall.Errno(status) return syscall.Errno(status)
} }
...@@ -723,7 +746,7 @@ func (n *Inode) NotifyEntry(name string) syscall.Errno { ...@@ -723,7 +746,7 @@ func (n *Inode) NotifyEntry(name string) syscall.Errno {
// to NotifyEntry, but also sends an event to inotify watchers. // to NotifyEntry, but also sends an event to inotify watchers.
func (n *Inode) NotifyDelete(name string, child *Inode) syscall.Errno { func (n *Inode) NotifyDelete(name string, child *Inode) syscall.Errno {
// XXX arg ordering? // XXX arg ordering?
return syscall.Errno(n.bridge.server.DeleteNotify(n.stableAttr.Ino, child.stableAttr.Ino, name)) return syscall.Errno(n.bridge.server.DeleteNotify(n.nodeId, child.nodeId, name))
} }
...@@ -731,16 +754,16 @@ func (n *Inode) NotifyDelete(name string, child *Inode) syscall.Errno { ...@@ -731,16 +754,16 @@ func (n *Inode) NotifyDelete(name string, child *Inode) syscall.Errno {
// inode should be flushed from buffers. // inode should be flushed from buffers.
func (n *Inode) NotifyContent(off, sz int64) syscall.Errno { func (n *Inode) NotifyContent(off, sz int64) syscall.Errno {
// XXX how does this work for directories? // XXX how does this work for directories?
return syscall.Errno(n.bridge.server.InodeNotify(n.stableAttr.Ino, off, sz)) return syscall.Errno(n.bridge.server.InodeNotify(n.nodeId, off, sz))
} }
// WriteCache stores data in the kernel cache. // WriteCache stores data in the kernel cache.
func (n *Inode) WriteCache(offset int64, data []byte) syscall.Errno { func (n *Inode) WriteCache(offset int64, data []byte) syscall.Errno {
return syscall.Errno(n.bridge.server.InodeNotifyStoreCache(n.stableAttr.Ino, offset, data)) return syscall.Errno(n.bridge.server.InodeNotifyStoreCache(n.nodeId, offset, data))
} }
// ReadCache reads data from the kernel cache. // ReadCache reads data from the kernel cache.
func (n *Inode) ReadCache(offset int64, dest []byte) (count int, errno syscall.Errno) { func (n *Inode) ReadCache(offset int64, dest []byte) (count int, errno syscall.Errno) {
c, s := n.bridge.server.InodeRetrieveCache(n.stableAttr.Ino, offset, dest) c, s := n.bridge.server.InodeRetrieveCache(n.nodeId, offset, dest)
return c, syscall.Errno(s) return c, syscall.Errno(s)
} }
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