forked from TrueCloudLab/frostfs-node
[#1367] writecache: Add background flushing objects limiter
To limit memory usage by background flush. Signed-off-by: Dmitrii Stepanov <d.stepanov@yadro.com>
This commit is contained in:
parent
8a6e3025a0
commit
e39378b1c3
12 changed files with 184 additions and 35 deletions
|
@ -68,6 +68,7 @@ func New(opts ...Option) Cache {
|
|||
maxBatchSize: bbolt.DefaultMaxBatchSize,
|
||||
maxBatchDelay: bbolt.DefaultMaxBatchDelay,
|
||||
metrics: DefaultMetrics(),
|
||||
flushSizeLimit: defaultFlushWorkersCount * defaultMaxObjectSize,
|
||||
},
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@ import (
|
|||
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
|
||||
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
|
||||
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
||||
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
||||
"go.etcd.io/bbolt"
|
||||
|
@ -41,19 +41,25 @@ func (c *cache) runFlushLoop(ctx context.Context) {
|
|||
if c.disableBackgroundFlush {
|
||||
return
|
||||
}
|
||||
fl := newFlushLimiter(c.flushSizeLimit)
|
||||
c.wg.Add(1)
|
||||
go func() {
|
||||
defer c.wg.Done()
|
||||
c.pushToFlushQueue(ctx)
|
||||
c.pushToFlushQueue(ctx, fl)
|
||||
}()
|
||||
|
||||
for range c.workersCount {
|
||||
c.wg.Add(1)
|
||||
go c.workerFlush(ctx)
|
||||
go c.workerFlush(ctx, fl)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *cache) pushToFlushQueue(ctx context.Context) {
|
||||
func (c *cache) pushToFlushQueue(ctx context.Context, fl *flushLimiter) {
|
||||
stopf := context.AfterFunc(ctx, func() {
|
||||
fl.close()
|
||||
})
|
||||
defer stopf()
|
||||
|
||||
tick := time.NewTicker(defaultFlushInterval)
|
||||
for {
|
||||
select {
|
||||
|
@ -65,6 +71,9 @@ func (c *cache) pushToFlushQueue(ctx context.Context) {
|
|||
}
|
||||
|
||||
err := c.fsTree.IterateInfo(ctx, func(oi fstree.ObjectInfo) error {
|
||||
if err := fl.acquire(oi.DataSize); err != nil {
|
||||
return err
|
||||
}
|
||||
select {
|
||||
case c.flushCh <- objectInfo{
|
||||
addr: oi.Address,
|
||||
|
@ -72,6 +81,7 @@ func (c *cache) pushToFlushQueue(ctx context.Context) {
|
|||
}:
|
||||
return nil
|
||||
case <-ctx.Done():
|
||||
fl.release(oi.DataSize)
|
||||
return ctx.Err()
|
||||
}
|
||||
})
|
||||
|
@ -86,37 +96,42 @@ func (c *cache) pushToFlushQueue(ctx context.Context) {
|
|||
}
|
||||
}
|
||||
|
||||
func (c *cache) workerFlush(ctx context.Context) {
|
||||
func (c *cache) workerFlush(ctx context.Context, fl *flushLimiter) {
|
||||
defer c.wg.Done()
|
||||
|
||||
var objInfo objectInfo
|
||||
for {
|
||||
select {
|
||||
case objInfo = <-c.flushCh:
|
||||
c.flushIfAnObjectExistsWorker(ctx, objInfo, fl)
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
|
||||
res, err := c.fsTree.Get(ctx, common.GetPrm{
|
||||
Address: objInfo.addr,
|
||||
})
|
||||
if err != nil {
|
||||
if !errors.As(err, new(*apistatus.ObjectNotFound)) {
|
||||
c.reportFlushError(logs.WritecacheCantGetObject, objInfo.addr.EncodeToString(), metaerr.Wrap(err))
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
err = c.flushObject(ctx, res.Object, res.RawData, StorageTypeFSTree)
|
||||
if err != nil {
|
||||
// Error is handled in flushObject.
|
||||
continue
|
||||
}
|
||||
|
||||
c.deleteFromDisk(ctx, objInfo.addr)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *cache) flushIfAnObjectExistsWorker(ctx context.Context, objInfo objectInfo, fl *flushLimiter) {
|
||||
defer fl.release(objInfo.size)
|
||||
|
||||
res, err := c.fsTree.Get(ctx, common.GetPrm{
|
||||
Address: objInfo.addr,
|
||||
})
|
||||
if err != nil {
|
||||
if !client.IsErrObjectNotFound(err) {
|
||||
c.reportFlushError(logs.WritecacheCantGetObject, objInfo.addr.EncodeToString(), metaerr.Wrap(err))
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
err = c.flushObject(ctx, res.Object, res.RawData, StorageTypeFSTree)
|
||||
if err != nil {
|
||||
// Error is handled in flushObject.
|
||||
return
|
||||
}
|
||||
|
||||
c.deleteFromDisk(ctx, objInfo.addr)
|
||||
}
|
||||
|
||||
func (c *cache) reportFlushError(msg string, addr string, err error) {
|
||||
if c.reportError != nil {
|
||||
c.reportError(msg, err)
|
||||
|
|
70
pkg/local_object_storage/writecache/limiter.go
Normal file
70
pkg/local_object_storage/writecache/limiter.go
Normal file
|
@ -0,0 +1,70 @@
|
|||
package writecache
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
)
|
||||
|
||||
var errLimiterClosed = errors.New("acquire failed: limiter closed")
|
||||
|
||||
// flushLimiter is used to limit the total size of objects
|
||||
// being flushed to blobstore at the same time. This is a necessary
|
||||
// limitation so that the flushing process does not have
|
||||
// a strong impact on user requests.
|
||||
type flushLimiter struct {
|
||||
count, size uint64
|
||||
maxSize uint64
|
||||
cond *sync.Cond
|
||||
closed bool
|
||||
}
|
||||
|
||||
func newFlushLimiter(maxSize uint64) *flushLimiter {
|
||||
return &flushLimiter{
|
||||
maxSize: maxSize,
|
||||
cond: sync.NewCond(&sync.Mutex{}),
|
||||
}
|
||||
}
|
||||
|
||||
func (l *flushLimiter) acquire(size uint64) error {
|
||||
l.cond.L.Lock()
|
||||
defer l.cond.L.Unlock()
|
||||
|
||||
// it is allowed to overflow maxSize to allow flushing objects with size > maxSize
|
||||
for l.count > 0 && l.size+size > l.maxSize && !l.closed {
|
||||
l.cond.Wait()
|
||||
if l.closed {
|
||||
return errLimiterClosed
|
||||
}
|
||||
}
|
||||
l.count++
|
||||
l.size += size
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *flushLimiter) release(size uint64) {
|
||||
l.cond.L.Lock()
|
||||
defer l.cond.L.Unlock()
|
||||
|
||||
if l.size >= size {
|
||||
l.size -= size
|
||||
} else {
|
||||
panic("flushLimiter: invalid size")
|
||||
}
|
||||
|
||||
if l.count > 0 {
|
||||
l.count--
|
||||
} else {
|
||||
panic("flushLimiter: invalid count")
|
||||
}
|
||||
|
||||
l.cond.Broadcast()
|
||||
}
|
||||
|
||||
func (l *flushLimiter) close() {
|
||||
l.cond.L.Lock()
|
||||
defer l.cond.L.Unlock()
|
||||
|
||||
l.closed = true
|
||||
|
||||
l.cond.Broadcast()
|
||||
}
|
27
pkg/local_object_storage/writecache/limiter_test.go
Normal file
27
pkg/local_object_storage/writecache/limiter_test.go
Normal file
|
@ -0,0 +1,27 @@
|
|||
package writecache
|
||||
|
||||
import (
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
"golang.org/x/sync/errgroup"
|
||||
)
|
||||
|
||||
func TestLimiter(t *testing.T) {
|
||||
var maxSize uint64 = 10
|
||||
var single uint64 = 3
|
||||
l := newFlushLimiter(uint64(maxSize))
|
||||
var currSize atomic.Int64
|
||||
var eg errgroup.Group
|
||||
for i := 0; i < 10_000; i++ {
|
||||
eg.Go(func() error {
|
||||
defer l.release(single)
|
||||
defer currSize.Add(-1)
|
||||
l.acquire(single)
|
||||
require.True(t, currSize.Add(1) <= 3)
|
||||
return nil
|
||||
})
|
||||
}
|
||||
require.NoError(t, eg.Wait())
|
||||
}
|
|
@ -44,6 +44,8 @@ type options struct {
|
|||
disableBackgroundFlush bool
|
||||
// pageSize is bbolt's page size config value
|
||||
pageSize int
|
||||
// flushSizeLimit is total size of flushing objects.
|
||||
flushSizeLimit uint64
|
||||
}
|
||||
|
||||
// WithLogger sets logger.
|
||||
|
@ -169,3 +171,10 @@ func WithPageSize(s int) Option {
|
|||
o.pageSize = s
|
||||
}
|
||||
}
|
||||
|
||||
// WithFlushSizeLimit sets flush size limit.
|
||||
func WithFlushSizeLimit(v uint64) Option {
|
||||
return func(o *options) {
|
||||
o.flushSizeLimit = v
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue