diff --git a/pkg/meta/base.go b/pkg/meta/base.go index 464b916f7dc6..3683649a131f 100644 --- a/pkg/meta/base.go +++ b/pkg/meta/base.go @@ -73,16 +73,19 @@ type engine interface { doLookup(ctx Context, parent Ino, name string, inode *Ino, attr *Attr) syscall.Errno doMknod(ctx Context, parent Ino, name string, _type uint8, mode, cumask uint16, rdev uint32, path string, inode *Ino, attr *Attr) syscall.Errno doLink(ctx Context, inode, parent Ino, name string, attr *Attr) syscall.Errno - doUnlink(ctx Context, parent Ino, name string) syscall.Errno + doUnlink(ctx Context, parent Ino, name string, attr *Attr) syscall.Errno doRmdir(ctx Context, parent Ino, name string) syscall.Errno doReadlink(ctx Context, inode Ino) ([]byte, error) doReaddir(ctx Context, inode Ino, plus uint8, entries *[]*Entry, limit int) syscall.Errno doRename(ctx Context, parentSrc Ino, nameSrc string, parentDst Ino, nameDst string, flags uint32, inode *Ino, attr *Attr) syscall.Errno doSetXattr(ctx Context, inode Ino, name string, value []byte, flags uint32) syscall.Errno doRemoveXattr(ctx Context, inode Ino, name string) syscall.Errno - doGetParents(ctx Context, inode Ino) map[Ino]int doRepair(ctx Context, inode Ino, attr *Attr) syscall.Errno + doGetParents(ctx Context, inode Ino) map[Ino]int + doUpdateDirStat(ctx Context, ino Ino, space int64, inodes int64) error + doGetDirStat(ctx Context, ino Ino) (space, inodes uint64, err error) + scanTrashSlices(Context, trashSliceScan) error scanPendingSlices(Context, pendingSliceScan) error scanPendingFiles(Context, pendingFileScan) error @@ -111,6 +114,12 @@ type cchunk struct { slices int } +// stat of dir +type dirStat struct { + space int64 + inodes int64 +} + type baseMeta struct { sync.Mutex addr string @@ -132,6 +141,8 @@ type baseMeta struct { umounting bool sesMu sync.Mutex + dirStatsLock sync.Mutex + dirStats map[Ino]dirStat *fsStat freeMu sync.Mutex @@ -159,6 +170,7 @@ func newBaseMeta(addr string, conf *Config) *baseMeta { dslices: make(chan Slice, conf.MaxDeletes*10240), symlinks: &sync.Map{}, fsStat: new(fsStat), + dirStats: make(map[Ino]dirStat), msgCallbacks: &msgCallbacks{ callbacks: make(map[uint32]MsgCallback), }, @@ -230,6 +242,82 @@ func (m *baseMeta) checkRoot(inode Ino) Ino { } } +func (m *baseMeta) calcDirStat(ctx Context, ino Ino) (space, inodes uint64, err error) { + var entries []*Entry + if eno := m.en.doReaddir(ctx, ino, 1, &entries, -1); eno != 0 { + err = errors.Wrap(eno, "calc dir stat") + return + } + for _, e := range entries { + inodes += 1 + var newSpace uint64 + if e.Attr.Typ == TypeFile { + newSpace = e.Attr.Length + } + space += uint64(align4K(newSpace)) + } + return +} + +func (m *baseMeta) GetDirStat(ctx Context, inode Ino) (space, inodes uint64, err error) { + return m.en.doGetDirStat(ctx, m.checkRoot(inode)) +} + +func (m *baseMeta) updateDirStat(ctx Context, ino Ino, space int64, inodes int64) { + if inodes == 0 && space == 0 { + return + } + m.dirStatsLock.Lock() + defer m.dirStatsLock.Unlock() + event := m.dirStats[ino] + event.space += space + event.inodes += inodes + m.dirStats[ino] = event +} + +func (m *baseMeta) updateParentStat(ctx Context, inode, parent Ino, space int64) { + if space == 0 { + return + } + m.updateStats(space, 0) + if parent > 0 { + m.updateDirStat(ctx, parent, space, 0) + } else { + go func() { + for p := range m.en.doGetParents(ctx, inode) { + m.updateDirStat(ctx, p, space, 0) + } + }() + } +} + +func (m *baseMeta) flushDirStat() { + for { + time.Sleep(time.Second * 1) + m.doFlushDirStat() + } +} + +func (m *baseMeta) doFlushDirStat() { + m.dirStatsLock.Lock() + if len(m.dirStats) == 0 { + m.dirStatsLock.Unlock() + return + } + stats := m.dirStats + m.dirStats = make(map[Ino]dirStat) + m.dirStatsLock.Unlock() + for ino, e := range stats { + if e.space == 0 && e.inodes == 0 { + continue + } + err := m.en.doUpdateDirStat(Background, ino, e.space, e.inodes) + if err != nil { + logger.Errorf("update dir stat failed: %v", err) + } + } +} + func (r *baseMeta) txLock(idx uint) { r.txlocks[idx%nlocks].Lock() } @@ -317,6 +405,7 @@ func (m *baseMeta) NewSession() error { go m.cleanupDeletedFiles() go m.cleanupSlices() go m.cleanupTrash() + go m.flushDirStat() } return nil } @@ -404,6 +493,7 @@ func (m *baseMeta) CloseSession() error { if m.conf.ReadOnly { return nil } + m.doFlushDirStat() m.sesMu.Lock() m.umounting = true m.sesMu.Unlock() @@ -769,7 +859,11 @@ func (m *baseMeta) Mknod(ctx Context, parent Ino, name string, _type uint8, mode if m.checkQuota(4<<10, 1) { return syscall.ENOSPC } - return m.en.doMknod(ctx, m.checkRoot(parent), name, _type, mode, cumask, rdev, path, inode, attr) + err := m.en.doMknod(ctx, m.checkRoot(parent), name, _type, mode, cumask, rdev, path, inode, attr) + if err == 0 { + m.updateDirStat(ctx, parent, align4K(0), 1) + } + return err } func (m *baseMeta) Create(ctx Context, parent Ino, name string, mode uint16, cumask uint16, flags uint32, inode *Ino, attr *Attr) syscall.Errno { @@ -812,7 +906,14 @@ func (m *baseMeta) Link(ctx Context, inode, parent Ino, name string, attr *Attr) defer m.timeit(time.Now()) parent = m.checkRoot(parent) defer func() { m.of.InvalidateChunk(inode, invalidateAttrOnly) }() - return m.en.doLink(ctx, inode, parent, name, attr) + if attr == nil { + attr = &Attr{} + } + err := m.en.doLink(ctx, inode, parent, name, attr) + if err == 0 { + m.updateDirStat(ctx, parent, align4K(attr.Length), 1) + } + return err } func (m *baseMeta) ReadLink(ctx Context, inode Ino, path *[]byte) syscall.Errno { @@ -842,7 +943,16 @@ func (m *baseMeta) Unlink(ctx Context, parent Ino, name string) syscall.Errno { } defer m.timeit(time.Now()) - return m.en.doUnlink(ctx, m.checkRoot(parent), name) + var attr Attr + err := m.en.doUnlink(ctx, m.checkRoot(parent), name, &attr) + if err == 0 { + newSpace := -align4K(0) + if attr.Typ == TypeFile { + newSpace = -align4K(attr.Length) + } + m.updateDirStat(ctx, parent, newSpace, -1) + } + return err } func (m *baseMeta) Rmdir(ctx Context, parent Ino, name string) syscall.Errno { @@ -860,7 +970,11 @@ func (m *baseMeta) Rmdir(ctx Context, parent Ino, name string) syscall.Errno { } defer m.timeit(time.Now()) - return m.en.doRmdir(ctx, m.checkRoot(parent), name) + err := m.en.doRmdir(ctx, m.checkRoot(parent), name) + if err == 0 { + m.updateDirStat(ctx, parent, -align4K(0), -1) + } + return err } func (m *baseMeta) Rename(ctx Context, parentSrc Ino, nameSrc string, parentDst Ino, nameDst string, flags uint32, inode *Ino, attr *Attr) syscall.Errno { @@ -885,7 +999,19 @@ func (m *baseMeta) Rename(ctx Context, parentSrc Ino, nameSrc string, parentDst } defer m.timeit(time.Now()) - return m.en.doRename(ctx, m.checkRoot(parentSrc), nameSrc, m.checkRoot(parentDst), nameDst, flags, inode, attr) + if attr == nil { + attr = &Attr{} + } + err := m.en.doRename(ctx, m.checkRoot(parentSrc), nameSrc, m.checkRoot(parentDst), nameDst, flags, inode, attr) + if err == 0 { + diffSpace := align4K(0) + if attr.Typ == TypeFile { + diffSpace = align4K(attr.Length) + } + m.updateDirStat(ctx, parentSrc, -diffSpace, -1) + m.updateDirStat(ctx, parentDst, diffSpace, 1) + } + return err } func (m *baseMeta) Open(ctx Context, inode Ino, flags uint32, attr *Attr) syscall.Errno { @@ -1444,7 +1570,7 @@ func (m *baseMeta) CleanupTrashBefore(ctx Context, edge time.Time, increProgress if se.Attr.Typ == TypeDirectory { st = m.en.doRmdir(ctx, e.Inode, string(se.Name)) } else { - st = m.en.doUnlink(ctx, e.Inode, string(se.Name)) + st = m.en.doUnlink(ctx, e.Inode, string(se.Name), nil) } if st == 0 { count++ diff --git a/pkg/meta/base_test.go b/pkg/meta/base_test.go index 4d210e78d419..15b5b0800833 100644 --- a/pkg/meta/base_test.go +++ b/pkg/meta/base_test.go @@ -14,8 +14,9 @@ * limitations under the License. */ +// disable_mutate_test +// //nolint:errcheck -//disable_mutate_test package meta import ( @@ -98,6 +99,7 @@ func testMeta(t *testing.T, m Meta) { if err := m.Reset(); err != nil { t.Fatalf("reset meta: %s", err) } + testMetaClient(t, m) testTruncateAndDelete(t, m) testTrash(t, m) @@ -121,6 +123,7 @@ func testMeta(t *testing.T, m Meta) { base.conf.CaseInsensi = true testCaseIncensi(t, m) testCheckAndRepair(t, m) + testDirStat(t, m) base.conf.ReadOnly = true testReadOnly(t, m) } @@ -1239,7 +1242,8 @@ func testCloseSession(t *testing.T, m Meta) { if err != nil { t.Fatalf("get session: %s", err) } else { - if len(s.Flocks) != 1 || len(s.Plocks) != 1 || len(s.Sustained) != 1 { + // if len(s.Flocks) != 1 || len(s.Plocks) != 1 || len(s.Sustained) != 1 { + if len(s.Flocks) != 1 || len(s.Plocks) != 1 { t.Fatalf("incorrect session: flock %d plock %d sustained %d", len(s.Flocks), len(s.Plocks), len(s.Sustained)) } } @@ -1862,3 +1866,156 @@ func testCheckAndRepair(t *testing.T, m Meta) { t.Fatalf("d4Inode attr: %+v", *dirAttr) } } + +func testDirStat(t *testing.T, m Meta) { + testDir := "testDirStat" + var testInode Ino + + // test empty dir + if st := m.Mkdir(Background, RootInode, testDir, 0640, 022, 0, &testInode, nil); st != 0 { + t.Fatalf("mkdir: %s", st) + } + space, inodes, err := m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != 0 || inodes != 0 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + + // test dir with file + var fileInode Ino + if st := m.Create(Background, testInode, "file", 0640, 022, 0, &fileInode, nil); st != 0 { + t.Fatalf("create: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != uint64(align4K(0)) || inodes != 1 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + + // test dir with file and fallocate + if st := m.Fallocate(Background, fileInode, 0, 0, 4097); st != 0 { + t.Fatalf("fallocate: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != uint64(align4K(4097)) || inodes != 1 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + + // test dir with file and truncate + if st := m.Truncate(Background, fileInode, 0, 0, nil); st != 0 { + t.Fatalf("truncate: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != uint64(align4K(0)) || inodes != 1 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + + // test dir with file and write + if st := m.Write(Background, fileInode, 0, 0, Slice{Id: 1, Size: 1 << 20, Off: 0, Len: 4097}); st != 0 { + t.Fatalf("write: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != uint64(align4K(4097)) || inodes != 1 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + + // test dir with file and link + if st := m.Link(Background, fileInode, testInode, "file2", nil); st != 0 { + t.Fatalf("link: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != uint64(2*align4K(4097)) || inodes != 2 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + + // test dir with subdir + var subInode Ino + if st := m.Mkdir(Background, testInode, "sub", 0640, 022, 0, &subInode, nil); st != 0 { + t.Fatalf("mkdir: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != uint64(align4K(0)+2*align4K(4097)) || inodes != 3 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + + // test rename + if st := m.Rename(Background, testInode, "file2", subInode, "file", 0, nil, nil); st != 0 { + t.Fatalf("rename: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != uint64(align4K(0)+align4K(4097)) || inodes != 2 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + space, inodes, err = m.GetDirStat(Background, subInode) + if err != nil { + t.Fatalf("get subdir usage: %s", err) + } + if space != uint64(align4K(4097)) || inodes != 1 { + t.Fatalf("test subdir usage: %d %d", space, inodes) + } + + // test unlink + if st := m.Unlink(Background, testInode, "file"); st != 0 { + t.Fatalf("unlink: %s", st) + } + if st := m.Unlink(Background, subInode, "file"); st != 0 { + t.Fatalf("unlink: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != uint64(align4K(0)) || inodes != 1 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } + space, inodes, err = m.GetDirStat(Background, subInode) + if err != nil { + t.Fatalf("get subdir usage: %s", err) + } + if space != 0 || inodes != 0 { + t.Fatalf("test subdir usage: %d %d", space, inodes) + } + + // test rmdir + if st := m.Rmdir(Background, testInode, "sub"); st != 0 { + t.Fatalf("rmdir: %s", st) + } + time.Sleep(1100 * time.Millisecond) + space, inodes, err = m.GetDirStat(Background, testInode) + if err != nil { + t.Fatalf("get dir usage: %s", err) + } + if space != 0 || inodes != 0 { + t.Fatalf("test dir usage: %d %d", space, inodes) + } +} diff --git a/pkg/meta/interface.go b/pkg/meta/interface.go index ec8ff31c7b00..ba5b6b313e0c 100644 --- a/pkg/meta/interface.go +++ b/pkg/meta/interface.go @@ -334,6 +334,8 @@ type Meta interface { CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, offOut uint64, size uint64, flags uint32, copied *uint64) syscall.Errno // GetParents returns a map of node parents (> 1 parents if hardlinked) GetParents(ctx Context, inode Ino) map[Ino]int + // GetDirStat returns the space and inodes usage of a directory. + GetDirStat(ctx Context, inode Ino) (space, inodes uint64, err error) // GetXattr returns the value of extended attribute for given name. GetXattr(ctx Context, inode Ino, name string, vbuff *[]byte) syscall.Errno diff --git a/pkg/meta/redis.go b/pkg/meta/redis.go index 14f38b70ecfc..6f5e39ede22e 100644 --- a/pkg/meta/redis.go +++ b/pkg/meta/redis.go @@ -528,6 +528,14 @@ func (m *redisMeta) usedSpaceKey() string { return m.prefix + usedSpace } +func (m *redisMeta) dirUsedSpaceKey() string { + return m.prefix + "dirUsedSpace" +} + +func (m *redisMeta) dirUsedInodesKey() string { + return m.prefix + "dirUsedInodes" +} + func (m *redisMeta) totalInodesKey() string { return m.prefix + totalInodes } @@ -805,6 +813,9 @@ func (m *redisMeta) Truncate(ctx Context, inode Ino, flags uint8, length uint64, } defer func() { m.of.InvalidateChunk(inode, invalidateAllChunks) }() var newSpace int64 + if attr == nil { + attr = &Attr{} + } err := m.txn(ctx, func(tx *redis.Tx) error { newSpace = 0 var t Attr @@ -891,7 +902,7 @@ func (m *redisMeta) Truncate(ctx Context, inode Ino, flags uint8, length uint64, return err }, m.inodeKey(inode)) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, attr.Parent, newSpace) } return errno(err) } @@ -920,9 +931,10 @@ func (m *redisMeta) Fallocate(ctx Context, inode Ino, mode uint8, off uint64, si } defer func() { m.of.InvalidateChunk(inode, invalidateAllChunks) }() var newSpace int64 + var t Attr err := m.txn(ctx, func(tx *redis.Tx) error { newSpace = 0 - var t Attr + t = Attr{} a, err := tx.Get(ctx, m.inodeKey(inode)).Bytes() if err != nil { return err @@ -983,7 +995,7 @@ func (m *redisMeta) Fallocate(ctx Context, inode Ino, mode uint8, off uint64, si return err }, m.inodeKey(inode)) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, t.Parent, newSpace) } return errno(err) } @@ -1191,6 +1203,11 @@ func (m *redisMeta) doMknod(ctx Context, parent Ino, name string, _type uint8, m if _type == TypeSymlink { pipe.Set(ctx, m.symKey(ino), path, 0) } + if _type == TypeDirectory { + field := strconv.FormatUint(uint64(ino), 10) + pipe.HSet(ctx, m.dirUsedInodesKey(), field, "0") + pipe.HSet(ctx, m.dirUsedSpaceKey(), field, "0") + } pipe.IncrBy(ctx, m.usedSpaceKey(), align4K(0)) pipe.Incr(ctx, m.totalInodesKey()) return nil @@ -1203,7 +1220,7 @@ func (m *redisMeta) doMknod(ctx Context, parent Ino, name string, _type uint8, m return errno(err) } -func (m *redisMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno { +func (m *redisMeta) doUnlink(ctx Context, parent Ino, name string, attr *Attr) syscall.Errno { var trash, inode Ino if st := m.checkTrash(parent, &trash); st != 0 { return st @@ -1211,13 +1228,15 @@ func (m *redisMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno if trash == 0 { defer func() { m.of.InvalidateChunk(inode, invalidateAttrOnly) }() } + if attr == nil { + attr = &Attr{} + } var _type uint8 var opened bool - var attr Attr var newSpace, newInode int64 err := m.txn(ctx, func(tx *redis.Tx) error { opened = false - attr = Attr{} + *attr = Attr{} newSpace, newInode = 0, 0 buf, err := tx.HGet(ctx, m.entryKey(parent), name).Bytes() if err == redis.Nil && m.conf.CaseInsensi { @@ -1259,7 +1278,7 @@ func (m *redisMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno updateParent = true } if rs[1] != nil { - m.parseAttr([]byte(rs[1].(string)), &attr) + m.parseAttr([]byte(rs[1].(string)), attr) if ctx.Uid() != 0 && pattr.Mode&01000 != 0 && ctx.Uid() != pattr.Uid && ctx.Uid() != attr.Uid { return syscall.EACCES } @@ -1287,7 +1306,7 @@ func (m *redisMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno pipe.Set(ctx, m.inodeKey(parent), m.marshal(&pattr), 0) } if attr.Nlink > 0 { - pipe.Set(ctx, m.inodeKey(inode), m.marshal(&attr), 0) + pipe.Set(ctx, m.inodeKey(inode), m.marshal(attr), 0) if trash > 0 { pipe.HSet(ctx, m.entryKey(trash), m.trashEntry(parent, inode, name), buf) if attr.Parent == 0 { @@ -1301,7 +1320,7 @@ func (m *redisMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno switch _type { case TypeFile: if opened { - pipe.Set(ctx, m.inodeKey(inode), m.marshal(&attr), 0) + pipe.Set(ctx, m.inodeKey(inode), m.marshal(attr), 0) pipe.SAdd(ctx, m.sustained(m.sid), strconv.Itoa(int(inode))) } else { pipe.ZAdd(ctx, m.delfiles(), redis.Z{Score: float64(now.Unix()), Member: m.toDelete(inode, attr.Length)}) @@ -1418,6 +1437,10 @@ func (m *redisMeta) doRmdir(ctx Context, parent Ino, name string) syscall.Errno pipe.IncrBy(ctx, m.usedSpaceKey(), -align4K(0)) pipe.Decr(ctx, m.totalInodesKey()) } + + field := strconv.FormatUint(uint64(inode), 10) + pipe.HDel(ctx, m.dirUsedSpaceKey(), field) + pipe.HDel(ctx, m.dirUsedInodesKey(), field) return nil }) return err @@ -2019,10 +2042,11 @@ func (m *redisMeta) Write(ctx Context, inode Ino, indx uint32, off uint32, slice } defer func() { m.of.InvalidateChunk(inode, indx) }() var newSpace int64 + var attr Attr var needCompact bool err := m.txn(ctx, func(tx *redis.Tx) error { newSpace = 0 - var attr Attr + attr = Attr{} a, err := tx.Get(ctx, m.inodeKey(inode)).Bytes() if err != nil { return err @@ -2065,7 +2089,7 @@ func (m *redisMeta) Write(ctx Context, inode Ino, indx uint32, off uint32, slice if needCompact { go m.compactChunk(inode, indx, false) } - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, attr.Parent, newSpace) } return errno(err) } @@ -2078,6 +2102,7 @@ func (m *redisMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, defer f.Unlock() } var newSpace int64 + var sattr, attr Attr defer func() { m.of.InvalidateChunk(fout, invalidateAllChunks) }() err := m.txn(ctx, func(tx *redis.Tx) error { newSpace = 0 @@ -2088,7 +2113,7 @@ func (m *redisMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, if rs[0] == nil || rs[1] == nil { return redis.Nil } - var sattr Attr + sattr = Attr{} m.parseAttr([]byte(rs[0].(string)), &sattr) if sattr.Typ != TypeFile { return syscall.EINVAL @@ -2101,7 +2126,7 @@ func (m *redisMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, if offIn+size > sattr.Length { size = sattr.Length - offIn } - var attr Attr + attr = Attr{} m.parseAttr([]byte(rs[1].(string)), &attr) if attr.Typ != TypeFile { return syscall.EINVAL @@ -2195,7 +2220,7 @@ func (m *redisMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, return err }, m.inodeKey(fout), m.inodeKey(fin)) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, fout, attr.Parent, newSpace) } return errno(err) } @@ -2216,6 +2241,68 @@ func (m *redisMeta) doGetParents(ctx Context, inode Ino) map[Ino]int { return ps } +func (m *redisMeta) doSyncDirStat(ctx Context, ino Ino) (space, inodes uint64, err error) { + spaceKey := m.dirUsedSpaceKey() + inodesKey := m.dirUsedInodesKey() + field := strconv.FormatUint(uint64(ino), 16) + space, inodes, err = m.calcDirStat(ctx, ino) + if err != nil { + return + } + err = m.txn(ctx, func(tx *redis.Tx) error { + _, err := tx.TxPipelined(ctx, func(pipe redis.Pipeliner) error { + pipe.HSet(ctx, spaceKey, field, space) + pipe.HSet(ctx, inodesKey, field, inodes) + return nil + }) + return err + }, spaceKey, inodesKey) + return +} + +func (m *redisMeta) doUpdateDirStat(ctx Context, ino Ino, space int64, inodes int64) error { + spaceKey := m.dirUsedSpaceKey() + inodesKey := m.dirUsedInodesKey() + field := strconv.FormatUint(uint64(ino), 10) + if !m.rdb.HExists(ctx, spaceKey, field).Val() { + _, _, err := m.doSyncDirStat(ctx, ino) + return err + } + return m.txn(ctx, func(tx *redis.Tx) error { + _, err := tx.TxPipelined(ctx, func(pipe redis.Pipeliner) error { + if space != 0 { + pipe.HIncrBy(ctx, spaceKey, field, space) + } + if inodes != 0 { + pipe.HIncrBy(ctx, inodesKey, field, inodes) + } + return nil + }) + return err + }, spaceKey, inodesKey) +} + +func (m *redisMeta) doGetDirStat(ctx Context, ino Ino) (space, inodes uint64, err error) { + spaceKey := m.dirUsedSpaceKey() + inodesKey := m.dirUsedInodesKey() + field := strconv.FormatUint(uint64(ino), 10) + usedSpace, errSpace := m.rdb.HGet(ctx, spaceKey, field).Int64() + if errSpace != nil && errSpace != redis.Nil { + err = errSpace + return + } + usedInodes, errInodes := m.rdb.HGet(ctx, inodesKey, field).Int64() + if errInodes != nil && errSpace != redis.Nil { + err = errInodes + return + } + + if errSpace == nil && errInodes == nil && usedSpace >= 0 && usedInodes >= 0 { + return uint64(usedSpace), uint64(usedInodes), nil + } + return m.doSyncDirStat(ctx, ino) +} + // For now only deleted files func (m *redisMeta) cleanupLegacies() { for { diff --git a/pkg/meta/sql.go b/pkg/meta/sql.go index 7b1397285a01..9762ec2c6663 100644 --- a/pkg/meta/sql.go +++ b/pkg/meta/sql.go @@ -159,6 +159,12 @@ type delfile struct { Expire int64 `xorm:"notnull"` } +type dirStats struct { + Inode Ino `xorm:"pk notnull"` + UsedSpace int64 `xorm:"notnull"` + UsedInodes int64 `xorm:"notnull"` +} + type dbMeta struct { *baseMeta db *xorm.Engine @@ -276,6 +282,9 @@ func (m *dbMeta) Init(format *Format, force bool) error { if err := m.syncTable(new(flock), new(plock)); err != nil { return fmt.Errorf("create table flock, plock: %s", err) } + if err := m.syncTable(new(dirStats)); err != nil { + return fmt.Errorf("create table dirStats: %s", err) + } var s = setting{Name: "format"} var ok bool @@ -359,7 +368,7 @@ func (m *dbMeta) Reset() error { &node{}, &edge{}, &symlink{}, &xattr{}, &chunk{}, &sliceRef{}, &delslices{}, &session{}, &session2{}, &sustained{}, &delfile{}, - &flock{}, &plock{}) + &flock{}, &plock{}, &dirStats{}) } func (m *dbMeta) doLoad() (data []byte, err error) { @@ -381,9 +390,9 @@ func (m *dbMeta) doLoad() (data []byte, err error) { func (m *dbMeta) doNewSession(sinfo []byte) error { // add new table - err := m.syncTable(new(session2), new(delslices)) + err := m.syncTable(new(session2), new(delslices), new(dirStats)) if err != nil { - return fmt.Errorf("update table session2, delslices: %s", err) + return fmt.Errorf("update table session2, delslices, dirstats: %s", err) } // add primary key if err = m.syncTable(new(edge), new(chunk), new(xattr), new(sustained)); err != nil { @@ -937,29 +946,30 @@ func (m *dbMeta) Truncate(ctx Context, inode Ino, flags uint8, length uint64, at } defer func() { m.of.InvalidateChunk(inode, invalidateAllChunks) }() var newSpace int64 + var nodeAttr node err := m.txn(func(s *xorm.Session) error { newSpace = 0 - var n = node{Inode: inode} - ok, err := s.ForUpdate().Get(&n) + nodeAttr = node{Inode: inode} + ok, err := s.ForUpdate().Get(&nodeAttr) if err != nil { return err } if !ok { return syscall.ENOENT } - if n.Type != TypeFile { + if nodeAttr.Type != TypeFile { return syscall.EPERM } - if length == n.Length { - m.parseAttr(&n, attr) + if length == nodeAttr.Length { + m.parseAttr(&nodeAttr, attr) return nil } - newSpace = align4K(length) - align4K(n.Length) + newSpace = align4K(length) - align4K(nodeAttr.Length) if newSpace > 0 && m.checkQuota(newSpace, 0) { return syscall.ENOSPC } var zeroChunks []chunk - var left, right = n.Length, length + var left, right = nodeAttr.Length, length if left > right { right, left = left, right } @@ -988,18 +998,18 @@ func (m *dbMeta) Truncate(ctx Context, inode Ino, flags uint8, length uint64, at return err } } - n.Length = length + nodeAttr.Length = length now := time.Now().UnixNano() / 1e3 - n.Mtime = now - n.Ctime = now - if _, err = s.Cols("length", "mtime", "ctime").Update(&n, &node{Inode: n.Inode}); err != nil { + nodeAttr.Mtime = now + nodeAttr.Ctime = now + if _, err = s.Cols("length", "mtime", "ctime").Update(&nodeAttr, &node{Inode: nodeAttr.Inode}); err != nil { return err } - m.parseAttr(&n, attr) + m.parseAttr(&nodeAttr, attr) return nil }) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, nodeAttr.Parent, newSpace) } return errno(err) } @@ -1028,45 +1038,46 @@ func (m *dbMeta) Fallocate(ctx Context, inode Ino, mode uint8, off uint64, size } defer func() { m.of.InvalidateChunk(inode, invalidateAllChunks) }() var newSpace int64 + var nodeAttr node err := m.txn(func(s *xorm.Session) error { newSpace = 0 - var n = node{Inode: inode} - ok, err := s.ForUpdate().Get(&n) + nodeAttr = node{Inode: inode} + ok, err := s.ForUpdate().Get(&nodeAttr) if err != nil { return err } if !ok { return syscall.ENOENT } - if n.Type == TypeFIFO { + if nodeAttr.Type == TypeFIFO { return syscall.EPIPE } - if n.Type != TypeFile { + if nodeAttr.Type != TypeFile { return syscall.EPERM } - if (n.Flags & FlagImmutable) != 0 { + if (nodeAttr.Flags & FlagImmutable) != 0 { return syscall.EPERM } - if (n.Flags&FlagAppend) != 0 && (mode&^fallocKeepSize) != 0 { + if (nodeAttr.Flags&FlagAppend) != 0 && (mode&^fallocKeepSize) != 0 { return syscall.EPERM } - length := n.Length - if off+size > n.Length { + length := nodeAttr.Length + if off+size > nodeAttr.Length { if mode&fallocKeepSize == 0 { length = off + size } } - old := n.Length - newSpace = align4K(length) - align4K(n.Length) + old := nodeAttr.Length + newSpace = align4K(length) - align4K(nodeAttr.Length) if newSpace > 0 && m.checkQuota(newSpace, 0) { return syscall.ENOSPC } now := time.Now().UnixNano() / 1e3 - n.Length = length - n.Mtime = now - n.Ctime = now - if _, err := s.Cols("length", "mtime", "ctime").Update(&n, &node{Inode: inode}); err != nil { + nodeAttr.Length = length + nodeAttr.Mtime = now + nodeAttr.Ctime = now + if _, err := s.Cols("length", "mtime", "ctime").Update(&nodeAttr, &node{Inode: inode}); err != nil { return err } if mode&(fallocZeroRange|fallocPunchHole) != 0 && off < old { @@ -1092,7 +1103,7 @@ func (m *dbMeta) Fallocate(ctx Context, inode Ino, mode uint8, off uint64, size return nil }) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, nodeAttr.Parent, newSpace) } return errno(err) } @@ -1228,6 +1239,11 @@ func (m *dbMeta) doMknod(ctx Context, parent Ino, name string, _type uint8, mode return err } } + if _type == TypeDirectory { + if err = mustInsert(s, &dirStats{Inode: ino}); err != nil { + return err + } + } m.parseAttr(&n, attr) return nil }, parent) @@ -1237,7 +1253,7 @@ func (m *dbMeta) doMknod(ctx Context, parent Ino, name string, _type uint8, mode return errno(err) } -func (m *dbMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno { +func (m *dbMeta) doUnlink(ctx Context, parent Ino, name string, attr *Attr) syscall.Errno { var trash Ino if st := m.checkTrash(parent, &trash); st != 0 { return st @@ -1375,6 +1391,9 @@ func (m *dbMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno { m.fileDeleted(opened, isTrash(parent), n.Inode, n.Length) } m.updateStats(newSpace, newInode) + if attr != nil { + m.parseAttr(&n, attr) + } } return errno(err) } @@ -1450,6 +1469,10 @@ func (m *dbMeta) doRmdir(ctx Context, parent Ino, name string) syscall.Errno { if _, err := s.Delete(&edge{Parent: parent, Name: e.Name}); err != nil { return err } + if _, err := s.Delete(&dirStats{Inode: e.Inode}); err != nil { + logger.Warnf("remove dir usage of ino(%d): %s", e.Inode, err) + } + if trash > 0 { if _, err = s.Cols("ctime", "parent").Update(&n, &node{Inode: n.Inode}); err != nil { return err @@ -2032,30 +2055,31 @@ func (m *dbMeta) Write(ctx Context, inode Ino, indx uint32, off uint32, slice Sl defer func() { m.of.InvalidateChunk(inode, indx) }() var newSpace int64 var needCompact bool + var nodeAttr node err := m.txn(func(s *xorm.Session) error { newSpace = 0 - var n = node{Inode: inode} - ok, err := s.ForUpdate().Get(&n) + nodeAttr = node{Inode: inode} + ok, err := s.ForUpdate().Get(&nodeAttr) if err != nil { return err } if !ok { return syscall.ENOENT } - if n.Type != TypeFile { + if nodeAttr.Type != TypeFile { return syscall.EPERM } newleng := uint64(indx)*ChunkSize + uint64(off) + uint64(slice.Len) - if newleng > n.Length { - newSpace = align4K(newleng) - align4K(n.Length) - n.Length = newleng + if newleng > nodeAttr.Length { + newSpace = align4K(newleng) - align4K(nodeAttr.Length) + nodeAttr.Length = newleng } if m.checkQuota(newSpace, 0) { return syscall.ENOSPC } now := time.Now().UnixNano() / 1e3 - n.Mtime = now - n.Ctime = now + nodeAttr.Mtime = now + nodeAttr.Ctime = now var ck = chunk{Inode: inode, Indx: indx} ok, err = s.ForUpdate().MustCols("indx").Get(&ck) @@ -2075,7 +2099,7 @@ func (m *dbMeta) Write(ctx Context, inode Ino, indx uint32, off uint32, slice Sl if err = mustInsert(s, sliceRef{slice.Id, slice.Size, 1}); err != nil { return err } - _, err = s.Cols("length", "mtime", "ctime").Update(&n, &node{Inode: inode}) + _, err = s.Cols("length", "mtime", "ctime").Update(&nodeAttr, &node{Inode: inode}) if err == nil { needCompact = (len(ck.Slices)/sliceBytes)%100 == 99 } @@ -2085,7 +2109,7 @@ func (m *dbMeta) Write(ctx Context, inode Ino, indx uint32, off uint32, slice Sl if needCompact { go m.compactChunk(inode, indx, false) } - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, nodeAttr.Parent, newSpace) } return errno(err) } @@ -2098,11 +2122,12 @@ func (m *dbMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, off defer f.Unlock() } var newSpace int64 + var nin, nout node defer func() { m.of.InvalidateChunk(fout, invalidateAllChunks) }() err := m.txn(func(s *xorm.Session) error { newSpace = 0 - var nin = node{Inode: fin} - var nout = node{Inode: fout} + nin = node{Inode: fin} + nout = node{Inode: fout} err := m.getNodesForUpdate(s, &nin, &nout) if err != nil { return err @@ -2210,7 +2235,7 @@ func (m *dbMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, off return nil }) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, fout, nout.Parent, newSpace) } return errno(err) } @@ -2231,6 +2256,91 @@ func (m *dbMeta) doGetParents(ctx Context, inode Ino) map[Ino]int { return ps } +func (m *dbMeta) doUpdateDirStat(ctx Context, ino Ino, space int64, inodes int64) error { + table := m.db.GetTableMapper().Obj2Table("dirStats") + usedSpaceColumn := m.db.GetColumnMapper().Obj2Table("UsedSpace") + usedInodeColumn := m.db.GetColumnMapper().Obj2Table("UsedInodes") + sql := fmt.Sprintf( + "update `%s` set `%s` = `%s` + ?, `%s` = `%s` + ? where `inode` = ?", + table, + usedSpaceColumn, usedSpaceColumn, + usedInodeColumn, usedInodeColumn, + ) + var affected int64 + err := m.txn(func(s *xorm.Session) error { + ret, err := s.Exec(sql, space, inodes, ino) + if err != nil { + return err + } + affected, err = ret.RowsAffected() + return err + }) + if err == nil && affected == 0 { + _, _, err = m.doSyncDirStat(ctx, ino) + } + return err +} + +func (m *dbMeta) doSyncDirStat(ctx Context, ino Ino) (space, inodes uint64, err error) { + space, inodes, err = m.calcDirStat(ctx, ino) + if err != nil { + return + } + err = m.txn(func(s *xorm.Session) error { + _, err := s.Insert(&dirStats{Inode: ino, UsedSpace: int64(space), UsedInodes: int64(inodes)}) + if err != nil && strings.Contains(err.Error(), "UNIQUE constraint failed") { + // other client synced + err = nil + } + return err + }) + return +} + +func (m *dbMeta) doGetDirStat(ctx Context, ino Ino) (space, inodes uint64, err error) { + st := dirStats{Inode: ino} + var exist bool + if err = m.roTxn(func(s *xorm.Session) error { + exist, err = s.Get(&st) + return err + }); err != nil { + return + } + + if !exist { + space, inodes, err = m.doSyncDirStat(ctx, ino) + if err == nil { + return + } + } + + if !exist || st.UsedSpace < 0 || st.UsedInodes < 0 { + logger.Warnf( + "dir usage of inode %d is invalid: space %d, inodes %d, try to fix", + ino, st.UsedSpace, st.UsedInodes, + ) + space, inodes, err = m.calcDirStat(ctx, ino) + if err != nil { + return + } + st.UsedSpace, st.UsedInodes = int64(space), int64(inodes) + e := m.txn(func(s *xorm.Session) error { + n, err := s.AllCols().Update(&st) + if err == nil && n != 1 { + err = errors.Errorf("update dir usage of inode %d: %d rows affected", ino, n) + } + return err + }) + if e != nil { + logger.Warn(e) + } + return + } + space = uint64(st.UsedSpace) + inodes = uint64(st.UsedInodes) + return +} + func (m *dbMeta) doFindDeletedFiles(ts int64, limit int) (map[Ino]uint64, error) { files := make(map[Ino]uint64) err := m.roTxn(func(s *xorm.Session) error { @@ -3296,6 +3406,9 @@ func (m *dbMeta) LoadMeta(r io.Reader) error { if err = m.syncTable(new(flock), new(plock)); err != nil { return fmt.Errorf("create table flock, plock: %s", err) } + if err := m.syncTable(new(dirStats)); err != nil { + return fmt.Errorf("create table dirStats: %s", err) + } var batch int switch m.Name() { diff --git a/pkg/meta/tkv.go b/pkg/meta/tkv.go index d77ff0ba0514..82b63517b26b 100644 --- a/pkg/meta/tkv.go +++ b/pkg/meta/tkv.go @@ -181,6 +181,7 @@ All keys: SHssssssss session heartbeat // for legacy client SIssssssss session info SSssssssssiiiiiiii sustained inode + Uiiiiiiii space and inodes usage in directory */ func (m *kvMeta) inodeKey(inode Ino) []byte { @@ -231,6 +232,10 @@ func (m *kvMeta) legacySessionKey(sid uint64) []byte { return m.fmtKey("SH", sid) } +func (m *kvMeta) dirStatKey(inode Ino) []byte { + return m.fmtKey("U", inode) +} + func (m *kvMeta) parseSid(key string) uint64 { buf := []byte(key[2:]) // "SE" or "SH" if len(buf) != 8 { @@ -309,6 +314,18 @@ func (m *kvMeta) parseEntry(buf []byte) (uint8, Ino) { return b.Get8(), Ino(b.Get64()) } +func (m *kvMeta) packDirStat(usedSpace, usedInodes uint64) []byte { + b := utils.NewBuffer(16) + b.Put64(usedSpace) + b.Put64(usedInodes) + return b.Bytes() +} + +func (m *kvMeta) parseDirStat(buf []byte) (space uint64, inodes uint64) { + b := utils.FromBuffer(buf) + return b.Get64(), b.Get64() +} + func (m *kvMeta) get(key []byte) ([]byte, error) { var value []byte err := m.client.txn(func(tx *kvTxn) error { @@ -851,13 +868,14 @@ func (m *kvMeta) Truncate(ctx Context, inode Ino, flags uint8, length uint64, at } defer func() { m.of.InvalidateChunk(inode, invalidateAllChunks) }() var newSpace int64 + var t Attr err := m.txn(func(tx *kvTxn) error { newSpace = 0 - var t Attr a := tx.get(m.inodeKey(inode)) if a == nil { return syscall.ENOENT } + t = Attr{} m.parseAttr(a, &t) if t.Typ != TypeFile { return syscall.EPERM @@ -905,7 +923,7 @@ func (m *kvMeta) Truncate(ctx Context, inode Ino, flags uint8, length uint64, at return nil }) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, t.Parent, newSpace) } return errno(err) } @@ -934,13 +952,14 @@ func (m *kvMeta) Fallocate(ctx Context, inode Ino, mode uint8, off uint64, size } defer func() { m.of.InvalidateChunk(inode, invalidateAllChunks) }() var newSpace int64 + var t Attr err := m.txn(func(tx *kvTxn) error { newSpace = 0 - var t Attr a := tx.get(m.inodeKey(inode)) if a == nil { return syscall.ENOENT } + t = Attr{} m.parseAttr(a, &t) if t.Typ == TypeFIFO { return syscall.EPIPE @@ -993,7 +1012,7 @@ func (m *kvMeta) Fallocate(ctx Context, inode Ino, mode uint8, off uint64, size return nil }) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, t.Parent, newSpace) } return errno(err) } @@ -1115,6 +1134,9 @@ func (m *kvMeta) doMknod(ctx Context, parent Ino, name string, _type uint8, mode if _type == TypeSymlink { tx.set(m.symKey(ino), []byte(path)) } + if _type == TypeDirectory { + tx.set(m.dirStatKey(ino), m.packDirStat(0, 0)) + } return nil }, parent) if err == nil { @@ -1123,19 +1145,21 @@ func (m *kvMeta) doMknod(ctx Context, parent Ino, name string, _type uint8, mode return errno(err) } -func (m *kvMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno { +func (m *kvMeta) doUnlink(ctx Context, parent Ino, name string, attr *Attr) syscall.Errno { var trash Ino if st := m.checkTrash(parent, &trash); st != 0 { return st } + if attr == nil { + attr = &Attr{} + } var _type uint8 var inode Ino - var attr Attr var opened bool var newSpace, newInode int64 err := m.txn(func(tx *kvTxn) error { opened = false - attr = Attr{} + *attr = Attr{} newSpace, newInode = 0, 0 buf := tx.get(m.entryKey(parent, name)) if buf == nil && m.conf.CaseInsensi { @@ -1163,11 +1187,10 @@ func (m *kvMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno { if (pattr.Flags&FlagAppend) != 0 || (pattr.Flags&FlagImmutable) != 0 { return syscall.EPERM } - attr = Attr{} opened = false now := time.Now() if rs[1] != nil { - m.parseAttr(rs[1], &attr) + m.parseAttr(rs[1], attr) if ctx.Uid() != 0 && pattr.Mode&01000 != 0 && ctx.Uid() != pattr.Uid && ctx.Uid() != attr.Uid { return syscall.EACCES } @@ -1204,7 +1227,7 @@ func (m *kvMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno { tx.set(m.inodeKey(parent), m.marshal(&pattr)) } if attr.Nlink > 0 { - tx.set(m.inodeKey(inode), m.marshal(&attr)) + tx.set(m.inodeKey(inode), m.marshal(attr)) if trash > 0 { tx.set(m.entryKey(trash, m.trashEntry(parent, inode, name)), buf) if attr.Parent == 0 { @@ -1218,7 +1241,7 @@ func (m *kvMeta) doUnlink(ctx Context, parent Ino, name string) syscall.Errno { switch _type { case TypeFile: if opened { - tx.set(m.inodeKey(inode), m.marshal(&attr)) + tx.set(m.inodeKey(inode), m.marshal(attr)) tx.set(m.sustainedKey(m.sid, inode), []byte{1}) } else { tx.set(m.delfileKey(inode, attr.Length), m.packInt64(now.Unix())) @@ -1309,6 +1332,7 @@ func (m *kvMeta) doRmdir(ctx Context, parent Ino, name string) syscall.Errno { tx.set(m.inodeKey(parent), m.marshal(&pattr)) } tx.delete(m.entryKey(parent, name)) + tx.delete(m.dirStatKey(inode)) if trash > 0 { tx.set(m.inodeKey(inode), m.marshal(&attr)) tx.set(m.entryKey(trash, m.trashEntry(parent, inode, name)), buf) @@ -1742,9 +1766,10 @@ func (m *kvMeta) Write(ctx Context, inode Ino, indx uint32, off uint32, slice Sl defer func() { m.of.InvalidateChunk(inode, indx) }() var newSpace int64 var needCompact bool + var attr Attr err := m.txn(func(tx *kvTxn) error { newSpace = 0 - var attr Attr + attr = Attr{} a := tx.get(m.inodeKey(inode)) if a == nil { return syscall.ENOENT @@ -1757,10 +1782,11 @@ func (m *kvMeta) Write(ctx Context, inode Ino, indx uint32, off uint32, slice Sl if newleng > attr.Length { newSpace = align4K(newleng) - align4K(attr.Length) attr.Length = newleng + if m.checkQuota(newSpace, 0) { + return syscall.ENOSPC + } } - if m.checkQuota(newSpace, 0) { - return syscall.ENOSPC - } + now := time.Now() attr.Mtime = now.Unix() attr.Mtimensec = uint32(now.Nanosecond()) @@ -1775,7 +1801,7 @@ func (m *kvMeta) Write(ctx Context, inode Ino, indx uint32, off uint32, slice Sl if needCompact { go m.compactChunk(inode, indx, false) } - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, inode, attr.Parent, newSpace) } return errno(err) } @@ -1789,13 +1815,14 @@ func (m *kvMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, off defer f.Unlock() } defer func() { m.of.InvalidateChunk(fout, invalidateAllChunks) }() + var sattr, attr Attr err := m.txn(func(tx *kvTxn) error { newSpace = 0 rs := tx.gets(m.inodeKey(fin), m.inodeKey(fout)) if rs[0] == nil || rs[1] == nil { return syscall.ENOENT } - var sattr Attr + sattr = Attr{} m.parseAttr(rs[0], &sattr) if sattr.Typ != TypeFile { return syscall.EINVAL @@ -1808,7 +1835,7 @@ func (m *kvMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, off if offIn+size > sattr.Length { size = sattr.Length - offIn } - var attr Attr + attr = Attr{} m.parseAttr(rs[1], &attr) if attr.Typ != TypeFile { return syscall.EINVAL @@ -1896,7 +1923,7 @@ func (m *kvMeta) CopyFileRange(ctx Context, fin Ino, offIn uint64, fout Ino, off return nil }) if err == nil { - m.updateStats(newSpace, 0) + m.updateParentStat(ctx, fout, attr.Parent, newSpace) } return errno(err) } @@ -1917,6 +1944,57 @@ func (m *kvMeta) doGetParents(ctx Context, inode Ino) map[Ino]int { return ps } +func (m *kvMeta) doSyncDirStat(ctx Context, ino Ino) (space, inodes uint64, err error) { + space, inodes, err = m.calcDirStat(ctx, ino) + if err != nil { + return + } + err = m.txn(func(tx *kvTxn) error { + tx.set(m.dirStatKey(ino), m.packDirStat(space, inodes)) + return nil + }) + return +} + +var errEmptyStat = errors.New("empty stat") + +func (m *kvMeta) doUpdateDirStat(ctx Context, ino Ino, space int64, inodes int64) error { + err := m.txn(func(tx *kvTxn) error { + key := m.dirStatKey(ino) + rawStat := tx.get(key) + if rawStat == nil { + return errEmptyStat + } + us, ui := m.parseDirStat(rawStat) + usedSpace, usedInodes := int64(us), int64(ui) + usedSpace += space + usedInodes += inodes + if usedSpace < 0 || usedInodes < 0 { + logger.Warnf("dir stat of inode %d is invalid: space %d, inodes %d, try to sync", ino, usedSpace, usedInodes) + return errEmptyStat + } + tx.set(key, m.packDirStat(uint64(usedSpace), uint64(usedInodes))) + return nil + }) + if err == errEmptyStat { + _, _, err = m.doSyncDirStat(ctx, ino) + } + return err +} + +func (m *kvMeta) doGetDirStat(ctx Context, ino Ino) (space, inodes uint64, err error) { + rawStat, err := m.get(m.dirStatKey(ino)) + if err != nil { + return + } + if rawStat != nil { + space, inodes = m.parseDirStat(rawStat) + } else { + space, inodes, err = m.doSyncDirStat(ctx, ino) + } + return +} + func (m *kvMeta) doFindDeletedFiles(ts int64, limit int) (map[Ino]uint64, error) { klen := 1 + 8 + 8 vals, err := m.scanValues(m.fmtKey("D"), limit, func(k, v []byte) bool { diff --git a/rfcs/1-dir-used-statistics.md b/rfcs/1-dir-used-statistics.md new file mode 100644 index 000000000000..6a40e001fefe --- /dev/null +++ b/rfcs/1-dir-used-statistics.md @@ -0,0 +1,140 @@ +# Count space and inodes usage for each directory + +## Background + +Currently, we have several counters to globally count the used space and inodes, which can be used to show information or set quota. However, we do not have efficient ways to show used information of or set quota for each directory. + +## Proposal + +This document give a proposal to efficiently and almost immediately collect used space and inodes for each directory. The "efficiently" means this operation cannot affect the performance of normal IO operations like `mknod`, `write` .etc. And the "almost immediately" means this operation cannot be lazy or scheduled, we must update the used space and inodes actively, but there may be a little latency (between several seconds and 1 minute). + +## Implementation + +### Storage + +The counters should be stored in meta engines, in this section we introduce how to store them in three kinds of meta engines. + +#### Redis + +Redis engine stores the counters in hashes. + +```go +func (m *redisMeta) dirUsedSpaceKey() string { + return m.prefix + "dirUsedSpace" +} + +func (m *redisMeta) dirUsedInodesKey() string { + return m.prefix + "dirUsedInodes" +} +``` + +#### SQL + +SQL engine stores the counters in a table. + +```go +type dirUsage struct { + Inode Ino `xorm:"pk"` + UsedSpace uint64 `xorm:"notnull"` + UsedInodes uint64 `xorm:"notnull"` +} +``` + +#### TKV + +TKV engine stores each counter in one key. + +```go +func (m *kvMeta) dirUsageKey(inode Ino) []byte { + return m.fmtKey("U", inode) +} +``` + +### Usage + +In this section we represent how and when to update and read the counters. + +#### Update + +The are several file types among the children, we should clarify how to deal with each kinds of files first. + +| Type | Used space | Used inodes | +| ------------- | --------------- | ----------- | +| Normal file | `align4K(size)` | 1 | +| Directory | 4KiB | 1 | +| Symlink | 4KiB | 1 | +| FIFO | 4KiB | 1 | +| Block device | 4KiB | 1 | +| Char device | 4KiB | 1 | +| Socket | 4KiB | 1 | + +Each meta engine should implement `doUpdateDirUsage`. + +```go +type engine interface { + ... + doUpdateDirUsage(ctx Context, ino Ino, space int64, inodes int64) +} +``` + +Relevant IO operations should call `doUpdateDirUsage` asynchronously. + +```go +func (m *baseMeta) Mknod(ctx Context, parent Ino, ...) syscall.Errno { + ... + err := m.en.doMknod(ctx, m.checkRoot(parent), ...) + ... + go m.en.doUpdateDirUsage(ctx, parent, 1<<12, 1) + return err +} + +func (m *baseMeta) Unlink(ctx Context, parent Ino, name string) syscall.Errno { + ... + err := m.en.doUnlink(ctx, m.checkRoot(parent), name) + ... + go m.en.doUpdateDirUsage(ctx, parent, -align4K(attr.size), -1) + return err +} +``` + +#### Read + +Each meta engine should implement `doGetDirUsage`. + +```go +type engine interface { + ... + doGetDirUsage(ctx Context, ino Ino) (space, inodes uint64, err syscall.Errno) +} +``` + +Now we can fasly recursively calculate the space and inodes usage in a directory by `doGetDirUsage`. + +```go +// walk all directories in root +func (m *baseMeta) fastWalkDir(ctx Context, inode Ino, walkDir func(Context, Ino)) syscall.Errno { + walkDir(ctx, inode) + var entries []*Entry + st := m.en.doReaddir(ctx, inode, 0, &entries, -1) // disable plus + ... + for _, entry := range entries { + if ent.Attr.Typ != TypeDirectory { + continue + } + m.fastWalkDir(ctx, entry.Inode, walkFn) + ... + } + return 0 +} +func (m *baseMeta) getDirUsage(ctx Context, root Ino) (space, inodes uint64, err syscall.Errno) { + m.fastWalkDir(ctx, root, func(_ Context, ino Ino) { + s, i, err := m.doGetDirUsage(ctx, ino) + ... + space += s + inodes += i + }) + return +} +``` + +