enhance: support specified version manifest write (#46331)

Related to #44956

**Support specified version manifest write**
- Add `baseVersion` parameter to `NewPackedRecordManifestWriter` and
`NewFFIPackedWriter` to support writing manifest based on a specific
version instead of always overwriting the latest
- Add `manifestPath` tracking in `BulkPackWriterV2` to maintain manifest
state across writes
- Add `GetManifestInfo` method to parse existing manifest path and
extract base path and version
- Add `UpdateManifestPath` metacache action to track manifest path in
segment info
- Update `transaction_begin` FFI call to use the specified base version

---------

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
This commit is contained in:
congqixia 2025-12-15 19:49:14 +08:00 committed by GitHub
parent 971085b033
commit 18fbaaca0a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 58 additions and 29 deletions

View File

@ -290,7 +290,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) initStorageV2Segments(rows int,
bw := syncmgr.NewBulkPackWriterV2(mc, sch, cm, s.mockAlloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{ bw := syncmgr.NewBulkPackWriterV2(mc, sch, cm, s.mockAlloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
StorageType: "local", StorageType: "local",
RootPath: rootPath, RootPath: rootPath,
}, columnGroups) }, columnGroups, "")
return bw.Write(context.Background(), pack) return bw.Write(context.Background(), pack)
} }

View File

@ -333,7 +333,7 @@ func (s *MixCompactionTaskStorageV2Suite) initStorageV2Segments(rows int, seed i
bw := syncmgr.NewBulkPackWriterV2(mc, s.meta.Schema, cm, alloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{ bw := syncmgr.NewBulkPackWriterV2(mc, s.meta.Schema, cm, alloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
StorageType: "local", StorageType: "local",
RootPath: rootPath, RootPath: rootPath,
}, columnGroups) }, columnGroups, "")
return bw.Write(context.Background(), pack) return bw.Write(context.Background(), pack)
} }

View File

