Revert "[#1367] writecache: Add background flushing objects limiter"
This reverts commit e39378b1c3
.
This commit is contained in:
parent
105278f1b3
commit
f44ba3b76b
12 changed files with 32 additions and 181 deletions
|
@ -163,7 +163,6 @@ type shardCfg struct {
|
||||||
countLimit uint64
|
countLimit uint64
|
||||||
noSync bool
|
noSync bool
|
||||||
pageSize int
|
pageSize int
|
||||||
flushSizeLimit uint64
|
|
||||||
}
|
}
|
||||||
|
|
||||||
piloramaCfg struct {
|
piloramaCfg struct {
|
||||||
|
@ -302,7 +301,6 @@ func (a *applicationConfiguration) setShardWriteCacheConfig(newConfig *shardCfg,
|
||||||
wc.sizeLimit = writeCacheCfg.SizeLimit()
|
wc.sizeLimit = writeCacheCfg.SizeLimit()
|
||||||
wc.countLimit = writeCacheCfg.CountLimit()
|
wc.countLimit = writeCacheCfg.CountLimit()
|
||||||
wc.noSync = writeCacheCfg.NoSync()
|
wc.noSync = writeCacheCfg.NoSync()
|
||||||
wc.flushSizeLimit = writeCacheCfg.MaxFlushingObjectsSize()
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -921,7 +919,6 @@ func (c *cfg) getWriteCacheOpts(shCfg shardCfg) []writecache.Option {
|
||||||
writecache.WithMaxBatchSize(wcRead.maxBatchSize),
|
writecache.WithMaxBatchSize(wcRead.maxBatchSize),
|
||||||
writecache.WithMaxBatchDelay(wcRead.maxBatchDelay),
|
writecache.WithMaxBatchDelay(wcRead.maxBatchDelay),
|
||||||
writecache.WithPageSize(wcRead.pageSize),
|
writecache.WithPageSize(wcRead.pageSize),
|
||||||
writecache.WithFlushSizeLimit(wcRead.flushSizeLimit),
|
|
||||||
writecache.WithMaxObjectSize(wcRead.maxObjSize),
|
writecache.WithMaxObjectSize(wcRead.maxObjSize),
|
||||||
writecache.WithSmallObjectSize(wcRead.smallObjectSize),
|
writecache.WithSmallObjectSize(wcRead.smallObjectSize),
|
||||||
writecache.WithFlushWorkersCount(wcRead.flushWorkerCount),
|
writecache.WithFlushWorkersCount(wcRead.flushWorkerCount),
|
||||||
|
|
|
@ -95,7 +95,6 @@ func TestEngineSection(t *testing.T) {
|
||||||
require.EqualValues(t, 3221225472, wc.SizeLimit())
|
require.EqualValues(t, 3221225472, wc.SizeLimit())
|
||||||
require.EqualValues(t, 4096, wc.BoltDB().PageSize())
|
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.Equal(t, "tmp/0/meta", meta.Path())
|
require.Equal(t, "tmp/0/meta", meta.Path())
|
||||||
require.Equal(t, fs.FileMode(0o644), meta.BoltDB().Perm())
|
require.Equal(t, fs.FileMode(0o644), meta.BoltDB().Perm())
|
||||||
|
@ -153,7 +152,6 @@ func TestEngineSection(t *testing.T) {
|
||||||
require.EqualValues(t, 4294967296, wc.SizeLimit())
|
require.EqualValues(t, 4294967296, wc.SizeLimit())
|
||||||
require.EqualValues(t, 0, wc.BoltDB().PageSize())
|
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.Equal(t, "tmp/1/meta", meta.Path())
|
require.Equal(t, "tmp/1/meta", meta.Path())
|
||||||
require.Equal(t, fs.FileMode(0o644), meta.BoltDB().Perm())
|
require.Equal(t, fs.FileMode(0o644), meta.BoltDB().Perm())
|
||||||
|
|
|
@ -24,8 +24,6 @@ const (
|
||||||
|
|
||||||
// CountLimitDefault is a default write-cache count limit.
|
// CountLimitDefault is a default write-cache count limit.
|
||||||
CountLimitDefault = 0
|
CountLimitDefault = 0
|
||||||
|
|
||||||
MaxFlushingObjectsSizeDefault = 128 << 20
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// From wraps config section into Config.
|
// From wraps config section into Config.
|
||||||
|
@ -147,19 +145,3 @@ func (x *Config) NoSync() bool {
|
||||||
func (x *Config) BoltDB() *boltdbconfig.Config {
|
func (x *Config) BoltDB() *boltdbconfig.Config {
|
||||||
return (*boltdbconfig.Config)(x)
|
return (*boltdbconfig.Config)(x)
|
||||||
}
|
}
|
||||||
|
|
||||||
// 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
|
|
||||||
}
|
|
||||||
|
|
|
@ -109,7 +109,6 @@ FROSTFS_STORAGE_SHARD_0_WRITECACHE_FLUSH_WORKER_COUNT=30
|
||||||
FROSTFS_STORAGE_SHARD_0_WRITECACHE_CAPACITY=3221225472
|
FROSTFS_STORAGE_SHARD_0_WRITECACHE_CAPACITY=3221225472
|
||||||
FROSTFS_STORAGE_SHARD_0_WRITECACHE_PAGE_SIZE=4096
|
FROSTFS_STORAGE_SHARD_0_WRITECACHE_PAGE_SIZE=4096
|
||||||
FROSTFS_STORAGE_SHARD_0_WRITECACHE_MAX_OBJECT_COUNT=49
|
FROSTFS_STORAGE_SHARD_0_WRITECACHE_MAX_OBJECT_COUNT=49
|
||||||
FROSTFS_STORAGE_SHARD_0_WRITECACHE_MAX_FLUSHING_OBJECTS_SIZE=100
|
|
||||||
### Metabase config
|
### Metabase config
|
||||||
FROSTFS_STORAGE_SHARD_0_METABASE_PATH=tmp/0/meta
|
FROSTFS_STORAGE_SHARD_0_METABASE_PATH=tmp/0/meta
|
||||||
FROSTFS_STORAGE_SHARD_0_METABASE_PERM=0644
|
FROSTFS_STORAGE_SHARD_0_METABASE_PERM=0644
|
||||||
|
|
|
@ -154,8 +154,7 @@
|
||||||
"flush_worker_count": 30,
|
"flush_worker_count": 30,
|
||||||
"capacity": 3221225472,
|
"capacity": 3221225472,
|
||||||
"page_size": 4096,
|
"page_size": 4096,
|
||||||
"max_object_count": 49,
|
"max_object_count": 49
|
||||||
"max_flushing_objects_size": 100
|
|
||||||
},
|
},
|
||||||
"metabase": {
|
"metabase": {
|
||||||
"path": "tmp/0/meta",
|
"path": "tmp/0/meta",
|
||||||
|
|
|
@ -179,7 +179,6 @@ storage:
|
||||||
capacity: 3221225472 # approximate write-cache total size, bytes
|
capacity: 3221225472 # approximate write-cache total size, bytes
|
||||||
max_object_count: 49
|
max_object_count: 49
|
||||||
page_size: 4k
|
page_size: 4k
|
||||||
max_flushing_objects_size: 100b
|
|
||||||
|
|
||||||
metabase:
|
metabase:
|
||||||
path: tmp/0/meta # metabase path
|
path: tmp/0/meta # metabase path
|
||||||
|
|
|
@ -293,18 +293,17 @@ writecache:
|
||||||
page_size: '4k'
|
page_size: '4k'
|
||||||
```
|
```
|
||||||
|
|
||||||
| Parameter | Type | Default value | Description |
|
| Parameter | Type | Default value | Description |
|
||||||
| --------------------------- | ---------- | ------------- | ----------------------------------------------------------------------------------------------------------------------------- |
|
|----------------------|------------|---------------|-------------------------------------------------------------------------------------------------------------------------------|
|
||||||
| `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. |
|
| `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_batch_size` | `int` | `1000` | Maximum amount of small object `PUT` operations to perform in a single transaction. |
|
||||||
| `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. |
|
||||||
| `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. |
|
||||||
| `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
|
||||||
|
|
|
@ -68,7 +68,6 @@ func New(opts ...Option) Cache {
|
||||||
maxBatchSize: bbolt.DefaultMaxBatchSize,
|
maxBatchSize: bbolt.DefaultMaxBatchSize,
|
||||||
maxBatchDelay: bbolt.DefaultMaxBatchDelay,
|
maxBatchDelay: bbolt.DefaultMaxBatchDelay,
|
||||||
metrics: DefaultMetrics(),
|
metrics: DefaultMetrics(),
|
||||||
flushSizeLimit: defaultFlushWorkersCount * defaultMaxObjectSize,
|
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -18,7 +18,7 @@ import (
|
||||||
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
|
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-node/pkg/local_object_storage/shard/mode"
|
||||||
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
|
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
|
||||||
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
|
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
|
||||||
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
||||||
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.etcd.io/bbolt"
|
||||||
|
@ -41,25 +41,19 @@ func (c *cache) runFlushLoop(ctx context.Context) {
|
||||||
if c.disableBackgroundFlush {
|
if c.disableBackgroundFlush {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
fl := newFlushLimiter(c.flushSizeLimit)
|
|
||||||
c.wg.Add(1)
|
c.wg.Add(1)
|
||||||
go func() {
|
go func() {
|
||||||
defer c.wg.Done()
|
defer c.wg.Done()
|
||||||
c.pushToFlushQueue(ctx, fl)
|
c.pushToFlushQueue(ctx)
|
||||||
}()
|
}()
|
||||||
|
|
||||||
for range c.workersCount {
|
for range c.workersCount {
|
||||||
c.wg.Add(1)
|
c.wg.Add(1)
|
||||||
go c.workerFlush(ctx, fl)
|
go c.workerFlush(ctx)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *cache) pushToFlushQueue(ctx context.Context, fl *flushLimiter) {
|
func (c *cache) pushToFlushQueue(ctx context.Context) {
|
||||||
stopf := context.AfterFunc(ctx, func() {
|
|
||||||
fl.close()
|
|
||||||
})
|
|
||||||
defer stopf()
|
|
||||||
|
|
||||||
tick := time.NewTicker(defaultFlushInterval)
|
tick := time.NewTicker(defaultFlushInterval)
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
|
@ -71,9 +65,6 @@ func (c *cache) pushToFlushQueue(ctx context.Context, fl *flushLimiter) {
|
||||||
}
|
}
|
||||||
|
|
||||||
err := c.fsTree.IterateInfo(ctx, func(oi fstree.ObjectInfo) error {
|
err := c.fsTree.IterateInfo(ctx, func(oi fstree.ObjectInfo) error {
|
||||||
if err := fl.acquire(oi.DataSize); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
select {
|
select {
|
||||||
case c.flushCh <- objectInfo{
|
case c.flushCh <- objectInfo{
|
||||||
addr: oi.Address,
|
addr: oi.Address,
|
||||||
|
@ -81,7 +72,6 @@ func (c *cache) pushToFlushQueue(ctx context.Context, fl *flushLimiter) {
|
||||||
}:
|
}:
|
||||||
return nil
|
return nil
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
fl.release(oi.DataSize)
|
|
||||||
return ctx.Err()
|
return ctx.Err()
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
@ -96,40 +86,35 @@ func (c *cache) pushToFlushQueue(ctx context.Context, fl *flushLimiter) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *cache) workerFlush(ctx context.Context, fl *flushLimiter) {
|
func (c *cache) workerFlush(ctx context.Context) {
|
||||||
defer c.wg.Done()
|
defer c.wg.Done()
|
||||||
|
|
||||||
var objInfo objectInfo
|
var objInfo objectInfo
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case objInfo = <-c.flushCh:
|
case objInfo = <-c.flushCh:
|
||||||
c.flushIfAnObjectExistsWorker(ctx, objInfo, fl)
|
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
func (c *cache) flushIfAnObjectExistsWorker(ctx context.Context, objInfo objectInfo, fl *flushLimiter) {
|
res, err := c.fsTree.Get(ctx, common.GetPrm{
|
||||||
defer fl.release(objInfo.size)
|
Address: objInfo.addr,
|
||||||
|
})
|
||||||
res, err := c.fsTree.Get(ctx, common.GetPrm{
|
if err != nil {
|
||||||
Address: objInfo.addr,
|
if !errors.As(err, new(*apistatus.ObjectNotFound)) {
|
||||||
})
|
c.reportFlushError(ctx, logs.WritecacheCantGetObject, objInfo.addr.EncodeToString(), metaerr.Wrap(err))
|
||||||
if err != nil {
|
}
|
||||||
if !client.IsErrObjectNotFound(err) {
|
continue
|
||||||
c.reportFlushError(ctx, logs.WritecacheCantGetObject, objInfo.addr.EncodeToString(), metaerr.Wrap(err))
|
|
||||||
}
|
}
|
||||||
return
|
|
||||||
}
|
|
||||||
|
|
||||||
err = c.flushObject(ctx, res.Object, res.RawData, StorageTypeFSTree)
|
err = c.flushObject(ctx, res.Object, res.RawData, StorageTypeFSTree)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// Error is handled in flushObject.
|
// Error is handled in flushObject.
|
||||||
return
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
c.deleteFromDisk(ctx, objInfo.addr)
|
c.deleteFromDisk(ctx, objInfo.addr)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *cache) reportFlushError(ctx context.Context, msg string, addr string, err error) {
|
func (c *cache) reportFlushError(ctx context.Context, msg string, addr string, err error) {
|
||||||
|
|
|
@ -1,70 +0,0 @@
|
||||||
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()
|
|
||||||
}
|
|
|
@ -1,27 +0,0 @@
|
||||||
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 range 10_000 {
|
|
||||||
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())
|
|
||||||
}
|
|
|
@ -45,8 +45,6 @@ type options struct {
|
||||||
disableBackgroundFlush bool
|
disableBackgroundFlush bool
|
||||||
// pageSize is bbolt's page size config value
|
// pageSize is bbolt's page size config value
|
||||||
pageSize int
|
pageSize int
|
||||||
// flushSizeLimit is total size of flushing objects.
|
|
||||||
flushSizeLimit uint64
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// WithLogger sets logger.
|
// WithLogger sets logger.
|
||||||
|
@ -172,10 +170,3 @@ func WithPageSize(s int) Option {
|
||||||
o.pageSize = s
|
o.pageSize = s
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// WithFlushSizeLimit sets flush size limit.
|
|
||||||
func WithFlushSizeLimit(v uint64) Option {
|
|
||||||
return func(o *options) {
|
|
||||||
o.flushSizeLimit = v
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
Loading…
Add table
Reference in a new issue