Fix GC race conditions in tests #668

Merged
fyrchik merged 11 commits from fyrchik/frostfs-node:fix-gc-test into master 2023-09-07 07:39:41 +00:00
14 changed files with 206 additions and 215 deletions

View file

@ -32,12 +32,6 @@ import (
"go.etcd.io/bbolt"
)
type epochState struct{}
func (s epochState) CurrentEpoch() uint64 {
return 0
}
type objAddr struct {
obj *objectSDK.Object
addr oid.Address

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -7,7 +7,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
cidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id/test"
"github.com/stretchr/testify/require"
@ -29,15 +28,14 @@ func TestShard_Delete(t *testing.T) {
func testShardDelete(t *testing.T, hasWriteCache bool) {
sh := newShard(t, hasWriteCache)
defer releaseShard(sh, t)
cnr := cidtest.ID()
obj := testutil.GenerateObjectWithCID(cnr)
testutil.AddAttribute(obj, "foo", "bar")
var putPrm shard.PutPrm
var getPrm shard.GetPrm
var putPrm PutPrm
var getPrm GetPrm
t.Run("big object", func(t *testing.T) {
testutil.AddPayload(obj, 1<<20)
@ -45,7 +43,7 @@ func testShardDelete(t *testing.T, hasWriteCache bool) {
putPrm.SetObject(obj)
getPrm.SetAddress(object.AddressOf(obj))
var delPrm shard.DeletePrm
var delPrm DeletePrm
delPrm.SetAddresses(object.AddressOf(obj))
_, err := sh.Put(context.Background(), putPrm)
@ -71,7 +69,7 @@ func testShardDelete(t *testing.T, hasWriteCache bool) {
putPrm.SetObject(obj)
getPrm.SetAddress(object.AddressOf(obj))
var delPrm shard.DeletePrm
var delPrm DeletePrm
delPrm.SetAddresses(object.AddressOf(obj))
_, err := sh.Put(context.Background(), putPrm)

View file

@ -119,6 +119,8 @@ type gcCfg struct {
expiredCollectorBatchSize int
metrics GCMectrics
testHookRemover func(ctx context.Context) gcRunResult
}
func defaultGCCfg() gcCfg {
@ -158,33 +160,37 @@ func (gc *gc) listenEvents(ctx context.Context) {
return
}
v, ok := gc.mEventHandler[event.typ()]
if !ok {
continue
}
gc.handleEvent(ctx, event)
}
}
v.cancelFunc()
v.prevGroup.Wait()
func (gc *gc) handleEvent(ctx context.Context, event Event) {
v, ok := gc.mEventHandler[event.typ()]
if !ok {
return
}
var runCtx context.Context
runCtx, v.cancelFunc = context.WithCancel(ctx)
v.cancelFunc()
v.prevGroup.Wait()
v.prevGroup.Add(len(v.handlers))
var runCtx context.Context
runCtx, v.cancelFunc = context.WithCancel(ctx)
for i := range v.handlers {
h := v.handlers[i]
v.prevGroup.Add(len(v.handlers))
err := gc.workerPool.Submit(func() {
defer v.prevGroup.Done()
h(runCtx, event)
})
if err != nil {
gc.log.Warn(logs.ShardCouldNotSubmitGCJobToWorkerPool,
zap.String("error", err.Error()),
)
for i := range v.handlers {
h := v.handlers[i]
v.prevGroup.Done()
}
err := gc.workerPool.Submit(func() {
defer v.prevGroup.Done()
h(runCtx, event)
})
if err != nil {
gc.log.Warn(logs.ShardCouldNotSubmitGCJobToWorkerPool,
zap.String("error", err.Error()),
)
v.prevGroup.Done()
}
}
}
@ -209,7 +215,12 @@ func (gc *gc) tickRemover(ctx context.Context) {
case <-timer.C:
startedAt := time.Now()
result := gc.remover(ctx)
var result gcRunResult
if gc.testHookRemover != nil {
result = gc.testHookRemover(ctx)
} else {
result = gc.remover(ctx)
}
timer.Reset(gc.removerInterval)
gc.metrics.AddRunDuration(time.Since(startedAt), result.success)
@ -220,7 +231,7 @@ func (gc *gc) tickRemover(ctx context.Context) {
func (gc *gc) stop() {
gc.onceStop.Do(func() {
gc.stopChannel <- struct{}{}
close(gc.stopChannel)
})
gc.log.Info(logs.ShardWaitingForGCWorkersToStop)

View file

@ -75,7 +75,7 @@ func Test_ObjectNotFoundIfNotDeletedFromMetabase(t *testing.T) {
}
sh = New(opts...)
sh.gcCfg.testHookRemover = func(context.Context) gcRunResult { return gcRunResult{} }
require.NoError(t, sh.Open())
require.NoError(t, sh.Init(context.Background()))
@ -116,13 +116,13 @@ func Test_ObjectNotFoundIfNotDeletedFromMetabase(t *testing.T) {
storageID, err := sh.metaBase.StorageID(context.Background(), metaStIDPrm)
require.NoError(t, err, "failed to get storage ID")
//check existance in blobstore
//check existence in blobstore
var bsExisted common.ExistsPrm
bsExisted.Address = addr
bsExisted.StorageID = storageID.StorageID()
exRes, err := sh.blobStor.Exists(context.Background(), bsExisted)
require.NoError(t, err, "failed to check blobstore existance")
require.True(t, exRes.Exists, "invalid blobstore existance result")
require.NoError(t, err, "failed to check blobstore existence")
require.True(t, exRes.Exists, "invalid blobstore existence result")
//drop from blobstor
var bsDeletePrm common.DeletePrm
@ -131,10 +131,10 @@ func Test_ObjectNotFoundIfNotDeletedFromMetabase(t *testing.T) {
_, err = sh.blobStor.Delete(context.Background(), bsDeletePrm)
require.NoError(t, err, "failed to delete from blobstore")
//check existance in blobstore
//check existence in blobstore
exRes, err = sh.blobStor.Exists(context.Background(), bsExisted)
require.NoError(t, err, "failed to check blobstore existance")
require.False(t, exRes.Exists, "invalid blobstore existance result")
require.NoError(t, err, "failed to check blobstore existence")
require.False(t, exRes.Exists, "invalid blobstore existence result")
//get should return object not found
_, err = sh.Get(context.Background(), getPrm)

View file

@ -1,17 +1,15 @@
package shard_test
package shard
import (
"context"
"errors"
"testing"
"time"
objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
objectCore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"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/shard"
writecacheconfig "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/config"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
cidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id/test"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
@ -27,13 +25,11 @@ func Test_GCDropsLockedExpiredSimpleObject(t *testing.T) {
Value: 100,
}
wcOpts := writecacheconfig.Options{
Type: writecacheconfig.TypeBBolt,
}
sh := newCustomShard(t, t.TempDir(), false, wcOpts, nil, []meta.Option{meta.WithEpochState(epoch)})
t.Cleanup(func() {
releaseShard(sh, t)
sh := newCustomShard(t, false, shardOptions{
metaOptions: []meta.Option{meta.WithEpochState(epoch)},
additionalShardOptions: []Option{WithGCWorkerPoolInitializer(func(int) util.WorkerPool {
return util.NewPseudoWorkerPool() // synchronous event processing
})},
})
cnr := cidtest.ID()
@ -55,7 +51,7 @@ func Test_GCDropsLockedExpiredSimpleObject(t *testing.T) {
lock.SetAttributes(lockExpirationAttr)
lockID, _ := lock.ID()
var putPrm shard.PutPrm
var putPrm PutPrm
putPrm.SetObject(obj)
_, err := sh.Put(context.Background(), putPrm)
@ -69,14 +65,12 @@ func Test_GCDropsLockedExpiredSimpleObject(t *testing.T) {
require.NoError(t, err)
epoch.Value = 105
sh.NotificationChannel() <- shard.EventNewEpoch(epoch.Value)
sh.gc.handleEvent(context.Background(), EventNewEpoch(epoch.Value))
var getPrm shard.GetPrm
var getPrm GetPrm
getPrm.SetAddress(objectCore.AddressOf(obj))
require.Eventually(t, func() bool {
_, err = sh.Get(context.Background(), getPrm)
return client.IsErrObjectNotFound(err)
}, 3*time.Second, 1*time.Second, "expired object must be deleted")
_, err = sh.Get(context.Background(), getPrm)
require.True(t, client.IsErrObjectNotFound(err), "expired object must be deleted")
}
func Test_GCDropsLockedExpiredComplexObject(t *testing.T) {
@ -127,13 +121,11 @@ func Test_GCDropsLockedExpiredComplexObject(t *testing.T) {
linkID, _ := link.ID()
wcOpts := writecacheconfig.Options{
Type: writecacheconfig.TypeBBolt,
}
sh := newCustomShard(t, t.TempDir(), false, wcOpts, nil, []meta.Option{meta.WithEpochState(epoch)})
t.Cleanup(func() {
releaseShard(sh, t)
sh := newCustomShard(t, false, shardOptions{
metaOptions: []meta.Option{meta.WithEpochState(epoch)},
additionalShardOptions: []Option{WithGCWorkerPoolInitializer(func(int) util.WorkerPool {
return util.NewPseudoWorkerPool() // synchronous event processing
})},
})
lock := testutil.GenerateObjectWithCID(cnr)
@ -141,7 +133,7 @@ func Test_GCDropsLockedExpiredComplexObject(t *testing.T) {
lock.SetAttributes(lockExpirationAttr)
lockID, _ := lock.ID()
var putPrm shard.PutPrm
var putPrm PutPrm
for _, child := range children {
putPrm.SetObject(child)
@ -160,7 +152,7 @@ func Test_GCDropsLockedExpiredComplexObject(t *testing.T) {
_, err = sh.Put(context.Background(), putPrm)
require.NoError(t, err)
var getPrm shard.GetPrm
var getPrm GetPrm
getPrm.SetAddress(objectCore.AddressOf(parent))
_, err = sh.Get(context.Background(), getPrm)
@ -168,10 +160,8 @@ func Test_GCDropsLockedExpiredComplexObject(t *testing.T) {
require.True(t, errors.As(err, &splitInfoError), "split info must be provided")
epoch.Value = 105
sh.NotificationChannel() <- shard.EventNewEpoch(epoch.Value)
sh.gc.handleEvent(context.Background(), EventNewEpoch(epoch.Value))
require.Eventually(t, func() bool {
_, err = sh.Get(context.Background(), getPrm)
return client.IsErrObjectNotFound(err)
}, 3*time.Second, 1*time.Second, "expired complex object must be deleted on epoch after lock expires")
_, err = sh.Get(context.Background(), getPrm)
require.True(t, client.IsErrObjectNotFound(err), "expired complex object must be deleted on epoch after lock expires")
}

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"bytes"
@ -9,7 +9,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
cidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id/test"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
@ -33,10 +32,9 @@ func TestShard_Get(t *testing.T) {
func testShardGet(t *testing.T, hasWriteCache bool) {
sh := newShard(t, hasWriteCache)
defer releaseShard(sh, t)
var putPrm shard.PutPrm
var getPrm shard.GetPrm
var putPrm PutPrm
var getPrm GetPrm
t.Run("small object", func(t *testing.T) {
obj := testutil.GenerateObject()
@ -116,7 +114,7 @@ func testShardGet(t *testing.T, hasWriteCache bool) {
})
}
func testGet(t *testing.T, sh *shard.Shard, getPrm shard.GetPrm, hasWriteCache bool) (shard.GetRes, error) {
func testGet(t *testing.T, sh *Shard, getPrm GetPrm, hasWriteCache bool) (GetRes, error) {
res, err := sh.Get(context.Background(), getPrm)
if hasWriteCache {
require.Eventually(t, func() bool {

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -8,7 +8,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
cidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id/test"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
@ -31,10 +30,9 @@ func TestShard_Head(t *testing.T) {
func testShardHead(t *testing.T, hasWriteCache bool) {
sh := newShard(t, hasWriteCache)
defer releaseShard(sh, t)
var putPrm shard.PutPrm
var headPrm shard.HeadPrm
var putPrm PutPrm
var headPrm HeadPrm
t.Run("regular object", func(t *testing.T) {
obj := testutil.GenerateObject()
@ -87,7 +85,7 @@ func testShardHead(t *testing.T, hasWriteCache bool) {
})
}
func testHead(t *testing.T, sh *shard.Shard, headPrm shard.HeadPrm, hasWriteCache bool) (shard.HeadRes, error) {
func testHead(t *testing.T, sh *Shard, headPrm HeadPrm, hasWriteCache bool) (HeadRes, error) {
res, err := sh.Head(context.Background(), headPrm)
if hasWriteCache {
require.Eventually(t, func() bool {

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -6,7 +6,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
cidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id/test"
"github.com/stretchr/testify/require"
@ -28,7 +27,6 @@ func TestShard_Inhume(t *testing.T) {
func testShardInhume(t *testing.T, hasWriteCache bool) {
sh := newShard(t, hasWriteCache)
defer releaseShard(sh, t)
cnr := cidtest.ID()
@ -37,13 +35,13 @@ func testShardInhume(t *testing.T, hasWriteCache bool) {
ts := testutil.GenerateObjectWithCID(cnr)
var putPrm shard.PutPrm
var putPrm PutPrm
putPrm.SetObject(obj)
var inhPrm shard.InhumePrm
var inhPrm InhumePrm
inhPrm.SetTarget(object.AddressOf(ts), object.AddressOf(obj))
var getPrm shard.GetPrm
var getPrm GetPrm
getPrm.SetAddress(object.AddressOf(obj))
_, err := sh.Put(context.Background(), putPrm)

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -7,7 +7,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
cidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id/test"
"github.com/stretchr/testify/require"
"golang.org/x/sync/errgroup"
@ -19,19 +18,17 @@ func TestShard_List(t *testing.T) {
t.Run("without write cache", func(t *testing.T) {
t.Parallel()
sh := newShard(t, false)
defer releaseShard(sh, t)
testShardList(t, sh)
})
t.Run("with write cache", func(t *testing.T) {
t.Parallel()
shWC := newShard(t, true)
defer releaseShard(shWC, t)
testShardList(t, shWC)
})
}
func testShardList(t *testing.T, sh *shard.Shard) {
func testShardList(t *testing.T, sh *Shard) {
const C = 10
const N = 5
@ -59,7 +56,7 @@ func testShardList(t *testing.T, sh *shard.Shard) {
objs[object.AddressOf(obj).EncodeToString()] = 0
mtx.Unlock()
var putPrm shard.PutPrm
var putPrm PutPrm
putPrm.SetObject(obj)
_, err := sh.Put(context.Background(), putPrm)

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -11,7 +11,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
"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/shard"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
@ -26,13 +25,13 @@ import (
func TestShard_Lock(t *testing.T) {
t.Parallel()
var sh *shard.Shard
var sh *Shard
rootPath := t.TempDir()
opts := []shard.Option{
shard.WithID(shard.NewIDFromBytes([]byte{})),
shard.WithLogger(&logger.Logger{Logger: zap.NewNop()}),
shard.WithBlobStorOptions(
opts := []Option{
WithID(NewIDFromBytes([]byte{})),
WithLogger(&logger.Logger{Logger: zap.NewNop()}),
WithBlobStorOptions(
blobstor.WithStorages([]blobstor.SubStorage{
{
Storage: blobovniczatree.NewBlobovniczaTree(
@ -49,16 +48,16 @@ func TestShard_Lock(t *testing.T) {
},
}),
),
shard.WithMetaBaseOptions(
WithMetaBaseOptions(
meta.WithPath(filepath.Join(rootPath, "meta")),
meta.WithEpochState(epochState{}),
),
shard.WithDeletedLockCallback(func(_ context.Context, addresses []oid.Address) {
WithDeletedLockCallback(func(_ context.Context, addresses []oid.Address) {
sh.HandleDeletedLocks(addresses)
}),
}
sh = shard.New(opts...)
sh = New(opts...)
require.NoError(t, sh.Open())
require.NoError(t, sh.Init(context.Background()))
@ -76,7 +75,7 @@ func TestShard_Lock(t *testing.T) {
// put the object
var putPrm shard.PutPrm
var putPrm PutPrm
putPrm.SetObject(obj)
_, err := sh.Put(context.Background(), putPrm)
@ -94,7 +93,7 @@ func TestShard_Lock(t *testing.T) {
t.Run("inhuming locked objects", func(t *testing.T) {
ts := testutil.GenerateObjectWithCID(cnr)
var inhumePrm shard.InhumePrm
var inhumePrm InhumePrm
inhumePrm.SetTarget(objectcore.AddressOf(ts), objectcore.AddressOf(obj))
var objLockedErr *apistatus.ObjectLocked
@ -110,7 +109,7 @@ func TestShard_Lock(t *testing.T) {
t.Run("inhuming lock objects", func(t *testing.T) {
ts := testutil.GenerateObjectWithCID(cnr)
var inhumePrm shard.InhumePrm
var inhumePrm InhumePrm
inhumePrm.SetTarget(objectcore.AddressOf(ts), objectcore.AddressOf(lock))
_, err = sh.Inhume(context.Background(), inhumePrm)
@ -122,7 +121,7 @@ func TestShard_Lock(t *testing.T) {
})
t.Run("force objects inhuming", func(t *testing.T) {
var inhumePrm shard.InhumePrm
var inhumePrm InhumePrm
inhumePrm.MarkAsGarbage(objectcore.AddressOf(lock))
inhumePrm.ForceRemoval()
@ -132,7 +131,7 @@ func TestShard_Lock(t *testing.T) {
// it should be possible to remove
// lock object now
inhumePrm = shard.InhumePrm{}
inhumePrm = InhumePrm{}
inhumePrm.MarkAsGarbage(objectcore.AddressOf(obj))
_, err = sh.Inhume(context.Background(), inhumePrm)
@ -140,7 +139,7 @@ func TestShard_Lock(t *testing.T) {
// check that object has been removed
var getPrm shard.GetPrm
var getPrm GetPrm
getPrm.SetAddress(objectcore.AddressOf(obj))
_, err = sh.Get(context.Background(), getPrm)
@ -160,7 +159,7 @@ func TestShard_IsLocked(t *testing.T) {
// put the object
var putPrm shard.PutPrm
var putPrm PutPrm
putPrm.SetObject(obj)
_, err := sh.Put(context.Background(), putPrm)

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -12,7 +12,6 @@ import (
"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/pilorama"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
@ -126,9 +125,6 @@ func (m *metricsStore) DeleteShardMetrics() {
m.errCounter = 0
}
const physical = "phy"
const logical = "logic"
func TestCounters(t *testing.T) {
t.Parallel()
@ -163,24 +159,22 @@ func TestCounters(t *testing.T) {
totalPayload += oSize
}
t.Run("put", func(t *testing.T) {
var prm shard.PutPrm
var prm PutPrm
for i := 0; i < objNumber; i++ {
prm.SetObject(oo[i])
for i := 0; i < objNumber; i++ {
prm.SetObject(oo[i])
_, err := sh.Put(context.Background(), prm)
require.NoError(t, err)
}
_, err := sh.Put(context.Background(), prm)
require.NoError(t, err)
}
require.Equal(t, uint64(objNumber), mm.getObjectCounter(physical))
require.Equal(t, uint64(objNumber), mm.getObjectCounter(logical))
require.Equal(t, expectedSizes, mm.containerSizes())
require.Equal(t, totalPayload, mm.payloadSize())
})
require.Equal(t, uint64(objNumber), mm.getObjectCounter(physical))
require.Equal(t, uint64(objNumber), mm.getObjectCounter(logical))
require.Equal(t, expectedSizes, mm.containerSizes())
require.Equal(t, totalPayload, mm.payloadSize())
t.Run("inhume_GC", func(t *testing.T) {
var prm shard.InhumePrm
var prm InhumePrm
inhumedNumber := objNumber / 4
for i := 0; i < inhumedNumber; i++ {
@ -199,7 +193,7 @@ func TestCounters(t *testing.T) {
})
t.Run("inhume_TS", func(t *testing.T) {
var prm shard.InhumePrm
var prm InhumePrm
ts := objectcore.AddressOf(testutil.GenerateObject())
phy := mm.getObjectCounter(physical)
@ -220,7 +214,7 @@ func TestCounters(t *testing.T) {
})
t.Run("Delete", func(t *testing.T) {
var prm shard.DeletePrm
var prm DeletePrm
phy := mm.getObjectCounter(physical)
logic := mm.getObjectCounter(logical)
@ -246,7 +240,7 @@ func TestCounters(t *testing.T) {
})
}
func shardWithMetrics(t *testing.T, path string) (*shard.Shard, *metricsStore) {
func shardWithMetrics(t *testing.T, path string) (*Shard, *metricsStore) {
blobOpts := []blobstor.Option{
blobstor.WithStorages([]blobstor.SubStorage{
{
@ -266,14 +260,14 @@ func shardWithMetrics(t *testing.T, path string) (*shard.Shard, *metricsStore) {
cnrSize: make(map[string]int64),
}
sh := shard.New(
shard.WithID(shard.NewIDFromBytes([]byte{})),
shard.WithBlobStorOptions(blobOpts...),
shard.WithPiloramaOptions(pilorama.WithPath(filepath.Join(path, "pilorama"))),
shard.WithMetaBaseOptions(
sh := New(
WithID(NewIDFromBytes([]byte{})),
WithBlobStorOptions(blobOpts...),
WithPiloramaOptions(pilorama.WithPath(filepath.Join(path, "pilorama"))),
WithMetaBaseOptions(
meta.WithPath(filepath.Join(path, "meta")),
meta.WithEpochState(epochState{})),
shard.WithMetricsWriter(mm),
WithMetricsWriter(mm),
)
require.NoError(t, sh.Open())
require.NoError(t, sh.Init(context.Background()))

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -11,7 +11,6 @@ import (
"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/internal/testutil"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
writecacheconfig "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/config"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/writecachebbolt"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger/test"
@ -77,25 +76,27 @@ func testShardGetRange(t *testing.T, hasWriteCache bool) {
},
}
sh := newCustomShard(t, t.TempDir(), hasWriteCache, wcOpts,
[]blobstor.Option{blobstor.WithStorages([]blobstor.SubStorage{
{
Storage: blobovniczatree.NewBlobovniczaTree(
blobovniczatree.WithLogger(test.NewLogger(t, true)),
blobovniczatree.WithRootPath(filepath.Join(t.TempDir(), "blob", "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(1),
blobovniczatree.WithBlobovniczaShallowWidth(1)),
Policy: func(_ *objectSDK.Object, data []byte) bool {
return len(data) <= smallObjectSize
sh := newCustomShard(t, hasWriteCache, shardOptions{
wcOpts: wcOpts,
bsOpts: []blobstor.Option{
blobstor.WithStorages([]blobstor.SubStorage{
{
Storage: blobovniczatree.NewBlobovniczaTree(
blobovniczatree.WithLogger(test.NewLogger(t, true)),
blobovniczatree.WithRootPath(filepath.Join(t.TempDir(), "blob", "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(1),
blobovniczatree.WithBlobovniczaShallowWidth(1)),
Policy: func(_ *objectSDK.Object, data []byte) bool {
return len(data) <= smallObjectSize
},
},
},
{
Storage: fstree.New(
fstree.WithPath(filepath.Join(t.TempDir(), "blob"))),
},
})},
nil)
defer releaseShard(sh, t)
{
Storage: fstree.New(
fstree.WithPath(filepath.Join(t.TempDir(), "blob"))),
},
}),
},
})
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
@ -106,13 +107,13 @@ func testShardGetRange(t *testing.T, hasWriteCache bool) {
addr := object.AddressOf(obj)
payload := slice.Copy(obj.Payload())
var putPrm shard.PutPrm
var putPrm PutPrm
putPrm.SetObject(obj)
_, err := sh.Put(context.Background(), putPrm)
require.NoError(t, err)
var rngPrm shard.RngPrm
var rngPrm RngPrm
rngPrm.SetAddress(addr)
rngPrm.SetRange(tc.rng.GetOffset(), tc.rng.GetLength())

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -11,7 +11,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/fstree"
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/pilorama"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
writecacheconfig "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/config"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/writecachebadger"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/writecachebbolt"
@ -31,39 +30,50 @@ func (s epochState) CurrentEpoch() uint64 {
return s.Value
}
func newShard(t testing.TB, enableWriteCache bool) *shard.Shard {
return newCustomShard(t, t.TempDir(), enableWriteCache,
writecacheconfig.Options{Type: writecacheconfig.TypeBBolt},
nil,
nil)
type shardOptions struct {
rootPath string
dontRelease bool
wcOpts writecacheconfig.Options
bsOpts []blobstor.Option
metaOptions []meta.Option
additionalShardOptions []Option
}
func newCustomShard(t testing.TB, rootPath string, enableWriteCache bool, wcOpts writecacheconfig.Options, bsOpts []blobstor.Option, metaOptions []meta.Option) *shard.Shard {
var sh *shard.Shard
if enableWriteCache {
rootPath = filepath.Join(rootPath, "wc")
switch wcOpts.Type {
case writecacheconfig.TypeBBolt:
wcOpts.BBoltOptions = append(
[]writecachebbolt.Option{writecachebbolt.WithPath(filepath.Join(rootPath, "wcache"))},
wcOpts.BBoltOptions...)
case writecacheconfig.TypeBadger:
wcOpts.BadgerOptions = append(
[]writecachebadger.Option{writecachebadger.WithPath(filepath.Join(rootPath, "wcache"))},
wcOpts.BadgerOptions...)
}
} else {
rootPath = filepath.Join(rootPath, "nowc")
func newShard(t testing.TB, enableWriteCache bool) *Shard {
return newCustomShard(t, enableWriteCache, shardOptions{})
}
func newCustomShard(t testing.TB, enableWriteCache bool, o shardOptions) *Shard {
if o.rootPath == "" {
o.rootPath = t.TempDir()
}
if enableWriteCache && o.wcOpts.Type == 0 {
o.wcOpts.Type = writecacheconfig.TypeBBolt
}
if bsOpts == nil {
bsOpts = []blobstor.Option{
var sh *Shard
if enableWriteCache {
switch o.wcOpts.Type {
case writecacheconfig.TypeBBolt:
o.wcOpts.BBoltOptions = append(
[]writecachebbolt.Option{writecachebbolt.WithPath(filepath.Join(o.rootPath, "wcache"))},
o.wcOpts.BBoltOptions...)
case writecacheconfig.TypeBadger:
o.wcOpts.BadgerOptions = append(
[]writecachebadger.Option{writecachebadger.WithPath(filepath.Join(o.rootPath, "wcache"))},
o.wcOpts.BadgerOptions...)
}
}
if o.bsOpts == nil {
o.bsOpts = []blobstor.Option{
blobstor.WithLogger(test.NewLogger(t, true)),
blobstor.WithStorages([]blobstor.SubStorage{
{
Storage: blobovniczatree.NewBlobovniczaTree(
blobovniczatree.WithLogger(test.NewLogger(t, true)),
blobovniczatree.WithRootPath(filepath.Join(rootPath, "blob", "blobovnicza")),
blobovniczatree.WithRootPath(filepath.Join(o.rootPath, "blob", "blobovnicza")),
blobovniczatree.WithBlobovniczaShallowDepth(1),
blobovniczatree.WithBlobovniczaShallowWidth(1)),
Policy: func(_ *objectSDK.Object, data []byte) bool {
@ -72,46 +82,51 @@ func newCustomShard(t testing.TB, rootPath string, enableWriteCache bool, wcOpts
},
{
Storage: fstree.New(
fstree.WithPath(filepath.Join(rootPath, "blob"))),
fstree.WithPath(filepath.Join(o.rootPath, "blob"))),
},
}),
}
}
opts := []shard.Option{
shard.WithID(shard.NewIDFromBytes([]byte{})),
shard.WithLogger(test.NewLogger(t, true)),
shard.WithBlobStorOptions(bsOpts...),
shard.WithMetaBaseOptions(
opts := []Option{
WithID(NewIDFromBytes([]byte{})),
WithLogger(test.NewLogger(t, true)),
WithBlobStorOptions(o.bsOpts...),
WithMetaBaseOptions(
append([]meta.Option{
meta.WithPath(filepath.Join(rootPath, "meta")), meta.WithEpochState(epochState{})},
metaOptions...)...,
meta.WithPath(filepath.Join(o.rootPath, "meta")), meta.WithEpochState(epochState{})},
o.metaOptions...)...,
),
shard.WithPiloramaOptions(pilorama.WithPath(filepath.Join(rootPath, "pilorama"))),
shard.WithWriteCache(enableWriteCache),
shard.WithWriteCacheOptions(wcOpts),
shard.WithDeletedLockCallback(func(_ context.Context, addresses []oid.Address) {
WithPiloramaOptions(pilorama.WithPath(filepath.Join(o.rootPath, "pilorama"))),
WithWriteCache(enableWriteCache),
WithWriteCacheOptions(o.wcOpts),
WithDeletedLockCallback(func(_ context.Context, addresses []oid.Address) {
sh.HandleDeletedLocks(addresses)
}),
shard.WithExpiredLocksCallback(func(ctx context.Context, epoch uint64, a []oid.Address) {
WithExpiredLocksCallback(func(ctx context.Context, epoch uint64, a []oid.Address) {
sh.HandleExpiredLocks(ctx, epoch, a)
}),
shard.WithGCWorkerPoolInitializer(func(sz int) util.WorkerPool {
WithGCWorkerPoolInitializer(func(sz int) util.WorkerPool {
pool, err := ants.NewPool(sz)
require.NoError(t, err)
return pool
}),
shard.WithGCRemoverSleepInterval(100 * time.Millisecond),
WithGCRemoverSleepInterval(100 * time.Millisecond),
}
opts = append(opts, o.additionalShardOptions...)
sh = shard.New(opts...)
sh = New(opts...)
require.NoError(t, sh.Open())
require.NoError(t, sh.Init(context.Background()))
if !o.dontRelease {
t.Cleanup(func() { releaseShard(sh, t) })
}
return sh
}
func releaseShard(s *shard.Shard, t testing.TB) {
func releaseShard(s *Shard, t testing.TB) {
require.NoError(t, s.Close())
}

View file

@ -1,4 +1,4 @@
package shard_test
package shard
import (
"context"
@ -7,7 +7,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
writecacheconfig "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/config"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/writecache/writecachebbolt"
cidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id/test"
@ -44,13 +43,13 @@ func TestWriteCacheObjectLoss(t *testing.T) {
},
}
sh := newCustomShard(t, dir, true, wcOpts, nil, nil)
sh := newCustomShard(t, true, shardOptions{dontRelease: true, rootPath: dir, wcOpts: wcOpts})
var errG errgroup.Group
for i := range objects {
obj := objects[i]
errG.Go(func() error {
var putPrm shard.PutPrm
var putPrm PutPrm
putPrm.SetObject(obj)
_, err := sh.Put(context.Background(), putPrm)
return err
@ -59,10 +58,9 @@ func TestWriteCacheObjectLoss(t *testing.T) {
require.NoError(t, errG.Wait())
require.NoError(t, sh.Close())
sh = newCustomShard(t, dir, true, wcOpts, nil, nil)
defer releaseShard(sh, t)
sh = newCustomShard(t, true, shardOptions{rootPath: dir, wcOpts: wcOpts})
var getPrm shard.GetPrm
var getPrm GetPrm
for i := range objects {
getPrm.SetAddress(object.AddressOf(objects[i]))