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:
congqixia 2026-01-06 11:55:23 +08:00 committed by GitHub
parent 56e82c78e1
commit fa2c3c404c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
17 changed files with 17 additions and 192 deletions

View File

@ -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

View File

@ -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(),

View File

@ -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(),

View File

@ -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(),
}

View File

@ -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

View File

@ -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()

View File

@ -47,6 +47,7 @@ import (
)
func TestMixCompactionTaskSuite(t *testing.T) {
t.Skip("v1 format shall not be written anymore")
suite.Run(t, new(MixCompactionTaskStorageV1Suite))
}

View File

@ -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())

View File

@ -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))
}
}

View File

@ -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"

View File

@ -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,

View File

@ -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)

View File

@ -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",

View File

@ -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)
}

View File

@ -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()

View File

@ -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()

View File

@ -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),
}
}