@ -111,7 +111,7 @@ func (s *NamespaceCompactorTestSuite) setupSortedSegments() {
bw := syncmgr.NewBulkPackWriterV2(mc, s.schema, cm, alloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{ bw := syncmgr.NewBulkPackWriterV2(mc, s.schema, cm, alloc, packed.DefaultWriteBufferSize, 0, &indexpb.StorageConfig{
StorageType: "local", StorageType: "local",
RootPath: rootPath, RootPath: rootPath,
}, columnGroups) }, columnGroups, "")
inserts, _, _, _, _, _, err := bw.Write(context.Background(), pack) inserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Require().NoError(err) s.Require().NoError(err)
s.sortedSegments = append(s.sortedSegments, &datapb.CompactionSegmentBinlogs{ s.sortedSegments = append(s.sortedSegments, &datapb.CompactionSegmentBinlogs{

View File

@ -243,6 +243,12 @@ func SetStartPosRecorded(flag bool) SegmentAction {
} }
} }
func UpdateManifestPath(manifestPath string) SegmentAction {
return func(info *SegmentInfo) {
info.manifestPath = manifestPath
}
}
// MergeSegmentAction is the util function to merge multiple SegmentActions into one. // MergeSegmentAction is the util function to merge multiple SegmentActions into one.
func MergeSegmentAction(actions ...SegmentAction) SegmentAction { func MergeSegmentAction(actions ...SegmentAction) SegmentAction {
return func(info *SegmentInfo) { return func(info *SegmentInfo) {

View File

@ -31,6 +31,7 @@ import (
"github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/storagecommon" "github.com/milvus-io/milvus/internal/storagecommon"
"github.com/milvus-io/milvus/internal/storagev2/packed"
"github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/hookutil"
"github.com/milvus-io/milvus/pkg/v2/common" "github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/log"
@ -51,11 +52,12 @@ type BulkPackWriterV2 struct {
storageConfig *indexpb.StorageConfig storageConfig *indexpb.StorageConfig
columnGroups []storagecommon.ColumnGroup columnGroups []storagecommon.ColumnGroup
manifestPath string
} }
func NewBulkPackWriterV2(metaCache metacache.MetaCache, schema *schemapb.CollectionSchema, chunkManager storage.ChunkManager, func NewBulkPackWriterV2(metaCache metacache.MetaCache, schema *schemapb.CollectionSchema, chunkManager storage.ChunkManager,
allocator allocator.Interface, bufferSize, multiPartUploadSize int64, allocator allocator.Interface, bufferSize, multiPartUploadSize int64,
storageConfig *indexpb.StorageConfig, columnGroups []storagecommon.ColumnGroup, writeRetryOpts ...retry.Option, storageConfig *indexpb.StorageConfig, columnGroups []storagecommon.ColumnGroup, curManifestPath string, writeRetryOpts ...retry.Option,
) *BulkPackWriterV2 { ) *BulkPackWriterV2 {
return &BulkPackWriterV2{ return &BulkPackWriterV2{
BulkPackWriter: &BulkPackWriter{ BulkPackWriter: &BulkPackWriter{
@ -70,6 +72,7 @@ func NewBulkPackWriterV2(metaCache metacache.MetaCache, schema *schemapb.Collect
multiPartUploadSize: multiPartUploadSize, multiPartUploadSize: multiPartUploadSize,
storageConfig: storageConfig, storageConfig: storageConfig,
columnGroups: columnGroups, columnGroups: columnGroups,
manifestPath: curManifestPath,
} }
} }
@ -179,6 +182,27 @@ func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (m
return logs, manifestPath, nil return logs, manifestPath, nil
} }
func (bw *BulkPackWriterV2) GetManifestInfo(pack *SyncPack) (basePath string, version int64, err error) {
// empty info, shall be first write,
// initialize manifestPath with -1 version
if bw.manifestPath == "" {
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID)
logicalPath := path.Join(bw.getRootPath(), common.SegmentInsertLogPath, k)
bucketName := bw.getBucketName()
// if storage config is not passed, use common config
storageType := paramtable.Get().CommonCfg.StorageType.GetValue()
if bw.storageConfig != nil {
storageType = bw.storageConfig.GetStorageType()
}
if storageType != "local" {
basePath = path.Join(bucketName, logicalPath)
}
return basePath, -1, nil
}
return packed.UnmarshalManfestPath(bw.manifestPath)
}
func (bw *BulkPackWriterV2) writeInsertsIntoStorage(_ context.Context, func (bw *BulkPackWriterV2) writeInsertsIntoStorage(_ context.Context,
pluginContextPtr *indexcgopb.StoragePluginContext, pluginContextPtr *indexcgopb.StoragePluginContext,
pack *SyncPack, pack *SyncPack,
@ -203,10 +227,12 @@ func (bw *BulkPackWriterV2) writeInsertsIntoStorage(_ context.Context,
} }
var manifestPath string var manifestPath string
if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() { if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() || bw.manifestPath != "" {
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID) basePath, version, err := bw.GetManifestInfo(pack)
basePath := path.Join(bw.getRootPath(), common.SegmentInsertLogPath, k) if err != nil {
w, err := storage.NewPackedRecordManifestWriter(bucketName, basePath, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr) return nil, "", err
}
w, err := storage.NewPackedRecordManifestWriter(bucketName, basePath, version, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
if err != nil { if err != nil {
return nil, "", err return nil, "", err
} }

View File

@ -143,7 +143,7 @@ func (s *PackWriterV2Suite) TestPackWriterV2_Write() {
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema)).WithDeleteData(deletes) pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema)).WithDeleteData(deletes)
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit) bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
gotInserts, _, _, _, _, _, err := bw.Write(context.Background(), pack) gotInserts, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.NoError(err) s.NoError(err)
@ -162,7 +162,7 @@ func (s *PackWriterV2Suite) TestWriteEmptyInsertData() {
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe() mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName) pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName)
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit) bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
_, _, _, _, _, _, err := bw.Write(context.Background(), pack) _, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.NoError(err) s.NoError(err)
@ -191,7 +191,7 @@ func (s *PackWriterV2Suite) TestNoPkField() {
buf.Append(data) buf.Append(data)
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf}) pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit) bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
_, _, _, _, _, _, err := bw.Write(context.Background(), pack) _, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err) s.Error(err)
@ -208,7 +208,7 @@ func (s *PackWriterV2Suite) TestAllocIDExhausedError() {
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe() mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema)) pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema))
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit) bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
_, _, _, _, _, _, err := bw.Write(context.Background(), pack) _, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err) s.Error(err)
@ -229,7 +229,7 @@ func (s *PackWriterV2Suite) TestWriteInsertDataError() {
buf.Append(data) buf.Append(data)
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf}) pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData([]*storage.InsertData{buf})
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit) bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil, s.currentSplit, "")
_, _, _, _, _, _, err := bw.Write(context.Background(), pack) _, _, _, _, _, _, err := bw.Write(context.Background(), pack)
s.Error(err) s.Error(err)

