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
import (
"context"
"log"
"math/rand"
"sync"
"syscall"
"time"
......@@ -61,10 +60,28 @@ type rawBridge struct {
// mu protects the following data. Locks for inodes must be
// taken before rawBridge.mu
mu sync.Mutex
nodes map[uint64]*Inode
mu sync.Mutex
// 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
// 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
freeFiles []uint32
}
......@@ -83,55 +100,26 @@ func (b *rawBridge) newInodeUnlocked(ops InodeEmbedder, id StableAttr, persisten
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 {
// Find free inode number.
for {
id.Ino = b.automaticIno
b.automaticIno++
_, ok := b.nodes[id.Ino]
_, ok := b.stableAttrs[id]
if !ok {
break
}
}
}
// Only the file type bits matter
id.Mode = id.Mode & syscall.S_IFMT
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)
initInode(ops.embed(), ops, id, b, persistent, b.nextNodeId)
b.nextNodeId++
return ops.embed()
}
......@@ -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 {
ch := b.newInodeUnlocked(ops, id, persistent)
if ch != ops.embed() {
......@@ -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.
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 == ".." {
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.
// This means it MAY have been deleted from nodes[] already. Add it back.
if child.lookupCount == 0 {
b.nodes[child.stableAttr.Ino] = child
// 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.
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.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 {
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.Attr.Ino = child.stableAttr.Ino
b.mu.Unlock()
unlockNodes(parent, child)
return fh
return child, fh
}
func (b *rawBridge) setEntryOutTimeout(out *fuse.EntryOut) {
......@@ -237,6 +257,8 @@ func NewNodeFS(root InodeEmbedder, opts *Options) fuse.RawFileSystem {
bridge := &rawBridge{
automaticIno: opts.FirstAutomaticIno,
server: opts.ServerCallbacks,
nextNodeId: 2, // the root node has nodeid 1
stableAttrs: make(map[StableAttr]*Inode),
}
if bridge.automaticIno == 1 {
bridge.automaticIno++
......@@ -256,15 +278,16 @@ func NewNodeFS(root InodeEmbedder, opts *Options) fuse.RawFileSystem {
initInode(root.embed(), root,
StableAttr{
Ino: 1,
Ino: root.embed().StableAttr().Ino,
Mode: fuse.S_IFDIR,
},
bridge,
false,
1,
)
bridge.root = root.embed()
bridge.root.lookupCount = 1
bridge.nodes = map[uint64]*Inode{
bridge.kernelNodeIds = map[uint64]*Inode{
1: bridge.root,
}
......@@ -287,7 +310,7 @@ func (b *rawBridge) String() string {
func (b *rawBridge) inode(id uint64, fh uint64) (*Inode, *fileEntry) {
b.mu.Lock()
defer b.mu.Unlock()
n, f := b.nodes[id], b.files[fh]
n, f := b.kernelNodeIds[id], b.files[fh]
if n == nil {
log.Panicf("unknown node %d", id)
}
......@@ -306,8 +329,8 @@ func (b *rawBridge) Lookup(cancel <-chan struct{}, header *fuse.InHeader, name s
return errnoToStatus(errno)
}
child, _ = b.addNewChild(parent, name, child, nil, 0, out)
child.setEntryOut(out)
b.addNewChild(parent, name, child, nil, 0, out)
b.setEntryOutTimeout(out)
return fuse.OK
}
......@@ -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)
}
child, _ = b.addNewChild(parent, name, child, nil, syscall.O_EXCL, out)
child.setEntryOut(out)
b.addNewChild(parent, name, child, nil, 0, out)
b.setEntryOutTimeout(out)
return fuse.OK
}
......@@ -401,8 +424,8 @@ func (b *rawBridge) Mknod(cancel <-chan struct{}, input *fuse.MknodIn, name stri
return errnoToStatus(errno)
}
child, _ = b.addNewChild(parent, name, child, nil, syscall.O_EXCL, out)
child.setEntryOut(out)
b.addNewChild(parent, name, child, nil, 0, out)
b.setEntryOutTimeout(out)
return fuse.OK
}
......@@ -428,8 +451,9 @@ func (b *rawBridge) Create(cancel <-chan struct{}, input *fuse.CreateIn, name st
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
child.setEntryOut(&out.EntryOut)
......@@ -540,8 +564,8 @@ func (b *rawBridge) Link(cancel <-chan struct{}, input *fuse.LinkIn, name string
return errnoToStatus(errno)
}
child, _ = b.addNewChild(parent, name, child, nil, 0, out)
child.setEntryOut(out)
b.addNewChild(parent, name, child, nil, 0, out)
b.setEntryOutTimeout(out)
return fuse.OK
}
......@@ -557,7 +581,7 @@ func (b *rawBridge) Symlink(cancel <-chan struct{}, header *fuse.InHeader, targe
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)
b.setEntryOutTimeout(out)
return fuse.OK
......@@ -769,7 +793,7 @@ func (b *rawBridge) releaseFileEntry(nid uint64, fh uint64) (*Inode, *fileEntry)
b.mu.Lock()
defer b.mu.Unlock()
n := b.nodes[nid]
n := b.kernelNodeIds[nid]
var entry *fileEntry
if fh > 0 {
last := len(n.openFiles) - 1
......@@ -994,7 +1018,7 @@ func (b *rawBridge) ReadDirPlus(cancel <-chan struct{}, input *fuse.ReadIn, out
entryOut.SetEntryTimeout(*b.options.NegativeTimeout)
}
} else {
b.addNewChild(n, e.Name, child, nil, 0, entryOut)
child, _ = b.addNewChild(n, e.Name, child, nil, 0, entryOut)
child.setEntryOut(entryOut)
b.setEntryOutTimeout(entryOut)
if e.Mode&syscall.S_IFMT != child.stableAttr.Mode&syscall.S_IFMT {
......
......@@ -112,7 +112,7 @@ func TestForget(t *testing.T) {
bridge := rawFS.(*rawBridge)
bridge.mu.Lock()
l := len(bridge.nodes)
l := len(bridge.kernelNodeIds)
bridge.mu.Unlock()
if l != 1 {
t.Fatalf("got %d live nodes, want 1", l)
......
......@@ -63,6 +63,11 @@ type Inode struct {
ops InodeEmbedder
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.
// file handles.
......@@ -115,11 +120,12 @@ func (n *Inode) EmbeddedInode() *Inode {
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.stableAttr = attr
n.bridge = bridge
n.persistent = persistent
n.nodeId = nodeId
n.parents = make(map[parentData]struct{})
if attr.Mode == fuse.S_IFDIR {
n.children = make(map[string]*Inode)
......@@ -128,7 +134,7 @@ func initInode(n *Inode, ops InodeEmbedder, attr StableAttr, bridge *rawBridge,
// Set node ID and mode in EntryOut
func (n *Inode) setEntryOut(out *fuse.EntryOut) {
out.NodeId = n.stableAttr.Ino
out.NodeId = n.nodeId
out.Ino = n.stableAttr.Ino
out.Mode = (out.Attr.Mode & 07777) | n.stableAttr.Mode
}
......@@ -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
// created and only one goroutine keeps referencing it.
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
ichild.changeCounter++
iparent.changeCounter++
......@@ -360,8 +375,8 @@ func (n *Inode) ForgetPersistent() {
// 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.
// and another node with the same ID is known, the new inode may be
// ignored, and the old one used instead.
func (n *Inode) NewInode(ctx context.Context, node InodeEmbedder, id StableAttr) *Inode {
return n.newInode(ctx, node, id, false)
}
......@@ -381,9 +396,8 @@ func (n *Inode) removeRef(nlookup uint64, dropPersistence bool) (forgotten bool,
if nlookup > 0 && dropPersistence {
log.Panic("only one allowed")
} 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 {
n.lookupCount -= nlookup
n.changeCounter++
} else if dropPersistence && n.persistent {
......@@ -391,13 +405,22 @@ func (n *Inode) removeRef(nlookup uint64, dropPersistence bool) (forgotten bool,
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:
for {
lockme = append(lockme[:0], n)
parents = parents[:0]
nChange := n.changeCounter
live = n.lookupCount > 0 || len(n.children) > 0 || n.persistent
forgotten = n.lookupCount == 0
for p := range n.parents {
parents = append(parents, p)
lockme = append(lockme, p.parent)
......@@ -417,6 +440,10 @@ retry:
}
for _, p := range parents {
if p.parent.children[p.name] != n {
// another node has replaced us already
continue
}
delete(p.parent.children, p.name)
p.parent.changeCounter++
}
......@@ -424,13 +451,9 @@ retry:
n.changeCounter++
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...)
break
}
......@@ -714,7 +737,7 @@ retry:
// tuple should be invalidated. On next access, a LOOKUP operation
// will be started.
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)
}
......@@ -723,7 +746,7 @@ func (n *Inode) NotifyEntry(name string) syscall.Errno {
// to NotifyEntry, but also sends an event to inotify watchers.
func (n *Inode) NotifyDelete(name string, child *Inode) syscall.Errno {
// 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 {
// inode should be flushed from buffers.
func (n *Inode) NotifyContent(off, sz int64) syscall.Errno {
// 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.
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.
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)
}
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