mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
enhance: Forbid writing V1 format and always use StorageV2 (#46791)
Related to #46595 Remove the EnableStorageV2 config option and enforce StorageV2 format across all write paths including compaction, import, write buffer, and streaming segment allocation. V1 format write tests are now skipped as writing V1 format is no longer supported. --------- Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
This commit is contained in:
parent
56e82c78e1
commit
fa2c3c404c
@ -1016,7 +1016,6 @@ common:
|
||||
warn: 1000 # minimum milliseconds for printing durations in warn level
|
||||
maxWLockConditionalWaitTime: 600 # maximum seconds for waiting wlock conditional
|
||||
storage:
|
||||
enablev2: true
|
||||
stv2:
|
||||
splitSystemColumn:
|
||||
enabled: true # enable split system column policy in storage v2
|
||||
|
||||
@ -35,12 +35,8 @@ type Params struct {
|
||||
}
|
||||
|
||||
func GenParams() Params {
|
||||
storageVersion := storage.StorageV1
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
return Params{
|
||||
StorageVersion: storageVersion,
|
||||
StorageVersion: storage.StorageV2,
|
||||
BinLogMaxSize: paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsUint64(),
|
||||
UseMergeSort: paramtable.Get().DataNodeCfg.UseMergeSort.GetAsBool(),
|
||||
MaxSegmentMergeSort: paramtable.Get().DataNodeCfg.MaxSegmentMergeSort.GetAsInt(),
|
||||
|
||||
@ -34,12 +34,8 @@ func TestGetJSONParams(t *testing.T) {
|
||||
var result Params
|
||||
err = json.Unmarshal([]byte(jsonStr), &result)
|
||||
assert.NoError(t, err)
|
||||
storageVersion := storage.StorageV1
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
assert.Equal(t, Params{
|
||||
StorageVersion: storageVersion,
|
||||
StorageVersion: storage.StorageV2,
|
||||
BinLogMaxSize: paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsUint64(),
|
||||
UseMergeSort: paramtable.Get().DataNodeCfg.UseMergeSort.GetAsBool(),
|
||||
MaxSegmentMergeSort: paramtable.Get().DataNodeCfg.MaxSegmentMergeSort.GetAsInt(),
|
||||
@ -84,12 +80,8 @@ func TestGetParamsFromJSON_EmptyJSON(t *testing.T) {
|
||||
emptyJSON := ``
|
||||
result, err := ParseParamsFromJSON(emptyJSON)
|
||||
assert.NoError(t, err)
|
||||
storageVersion := storage.StorageV1
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
assert.Equal(t, Params{
|
||||
StorageVersion: storageVersion,
|
||||
StorageVersion: storage.StorageV2,
|
||||
BinLogMaxSize: paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsUint64(),
|
||||
UseMergeSort: paramtable.Get().DataNodeCfg.UseMergeSort.GetAsBool(),
|
||||
MaxSegmentMergeSort: paramtable.Get().DataNodeCfg.MaxSegmentMergeSort.GetAsInt(),
|
||||
|
||||
@ -171,11 +171,6 @@ func AssignSegments(job ImportJob, task ImportTask, alloc allocator.Allocator, m
|
||||
segmentLevel = datapb.SegmentLevel_L0
|
||||
}
|
||||
|
||||
storageVersion := storage.StorageV1
|
||||
if Params.CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
|
||||
// alloc new segments
|
||||
segments := make([]int64, 0)
|
||||
addSegment := func(vchannel string, partitionID int64, size int64) error {
|
||||
@ -184,7 +179,7 @@ func AssignSegments(job ImportJob, task ImportTask, alloc allocator.Allocator, m
|
||||
for size > 0 {
|
||||
segmentInfo, err := AllocImportSegment(ctx, alloc, meta,
|
||||
task.GetJobID(), task.GetTaskID(), task.GetCollectionID(),
|
||||
partitionID, vchannel, job.GetDataTs(), segmentLevel, storageVersion)
|
||||
partitionID, vchannel, job.GetDataTs(), segmentLevel, storage.StorageV2)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@ -360,10 +355,6 @@ func AssembleImportRequest(task ImportTask, job ImportJob, meta *meta, alloc all
|
||||
return fileStat.GetImportFile()
|
||||
})
|
||||
|
||||
storageVersion := storage.StorageV1
|
||||
if Params.CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
req := &datapb.ImportRequest{
|
||||
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
|
||||
JobID: task.GetJobID(),
|
||||
@ -379,7 +370,7 @@ func AssembleImportRequest(task ImportTask, job ImportJob, meta *meta, alloc all
|
||||
RequestSegments: requestSegments,
|
||||
StorageConfig: createStorageConfig(),
|
||||
TaskSlot: task.GetTaskSlot(),
|
||||
StorageVersion: storageVersion,
|
||||
StorageVersion: storage.StorageV2,
|
||||
PluginContext: GetReadPluginContext(job.GetOptions()),
|
||||
UseLoonFfi: Params.CommonCfg.UseLoonFFI.GetAsBool(),
|
||||
}
|
||||
|
||||
@ -53,12 +53,10 @@ type ClusteringCompactionTaskStorageV2Suite struct {
|
||||
|
||||
func (s *ClusteringCompactionTaskStorageV2Suite) SetupTest() {
|
||||
s.setupTest()
|
||||
paramtable.Get().Save("common.storage.enableV2", "true")
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
}
|
||||
|
||||
func (s *ClusteringCompactionTaskStorageV2Suite) TearDownTest() {
|
||||
paramtable.Get().Reset("common.storage.enableV2")
|
||||
os.RemoveAll(paramtable.Get().LocalStorageCfg.Path.GetValue() + "insert_log")
|
||||
os.RemoveAll(paramtable.Get().LocalStorageCfg.Path.GetValue() + "delta_log")
|
||||
os.RemoveAll(paramtable.Get().LocalStorageCfg.Path.GetValue() + "stats_log")
|
||||
@ -142,80 +140,6 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2To
|
||||
)
|
||||
}
|
||||
|
||||
func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2ToV1Format() {
|
||||
paramtable.Get().Save("common.storage.enableV2", "false")
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
|
||||
var segmentID int64 = 1001
|
||||
|
||||
fBinlogs, deltalogs, _, _, _, _, err := s.initStorageV2Segments(10240, segmentID)
|
||||
s.NoError(err)
|
||||
|
||||
dblobs, err := getInt64DeltaBlobs(
|
||||
1,
|
||||
[]int64{100},
|
||||
[]uint64{tsoutil.ComposeTSByTime(getMilvusBirthday().Add(time.Second), 0)},
|
||||
)
|
||||
s.Require().NoError(err)
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, []string{deltalogs.GetBinlogs()[0].GetLogPath()}).
|
||||
Return([][]byte{dblobs.GetValue()}, nil).Once()
|
||||
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{
|
||||
{
|
||||
CollectionID: 1,
|
||||
SegmentID: segmentID,
|
||||
FieldBinlogs: storage.SortFieldBinlogs(fBinlogs),
|
||||
Deltalogs: []*datapb.FieldBinlog{deltalogs},
|
||||
StorageVersion: storage.StorageV2,
|
||||
},
|
||||
}
|
||||
|
||||
s.task.plan.Schema = genCollectionSchema()
|
||||
s.task.plan.ClusteringKeyField = 100
|
||||
s.task.plan.PreferSegmentRows = 2048
|
||||
s.task.plan.MaxSegmentRows = 2048
|
||||
s.task.plan.MaxSize = 1024 * 1024 * 1024 // max segment size = 1GB, we won't touch this value
|
||||
s.task.plan.PreAllocatedSegmentIDs = &datapb.IDRange{
|
||||
Begin: 1,
|
||||
End: 101,
|
||||
}
|
||||
s.task.plan.PreAllocatedLogIDs = &datapb.IDRange{
|
||||
Begin: 200,
|
||||
End: 2000,
|
||||
}
|
||||
|
||||
compactionResult, err := s.task.Compact()
|
||||
s.Require().NoError(err)
|
||||
s.Equal(5, len(s.task.clusterBuffers))
|
||||
s.Equal(5, len(compactionResult.GetSegments()))
|
||||
totalBinlogNum := 0
|
||||
totalRowNum := int64(0)
|
||||
for _, fb := range compactionResult.GetSegments()[0].GetInsertLogs() {
|
||||
for _, b := range fb.GetBinlogs() {
|
||||
totalBinlogNum++
|
||||
if fb.GetFieldID() == 100 {
|
||||
totalRowNum += b.GetEntriesNum()
|
||||
}
|
||||
}
|
||||
}
|
||||
statsBinlogNum := 0
|
||||
statsRowNum := int64(0)
|
||||
for _, sb := range compactionResult.GetSegments()[0].GetField2StatslogPaths() {
|
||||
for _, b := range sb.GetBinlogs() {
|
||||
statsBinlogNum++
|
||||
statsRowNum += b.GetEntriesNum()
|
||||
}
|
||||
}
|
||||
s.Equal(1, totalBinlogNum/len(s.plan.Schema.GetFields()))
|
||||
s.Equal(1, statsBinlogNum)
|
||||
s.Equal(totalRowNum, statsRowNum)
|
||||
s.EqualValues(10239,
|
||||
lo.SumBy(compactionResult.GetSegments(), func(seg *datapb.CompactionSegment) int64 {
|
||||
return seg.GetNumOfRows()
|
||||
}),
|
||||
)
|
||||
}
|
||||
|
||||
func (s *ClusteringCompactionTaskStorageV2Suite) TestCompactionWithBM25Function() {
|
||||
// 8 + 8 + 8 + 7 + 8 = 39
|
||||
// 39*1024 = 39936
|
||||
|
||||
@ -58,14 +58,12 @@ type MixCompactionTaskStorageV2Suite struct {
|
||||
func (s *MixCompactionTaskStorageV2Suite) SetupTest() {
|
||||
s.setupTest()
|
||||
paramtable.Get().Save("common.storageType", "local")
|
||||
paramtable.Get().Save("common.storage.enableV2", "true")
|
||||
initcore.InitStorageV2FileSystem(paramtable.Get())
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
}
|
||||
|
||||
func (s *MixCompactionTaskStorageV2Suite) TearDownTest() {
|
||||
paramtable.Get().Reset("common.storageType")
|
||||
paramtable.Get().Reset("common.storage.enableV2")
|
||||
os.RemoveAll(paramtable.Get().LocalStorageCfg.Path.GetValue() + "insert_log")
|
||||
os.RemoveAll(paramtable.Get().LocalStorageCfg.Path.GetValue() + "delta_log")
|
||||
os.RemoveAll(paramtable.Get().LocalStorageCfg.Path.GetValue() + "stats_log")
|
||||
@ -181,43 +179,6 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV2Format() {
|
||||
s.Empty(segment.Deltalogs)
|
||||
}
|
||||
|
||||
func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV1Format() {
|
||||
paramtable.Get().Save("common.storage.enableV2", "false")
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
|
||||
alloc := allocator.NewLocalAllocator(7777777, math.MaxInt64)
|
||||
|
||||
s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0)
|
||||
|
||||
v2Segments := []int64{10, 11}
|
||||
for _, segID := range v2Segments {
|
||||
binlogs, _, _, _, _, _, err := s.initStorageV2Segments(1, segID, alloc)
|
||||
s.NoError(err)
|
||||
s.task.plan.SegmentBinlogs = append(s.task.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
|
||||
CollectionID: 1,
|
||||
SegmentID: segID,
|
||||
FieldBinlogs: storage.SortFieldBinlogs(binlogs),
|
||||
Deltalogs: []*datapb.FieldBinlog{},
|
||||
StorageVersion: storage.StorageV2,
|
||||
})
|
||||
}
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err)
|
||||
s.NotNil(result)
|
||||
|
||||
s.Equal(s.task.plan.GetPlanID(), result.GetPlanID())
|
||||
s.Equal(1, len(result.GetSegments()))
|
||||
|
||||
segment := result.GetSegments()[0]
|
||||
s.EqualValues(19531, segment.GetSegmentID())
|
||||
s.EqualValues(2, segment.GetNumOfRows())
|
||||
// each field has only one insert log for storage v1
|
||||
s.EqualValues(len(s.task.plan.Schema.Fields), len(segment.GetInsertLogs()))
|
||||
s.NotEmpty(segment.Field2StatslogPaths)
|
||||
s.Empty(segment.Deltalogs)
|
||||
}
|
||||
|
||||
func (s *MixCompactionTaskStorageV2Suite) TestCompactTwoToOne() {
|
||||
s.prepareCompactTwoToOneSegments()
|
||||
result, err := s.task.Compact()
|
||||
|
||||
@ -47,6 +47,7 @@ import (
|
||||
)
|
||||
|
||||
func TestMixCompactionTaskSuite(t *testing.T) {
|
||||
t.Skip("v1 format shall not be written anymore")
|
||||
suite.Run(t, new(MixCompactionTaskStorageV1Suite))
|
||||
}
|
||||
|
||||
|
||||
@ -40,7 +40,6 @@ type NamespaceCompactorTestSuite struct {
|
||||
func (s *NamespaceCompactorTestSuite) SetupSuite() {
|
||||
paramtable.Get().Init(paramtable.NewBaseTable())
|
||||
paramtable.Get().Save("common.storageType", "local")
|
||||
paramtable.Get().Save("common.storage.enableV2", "true")
|
||||
initcore.InitStorageV2FileSystem(paramtable.Get())
|
||||
|
||||
s.binlogIO = mock_util.NewMockBinlogIO(s.T())
|
||||
|
||||
@ -514,10 +514,6 @@ func (wb *writeBufferBase) CreateNewGrowingSegment(partitionID int64, segmentID
|
||||
_, ok := wb.metaCache.GetSegmentByID(segmentID)
|
||||
// new segment
|
||||
if !ok {
|
||||
storageVersion := storage.StorageV1
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
segmentInfo := &datapb.SegmentInfo{
|
||||
ID: segmentID,
|
||||
PartitionID: partitionID,
|
||||
@ -525,7 +521,7 @@ func (wb *writeBufferBase) CreateNewGrowingSegment(partitionID int64, segmentID
|
||||
InsertChannel: wb.channelName,
|
||||
StartPosition: startPos,
|
||||
State: commonpb.SegmentState_Growing,
|
||||
StorageVersion: storageVersion,
|
||||
StorageVersion: storage.StorageV2,
|
||||
}
|
||||
// set manifest path when creating segment
|
||||
if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() {
|
||||
@ -540,7 +536,7 @@ func (wb *writeBufferBase) CreateNewGrowingSegment(partitionID int64, segmentID
|
||||
wb.metaCache.AddSegment(segmentInfo, func(_ *datapb.SegmentInfo) pkoracle.PkStat {
|
||||
return pkoracle.NewBloomFilterSetWithBatchSize(wb.getEstBatchSize())
|
||||
}, metacache.NewBM25StatsFactory, metacache.SetStartPosRecorded(false))
|
||||
log.Info("add growing segment", zap.Int64("segmentID", segmentID), zap.String("channel", wb.channelName), zap.Int64("storage version", storageVersion))
|
||||
log.Info("add growing segment", zap.Int64("segmentID", segmentID), zap.String("channel", wb.channelName), zap.Int64("storage version", storage.StorageV2))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -926,7 +926,6 @@ func TestProxy(t *testing.T) {
|
||||
params.Save(params.CommonCfg.SessionRetryTimes.Key, "500")
|
||||
params.Save(params.CommonCfg.GracefulStopTimeout.Key, "3600")
|
||||
|
||||
params.Save(params.CommonCfg.EnableStorageV2.Key, "true")
|
||||
params.RootCoordGrpcServerCfg.IP = "localhost"
|
||||
params.QueryCoordGrpcServerCfg.IP = "localhost"
|
||||
params.DataCoordGrpcServerCfg.IP = "localhost"
|
||||
|
||||
@ -14,7 +14,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
)
|
||||
|
||||
// asyncAllocSegment allocates a new growing segment asynchronously.
|
||||
@ -108,10 +107,6 @@ func (w *segmentAllocWorker) generateNewGrowingSegmentMessage() error {
|
||||
w.Logger().Warn("failed to allocate segment id", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
storageVersion := storage.StorageV1
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
// Getnerate growing segment limitation.
|
||||
limitation := getSegmentLimitationPolicy().GenerateLimitation(datapb.SegmentLevel_L1)
|
||||
// Create a new segment by sending a create segment message into wal directly.
|
||||
@ -121,7 +116,7 @@ func (w *segmentAllocWorker) generateNewGrowingSegmentMessage() error {
|
||||
CollectionId: w.collectionID,
|
||||
PartitionId: w.partitionID,
|
||||
SegmentId: int64(segmentID),
|
||||
StorageVersion: storageVersion,
|
||||
StorageVersion: storage.StorageV2,
|
||||
MaxRows: limitation.SegmentRows,
|
||||
MaxSegmentSize: limitation.SegmentSize,
|
||||
Level: datapb.SegmentLevel_L1,
|
||||
|
||||
@ -155,11 +155,9 @@ func doInitQueryNodeOnce(ctx context.Context) error {
|
||||
return err
|
||||
}
|
||||
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
err = InitStorageV2FileSystem(paramtable.Get())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = InitStorageV2FileSystem(paramtable.Get())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = InitMmapManager(paramtable.Get(), nodeID)
|
||||
|
||||
@ -283,7 +283,6 @@ type commonConfig struct {
|
||||
MaxWLockConditionalWaitTime ParamItem `refreshable:"true"`
|
||||
|
||||
// storage v2
|
||||
EnableStorageV2 ParamItem `refreshable:"false"`
|
||||
Stv2SplitSystemColumn ParamItem `refreshable:"true"`
|
||||
Stv2SystemColumnIncludePK ParamItem `refreshable:"true"`
|
||||
Stv2SystemColumnIncludePartitionKey ParamItem `refreshable:"true"`
|
||||
@ -947,14 +946,6 @@ Large numeric passwords require double quotes to avoid yaml parsing precision is
|
||||
}
|
||||
p.MaxWLockConditionalWaitTime.Init(base.mgr)
|
||||
|
||||
p.EnableStorageV2 = ParamItem{
|
||||
Key: "common.storage.enablev2",
|
||||
Version: "2.3.1",
|
||||
DefaultValue: "true",
|
||||
Export: true,
|
||||
}
|
||||
p.EnableStorageV2.Init(base.mgr)
|
||||
|
||||
p.UseLoonFFI = ParamItem{
|
||||
Key: "common.storage.useLoonFFI",
|
||||
Version: "2.6.7",
|
||||
|
||||
@ -240,7 +240,7 @@ func (s *CompactionSuite) TestMixCompaction() {
|
||||
defer cancel()
|
||||
|
||||
collectionName := "TestCompaction_" + funcutil.GenRandomStr()
|
||||
s.assertMixCompaction(ctx, collectionName, paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool())
|
||||
s.assertMixCompaction(ctx, collectionName, true)
|
||||
s.assertQuery(ctx, collectionName)
|
||||
|
||||
// drop collection
|
||||
@ -254,9 +254,8 @@ func (s *CompactionSuite) TestMixCompaction() {
|
||||
}
|
||||
|
||||
func (s *CompactionSuite) TestMixCompactionV2() {
|
||||
s.T().Skip("skip v2 compaction test")
|
||||
// s.T().Skip("skip v2 compaction test")
|
||||
revertGuard := s.Cluster.MustModifyMilvusConfig(map[string]string{
|
||||
paramtable.Get().CommonCfg.EnableStorageV2.Key: "true",
|
||||
paramtable.Get().DataCoordCfg.IndexBasedCompaction.Key: "false",
|
||||
})
|
||||
defer revertGuard()
|
||||
@ -265,5 +264,5 @@ func (s *CompactionSuite) TestMixCompactionV2() {
|
||||
defer cancel()
|
||||
|
||||
collectionName := "TestCompaction_" + funcutil.GenRandomStr()
|
||||
s.assertMixCompaction(ctx, collectionName, paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool())
|
||||
s.assertMixCompaction(ctx, collectionName, true)
|
||||
}
|
||||
|
||||
@ -30,7 +30,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metric"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
"github.com/milvus-io/milvus/tests/integration"
|
||||
)
|
||||
@ -49,11 +48,6 @@ type TestArrayStructSuite struct {
|
||||
}
|
||||
|
||||
func (s *TestArrayStructSuite) run() {
|
||||
revertGuard := s.Cluster.MustModifyMilvusConfig(map[string]string{
|
||||
paramtable.Get().CommonCfg.EnableStorageV2.Key: "true",
|
||||
})
|
||||
defer revertGuard()
|
||||
|
||||
ctx, cancel := context.WithCancel(s.Cluster.GetContext())
|
||||
defer cancel()
|
||||
|
||||
|
||||
@ -31,7 +31,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metric"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/tests/integration"
|
||||
)
|
||||
|
||||
@ -122,11 +121,6 @@ func (s *HelloMilvusSuite) TestInsertStorageV2() {
|
||||
ctx, cancel := context.WithCancel(c.GetContext())
|
||||
defer cancel()
|
||||
|
||||
revertGuard := s.Cluster.MustModifyMilvusConfig(map[string]string{
|
||||
paramtable.Get().CommonCfg.EnableStorageV2.Key: "true",
|
||||
})
|
||||
defer revertGuard()
|
||||
|
||||
prefix := "TestInsert"
|
||||
dbName := ""
|
||||
collectionName := prefix + funcutil.GenRandomStr()
|
||||
|
||||
@ -30,6 +30,7 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
@ -252,11 +253,6 @@ func (s *BulkInsertSuite) PrepareSourceCollection(dim int, dmlGroup *DMLGroup) *
|
||||
collectionID := showCollectionsResp.GetCollectionIds()[0]
|
||||
partitionID := showPartitionsResp.GetPartitionIDs()[0]
|
||||
|
||||
storageVersion := 0
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = 2
|
||||
}
|
||||
|
||||
return &SourceCollectionInfo{
|
||||
collectionID: collectionID,
|
||||
partitionID: partitionID,
|
||||
@ -267,7 +263,7 @@ func (s *BulkInsertSuite) PrepareSourceCollection(dim int, dmlGroup *DMLGroup) *
|
||||
return segment.GetID()
|
||||
}),
|
||||
insertedIDs: totalInsertedIDs,
|
||||
storageVersion: storageVersion,
|
||||
storageVersion: int(storage.StorageV2),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user