View File

@ -136,10 +136,9 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
switch segmentInfo.GetStorageVersion() { switch segmentInfo.GetStorageVersion() {
case storage.StorageV2: case storage.StorageV2:
// TODO change to return manifest after integrated
// New sync task means needs to flush data immediately, so do not need to buffer data in writer again. // New sync task means needs to flush data immediately, so do not need to buffer data in writer again.
writer := NewBulkPackWriterV2(t.metacache, t.schema, t.chunkManager, t.allocator, 0, writer := NewBulkPackWriterV2(t.metacache, t.schema, t.chunkManager, t.allocator, 0,
packed.DefaultMultiPartUploadSize, t.storageConfig, columnGroups, t.writeRetryOpts...) packed.DefaultMultiPartUploadSize, t.storageConfig, columnGroups, segmentInfo.ManifestPath(), t.writeRetryOpts...)
t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.manifestPath, t.flushedSize, err = writer.Write(ctx, t.pack) t.insertBinlogs, t.deltaBinlog, t.statsBinlogs, t.bm25Binlogs, t.manifestPath, t.flushedSize, err = writer.Write(ctx, t.pack)
if err != nil { if err != nil {
log.Warn("failed to write sync data with storage v2 format", zap.Error(err)) log.Warn("failed to write sync data with storage v2 format", zap.Error(err))
@ -181,7 +180,7 @@ func (t *SyncTask) Run(ctx context.Context) (err error) {
t.pack.ReleaseData() t.pack.ReleaseData()
actions := []metacache.SegmentAction{metacache.FinishSyncing(t.batchRows)} actions := []metacache.SegmentAction{metacache.FinishSyncing(t.batchRows), metacache.UpdateManifestPath(t.manifestPath)}
if columnGroups != nil { if columnGroups != nil {
actions = append(actions, metacache.UpdateCurrentSplit(columnGroups)) actions = append(actions, metacache.UpdateCurrentSplit(columnGroups))
} }

View File

@ -361,7 +361,10 @@ func (pw *PackedManifestRecordWriter) initWriters(r Record) error {
var err error var err error
k := metautil.JoinIDPath(pw.collectionID, pw.partitionID, pw.segmentID) k := metautil.JoinIDPath(pw.collectionID, pw.partitionID, pw.segmentID)
basePath := path.Join(pw.storageConfig.GetRootPath(), common.SegmentInsertLogPath, k) basePath := path.Join(pw.storageConfig.GetRootPath(), common.SegmentInsertLogPath, k)
pw.writer, err = NewPackedRecordManifestWriter(pw.storageConfig.GetBucketName(), basePath, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext) if pw.storageConfig.StorageType != "local" {
basePath = path.Join(pw.storageConfig.GetBucketName(), basePath)
}
pw.writer, err = NewPackedRecordManifestWriter(pw.storageConfig.GetBucketName(), basePath, -1, pw.schema, pw.bufferSize, pw.multiPartUploadSize, pw.columnGroups, pw.storageConfig, pw.storagePluginContext)
if err != nil { if err != nil {
return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error())) return merr.WrapErrServiceInternal(fmt.Sprintf("can not new packed record writer %s", err.Error()))
} }

View File

@ -296,6 +296,7 @@ func (pw *packedRecordManifestWriter) Close() error {
func NewPackedRecordManifestWriter( func NewPackedRecordManifestWriter(
bucketName string, bucketName string,
basePath string, basePath string,
baseVersion int64,
schema *schemapb.CollectionSchema, schema *schemapb.CollectionSchema,
bufferSize int64, bufferSize int64,
multiPartUploadSize int64, multiPartUploadSize int64,
@ -314,16 +315,8 @@ func NewPackedRecordManifestWriter(
return nil, merr.WrapErrServiceInternal( return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error())) fmt.Sprintf("can not convert collection schema %s to arrow schema: %s", schema.Name, err.Error()))
} }
// if storage config is not passed, use common config
storageType := paramtable.Get().CommonCfg.StorageType.GetValue() writer, err := packed.NewFFIPackedWriter(basePath, baseVersion, arrowSchema, columnGroups, storageConfig, storagePluginContext)
if storageConfig != nil {
storageType = storageConfig.GetStorageType()
}
ffiBasePath := basePath
if storageType != "local" {
ffiBasePath = path.Join(bucketName, basePath)
}
writer, err := packed.NewFFIPackedWriter(ffiBasePath, arrowSchema, columnGroups, storageConfig, storagePluginContext)
if err != nil { if err != nil {
return nil, merr.WrapErrServiceInternal( return nil, merr.WrapErrServiceInternal(
fmt.Sprintf("can not new packed record writer %s", err.Error())) fmt.Sprintf("can not new packed record writer %s", err.Error()))

View File

@ -74,7 +74,7 @@ func createStorageConfig() *indexpb.StorageConfig {
return storageConfig return storageConfig
} }
func NewFFIPackedWriter(basePath string, schema *arrow.Schema, columnGroups []storagecommon.ColumnGroup, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext) (*FFIPackedWriter, error) { func NewFFIPackedWriter(basePath string, baseVersion int64, schema *arrow.Schema, columnGroups []storagecommon.ColumnGroup, storageConfig *indexpb.StorageConfig, storagePluginContext *indexcgopb.StoragePluginContext) (*FFIPackedWriter, error) {
cBasePath := C.CString(basePath) cBasePath := C.CString(basePath)
defer C.free(unsafe.Pointer(cBasePath)) defer C.free(unsafe.Pointer(cBasePath))
@ -143,6 +143,7 @@ func NewFFIPackedWriter(basePath string, schema *arrow.Schema, columnGroups []st
return &FFIPackedWriter{ return &FFIPackedWriter{
basePath: basePath, basePath: basePath,
baseVersion: baseVersion,
cWriterHandle: writerHandle, cWriterHandle: writerHandle,
cProperties: cProperties, cProperties: cProperties,
}, nil }, nil
@ -178,7 +179,7 @@ func (pw *FFIPackedWriter) Close() (string, error) {
// TODO pass version // TODO pass version
// use -1 as latest // use -1 as latest
result = C.transaction_begin(cBasePath, pw.cProperties, &transationHandle, C.int64_t(-1)) result = C.transaction_begin(cBasePath, pw.cProperties, &transationHandle, C.int64_t(pw.baseVersion))
if err := HandleFFIResult(result); err != nil { if err := HandleFFIResult(result); err != nil {
return "", err return "", err
} }

View File

@ -36,6 +36,7 @@ type PackedWriter struct {
type FFIPackedWriter struct { type FFIPackedWriter struct {
basePath string basePath string
baseVersion int64
cWriterHandle C.WriterHandle cWriterHandle C.WriterHandle
cProperties *C.Properties cProperties *C.Properties
} }