[#421] Try using badger for the write-cache

Signed-off-by: Alejandro Lopez <a.lopez@yadro.com>
This commit is contained in:
Alejandro Lopez 2023-06-22 14:55:30 +03:00 committed by Evgenii Stratonikov
parent 65c72f3e0b
commit 1a0cb0f34a
56 changed files with 2234 additions and 747 deletions

View file

@ -0,0 +1,129 @@
package writecachebadger
import (
"sync"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"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/util/logger"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.uber.org/zap"
)
type cache struct {
options
mode mode.Mode
modeMtx sync.RWMutex
// flushCh is a channel with objects to flush.
flushCh chan *objectSDK.Object
// closeCh is close channel, protected by modeMtx.
closeCh chan struct{}
// wg is a wait group for flush workers.
wg sync.WaitGroup
// store contains underlying database.
store
}
// wcStorageType is used for write-cache operations logging.
const wcStorageType = "write-cache"
type objectInfo struct {
addr oid.Address
data []byte
obj *objectSDK.Object
}
const (
defaultMaxObjectSize = 64 << 20 // 64 MiB
defaultSmallObjectSize = 32 << 10 // 32 KiB
defaultMaxCacheSize = 1 << 30 // 1 GiB
)
// New creates new writecache instance.
func New(opts ...Option) writecache.Cache {
c := &cache{
flushCh: make(chan *objectSDK.Object),
mode: mode.ReadWrite,
options: options{
log: &logger.Logger{Logger: zap.NewNop()},
maxObjectSize: defaultMaxObjectSize,
workersCount: defaultFlushWorkersCount,
maxCacheSize: defaultMaxCacheSize,
metrics: writecache.DefaultMetrics(),
},
}
for i := range opts {
opts[i](&c.options)
}
return c
}
// SetLogger sets logger. It is used after the shard ID was generated to use it in logs.
func (c *cache) SetLogger(l *logger.Logger) {
c.log = l
}
func (c *cache) DumpInfo() writecache.Info {
return writecache.Info{
Path: c.path,
}
}
// Open opens and initializes database. Reads object counters from the ObjectCounters instance.
func (c *cache) Open(readOnly bool) error {
err := c.openStore(readOnly)
if err != nil {
return metaerr.Wrap(err)
}
// Opening after Close is done during maintenance mode,
// thus we need to create a channel here.
c.closeCh = make(chan struct{})
return metaerr.Wrap(c.initCounters())
}
// Init runs necessary services.
func (c *cache) Init() error {
c.log.Info(logs.WritecacheBadgerInitExperimental)
c.metrics.SetMode(c.mode)
c.runFlushLoop()
c.runGCLoop()
return nil
}
// Close closes db connection and stops services. Executes ObjectCounters.FlushAndClose op.
func (c *cache) Close() error {
// We cannot lock mutex for the whole operation duration
// because it is taken by some background workers, so `wg.Wait()` is done without modeMtx.
c.modeMtx.Lock()
if c.closeCh != nil {
close(c.closeCh)
}
c.mode = mode.DegradedReadOnly // prevent new operations from being processed
c.modeMtx.Unlock()
c.wg.Wait()
c.modeMtx.Lock()
defer c.modeMtx.Unlock()
c.closeCh = nil
var err error
if c.db != nil {
err = c.db.Close()
if err != nil {
c.db = nil
}
}
c.metrics.Close()
return nil
}

View file

@ -0,0 +1,70 @@
package writecachebadger
import (
"context"
"time"
storagelog "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/log"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"github.com/dgraph-io/badger/v4"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
// Delete removes object from write-cache.
//
// Returns an error of type apistatus.ObjectNotFound if object is missing in write-cache.
func (c *cache) Delete(ctx context.Context, addr oid.Address) error {
_, span := tracing.StartSpanFromContext(ctx, "writecache.Delete",
trace.WithAttributes(
attribute.String("address", addr.EncodeToString()),
))
defer span.End()
deleted := false
storageType := writecache.StorageTypeUndefined
startedAt := time.Now()
defer func() {
c.metrics.Delete(time.Since(startedAt), deleted, storageType)
}()
c.modeMtx.RLock()
defer c.modeMtx.RUnlock()
if c.readOnly() {
return writecache.ErrReadOnly
}
saddr := addr.EncodeToString()
err := c.db.Update(func(tx *badger.Txn) error {
it, err := tx.Get([]byte(saddr))
if err != nil {
if err == badger.ErrKeyNotFound {
return logicerr.Wrap(apistatus.ObjectNotFound{})
}
return err
}
if it.ValueSize() > 0 {
storageType = writecache.StorageTypeDB
err := tx.Delete([]byte(saddr))
if err == nil {
storagelog.Write(c.log,
storagelog.AddressField(saddr),
storagelog.StorageTypeField(wcStorageType),
storagelog.OpField("db DELETE"),
)
deleted = true
c.objCounters.DecDB()
}
return err
}
return nil
})
return metaerr.Wrap(err)
}

View file

@ -0,0 +1,257 @@
package writecachebadger
import (
"bytes"
"context"
"encoding/hex"
"errors"
"fmt"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
objectCore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/common"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
"github.com/dgraph-io/badger/v4"
"github.com/mr-tron/base58"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
"go.uber.org/zap"
)
const (
// flushBatchSize is amount of keys which will be read from cache to be flushed
// to the main storage. It is used to reduce contention between cache put
// and cache persist.
flushBatchSize = 512
// defaultFlushWorkersCount is number of workers for putting objects in main storage.
defaultFlushWorkersCount = 20
// defaultFlushInterval is default time interval between successive flushes.
defaultFlushInterval = time.Second
)
// runFlushLoop starts background workers which periodically flush objects to the blobstor.
func (c *cache) runFlushLoop() {
for i := 0; i < c.workersCount; i++ {
c.wg.Add(1)
go c.workerFlushSmall()
}
c.wg.Add(1)
go func() {
defer c.wg.Done()
tt := time.NewTimer(defaultFlushInterval)
defer tt.Stop()
for {
select {
case <-tt.C:
c.flushSmallObjects()
tt.Reset(defaultFlushInterval)
case <-c.closeCh:
return
}
}
}()
}
func (c *cache) flushSmallObjects() {
var lastKey internalKey
var m []objectInfo
for {
select {
case <-c.closeCh:
return
default:
}
m = m[:0]
c.modeMtx.RLock()
if c.readOnly() {
c.modeMtx.RUnlock()
time.Sleep(time.Second)
continue
}
_ = c.db.View(func(tx *badger.Txn) error {
it := tx.NewIterator(badger.DefaultIteratorOptions)
defer it.Close()
if len(lastKey) == 0 {
it.Rewind()
} else {
it.Seek(lastKey[:])
if it.Valid() && bytes.Equal(it.Item().Key(), lastKey[:]) {
it.Next()
}
}
for ; it.Valid() && len(m) < flushBatchSize; it.Next() {
if got, want := int(it.Item().KeySize()), len(lastKey); got != want {
return fmt.Errorf("invalid db key len: got %d, want %d", got, want)
}
it.Item().KeyCopy(lastKey[:])
value, err := it.Item().ValueCopy(nil)
if err != nil {
return err
}
m = append(m, objectInfo{
addr: lastKey.address(),
data: value,
})
}
return nil
})
var count int
for i := range m {
obj := objectSDK.New()
if err := obj.Unmarshal(m[i].data); err != nil {
continue
}
count++
select {
case c.flushCh <- obj:
case <-c.closeCh:
c.modeMtx.RUnlock()
return
}
}
if count == 0 {
c.modeMtx.RUnlock()
break
}
c.modeMtx.RUnlock()
c.log.Debug(logs.WritecacheTriedToFlushItemsFromWritecache,
zap.Int("count", count),
zap.String("start", base58.Encode(lastKey[:])))
}
}
func (c *cache) reportFlushError(msg string, addr string, err error) {
if c.reportError != nil {
c.reportError(msg, err)
} else {
c.log.Error(msg,
zap.String("address", addr),
zap.Error(err))
}
}
// workerFlushSmall writes small objects to the main storage.
func (c *cache) workerFlushSmall() {
defer c.wg.Done()
var obj *objectSDK.Object
for {
// Give priority to direct put.
select {
case obj = <-c.flushCh:
case <-c.closeCh:
return
}
err := c.flushObject(context.TODO(), obj, nil, writecache.StorageTypeDB)
if err != nil {
// Error is handled in flushObject.
continue
}
c.deleteFromDB([]string{objectCore.AddressOf(obj).EncodeToString()})
}
}
// flushObject is used to write object directly to the main storage.
func (c *cache) flushObject(ctx context.Context, obj *objectSDK.Object, data []byte, st writecache.StorageType) error {
var err error
defer func() {
c.metrics.Flush(err == nil, st)
}()
addr := objectCore.AddressOf(obj)
var prm common.PutPrm
prm.Object = obj
prm.RawData = data
res, err := c.blobstor.Put(ctx, prm)
if err != nil {
if !errors.Is(err, common.ErrNoSpace) && !errors.Is(err, common.ErrReadOnly) &&
!errors.Is(err, blobstor.ErrNoPlaceFound) {
c.reportFlushError("can't flush an object to blobstor",
addr.EncodeToString(), err)
}
return err
}
var updPrm meta.UpdateStorageIDPrm
updPrm.SetAddress(addr)
updPrm.SetStorageID(res.StorageID)
_, err = c.metabase.UpdateStorageID(updPrm)
if err != nil {
c.reportFlushError("can't update object storage ID",
addr.EncodeToString(), err)
}
return err
}
// Flush flushes all objects from the write-cache to the main storage.
// Write-cache must be in readonly mode to ensure correctness of an operation and
// to prevent interference with background flush workers.
func (c *cache) Flush(ctx context.Context, ignoreErrors bool) error {
ctx, span := tracing.StartSpanFromContext(ctx, "writecache.Flush",
trace.WithAttributes(
attribute.Bool("ignore_errors", ignoreErrors),
))
defer span.End()
c.modeMtx.RLock()
defer c.modeMtx.RUnlock()
return c.flush(ctx, ignoreErrors)
}
func (c *cache) flush(ctx context.Context, ignoreErrors bool) error {
return c.db.View(func(tx *badger.Txn) error {
it := tx.NewIterator(badger.DefaultIteratorOptions)
defer it.Close()
var key internalKey
for it.Rewind(); it.Valid(); it.Next() {
if got, want := int(it.Item().KeySize()), len(key); got != want {
err := fmt.Errorf("invalid db key len: got %d, want %d", got, want)
c.reportFlushError("can't decode object address from the DB", hex.EncodeToString(it.Item().Key()), metaerr.Wrap(err))
if ignoreErrors {
continue
}
return err
}
if err := it.Item().Value(func(data []byte) error {
var obj objectSDK.Object
if err := obj.Unmarshal(data); err != nil {
copy(key[:], it.Item().Key())
c.reportFlushError("can't unmarshal an object from the DB", key.address().EncodeToString(), metaerr.Wrap(err))
if ignoreErrors {
return nil
}
return err
}
return c.flushObject(ctx, &obj, data, writecache.StorageTypeDB)
}); err != nil {
return err
}
}
return nil
})
}

View file

@ -0,0 +1,65 @@
package writecachebadger
import (
"path/filepath"
"sync/atomic"
"testing"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/writecachetest"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
oidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id/test"
"github.com/dgraph-io/badger/v4"
"github.com/stretchr/testify/require"
"go.uber.org/zap/zaptest"
)
func TestFlush(t *testing.T) {
createCacheFn := func(t *testing.T, smallSize uint64, mb *meta.DB, bs *blobstor.BlobStor, opts ...Option) writecache.Cache {
return New(
append([]Option{
WithLogger(&logger.Logger{Logger: zaptest.NewLogger(t)}),
WithPath(filepath.Join(t.TempDir(), "writecache")),
WithMetabase(mb),
WithBlobstor(bs),
WithGCInterval(1 * time.Second),
}, opts...)...)
}
errCountOpt := func() (Option, *atomic.Uint32) {
cnt := &atomic.Uint32{}
return WithReportErrorFunc(func(string, error) {
cnt.Add(1)
}), cnt
}
failures := []writecachetest.TestFailureInjector[Option]{
{
Desc: "db, invalid address",
InjectFn: func(t *testing.T, wc writecache.Cache) {
c := wc.(*cache)
obj := testutil.GenerateObject()
data, err := obj.Marshal()
require.NoError(t, err)
require.NoError(t, c.db.Update(func(tx *badger.Txn) error {
return tx.Set([]byte{1, 2, 3}, data)
}))
},
},
{
Desc: "db, invalid object",
InjectFn: func(t *testing.T, wc writecache.Cache) {
c := wc.(*cache)
require.NoError(t, c.db.Update(func(tx *badger.Txn) error {
return tx.Set([]byte(oidtest.Address().EncodeToString()), []byte{1, 2, 3})
}))
},
},
}
writecachetest.TestFlush(t, createCacheFn, errCountOpt, failures...)
}

View file

@ -0,0 +1,31 @@
package writecachebadger
import (
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
)
func (c *cache) runGCLoop() {
c.wg.Add(1)
go func() {
defer c.wg.Done()
t := time.NewTicker(c.gcInterval)
defer t.Stop()
for {
select {
case <-c.closeCh:
return
case <-t.C:
// 0.5 is the recommended value so that write amplification of the value log is 2.
// See https://pkg.go.dev/github.com/dgraph-io/badger/v4#DB.RunValueLogGC for more info.
for c.db.RunValueLogGC(0.5) == nil {
c.log.Debug(logs.WritecacheDBValueLogGCRunCompleted)
}
}
}
}()
}

View file

@ -0,0 +1,20 @@
package writecachebadger
import (
"testing"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/storagetest"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
"go.uber.org/zap/zaptest"
)
func TestGeneric(t *testing.T) {
storagetest.TestAll(t, func(t *testing.T) storagetest.Component {
return New(
WithLogger(&logger.Logger{Logger: zaptest.NewLogger(t)}),
WithFlushWorkersCount(2),
WithPath(t.TempDir()),
WithGCInterval(1*time.Second))
})
}

View file

@ -0,0 +1,95 @@
package writecachebadger
import (
"context"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"github.com/dgraph-io/badger/v4"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
// Get returns object from write-cache.
//
// Returns an error of type apistatus.ObjectNotFound if the requested object is missing in write-cache.
func (c *cache) Get(ctx context.Context, addr oid.Address) (*objectSDK.Object, error) {
_, span := tracing.StartSpanFromContext(ctx, "writecache.Get",
trace.WithAttributes(
attribute.String("address", addr.EncodeToString()),
))
defer span.End()
obj, err := c.getInternal(addr)
return obj, metaerr.Wrap(err)
}
func (c *cache) getInternal(addr oid.Address) (*objectSDK.Object, error) {
found := false
storageType := writecache.StorageTypeUndefined
startedAt := time.Now()
defer func() {
c.metrics.Get(time.Since(startedAt), found, storageType)
}()
k := addr2key(addr)
value, err := Get(c.db, k[:])
if err == nil {
obj := objectSDK.New()
found = true
storageType = writecache.StorageTypeDB
return obj, obj.Unmarshal(value)
}
return nil, logicerr.Wrap(apistatus.ObjectNotFound{})
}
// Head returns object header from write-cache.
//
// Returns an error of type apistatus.ObjectNotFound if the requested object is missing in write-cache.
func (c *cache) Head(ctx context.Context, addr oid.Address) (*objectSDK.Object, error) {
_, span := tracing.StartSpanFromContext(ctx, "writecache.Head",
trace.WithAttributes(
attribute.String("address", addr.EncodeToString()),
))
defer span.End()
obj, err := c.getInternal(addr)
if err != nil {
return nil, metaerr.Wrap(err)
}
return obj.CutPayload(), nil
}
// Get fetches object from the underlying database.
// Key should be a stringified address.
//
// Returns an error of type apistatus.ObjectNotFound if the requested object is missing in db.
func Get(db *badger.DB, key []byte) ([]byte, error) {
var value []byte
err := db.View(func(tx *badger.Txn) error {
it, err := tx.Get(key)
if err != nil {
if err == badger.ErrKeyNotFound {
return logicerr.Wrap(apistatus.ObjectNotFound{})
}
return err
}
v, err := it.ValueCopy(nil)
if err != nil {
return err
}
value = v
return nil
})
return value, metaerr.Wrap(err)
}

View file

@ -0,0 +1,32 @@
package writecachebadger
import (
"fmt"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"github.com/dgraph-io/badger/v4"
)
// IterateDB iterates over all objects stored in badger.DB instance and passes them to f until error return.
// It is assumed that db is an underlying database of some WriteCache instance.
//
// DB must not be nil and should be opened.
func IterateDB(db *badger.DB, f func(oid.Address) error) error {
return metaerr.Wrap(db.View(func(tx *badger.Txn) error {
opts := badger.DefaultIteratorOptions
opts.PrefetchValues = false
it := tx.NewIterator(opts)
for it.Rewind(); it.Valid(); it.Next() {
var key internalKey
if got, want := len(it.Item().Key()), len(key); got != want {
return fmt.Errorf("invalid db key len: got %d, want %d", got, want)
}
copy(key[:], it.Item().Key())
if err := f(key.address()); err != nil {
return err
}
}
return nil
}))
}

View file

@ -0,0 +1,75 @@
package writecachebadger
import (
"context"
"fmt"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
// SetMode sets write-cache mode of operation.
// When shard is put in read-only mode all objects in memory are flushed to disk
// and all background jobs are suspended.
func (c *cache) SetMode(m mode.Mode) error {
ctx, span := tracing.StartSpanFromContext(context.TODO(), "writecache.SetMode",
trace.WithAttributes(
attribute.String("mode", m.String()),
))
defer span.End()
err := c.setMode(ctx, m)
if err == nil {
c.metrics.SetMode(m)
}
return err
}
// setMode applies new mode. Must be called with cache.modeMtx lock taken.
func (c *cache) setMode(ctx context.Context, m mode.Mode) error {
var err error
turnOffMeta := m.NoMetabase()
if turnOffMeta && !c.mode.NoMetabase() {
err = c.flush(ctx, true)
if err != nil {
return err
}
}
if c.db != nil {
if err = c.db.Close(); err != nil {
return fmt.Errorf("can't close write-cache database: %w", err)
}
}
// Suspend producers to ensure there are channel send operations in fly.
// flushCh is populated by `flush` with `modeMtx` taken, thus waiting until it is empty
// guarantees that there are no in-fly operations.
for len(c.flushCh) != 0 {
c.log.Info(logs.WritecacheWaitingForChannelsToFlush)
time.Sleep(time.Second)
}
if turnOffMeta {
c.mode = m
return nil
}
if err = c.openStore(m.ReadOnly()); err != nil {
return err
}
c.mode = m
return nil
}
// readOnly returns true if current mode is read-only.
// `c.modeMtx` must be taken.
func (c *cache) readOnly() bool {
return c.mode.ReadOnly()
}

View file

@ -0,0 +1,141 @@
package writecachebadger
import (
"context"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/common"
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
"go.uber.org/zap"
)
// Option represents write-cache configuration option.
type Option func(*options)
// meta is an interface for a metabase.
type metabase interface {
Exists(context.Context, meta.ExistsPrm) (meta.ExistsRes, error)
StorageID(context.Context, meta.StorageIDPrm) (meta.StorageIDRes, error)
UpdateStorageID(meta.UpdateStorageIDPrm) (meta.UpdateStorageIDRes, error)
}
// blob is an interface for the blobstor.
type blob interface {
Put(context.Context, common.PutPrm) (common.PutRes, error)
NeedsCompression(obj *objectSDK.Object) bool
Exists(ctx context.Context, res common.ExistsPrm) (common.ExistsRes, error)
}
type options struct {
log *logger.Logger
// path is a path to a directory for write-cache.
path string
// blobstor is the main persistent storage.
blobstor blob
// metabase is the metabase instance.
metabase metabase
// maxObjectSize is the maximum size of the object stored in the write-cache.
maxObjectSize uint64
// workersCount is the number of workers flushing objects in parallel.
workersCount int
// maxCacheSize is the maximum total size of all objects saved in cache (DB + FS).
// 1 GiB by default.
maxCacheSize uint64
// objCounters contains atomic counters for the number of objects stored in cache.
objCounters counters
// noSync is true iff FSTree allows unsynchronized writes.
noSync bool
// reportError is the function called when encountering disk errors in background workers.
reportError func(string, error)
// metrics is metrics implementation
metrics writecache.Metrics
// gcInterval is the interval duration to run the GC cycle.
gcInterval time.Duration
}
// WithLogger sets logger.
func WithLogger(log *logger.Logger) Option {
return func(o *options) {
o.log = &logger.Logger{Logger: log.With(zap.String("component", "WriteCache"))}
}
}
// WithPath sets path to writecache db.
func WithPath(path string) Option {
return func(o *options) {
o.path = path
}
}
// WithBlobstor sets main object storage.
func WithBlobstor(bs *blobstor.BlobStor) Option {
return func(o *options) {
o.blobstor = bs
}
}
// WithMetabase sets metabase.
func WithMetabase(db *meta.DB) Option {
return func(o *options) {
o.metabase = db
}
}
// WithMaxObjectSize sets maximum object size to be stored in write-cache.
func WithMaxObjectSize(sz uint64) Option {
return func(o *options) {
if sz > 0 {
o.maxObjectSize = sz
}
}
}
func WithFlushWorkersCount(c int) Option {
return func(o *options) {
if c > 0 {
o.workersCount = c
}
}
}
// WithMaxCacheSize sets maximum write-cache size in bytes.
func WithMaxCacheSize(sz uint64) Option {
return func(o *options) {
o.maxCacheSize = sz
}
}
// 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
// we cannot yet properly handle the corrupted database during the startup. This SHOULD NOT
// be relied upon and may be changed in future.
func WithNoSync(noSync bool) Option {
return func(o *options) {
o.noSync = noSync
}
}
// WithReportErrorFunc sets error reporting function.
func WithReportErrorFunc(f func(string, error)) Option {
return func(o *options) {
o.reportError = f
}
}
// WithMetrics sets metrics implementation.
func WithMetrics(metrics writecache.Metrics) Option {
return func(o *options) {
o.metrics = metrics
}
}
// WithGCInterval sets the duration of the interval to run GC cycles.
func WithGCInterval(d time.Duration) Option {
return func(o *options) {
o.gcInterval = d
}
}

View file

@ -0,0 +1,82 @@
package writecachebadger
import (
"context"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/common"
storagelog "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/log"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
// Put puts object to write-cache.
//
// Returns ErrReadOnly if write-cache is in R/O mode.
// Returns ErrNotInitialized if write-cache has not been initialized yet.
// Returns ErrOutOfSpace if saving an object leads to WC's size overflow.
// Returns ErrBigObject if an objects exceeds maximum object size.
func (c *cache) Put(ctx context.Context, prm common.PutPrm) (common.PutRes, error) {
_, span := tracing.StartSpanFromContext(ctx, "writecache.Put",
trace.WithAttributes(
attribute.String("address", prm.Address.EncodeToString()),
attribute.Bool("dont_compress", prm.DontCompress),
))
defer span.End()
startedAt := time.Now()
added := false
storageType := writecache.StorageTypeUndefined
defer func() {
c.metrics.Put(time.Since(startedAt), added, storageType)
}()
c.modeMtx.RLock()
defer c.modeMtx.RUnlock()
if c.readOnly() {
return common.PutRes{}, writecache.ErrReadOnly
}
sz := uint64(len(prm.RawData))
if sz > c.maxObjectSize {
return common.PutRes{}, writecache.ErrBigObject
}
oi := objectInfo{
addr: prm.Address,
obj: prm.Object,
data: prm.RawData,
}
storageType = writecache.StorageTypeDB
err := c.put(oi)
if err == nil {
added = true
}
return common.PutRes{}, err
}
// put persists objects to the write-cache database and
// pushes the to the flush workers queue.
func (c *cache) put(obj objectInfo) error {
cacheSize := c.estimateCacheSize()
if c.maxCacheSize < c.incSizeDB(cacheSize) {
return writecache.ErrOutOfSpace
}
wb := c.db.NewWriteBatch()
k := addr2key(obj.addr)
_ = wb.Set(k[:], obj.data)
err := wb.Flush()
if err == nil {
storagelog.Write(c.log,
storagelog.AddressField(obj.addr),
storagelog.StorageTypeField(wcStorageType),
storagelog.OpField("db PUT"),
)
c.objCounters.IncDB()
}
return err
}

View file

@ -0,0 +1,57 @@
package writecachebadger
import (
"fmt"
"math"
"sync/atomic"
"github.com/dgraph-io/badger/v4"
)
func (c *cache) estimateCacheSize() uint64 {
onDiskSize, _ := c.db.EstimateSize(nil)
c.metrics.SetEstimateSize(onDiskSize, 0)
return onDiskSize
}
func (c *cache) incSizeDB(sz uint64) uint64 {
return sz + c.maxObjectSize
}
type counters struct {
cDB atomic.Uint64
}
func (x *counters) IncDB() {
x.cDB.Add(1)
}
func (x *counters) DecDB() {
x.cDB.Add(math.MaxUint64)
}
func (x *counters) DB() uint64 {
return x.cDB.Load()
}
func (c *cache) initCounters() error {
var inDB uint64
err := c.db.View(func(tx *badger.Txn) error {
opts := badger.DefaultIteratorOptions
opts.PrefetchValues = false
it := tx.NewIterator(opts)
defer it.Close()
for it.Rewind(); it.Valid(); it.Next() {
inDB++
}
return nil
})
if err != nil {
return fmt.Errorf("could not read write-cache DB counter: %w", err)
}
c.objCounters.cDB.Store(inDB)
c.metrics.SetActualCounters(inDB, 0)
return nil
}

View file

@ -0,0 +1,91 @@
package writecachebadger
import (
"fmt"
"os"
"path/filepath"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
storagelog "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/log"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"github.com/dgraph-io/badger/v4"
"go.uber.org/zap"
)
// store represents persistent storage with in-memory LRU cache
// for flushed items on top of it.
type store struct {
db *badger.DB
}
type internalKey [len(cid.ID{}) + len(oid.ID{})]byte
func (k internalKey) address() oid.Address {
var addr oid.Address
var cnr cid.ID
var obj oid.ID
copy(cnr[:], k[:len(cnr)])
copy(obj[:], k[len(cnr):])
addr.SetContainer(cnr)
addr.SetObject(obj)
return addr
}
func addr2key(addr oid.Address) internalKey {
var key internalKey
cnr, obj := addr.Container(), addr.Object()
copy(key[:len(cnr)], cnr[:])
copy(key[len(cnr):], obj[:])
return key
}
const dbName = "small.badger"
func (c *cache) openStore(readOnly bool) error {
err := util.MkdirAllX(c.path, os.ModePerm)
if err != nil {
return err
}
c.db, err = OpenDB(filepath.Join(c.path, dbName), readOnly, c.log)
if err != nil {
return fmt.Errorf("could not open database: %w", err)
}
return nil
}
func (c *cache) deleteFromDB(keys []string) []string {
if len(keys) == 0 {
return keys
}
wb := c.db.NewWriteBatch()
var errorIndex int
for errorIndex = range keys {
if err := wb.Delete([]byte(keys[errorIndex])); err != nil {
break
}
}
for i := 0; i < errorIndex; i++ {
c.objCounters.DecDB()
c.metrics.Evict(writecache.StorageTypeDB)
storagelog.Write(c.log,
storagelog.AddressField(keys[i]),
storagelog.StorageTypeField(wcStorageType),
storagelog.OpField("db DELETE"),
)
}
if err := wb.Flush(); err != nil {
c.log.Error(logs.WritecacheCantRemoveObjectsFromTheDatabase, zap.Error(err))
}
copy(keys, keys[errorIndex:])
return keys[:len(keys)-errorIndex]
}

View file

@ -0,0 +1,36 @@
package writecachebadger
import (
"fmt"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
"github.com/dgraph-io/badger/v4"
)
// OpenDB opens a badger instance for write-cache. Opens in read-only mode if ro is true.
func OpenDB(p string, ro bool, l *logger.Logger) (*badger.DB, error) {
return badger.Open(badger.DefaultOptions(p).
WithReadOnly(ro).
WithLoggingLevel(badger.ERROR).
WithLogger(badgerLoggerWrapper{l}))
}
type badgerLoggerWrapper struct {
l *logger.Logger
}
func (w badgerLoggerWrapper) Errorf(msg string, args ...any) {
w.l.Error(fmt.Sprintf(msg, args...))
}
func (w badgerLoggerWrapper) Warningf(msg string, args ...any) {
w.l.Error(fmt.Sprintf(msg, args...))
}
func (w badgerLoggerWrapper) Infof(msg string, args ...any) {
w.l.Error(fmt.Sprintf(msg, args...))
}
func (w badgerLoggerWrapper) Debugf(msg string, args ...any) {
w.l.Error(fmt.Sprintf(msg, args...))
}