[#1004] blobovnicza: Use TTL for blobovnicza tree cache
All checks were successful
DCO action / DCO (pull_request) Successful in 12m46s
Vulncheck / Vulncheck (pull_request) Successful in 14m10s
Build / Build Components (1.22) (pull_request) Successful in 15m23s
Build / Build Components (1.21) (pull_request) Successful in 15m38s
Tests and linters / Staticcheck (pull_request) Successful in 3m36s
Tests and linters / gopls check (pull_request) Successful in 4m0s
Tests and linters / Lint (pull_request) Successful in 5m8s
Tests and linters / Tests (1.21) (pull_request) Successful in 7m41s
Tests and linters / Tests with -race (pull_request) Successful in 7m46s
Tests and linters / Tests (1.22) (pull_request) Successful in 7m58s

Signed-off-by: Anton Nikiforov <an.nikiforov@yadro.com>
This commit is contained in:
Anton Nikiforov 2024-03-01 14:43:26 +03:00
parent 112a7c690f
commit 411a8d0245
26 changed files with 203 additions and 65 deletions

View file

@ -1,6 +1,7 @@
package blobovniczatree
import (
"context"
"errors"
"strconv"
"strings"
@ -72,7 +73,7 @@ const (
)
// NewBlobovniczaTree returns new instance of blobovniczas tree.
func NewBlobovniczaTree(opts ...Option) (blz *Blobovniczas) {
func NewBlobovniczaTree(ctx context.Context, opts ...Option) (blz *Blobovniczas) {
blz = new(Blobovniczas)
initConfig(&blz.cfg)
@ -86,7 +87,8 @@ func NewBlobovniczaTree(opts ...Option) (blz *Blobovniczas) {
blz.commondbManager = newDBManager(blz.rootPath, blz.blzOpts, blz.readOnly, blz.metrics.Blobovnicza(), blz.log)
blz.activeDBManager = newActiveDBManager(blz.commondbManager, blz.blzLeafWidth)
blz.dbCache = newDBCache(blz.openedCacheSize, blz.commondbManager)
blz.dbCache = newDBCache(ctx, blz.openedCacheSize,
blz.openedCacheTTL, blz.openedCacheExpInterval, blz.commondbManager)
blz.deleteProtectedObjects = newAddressMap()
blz.dbFilesGuard = &sync.RWMutex{}
blz.rebuildGuard = &sync.RWMutex{}

View file

@ -1,11 +1,12 @@
package blobovniczatree
import (
"fmt"
"context"
"sync"
"time"
utilSync "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/sync"
"github.com/hashicorp/golang-lru/v2/simplelru"
cache "github.com/go-pkgz/expirable-cache/v3"
)
// dbCache caches sharedDB instances that are NOT open for Put.
@ -13,30 +14,55 @@ import (
// Uses dbManager for opening/closing sharedDB instances.
// Stores a reference to an cached sharedDB, so dbManager does not close it.
type dbCache struct {
cacheGuard *sync.RWMutex
cache simplelru.LRUCache[string, *sharedDB]
cacheGuard *sync.Mutex
cache cache.Cache[string, *sharedDB]
pathLock *utilSync.KeyLocker[string] // the order of locks is important: pathLock first, cacheGuard second
closed bool
nonCached map[string]struct{}
wg sync.WaitGroup
cancel context.CancelFunc
dbManager *dbManager
}
func newDBCache(size int, dbManager *dbManager) *dbCache {
cache, err := simplelru.NewLRU(size, func(_ string, evictedDB *sharedDB) {
evictedDB.Close()
})
if err != nil {
// occurs only if the size is not positive
panic(fmt.Errorf("could not create LRU cache of size %d: %w", size, err))
}
return &dbCache{
cacheGuard: &sync.RWMutex{},
cache: cache,
func newDBCache(parentCtx context.Context, size int,
ttl time.Duration, expInterval time.Duration,
dbManager *dbManager,
) *dbCache {
ch := cache.NewCache[string, *sharedDB]().
WithTTL(ttl).WithLRU().WithMaxKeys(size).
WithOnEvicted(func(_ string, db *sharedDB) {
db.Close()
})
ctx, cancel := context.WithCancel(parentCtx)
res := &dbCache{
cacheGuard: &sync.Mutex{},
wg: sync.WaitGroup{},
cancel: cancel,
cache: ch,
dbManager: dbManager,
pathLock: utilSync.NewKeyLocker[string](),
nonCached: make(map[string]struct{}),
}
if ttl > 0 {
res.wg.Add(1)
go func() {
ticker := time.NewTicker(expInterval)
defer ticker.Stop()
for {
select {
case <-ctx.Done():
res.wg.Done()
return
case <-ticker.C:
res.cacheGuard.Lock()
res.cache.DeleteExpired()
res.cacheGuard.Unlock()
}
}
}()
}
return res
}
func (c *dbCache) Open() {
@ -49,6 +75,8 @@ func (c *dbCache) Open() {
func (c *dbCache) Close() {
c.cacheGuard.Lock()
defer c.cacheGuard.Unlock()
c.cancel()
c.wg.Wait()
c.cache.Purge()
c.closed = true
}
@ -88,6 +116,8 @@ func (c *dbCache) getExisted(path string) *sharedDB {
if value, ok := c.cache.Get(path); ok {
return value
} else if value != nil {
c.cache.Invalidate(path)
}
return nil
}
@ -119,10 +149,9 @@ func (c *dbCache) put(path string, db *sharedDB) bool {
_, isNonCached := c.nonCached[path]
if !isNonCached && !c.closed {
c.cache.Add(path, db)
return true
if isNonCached || c.closed {
return false
}
return false
c.cache.Add(path, db)
return true
}

View file

@ -17,6 +17,7 @@ func TestBlobovniczaTree_Concurrency(t *testing.T) {
const n = 1000
st := NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(1024),
WithBlobovniczaShallowWidth(10),

View file

@ -49,6 +49,7 @@ func createTestTree(t *testing.T, currentDepth, depth, width uint64, path string
func openAndCloseTestTree(t *testing.T, depth, width uint64, path string) {
blz := NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(depth),
WithBlobovniczaShallowWidth(width),
WithRootPath(path),
@ -78,6 +79,7 @@ func TestObjectsAvailableAfterDepthAndWidthEdit(t *testing.T) {
rootDir := t.TempDir()
blz := NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(3),
WithBlobovniczaShallowWidth(5),
WithRootPath(rootDir),
@ -115,6 +117,7 @@ func TestObjectsAvailableAfterDepthAndWidthEdit(t *testing.T) {
// change depth and width
blz = NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(5),
WithBlobovniczaShallowWidth(2),
WithRootPath(rootDir),
@ -152,6 +155,7 @@ func TestObjectsAvailableAfterDepthAndWidthEdit(t *testing.T) {
// change depth and width back
blz = NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(3),
WithBlobovniczaShallowWidth(5),
WithRootPath(rootDir),

View file

@ -17,6 +17,7 @@ import (
func TestExistsInvalidStorageID(t *testing.T) {
dir := t.TempDir()
b := NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(1024),
WithBlobovniczaShallowWidth(2),

View file

@ -1,6 +1,7 @@
package blobovniczatree
import (
"context"
"testing"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/common"
@ -13,6 +14,7 @@ func TestGeneric(t *testing.T) {
helper := func(t *testing.T, dir string) common.Storage {
return NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(maxObjectSize),
WithBlobovniczaShallowWidth(2),
@ -40,6 +42,7 @@ func TestControl(t *testing.T) {
newTree := func(t *testing.T) common.Storage {
return NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(maxObjectSize),
WithBlobovniczaShallowWidth(2),

View file

@ -12,6 +12,7 @@ func TestIterateSortedLeavesAndDBPathsAreSame(t *testing.T) {
t.Parallel()
blz := NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(3),
WithBlobovniczaShallowWidth(5),
WithRootPath(t.TempDir()),

View file

@ -27,32 +27,40 @@ type cfg struct {
blzInitWorkerCount int
blzMoveBatchSize int
createDBInAdvance bool
// TTL for blobovnicza's cache
openedCacheTTL time.Duration
// Interval for deletion expired blobovnicza's
openedCacheExpInterval time.Duration
}
type Option func(*cfg)
const (
defaultPerm = 0o700
defaultOpenedCacheSize = 50
defaultBlzShallowDepth = 2
defaultBlzShallowWidth = 16
defaultWaitBeforeDropDB = 10 * time.Second
defaultBlzInitWorkerCount = 5
defaulBlzMoveBatchSize = 10000
defaultPerm = 0o700
defaultOpenedCacheSize = 50
defaultOpenedCacheTTL = 0 // means expiring is off
defaultOpenedCacheInterval = 15 * time.Second
defaultBlzShallowDepth = 2
defaultBlzShallowWidth = 16
defaultWaitBeforeDropDB = 10 * time.Second
defaultBlzInitWorkerCount = 5
defaulBlzMoveBatchSize = 10000
)
func initConfig(c *cfg) {
*c = cfg{
log: &logger.Logger{Logger: zap.L()},
perm: defaultPerm,
openedCacheSize: defaultOpenedCacheSize,
blzShallowDepth: defaultBlzShallowDepth,
blzShallowWidth: defaultBlzShallowWidth,
reportError: func(string, error) {},
metrics: &noopMetrics{},
waitBeforeDropDB: defaultWaitBeforeDropDB,
blzInitWorkerCount: defaultBlzInitWorkerCount,
blzMoveBatchSize: defaulBlzMoveBatchSize,
log: &logger.Logger{Logger: zap.L()},
perm: defaultPerm,
openedCacheSize: defaultOpenedCacheSize,
openedCacheTTL: defaultOpenedCacheTTL,
openedCacheExpInterval: defaultOpenedCacheInterval,
blzShallowDepth: defaultBlzShallowDepth,
blzShallowWidth: defaultBlzShallowWidth,
reportError: func(string, error) {},
metrics: &noopMetrics{},
waitBeforeDropDB: defaultWaitBeforeDropDB,
blzInitWorkerCount: defaultBlzInitWorkerCount,
blzMoveBatchSize: defaulBlzMoveBatchSize,
}
}
@ -105,6 +113,18 @@ func WithOpenedCacheSize(sz int) Option {
}
}
func WithOpenedCacheTTL(ttl time.Duration) Option {
return func(c *cfg) {
c.openedCacheTTL = ttl
}
}
func WithOpenedCacheExpInterval(expInterval time.Duration) Option {
return func(c *cfg) {
c.openedCacheExpInterval = expInterval
}
}
func WithObjectSizeLimit(sz uint64) Option {
return func(c *cfg) {
c.blzOpts = append(c.blzOpts, blobovnicza.WithObjectSizeLimit(sz))

View file

@ -129,6 +129,7 @@ func testRebuildFailoverObjectDeletedFromSource(t *testing.T) {
func testRebuildFailoverValidate(t *testing.T, dir string, obj *objectSDK.Object, mustUpdateStorageID bool) {
b := NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(2048),
WithBlobovniczaShallowWidth(2),

View file

@ -43,6 +43,7 @@ func TestBlobovniczaTreeRebuildLargeObject(t *testing.T) {
dir := t.TempDir()
b := NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(64*1024), // 64KB object size limit
WithBlobovniczaShallowWidth(5),
@ -70,6 +71,7 @@ func TestBlobovniczaTreeRebuildLargeObject(t *testing.T) {
require.NoError(t, b.Close())
b = NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(32*1024), // 32KB object size limit
WithBlobovniczaShallowWidth(5),
@ -108,6 +110,7 @@ func TestBlobovniczaTreeRebuildLargeObject(t *testing.T) {
func testBlobovniczaTreeRebuildHelper(t *testing.T, sourceDepth, sourceWidth, targetDepth, targetWidth uint64, shouldMigrate bool) {
dir := t.TempDir()
b := NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(2048),
WithBlobovniczaShallowWidth(sourceWidth),
@ -148,6 +151,7 @@ func testBlobovniczaTreeRebuildHelper(t *testing.T, sourceDepth, sourceWidth, ta
require.NoError(t, b.Close())
b = NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(2048),
WithBlobovniczaShallowWidth(targetWidth),

View file

@ -18,6 +18,7 @@ import (
func defaultTestStorages(p string, smallSizeLimit uint64) ([]SubStorage, *teststore.TestStore, *teststore.TestStore) {
smallFileStorage := teststore.New(teststore.WithSubstorage(blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithRootPath(filepath.Join(p, "blobovniczas")),
blobovniczatree.WithBlobovniczaShallowWidth(1)), // default width is 16, slow init
))
@ -117,6 +118,7 @@ func TestBlobstor_needsCompression(t *testing.T) {
WithStorages([]SubStorage{
{
Storage: blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithRootPath(filepath.Join(dir, "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowWidth(1)), // default width is 16, slow init
Policy: func(_ *objectSDK.Object, data []byte) bool {

View file

@ -73,6 +73,7 @@ var storages = []storage{
desc: "blobovniczatree",
create: func(dir string) common.Storage {
return blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithRootPath(dir),
)
},