Commit 4a79a89d authored by Han-Wen Nienhuys's avatar Han-Wen Nienhuys

Rewrite Mount/Unmount logic.

* In-process mounts are now always on synthetic inodes.  FileSystems
  no longer have to generate GetAttr() responses for mount points.

  Advantages:

  - Simplifies FileSystems: there is no need to represent the state
    between mountpoint appearing and Mount() succeeding, and mount
    points appear automatically in ReadDir().

  - Mount point inodes are always associated with one filesystem over
    their lifetime.  This simplifies synchronization around unmount calls,
    which could previously lead to "unlock of unlocked mutex" panics.

* (Un)mount always takes 2 locks: treeLock from the parent, and the
  treeLock of the filesystem itself, as the parent/child relation
  crosses the boundary between filesystems..

* Remove unmountPending variable; Unmount now removes all known inodes
  directly, removing reliance on eventual consistency.

* Document Mount(), Unmount()

* Forced forgets (eg. when fusermount -u is called) will also cause
  FileSystem.Unmount() to be called.

* Changes to accomodate new conventions in ZipFs/UnionFs

* Configure MultiZipFs using symlinks rather than catching file
  writes.

* Remove return from Mount() api call.
parent 6a591a3e
...@@ -44,7 +44,7 @@ type FileSystem interface { ...@@ -44,7 +44,7 @@ type FileSystem interface {
SetXAttr(name string, attr string, data []byte, flags int) Status SetXAttr(name string, attr string, data []byte, flags int) Status
// Called after mount. // Called after mount.
Mount(connector *FileSystemConnector) Status Mount(connector *FileSystemConnector)
Unmount() Unmount()
// File handling. If opening for writing, the file's mtime // File handling. If opening for writing, the file's mtime
......
...@@ -260,8 +260,7 @@ func (me *DefaultFileSystem) OpenDir(name string) (stream chan DirEntry, status ...@@ -260,8 +260,7 @@ func (me *DefaultFileSystem) OpenDir(name string) (stream chan DirEntry, status
return nil, ENOSYS return nil, ENOSYS
} }
func (me *DefaultFileSystem) Mount(conn *FileSystemConnector) Status { func (me *DefaultFileSystem) Mount(conn *FileSystemConnector) {
return OK
} }
func (me *DefaultFileSystem) Unmount() { func (me *DefaultFileSystem) Unmount() {
......
...@@ -66,44 +66,47 @@ func (me *DirEntryList) Bytes() []byte { ...@@ -66,44 +66,47 @@ func (me *DirEntryList) Bytes() []byte {
//////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////
type Dir struct { type Dir struct {
extra []DirEntry
stream chan DirEntry stream chan DirEntry
leftOver DirEntry leftOver DirEntry
} }
func (me *Dir) inode(name string) uint64 { func (me *Dir) ReadDir(input *ReadIn) (*DirEntryList, Status) {
if me.stream == nil && len(me.extra) == 0 {
return nil, OK
}
// We could also return // We could also return
// me.connector.lookupUpdate(me.parentIno, name).NodeId but it // me.connector.lookupUpdate(me.parentIno, name).NodeId but it
// appears FUSE will issue a LOOKUP afterwards for the entry // appears FUSE will issue a LOOKUP afterwards for the entry
// anyway, so we skip hash table update here. // anyway, so we skip hash table update here.
return FUSE_UNKNOWN_INO inode := uint64(FUSE_UNKNOWN_INO)
}
func (me *Dir) ReadDir(input *ReadIn) (*DirEntryList, Status) {
if me.stream == nil {
return nil, OK
}
list := NewDirEntryList(int(input.Size)) list := NewDirEntryList(int(input.Size))
if me.leftOver.Name != "" { if me.leftOver.Name != "" {
n := me.leftOver.Name n := me.leftOver.Name
i := me.inode(n) success := list.AddString(n, inode, me.leftOver.Mode)
success := list.AddString(n, i, me.leftOver.Mode)
if !success { if !success {
panic("No space for single entry.") panic("No space for single entry.")
} }
me.leftOver.Name = "" me.leftOver.Name = ""
} }
for len(me.extra) > 0 {
e := me.extra[len(me.extra)-1]
me.extra = me.extra[:len(me.extra)-1]
success := list.AddString(e.Name, inode, e.Mode)
if !success {
me.leftOver = e
return list, OK
}
}
for { for {
d, isOpen := <-me.stream d, isOpen := <-me.stream
if !isOpen { if !isOpen {
me.stream = nil me.stream = nil
break break
} }
i := me.inode(d.Name) if !list.AddString(d.Name, inode, d.Mode) {
if !list.AddString(d.Name, i, d.Mode) {
me.leftOver = d me.leftOver = d
break break
} }
......
...@@ -95,9 +95,9 @@ func (me *LockingFileSystem) OpenDir(name string) (stream chan DirEntry, status ...@@ -95,9 +95,9 @@ func (me *LockingFileSystem) OpenDir(name string) (stream chan DirEntry, status
return me.FileSystem.OpenDir(name) return me.FileSystem.OpenDir(name)
} }
func (me *LockingFileSystem) Mount(conn *FileSystemConnector) Status { func (me *LockingFileSystem) Mount(conn *FileSystemConnector) {
defer me.locked()() defer me.locked()()
return me.FileSystem.Mount(conn) me.FileSystem.Mount(conn)
} }
func (me *LockingFileSystem) Unmount() { func (me *LockingFileSystem) Unmount() {
......
...@@ -114,9 +114,9 @@ func (me *LoggingFileSystem) OpenDir(name string) (stream chan DirEntry, status ...@@ -114,9 +114,9 @@ func (me *LoggingFileSystem) OpenDir(name string) (stream chan DirEntry, status
return me.FileSystem.OpenDir(name) return me.FileSystem.OpenDir(name)
} }
func (me *LoggingFileSystem) Mount(conn *FileSystemConnector) Status { func (me *LoggingFileSystem) Mount(conn *FileSystemConnector) {
me.Print("Mount", "") me.Print("Mount", "")
return me.FileSystem.Mount(conn) me.FileSystem.Mount(conn)
} }
func (me *LoggingFileSystem) Unmount() { func (me *LoggingFileSystem) Unmount() {
......
...@@ -11,7 +11,6 @@ import ( ...@@ -11,7 +11,6 @@ import (
"strings" "strings"
"syscall" "syscall"
"testing" "testing"
"time"
) )
var _ = strings.Join var _ = strings.Join
...@@ -618,6 +617,19 @@ func TestLargeDirRead(t *testing.T) { ...@@ -618,6 +617,19 @@ func TestLargeDirRead(t *testing.T) {
ts.testLargeDirRead() ts.testLargeDirRead()
} }
func TestRootDir(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
d, err := os.Open(ts.mountPoint)
CheckSuccess(err)
_, err = d.Readdirnames(-1)
CheckSuccess(err)
err = d.Close()
CheckSuccess(err)
}
func TestDelRename(t *testing.T) { func TestDelRename(t *testing.T) {
ts := new(testCase) ts := new(testCase)
ts.Setup(t) ts.Setup(t)
...@@ -646,97 +658,3 @@ func TestIoctl(t *testing.T) { ...@@ -646,97 +658,3 @@ func TestIoctl(t *testing.T) {
fmt.Println("ioctl", v, e) fmt.Println("ioctl", v, e)
} }
func TestRecursiveMount(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
f, err := os.OpenFile(filepath.Join(ts.mountPoint, "hello.txt"),
os.O_WRONLY|os.O_CREATE, 0777)
CheckSuccess(err)
f.WriteString("bla")
f.Close()
pfs2 := NewLoopbackFileSystem(ts.origDir)
code := ts.connector.Mount("/hello.txt", pfs2, nil)
if code != EINVAL {
t.Error("expect EINVAL", code)
}
submnt := filepath.Join(ts.mountPoint, "mnt")
err = os.Mkdir(submnt, 0777)
CheckSuccess(err)
code = ts.connector.Mount("/mnt", pfs2, nil)
if code != OK {
t.Errorf("mkdir")
}
_, err = os.Lstat(submnt)
CheckSuccess(err)
_, err = os.Lstat(filepath.Join(submnt, "hello.txt"))
CheckSuccess(err)
f, err = os.Open(filepath.Join(submnt, "hello.txt"))
CheckSuccess(err)
code = ts.connector.Unmount("/mnt")
if code != EBUSY {
t.Error("expect EBUSY")
}
err = os.Rename(ts.mountPoint+"/mnt", ts.mountPoint+"/foobar")
CheckSuccess(err)
f.Close()
log.Println("Waiting for kernel to flush file-close to fuse...")
time.Sleep(1.5e9 * testTtl)
code = ts.connector.Unmount("/doesnotexist")
if code != EINVAL {
t.Fatal("expect EINVAL", code)
}
code = ts.connector.Unmount("/foobar")
if code != OK {
t.Error("umount failed.", code)
}
}
func TestDeletedUnmount(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
submnt := filepath.Join(ts.mountPoint, "mnt")
err := os.Mkdir(submnt, 0777)
CheckSuccess(err)
pfs2 := NewLoopbackFileSystem(ts.origDir)
code := ts.connector.Mount("/mnt", pfs2, nil)
if !code.Ok() {
t.Fatal("err")
}
f, err := os.Create(filepath.Join(submnt, "hello.txt"))
CheckSuccess(err)
log.Println("Removing")
err = os.Remove(filepath.Join(submnt, "hello.txt"))
CheckSuccess(err)
log.Println("Removing")
_, err = f.Write([]byte("bla"))
CheckSuccess(err)
code = ts.connector.Unmount("/mnt")
if code != EBUSY {
t.Error("expect EBUSY", code)
}
f.Close()
time.Sleep(1.5e9 * testTtl)
code = ts.connector.Unmount("/mnt")
if !code.Ok() {
t.Error("should succeed", code)
}
}
package fuse
import (
"log"
"os"
"testing"
"time"
"path/filepath"
"io/ioutil"
)
func TestMountOnExisting(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
err := os.Mkdir(ts.mountPoint + "/mnt", 0777)
CheckSuccess(err)
fs := &DefaultFileSystem{}
code := ts.connector.Mount("/mnt", fs, nil)
if code != EBUSY {
t.Fatal("expect EBUSY:", code)
}
err = os.Remove(ts.mountPoint + "/mnt")
CheckSuccess(err)
code = ts.connector.Mount("/mnt", fs, nil)
if !code.Ok() {
t.Fatal("expect OK:", code)
}
}
func TestUnmountNoExist(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
code := ts.connector.Unmount("/doesnotexist")
if code != EINVAL {
t.Fatal("expect EINVAL", code)
}
}
func TestMountRename(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
fs := NewLoopbackFileSystem(ts.origDir)
code := ts.connector.Mount("/mnt", fs, nil)
if !code.Ok() {
t.Fatal("mount should succeed")
}
err := os.Rename(ts.mountPoint+"/mnt", ts.mountPoint+"/foobar")
if OsErrorToErrno(err) != EBUSY {
t.Fatal("rename mount point should fail with EBUSY:", err)
}
}
func TestMountReaddir(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
fs := NewLoopbackFileSystem(ts.origDir)
code := ts.connector.Mount("/mnt", fs, nil)
if !code.Ok() {
t.Fatal("mount should succeed")
}
entries, err := ioutil.ReadDir(ts.mountPoint)
CheckSuccess(err)
if len(entries) != 1 || entries[0].Name != "mnt" {
t.Error("wrong readdir result", entries)
}
}
func TestRecursiveMount(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
err := ioutil.WriteFile(ts.origDir + "/hello.txt", []byte("blabla"), 0644)
CheckSuccess(err)
fs := NewLoopbackFileSystem(ts.origDir)
code := ts.connector.Mount("/mnt", fs, nil)
if !code.Ok() {
t.Fatal("mount should succeed")
}
submnt := ts.mountPoint + "/mnt"
_, err = os.Lstat(submnt)
CheckSuccess(err)
_, err = os.Lstat(filepath.Join(submnt, "hello.txt"))
CheckSuccess(err)
f, err := os.Open(filepath.Join(submnt, "hello.txt"))
CheckSuccess(err)
log.Println("Attempting unmount, should fail")
code = ts.connector.Unmount("/mnt")
if code != EBUSY {
t.Error("expect EBUSY")
}
f.Close()
log.Println("Waiting for kernel to flush file-close to fuse...")
time.Sleep(1.5e9 * testTtl)
log.Println("Attempting unmount, should succeed")
code = ts.connector.Unmount("/mnt")
if code != OK {
t.Error("umount failed.", code)
}
}
func TestDeletedUnmount(t *testing.T) {
ts := new(testCase)
ts.Setup(t)
defer ts.Cleanup()
submnt := filepath.Join(ts.mountPoint, "mnt")
pfs2 := NewLoopbackFileSystem(ts.origDir)
code := ts.connector.Mount("/mnt", pfs2, nil)
if !code.Ok() {
t.Fatal("Mount error", code)
}
f, err := os.Create(filepath.Join(submnt, "hello.txt"))
CheckSuccess(err)
log.Println("Removing")
err = os.Remove(filepath.Join(submnt, "hello.txt"))
CheckSuccess(err)
log.Println("Removing")
_, err = f.Write([]byte("bla"))
CheckSuccess(err)
code = ts.connector.Unmount("/mnt")
if code != EBUSY {
t.Error("expect EBUSY for unmount with open files", code)
}
f.Close()
time.Sleep(1.5e9 * testTtl)
code = ts.connector.Unmount("/mnt")
if !code.Ok() {
t.Error("should succeed", code)
}
}
...@@ -42,16 +42,8 @@ type mountData struct { ...@@ -42,16 +42,8 @@ type mountData struct {
// If non-nil the file system mounted here. // If non-nil the file system mounted here.
fs FileSystem fs FileSystem
// If yes, we are looking to unmount the mounted fs. // Node that we were mounted on.
// mountPoint *inode
// To be technically correct, we'd have to have a mutex
// protecting this. We don't, keeping the following in mind:
//
// * eventual consistency is OK here
//
// * the kernel controls when to ask for updates,
// so we can't make entries disappear directly anyway.
unmountPending bool
// We could have separate treeLocks per mount; something to // We could have separate treeLocks per mount; something to
// consider if we can measure significant contention for // consider if we can measure significant contention for
...@@ -109,67 +101,86 @@ func (me *mountData) registerFileHandle(node *inode, dir rawDir, f File, flags u ...@@ -109,67 +101,86 @@ func (me *mountData) registerFileHandle(node *inode, dir rawDir, f File, flags u
// Tests should set to true. // Tests should set to true.
var paranoia = false var paranoia = false
// TODO should rename to dentry? // The inode is a combination of dentry (entry in the file/directory
// tree) and inode. We do this, since in the high-level API, each
// files and inodes correspond one-to-one.
type inode struct { type inode struct {
Handled Handled
Parent *inode
Children map[string]*inode // Constant during lifetime.
NodeId uint64 NodeId uint64
Name string
LookupCount int
// Number of open files and its protection.
OpenCountMutex sync.Mutex OpenCountMutex sync.Mutex
OpenCount int OpenCount int
// me.mount.treeLock; we need store this mutex separately,
// since unmount may set me.mount = nil during Unmount().
// Constant during lifetime.
//
// If multiple treeLocks must be acquired, the treeLocks
// closer to the root must be acquired first.
treeLock *sync.RWMutex
// All data below is protected by treeLock.
Name string
Parent *inode
Children map[string]*inode
Mounts map[string]*mountData
LookupCount int
// Non-nil if this is a mountpoint. // Non-nil if this is a mountpoint.
mountPoint *mountData mountPoint *mountData
// The point under which this node is. Should be non-nil for // The file system to which this node belongs. Is constant
// all nodes. // during the lifetime, except upon Unmount() when it is set
// to nil.
mount *mountData mount *mountData
} }
// TotalOpenCount counts open files. It should only be entered from // Must be called with treeLock held.
// an inode which is a mountpoint. func (me *inode) canUnmount() bool {
func (me *inode) TotalOpenCount() int {
o := 0
if me.mountPoint != nil {
me.mountPoint.treeLock.RLock()
defer me.mountPoint.treeLock.RUnlock()
o += me.mountPoint.openFiles.Count()
}
for _, v := range me.Children { for _, v := range me.Children {
o += v.TotalOpenCount() if v.mountPoint != nil {
} // This access may be out of date, but it is no
return o // problem to err on the safe side.
} return false
}
// TotalMountCount counts mountpoints. It should only be entered from if !v.canUnmount() {
// an inode which is a mountpoint. return false
func (me *inode) TotalMountCount() int {
o := 0
if me.mountPoint != nil {
if me.mountPoint.unmountPending {
return 0
} }
o++
me.mountPoint.treeLock.RLock()
defer me.mountPoint.treeLock.RUnlock()
} }
me.OpenCountMutex.Lock()
defer me.OpenCountMutex.Unlock()
return me.OpenCount == 0
}
// Must be called with treeLock held
func (me *inode) recursiveUnmount() {
for _, v := range me.Children { for _, v := range me.Children {
o += v.TotalMountCount() v.recursiveUnmount()
} }
return o me.mount = nil
} }
func (me *inode) IsDir() bool { func (me *inode) IsDir() bool {
return me.Children != nil return me.Children != nil
} }
func (me *inode) GetMountDirEntries() (out []DirEntry) {
me.treeLock.RLock()
defer me.treeLock.RUnlock()
for k, _ := range me.Mounts {
out = append(out, DirEntry{
Name: k,
Mode: S_IFDIR,
})
}
return out
}
const initDirSize = 20 const initDirSize = 20
func (me *inode) verify(cur *mountData) { func (me *inode) verify(cur *mountData) {
...@@ -178,8 +189,8 @@ func (me *inode) verify(cur *mountData) { ...@@ -178,8 +189,8 @@ func (me *inode) verify(cur *mountData) {
panic(fmt.Sprintf("node %v %d should be dead: %v %v", p, me.NodeId, len(me.Children), me.LookupCount)) panic(fmt.Sprintf("node %v %d should be dead: %v %v", p, me.NodeId, len(me.Children), me.LookupCount))
} }
if me.mountPoint != nil { if me.mountPoint != nil {
if me.mountPoint.unmountPending && me.mountPoint.openFiles.Count() > 0 { if me != me.mountPoint.mountPoint {
panic(fmt.Sprintf("cannot have open files for pending unmount")) panic("mountpoint mismatch")
} }
cur = me.mountPoint cur = me.mountPoint
} }
...@@ -187,6 +198,13 @@ func (me *inode) verify(cur *mountData) { ...@@ -187,6 +198,13 @@ func (me *inode) verify(cur *mountData) {
panic(fmt.Sprintf("me.mount not set correctly %v %v", me.mount, cur)) panic(fmt.Sprintf("me.mount not set correctly %v %v", me.mount, cur))
} }
for name, m := range me.Mounts {
if m.mountPoint != me.Children[name] {
panic(fmt.Sprintf("mountpoint parent mismatch: node:%v name:%v ch:%v",
me.mountPoint, name, me.Children))
}
}
for n, ch := range me.Children { for n, ch := range me.Children {
if ch == nil { if ch == nil {
panic("Found nil child.") panic("Found nil child.")
...@@ -216,8 +234,12 @@ func (me *inode) GetFullPath() (path string) { ...@@ -216,8 +234,12 @@ func (me *inode) GetFullPath() (path string) {
// inode. It returns nil for mount if the file was deleted or the // inode. It returns nil for mount if the file was deleted or the
// filesystem unmounted. // filesystem unmounted.
func (me *inode) GetPath() (path string, mount *mountData) { func (me *inode) GetPath() (path string, mount *mountData) {
me.mount.treeLock.RLock() me.treeLock.RLock()
defer me.mount.treeLock.RUnlock() defer me.treeLock.RUnlock()
if me.mount == nil {
// Node from unmounted file system.
return ".deleted", nil
}
if me.NodeId != FUSE_ROOT_ID && me.Parent == nil { if me.NodeId != FUSE_ROOT_ID && me.Parent == nil {
// Deleted node. Treat as if the filesystem was unmounted. // Deleted node. Treat as if the filesystem was unmounted.
...@@ -234,10 +256,6 @@ func (me *inode) GetPath() (path string, mount *mountData) { ...@@ -234,10 +256,6 @@ func (me *inode) GetPath() (path string, mount *mountData) {
panic(fmt.Sprintf("did not find parent with mount: %v", rev_components)) panic(fmt.Sprintf("did not find parent with mount: %v", rev_components))
} }
mount = inode.mountPoint mount = inode.mountPoint
if mount.unmountPending {
return "", nil
}
return ReverseJoin(rev_components, "/"), mount return ReverseJoin(rev_components, "/"), mount
} }
...@@ -255,15 +273,6 @@ func (me *inode) setParent(newParent *inode) { ...@@ -255,15 +273,6 @@ func (me *inode) setParent(newParent *inode) {
} }
} }
oldParent.Children[me.Name] = nil, false oldParent.Children[me.Name] = nil, false
if oldParent.mountPoint != nil && oldParent.mountPoint.unmountPending &&
len(oldParent.Children) == 0 {
oldParent.mountPoint = nil
if oldParent.Parent != nil {
oldParent.mount = oldParent.Parent.mount
}
}
me.Parent = nil me.Parent = nil
} }
if newParent != nil { if newParent != nil {
...@@ -300,10 +309,7 @@ type FileSystemConnector struct { ...@@ -300,10 +309,7 @@ type FileSystemConnector struct {
} }
func (me *FileSystemConnector) Statistics() string { func (me *FileSystemConnector) Statistics() string {
root := me.rootNode return fmt.Sprintf("Inodes %20d\n", me.inodeMap.Count())
return fmt.Sprintf("Mounts %20d\nFiles %20d\nInodes %20d\n",
root.TotalMountCount(),
root.TotalOpenCount(), me.inodeMap.Count())
} }
func (me *FileSystemConnector) getOpenedFile(h uint64) *openedFile { func (me *FileSystemConnector) getOpenedFile(h uint64) *openedFile {
...@@ -342,8 +348,8 @@ func (me *FileSystemConnector) newInode(root bool, isDir bool) *inode { ...@@ -342,8 +348,8 @@ func (me *FileSystemConnector) newInode(root bool, isDir bool) *inode {
func (me *FileSystemConnector) lookupUpdate(parent *inode, name string, isDir bool, lookupCount int) *inode { func (me *FileSystemConnector) lookupUpdate(parent *inode, name string, isDir bool, lookupCount int) *inode {
defer me.verify() defer me.verify()
parent.mount.treeLock.Lock() parent.treeLock.Lock()
defer parent.mount.treeLock.Unlock() defer parent.treeLock.Unlock()
data, ok := parent.Children[name] data, ok := parent.Children[name]
if !ok { if !ok {
...@@ -351,11 +357,29 @@ func (me *FileSystemConnector) lookupUpdate(parent *inode, name string, isDir bo ...@@ -351,11 +357,29 @@ func (me *FileSystemConnector) lookupUpdate(parent *inode, name string, isDir bo
data.Name = name data.Name = name
data.setParent(parent) data.setParent(parent)
data.mount = parent.mount data.mount = parent.mount
data.treeLock = &data.mount.treeLock
} }
data.LookupCount += lookupCount data.LookupCount += lookupCount
return data return data
} }
func (me *FileSystemConnector) lookupMount(parent *inode, name string, lookupCount int) (path string, mount *mountData, isMount bool) {
parent.treeLock.RLock()
defer parent.treeLock.RUnlock()
if parent.Mounts == nil {
return "", nil, false
}
mount, ok := parent.Mounts[name]
if ok {
mount.treeLock.Lock()
defer mount.treeLock.Unlock()
mount.mountPoint.LookupCount += lookupCount
return "", mount, true
}
return "", nil, false
}
func (me *FileSystemConnector) getInodeData(nodeid uint64) *inode { func (me *FileSystemConnector) getInodeData(nodeid uint64) *inode {
if nodeid == FUSE_ROOT_ID { if nodeid == FUSE_ROOT_ID {
return me.rootNode return me.rootNode
...@@ -373,16 +397,25 @@ func (me *FileSystemConnector) forgetUpdate(nodeId uint64, forgetCount int) { ...@@ -373,16 +397,25 @@ func (me *FileSystemConnector) forgetUpdate(nodeId uint64, forgetCount int) {
} }
func (me *FileSystemConnector) considerDropInode(n *inode) { func (me *FileSystemConnector) considerDropInode(n *inode) {
n.mount.treeLock.Lock() if n.Parent != nil {
defer n.mount.treeLock.Unlock() n.Parent.treeLock.Lock()
defer n.Parent.treeLock.Unlock()
}
if n.Parent == nil || n.Parent.treeLock != n.treeLock {
n.treeLock.Lock()
defer n.treeLock.Unlock()
}
n.OpenCountMutex.Lock() n.OpenCountMutex.Lock()
defer n.OpenCountMutex.Unlock() defer n.OpenCountMutex.Unlock()
dropInode := n.LookupCount <= 0 && len(n.Children) == 0 && dropInode := n.LookupCount <= 0 && len(n.Children) == 0 &&
(n.mountPoint == nil || n.mountPoint.unmountPending) && n.OpenCount <= 0 && n != me.rootNode
n.OpenCount <= 0
if dropInode { if dropInode {
n.setParent(nil) if n.mountPoint != nil {
me.unsafeUnmountNode(n)
} else {
n.setParent(nil)
}
if n != me.rootNode { if n != me.rootNode {
me.inodeMap.Forget(n.NodeId) me.inodeMap.Forget(n.NodeId)
} }
...@@ -391,11 +424,12 @@ func (me *FileSystemConnector) considerDropInode(n *inode) { ...@@ -391,11 +424,12 @@ func (me *FileSystemConnector) considerDropInode(n *inode) {
func (me *FileSystemConnector) renameUpdate(oldParent *inode, oldName string, newParent *inode, newName string) { func (me *FileSystemConnector) renameUpdate(oldParent *inode, oldName string, newParent *inode, newName string) {
defer me.verify() defer me.verify()
oldParent.treeLock.Lock()
defer oldParent.treeLock.Unlock()
if oldParent.mount != newParent.mount { if oldParent.mount != newParent.mount {
panic("Cross mount rename") panic("Cross mount rename")
} }
oldParent.mount.treeLock.Lock()
defer oldParent.mount.treeLock.Unlock()
node := oldParent.Children[oldName] node := oldParent.Children[oldName]
if node == nil { if node == nil {
...@@ -414,8 +448,8 @@ func (me *FileSystemConnector) renameUpdate(oldParent *inode, oldName string, ne ...@@ -414,8 +448,8 @@ func (me *FileSystemConnector) renameUpdate(oldParent *inode, oldName string, ne
func (me *FileSystemConnector) unlinkUpdate(parent *inode, name string) { func (me *FileSystemConnector) unlinkUpdate(parent *inode, name string) {
defer me.verify() defer me.verify()
parent.mount.treeLock.Lock() parent.treeLock.Lock()
defer parent.mount.treeLock.Unlock() defer parent.treeLock.Unlock()
node := parent.Children[name] node := parent.Children[name]
node.setParent(nil) node.setParent(nil)
...@@ -453,66 +487,86 @@ func EmptyFileSystemConnector() (out *FileSystemConnector) { ...@@ -453,66 +487,86 @@ func EmptyFileSystemConnector() (out *FileSystemConnector) {
out = new(FileSystemConnector) out = new(FileSystemConnector)
out.inodeMap = NewHandleMap() out.inodeMap = NewHandleMap()
rootData := out.newInode(true, true) out.newInode(true, true)
rootData.Children = make(map[string]*inode, initDirSize)
out.verify() out.verify()
return out return out
} }
// Mount() generates a synthetic directory node, and mounts the file
// system there. If opts is nil, the mount options of the root file
// system are inherited. The encompassing filesystem should pretend
// the mount point does not exist. If it does, it will generate an
// inode with the same, which will cause Mount() to return EBUSY.
//
// Return values:
//
// ENOENT: the directory containing the mount point does not exist.
//
// EBUSY: the intended mount point already exists.
func (me *FileSystemConnector) Mount(mountPoint string, fs FileSystem, opts *FileSystemOptions) Status { func (me *FileSystemConnector) Mount(mountPoint string, fs FileSystem, opts *FileSystemOptions) Status {
var node *inode var node *inode
var parent *inode
if mountPoint != "/" { if mountPoint != "/" {
dirParent, base := filepath.Split(mountPoint) dirParent, base := filepath.Split(mountPoint)
dirParentNode := me.findInode(dirParent) parent = me.findInode(dirParent)
if dirParentNode == nil { if parent == nil {
log.Println("Could not find mountpoint:", mountPoint) log.Println("Could not find mountpoint parent:", dirParent)
return ENOENT return ENOENT
} }
// Make sure we know the mount point.
_, _, node = me.internalLookupWithNode(dirParentNode, base, 0)
} else {
node = me.rootNode
}
if node == nil {
log.Println("Could not find mountpoint:", mountPoint)
return ENOENT
}
if !node.IsDir() { parent.treeLock.Lock()
return EINVAL defer parent.treeLock.Unlock()
} if parent.mount == nil {
return ENOENT
}
node = parent.Children[base]
if node != nil {
return EBUSY
}
if node != me.rootNode { node = me.newInode(false, true)
node.mount.treeLock.Lock() node.Name = base
defer node.mount.treeLock.Unlock() node.setParent(parent)
if opts == nil {
opts = me.rootNode.mountPoint.options
}
} else {
node = me.rootNode
if opts == nil {
opts = NewFileSystemOptions()
}
} }
hasChildren := len(node.Children) > 0 node.mountPoint = newMount(fs)
if hasChildren { node.treeLock = &node.mountPoint.treeLock
return EBUSY node.mount = node.mountPoint
node.mountPoint.mountPoint = node
if parent != nil {
if parent.Mounts == nil {
parent.Mounts = make(map[string]*mountData)
}
parent.Mounts[node.Name] = node.mountPoint
} }
code := fs.Mount(me) node.mountPoint.options = opts
if code != OK {
log.Println("Mount error: ", mountPoint, code)
return code
}
if me.Debug { if me.Debug {
log.Println("Mount: ", fs, "on", mountPoint, node) log.Println("Mount: ", fs, "on dir", mountPoint,
"parent", parent)
} }
fs.Mount(me)
node.mountPoint = newMount(fs)
node.mount = node.mountPoint
if opts == nil {
opts = NewFileSystemOptions()
}
node.mountPoint.options = opts
return OK return OK
} }
// Unmount() tries to unmount the given path. Because of kernel-side
// caching, it may takes a few seconds for files to disappear when
// viewed from user-space.
//
// Returns the following error codes:
//
// EINVAL: path does not exist, or is not a mount point.
//
// EBUSY: there are open files, or submounts below this node.
func (me *FileSystemConnector) Unmount(path string) Status { func (me *FileSystemConnector) Unmount(path string) Status {
node := me.findInode(path) node := me.findInode(path)
if node == nil { if node == nil {
...@@ -520,33 +574,53 @@ func (me *FileSystemConnector) Unmount(path string) Status { ...@@ -520,33 +574,53 @@ func (me *FileSystemConnector) Unmount(path string) Status {
return EINVAL return EINVAL
} }
// Need to lock to look at node.Children parentNode := node.Parent
unmountError := OK if parentNode == nil {
// attempt to unmount root?
return EINVAL
}
// Must lock parent to update tree structure.
parentNode.treeLock.Lock()
defer parentNode.treeLock.Unlock()
mount := node.mountPoint if node.treeLock != parentNode.treeLock {
if mount == nil || mount.unmountPending { node.treeLock.Lock()
unmountError = EINVAL defer node.treeLock.Unlock()
}
if node.mountPoint == nil {
return EINVAL
} }
// don't use defer: we don't want to call out to if node.mountPoint.openFiles.Count() > 0 {
// mount.fs.Unmount() with lock held. return EBUSY
if unmountError.Ok() && (node.TotalOpenCount() > 0 || node.TotalMountCount() > 1) {
unmountError = EBUSY
} }
if unmountError.Ok() { if !node.canUnmount() {
// We settle for eventual consistency. return EBUSY
mount.unmountPending = true
} }
if unmountError.Ok() { me.unsafeUnmountNode(node)
if me.Debug { return OK
log.Println("Unmount: ", mount) }
}
// Assumes node.treeLock and node.Parent.treeLock have been taken.
func (me *FileSystemConnector) unsafeUnmountNode(node *inode) {
if node == me.rootNode {
return
}
node.recursiveUnmount()
unmounted := node.mountPoint
unmounted.mountPoint = nil
node.mountPoint = nil
mount.fs.Unmount() parentNode := node.Parent
node.Parent = nil
if parentNode != nil {
parentNode.Mounts[node.Name] = nil, false
parentNode.Children[node.Name] = nil, false
} }
return unmountError unmounted.fs.Unmount()
} }
func (me *FileSystemConnector) GetPath(nodeid uint64) (path string, mount *mountData, node *inode) { func (me *FileSystemConnector) GetPath(nodeid uint64) (path string, mount *mountData, node *inode) {
...@@ -561,23 +635,21 @@ func (me *FileSystemConnector) GetPath(nodeid uint64) (path string, mount *mount ...@@ -561,23 +635,21 @@ func (me *FileSystemConnector) GetPath(nodeid uint64) (path string, mount *mount
} }
func (me *FileSystemConnector) getOpenFileData(nodeid uint64, fh uint64) (f File, m *mountData, p string) { func (me *FileSystemConnector) getOpenFileData(nodeid uint64, fh uint64) (f File, m *mountData, p string) {
node := me.getInodeData(nodeid)
node.treeLock.RLock()
defer node.treeLock.RUnlock()
if fh != 0 { if fh != 0 {
opened := me.getOpenedFile(fh) opened := me.getOpenedFile(fh)
m = opened.mountData m = opened.mountData
f = opened.file f = opened.file
} }
node := me.getInodeData(nodeid)
node.mount.treeLock.RLock()
defer node.mount.treeLock.RUnlock()
path, maybeNil := node.GetPath() path, mount := node.GetPath()
// If the file was deleted, GetPath() will return nil.
if maybeNil != nil {
if m != nil && maybeNil != m {
panic("mount mismatch")
}
m = maybeNil // If the file was deleted, GetPath() will return nil.
if mount != nil {
m = mount
p = path p = path
} }
return return
......
...@@ -37,11 +37,17 @@ func (me *FileSystemConnector) internalLookup(parent *inode, name string, lookup ...@@ -37,11 +37,17 @@ func (me *FileSystemConnector) internalLookup(parent *inode, name string, lookup
} }
func (me *FileSystemConnector) internalLookupWithNode(parent *inode, name string, lookupCount int) (out *EntryOut, status Status, node *inode) { func (me *FileSystemConnector) internalLookupWithNode(parent *inode, name string, lookupCount int) (out *EntryOut, status Status, node *inode) {
// TODO - fuse.c has special case code for name == "." and fullPath, mount, isMountPoint := me.lookupMount(parent, name, lookupCount)
// "..", those lookups happen if FUSE_EXPORT_SUPPORT is set in if isMountPoint {
// Init. node = mount.mountPoint
fullPath, mount := parent.GetPath() } else {
fullPath, mount = parent.GetPath()
fullPath = filepath.Join(fullPath, name)
}
if mount == nil { if mount == nil {
fmt.Println(me.rootNode)
fmt.Println(me.rootNode.mountPoint)
timeout := me.rootNode.mountPoint.options.NegativeTimeout timeout := me.rootNode.mountPoint.options.NegativeTimeout
if timeout > 0 { if timeout > 0 {
return NegativeEntry(timeout), OK, nil return NegativeEntry(timeout), OK, nil
...@@ -49,10 +55,7 @@ func (me *FileSystemConnector) internalLookupWithNode(parent *inode, name string ...@@ -49,10 +55,7 @@ func (me *FileSystemConnector) internalLookupWithNode(parent *inode, name string
return nil, ENOENT, nil return nil, ENOENT, nil
} }
} }
fullPath = filepath.Join(fullPath, name)
fi, err := mount.fs.GetAttr(fullPath) fi, err := mount.fs.GetAttr(fullPath)
if err == ENOENT && mount.options.NegativeTimeout > 0.0 { if err == ENOENT && mount.options.NegativeTimeout > 0.0 {
return NegativeEntry(mount.options.NegativeTimeout), OK, nil return NegativeEntry(mount.options.NegativeTimeout), OK, nil
} }
...@@ -60,11 +63,12 @@ func (me *FileSystemConnector) internalLookupWithNode(parent *inode, name string ...@@ -60,11 +63,12 @@ func (me *FileSystemConnector) internalLookupWithNode(parent *inode, name string
if err != OK { if err != OK {
return nil, err, nil return nil, err, nil
} }
if !isMountPoint {
data := me.lookupUpdate(parent, name, fi.IsDirectory(), lookupCount) node = me.lookupUpdate(parent, name, fi.IsDirectory(), lookupCount)
}
out = &EntryOut{ out = &EntryOut{
NodeId: data.NodeId, NodeId: node.NodeId,
Generation: 1, // where to get the generation? Generation: 1, // where to get the generation?
} }
SplitNs(mount.options.EntryTimeout, &out.EntryValid, &out.EntryValidNsec) SplitNs(mount.options.EntryTimeout, &out.EntryValid, &out.EntryValidNsec)
...@@ -74,10 +78,10 @@ func (me *FileSystemConnector) internalLookupWithNode(parent *inode, name string ...@@ -74,10 +78,10 @@ func (me *FileSystemConnector) internalLookupWithNode(parent *inode, name string
} }
CopyFileInfo(fi, &out.Attr) CopyFileInfo(fi, &out.Attr)
out.Attr.Ino = data.NodeId out.Attr.Ino = node.NodeId
mount.setOwner(&out.Attr) mount.setOwner(&out.Attr)
return out, OK, data return out, OK, node
} }
...@@ -137,6 +141,7 @@ func (me *FileSystemConnector) OpenDir(header *InHeader, input *OpenIn) (flags u ...@@ -137,6 +141,7 @@ func (me *FileSystemConnector) OpenDir(header *InHeader, input *OpenIn) (flags u
} }
de := &Dir{ de := &Dir{
extra: node.GetMountDirEntries(),
stream: stream, stream: stream,
} }
h := mount.registerFileHandle(node, de, nil, input.Flags) h := mount.registerFileHandle(node, de, nil, input.Flags)
...@@ -328,6 +333,10 @@ func (me *FileSystemConnector) Rename(header *InHeader, input *RenameIn, oldName ...@@ -328,6 +333,10 @@ func (me *FileSystemConnector) Rename(header *InHeader, input *RenameIn, oldName
if mount == nil || oldMount == nil { if mount == nil || oldMount == nil {
return ENOENT return ENOENT
} }
_, _, isMountPoint := me.lookupMount(oldParent, oldName, 0)
if isMountPoint {
return EBUSY
}
if mount != oldMount { if mount != oldMount {
return EXDEV return EXDEV
} }
...@@ -382,6 +391,10 @@ func (me *FileSystemConnector) Create(header *InHeader, input *CreateIn, name st ...@@ -382,6 +391,10 @@ func (me *FileSystemConnector) Create(header *InHeader, input *CreateIn, name st
} }
out, code, inode := me.internalLookupWithNode(parent, name, 1) out, code, inode := me.internalLookupWithNode(parent, name, 1)
if inode == nil {
msg := fmt.Sprintf("Create succeded, but GetAttr returned no entry %v", fullPath)
panic(msg)
}
return 0, mount.registerFileHandle(inode, nil, f, input.Flags), out, code return 0, mount.registerFileHandle(inode, nil, f, input.Flags), out, code
} }
......
...@@ -135,9 +135,9 @@ func (me *TimingFileSystem) OpenDir(name string) (stream chan DirEntry, status S ...@@ -135,9 +135,9 @@ func (me *TimingFileSystem) OpenDir(name string) (stream chan DirEntry, status S
return me.FileSystem.OpenDir(name) return me.FileSystem.OpenDir(name)
} }
func (me *TimingFileSystem) Mount(conn *FileSystemConnector) Status { func (me *TimingFileSystem) Mount(conn *FileSystemConnector) {
defer me.startTimer("Mount", "")() defer me.startTimer("Mount", "")()
return me.FileSystem.Mount(conn) me.FileSystem.Mount(conn)
} }
func (me *TimingFileSystem) Unmount() { func (me *TimingFileSystem) Unmount() {
......
...@@ -60,12 +60,11 @@ func NewAutoUnionFs(directory string, options AutoUnionFsOptions) *AutoUnionFs { ...@@ -60,12 +60,11 @@ func NewAutoUnionFs(directory string, options AutoUnionFsOptions) *AutoUnionFs {
return a return a
} }
func (me *AutoUnionFs) Mount(connector *fuse.FileSystemConnector) fuse.Status { func (me *AutoUnionFs) Mount(connector *fuse.FileSystemConnector) {
me.connector = connector me.connector = connector
if me.options.UpdateOnMount { if me.options.UpdateOnMount {
time.AfterFunc(0.1e9, func() { me.updateKnownFses() }) time.AfterFunc(0.1e9, func() { me.updateKnownFses() })
} }
return fuse.OK
} }
func (me *AutoUnionFs) addAutomaticFs(roots []string) { func (me *AutoUnionFs) addAutomaticFs(roots []string) {
...@@ -343,15 +342,12 @@ func (me *AutoUnionFs) OpenDir(name string) (stream chan fuse.DirEntry, status f ...@@ -343,15 +342,12 @@ func (me *AutoUnionFs) OpenDir(name string) (stream chan fuse.DirEntry, status f
defer me.lock.RUnlock() defer me.lock.RUnlock()
stream = make(chan fuse.DirEntry, len(me.knownFileSystems)+5) stream = make(chan fuse.DirEntry, len(me.knownFileSystems)+5)
for k, _ := range me.knownFileSystems { if name == _CONFIG {
mode := fuse.S_IFDIR | 0755 for k, _ := range me.knownFileSystems {
if name == _CONFIG { stream <- fuse.DirEntry{
mode = syscall.S_IFLNK | 0644 Name: k,
} Mode: syscall.S_IFLNK | 0644,
}
stream <- fuse.DirEntry{
Name: k,
Mode: uint32(mode),
} }
} }
......
...@@ -74,6 +74,7 @@ func TestAutoFsSymlink(t *testing.T) { ...@@ -74,6 +74,7 @@ func TestAutoFsSymlink(t *testing.T) {
err := os.Mkdir(wd+"/store/backing1", 0755) err := os.Mkdir(wd+"/store/backing1", 0755)
CheckSuccess(err) CheckSuccess(err)
os.Symlink(wd+"/ro", wd+"/store/backing1/READONLY") os.Symlink(wd+"/ro", wd+"/store/backing1/READONLY")
CheckSuccess(err) CheckSuccess(err)
...@@ -83,6 +84,12 @@ func TestAutoFsSymlink(t *testing.T) { ...@@ -83,6 +84,12 @@ func TestAutoFsSymlink(t *testing.T) {
fi, err := os.Lstat(wd + "/mount/manual1/file1") fi, err := os.Lstat(wd + "/mount/manual1/file1")
CheckSuccess(err) CheckSuccess(err)
entries, err := ioutil.ReadDir(wd + "/mount")
CheckSuccess(err)
if len(entries) != 3 {
t.Error("readdir mismatch", entries)
}
err = os.Remove(wd + "/mount/config/manual1") err = os.Remove(wd + "/mount/config/manual1")
CheckSuccess(err) CheckSuccess(err)
......
...@@ -5,8 +5,8 @@ package zipfs ...@@ -5,8 +5,8 @@ package zipfs
This provides a practical example of mounting Go-fuse path filesystems This provides a practical example of mounting Go-fuse path filesystems
on top of each other. on top of each other.
It is a file system that configures a Zip filesystem at /zipmount when writing It is a file system that configures a Zip filesystem at /zipmount when
path/to/zipfile to /config/zipmount symlinking path/to/zipfile to /config/zipmount
*/ */
...@@ -16,7 +16,6 @@ import ( ...@@ -16,7 +16,6 @@ import (
"os" "os"
"path/filepath" "path/filepath"
"sync" "sync"
"strings"
) )
var _ = log.Printf var _ = log.Printf
...@@ -25,47 +24,6 @@ const ( ...@@ -25,47 +24,6 @@ const (
CONFIG_PREFIX = "config/" CONFIG_PREFIX = "config/"
) )
// zipCreateFile is a placeholder file to receive the write containing
// the path to the zip file.
type zipCreateFile struct {
// Basename of the entry in the FS.
Basename string
zfs *MultiZipFs
fuse.DefaultFile
}
func (me *zipCreateFile) Write(input *fuse.WriteIn, nameBytes []byte) (uint32, fuse.Status) {
if me.zfs == nil {
// TODO
return 0, fuse.EPERM
}
zipFile := string(nameBytes)
zipFile = strings.Trim(zipFile, "\n ")
fs, err := NewArchiveFileSystem(zipFile)
if err != nil {
// TODO
log.Println("NewZipArchiveFileSystem failed.")
me.zfs.pendingZips[me.Basename] = false, false
return 0, fuse.ENOSYS
}
code := me.zfs.Connector.Mount("/"+filepath.Base(me.Basename), fs, nil)
if code != fuse.OK {
return 0, code
}
// TODO. locks?
me.zfs.zips[me.Basename] = fs
me.zfs.dirZipFileMap[me.Basename] = zipFile
me.zfs.pendingZips[me.Basename] = false, false
me.zfs = nil
return uint32(len(nameBytes)), code
}
//////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////
...@@ -76,7 +34,6 @@ type MultiZipFs struct { ...@@ -76,7 +34,6 @@ type MultiZipFs struct {
Connector *fuse.FileSystemConnector Connector *fuse.FileSystemConnector
lock sync.RWMutex lock sync.RWMutex
zips map[string]*MemTreeFileSystem zips map[string]*MemTreeFileSystem
pendingZips map[string]bool
dirZipFileMap map[string]string dirZipFileMap map[string]string
fuse.DefaultFileSystem fuse.DefaultFileSystem
...@@ -85,43 +42,19 @@ type MultiZipFs struct { ...@@ -85,43 +42,19 @@ type MultiZipFs struct {
func NewMultiZipFs() *MultiZipFs { func NewMultiZipFs() *MultiZipFs {
m := new(MultiZipFs) m := new(MultiZipFs)
m.zips = make(map[string]*MemTreeFileSystem) m.zips = make(map[string]*MemTreeFileSystem)
m.pendingZips = make(map[string]bool)
m.dirZipFileMap = make(map[string]string) m.dirZipFileMap = make(map[string]string)
return m return m
} }
func (me *MultiZipFs) Mount(connector *fuse.FileSystemConnector) fuse.Status { func (me *MultiZipFs) Mount(connector *fuse.FileSystemConnector) {
me.Connector = connector me.Connector = connector
return fuse.OK
} }
func (me *MultiZipFs) OpenDir(name string) (stream chan fuse.DirEntry, code fuse.Status) { func (me *MultiZipFs) OpenDir(name string) (stream chan fuse.DirEntry, code fuse.Status) {
me.lock.RLock() me.lock.RLock()
defer me.lock.RUnlock() defer me.lock.RUnlock()
// We don't use a goroutine, since we don't want to hold the stream = make(chan fuse.DirEntry, len(me.zips)+2)
// lock.
stream = make(chan fuse.DirEntry,
len(me.pendingZips)+len(me.zips)+2)
submode := uint32(fuse.S_IFDIR | 0700)
if name == "config" {
submode = fuse.S_IFREG | 0600
}
for k, _ := range me.zips {
var d fuse.DirEntry
d.Name = k
d.Mode = submode
stream <- fuse.DirEntry(d)
}
for k, _ := range me.pendingZips {
var d fuse.DirEntry
d.Name = k
d.Mode = submode
stream <- fuse.DirEntry(d)
}
if name == "" { if name == "" {
var d fuse.DirEntry var d fuse.DirEntry
d.Name = "config" d.Name = "config"
...@@ -129,6 +62,15 @@ func (me *MultiZipFs) OpenDir(name string) (stream chan fuse.DirEntry, code fuse ...@@ -129,6 +62,15 @@ func (me *MultiZipFs) OpenDir(name string) (stream chan fuse.DirEntry, code fuse
stream <- fuse.DirEntry(d) stream <- fuse.DirEntry(d)
} }
if name == "config" {
for k, _ := range me.zips {
var d fuse.DirEntry
d.Name = k
d.Mode = fuse.S_IFLNK
stream <- fuse.DirEntry(d)
}
}
close(stream) close(stream)
return stream, fuse.OK return stream, fuse.OK
} }
...@@ -153,7 +95,7 @@ func (me *MultiZipFs) GetAttr(name string) (*os.FileInfo, fuse.Status) { ...@@ -153,7 +95,7 @@ func (me *MultiZipFs) GetAttr(name string) (*os.FileInfo, fuse.Status) {
} }
submode := uint32(fuse.S_IFDIR | 0700) submode := uint32(fuse.S_IFDIR | 0700)
if dir == CONFIG_PREFIX { if dir == CONFIG_PREFIX {
submode = fuse.S_IFREG | 0600 submode = fuse.S_IFLNK | 0600
} }
me.lock.RLock() me.lock.RLock()
...@@ -164,10 +106,6 @@ func (me *MultiZipFs) GetAttr(name string) (*os.FileInfo, fuse.Status) { ...@@ -164,10 +106,6 @@ func (me *MultiZipFs) GetAttr(name string) (*os.FileInfo, fuse.Status) {
if hasDir { if hasDir {
return a, fuse.OK return a, fuse.OK
} }
_, hasDir = me.pendingZips[base]
if hasDir {
return a, fuse.OK
}
return nil, fuse.ENOENT return nil, fuse.ENOENT
} }
...@@ -180,6 +118,10 @@ func (me *MultiZipFs) Unlink(name string) (code fuse.Status) { ...@@ -180,6 +118,10 @@ func (me *MultiZipFs) Unlink(name string) (code fuse.Status) {
_, ok := me.zips[basename] _, ok := me.zips[basename]
if ok { if ok {
code = me.Connector.Unmount("/" + basename)
if !code.Ok() {
return code
}
me.zips[basename] = nil, false me.zips[basename] = nil, false
me.dirZipFileMap[basename] = "", false me.dirZipFileMap[basename] = "", false
return fuse.OK return fuse.OK
...@@ -190,41 +132,48 @@ func (me *MultiZipFs) Unlink(name string) (code fuse.Status) { ...@@ -190,41 +132,48 @@ func (me *MultiZipFs) Unlink(name string) (code fuse.Status) {
return fuse.EPERM return fuse.EPERM
} }
func (me *MultiZipFs) Open(name string, flags uint32) (file fuse.File, code fuse.Status) { func (me *MultiZipFs) Readlink(path string) (val string, code fuse.Status) {
if 0 != flags&uint32(fuse.O_ANYWRITE) { dir, base := filepath.Split(path)
return nil, fuse.EPERM if dir != CONFIG_PREFIX {
return "", fuse.ENOENT
} }
dir, basename := filepath.Split(name) me.lock.Lock()
if dir == CONFIG_PREFIX { defer me.lock.Unlock()
me.lock.RLock()
defer me.lock.RUnlock()
orig, ok := me.dirZipFileMap[basename]
if !ok {
return nil, fuse.ENOENT
}
return fuse.NewReadOnlyFile([]byte(orig)), fuse.OK zipfile, ok := me.dirZipFileMap[base]
if !ok {
return "", fuse.ENOENT
} }
return zipfile, fuse.OK
return nil, fuse.ENOENT
} }
func (me *MultiZipFs) Symlink(value string, linkName string) (code fuse.Status) {
func (me *MultiZipFs) Create(name string, flags uint32, mode uint32) (file fuse.File, code fuse.Status) { dir, base := filepath.Split(linkName)
dir, base := filepath.Split(name)
if dir != CONFIG_PREFIX { if dir != CONFIG_PREFIX {
return nil, fuse.EPERM return fuse.EPERM
} }
z := new(zipCreateFile)
z.Basename = base
z.zfs = me
me.lock.Lock() me.lock.Lock()
defer me.lock.Unlock() defer me.lock.Unlock()
me.pendingZips[z.Basename] = true _, ok := me.dirZipFileMap[base]
if ok {
return fuse.EBUSY
}
return z, fuse.OK fs, err := NewArchiveFileSystem(value)
if err != nil {
log.Println("NewZipArchiveFileSystem failed.", err)
return fuse.EINVAL
}
code = me.Connector.Mount("/"+base, fs, nil)
if !code.Ok() {
return code
}
me.dirZipFileMap[base] = value
me.zips[base] = fs
return fuse.OK
} }
...@@ -2,6 +2,7 @@ package zipfs ...@@ -2,6 +2,7 @@ package zipfs
import ( import (
"github.com/hanwen/go-fuse/fuse" "github.com/hanwen/go-fuse/fuse"
"io/ioutil"
"log" "log"
"os" "os"
"testing" "testing"
...@@ -13,85 +14,84 @@ var CheckSuccess = fuse.CheckSuccess ...@@ -13,85 +14,84 @@ var CheckSuccess = fuse.CheckSuccess
const testTtl = 0.1 const testTtl = 0.1
func TestMultiZipFs(t *testing.T) {
var err os.Error
wd, err := os.Getwd()
zipFile := wd + "/test.zip"
func setupMzfs() (mountPoint string, cleanup func()) {
fs := NewMultiZipFs() fs := NewMultiZipFs()
mountPoint := fuse.MakeTempDir() mountPoint = fuse.MakeTempDir()
state, _, err := fuse.MountFileSystem(mountPoint, fs, &fuse.FileSystemOptions{ state, _, err := fuse.MountFileSystem(mountPoint, fs, &fuse.FileSystemOptions{
EntryTimeout: testTtl, EntryTimeout: testTtl,
AttrTimeout: testTtl, AttrTimeout: testTtl,
NegativeTimeout: 0.0, NegativeTimeout: 0.0,
}) })
defer os.RemoveAll(mountPoint)
CheckSuccess(err) CheckSuccess(err)
defer state.Unmount()
state.Debug = true state.Debug = true
go state.Loop(true) go state.Loop(true)
f, err := os.Open(mountPoint + "") return mountPoint, func() {
CheckSuccess(err) state.Unmount()
os.RemoveAll(mountPoint)
names, err := f.Readdirnames(-1)
CheckSuccess(err)
if len(names) != 1 || string(names[0]) != "config" {
t.Errorf("wrong names return. %v", names)
} }
err = f.Close() }
CheckSuccess(err)
f, err = os.Create(mountPoint + "/random") func TestMultiZipReadonly(t *testing.T) {
mountPoint, cleanup := setupMzfs()
defer cleanup()
_, err := os.Create(mountPoint + "/random")
if err == nil { if err == nil {
t.Error("Must fail writing in root.") t.Error("Must fail writing in root.")
} }
f, err = os.OpenFile(mountPoint+"/config/zipmount", os.O_WRONLY, 0) _, err = os.OpenFile(mountPoint+"/config/zipmount", os.O_WRONLY, 0)
if err == nil { if err == nil {
t.Error("Must fail without O_CREATE") t.Error("Must fail without O_CREATE")
} }
f, err = os.Create(mountPoint + "/config/zipmount") }
func TestMultiZipFs(t *testing.T) {
mountPoint, cleanup := setupMzfs()
defer cleanup()
wd, err := os.Getwd()
zipFile := wd + "/test.zip"
entries, err := ioutil.ReadDir(mountPoint)
CheckSuccess(err)
if len(entries) != 1 || string(entries[0].Name) != "config" {
t.Errorf("wrong names return. %v", entries)
}
err = os.Symlink(zipFile, mountPoint + "/config/zipmount")
CheckSuccess(err) CheckSuccess(err)
// Directory exists, but is empty.
fi, err := os.Lstat(mountPoint + "/zipmount") fi, err := os.Lstat(mountPoint + "/zipmount")
CheckSuccess(err)
if !fi.IsDirectory() { if !fi.IsDirectory() {
t.Errorf("Expect directory at /zipmount") t.Errorf("Expect directory at /zipmount")
} }
// Open the zip file. entries, err = ioutil.ReadDir(mountPoint)
_, err = f.Write([]byte(zipFile))
CheckSuccess(err) CheckSuccess(err)
if len(entries) != 2 {
_, err = f.Write([]byte(zipFile)) t.Error("Expect 2 entries", entries)
if err == nil {
t.Error("Must fail second write.")
} }
err = f.Close() val, err := os.Readlink(mountPoint + "/config/zipmount")
CheckSuccess(err) CheckSuccess(err)
fi, err = os.Lstat(mountPoint + "/zipmount") if val != zipFile {
if !fi.IsDirectory() { t.Errorf("expected %v got %v", zipFile, val)
t.Errorf("Expect directory at /zipmount")
} }
// Check that zipfs itself works. // Check that zipfs itself works.
fi, err = os.Stat(mountPoint + "/zipmount/subdir") fi, err = os.Stat(mountPoint + "/zipmount/subdir")
CheckSuccess(err) CheckSuccess(err)
if !fi.IsDirectory() { if !fi.IsDirectory() {
t.Error("directory type", fi) t.Error("directory type", fi)
} }
// Removing the config dir unmount // Removing the config dir unmount
err = os.Remove(mountPoint + "/config/zipmount") err = os.Remove(mountPoint + "/config/zipmount")
CheckSuccess(err) CheckSuccess(err)
// This is ugly but necessary: We don't have ways to signal // This is ugly but necessary: We don't have ways to signal
// back to FUSE that the file disappeared. // back to FUSE that the file disappeared.
time.Sleep(1.5e9 * testTtl) time.Sleep(1.5e9 * testTtl)
......
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