[#1367] writecache: Drop BBolt related config variables

Signed-off-by: Dmitrii Stepanov <d.stepanov@yadro.com>
This commit is contained in:
Dmitrii Stepanov 2024-09-10 13:12:17 +03:00
parent e39378b1c3
commit 25d2ae8aaf
12 changed files with 17 additions and 118 deletions

View file

@ -25,7 +25,7 @@ func init() {
func inspectFunc(cmd *cobra.Command, _ []string) { func inspectFunc(cmd *cobra.Command, _ []string) {
var data []byte var data []byte
db, err := writecache.OpenDB(vPath, true, os.OpenFile, 0) db, err := writecache.OpenDB(vPath, true, os.OpenFile)
common.ExitOnErr(cmd, common.Errf("could not open write-cache db: %w", err)) common.ExitOnErr(cmd, common.Errf("could not open write-cache db: %w", err))
defer db.Close() defer db.Close()

View file

@ -31,7 +31,7 @@ func listFunc(cmd *cobra.Command, _ []string) {
return err return err
} }
db, err := writecache.OpenDB(vPath, true, os.OpenFile, 0) db, err := writecache.OpenDB(vPath, true, os.OpenFile)
common.ExitOnErr(cmd, common.Errf("could not open write-cache db: %w", err)) common.ExitOnErr(cmd, common.Errf("could not open write-cache db: %w", err))
defer db.Close() defer db.Close()

View file

@ -145,15 +145,11 @@ type shardCfg struct {
writecacheCfg struct { writecacheCfg struct {
enabled bool enabled bool
path string path string
maxBatchSize int
maxBatchDelay time.Duration
smallObjectSize uint64
maxObjSize uint64 maxObjSize uint64
flushWorkerCount int flushWorkerCount int
sizeLimit uint64 sizeLimit uint64
countLimit uint64 countLimit uint64
noSync bool noSync bool
pageSize int
flushSizeLimit uint64 flushSizeLimit uint64
} }
@ -270,11 +266,7 @@ func (a *applicationConfiguration) setShardWriteCacheConfig(newConfig *shardCfg,
wc.enabled = true wc.enabled = true
wc.path = writeCacheCfg.Path() wc.path = writeCacheCfg.Path()
wc.maxBatchSize = writeCacheCfg.BoltDB().MaxBatchSize()
wc.maxBatchDelay = writeCacheCfg.BoltDB().MaxBatchDelay()
wc.pageSize = writeCacheCfg.BoltDB().PageSize()
wc.maxObjSize = writeCacheCfg.MaxObjectSize() wc.maxObjSize = writeCacheCfg.MaxObjectSize()
wc.smallObjectSize = writeCacheCfg.SmallObjectSize()
wc.flushWorkerCount = writeCacheCfg.WorkerCount() wc.flushWorkerCount = writeCacheCfg.WorkerCount()
wc.sizeLimit = writeCacheCfg.SizeLimit() wc.sizeLimit = writeCacheCfg.SizeLimit()
wc.countLimit = writeCacheCfg.CountLimit() wc.countLimit = writeCacheCfg.CountLimit()
@ -864,12 +856,8 @@ func (c *cfg) getWriteCacheOpts(shCfg shardCfg) []writecache.Option {
if wcRead := shCfg.writecacheCfg; wcRead.enabled { if wcRead := shCfg.writecacheCfg; wcRead.enabled {
writeCacheOpts = append(writeCacheOpts, writeCacheOpts = append(writeCacheOpts,
writecache.WithPath(wcRead.path), writecache.WithPath(wcRead.path),
writecache.WithMaxBatchSize(wcRead.maxBatchSize),
writecache.WithMaxBatchDelay(wcRead.maxBatchDelay),
writecache.WithPageSize(wcRead.pageSize),
writecache.WithFlushSizeLimit(wcRead.flushSizeLimit), writecache.WithFlushSizeLimit(wcRead.flushSizeLimit),
writecache.WithMaxObjectSize(wcRead.maxObjSize), writecache.WithMaxObjectSize(wcRead.maxObjSize),
writecache.WithSmallObjectSize(wcRead.smallObjectSize),
writecache.WithFlushWorkersCount(wcRead.flushWorkerCount), writecache.WithFlushWorkersCount(wcRead.flushWorkerCount),
writecache.WithMaxCacheSize(wcRead.sizeLimit), writecache.WithMaxCacheSize(wcRead.sizeLimit),
writecache.WithMaxCacheCount(wcRead.countLimit), writecache.WithMaxCacheCount(wcRead.countLimit),

View file

@ -73,11 +73,9 @@ func TestEngineSection(t *testing.T) {
require.Equal(t, true, wc.NoSync()) require.Equal(t, true, wc.NoSync())
require.Equal(t, "tmp/0/cache", wc.Path()) require.Equal(t, "tmp/0/cache", wc.Path())
require.EqualValues(t, 16384, wc.SmallObjectSize())
require.EqualValues(t, 134217728, wc.MaxObjectSize()) require.EqualValues(t, 134217728, wc.MaxObjectSize())
require.EqualValues(t, 30, wc.WorkerCount()) require.EqualValues(t, 30, wc.WorkerCount())
require.EqualValues(t, 3221225472, wc.SizeLimit()) require.EqualValues(t, 3221225472, wc.SizeLimit())
require.EqualValues(t, 4096, wc.BoltDB().PageSize())
require.EqualValues(t, 49, wc.CountLimit()) require.EqualValues(t, 49, wc.CountLimit())
require.EqualValues(t, uint64(100), wc.MaxFlushingObjectsSize()) require.EqualValues(t, uint64(100), wc.MaxFlushingObjectsSize())
@ -131,11 +129,9 @@ func TestEngineSection(t *testing.T) {
require.Equal(t, false, wc.NoSync()) require.Equal(t, false, wc.NoSync())
require.Equal(t, "tmp/1/cache", wc.Path()) require.Equal(t, "tmp/1/cache", wc.Path())
require.EqualValues(t, 16384, wc.SmallObjectSize())
require.EqualValues(t, 134217728, wc.MaxObjectSize()) require.EqualValues(t, 134217728, wc.MaxObjectSize())
require.EqualValues(t, 30, wc.WorkerCount()) require.EqualValues(t, 30, wc.WorkerCount())
require.EqualValues(t, 4294967296, wc.SizeLimit()) require.EqualValues(t, 4294967296, wc.SizeLimit())
require.EqualValues(t, 0, wc.BoltDB().PageSize())
require.EqualValues(t, writecacheconfig.CountLimitDefault, wc.CountLimit()) require.EqualValues(t, writecacheconfig.CountLimitDefault, wc.CountLimit())
require.EqualValues(t, writecacheconfig.MaxFlushingObjectsSizeDefault, wc.MaxFlushingObjectsSize()) require.EqualValues(t, writecacheconfig.MaxFlushingObjectsSizeDefault, wc.MaxFlushingObjectsSize())

View file

@ -2,7 +2,6 @@ package writecacheconfig
import ( import (
"git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-node/config" "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 // Config is a wrapper over the config section
@ -10,9 +9,6 @@ import (
type Config config.Config type Config config.Config
const ( const (
// SmallSizeDefault is a default size of small objects.
SmallSizeDefault = 32 << 10
// MaxSizeDefault is a default value of the object payload size limit. // MaxSizeDefault is a default value of the object payload size limit.
MaxSizeDefault = 64 << 20 MaxSizeDefault = 64 << 20
@ -56,22 +52,6 @@ func (x *Config) Path() string {
return p return p
} }
// SmallObjectSize returns the value of "small_object_size" config parameter.
//
// Returns SmallSizeDefault if the value is not a positive number.
func (x *Config) SmallObjectSize() uint64 {
s := config.SizeInBytesSafe(
(*config.Config)(x),
"small_object_size",
)
if s > 0 {
return s
}
return SmallSizeDefault
}
// MaxObjectSize returns the value of "max_object_size" config parameter. // MaxObjectSize returns the value of "max_object_size" config parameter.
// //
// Returns MaxSizeDefault if the value is not a positive number. // Returns MaxSizeDefault if the value is not a positive number.
@ -143,11 +123,6 @@ func (x *Config) NoSync() bool {
return config.BoolSafe((*config.Config)(x), "no_sync") return config.BoolSafe((*config.Config)(x), "no_sync")
} }
// BoltDB returns config instance for querying bolt db specific parameters.
func (x *Config) BoltDB() *boltdbconfig.Config {
return (*boltdbconfig.Config)(x)
}
// MaxFlushingObjectsSize returns the value of "max_flushing_objects_size" config parameter. // MaxFlushingObjectsSize returns the value of "max_flushing_objects_size" config parameter.
// //
// Returns MaxFlushingObjectsSizeDefault if the value is not a positive number. // Returns MaxFlushingObjectsSizeDefault if the value is not a positive number.

View file

@ -287,10 +287,8 @@ writecache:
enabled: true enabled: true
path: /path/to/writecache path: /path/to/writecache
capacity: 4294967296 capacity: 4294967296
small_object_size: 16384
max_object_size: 134217728 max_object_size: 134217728
flush_worker_count: 30 flush_worker_count: 30
page_size: '4k'
``` ```
| Parameter | Type | Default value | Description | | Parameter | Type | Default value | Description |
@ -298,13 +296,9 @@ writecache:
| `path` | `string` | | Path to the metabase file. | | `path` | `string` | | Path to the metabase file. |
| `capacity` | `size` | `1G` | Approximate maximum size of the writecache. If the writecache is full, objects are written to the blobstor directly. | | `capacity` | `size` | `1G` | Approximate maximum size of the writecache. If the writecache is full, objects are written to the blobstor directly. |
| `max_object_count` | `int` | unrestricted | Approximate maximum objects count in the writecache. If the writecache is full, objects are written to the blobstor directly. | | `max_object_count` | `int` | unrestricted | Approximate maximum objects count in the writecache. If the writecache is full, objects are written to the blobstor directly. |
| `small_object_size` | `size` | `32K` | Maximum object size for "small" objects. This objects are stored in a key-value database instead of a file-system. |
| `max_object_size` | `size` | `64M` | Maximum object size allowed to be stored in the writecache. | | `max_object_size` | `size` | `64M` | Maximum object size allowed to be stored in the writecache. |
| `flush_worker_count` | `int` | `20` | Amount of background workers that move data from the writecache to the blobstor. | | `flush_worker_count` | `int` | `20` | Amount of background workers that move data from the writecache to the blobstor. |
| `max_flushing_objects_size` | `size` | `512M` | Max total size of background flushing objects. | | `max_flushing_objects_size` | `size` | `512M` | Max total size of background flushing objects. |
| `max_batch_size` | `int` | `1000` | Maximum amount of small object `PUT` operations to perform in a single transaction. |
| `max_batch_delay` | `duration` | `10ms` | Maximum delay before a batch starts. |
| `page_size` | `size` | `0` | Page size overrides the default OS page size for small objects storage. Does not affect the existing storage. |
# `node` section # `node` section

View file

@ -118,6 +118,5 @@ func newCache(b *testing.B) writecache.Cache {
writecache.WithBlobstor(bs), writecache.WithBlobstor(bs),
writecache.WithMetabase(testMetabase{}), writecache.WithMetabase(testMetabase{}),
writecache.WithMaxCacheSize(256<<30), writecache.WithMaxCacheSize(256<<30),
writecache.WithSmallObjectSize(128<<10),
) )
} }

View file

@ -11,7 +11,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id" oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.etcd.io/bbolt"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -43,7 +42,6 @@ type objectInfo struct {
const ( const (
defaultMaxObjectSize = 64 * 1024 * 1024 // 64 MiB defaultMaxObjectSize = 64 * 1024 * 1024 // 64 MiB
defaultSmallObjectSize = 32 * 1024 // 32 KiB
defaultMaxCacheSize = 1 << 30 // 1 GiB defaultMaxCacheSize = 1 << 30 // 1 GiB
) )
@ -62,11 +60,8 @@ func New(opts ...Option) Cache {
options: options{ options: options{
log: &logger.Logger{Logger: zap.NewNop()}, log: &logger.Logger{Logger: zap.NewNop()},
maxObjectSize: defaultMaxObjectSize, maxObjectSize: defaultMaxObjectSize,
smallObjectSize: defaultSmallObjectSize,
workersCount: defaultFlushWorkersCount, workersCount: defaultFlushWorkersCount,
maxCacheSize: defaultMaxCacheSize, maxCacheSize: defaultMaxCacheSize,
maxBatchSize: bbolt.DefaultMaxBatchSize,
maxBatchDelay: bbolt.DefaultMaxBatchDelay,
metrics: DefaultMetrics(), metrics: DefaultMetrics(),
flushSizeLimit: defaultFlushWorkersCount * defaultMaxObjectSize, flushSizeLimit: defaultFlushWorkersCount * defaultMaxObjectSize,
}, },

View file

@ -253,7 +253,7 @@ func (c *cache) flushAndDropBBoltDB(ctx context.Context) error {
if err != nil { if err != nil {
return fmt.Errorf("could not check write-cache database existence: %w", err) return fmt.Errorf("could not check write-cache database existence: %w", err)
} }
db, err := OpenDB(c.path, true, os.OpenFile, c.pageSize) db, err := OpenDB(c.path, true, os.OpenFile)
if err != nil { if err != nil {
return fmt.Errorf("could not open write-cache database: %w", err) return fmt.Errorf("could not open write-cache database: %w", err)
} }

View file

@ -25,12 +25,11 @@ import (
func TestFlush(t *testing.T) { func TestFlush(t *testing.T) {
testlogger := test.NewLogger(t) testlogger := test.NewLogger(t)
createCacheFn := func(t *testing.T, smallSize uint64, mb *meta.DB, bs MainStorage, opts ...Option) Cache { createCacheFn := func(t *testing.T, mb *meta.DB, bs MainStorage, opts ...Option) Cache {
return New( return New(
append([]Option{ append([]Option{
WithLogger(testlogger), WithLogger(testlogger),
WithPath(filepath.Join(t.TempDir(), "writecache")), WithPath(filepath.Join(t.TempDir(), "writecache")),
WithSmallObjectSize(smallSize),
WithMetabase(mb), WithMetabase(mb),
WithBlobstor(bs), WithBlobstor(bs),
WithDisableBackgroundFlush(), WithDisableBackgroundFlush(),
@ -92,7 +91,6 @@ const (
type CreateCacheFunc[Option any] func( type CreateCacheFunc[Option any] func(
t *testing.T, t *testing.T,
smallSize uint64,
meta *meta.DB, meta *meta.DB,
bs MainStorage, bs MainStorage,
opts ...Option, opts ...Option,
@ -115,7 +113,7 @@ func runFlushTest[Option any](
failures ...TestFailureInjector[Option], failures ...TestFailureInjector[Option],
) { ) {
t.Run("no errors", func(t *testing.T) { t.Run("no errors", func(t *testing.T) {
wc, bs, mb := newCache(t, createCacheFn, smallSize) wc, bs, mb := newCache(t, createCacheFn)
defer func() { require.NoError(t, wc.Close()) }() defer func() { require.NoError(t, wc.Close()) }()
objects := putObjects(t, wc) objects := putObjects(t, wc)
@ -128,7 +126,7 @@ func runFlushTest[Option any](
}) })
t.Run("flush on moving to degraded mode", func(t *testing.T) { t.Run("flush on moving to degraded mode", func(t *testing.T) {
wc, bs, mb := newCache(t, createCacheFn, smallSize) wc, bs, mb := newCache(t, createCacheFn)
defer func() { require.NoError(t, wc.Close()) }() defer func() { require.NoError(t, wc.Close()) }()
objects := putObjects(t, wc) objects := putObjects(t, wc)
@ -146,7 +144,7 @@ func runFlushTest[Option any](
for _, f := range failures { for _, f := range failures {
t.Run(f.Desc, func(t *testing.T) { t.Run(f.Desc, func(t *testing.T) {
errCountOpt, errCount := errCountOption() errCountOpt, errCount := errCountOption()
wc, bs, mb := newCache(t, createCacheFn, smallSize, errCountOpt) wc, bs, mb := newCache(t, createCacheFn, errCountOpt)
defer func() { require.NoError(t, wc.Close()) }() defer func() { require.NoError(t, wc.Close()) }()
objects := putObjects(t, wc) objects := putObjects(t, wc)
f.InjectFn(t, wc) f.InjectFn(t, wc)
@ -168,7 +166,6 @@ func runFlushTest[Option any](
func newCache[Option any]( func newCache[Option any](
t *testing.T, t *testing.T,
createCacheFn CreateCacheFunc[Option], createCacheFn CreateCacheFunc[Option],
smallSize uint64,
opts ...Option, opts ...Option,
) (Cache, *blobstor.BlobStor, *meta.DB) { ) (Cache, *blobstor.BlobStor, *meta.DB) {
dir := t.TempDir() dir := t.TempDir()
@ -189,7 +186,7 @@ func newCache[Option any](
require.NoError(t, bs.Open(context.Background(), mode.ReadWrite)) require.NoError(t, bs.Open(context.Background(), mode.ReadWrite))
require.NoError(t, bs.Init()) require.NoError(t, bs.Init())
wc := createCacheFn(t, smallSize, mb, bs, opts...) wc := createCacheFn(t, mb, bs, opts...)
require.NoError(t, wc.Open(context.Background(), mode.ReadWrite)) require.NoError(t, wc.Open(context.Background(), mode.ReadWrite))
require.NoError(t, wc.Init()) require.NoError(t, wc.Init())

View file

@ -1,8 +1,6 @@
package writecache package writecache
import ( import (
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -20,8 +18,6 @@ type options struct {
metabase Metabase metabase Metabase
// maxObjectSize is the maximum size of the object stored in the write-cache. // maxObjectSize is the maximum size of the object stored in the write-cache.
maxObjectSize uint64 maxObjectSize uint64
// smallObjectSize is the maximum size of the object stored in the database.
smallObjectSize uint64
// workersCount is the number of workers flushing objects in parallel. // workersCount is the number of workers flushing objects in parallel.
workersCount int workersCount int
// maxCacheSize is the maximum total size of all objects saved in cache (DB + FS). // maxCacheSize is the maximum total size of all objects saved in cache (DB + FS).
@ -30,10 +26,6 @@ type options struct {
// maxCacheCount is the maximum total count of all object saved in cache. // maxCacheCount is the maximum total count of all object saved in cache.
// 0 (no limit) by default. // 0 (no limit) by default.
maxCacheCount uint64 maxCacheCount uint64
// maxBatchSize is the maximum batch size for the small object database.
maxBatchSize int
// maxBatchDelay is the maximum batch wait time for the small object database.
maxBatchDelay time.Duration
// noSync is true iff FSTree allows unsynchronized writes. // noSync is true iff FSTree allows unsynchronized writes.
noSync bool noSync bool
// reportError is the function called when encountering disk errors in background workers. // reportError is the function called when encountering disk errors in background workers.
@ -42,8 +34,6 @@ type options struct {
metrics Metrics metrics Metrics
// disableBackgroundFlush is for testing purposes only. // disableBackgroundFlush is for testing purposes only.
disableBackgroundFlush bool disableBackgroundFlush bool
// pageSize is bbolt's page size config value
pageSize int
// flushSizeLimit is total size of flushing objects. // flushSizeLimit is total size of flushing objects.
flushSizeLimit uint64 flushSizeLimit uint64
} }
@ -85,15 +75,6 @@ func WithMaxObjectSize(sz uint64) Option {
} }
} }
// WithSmallObjectSize sets maximum object size to be stored in write-cache.
func WithSmallObjectSize(sz uint64) Option {
return func(o *options) {
if sz > 0 {
o.smallObjectSize = sz
}
}
}
func WithFlushWorkersCount(c int) Option { func WithFlushWorkersCount(c int) Option {
return func(o *options) { return func(o *options) {
if c > 0 { if c > 0 {
@ -116,24 +97,6 @@ func WithMaxCacheCount(v uint64) Option {
} }
} }
// WithMaxBatchSize sets max batch size for the small object database.
func WithMaxBatchSize(sz int) Option {
return func(o *options) {
if sz > 0 {
o.maxBatchSize = sz
}
}
}
// WithMaxBatchDelay sets max batch delay for the small object database.
func WithMaxBatchDelay(d time.Duration) Option {
return func(o *options) {
if d > 0 {
o.maxBatchDelay = d
}
}
}
// WithNoSync sets an option to allow returning to caller on PUT before write is persisted. // WithNoSync sets an option to allow returning to caller on PUT before write is persisted.
// Note, that we use this flag for FSTree only and DO NOT use it for a bolt DB because // Note, that we use this flag for FSTree only and DO NOT use it for a bolt DB because
// we cannot yet properly handle the corrupted database during the startup. This SHOULD NOT // we cannot yet properly handle the corrupted database during the startup. This SHOULD NOT
@ -165,13 +128,6 @@ func WithDisableBackgroundFlush() Option {
} }
} }
// WithPageSize sets bbolt's page size.
func WithPageSize(s int) Option {
return func(o *options) {
o.pageSize = s
}
}
// WithFlushSizeLimit sets flush size limit. // WithFlushSizeLimit sets flush size limit.
func WithFlushSizeLimit(v uint64) Option { func WithFlushSizeLimit(v uint64) Option {
return func(o *options) { return func(o *options) {

View file

@ -10,12 +10,11 @@ import (
) )
// OpenDB opens BoltDB instance for write-cache. Opens in read-only mode if ro is true. // OpenDB opens BoltDB instance for write-cache. Opens in read-only mode if ro is true.
func OpenDB(p string, ro bool, openFile func(string, int, fs.FileMode) (*os.File, error), pageSize int) (*bbolt.DB, error) { func OpenDB(p string, ro bool, openFile func(string, int, fs.FileMode) (*os.File, error)) (*bbolt.DB, error) {
return bbolt.Open(filepath.Join(p, dbName), os.ModePerm, &bbolt.Options{ return bbolt.Open(filepath.Join(p, dbName), os.ModePerm, &bbolt.Options{
NoFreelistSync: true, NoFreelistSync: true,
ReadOnly: ro, ReadOnly: ro,
Timeout: 100 * time.Millisecond, Timeout: 100 * time.Millisecond,
OpenFile: openFile, OpenFile: openFile,
PageSize: pageSize,
}) })
} }