mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
Cherry-pick from master pr: #45061 #45488 #45803 #46017 #44991 #45132 #45723 #45726 #45798 #45897 #45918 #44998 This feature integrates the Storage V2 (Loon) FFI interface as a unified storage layer for segment loading and index building in Milvus. It enables manifest-based data access, replacing the traditional binlog-based approach with a more efficient columnar storage format. Key changes: ### Segment Self-Managed Loading Architecture - Move segment loading orchestration from Go layer to C++ segcore - Add NewSegmentWithLoadInfo() API for passing load info during segment creation - Implement SetLoadInfo() and Load() methods in SegmentInterface - Support parallel loading of indexed and non-indexed fields - Enable both sealed and growing segments to self-manage loading ### Storage V2 FFI Integration - Integrate milvus-storage library's FFI interface for packed columnar data - Add manifest path support throughout the data path (SegmentInfo, LoadInfo) - Implement ManifestReader for generating manifests from binlogs - Support zero-copy data exchange using Arrow C Data Interface - Add ToCStorageConfig() for Go-to-C storage config conversion ### Manifest-Based Index Building - Extend FileManagerContext to carry loon_ffi_properties - Implement GetFieldDatasFromManifest() using Arrow C Stream interface - Support manifest-based reading in DiskFileManagerImpl and MemFileManagerImpl - Add fallback to traditional segment insert files when manifest unavailable ### Compaction Pipeline Updates - Include manifest path in all compaction task builders (clustering, L0, mix) - Update BulkPackWriterV2 to return manifest path - Propagate manifest metadata through compaction pipeline ### Configuration & Protocol - Add common.storageV2.useLoonFFI config option (default: false) - Add manifest_path field to SegmentLoadInfo and related proto messages - Add manifest field to compaction segment messages ### Bug Fixes - Fix mmap settings not applied during segment load (key typo fix) - Populate index info after segment loading to prevent redundant load tasks - Fix memory corruption by removing premature transaction handle destruction Related issues: #44956, #45060, #39173 ## Individual Cherry-Picked Commits 1. **e1c923b5cc** - fix: apply mmap settings correctly during segment load (#46017) 2. **63b912370b** - enhance: use milvus-storage internal C++ Reader API for Loon FFI (#45897) 3. **bfc192faa5** - enhance: Resolve issues integrating loon FFI (#45918) 4. **fb18564631** - enhance: support manifest-based index building with Loon FFI reader (#45726) 5. **b9ec2392b9** - enhance: integrate StorageV2 FFI interface for manifest-based segment loading (#45798) 6. **66db3c32e6** - enhance: integrate Storage V2 FFI interface for unified storage access (#45723) 7. **ae789273ac** - fix: populate index info after segment loading to prevent redundant load tasks (#45803) 8. **49688b0be2** - enhance: Move segment loading logic from Go layer to segcore for self-managed loading (#45488) 9. **5b2df88bac** - enhance: [StorageV2] Integrate FFI interface for packed reader (#45132) 10. **91ff5706ac** - enhance: [StorageV2] add manifest path support for FFI integration (#44991) 11. **2192bb4a85** - enhance: add NewSegmentWithLoadInfo API to support segment self-managed loading (#45061) 12. **4296b01da0** - enhance: update delta log serialization APIs to integrate storage V2 (#44998) ## Technical Details ### Architecture Changes - **Before**: Go layer orchestrated segment loading, making multiple CGO calls - **After**: Segments autonomously manage loading in C++ layer with single entry point ### Storage Access Pattern - **Before**: Read individual binlog files through Go storage layer - **After**: Read manifest file that references packed columnar data via FFI ### Benefits - Reduced cross-language call overhead - Better resource management at C++ level - Improved I/O performance through batched streaming reads - Cleaner separation of concerns between Go and C++ layers - Foundation for proactive schema evolution handling --------- Signed-off-by: Ted Xu <ted.xu@zilliz.com> Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> Co-authored-by: Ted Xu <ted.xu@zilliz.com>
410 lines
12 KiB
Go
410 lines
12 KiB
Go
// Licensed to the LF AI & Data foundation under one
|
|
// or more contributor license agreements. See the NOTICE file
|
|
// distributed with this work for additional information
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
// to you under the Apache License, Version 2.0 (the
|
|
// "License"); you may not use this file except in compliance
|
|
// with the License. You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package syncmgr
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
"path"
|
|
|
|
"github.com/apache/arrow/go/v17/arrow"
|
|
"github.com/apache/arrow/go/v17/arrow/array"
|
|
"github.com/apache/arrow/go/v17/arrow/memory"
|
|
"github.com/samber/lo"
|
|
"go.uber.org/zap"
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
|
"github.com/milvus-io/milvus/internal/allocator"
|
|
"github.com/milvus-io/milvus/internal/flushcommon/metacache"
|
|
"github.com/milvus-io/milvus/internal/storage"
|
|
"github.com/milvus-io/milvus/pkg/v2/common"
|
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
|
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/retry"
|
|
)
|
|
|
|
type PackWriter interface {
|
|
Write(ctx context.Context, pack *SyncPack) (
|
|
inserts []*datapb.Binlog, deletes *datapb.Binlog, stats *datapb.Binlog, bm25Stats *datapb.Binlog,
|
|
size int64, err error)
|
|
}
|
|
|
|
type BulkPackWriter struct {
|
|
metaCache metacache.MetaCache
|
|
schema *schemapb.CollectionSchema
|
|
chunkManager storage.ChunkManager
|
|
allocator allocator.Interface
|
|
writeRetryOpts []retry.Option
|
|
|
|
// prefetched log ids
|
|
ids []int64
|
|
sizeWritten int64
|
|
}
|
|
|
|
func NewBulkPackWriter(metaCache metacache.MetaCache,
|
|
schema *schemapb.CollectionSchema,
|
|
chunkManager storage.ChunkManager,
|
|
allocator allocator.Interface, writeRetryOpts ...retry.Option,
|
|
) *BulkPackWriter {
|
|
return &BulkPackWriter{
|
|
metaCache: metaCache,
|
|
schema: schema,
|
|
chunkManager: chunkManager,
|
|
allocator: allocator,
|
|
writeRetryOpts: writeRetryOpts,
|
|
}
|
|
}
|
|
|
|
func (bw *BulkPackWriter) Write(ctx context.Context, pack *SyncPack) (
|
|
inserts map[int64]*datapb.FieldBinlog,
|
|
deltas *datapb.FieldBinlog,
|
|
stats map[int64]*datapb.FieldBinlog,
|
|
bm25Stats map[int64]*datapb.FieldBinlog,
|
|
size int64,
|
|
err error,
|
|
) {
|
|
err = bw.prefetchIDs(pack)
|
|
if err != nil {
|
|
log.Warn("failed allocate ids for sync task", zap.Error(err))
|
|
return
|
|
}
|
|
|
|
if inserts, err = bw.writeInserts(ctx, pack); err != nil {
|
|
log.Error("failed to write insert data", zap.Error(err))
|
|
return
|
|
}
|
|
if stats, err = bw.writeStats(ctx, pack); err != nil {
|
|
log.Error("failed to process stats blob", zap.Error(err))
|
|
return
|
|
}
|
|
if deltas, err = bw.writeDelta(ctx, pack); err != nil {
|
|
log.Error("failed to process delta blob", zap.Error(err))
|
|
return
|
|
}
|
|
if bm25Stats, err = bw.writeBM25Stasts(ctx, pack); err != nil {
|
|
log.Error("failed to process bm25 stats blob", zap.Error(err))
|
|
return
|
|
}
|
|
|
|
size = bw.sizeWritten
|
|
|
|
return
|
|
}
|
|
|
|
// prefetchIDs pre-allcates ids depending on the number of blobs current task contains.
|
|
func (bw *BulkPackWriter) prefetchIDs(pack *SyncPack) error {
|
|
totalIDCount := 0
|
|
if len(pack.insertData) > 0 {
|
|
totalIDCount += len(pack.insertData[0].Data) * 2 // binlogs and statslogs
|
|
}
|
|
if pack.isFlush {
|
|
totalIDCount++ // merged stats log
|
|
}
|
|
if pack.deltaData != nil {
|
|
totalIDCount++
|
|
}
|
|
if pack.bm25Stats != nil {
|
|
totalIDCount += len(pack.bm25Stats)
|
|
if pack.isFlush {
|
|
totalIDCount++ // merged bm25 stats
|
|
}
|
|
}
|
|
|
|
if totalIDCount == 0 {
|
|
return nil
|
|
}
|
|
start, _, err := bw.allocator.Alloc(uint32(totalIDCount))
|
|
if err != nil {
|
|
return err
|
|
}
|
|
bw.ids = lo.RangeFrom(start, totalIDCount)
|
|
return nil
|
|
}
|
|
|
|
func (bw *BulkPackWriter) nextID() int64 {
|
|
if len(bw.ids) == 0 {
|
|
panic("pre-fetched ids exhausted")
|
|
}
|
|
r := bw.ids[0]
|
|
bw.ids = bw.ids[1:]
|
|
return r
|
|
}
|
|
|
|
func (bw *BulkPackWriter) writeLog(ctx context.Context, blob *storage.Blob,
|
|
root, p string, pack *SyncPack,
|
|
) (*datapb.Binlog, error) {
|
|
key := path.Join(bw.chunkManager.RootPath(), root, p)
|
|
err := retry.Do(ctx, func() error {
|
|
return bw.chunkManager.Write(ctx, key, blob.Value)
|
|
}, bw.writeRetryOpts...)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
size := int64(len(blob.GetValue()))
|
|
bw.sizeWritten += size
|
|
return &datapb.Binlog{
|
|
EntriesNum: blob.RowNum,
|
|
TimestampFrom: pack.tsFrom,
|
|
TimestampTo: pack.tsTo,
|
|
LogPath: key,
|
|
LogSize: size,
|
|
MemorySize: blob.MemorySize,
|
|
}, nil
|
|
}
|
|
|
|
func (bw *BulkPackWriter) writeInserts(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, error) {
|
|
if len(pack.insertData) == 0 {
|
|
return make(map[int64]*datapb.FieldBinlog), nil
|
|
}
|
|
|
|
serializer, err := NewStorageSerializer(bw.metaCache, bw.schema)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
binlogBlobs, err := serializer.serializeBinlog(ctx, pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
logs := make(map[int64]*datapb.FieldBinlog)
|
|
for fieldID, blob := range binlogBlobs {
|
|
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID, fieldID, bw.nextID())
|
|
binlog, err := bw.writeLog(ctx, blob, common.SegmentInsertLogPath, k, pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
logs[fieldID] = &datapb.FieldBinlog{
|
|
FieldID: fieldID,
|
|
Binlogs: []*datapb.Binlog{binlog},
|
|
}
|
|
}
|
|
return logs, nil
|
|
}
|
|
|
|
func (bw *BulkPackWriter) writeStats(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, error) {
|
|
if len(pack.insertData) == 0 {
|
|
// TODO: we should not skip here, if the flush operation don't carry any insert data,
|
|
// the merge stats operation will be skipped, which is a bad case.
|
|
return make(map[int64]*datapb.FieldBinlog), nil
|
|
}
|
|
serializer, err := NewStorageSerializer(bw.metaCache, bw.schema)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
singlePKStats, batchStatsBlob, err := serializer.serializeStatslog(pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
actions := []metacache.SegmentAction{metacache.RollStats(singlePKStats)}
|
|
bw.metaCache.UpdateSegments(metacache.MergeSegmentAction(actions...), metacache.WithSegmentIDs(pack.segmentID))
|
|
|
|
pkFieldID := serializer.pkField.GetFieldID()
|
|
binlogs := make([]*datapb.Binlog, 0)
|
|
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID, pkFieldID, bw.nextID())
|
|
if binlog, err := bw.writeLog(ctx, batchStatsBlob, common.SegmentStatslogPath, k, pack); err != nil {
|
|
return nil, err
|
|
} else {
|
|
binlogs = append(binlogs, binlog)
|
|
}
|
|
|
|
if pack.isFlush && pack.level != datapb.SegmentLevel_L0 {
|
|
mergedStatsBlob, err := serializer.serializeMergedPkStats(pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID, pkFieldID, int64(storage.CompoundStatsType))
|
|
binlog, err := bw.writeLog(ctx, mergedStatsBlob, common.SegmentStatslogPath, k, pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
binlogs = append(binlogs, binlog)
|
|
}
|
|
|
|
logs := make(map[int64]*datapb.FieldBinlog)
|
|
logs[pkFieldID] = &datapb.FieldBinlog{
|
|
FieldID: pkFieldID,
|
|
Binlogs: binlogs,
|
|
}
|
|
return logs, nil
|
|
}
|
|
|
|
func (bw *BulkPackWriter) writeBM25Stasts(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, error) {
|
|
if len(pack.bm25Stats) == 0 {
|
|
// TODO: we should not skip here, if the flush operation don't carry any insert data,
|
|
// the merge stats operation will be skipped, which is a bad case.
|
|
return make(map[int64]*datapb.FieldBinlog), nil
|
|
}
|
|
|
|
serializer, err := NewStorageSerializer(bw.metaCache, bw.schema)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
bm25Blobs, err := serializer.serializeBM25Stats(pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
logs := make(map[int64]*datapb.FieldBinlog)
|
|
for fieldID, blob := range bm25Blobs {
|
|
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID, fieldID, bw.nextID())
|
|
binlog, err := bw.writeLog(ctx, blob, common.SegmentBm25LogPath, k, pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
logs[fieldID] = &datapb.FieldBinlog{
|
|
FieldID: fieldID,
|
|
Binlogs: []*datapb.Binlog{binlog},
|
|
}
|
|
}
|
|
|
|
actions := []metacache.SegmentAction{metacache.MergeBm25Stats(pack.bm25Stats)}
|
|
bw.metaCache.UpdateSegments(metacache.MergeSegmentAction(actions...), metacache.WithSegmentIDs(pack.segmentID))
|
|
|
|
if pack.isFlush {
|
|
if pack.level != datapb.SegmentLevel_L0 {
|
|
if hasBM25Function(bw.schema) {
|
|
mergedBM25Blob, err := serializer.serializeMergedBM25Stats(pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
for fieldID, blob := range mergedBM25Blob {
|
|
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID, fieldID, int64(storage.CompoundStatsType))
|
|
binlog, err := bw.writeLog(ctx, blob, common.SegmentBm25LogPath, k, pack)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
fieldBinlog, ok := logs[fieldID]
|
|
if !ok {
|
|
fieldBinlog = &datapb.FieldBinlog{
|
|
FieldID: fieldID,
|
|
}
|
|
logs[fieldID] = fieldBinlog
|
|
}
|
|
fieldBinlog.Binlogs = append(fieldBinlog.Binlogs, binlog)
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return logs, nil
|
|
}
|
|
|
|
func (bw *BulkPackWriter) writeDelta(ctx context.Context, pack *SyncPack) (*datapb.FieldBinlog, error) {
|
|
if pack.deltaData == nil {
|
|
return &datapb.FieldBinlog{}, nil
|
|
}
|
|
|
|
pkField := func() *schemapb.FieldSchema {
|
|
for _, field := range bw.schema.Fields {
|
|
if field.IsPrimaryKey {
|
|
return field
|
|
}
|
|
}
|
|
return nil
|
|
}()
|
|
if pkField == nil {
|
|
return nil, fmt.Errorf("primary key field not found")
|
|
}
|
|
|
|
logID := bw.nextID()
|
|
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID, logID)
|
|
path := path.Join(bw.chunkManager.RootPath(), common.SegmentDeltaLogPath, k)
|
|
writer, err := storage.NewDeltalogWriter(
|
|
ctx, pack.collectionID, pack.partitionID, pack.segmentID, logID, pkField.DataType, path,
|
|
storage.WithUploader(func(ctx context.Context, kvs map[string][]byte) error {
|
|
// Get the only blob in the map
|
|
if len(kvs) != 1 {
|
|
return fmt.Errorf("expected 1 blob, got %d", len(kvs))
|
|
}
|
|
for _, blob := range kvs {
|
|
return bw.chunkManager.Write(ctx, path, blob)
|
|
}
|
|
return nil
|
|
}),
|
|
)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
pkType := func() arrow.DataType {
|
|
switch pkField.DataType {
|
|
case schemapb.DataType_Int64:
|
|
return arrow.PrimitiveTypes.Int64
|
|
case schemapb.DataType_VarChar:
|
|
return arrow.BinaryTypes.String
|
|
default:
|
|
return nil
|
|
}
|
|
}()
|
|
if pkType == nil {
|
|
return nil, fmt.Errorf("unexpected pk type %v", pkField.DataType)
|
|
}
|
|
|
|
pkBuilder := array.NewBuilder(memory.DefaultAllocator, pkType)
|
|
tsBuilder := array.NewBuilder(memory.DefaultAllocator, arrow.PrimitiveTypes.Int64)
|
|
defer pkBuilder.Release()
|
|
defer tsBuilder.Release()
|
|
|
|
for i := int64(0); i < pack.deltaData.RowCount; i++ {
|
|
switch pkField.DataType {
|
|
case schemapb.DataType_Int64:
|
|
pkBuilder.(*array.Int64Builder).Append(pack.deltaData.Pks[i].GetValue().(int64))
|
|
case schemapb.DataType_VarChar:
|
|
pkBuilder.(*array.StringBuilder).Append(pack.deltaData.Pks[i].GetValue().(string))
|
|
default:
|
|
return nil, fmt.Errorf("unexpected pk type %v", pkField.DataType)
|
|
}
|
|
tsBuilder.(*array.Int64Builder).Append(int64(pack.deltaData.Tss[i]))
|
|
}
|
|
|
|
pkArray := pkBuilder.NewArray()
|
|
tsArray := tsBuilder.NewArray()
|
|
record := storage.NewSimpleArrowRecord(array.NewRecord(arrow.NewSchema([]arrow.Field{
|
|
{Name: "pk", Type: pkType},
|
|
{Name: "ts", Type: arrow.PrimitiveTypes.Int64},
|
|
}, nil), []arrow.Array{pkArray, tsArray}, pack.deltaData.RowCount), map[storage.FieldID]int{
|
|
common.RowIDField: 0,
|
|
common.TimeStampField: 1,
|
|
})
|
|
err = writer.Write(record)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
err = writer.Close()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
deltalog := &datapb.Binlog{
|
|
EntriesNum: pack.deltaData.RowCount,
|
|
TimestampFrom: pack.tsFrom,
|
|
TimestampTo: pack.tsTo,
|
|
LogPath: path,
|
|
LogSize: pack.deltaData.Size() / 4, // Not used
|
|
MemorySize: pack.deltaData.Size(),
|
|
}
|
|
bw.sizeWritten += deltalog.LogSize
|
|
|
|
return &datapb.FieldBinlog{
|
|
FieldID: pkField.GetFieldID(),
|
|
Binlogs: []*datapb.Binlog{deltalog},
|
|
}, nil
|
|
}
|