Commit 89a1a0c7 authored by Han-Wen Nienhuys's avatar Han-Wen Nienhuys

nodefs: introduce FileID, an ID type for persistent

identification

This fixes another possible race condition if the same file is looked
up through 2 different paths.

Remove FindChildByOpaqueID(), since it is still susceptible to race
conditions: two lookup calls might be racing, and both may see
FindChildByOpaqueID == nil, creating two Node objects

Instead, NewInode() automatically discards a new Node if it finds one
with existing opaqueID.

The FileID struct has (uint64,uint64) for Dev, Ino. While Dev
typically is uint32 (the FUSE protocol has uint32), it is uint64
analogous to syscall.Stat_t. The resulting 128 bits of ID space is
also ample space for storing hashes of other IDs (eg strings)
parent 9778688e
......@@ -79,7 +79,7 @@ type Node interface {
//
// See InodeOf for public API to retrieve an inode from Node.
inode() *Inode
setInode(*Inode) (set bool)
setInode(*Inode)
// Lookup should find a direct child of the node by child name.
//
......
......@@ -38,11 +38,56 @@ type rawBridge struct {
files []fileEntry
freeFiles []uint64
byFileID map[FileID]*Inode
}
// newInode creates creates new inode pointing to node.
func (b *rawBridge) newInode(node Node, mode uint32, id FileID, persistent bool) *Inode {
b.mu.Lock()
defer b.mu.Unlock()
if !id.Zero() {
// 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 FileIDs ensure that we return the
// same node.
old := b.byFileID[id]
if old != nil {
return old
}
}
inode := &Inode{
mode: mode ^ 07777,
node: node,
bridge: b,
persistent: persistent,
parents: make(map[parentData]struct{}),
}
if mode&fuse.S_IFDIR != 0 {
inode.children = make(map[string]*Inode)
}
if !id.Zero() {
b.byFileID[id] = inode
}
node.setInode(inode)
return inode
}
func NewNodeFS(root Node, opts *Options) fuse.RawFileSystem {
bridge := &rawBridge{
RawFileSystem: fuse.NewDefaultRawFileSystem(),
byFileID: make(map[FileID]*Inode),
}
if opts != nil {
......@@ -76,7 +121,14 @@ func NewNodeFS(root Node, opts *Options) fuse.RawFileSystem {
func (b *rawBridge) inode(id uint64, fh uint64) (*Inode, fileEntry) {
b.mu.Lock()
defer b.mu.Unlock()
return b.nodes[id].inode, b.files[fh]
n, f := b.nodes[id].inode, b.files[fh]
if n == nil {
log.Panicf("unknown node %d", id)
}
if fh != 0 && f.file == nil {
log.Panicf("unknown fh %d", fh)
}
return n, f
}
func (b *rawBridge) Lookup(header *fuse.InHeader, name string, out *fuse.EntryOut) (status fuse.Status) {
......@@ -338,10 +390,7 @@ func (b *rawBridge) Rename(input *fuse.RenameIn, oldName string, newName string)
p2, _ := b.inode(input.Newdir, 0)
if code := p1.node.Rename(context.TODO(), oldName, p2.node, newName); code.Ok() {
// NOSUBMIT - is it better to have the user code do
// this? Maybe the user code wants a transaction over
// more nodes?
p1.MvChild(oldName, p2, newName)
p1.MvChild(oldName, p2, newName, true)
}
return code
}
......
......@@ -6,9 +6,7 @@ package nodefs
import (
"context"
"sync/atomic"
"time"
"unsafe"
"github.com/hanwen/go-fuse/fuse"
)
......@@ -20,32 +18,12 @@ type DefaultNode struct {
inode_ *Inode
}
// set/retrieve inode.
//
// node -> inode association, can be simultaneously tried to be set, if for e.g.
//
// root
// / \
// dir1 dir2
// \ /
// file
//
// dir1.Lookup("file") and dir2.Lookup("file") are executed simultaneously.
//
// We use atomics so that only one set can win
//
// 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) setInode(inode *Inode) {
dn.inode_ = inode
}
func (dn *DefaultNode) inode() *Inode {
return (*Inode)(atomic.LoadPointer(
(*unsafe.Pointer)(unsafe.Pointer(&dn.inode_))))
return dn.inode_
}
func (n *DefaultNode) Read(ctx context.Context, f File, dest []byte, off int64) (fuse.ReadResult, fuse.Status) {
......
......@@ -11,8 +11,6 @@ import (
"strings"
"sync"
"unsafe"
"github.com/hanwen/go-fuse/fuse"
)
var _ = log.Println
......@@ -22,6 +20,19 @@ type parentData struct {
parent *Inode
}
// FileID provides a identifier for file objects defined by FUSE
// filesystems.
//
// XXX name: PersistentID ? NodeID ?
type FileID struct {
Dev uint64 // XXX Rdev?
Ino uint64
}
func (i *FileID) Zero() bool {
return i.Dev == 0 && i.Ino == 0
}
// Inode is a node in VFS tree. Inodes are one-to-one mapped to Node
// instances, which is the extension interface for file systems. One
// can create fully-formed trees of Inodes ahead of time by creating
......@@ -29,7 +40,7 @@ type parentData struct {
type Inode struct {
// The filetype bits from the mode.
mode uint32
opaqueID uint64
opaqueID FileID
node Node
bridge *rawBridge
......@@ -77,22 +88,6 @@ func (n *Inode) debugString() string {
return fmt.Sprintf("%d: %s", n.nodeID, strings.Join(ss, ","))
}
// 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,
parents: make(map[parentData]struct{}),
}
if mode&fuse.S_IFDIR != 0 {
inode.children = make(map[string]*Inode)
}
return inode
}
// sortNodes rearranges inode group in consistent order.
//
// The nodes are ordered by their in-RAM address, which gives consistency
......@@ -234,20 +229,6 @@ func (n *Inode) FindChildByMode(name string, mode uint32) *Inode {
return nil
}
// Finds a child with the given name and ID. Returns nil if not found.
func (n *Inode) FindChildByOpaqueID(name string, opaqueID uint64) *Inode {
n.mu.Lock()
defer n.mu.Unlock()
ch := n.children[name]
if ch != nil && ch.opaqueID == opaqueID {
return ch
}
return nil
}
// setEntry does `iparent[name] = ichild` linking.
//
// setEntry must not be called simultaneously for any of iparent or ichild.
......@@ -263,7 +244,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(node Node, mode uint32, opaque uint64) *Inode {
func (n *Inode) NewPersistentInode(node Node, mode uint32, opaque FileID) *Inode {
return n.newInode(node, mode, opaque, true)
}
......@@ -275,31 +256,16 @@ func (n *Inode) ForgetPersistent() {
}
// NewInode returns an inode for the given Node. The mode should be
// standard mode argument (eg. S_IFDIR). The opaqueID argument can be
// used to signal changes in the tree structure during lookup (see
// FindChildByOpaqueID). For a loopback file system, the inode number
// of the underlying file is a good candidate.
func (n *Inode) NewInode(node Node, mode uint32, opaqueID uint64) *Inode {
// standard mode argument (eg. S_IFDIR). The opaqueID argument, if
// non-zero, is used to implement hard-links. If opaqueID 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(node Node, mode uint32, opaqueID FileID) *Inode {
return n.newInode(node, mode, opaqueID, false)
}
func (n *Inode) newInode(node Node, mode uint32, opaqueID uint64, persistent bool) *Inode {
ch := &Inode{
mode: mode ^ 07777,
node: node,
opaqueID: opaqueID,
bridge: n.bridge,
persistent: persistent,
parents: make(map[parentData]struct{}),
}
if mode&fuse.S_IFDIR != 0 {
ch.children = make(map[string]*Inode)
}
if node.setInode(ch) {
return ch
}
return node.inode()
func (n *Inode) newInode(node Node, mode uint32, opaqueID FileID, persistent bool) *Inode {
return n.bridge.newInode(node, mode, opaqueID, persistent)
}
// removeRef decreases references. Returns if this operation caused
......@@ -355,12 +321,15 @@ retry:
panic("lookupCount changed")
}
if n.nodeID != 0 {
n.bridge.mu.Lock()
if n.nodeID != 0 {
n.bridge.unregisterNode(n.nodeID)
n.bridge.mu.Unlock()
n.nodeID = 0
}
if !n.opaqueID.Zero() {
delete(n.bridge.byFileID, n.opaqueID)
}
n.bridge.mu.Unlock()
unlockNodes(lockme...)
break
......@@ -427,8 +396,10 @@ retry:
return true, true
}
// TODO - RENAME_NOREPLACE, RENAME_EXCHANGE
func (n *Inode) MvChild(old string, newParent *Inode, newName string) {
// TODO - RENAME_NOREPLACE, RENAME_EXCHANGE MvChild executes a
// rename. If overwrite is set, a child at the destination will be
// overwritten.
func (n *Inode) MvChild(old string, newParent *Inode, newName string, overwrite bool) bool {
retry:
for {
lockNode2(n, newParent)
......@@ -439,13 +410,26 @@ retry:
destChild := newParent.children[newName]
unlockNode2(n, newParent)
if destChild != nil && !overwrite {
return false
}
lockNodes(n, newParent, oldChild, destChild)
if counter2 != newParent.changeCounter || counter1 != n.changeCounter {
unlockNodes(n, newParent, oldChild, destChild)
continue retry
}
if oldChild != nil {
delete(n.children, old)
delete(oldChild.parents, parentData{old, n})
n.changeCounter++
oldChild.changeCounter++
}
if destChild != nil {
// This can cause the child to be slated for
// removal; see below
delete(newParent.children, newName)
delete(destChild.parents, parentData{newName, newParent})
destChild.changeCounter++
......@@ -456,14 +440,16 @@ retry:
newParent.children[newName] = oldChild
newParent.changeCounter++
delete(n.children, old)
delete(oldChild.parents, parentData{old, n})
oldChild.parents[parentData{newName, newParent}] = struct{}{}
oldChild.changeCounter++
}
unlockNodes(n, newParent, oldChild, destChild)
return
if destChild != nil {
// XXX would be better to do this under lock above too.
destChild.removeRef(0, false)
}
return true
}
}
......@@ -52,13 +52,13 @@ func (n *loopbackNode) Lookup(ctx context.Context, name string, out *fuse.EntryO
out.Attr.FromStat(&st)
ch := InodeOf(n).FindChildByOpaqueID(name, out.Attr.Ino)
if ch != nil {
return ch, fuse.OK
opaque := FileID{
Dev: uint64(out.Attr.Rdev),
Ino: out.Attr.Ino,
}
node := &loopbackNode{rootNode: n.rootNode}
ch = n.inode().NewInode(node, out.Attr.Mode, out.Attr.Ino)
ch := n.inode().NewInode(node, out.Attr.Mode, opaque)
return ch, fuse.OK
}
......@@ -77,7 +77,11 @@ func (n *loopbackNode) Mknod(ctx context.Context, name string, mode, rdev uint32
out.Attr.FromStat(&st)
node := &loopbackNode{rootNode: n.rootNode}
ch := n.inode().NewInode(node, out.Attr.Mode, out.Attr.Ino)
opaque := FileID{
Dev: uint64(out.Attr.Rdev),
Ino: out.Attr.Ino,
}
ch := n.inode().NewInode(node, out.Attr.Mode, opaque)
return ch, fuse.OK
}
......@@ -98,7 +102,11 @@ func (n *loopbackNode) Mkdir(ctx context.Context, name string, mode uint32, out
out.Attr.FromStat(&st)
node := &loopbackNode{rootNode: n.rootNode}
ch := n.inode().NewInode(node, out.Attr.Mode, out.Attr.Ino)
opaque := FileID{
Dev: uint64(out.Attr.Rdev),
Ino: out.Attr.Ino,
}
ch := n.inode().NewInode(node, out.Attr.Mode, opaque)
return ch, fuse.OK
}
......@@ -144,7 +152,12 @@ func (n *loopbackNode) Create(ctx context.Context, name string, flags uint32, mo
}
node := &loopbackNode{rootNode: n.rootNode}
ch := n.inode().NewInode(node, st.Mode, st.Ino)
opaque := FileID{
Dev: st.Rdev,
Ino: st.Ino,
}
ch := n.inode().NewInode(node, st.Mode, opaque)
return ch, NewLoopbackFile(f), 0, fuse.OK
}
......
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