[#9999] config: Drop blobstor, metabase, writecache configs

Signed-off-by: Dmitrii Stepanov <d.stepanov@yadro.com>
This commit is contained in:
Dmitrii Stepanov 2025-02-24 18:10:58 +03:00
parent 02f3a7f65c
commit 82f64ed2bc
Signed by: dstepanov-yadro
GPG key ID: 237AF1A763293BC0
11 changed files with 7 additions and 905 deletions

View file

@ -9,8 +9,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-adm/internal/commonflags"
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
engineconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine"
shardconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard"
morphconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/morph"
nodeconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/node"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/container"
@ -109,12 +107,6 @@ func upgrade(cmd *cobra.Command, _ []string) error {
func getMetabasePaths(appCfg *config.Config) ([]string, error) {
var paths []string
if err := engineconfig.IterateShards(appCfg, false, func(sc *shardconfig.Config) error {
paths = append(paths, sc.Metabase().Path())
return nil
}); err != nil {
return nil, fmt.Errorf("get metabase paths: %w", err)
}
return paths, nil
}

View file

@ -8,7 +8,6 @@ import (
"net"
"os"
"os/signal"
"path/filepath"
"strings"
"sync"
"sync/atomic"
@ -21,8 +20,6 @@ import (
contractsconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/contracts"
engineconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine"
shardconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard"
blobovniczaconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/blobovnicza"
fstreeconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/fstree"
loggerconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/logger"
morphconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/morph"
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/multinet"
@ -37,16 +34,11 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/container"
frostfsidcore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/frostfsid"
netmapCore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/netmap"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/blobovniczatree"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/engine"
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
lsmetrics "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metrics"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/pilorama"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
shardmode "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client"
containerClient "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/container"
nmClient "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/netmap"
@ -71,7 +63,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
netmapV2 "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/api/netmap"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/netmap"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/user"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/version"
policy_engine "git.frostfs.info/TrueCloudLab/policy-engine/pkg/engine"
@ -79,7 +70,6 @@ import (
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
neogoutil "github.com/nspcc-dev/neo-go/pkg/util"
"github.com/panjf2000/ants/v2"
"go.etcd.io/bbolt"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"
"google.golang.org/grpc"
@ -129,20 +119,8 @@ type shardCfg struct {
estimateCompressibility bool
estimateCompressibilityThreshold float64
smallSizeObjectLimit uint64
uncompressableContentType []string
refillMetabase bool
refillMetabaseWorkersCount int
mode shardmode.Mode
metaCfg struct {
path string
perm fs.FileMode
maxBatchSize int
maxBatchDelay time.Duration
}
subStorages []subStorageCfg
uncompressableContentType []string
mode shardmode.Mode
gcCfg struct {
removerBatchSize int
@ -151,17 +129,6 @@ type shardCfg struct {
expiredCollectorWorkerCount int
}
writecacheCfg struct {
enabled bool
path string
maxObjSize uint64
flushWorkerCount int
sizeLimit uint64
countLimit uint64
noSync bool
flushSizeLimit uint64
}
piloramaCfg struct {
enabled bool
path string
@ -178,9 +145,6 @@ func (c *shardCfg) id() string {
// This calculation should be kept in sync with
// pkg/local_object_storage/engine/control.go file.
var sb strings.Builder
for i := range c.subStorages {
sb.WriteString(filepath.Clean(c.subStorages[i].path))
}
return sb.String()
}
@ -256,25 +220,14 @@ func (a *applicationConfiguration) readConfig(c *config.Config) error {
func (a *applicationConfiguration) updateShardConfig(c *config.Config, oldConfig *shardconfig.Config) error {
var newConfig shardCfg
newConfig.refillMetabase = oldConfig.RefillMetabase()
newConfig.refillMetabaseWorkersCount = oldConfig.RefillMetabaseWorkersCount()
newConfig.mode = oldConfig.Mode()
newConfig.compress = oldConfig.Compress()
newConfig.estimateCompressibility = oldConfig.EstimateCompressibility()
newConfig.estimateCompressibilityThreshold = oldConfig.EstimateCompressibilityThreshold()
newConfig.uncompressableContentType = oldConfig.UncompressableContentTypes()
newConfig.smallSizeObjectLimit = oldConfig.SmallSizeLimit()
a.setShardWriteCacheConfig(&newConfig, oldConfig)
a.setShardPiloramaConfig(c, &newConfig, oldConfig)
if err := a.setShardStorageConfig(&newConfig, oldConfig); err != nil {
return err
}
a.setMetabaseConfig(&newConfig, oldConfig)
a.setGCConfig(&newConfig, oldConfig)
a.EngineCfg.shards = append(a.EngineCfg.shards, newConfig)
@ -282,22 +235,6 @@ func (a *applicationConfiguration) updateShardConfig(c *config.Config, oldConfig
return nil
}
func (a *applicationConfiguration) setShardWriteCacheConfig(newConfig *shardCfg, oldConfig *shardconfig.Config) {
writeCacheCfg := oldConfig.WriteCache()
if writeCacheCfg.Enabled() {
wc := &newConfig.writecacheCfg
wc.enabled = true
wc.path = writeCacheCfg.Path()
wc.maxObjSize = writeCacheCfg.MaxObjectSize()
wc.flushWorkerCount = writeCacheCfg.WorkerCount()
wc.sizeLimit = writeCacheCfg.SizeLimit()
wc.countLimit = writeCacheCfg.CountLimit()
wc.noSync = writeCacheCfg.NoSync()
wc.flushSizeLimit = writeCacheCfg.MaxFlushingObjectsSize()
}
}
func (a *applicationConfiguration) setShardPiloramaConfig(c *config.Config, newConfig *shardCfg, oldConfig *shardconfig.Config) {
if config.BoolSafe(c.Sub("tree"), "enabled") {
piloramaCfg := oldConfig.Pilorama()
@ -312,55 +249,6 @@ func (a *applicationConfiguration) setShardPiloramaConfig(c *config.Config, newC
}
}
func (a *applicationConfiguration) setShardStorageConfig(newConfig *shardCfg, oldConfig *shardconfig.Config) error {
blobStorCfg := oldConfig.BlobStor()
storagesCfg := blobStorCfg.Storages()
ss := make([]subStorageCfg, 0, len(storagesCfg))
for i := range storagesCfg {
var sCfg subStorageCfg
sCfg.typ = storagesCfg[i].Type()
sCfg.path = storagesCfg[i].Path()
sCfg.perm = storagesCfg[i].Perm()
switch storagesCfg[i].Type() {
case blobovniczatree.Type:
sub := blobovniczaconfig.From((*config.Config)(storagesCfg[i]))
sCfg.size = sub.Size()
sCfg.depth = sub.ShallowDepth()
sCfg.width = sub.ShallowWidth()
sCfg.openedCacheSize = sub.OpenedCacheSize()
sCfg.openedCacheTTL = sub.OpenedCacheTTL()
sCfg.openedCacheExpInterval = sub.OpenedCacheExpInterval()
sCfg.initWorkerCount = sub.InitWorkerCount()
sCfg.rebuildDropTimeout = sub.RebuildDropTimeout()
case fstree.Type:
sub := fstreeconfig.From((*config.Config)(storagesCfg[i]))
sCfg.depth = sub.Depth()
sCfg.noSync = sub.NoSync()
default:
return fmt.Errorf("invalid storage type: %s", storagesCfg[i].Type())
}
ss = append(ss, sCfg)
}
newConfig.subStorages = ss
return nil
}
func (a *applicationConfiguration) setMetabaseConfig(newConfig *shardCfg, oldConfig *shardconfig.Config) {
metabaseCfg := oldConfig.Metabase()
m := &newConfig.metaCfg
m.path = metabaseCfg.Path()
m.perm = metabaseCfg.BoltDB().Perm()
m.maxBatchDelay = metabaseCfg.BoltDB().MaxBatchDelay()
m.maxBatchSize = metabaseCfg.BoltDB().MaxBatchSize()
}
func (a *applicationConfiguration) setGCConfig(newConfig *shardCfg, oldConfig *shardconfig.Config) {
gcCfg := oldConfig.GC()
newConfig.gcCfg.removerBatchSize = gcCfg.RemoverBatchSize()
@ -894,33 +782,16 @@ type shardOptsWithID struct {
shOpts []shard.Option
}
func (c *cfg) shardOpts(ctx context.Context) []shardOptsWithID {
func (c *cfg) shardOpts() []shardOptsWithID {
shards := make([]shardOptsWithID, 0, len(c.EngineCfg.shards))
for _, shCfg := range c.EngineCfg.shards {
shards = append(shards, c.getShardOpts(ctx, shCfg))
shards = append(shards, c.getShardOpts(shCfg))
}
return shards
}
func (c *cfg) getWriteCacheOpts(shCfg shardCfg) []writecache.Option {
var writeCacheOpts []writecache.Option
if wcRead := shCfg.writecacheCfg; wcRead.enabled {
writeCacheOpts = append(writeCacheOpts,
writecache.WithPath(wcRead.path),
writecache.WithFlushSizeLimit(wcRead.flushSizeLimit),
writecache.WithMaxObjectSize(wcRead.maxObjSize),
writecache.WithFlushWorkersCount(wcRead.flushWorkerCount),
writecache.WithMaxCacheSize(wcRead.sizeLimit),
writecache.WithMaxCacheCount(wcRead.countLimit),
writecache.WithNoSync(wcRead.noSync),
writecache.WithLogger(c.log),
)
}
return writeCacheOpts
}
func (c *cfg) getPiloramaOpts(shCfg shardCfg) []pilorama.Option {
var piloramaOpts []pilorama.Option
if prRead := shCfg.piloramaCfg; prRead.enabled {
@ -938,113 +809,15 @@ func (c *cfg) getPiloramaOpts(shCfg shardCfg) []pilorama.Option {
return piloramaOpts
}
func (c *cfg) getSubstorageOpts(ctx context.Context, shCfg shardCfg) []blobstor.SubStorage {
var ss []blobstor.SubStorage
for _, sRead := range shCfg.subStorages {
switch sRead.typ {
case blobovniczatree.Type:
blobTreeOpts := []blobovniczatree.Option{
blobovniczatree.WithRootPath(sRead.path),
blobovniczatree.WithPermissions(sRead.perm),
blobovniczatree.WithBlobovniczaSize(sRead.size),
blobovniczatree.WithBlobovniczaShallowDepth(sRead.depth),
blobovniczatree.WithBlobovniczaShallowWidth(sRead.width),
blobovniczatree.WithOpenedCacheSize(sRead.openedCacheSize),
blobovniczatree.WithOpenedCacheTTL(sRead.openedCacheTTL),
blobovniczatree.WithOpenedCacheExpInterval(sRead.openedCacheExpInterval),
blobovniczatree.WithInitWorkerCount(sRead.initWorkerCount),
blobovniczatree.WithWaitBeforeDropDB(sRead.rebuildDropTimeout),
blobovniczatree.WithLogger(c.log),
blobovniczatree.WithObjectSizeLimit(shCfg.smallSizeObjectLimit),
}
if c.metricsCollector != nil {
blobTreeOpts = append(blobTreeOpts,
blobovniczatree.WithMetrics(
lsmetrics.NewBlobovniczaTreeMetrics(sRead.path, c.metricsCollector.BlobobvnizcaTreeMetrics()),
),
)
}
ss = append(ss, blobstor.SubStorage{
Storage: blobovniczatree.NewBlobovniczaTree(ctx, blobTreeOpts...),
Policy: func(_ *objectSDK.Object, data []byte) bool {
return uint64(len(data)) < shCfg.smallSizeObjectLimit
},
})
case fstree.Type:
fstreeOpts := []fstree.Option{
fstree.WithPath(sRead.path),
fstree.WithPerm(sRead.perm),
fstree.WithDepth(sRead.depth),
fstree.WithNoSync(sRead.noSync),
fstree.WithLogger(c.log),
}
if c.metricsCollector != nil {
fstreeOpts = append(fstreeOpts,
fstree.WithMetrics(
lsmetrics.NewFSTreeMetricsWithoutShardID(sRead.path, c.metricsCollector.FSTree()),
),
)
}
ss = append(ss, blobstor.SubStorage{
Storage: fstree.New(fstreeOpts...),
Policy: func(_ *objectSDK.Object, _ []byte) bool {
return true
},
})
default:
// should never happen, that has already
// been handled: when the config was read
}
}
return ss
}
func (c *cfg) getShardOpts(ctx context.Context, shCfg shardCfg) shardOptsWithID {
writeCacheOpts := c.getWriteCacheOpts(shCfg)
func (c *cfg) getShardOpts(shCfg shardCfg) shardOptsWithID {
piloramaOpts := c.getPiloramaOpts(shCfg)
ss := c.getSubstorageOpts(ctx, shCfg)
blobstoreOpts := []blobstor.Option{
blobstor.WithCompressObjects(shCfg.compress),
blobstor.WithUncompressableContentTypes(shCfg.uncompressableContentType),
blobstor.WithCompressibilityEstimate(shCfg.estimateCompressibility),
blobstor.WithCompressibilityEstimateThreshold(shCfg.estimateCompressibilityThreshold),
blobstor.WithStorages(ss),
blobstor.WithLogger(c.log),
}
if c.metricsCollector != nil {
blobstoreOpts = append(blobstoreOpts, blobstor.WithMetrics(lsmetrics.NewBlobstoreMetrics(c.metricsCollector.Blobstore())))
}
mbOptions := []meta.Option{
meta.WithPath(shCfg.metaCfg.path),
meta.WithPermissions(shCfg.metaCfg.perm),
meta.WithMaxBatchSize(shCfg.metaCfg.maxBatchSize),
meta.WithMaxBatchDelay(shCfg.metaCfg.maxBatchDelay),
meta.WithBoltDBOptions(&bbolt.Options{
Timeout: 100 * time.Millisecond,
}),
meta.WithLogger(c.log),
meta.WithEpochState(c.cfgNetmap.state),
}
if c.metricsCollector != nil {
mbOptions = append(mbOptions, meta.WithMetrics(lsmetrics.NewMetabaseMetrics(shCfg.metaCfg.path, c.metricsCollector.MetabaseMetrics())))
}
var sh shardOptsWithID
sh.configID = shCfg.id()
sh.shOpts = []shard.Option{
shard.WithLogger(c.log),
shard.WithRefillMetabase(shCfg.refillMetabase),
shard.WithRefillMetabaseWorkersCount(shCfg.refillMetabaseWorkersCount),
shard.WithMode(shCfg.mode),
shard.WithBlobStorOptions(blobstoreOpts...),
shard.WithMetaBaseOptions(mbOptions...),
shard.WithPiloramaOptions(piloramaOpts...),
shard.WithWriteCache(shCfg.writecacheCfg.enabled),
shard.WithWriteCacheOptions(writeCacheOpts),
shard.WithRemoverBatchSize(shCfg.gcCfg.removerBatchSize),
shard.WithGCRemoverSleepInterval(shCfg.gcCfg.removerSleepInterval),
shard.WithExpiredCollectorBatchSize(shCfg.gcCfg.expiredCollectorBatchSize),
@ -1097,7 +870,7 @@ func initLocalStorage(ctx context.Context, c *cfg) {
c.cfgObject.getSvc = new(getsvc.Service)
var shardsAttached int
for _, optsWithMeta := range c.shardOpts(ctx) {
for _, optsWithMeta := range c.shardOpts() {
id, err := ls.AddShard(ctx, append(optsWithMeta.shOpts,
shard.WithTombstoneSource(c.createTombstoneSource()),
shard.WithContainerInfoProvider(c.createContainerInfoProvider(ctx)))...)
@ -1344,7 +1117,7 @@ func (c *cfg) reloadConfig(ctx context.Context) {
// Storage Engine
var rcfg engine.ReConfiguration
for _, optsWithID := range c.shardOpts(ctx) {
for _, optsWithID := range c.shardOpts() {
rcfg.AddShard(optsWithID.configID, append(optsWithID.shOpts,
shard.WithTombstoneSource(c.createTombstoneSource()),
shard.WithContainerInfoProvider(c.createContainerInfoProvider(ctx)),

View file

@ -8,11 +8,8 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
engineconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine"
shardconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard"
blobovniczaconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/blobovnicza"
fstreeconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/fstree"
gcconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/gc"
piloramaconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/pilorama"
writecacheconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/writecache"
configtest "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/test"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"github.com/stretchr/testify/require"
@ -20,13 +17,10 @@ import (
func TestIterateShards(t *testing.T) {
fileConfigTest := func(c *config.Config) {
var res []string
require.NoError(t,
engineconfig.IterateShards(c, false, func(sc *shardconfig.Config) error {
res = append(res, sc.Metabase().Path())
return nil
}))
require.Equal(t, []string{"abc", "xyz"}, res)
}
const cfgDir = "./testdata/shards"
@ -70,10 +64,6 @@ func TestEngineSection(t *testing.T) {
num++
}()
wc := sc.WriteCache()
meta := sc.Metabase()
blob := sc.BlobStor()
ss := blob.Storages()
pl := sc.Pilorama()
gc := sc.GC()
@ -85,55 +75,18 @@ func TestEngineSection(t *testing.T) {
require.Equal(t, pl.MaxBatchDelay(), 10*time.Millisecond)
require.Equal(t, pl.MaxBatchSize(), 200)
require.Equal(t, false, wc.Enabled())
require.Equal(t, true, wc.NoSync())
require.Equal(t, "tmp/0/cache", wc.Path())
require.EqualValues(t, 134217728, wc.MaxObjectSize())
require.EqualValues(t, 30, wc.WorkerCount())
require.EqualValues(t, 3221225472, wc.SizeLimit())
require.EqualValues(t, 49, wc.CountLimit())
require.EqualValues(t, uint64(100), wc.MaxFlushingObjectsSize())
require.Equal(t, "tmp/0/meta", meta.Path())
require.Equal(t, fs.FileMode(0o644), meta.BoltDB().Perm())
require.Equal(t, 100, meta.BoltDB().MaxBatchSize())
require.Equal(t, 10*time.Millisecond, meta.BoltDB().MaxBatchDelay())
require.Equal(t, true, sc.Compress())
require.Equal(t, []string{"audio/*", "video/*"}, sc.UncompressableContentTypes())
require.Equal(t, true, sc.EstimateCompressibility())
require.Equal(t, float64(0.7), sc.EstimateCompressibilityThreshold())
require.EqualValues(t, 102400, sc.SmallSizeLimit())
require.Equal(t, 2, len(ss))
blz := blobovniczaconfig.From((*config.Config)(ss[0]))
require.Equal(t, "tmp/0/blob/blobovnicza", ss[0].Path())
require.EqualValues(t, 0o644, blz.BoltDB().Perm())
require.EqualValues(t, 4194304, blz.Size())
require.EqualValues(t, 1, blz.ShallowDepth())
require.EqualValues(t, 4, blz.ShallowWidth())
require.EqualValues(t, 50, blz.OpenedCacheSize())
require.EqualValues(t, time.Minute, blz.OpenedCacheTTL())
require.EqualValues(t, 30*time.Second, blz.OpenedCacheExpInterval())
require.EqualValues(t, 10, blz.InitWorkerCount())
require.EqualValues(t, 30*time.Second, blz.RebuildDropTimeout())
require.Equal(t, "tmp/0/blob", ss[1].Path())
require.EqualValues(t, 0o644, ss[1].Perm())
fst := fstreeconfig.From((*config.Config)(ss[1]))
require.EqualValues(t, 5, fst.Depth())
require.Equal(t, false, fst.NoSync())
require.EqualValues(t, 150, gc.RemoverBatchSize())
require.Equal(t, 2*time.Minute, gc.RemoverSleepInterval())
require.Equal(t, 1500, gc.ExpiredCollectorBatchSize())
require.Equal(t, 15, gc.ExpiredCollectorWorkerCount())
require.Equal(t, false, sc.RefillMetabase())
require.Equal(t, mode.ReadOnly, sc.Mode())
require.Equal(t, 100, sc.RefillMetabaseWorkersCount())
case 1:
require.Equal(t, "tmp/1/blob/pilorama.db", pl.Path())
require.Equal(t, fs.FileMode(0o644), pl.Perm())
@ -141,53 +94,16 @@ func TestEngineSection(t *testing.T) {
require.Equal(t, 5*time.Millisecond, pl.MaxBatchDelay())
require.Equal(t, 100, pl.MaxBatchSize())
require.Equal(t, true, wc.Enabled())
require.Equal(t, false, wc.NoSync())
require.Equal(t, "tmp/1/cache", wc.Path())
require.EqualValues(t, 134217728, wc.MaxObjectSize())
require.EqualValues(t, 30, wc.WorkerCount())
require.EqualValues(t, 4294967296, wc.SizeLimit())
require.EqualValues(t, writecacheconfig.CountLimitDefault, wc.CountLimit())
require.EqualValues(t, writecacheconfig.MaxFlushingObjectsSizeDefault, wc.MaxFlushingObjectsSize())
require.Equal(t, "tmp/1/meta", meta.Path())
require.Equal(t, fs.FileMode(0o644), meta.BoltDB().Perm())
require.Equal(t, 200, meta.BoltDB().MaxBatchSize())
require.Equal(t, 20*time.Millisecond, meta.BoltDB().MaxBatchDelay())
require.Equal(t, false, sc.Compress())
require.Equal(t, []string(nil), sc.UncompressableContentTypes())
require.EqualValues(t, 102400, sc.SmallSizeLimit())
require.Equal(t, 2, len(ss))
blz := blobovniczaconfig.From((*config.Config)(ss[0]))
require.Equal(t, "tmp/1/blob/blobovnicza", ss[0].Path())
require.EqualValues(t, 4194304, blz.Size())
require.EqualValues(t, 1, blz.ShallowDepth())
require.EqualValues(t, 4, blz.ShallowWidth())
require.EqualValues(t, 50, blz.OpenedCacheSize())
require.EqualValues(t, 5*time.Minute, blz.OpenedCacheTTL())
require.EqualValues(t, 15*time.Second, blz.OpenedCacheExpInterval())
require.EqualValues(t, blobovniczaconfig.InitWorkerCountDefault, blz.InitWorkerCount())
require.EqualValues(t, blobovniczaconfig.RebuildDropTimeoutDefault, blz.RebuildDropTimeout())
require.Equal(t, "tmp/1/blob", ss[1].Path())
require.EqualValues(t, 0o644, ss[1].Perm())
fst := fstreeconfig.From((*config.Config)(ss[1]))
require.EqualValues(t, 5, fst.Depth())
require.Equal(t, true, fst.NoSync())
require.EqualValues(t, 200, gc.RemoverBatchSize())
require.Equal(t, 5*time.Minute, gc.RemoverSleepInterval())
require.Equal(t, gcconfig.ExpiredCollectorBatchSizeDefault, gc.ExpiredCollectorBatchSize())
require.Equal(t, gcconfig.ExpiredCollectorWorkersCountDefault, gc.ExpiredCollectorWorkerCount())
require.Equal(t, true, sc.RefillMetabase())
require.Equal(t, mode.ReadWrite, sc.Mode())
require.Equal(t, shardconfig.RefillMetabaseWorkersCountDefault, sc.RefillMetabaseWorkersCount())
}
return nil
})

View file

@ -1,180 +0,0 @@
package blobovniczaconfig
import (
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
boltdbconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/boltdb"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/blobovniczatree"
)
// Config is a wrapper over the config section
// which provides access to Blobovnicza configurations.
type Config config.Config
const (
// SizeDefault is a default limit of estimates of Blobovnicza size.
SizeDefault = 1 << 30
// ShallowDepthDefault is a default shallow dir depth.
ShallowDepthDefault = 2
// ShallowWidthDefault is a default shallow dir width.
ShallowWidthDefault = 16
// OpenedCacheSizeDefault is a default cache size of opened Blobovnicza's.
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 = 5
// RebuildDropTimeoutDefault is a default timeout value to wait before drop single blobovnicza.
RebuildDropTimeoutDefault = 10 * time.Second
)
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Type returns the storage type.
func (x *Config) Type() string {
return blobovniczatree.Type
}
// Size returns the value of "size" config parameter.
//
// Returns SizeDefault if the value is not a positive number.
func (x *Config) Size() uint64 {
s := config.SizeInBytesSafe(
(*config.Config)(x),
"size",
)
if s > 0 {
return s
}
return SizeDefault
}
// ShallowDepth returns the value of "depth" config parameter.
//
// Returns ShallowDepthDefault if the value is not a positive number.
func (x *Config) ShallowDepth() uint64 {
d := config.UintSafe(
(*config.Config)(x),
"depth",
)
if d > 0 {
return d
}
return ShallowDepthDefault
}
// ShallowWidth returns the value of "width" config parameter.
//
// Returns ShallowWidthDefault if the value is not a positive number.
func (x *Config) ShallowWidth() uint64 {
d := config.UintSafe(
(*config.Config)(x),
"width",
)
if d > 0 {
return d
}
return ShallowWidthDefault
}
// OpenedCacheSize returns the value of "opened_cache_capacity" config parameter.
//
// Returns OpenedCacheSizeDefault if the value is not a positive number.
func (x *Config) OpenedCacheSize() int {
d := config.IntSafe(
(*config.Config)(x),
"opened_cache_capacity",
)
if d > 0 {
return int(d)
}
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.
func (x *Config) BoltDB() *boltdbconfig.Config {
return (*boltdbconfig.Config)(x)
}
// InitWorkerCount returns the value of "init_worker_count" config parameter.
//
// Returns InitWorkerCountDefault if the value is not a positive number.
func (x *Config) InitWorkerCount() int {
d := config.IntSafe(
(*config.Config)(x),
"init_worker_count",
)
if d > 0 {
return int(d)
}
return InitWorkerCountDefault
}
// RebuildDropTimeout returns the value of "rebuild_drop_timeout" config parameter.
//
// Returns RebuildDropTimeoutDefault if the value is not defined or invalid.
func (x *Config) RebuildDropTimeout() time.Duration {
d := config.DurationSafe(
(*config.Config)(x),
"rebuild_drop_timeout",
)
if d > 0 {
return d
}
return RebuildDropTimeoutDefault
}

View file

@ -1,33 +0,0 @@
package blobstorconfig
import (
"strconv"
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor/storage"
)
// Config is a wrapper over the config section
// which provides access to BlobStor configurations.
type Config config.Config
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Storages returns the value of storage subcomponents.
func (x *Config) Storages() []*storage.Config {
var ss []*storage.Config
for i := 0; ; i++ {
typ := config.String(
(*config.Config)(x),
strconv.Itoa(i)+".type")
if typ == "" {
return ss
}
sub := storage.From((*config.Config)(x).Sub(strconv.Itoa(i)))
ss = append(ss, sub)
}
}

View file

@ -1,47 +0,0 @@
package fstree
import (
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
)
// Config is a wrapper over the config section
// which provides access to Blobovnicza configurations.
type Config config.Config
// DepthDefault is a default shallow dir depth.
const DepthDefault = 4
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Type returns the storage type.
func (x *Config) Type() string {
return fstree.Type
}
// Depth returns the value of "depth" config parameter.
//
// Returns DepthDefault if the value is out of
// [1:fstree.MaxDepth] range.
func (x *Config) Depth() uint64 {
d := config.UintSafe(
(*config.Config)(x),
"depth",
)
if d >= 1 && d <= fstree.MaxDepth {
return d
}
return DepthDefault
}
// NoSync returns the value of "no_sync" config parameter.
//
// Returns false if the value is not a boolean or is missing.
func (x *Config) NoSync() bool {
return config.BoolSafe((*config.Config)(x), "no_sync")
}

View file

@ -1,55 +0,0 @@
package storage
import (
"io/fs"
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
type Config config.Config
// PermDefault are default permission bits for BlobStor data.
const PermDefault = 0o660
func From(x *config.Config) *Config {
return (*Config)(x)
}
// Type returns storage type.
func (x *Config) Type() string {
return config.String(
(*config.Config)(x),
"type")
}
// Path returns the value of "path" config parameter.
//
// Panics if the value is not a non-empty string.
func (x *Config) Path() string {
p := config.String(
(*config.Config)(x),
"path",
)
if p == "" {
panic("blobstor path not set")
}
return p
}
// Perm returns the value of "perm" config parameter as a fs.FileMode.
//
// Returns PermDefault if the value is not a non-zero number.
func (x *Config) Perm() fs.FileMode {
p := config.UintSafe(
(*config.Config)(x),
"perm",
)
if p == 0 {
p = PermDefault
}
return fs.FileMode(p)
}

View file

@ -2,11 +2,8 @@ package shardconfig
import (
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
blobstorconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/blobstor"
gcconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/gc"
metabaseconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/metabase"
piloramaconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/pilorama"
writecacheconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
)
@ -85,30 +82,6 @@ func (x *Config) SmallSizeLimit() uint64 {
return SmallSizeLimitDefault
}
// BlobStor returns "blobstor" subsection as a blobstorconfig.Config.
func (x *Config) BlobStor() *blobstorconfig.Config {
return blobstorconfig.From(
(*config.Config)(x).
Sub("blobstor"),
)
}
// Metabase returns "metabase" subsection as a metabaseconfig.Config.
func (x *Config) Metabase() *metabaseconfig.Config {
return metabaseconfig.From(
(*config.Config)(x).
Sub("metabase"),
)
}
// WriteCache returns "writecache" subsection as a writecacheconfig.Config.
func (x *Config) WriteCache() *writecacheconfig.Config {
return writecacheconfig.From(
(*config.Config)(x).
Sub("writecache"),
)
}
// Pilorama returns "pilorama" subsection as a piloramaconfig.Config.
func (x *Config) Pilorama() *piloramaconfig.Config {
return piloramaconfig.From(
@ -125,30 +98,6 @@ func (x *Config) GC() *gcconfig.Config {
)
}
// RefillMetabase returns the value of "resync_metabase" config parameter.
//
// Returns false if the value is not a valid bool.
func (x *Config) RefillMetabase() bool {
return config.BoolSafe(
(*config.Config)(x),
"resync_metabase",
)
}
// RefillMetabaseWorkersCount returns the value of "resync_metabase_worker_count" config parameter.
//
// Returns RefillMetabaseWorkersCountDefault if the value is not a positive number.
func (x *Config) RefillMetabaseWorkersCount() int {
v := config.IntSafe(
(*config.Config)(x),
"resync_metabase_worker_count",
)
if v > 0 {
return int(v)
}
return RefillMetabaseWorkersCountDefault
}
// Mode return the value of "mode" config parameter.
//
// Panics if read the value is not one of predefined

View file

@ -1,36 +0,0 @@
package metabaseconfig
import (
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
boltdbconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard/boltdb"
)
// Config is a wrapper over the config section
// which provides access to Metabase configurations.
type Config config.Config
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Path returns the value of "path" config parameter.
//
// Panics if the value is not a non-empty string.
func (x *Config) Path() string {
p := config.String(
(*config.Config)(x),
"path",
)
if p == "" {
panic("metabase path not set")
}
return p
}
// BoltDB returns config instance for querying bolt db specific parameters.
func (x *Config) BoltDB() *boltdbconfig.Config {
return (*boltdbconfig.Config)(x)
}

View file

@ -1,140 +0,0 @@
package writecacheconfig
import (
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config"
)
// Config is a wrapper over the config section
// which provides access to WriteCache configurations.
type Config config.Config
const (
// MaxSizeDefault is a default value of the object payload size limit.
MaxSizeDefault = 64 << 20
// WorkersNumberDefault is a default number of workers.
WorkersNumberDefault = 20
// SizeLimitDefault is a default write-cache size limit.
SizeLimitDefault = 1 << 30
// CountLimitDefault is a default write-cache count limit.
CountLimitDefault = 0
MaxFlushingObjectsSizeDefault = 128 << 20
)
// From wraps config section into Config.
func From(c *config.Config) *Config {
return (*Config)(c)
}
// Enabled returns true if write-cache is enabled and false otherwise.
//
// Panics if the value is not a boolean.
func (x *Config) Enabled() bool {
return config.Bool((*config.Config)(x), "enabled")
}
// Path returns the value of "path" config parameter.
//
// Panics if the value is not a non-empty string.
func (x *Config) Path() string {
p := config.String(
(*config.Config)(x),
"path",
)
if p == "" {
panic("write cache path not set")
}
return p
}
// MaxObjectSize returns the value of "max_object_size" config parameter.
//
// Returns MaxSizeDefault if the value is not a positive number.
func (x *Config) MaxObjectSize() uint64 {
s := config.SizeInBytesSafe(
(*config.Config)(x),
"max_object_size",
)
if s > 0 {
return s
}
return MaxSizeDefault
}
// WorkerCount returns the value of "flush_worker_count" config parameter.
//
// Returns WorkersNumberDefault if the value is not a positive number.
func (x *Config) WorkerCount() int {
c := config.IntSafe(
(*config.Config)(x),
"flush_worker_count",
)
if c > 0 {
return int(c)
}
return WorkersNumberDefault
}
// SizeLimit returns the value of "capacity" config parameter.
//
// Returns SizeLimitDefault if the value is not a positive number.
func (x *Config) SizeLimit() uint64 {
c := config.SizeInBytesSafe(
(*config.Config)(x),
"capacity",
)
if c > 0 {
return c
}
return SizeLimitDefault
}
// CountLimit returns the value of "max_object_count" config parameter.
//
// Returns CountLimitDefault if the value is not a positive number.
func (x *Config) CountLimit() uint64 {
c := config.SizeInBytesSafe(
(*config.Config)(x),
"max_object_count",
)
if c > 0 {
return c
}
return CountLimitDefault
}
// NoSync returns the value of "no_sync" config parameter.
//
// Returns false if the value is not a boolean.
func (x *Config) NoSync() bool {
return config.BoolSafe((*config.Config)(x), "no_sync")
}
// MaxFlushingObjectsSize returns the value of "max_flushing_objects_size" config parameter.
//
// Returns MaxFlushingObjectsSizeDefault if the value is not a positive number.
func (x *Config) MaxFlushingObjectsSize() uint64 {
s := config.SizeInBytesSafe(
(*config.Config)(x),
"max_flushing_objects_size",
)
if s > 0 {
return s
}
return MaxFlushingObjectsSizeDefault
}

View file

@ -9,8 +9,6 @@ import (
shardconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/engine/shard"
loggerconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/logger"
treeconfig "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config/tree"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/blobovniczatree"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
)
@ -35,17 +33,6 @@ func validateConfig(c *config.Config) error {
shardNum := 0
paths := make(map[string]pathDescription)
return engineconfig.IterateShards(c, false, func(sc *shardconfig.Config) error {
if sc.WriteCache().Enabled() {
err := addPath(paths, "writecache", shardNum, sc.WriteCache().Path())
if err != nil {
return err
}
}
if err := addPath(paths, "metabase", shardNum, sc.Metabase().Path()); err != nil {
return err
}
treeConfig := treeconfig.Tree(c)
if treeConfig.Enabled() {
err := addPath(paths, "pilorama", shardNum, sc.Pilorama().Path())
@ -54,30 +41,6 @@ func validateConfig(c *config.Config) error {
}
}
blobstor := sc.BlobStor().Storages()
if len(blobstor) != 2 {
return fmt.Errorf("blobstor section must have 2 components, got: %d", len(blobstor))
}
for i := range blobstor {
switch blobstor[i].Type() {
case fstree.Type, blobovniczatree.Type:
default:
return fmt.Errorf("unexpected storage type: %s (shard %d)", blobstor[i].Type(), shardNum)
}
if blobstor[i].Perm()&0o600 != 0o600 {
return fmt.Errorf("invalid permissions for blobstor component: %s, "+
"expected at least rw- for the owner (shard %d)",
blobstor[i].Perm(), shardNum)
}
if blobstor[i].Path() == "" {
return fmt.Errorf("blobstor component path is empty (shard %d)", shardNum)
}
err := addPath(paths, fmt.Sprintf("blobstor[%d]", i), shardNum, blobstor[i].Path())
if err != nil {
return err
}
}
shardNum++
return nil
})