forked from TrueCloudLab/frostfs-node
[#9999] config: Drop blobstor, metabase, writecache configs
Signed-off-by: Dmitrii Stepanov <d.stepanov@yadro.com>
This commit is contained in:
parent
02f3a7f65c
commit
82f64ed2bc
11 changed files with 7 additions and 905 deletions
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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)),
|
||||
|
|
|
@ -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
|
||||
})
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
|
@ -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")
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
})
|
||||
|
|
Loading…
Add table
Reference in a new issue