milvus/internal/flushcommon/syncmgr/pack_writer_v2.go
congqixia c01fd94a6a
enhance: integrate Storage V2 FFI interface for unified storage access (#45723)
Related #44956
This commit integrates the Storage V2 FFI (Foreign Function Interface)
interface throughout the Milvus codebase, enabling unified storage
access through the Loon FFI layer. This is a significant step towards
standardizing storage operations across different storage versions.

1. Configuration Support
- **configs/milvus.yaml**: Added `useLoonFFI` configuration flag under
`common.storage.file.splitByAvgSize` section
- Allows runtime toggle between traditional binlog readers and new
FFI-based manifest readers
  - Default: `false` (maintains backward compatibility)

2. Core FFI Infrastructure

Enhanced Utilities (internal/core/src/storage/loon_ffi/util.cpp/h)
- **ToCStorageConfig()**: Converts Go's `StorageConfig` to C's
`CStorageConfig` struct for FFI calls
- **GetManifest()**: Parses manifest JSON and retrieves latest column
groups using FFI
  - Accepts manifest path with `base_path` and `ver` fields
  - Calls `get_latest_column_groups()` FFI function
  - Returns column group information as string
  - Comprehensive error handling for JSON parsing and FFI errors

3. Dependency Updates
- **internal/core/thirdparty/milvus-storage/CMakeLists.txt**:
  - Updated milvus-storage version from `0883026` to `302143c`
  - Ensures compatibility with latest FFI interfaces

4. Data Coordinator Changes

All compaction task builders now include manifest path in segment
binlogs:

- **compaction_task_clustering.go**: Added `Manifest:
segInfo.GetManifestPath()` to segment binlogs
- **compaction_task_l0.go**: Added manifest path to both L0 segment
selection and compaction plan building
- **compaction_task_mix.go**: Added manifest path to mixed compaction
segment binlogs
- **meta.go**: Updated metadata completion logic:
- `completeClusterCompactionMutation()`: Set `ManifestPath` in new
segment info
- `completeMixCompactionMutation()`: Preserve manifest path in compacted
segments
- `completeSortCompactionMutation()`: Include manifest path in sorted
segments

5. Data Node Compactor Enhancements

All compactors updated to support dual-mode reading (binlog vs
manifest):

6. Flush & Sync Manager Updates

Pack Writer V2 (pack_writer_v2.go)
- **BulkPackWriterV2.Write()**: Extended return signature to include
`manifest string`
- Implementation:
  - Generate manifest path: `path.Join(pack.segmentID, "manifest.json")`
  - Write packed data using FFI-based writer
  - Return manifest path along with binlogs, deltas, and stats

Task Handling (task.go)
- Updated all sync task result handling to accommodate new manifest
return value
- Ensured backward compatibility for callers not using manifest

7. Go Storage Layer Integration

New Interfaces and Implementations
- **record_reader.go**: Interface for unified record reading across
storage versions
- **record_writer.go**: Interface for unified record writing across
storage versions
- **binlog_record_writer.go**: Concrete implementation for traditional
binlog-based writing

Enhanced Schema Support (schema.go, schema_test.go)
- Schema conversion utilities to support FFI-based storage operations
- Ensures proper Arrow schema mapping for V2 storage

Serialization Updates
- **serde.go, serde_events.go, serde_events_v2.go**: Updated to work
with new reader/writer interfaces
- Test files updated to validate dual-mode serialization

8. Storage V2 Packed Format

FFI Common (storagev2/packed/ffi_common.go)
- Common FFI utilities and type conversions for packed storage format

Packed Writer FFI (storagev2/packed/packed_writer_ffi.go)
- FFI-based implementation of packed writer
- Integrates with Loon storage layer for efficient columnar writes

Packed Reader FFI (storagev2/packed/packed_reader_ffi.go)
- Already existed, now complemented by writer implementation

9. Protocol Buffer Updates

data_coord.proto & datapb/data_coord.pb.go
- Added `manifest` field to compaction segment messages
- Enables passing manifest metadata through compaction pipeline

worker.proto & workerpb/worker.pb.go
- Added compaction parameter for `useLoonFFI` flag
- Allows workers to receive FFI configuration from coordinator

10. Parameter Configuration

component_param.go
- Added `UseLoonFFI` parameter to compaction configuration
- Reads from `common.storage.file.useLoonFFI` config path
- Default: `false` for safe rollout

11. Test Updates
- **clustering_compactor_storage_v2_test.go**: Updated signatures to
handle manifest return value
- **mix_compactor_storage_v2_test.go**: Updated test helpers for
manifest support
- **namespace_compactor_test.go**: Adjusted writer calls to expect
manifest
- **pack_writer_v2_test.go**: Validated manifest generation in pack
writing

This integration follows a **dual-mode approach**:
1. **Legacy Path**: Traditional binlog-based reading/writing (when
`useLoonFFI=false` or no manifest)
2. **FFI Path**: Manifest-based reading/writing through Loon FFI (when
`useLoonFFI=true` and manifest exists)

---------

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
2025-11-24 19:57:07 +08:00

274 lines
8.5 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"
"encoding/base64"
"math"
"path"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/apache/arrow/go/v17/arrow/memory"
"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/internal/storagecommon"
"github.com/milvus-io/milvus/internal/util/hookutil"
"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/proto/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/retry"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type BulkPackWriterV2 struct {
*BulkPackWriter
schema *schemapb.CollectionSchema
bufferSize int64
multiPartUploadSize int64
storageConfig *indexpb.StorageConfig
columnGroups []storagecommon.ColumnGroup
}
func NewBulkPackWriterV2(metaCache metacache.MetaCache, schema *schemapb.CollectionSchema, chunkManager storage.ChunkManager,
allocator allocator.Interface, bufferSize, multiPartUploadSize int64,
storageConfig *indexpb.StorageConfig, columnGroups []storagecommon.ColumnGroup, writeRetryOpts ...retry.Option,
) *BulkPackWriterV2 {
return &BulkPackWriterV2{
BulkPackWriter: &BulkPackWriter{
metaCache: metaCache,
schema: schema,
chunkManager: chunkManager,
allocator: allocator,
writeRetryOpts: writeRetryOpts,
},
schema: schema,
bufferSize: bufferSize,
multiPartUploadSize: multiPartUploadSize,
storageConfig: storageConfig,
columnGroups: columnGroups,
}
}
func (bw *BulkPackWriterV2) Write(ctx context.Context, pack *SyncPack) (
inserts map[int64]*datapb.FieldBinlog,
deltas *datapb.FieldBinlog,
stats map[int64]*datapb.FieldBinlog,
bm25Stats map[int64]*datapb.FieldBinlog,
manifest string,
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, manifest, 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
}
// getRootPath returns the rootPath current task shall use.
// when storageConfig is set, use the rootPath in it.
// otherwise, use chunkManager.RootPath() instead.
func (bw *BulkPackWriterV2) getRootPath() string {
if bw.storageConfig != nil {
return bw.storageConfig.RootPath
}
return bw.chunkManager.RootPath()
}
func (bw *BulkPackWriterV2) getBucketName() string {
if bw.storageConfig != nil {
return bw.storageConfig.BucketName
}
return paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue()
}
func (bw *BulkPackWriterV2) writeInserts(ctx context.Context, pack *SyncPack) (map[int64]*datapb.FieldBinlog, string, error) {
if len(pack.insertData) == 0 {
return make(map[int64]*datapb.FieldBinlog), "", nil
}
columnGroups := bw.columnGroups
rec, err := bw.serializeBinlog(ctx, pack)
if err != nil {
return nil, "", err
}
logs := make(map[int64]*datapb.FieldBinlog)
tsArray := rec.Column(common.TimeStampField).(*array.Int64)
rows := rec.Len()
var tsFrom uint64 = math.MaxUint64
var tsTo uint64 = 0
for i := 0; i < rows; i++ {
ts := typeutil.Timestamp(tsArray.Value(i))
if ts < tsFrom {
tsFrom = ts
}
if ts > tsTo {
tsTo = ts
}
}
bucketName := bw.getBucketName()
var pluginContextPtr *indexcgopb.StoragePluginContext
if hookutil.IsClusterEncyptionEnabled() {
ez := hookutil.GetEzByCollProperties(bw.schema.GetProperties(), pack.collectionID)
if ez != nil {
unsafe := hookutil.GetCipher().GetUnsafeKey(ez.EzID, ez.CollectionID)
if len(unsafe) > 0 {
pluginContext := indexcgopb.StoragePluginContext{
EncryptionZoneId: ez.EzID,
CollectionId: ez.CollectionID,
EncryptionKey: base64.StdEncoding.EncodeToString(unsafe),
}
pluginContextPtr = &pluginContext
}
}
}
doWrite := func(w storage.RecordWriter) error {
if err = w.Write(rec); err != nil {
return err
}
// close first the get stats & output
return w.Close()
}
var manifestPath string
if paramtable.Get().CommonCfg.UseLoonFFI.GetAsBool() {
k := metautil.JoinIDPath(pack.collectionID, pack.partitionID, pack.segmentID)
basePath := path.Join(bw.getRootPath(), common.SegmentInsertLogPath, k)
w, err := storage.NewPackedRecordManifestWriter(bucketName, basePath, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
if err != nil {
return nil, "", err
}
if err = doWrite(w); err != nil {
return nil, "", err
}
for _, columnGroup := range columnGroups {
columnGroupID := columnGroup.GroupID
logs[columnGroupID] = &datapb.FieldBinlog{
FieldID: columnGroupID,
ChildFields: columnGroup.Fields,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(w.GetColumnGroupWrittenCompressed(columnGroup.GroupID)),
MemorySize: int64(w.GetColumnGroupWrittenUncompressed(columnGroup.GroupID)),
LogPath: w.GetWrittenPaths(columnGroupID),
EntriesNum: w.GetWrittenRowNum(),
TimestampFrom: tsFrom,
TimestampTo: tsTo,
},
},
}
}
manifestPath = w.GetWrittenManifest()
} else {
paths := make([]string, 0)
for _, columnGroup := range columnGroups {
path := metautil.BuildInsertLogPath(bw.getRootPath(), pack.collectionID, pack.partitionID, pack.segmentID, columnGroup.GroupID, bw.nextID())
paths = append(paths, path)
}
w, err := storage.NewPackedRecordWriter(bucketName, paths, bw.schema, bw.bufferSize, bw.multiPartUploadSize, columnGroups, bw.storageConfig, pluginContextPtr)
if err != nil {
return nil, "", err
}
if err = doWrite(w); err != nil {
return nil, "", err
}
// workaround to store row num
for _, columnGroup := range columnGroups {
columnGroupID := columnGroup.GroupID
logs[columnGroupID] = &datapb.FieldBinlog{
FieldID: columnGroupID,
ChildFields: columnGroup.Fields,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(w.GetColumnGroupWrittenCompressed(columnGroup.GroupID)),
MemorySize: int64(w.GetColumnGroupWrittenUncompressed(columnGroup.GroupID)),
LogPath: w.GetWrittenPaths(columnGroupID),
EntriesNum: w.GetWrittenRowNum(),
TimestampFrom: tsFrom,
TimestampTo: tsTo,
},
},
}
}
}
return logs, manifestPath, nil
}
func (bw *BulkPackWriterV2) serializeBinlog(_ context.Context, pack *SyncPack) (storage.Record, error) {
if len(pack.insertData) == 0 {
return nil, nil
}
arrowSchema, err := storage.ConvertToArrowSchema(bw.schema, true)
if err != nil {
return nil, err
}
builder := array.NewRecordBuilder(memory.DefaultAllocator, arrowSchema)
defer builder.Release()
for _, chunk := range pack.insertData {
if err := storage.BuildRecord(builder, chunk, bw.schema); err != nil {
return nil, err
}
}
rec := builder.NewRecord()
allFields := typeutil.GetAllFieldSchemas(bw.schema)
field2Col := make(map[storage.FieldID]int, len(allFields))
for c, field := range allFields {
field2Col[field.FieldID] = c
}
return storage.NewSimpleArrowRecord(rec, field2Col), nil
}