Browse Source

multiple fixes

* adjust isOpen count
* move ContinuousDirtyPages lock to filehandle
* fix problem with MergeIntoVisibles, avoid reusing slices
* let filer delete the garbage
pull/1445/head
Chris Lu 4 years ago
parent
commit
aee27ccbe1
  1. 27
      weed/filer2/filechunks.go
  2. 1
      weed/filesys/dir.go
  3. 27
      weed/filesys/dirty_page.go
  4. 17
      weed/filesys/file.go
  5. 53
      weed/filesys/filehandle.go
  6. 4
      weed/filesys/meta_cache/meta_cache.go
  7. 2
      weed/filesys/meta_cache/meta_cache_init.go
  8. 4
      weed/filesys/wfs.go
  9. 2
      weed/filesys/xattr.go
  10. 5
      weed/util/bounded_tree/bounded_tree.go

27
weed/filer2/filechunks.go

@ -158,9 +158,9 @@ func logPrintf(name string, visibles []VisibleInterval) {
/* /*
glog.V(0).Infof("%s len %d", name, len(visibles)) glog.V(0).Infof("%s len %d", name, len(visibles))
for _, v := range visibles { for _, v := range visibles {
glog.V(0).Infof("%s: [%d,%d)", name, v.start, v.stop)
glog.V(0).Infof("%s: [%d,%d) %s %d", name, v.start, v.stop, v.fileId, v.chunkOffset)
} }
*/
*/
} }
var bufPool = sync.Pool{ var bufPool = sync.Pool{
@ -169,7 +169,7 @@ var bufPool = sync.Pool{
}, },
} }
func MergeIntoVisibles(visibles, newVisibles []VisibleInterval, chunk *filer_pb.FileChunk) []VisibleInterval {
func MergeIntoVisibles(visibles []VisibleInterval, chunk *filer_pb.FileChunk) (newVisibles []VisibleInterval) {
newV := newVisibleInterval(chunk.Offset, chunk.Offset+int64(chunk.Size), chunk.GetFileIdString(), chunk.Mtime, 0, chunk.Size, chunk.CipherKey, chunk.IsCompressed) newV := newVisibleInterval(chunk.Offset, chunk.Offset+int64(chunk.Size), chunk.GetFileIdString(), chunk.Mtime, 0, chunk.Size, chunk.CipherKey, chunk.IsCompressed)
@ -183,16 +183,22 @@ func MergeIntoVisibles(visibles, newVisibles []VisibleInterval, chunk *filer_pb.
} }
logPrintf(" before", visibles) logPrintf(" before", visibles)
// glog.V(0).Infof("newVisibles %d adding chunk [%d,%d) %s size:%d", len(newVisibles), chunk.Offset, chunk.Offset+int64(chunk.Size), chunk.GetFileIdString(), chunk.Size)
chunkStop := chunk.Offset + int64(chunk.Size) chunkStop := chunk.Offset + int64(chunk.Size)
for _, v := range visibles { for _, v := range visibles {
if v.start < chunk.Offset && chunk.Offset < v.stop { if v.start < chunk.Offset && chunk.Offset < v.stop {
newVisibles = append(newVisibles, newVisibleInterval(v.start, chunk.Offset, v.fileId, v.modifiedTime, v.chunkOffset, v.chunkSize, v.cipherKey, v.isGzipped))
t := newVisibleInterval(v.start, chunk.Offset, v.fileId, v.modifiedTime, v.chunkOffset, v.chunkSize, v.cipherKey, v.isGzipped)
newVisibles = append(newVisibles, t)
// glog.V(0).Infof("visible %d [%d,%d) =1> [%d,%d)", i, v.start, v.stop, t.start, t.stop)
} }
if v.start < chunkStop && chunkStop < v.stop { if v.start < chunkStop && chunkStop < v.stop {
newVisibles = append(newVisibles, newVisibleInterval(chunkStop, v.stop, v.fileId, v.modifiedTime, v.chunkOffset+(chunkStop-v.start), v.chunkSize, v.cipherKey, v.isGzipped))
t := newVisibleInterval(chunkStop, v.stop, v.fileId, v.modifiedTime, v.chunkOffset+(chunkStop-v.start), v.chunkSize, v.cipherKey, v.isGzipped)
newVisibles = append(newVisibles, t)
// glog.V(0).Infof("visible %d [%d,%d) =2> [%d,%d)", i, v.start, v.stop, t.start, t.stop)
} }
if chunkStop <= v.start || v.stop <= chunk.Offset { if chunkStop <= v.start || v.stop <= chunk.Offset {
newVisibles = append(newVisibles, v) newVisibles = append(newVisibles, v)
// glog.V(0).Infof("visible %d [%d,%d) =3> [%d,%d)", i, v.start, v.stop, v.start, v.stop)
} }
} }
newVisibles = append(newVisibles, newV) newVisibles = append(newVisibles, newV)
@ -219,17 +225,16 @@ func NonOverlappingVisibleIntervals(lookupFileIdFn LookupFileIdFunctionType, chu
chunks, _, err = ResolveChunkManifest(lookupFileIdFn, chunks) chunks, _, err = ResolveChunkManifest(lookupFileIdFn, chunks)
sort.Slice(chunks, func(i, j int) bool { sort.Slice(chunks, func(i, j int) bool {
return chunks[i].Fid.FileKey < chunks[j].Fid.FileKey
if chunks[i].Mtime == chunks[j].Mtime {
return chunks[i].Fid.FileKey < chunks[j].Fid.FileKey
}
return chunks[i].Mtime < chunks[j].Mtime // keep this to make tests run
}) })
var newVisibles []VisibleInterval
for _, chunk := range chunks { for _, chunk := range chunks {
// glog.V(0).Infof("merge [%d,%d)", chunk.Offset, chunk.Offset+int64(chunk.Size)) // glog.V(0).Infof("merge [%d,%d)", chunk.Offset, chunk.Offset+int64(chunk.Size))
newVisibles = MergeIntoVisibles(visibles, newVisibles, chunk)
t := visibles[:0]
visibles = newVisibles
newVisibles = t
visibles = MergeIntoVisibles(visibles, chunk)
logPrintf("add", visibles) logPrintf("add", visibles)

1
weed/filesys/dir.go

@ -168,7 +168,6 @@ func (dir *Dir) Create(ctx context.Context, req *fuse.CreateRequest,
node = dir.newFile(req.Name, request.Entry) node = dir.newFile(req.Name, request.Entry)
file := node.(*File) file := node.(*File)
file.isOpen++
fh := dir.wfs.AcquireHandle(file, req.Uid, req.Gid) fh := dir.wfs.AcquireHandle(file, req.Uid, req.Gid)
return file, fh, nil return file, fh, nil

27
weed/filesys/dirty_page.go

@ -29,9 +29,6 @@ var counter = int32(0)
func (pages *ContinuousDirtyPages) AddPage(offset int64, data []byte) (chunks []*filer_pb.FileChunk, err error) { func (pages *ContinuousDirtyPages) AddPage(offset int64, data []byte) (chunks []*filer_pb.FileChunk, err error) {
pages.lock.Lock()
defer pages.lock.Unlock()
glog.V(5).Infof("%s AddPage [%d,%d) of %d bytes", pages.f.fullpath(), offset, offset+int64(len(data)), pages.f.entry.Attributes.FileSize) glog.V(5).Infof("%s AddPage [%d,%d) of %d bytes", pages.f.fullpath(), offset, offset+int64(len(data)), pages.f.entry.Attributes.FileSize)
if len(data) > int(pages.f.wfs.option.ChunkSizeLimit) { if len(data) > int(pages.f.wfs.option.ChunkSizeLimit) {
@ -82,14 +79,6 @@ func (pages *ContinuousDirtyPages) flushAndSave(offset int64, data []byte) (chun
return return
} }
func (pages *ContinuousDirtyPages) FlushToStorage() (chunks []*filer_pb.FileChunk, err error) {
pages.lock.Lock()
defer pages.lock.Unlock()
return pages.saveExistingPagesToStorage()
}
func (pages *ContinuousDirtyPages) saveExistingPagesToStorage() (chunks []*filer_pb.FileChunk, err error) { func (pages *ContinuousDirtyPages) saveExistingPagesToStorage() (chunks []*filer_pb.FileChunk, err error) {
var hasSavedData bool var hasSavedData bool
@ -103,7 +92,9 @@ func (pages *ContinuousDirtyPages) saveExistingPagesToStorage() (chunks []*filer
} }
if err == nil { if err == nil {
chunks = append(chunks, chunk)
if chunk != nil {
chunks = append(chunks, chunk)
}
} else { } else {
return return
} }
@ -121,9 +112,14 @@ func (pages *ContinuousDirtyPages) saveExistingLargestPageToStorage() (chunk *fi
fileSize := int64(pages.f.entry.Attributes.FileSize) fileSize := int64(pages.f.entry.Attributes.FileSize)
for { for {
chunkSize := min(maxList.Size(), fileSize-maxList.Offset()) chunkSize := min(maxList.Size(), fileSize-maxList.Offset())
if chunkSize == 0 {
return
}
chunk, err = pages.saveToStorage(maxList.ToReader(), maxList.Offset(), chunkSize) chunk, err = pages.saveToStorage(maxList.ToReader(), maxList.Offset(), chunkSize)
if err == nil { if err == nil {
hasSavedData = true
if chunk != nil {
hasSavedData = true
}
glog.V(4).Infof("saveToStorage %s %s [%d,%d) of %d bytes", pages.f.fullpath(), chunk.GetFileIdString(), maxList.Offset(), maxList.Offset()+chunkSize, fileSize) glog.V(4).Infof("saveToStorage %s %s [%d,%d) of %d bytes", pages.f.fullpath(), chunk.GetFileIdString(), maxList.Offset(), maxList.Offset()+chunkSize, fileSize)
return return
} else { } else {
@ -170,10 +166,5 @@ func min(x, y int64) int64 {
} }
func (pages *ContinuousDirtyPages) ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64) { func (pages *ContinuousDirtyPages) ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64) {
pages.lock.Lock()
defer pages.lock.Unlock()
return pages.intervals.ReadDataAt(data, startOffset) return pages.intervals.ReadDataAt(data, startOffset)
} }

17
weed/filesys/file.go

@ -87,8 +87,6 @@ func (file *File) Open(ctx context.Context, req *fuse.OpenRequest, resp *fuse.Op
glog.V(4).Infof("file %v open %+v", file.fullpath(), req) glog.V(4).Infof("file %v open %+v", file.fullpath(), req)
file.isOpen++
handle := file.wfs.AcquireHandle(file, req.Uid, req.Gid) handle := file.wfs.AcquireHandle(file, req.Uid, req.Gid)
resp.Handle = fuse.HandleID(handle.handle) resp.Handle = fuse.HandleID(handle.handle)
@ -120,7 +118,7 @@ func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *f
if req.Valid.Size() { if req.Valid.Size() {
glog.V(4).Infof("%v file setattr set size=%v chunks=%d", file.fullpath(), req.Size, len(file.entry.Chunks)) glog.V(4).Infof("%v file setattr set size=%v chunks=%d", file.fullpath(), req.Size, len(file.entry.Chunks))
if req.Size < filer2.TotalSize(file.entry.Chunks) {
if req.Size < filer2.FileSize(file.entry) {
// fmt.Printf("truncate %v \n", fullPath) // fmt.Printf("truncate %v \n", fullPath)
var chunks []*filer_pb.FileChunk var chunks []*filer_pb.FileChunk
var truncatedChunks []*filer_pb.FileChunk var truncatedChunks []*filer_pb.FileChunk
@ -252,7 +250,7 @@ func (file *File) Forget() {
} }
func (file *File) maybeLoadEntry(ctx context.Context) error { func (file *File) maybeLoadEntry(ctx context.Context) error {
if file.entry == nil || file.isOpen <= 0 {
if file.entry == nil && file.isOpen <= 0 {
entry, err := file.wfs.maybeLoadEntry(file.dir.FullPath(), file.Name) entry, err := file.wfs.maybeLoadEntry(file.dir.FullPath(), file.Name)
if err != nil { if err != nil {
glog.V(3).Infof("maybeLoadEntry file %s/%s: %v", file.dir.FullPath(), file.Name, err) glog.V(3).Infof("maybeLoadEntry file %s/%s: %v", file.dir.FullPath(), file.Name, err)
@ -268,15 +266,14 @@ func (file *File) maybeLoadEntry(ctx context.Context) error {
func (file *File) addChunks(chunks []*filer_pb.FileChunk) { func (file *File) addChunks(chunks []*filer_pb.FileChunk) {
sort.Slice(chunks, func(i, j int) bool { sort.Slice(chunks, func(i, j int) bool {
return chunks[i].Fid.FileKey < chunks[j].Fid.FileKey
if chunks[i].Mtime == chunks[j].Mtime {
return chunks[i].Fid.FileKey < chunks[j].Fid.FileKey
}
return chunks[i].Mtime < chunks[j].Mtime
}) })
var newVisibles []filer2.VisibleInterval
for _, chunk := range chunks { for _, chunk := range chunks {
newVisibles = filer2.MergeIntoVisibles(file.entryViewCache, newVisibles, chunk)
t := file.entryViewCache[:0]
file.entryViewCache = newVisibles
newVisibles = t
file.entryViewCache = filer2.MergeIntoVisibles(file.entryViewCache, chunk)
} }
file.reader = nil file.reader = nil

53
weed/filesys/filehandle.go

@ -7,6 +7,7 @@ import (
"math" "math"
"net/http" "net/http"
"os" "os"
"sync"
"time" "time"
"github.com/seaweedfs/fuse" "github.com/seaweedfs/fuse"
@ -22,6 +23,7 @@ type FileHandle struct {
dirtyPages *ContinuousDirtyPages dirtyPages *ContinuousDirtyPages
contentType string contentType string
handle uint64 handle uint64
sync.RWMutex
f *File f *File
RequestId fuse.RequestID // unique ID for request RequestId fuse.RequestID // unique ID for request
@ -41,6 +43,7 @@ func newFileHandle(file *File, uid, gid uint32) *FileHandle {
if fh.f.entry != nil { if fh.f.entry != nil {
fh.f.entry.Attributes.FileSize = filer2.FileSize(fh.f.entry) fh.f.entry.Attributes.FileSize = filer2.FileSize(fh.f.entry)
} }
return fh return fh
} }
@ -55,6 +58,12 @@ var _ = fs.HandleReleaser(&FileHandle{})
func (fh *FileHandle) Read(ctx context.Context, req *fuse.ReadRequest, resp *fuse.ReadResponse) error { func (fh *FileHandle) Read(ctx context.Context, req *fuse.ReadRequest, resp *fuse.ReadResponse) error {
glog.V(4).Infof("%s read fh %d: [%d,%d) size %d resp.Data cap=%d", fh.f.fullpath(), fh.handle, req.Offset, req.Offset+int64(req.Size), req.Size, cap(resp.Data)) glog.V(4).Infof("%s read fh %d: [%d,%d) size %d resp.Data cap=%d", fh.f.fullpath(), fh.handle, req.Offset, req.Offset+int64(req.Size), req.Size, cap(resp.Data))
fh.RLock()
defer fh.RUnlock()
if req.Size <= 0 {
return nil
}
buff := resp.Data[:cap(resp.Data)] buff := resp.Data[:cap(resp.Data)]
if req.Size > cap(resp.Data) { if req.Size > cap(resp.Data) {
@ -65,7 +74,7 @@ func (fh *FileHandle) Read(ctx context.Context, req *fuse.ReadRequest, resp *fus
totalRead, err := fh.readFromChunks(buff, req.Offset) totalRead, err := fh.readFromChunks(buff, req.Offset)
if err == nil { if err == nil {
maxStop := fh.readFromDirtyPages(buff, req.Offset) maxStop := fh.readFromDirtyPages(buff, req.Offset)
totalRead = max(maxStop - req.Offset, totalRead)
totalRead = max(maxStop-req.Offset, totalRead)
} }
if err != nil { if err != nil {
@ -77,13 +86,15 @@ func (fh *FileHandle) Read(ctx context.Context, req *fuse.ReadRequest, resp *fus
glog.Warningf("%s FileHandle Read %d: [%d,%d) size %d totalRead %d", fh.f.fullpath(), fh.handle, req.Offset, req.Offset+int64(req.Size), req.Size, totalRead) glog.Warningf("%s FileHandle Read %d: [%d,%d) size %d totalRead %d", fh.f.fullpath(), fh.handle, req.Offset, req.Offset+int64(req.Size), req.Size, totalRead)
totalRead = min(int64(len(buff)), totalRead) totalRead = min(int64(len(buff)), totalRead)
} }
resp.Data = buff[:totalRead]
// resp.Data = buff[:totalRead]
resp.Data = buff
return err return err
} }
func (fh *FileHandle) readFromDirtyPages(buff []byte, startOffset int64) (maxStop int64) { func (fh *FileHandle) readFromDirtyPages(buff []byte, startOffset int64) (maxStop int64) {
return fh.dirtyPages.ReadDirtyDataAt(buff, startOffset)
maxStop = fh.dirtyPages.ReadDirtyDataAt(buff, startOffset)
return
} }
func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) { func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
@ -127,6 +138,9 @@ func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
// Write to the file handle // Write to the file handle
func (fh *FileHandle) Write(ctx context.Context, req *fuse.WriteRequest, resp *fuse.WriteResponse) error { func (fh *FileHandle) Write(ctx context.Context, req *fuse.WriteRequest, resp *fuse.WriteResponse) error {
fh.Lock()
defer fh.Unlock()
// write the request to volume servers // write the request to volume servers
data := make([]byte, len(req.Data)) data := make([]byte, len(req.Data))
copy(data, req.Data) copy(data, req.Data)
@ -162,19 +176,30 @@ func (fh *FileHandle) Release(ctx context.Context, req *fuse.ReleaseRequest) err
glog.V(4).Infof("Release %v fh %d", fh.f.fullpath(), fh.handle) glog.V(4).Infof("Release %v fh %d", fh.f.fullpath(), fh.handle)
fh.Lock()
defer fh.Unlock()
fh.f.isOpen-- fh.f.isOpen--
if fh.f.isOpen <= 0 {
if fh.f.isOpen < 0 {
glog.V(0).Infof("Release reset %s open count %d => %d", fh.f.Name, fh.f.isOpen, 0)
fh.f.isOpen = 0
return nil
}
if fh.f.isOpen == 0 {
fh.doFlush(ctx, req.Header) fh.doFlush(ctx, req.Header)
fh.f.wfs.ReleaseHandle(fh.f.fullpath(), fuse.HandleID(fh.handle)) fh.f.wfs.ReleaseHandle(fh.f.fullpath(), fuse.HandleID(fh.handle))
fh.f.entryViewCache = nil
fh.f.reader = nil
} }
return nil return nil
} }
func (fh *FileHandle) Flush(ctx context.Context, req *fuse.FlushRequest) error { func (fh *FileHandle) Flush(ctx context.Context, req *fuse.FlushRequest) error {
fh.Lock()
defer fh.Unlock()
return fh.doFlush(ctx, req.Header) return fh.doFlush(ctx, req.Header)
} }
@ -183,13 +208,14 @@ func (fh *FileHandle) doFlush(ctx context.Context, header fuse.Header) error {
// send the data to the OS // send the data to the OS
glog.V(4).Infof("doFlush %s fh %d %v", fh.f.fullpath(), fh.handle, header) glog.V(4).Infof("doFlush %s fh %d %v", fh.f.fullpath(), fh.handle, header)
chunks, err := fh.dirtyPages.FlushToStorage()
chunks, err := fh.dirtyPages.saveExistingPagesToStorage()
if err != nil { if err != nil {
glog.Errorf("flush %s: %v", fh.f.fullpath(), err) glog.Errorf("flush %s: %v", fh.f.fullpath(), err)
return fuse.EIO return fuse.EIO
} }
if len(chunks) > 0 { if len(chunks) > 0 {
fh.f.addChunks(chunks) fh.f.addChunks(chunks)
fh.f.dirtyMetadata = true fh.f.dirtyMetadata = true
} }
@ -227,18 +253,14 @@ func (fh *FileHandle) doFlush(ctx context.Context, header fuse.Header) error {
glog.V(4).Infof("%s chunks %d: %v [%d,%d)", fh.f.fullpath(), i, chunk.GetFileIdString(), chunk.Offset, chunk.Offset+int64(chunk.Size)) glog.V(4).Infof("%s chunks %d: %v [%d,%d)", fh.f.fullpath(), i, chunk.GetFileIdString(), chunk.Offset, chunk.Offset+int64(chunk.Size))
} }
chunks, garbages := filer2.CompactFileChunks(filer2.LookupFn(fh.f.wfs), fh.f.entry.Chunks)
chunks, _ := filer2.CompactFileChunks(filer2.LookupFn(fh.f.wfs), fh.f.entry.Chunks)
chunks, manifestErr := filer2.MaybeManifestize(fh.f.wfs.saveDataAsChunk(fh.f.dir.FullPath()), chunks) chunks, manifestErr := filer2.MaybeManifestize(fh.f.wfs.saveDataAsChunk(fh.f.dir.FullPath()), chunks)
if manifestErr != nil { if manifestErr != nil {
// not good, but should be ok // not good, but should be ok
glog.V(0).Infof("MaybeManifestize: %v", manifestErr) glog.V(0).Infof("MaybeManifestize: %v", manifestErr)
} }
fh.f.entry.Chunks = chunks fh.f.entry.Chunks = chunks
// fh.f.entryViewCache = nil
// special handling of one chunk md5
if len(chunks) == 1 {
}
fh.f.entryViewCache = nil
if err := filer_pb.CreateEntry(client, request); err != nil { if err := filer_pb.CreateEntry(client, request); err != nil {
glog.Errorf("fh flush create %s: %v", fh.f.fullpath(), err) glog.Errorf("fh flush create %s: %v", fh.f.fullpath(), err)
@ -247,11 +269,6 @@ func (fh *FileHandle) doFlush(ctx context.Context, header fuse.Header) error {
fh.f.wfs.metaCache.InsertEntry(context.Background(), filer2.FromPbEntry(request.Directory, request.Entry)) fh.f.wfs.metaCache.InsertEntry(context.Background(), filer2.FromPbEntry(request.Directory, request.Entry))
fh.f.wfs.deleteFileChunks(garbages)
for i, chunk := range garbages {
glog.V(4).Infof("garbage %s chunks %d: %v [%d,%d)", fh.f.fullpath(), i, chunk.GetFileIdString(), chunk.Offset, chunk.Offset+int64(chunk.Size))
}
return nil return nil
}) })

4
weed/filesys/meta_cache/meta_cache.go

@ -50,6 +50,10 @@ func openMetaStore(dbFolder string) filer2.FilerStore {
func (mc *MetaCache) InsertEntry(ctx context.Context, entry *filer2.Entry) error { func (mc *MetaCache) InsertEntry(ctx context.Context, entry *filer2.Entry) error {
mc.Lock() mc.Lock()
defer mc.Unlock() defer mc.Unlock()
return mc.doInsertEntry(ctx, entry)
}
func (mc *MetaCache) doInsertEntry(ctx context.Context, entry *filer2.Entry) error {
filer_pb.BeforeEntrySerialization(entry.Chunks) filer_pb.BeforeEntrySerialization(entry.Chunks)
return mc.actualStore.InsertEntry(ctx, entry) return mc.actualStore.InsertEntry(ctx, entry)
} }

2
weed/filesys/meta_cache/meta_cache_init.go

@ -18,7 +18,7 @@ func EnsureVisited(mc *MetaCache, client filer_pb.FilerClient, dirPath util.Full
err = filer_pb.ReadDirAllEntries(client, dirPath, "", func(pbEntry *filer_pb.Entry, isLast bool) error { err = filer_pb.ReadDirAllEntries(client, dirPath, "", func(pbEntry *filer_pb.Entry, isLast bool) error {
entry := filer2.FromPbEntry(string(dirPath), pbEntry) entry := filer2.FromPbEntry(string(dirPath), pbEntry)
if err := mc.InsertEntry(context.Background(), entry); err != nil {
if err := mc.doInsertEntry(context.Background(), entry); err != nil {
glog.V(0).Infof("read %s: %v", entry.FullPath, err) glog.V(0).Infof("read %s: %v", entry.FullPath, err)
return err return err
} }

4
weed/filesys/wfs.go

@ -118,10 +118,14 @@ func (wfs *WFS) AcquireHandle(file *File, uid, gid uint32) (fileHandle *FileHand
inodeId := file.fullpath().AsInode() inodeId := file.fullpath().AsInode()
existingHandle, found := wfs.handles[inodeId] existingHandle, found := wfs.handles[inodeId]
if found && existingHandle != nil { if found && existingHandle != nil {
file.isOpen++
return existingHandle return existingHandle
} }
fileHandle = newFileHandle(file, uid, gid) fileHandle = newFileHandle(file, uid, gid)
file.maybeLoadEntry(context.Background())
file.isOpen++
wfs.handles[inodeId] = fileHandle wfs.handles[inodeId] = fileHandle
fileHandle.handle = inodeId fileHandle.handle = inodeId

2
weed/filesys/xattr.go

@ -119,5 +119,5 @@ func (wfs *WFS) maybeLoadEntry(dir, name string) (entry *filer_pb.Entry, err err
if cacheErr == filer_pb.ErrNotFound { if cacheErr == filer_pb.ErrNotFound {
return nil, fuse.ENOENT return nil, fuse.ENOENT
} }
return cachedEntry.ToProtoEntry(), nil
return cachedEntry.ToProtoEntry(), cacheErr
} }

5
weed/util/bounded_tree/bounded_tree.go

@ -15,7 +15,7 @@ type Node struct {
type BoundedTree struct { type BoundedTree struct {
root *Node root *Node
sync.Mutex
sync.RWMutex
} }
func NewBoundedTree() *BoundedTree { func NewBoundedTree() *BoundedTree {
@ -131,6 +131,9 @@ func (n *Node) getChild(childName string) *Node {
func (t *BoundedTree) HasVisited(p util.FullPath) bool { func (t *BoundedTree) HasVisited(p util.FullPath) bool {
t.RLock()
defer t.RUnlock()
if t.root == nil { if t.root == nil {
return true return true
} }

Loading…
Cancel
Save