[#1004] blobovnicza: Use TTL for blobovnicza tree cache

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 201 additions and 65 deletions

View file

@ -61,6 +61,8 @@ storage:
depth: 1 # max depth of object tree storage in key-value DB depth: 1 # max depth of object tree storage in key-value DB
width: 4 # max width of object tree storage in key-value DB width: 4 # max width of object tree storage in key-value DB
opened_cache_capacity: 50 # maximum number of opened database files opened_cache_capacity: 50 # maximum number of opened database files
opened_cache_ttl: 5m # ttl for opened database file
opened_cache_exp_interval: 15s # cache cleanup interval for expired blobovnicza's
gc: gc:
remover_batch_size: 200 # number of objects to be removed by the garbage collector remover_batch_size: 200 # number of objects to be removed by the garbage collector

View file

@ -185,13 +185,15 @@ type subStorageCfg struct {
noSync bool noSync bool
// blobovnicza-specific // blobovnicza-specific
size uint64 size uint64
width uint64 width uint64
leafWidth uint64 leafWidth uint64
openedCacheSize int openedCacheSize int
initWorkerCount int initWorkerCount int
initInAdvance bool initInAdvance bool
rebuildDropTimeout time.Duration rebuildDropTimeout time.Duration
openedCacheTTL time.Duration
openedCacheExpInterval time.Duration
} }
// readConfig fills applicationConfiguration with raw configuration values // readConfig fills applicationConfiguration with raw configuration values
@ -313,6 +315,8 @@ func (a *applicationConfiguration) setShardStorageConfig(newConfig *shardCfg, ol
sCfg.width = sub.ShallowWidth() sCfg.width = sub.ShallowWidth()
sCfg.leafWidth = sub.LeafWidth() sCfg.leafWidth = sub.LeafWidth()
sCfg.openedCacheSize = sub.OpenedCacheSize() sCfg.openedCacheSize = sub.OpenedCacheSize()
sCfg.openedCacheTTL = sub.OpenedCacheTTL()
sCfg.openedCacheExpInterval = sub.OpenedCacheExpInterval()
sCfg.initWorkerCount = sub.InitWorkerCount() sCfg.initWorkerCount = sub.InitWorkerCount()
sCfg.initInAdvance = sub.InitInAdvance() sCfg.initInAdvance = sub.InitInAdvance()
sCfg.rebuildDropTimeout = sub.RebuildDropTimeout() sCfg.rebuildDropTimeout = sub.RebuildDropTimeout()
@ -846,11 +850,11 @@ type shardOptsWithID struct {
shOpts []shard.Option shOpts []shard.Option
} }
func (c *cfg) shardOpts() []shardOptsWithID { func (c *cfg) shardOpts(ctx context.Context) []shardOptsWithID {
shards := make([]shardOptsWithID, 0, len(c.EngineCfg.shards)) shards := make([]shardOptsWithID, 0, len(c.EngineCfg.shards))
for _, shCfg := range c.EngineCfg.shards { for _, shCfg := range c.EngineCfg.shards {
shards = append(shards, c.getShardOpts(shCfg)) shards = append(shards, c.getShardOpts(ctx, shCfg))
} }
return shards return shards
@ -891,7 +895,7 @@ func (c *cfg) getPiloramaOpts(shCfg shardCfg) []pilorama.Option {
return piloramaOpts return piloramaOpts
} }
func (c *cfg) getSubstorageOpts(shCfg shardCfg) []blobstor.SubStorage { func (c *cfg) getSubstorageOpts(ctx context.Context, shCfg shardCfg) []blobstor.SubStorage {
var ss []blobstor.SubStorage var ss []blobstor.SubStorage
for _, sRead := range shCfg.subStorages { for _, sRead := range shCfg.subStorages {
switch sRead.typ { switch sRead.typ {
@ -904,6 +908,8 @@ func (c *cfg) getSubstorageOpts(shCfg shardCfg) []blobstor.SubStorage {
blobovniczatree.WithBlobovniczaShallowWidth(sRead.width), blobovniczatree.WithBlobovniczaShallowWidth(sRead.width),
blobovniczatree.WithBlobovniczaLeafWidth(sRead.leafWidth), blobovniczatree.WithBlobovniczaLeafWidth(sRead.leafWidth),
blobovniczatree.WithOpenedCacheSize(sRead.openedCacheSize), blobovniczatree.WithOpenedCacheSize(sRead.openedCacheSize),
blobovniczatree.WithOpenedCacheTTL(sRead.openedCacheTTL),
blobovniczatree.WithOpenedCacheExpInterval(sRead.openedCacheExpInterval),
blobovniczatree.WithInitWorkerCount(sRead.initWorkerCount), blobovniczatree.WithInitWorkerCount(sRead.initWorkerCount),
blobovniczatree.WithInitInAdvance(sRead.initInAdvance), blobovniczatree.WithInitInAdvance(sRead.initInAdvance),
blobovniczatree.WithWaitBeforeDropDB(sRead.rebuildDropTimeout), blobovniczatree.WithWaitBeforeDropDB(sRead.rebuildDropTimeout),
@ -919,7 +925,7 @@ func (c *cfg) getSubstorageOpts(shCfg shardCfg) []blobstor.SubStorage {
) )
} }
ss = append(ss, blobstor.SubStorage{ ss = append(ss, blobstor.SubStorage{
Storage: blobovniczatree.NewBlobovniczaTree(blobTreeOpts...), Storage: blobovniczatree.NewBlobovniczaTree(ctx, blobTreeOpts...),
Policy: func(_ *objectSDK.Object, data []byte) bool { Policy: func(_ *objectSDK.Object, data []byte) bool {
return uint64(len(data)) < shCfg.smallSizeObjectLimit return uint64(len(data)) < shCfg.smallSizeObjectLimit
}, },
@ -954,10 +960,10 @@ func (c *cfg) getSubstorageOpts(shCfg shardCfg) []blobstor.SubStorage {
return ss return ss
} }
func (c *cfg) getShardOpts(shCfg shardCfg) shardOptsWithID { func (c *cfg) getShardOpts(ctx context.Context, shCfg shardCfg) shardOptsWithID {
writeCacheOpts := c.getWriteCacheOpts(shCfg) writeCacheOpts := c.getWriteCacheOpts(shCfg)
piloramaOpts := c.getPiloramaOpts(shCfg) piloramaOpts := c.getPiloramaOpts(shCfg)
ss := c.getSubstorageOpts(shCfg) ss := c.getSubstorageOpts(ctx, shCfg)
blobstoreOpts := []blobstor.Option{ blobstoreOpts := []blobstor.Option{
blobstor.WithCompressObjects(shCfg.compress), blobstor.WithCompressObjects(shCfg.compress),
@ -1049,7 +1055,7 @@ func initLocalStorage(ctx context.Context, c *cfg) {
c.cfgObject.getSvc = new(getsvc.Service) c.cfgObject.getSvc = new(getsvc.Service)
var shardsAttached int var shardsAttached int
for _, optsWithMeta := range c.shardOpts() { for _, optsWithMeta := range c.shardOpts(ctx) {
id, err := ls.AddShard(ctx, append(optsWithMeta.shOpts, shard.WithTombstoneSource(c.createTombstoneSource()))...) id, err := ls.AddShard(ctx, append(optsWithMeta.shOpts, shard.WithTombstoneSource(c.createTombstoneSource()))...)
if err != nil { if err != nil {
c.log.Error(logs.FrostFSNodeFailedToAttachShardToEngine, zap.Error(err)) c.log.Error(logs.FrostFSNodeFailedToAttachShardToEngine, zap.Error(err))
@ -1289,7 +1295,7 @@ func (c *cfg) reloadConfig(ctx context.Context) {
// Storage Engine // Storage Engine
var rcfg engine.ReConfiguration var rcfg engine.ReConfiguration
for _, optsWithID := range c.shardOpts() { for _, optsWithID := range c.shardOpts(ctx) {
rcfg.AddShard(optsWithID.configID, append(optsWithID.shOpts, shard.WithTombstoneSource(c.createTombstoneSource()))) rcfg.AddShard(optsWithID.configID, append(optsWithID.shOpts, shard.WithTombstoneSource(c.createTombstoneSource())))
} }

View file

@ -25,6 +25,12 @@ const (
// OpenedCacheSizeDefault is a default cache size of opened Blobovnicza's. // OpenedCacheSizeDefault is a default cache size of opened Blobovnicza's.
OpenedCacheSizeDefault = 16 OpenedCacheSizeDefault = 16
// OpenedCacheTTLDefault is a default cache ttl of opened Blobovnicza's.
OpenedCacheTTLDefault = 0 // means expiring is off
// OpenedCacheExpIntervalDefault is a default cache cleanup interval for expired Blobovnicza's.
OpenedCacheExpIntervalDefault = 15 * time.Second
// InitWorkerCountDefault is a default workers count to initialize Blobovnicza's. // InitWorkerCountDefault is a default workers count to initialize Blobovnicza's.
InitWorkerCountDefault = 5 InitWorkerCountDefault = 5
@ -106,6 +112,38 @@ func (x *Config) OpenedCacheSize() int {
return OpenedCacheSizeDefault return OpenedCacheSizeDefault
} }
// OpenedCacheTTL returns the value of "opened_cache_ttl" config parameter.
//
// Returns OpenedCacheTTLDefault if the value is not a positive number.
func (x *Config) OpenedCacheTTL() time.Duration {
d := config.DurationSafe(
(*config.Config)(x),
"opened_cache_ttl",
)
if d > 0 {
return d
}
return OpenedCacheTTLDefault
}
// OpenedCacheExpInterval returns the value of "opened_cache_exp_interval" config parameter.
//
// Returns OpenedCacheExpIntervalDefault if the value is not a positive number.
func (x *Config) OpenedCacheExpInterval() time.Duration {
d := config.DurationSafe(
(*config.Config)(x),
"opened_cache_exp_interval",
)
if d > 0 {
return d
}
return OpenedCacheExpIntervalDefault
}
// BoltDB returns config instance for querying bolt db specific parameters. // BoltDB returns config instance for querying bolt db specific parameters.
func (x *Config) BoltDB() *boltdbconfig.Config { func (x *Config) BoltDB() *boltdbconfig.Config {
return (*boltdbconfig.Config)(x) return (*boltdbconfig.Config)(x)

View file

@ -128,6 +128,8 @@ FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_SIZE=4194304
FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_DEPTH=1 FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_DEPTH=1
FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_WIDTH=4 FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_WIDTH=4
FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_OPENED_CACHE_CAPACITY=50 FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_OPENED_CACHE_CAPACITY=50
FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_OPENED_CACHE_TTL=5m
FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_OPENED_CACHE_EXP_INTERVAL=15s
FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_LEAF_WIDTH=10 FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_LEAF_WIDTH=10
FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_INIT_WORKER_COUNT=10 FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_INIT_WORKER_COUNT=10
FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_INIT_IN_ADVANCE=TRUE FROSTFS_STORAGE_SHARD_0_BLOBSTOR_0_INIT_IN_ADVANCE=TRUE
@ -178,6 +180,8 @@ FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_SIZE=4194304
FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_DEPTH=1 FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_DEPTH=1
FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_WIDTH=4 FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_WIDTH=4
FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_OPENED_CACHE_CAPACITY=50 FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_OPENED_CACHE_CAPACITY=50
FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_OPENED_CACHE_TTL=5m
FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_OPENED_CACHE_EXP_INTERVAL=15s
FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_LEAF_WIDTH=10 FROSTFS_STORAGE_SHARD_1_BLOBSTOR_0_LEAF_WIDTH=10
### FSTree config ### FSTree config
FROSTFS_STORAGE_SHARD_1_BLOBSTOR_1_TYPE=fstree FROSTFS_STORAGE_SHARD_1_BLOBSTOR_1_TYPE=fstree

View file

@ -176,6 +176,8 @@
"depth": 1, "depth": 1,
"width": 4, "width": 4,
"opened_cache_capacity": 50, "opened_cache_capacity": 50,
"opened_cache_ttl": "5m",
"opened_cache_exp_interval": "15s",
"leaf_width": 10, "leaf_width": 10,
"init_worker_count": 10, "init_worker_count": 10,
"init_in_advance": true, "init_in_advance": true,
@ -229,6 +231,8 @@
"depth": 1, "depth": 1,
"width": 4, "width": 4,
"opened_cache_capacity": 50, "opened_cache_capacity": 50,
"opened_cache_ttl": "5m",
"opened_cache_exp_interval": "15s",
"leaf_width": 10 "leaf_width": 10
}, },
{ {

View file

@ -151,6 +151,8 @@ storage:
depth: 1 # max depth of object tree storage in key-value DB depth: 1 # max depth of object tree storage in key-value DB
width: 4 # max width of object tree storage in key-value DB width: 4 # max width of object tree storage in key-value DB
opened_cache_capacity: 50 # maximum number of opened database files opened_cache_capacity: 50 # maximum number of opened database files
opened_cache_ttl: 5m # ttl for opened database file
opened_cache_exp_interval: 15s # cache cleanup interval for expired blobovnicza's
leaf_width: 10 # max count of key-value DB on leafs of object tree storage leaf_width: 10 # max count of key-value DB on leafs of object tree storage
- perm: 0644 # permissions for blobstor files(directories: +x for current user and group) - perm: 0644 # permissions for blobstor files(directories: +x for current user and group)
depth: 5 # max depth of object tree storage in FS depth: 5 # max depth of object tree storage in FS

View file

@ -212,6 +212,8 @@ blobstor:
depth: 1 depth: 1
width: 4 width: 4
opened_cache_capacity: 50 opened_cache_capacity: 50
opened_cache_ttl: 5m
opened_cache_exp_interval: 15s
``` ```
#### Common options for sub-storages #### Common options for sub-storages
@ -228,17 +230,19 @@ blobstor:
| `depth` | `int` | `4` | File-system tree depth. | | `depth` | `int` | `4` | File-system tree depth. |
#### `blobovnicza` type options #### `blobovnicza` type options
| Parameter | Type | Default value | Description | | Parameter | Type | Default value | Description |
| ----------------------- | ---------- | ------------- | --------------------------------------------------------------------- | |-----------------------------| ---------- |---------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| `path` | `string` | | Path to the root of the blobstor. | | `path` | `string` | | Path to the root of the blobstor. |
| `perm` | file mode | `0660` | Default permission for created files and directories. | | `perm` | file mode | `0660` | Default permission for created files and directories. |
| `size` | `size` | `1 G` | Maximum size of a single blobovnicza | | `size` | `size` | `1 G` | Maximum size of a single blobovnicza |
| `depth` | `int` | `2` | Blobovnicza tree depth. | | `depth` | `int` | `2` | Blobovnicza tree depth. |
| `width` | `int` | `16` | Blobovnicza tree width. | | `width` | `int` | `16` | Blobovnicza tree width. |
| `opened_cache_capacity` | `int` | `16` | Maximum number of simultaneously opened blobovniczas. | | `opened_cache_capacity` | `int` | `16` | Maximum number of simultaneously opened blobovniczas. |
| `init_worker_count` | `int` | `5` | Maximum number of concurrent initialization workers. | | `opened_cache_ttl` | `duration` | `0` | TTL in cache for opened blobovniczas(disabled by default). In case of heavy random-read and 10 shards each with 10_000 databases and accessing 400 objects per-second we will access each db approximately once per ((10 * 10_000 / 400) = 250 seconds <= 300 seconds = 5 min). Also take in mind that in this scenario they will probably be closed earlier because of the cache capacity, so bigger values are likely to be of no use. |
| `init_in_advance` | `bool` | `false` | If `true`, than all the blobovnicza files will be created on startup. | | `opened_cache_exp_interval` | `duration` | `15s` | Cache cleanup interval for expired blobovnicza's. |
| `rebuild_drop_timeout` | `duration` | `10s` | Timeout before drop empty blobovnicza file during rebuild. | | `init_worker_count` | `int` | `5` | Maximum number of concurrent initialization workers. |
| `init_in_advance` | `bool` | `false` | If `true`, than all the blobovnicza files will be created on startup. |
| `rebuild_drop_timeout` | `duration` | `10s` | Timeout before drop empty blobovnicza file during rebuild. |
### `gc` subsection ### `gc` subsection

1
go.mod
View file

@ -16,6 +16,7 @@ require (
github.com/chzyer/readline v1.5.1 github.com/chzyer/readline v1.5.1
github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc
github.com/flynn-archive/go-shlex v0.0.0-20150515145356-3f9db97f8568 github.com/flynn-archive/go-shlex v0.0.0-20150515145356-3f9db97f8568
github.com/go-pkgz/expirable-cache/v3 v3.0.0
github.com/google/uuid v1.6.0 github.com/google/uuid v1.6.0
github.com/hashicorp/golang-lru/v2 v2.0.7 github.com/hashicorp/golang-lru/v2 v2.0.7
github.com/klauspost/compress v1.17.4 github.com/klauspost/compress v1.17.4

BIN
go.sum

Binary file not shown.

View file

@ -1,6 +1,7 @@
package blobovniczatree package blobovniczatree
import ( import (
"context"
"errors" "errors"
"strconv" "strconv"
"strings" "strings"
@ -72,7 +73,7 @@ const (
) )
// NewBlobovniczaTree returns new instance of blobovniczas tree. // 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) blz = new(Blobovniczas)
initConfig(&blz.cfg) 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.commondbManager = newDBManager(blz.rootPath, blz.blzOpts, blz.readOnly, blz.metrics.Blobovnicza(), blz.log)
blz.activeDBManager = newActiveDBManager(blz.commondbManager, blz.blzLeafWidth) 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.deleteProtectedObjects = newAddressMap()
blz.dbFilesGuard = &sync.RWMutex{} blz.dbFilesGuard = &sync.RWMutex{}
blz.rebuildGuard = &sync.RWMutex{} blz.rebuildGuard = &sync.RWMutex{}

View file

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

View file

@ -17,6 +17,7 @@ func TestBlobovniczaTree_Concurrency(t *testing.T) {
const n = 1000 const n = 1000
st := NewBlobovniczaTree( st := NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)), WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(1024), WithObjectSizeLimit(1024),
WithBlobovniczaShallowWidth(10), 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) { func openAndCloseTestTree(t *testing.T, depth, width uint64, path string) {
blz := NewBlobovniczaTree( blz := NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(depth), WithBlobovniczaShallowDepth(depth),
WithBlobovniczaShallowWidth(width), WithBlobovniczaShallowWidth(width),
WithRootPath(path), WithRootPath(path),
@ -78,6 +79,7 @@ func TestObjectsAvailableAfterDepthAndWidthEdit(t *testing.T) {
rootDir := t.TempDir() rootDir := t.TempDir()
blz := NewBlobovniczaTree( blz := NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(3), WithBlobovniczaShallowDepth(3),
WithBlobovniczaShallowWidth(5), WithBlobovniczaShallowWidth(5),
WithRootPath(rootDir), WithRootPath(rootDir),
@ -115,6 +117,7 @@ func TestObjectsAvailableAfterDepthAndWidthEdit(t *testing.T) {
// change depth and width // change depth and width
blz = NewBlobovniczaTree( blz = NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(5), WithBlobovniczaShallowDepth(5),
WithBlobovniczaShallowWidth(2), WithBlobovniczaShallowWidth(2),
WithRootPath(rootDir), WithRootPath(rootDir),
@ -152,6 +155,7 @@ func TestObjectsAvailableAfterDepthAndWidthEdit(t *testing.T) {
// change depth and width back // change depth and width back
blz = NewBlobovniczaTree( blz = NewBlobovniczaTree(
context.Background(),
WithBlobovniczaShallowDepth(3), WithBlobovniczaShallowDepth(3),
WithBlobovniczaShallowWidth(5), WithBlobovniczaShallowWidth(5),
WithRootPath(rootDir), WithRootPath(rootDir),

View file

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

View file

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

View file

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

View file

@ -27,32 +27,40 @@ type cfg struct {
blzInitWorkerCount int blzInitWorkerCount int
blzMoveBatchSize int blzMoveBatchSize int
createDBInAdvance bool createDBInAdvance bool
// TTL for blobovnicza's cache
openedCacheTTL time.Duration
// Interval for deletion expired blobovnicza's
openedCacheExpInterval time.Duration
} }
type Option func(*cfg) type Option func(*cfg)
const ( const (
defaultPerm = 0o700 defaultPerm = 0o700
defaultOpenedCacheSize = 50 defaultOpenedCacheSize = 50
defaultBlzShallowDepth = 2 defaultOpenedCacheTTL = 0 // means expiring is off
defaultBlzShallowWidth = 16 defaultOpenedCacheInterval = 15 * time.Second
defaultWaitBeforeDropDB = 10 * time.Second defaultBlzShallowDepth = 2
defaultBlzInitWorkerCount = 5 defaultBlzShallowWidth = 16
defaulBlzMoveBatchSize = 10000 defaultWaitBeforeDropDB = 10 * time.Second
defaultBlzInitWorkerCount = 5
defaulBlzMoveBatchSize = 10000
) )
func initConfig(c *cfg) { func initConfig(c *cfg) {
*c = cfg{ *c = cfg{
log: &logger.Logger{Logger: zap.L()}, log: &logger.Logger{Logger: zap.L()},
perm: defaultPerm, perm: defaultPerm,
openedCacheSize: defaultOpenedCacheSize, openedCacheSize: defaultOpenedCacheSize,
blzShallowDepth: defaultBlzShallowDepth, openedCacheTTL: defaultOpenedCacheTTL,
blzShallowWidth: defaultBlzShallowWidth, openedCacheExpInterval: defaultOpenedCacheInterval,
reportError: func(string, error) {}, blzShallowDepth: defaultBlzShallowDepth,
metrics: &noopMetrics{}, blzShallowWidth: defaultBlzShallowWidth,
waitBeforeDropDB: defaultWaitBeforeDropDB, reportError: func(string, error) {},
blzInitWorkerCount: defaultBlzInitWorkerCount, metrics: &noopMetrics{},
blzMoveBatchSize: defaulBlzMoveBatchSize, 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 { func WithObjectSizeLimit(sz uint64) Option {
return func(c *cfg) { return func(c *cfg) {
c.blzOpts = append(c.blzOpts, blobovnicza.WithObjectSizeLimit(sz)) 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) { func testRebuildFailoverValidate(t *testing.T, dir string, obj *objectSDK.Object, mustUpdateStorageID bool) {
b := NewBlobovniczaTree( b := NewBlobovniczaTree(
context.Background(),
WithLogger(test.NewLogger(t)), WithLogger(test.NewLogger(t)),
WithObjectSizeLimit(2048), WithObjectSizeLimit(2048),
WithBlobovniczaShallowWidth(2), WithBlobovniczaShallowWidth(2),

View file

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

View file

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

View file

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

View file

@ -137,6 +137,7 @@ func newStorages(t testing.TB, root string, smallSize uint64) []blobstor.SubStor
return []blobstor.SubStorage{ return []blobstor.SubStorage{
{ {
Storage: blobovniczatree.NewBlobovniczaTree( Storage: blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithRootPath(filepath.Join(root, "blobovnicza")), blobovniczatree.WithRootPath(filepath.Join(root, "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(1), blobovniczatree.WithBlobovniczaShallowDepth(1),
blobovniczatree.WithBlobovniczaShallowWidth(1), blobovniczatree.WithBlobovniczaShallowWidth(1),
@ -158,6 +159,7 @@ func newStorages(t testing.TB, root string, smallSize uint64) []blobstor.SubStor
func newTestStorages(root string, smallSize uint64) ([]blobstor.SubStorage, *teststore.TestStore, *teststore.TestStore) { func newTestStorages(root string, smallSize uint64) ([]blobstor.SubStorage, *teststore.TestStore, *teststore.TestStore) {
smallFileStorage := teststore.New( smallFileStorage := teststore.New(
teststore.WithSubstorage(blobovniczatree.NewBlobovniczaTree( teststore.WithSubstorage(blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithRootPath(filepath.Join(root, "blobovnicza")), blobovniczatree.WithRootPath(filepath.Join(root, "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(1), blobovniczatree.WithBlobovniczaShallowDepth(1),
blobovniczatree.WithBlobovniczaShallowWidth(1), blobovniczatree.WithBlobovniczaShallowWidth(1),

View file

@ -36,6 +36,7 @@ func Test_ObjectNotFoundIfNotDeletedFromMetabase(t *testing.T) {
blobstor.WithStorages([]blobstor.SubStorage{ blobstor.WithStorages([]blobstor.SubStorage{
{ {
Storage: blobovniczatree.NewBlobovniczaTree( Storage: blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithLogger(test.NewLogger(t)), blobovniczatree.WithLogger(test.NewLogger(t)),
blobovniczatree.WithRootPath(filepath.Join(rootPath, "blob", "blobovnicza")), blobovniczatree.WithRootPath(filepath.Join(rootPath, "blob", "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(1), blobovniczatree.WithBlobovniczaShallowDepth(1),

View file

@ -35,6 +35,7 @@ func TestShard_Lock(t *testing.T) {
blobstor.WithStorages([]blobstor.SubStorage{ blobstor.WithStorages([]blobstor.SubStorage{
{ {
Storage: blobovniczatree.NewBlobovniczaTree( Storage: blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithRootPath(filepath.Join(rootPath, "blob", "blobovnicza")), blobovniczatree.WithRootPath(filepath.Join(rootPath, "blob", "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(2), blobovniczatree.WithBlobovniczaShallowDepth(2),
blobovniczatree.WithBlobovniczaShallowWidth(2)), blobovniczatree.WithBlobovniczaShallowWidth(2)),

View file

@ -78,6 +78,7 @@ func testShardGetRange(t *testing.T, hasWriteCache bool) {
blobstor.WithStorages([]blobstor.SubStorage{ blobstor.WithStorages([]blobstor.SubStorage{
{ {
Storage: blobovniczatree.NewBlobovniczaTree( Storage: blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithLogger(test.NewLogger(t)), blobovniczatree.WithLogger(test.NewLogger(t)),
blobovniczatree.WithRootPath(filepath.Join(t.TempDir(), "blob", "blobovnicza")), blobovniczatree.WithRootPath(filepath.Join(t.TempDir(), "blob", "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(1), blobovniczatree.WithBlobovniczaShallowDepth(1),

View file

@ -59,6 +59,7 @@ func newCustomShard(t testing.TB, enableWriteCache bool, o shardOptions) *Shard
blobstor.WithStorages([]blobstor.SubStorage{ blobstor.WithStorages([]blobstor.SubStorage{
{ {
Storage: blobovniczatree.NewBlobovniczaTree( Storage: blobovniczatree.NewBlobovniczaTree(
context.Background(),
blobovniczatree.WithLogger(test.NewLogger(t)), blobovniczatree.WithLogger(test.NewLogger(t)),
blobovniczatree.WithRootPath(filepath.Join(o.rootPath, "blob", "blobovnicza")), blobovniczatree.WithRootPath(filepath.Join(o.rootPath, "blob", "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(1), blobovniczatree.WithBlobovniczaShallowDepth(1),