mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
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>
274 lines
8.5 KiB
Go
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
|
|
}
|