Fix GC race conditions in tests #668
14 changed files with 206 additions and 215 deletions
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()))
|
||||
|
|
|
@ -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())
|
||||
|
||||
|
|
|
@ -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())
|
||||
}
|
||||
|
|
|
@ -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]))
|
||||
|
|
Loading…
Add table
Reference in a new issue