milvus/internal/datanode/compactor/mix_compactor.go
congqixia f94b04e642
feat: [2.6] integrate Loon FFI for manifest-based segment loading and index building (#46076)
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>
2025-12-04 17:09:12 +08:00

447 lines
13 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 compactor
import (
"context"
"fmt"
sio "io"
"math"
"time"
"github.com/apache/arrow/go/v17/arrow/array"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.opentelemetry.io/otel"
"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/compaction"
"github.com/milvus-io/milvus/internal/flushcommon/io"
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
"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/metrics"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/timerecord"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type mixCompactionTask struct {
binlogIO io.BinlogIO
currentTime time.Time
plan *datapb.CompactionPlan
ctx context.Context
cancel context.CancelFunc
collectionID int64
partitionID int64
targetSize int64
maxRows int64
bm25FieldIDs []int64
done chan struct{}
tr *timerecord.TimeRecorder
compactionParams compaction.Params
sortByFieldIDs []int64
}
var _ Compactor = (*mixCompactionTask)(nil)
func NewMixCompactionTask(
ctx context.Context,
binlogIO io.BinlogIO,
plan *datapb.CompactionPlan,
compactionParams compaction.Params,
sortByFieldIDs []int64,
) *mixCompactionTask {
ctx1, cancel := context.WithCancel(ctx)
return &mixCompactionTask{
ctx: ctx1,
cancel: cancel,
binlogIO: binlogIO,
plan: plan,
tr: timerecord.NewTimeRecorder("mergeSplit compaction"),
currentTime: time.Now(),
done: make(chan struct{}, 1),
compactionParams: compactionParams,
sortByFieldIDs: sortByFieldIDs,
}
}
// preCompact exams whether its a valid compaction plan, and init the collectionID and partitionID
func (t *mixCompactionTask) preCompact() error {
if ok := funcutil.CheckCtxValid(t.ctx); !ok {
return t.ctx.Err()
}
if len(t.plan.GetSegmentBinlogs()) < 1 {
return errors.Newf("compaction plan is illegal, there's no segments in compaction plan, planID = %d", t.GetPlanID())
}
if t.plan.GetMaxSize() == 0 {
return errors.Newf("compaction plan is illegal, empty maxSize, planID = %d", t.GetPlanID())
}
t.collectionID = t.plan.GetSegmentBinlogs()[0].GetCollectionID()
t.partitionID = t.plan.GetSegmentBinlogs()[0].GetPartitionID()
t.targetSize = t.plan.GetMaxSize()
t.bm25FieldIDs = GetBM25FieldIDs(t.plan.GetSchema())
currSize := int64(0)
for _, segmentBinlog := range t.plan.GetSegmentBinlogs() {
for i, fieldBinlog := range segmentBinlog.GetFieldBinlogs() {
for _, binlog := range fieldBinlog.GetBinlogs() {
// numRows just need to add entries num of ONE field.
if i == 0 {
t.maxRows += binlog.GetEntriesNum()
}
// MemorySize might be incorrectly
currSize += binlog.GetMemorySize()
}
}
}
outputSegmentCount := int64(math.Ceil(float64(currSize) / float64(t.targetSize)))
log.Info("preCompaction analyze",
zap.Int64("planID", t.GetPlanID()),
zap.Int64("inputSize", currSize),
zap.Int64("targetSize", t.targetSize),
zap.Int("inputSegmentCount", len(t.plan.GetSegmentBinlogs())),
zap.Int64("estimatedOutputSegmentCount", outputSegmentCount),
)
return nil
}
func (t *mixCompactionTask) mergeSplit(
ctx context.Context,
) ([]*datapb.CompactionSegment, error) {
_ = t.tr.RecordSpan()
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "MergeSplit")
defer span.End()
log := log.With(zap.Int64("planID", t.GetPlanID()))
segIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedSegmentIDs().GetBegin(), t.plan.GetPreAllocatedSegmentIDs().GetEnd())
logIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedLogIDs().GetBegin(), t.plan.GetPreAllocatedLogIDs().GetEnd())
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
mWriter, err := NewMultiSegmentWriter(ctx, t.binlogIO, compAlloc, t.plan.GetMaxSize(), t.plan.GetSchema(), t.compactionParams, t.maxRows, t.partitionID, t.collectionID, t.GetChannelName(), 4096, storage.WithStorageConfig(t.compactionParams.StorageConfig), storage.WithUseLoonFFI(t.compactionParams.UseLoonFFI))
if err != nil {
return nil, err
}
deletedRowCount := int64(0)
expiredRowCount := int64(0)
pkField, err := typeutil.GetPrimaryFieldSchema(t.plan.GetSchema())
if err != nil {
log.Warn("failed to get pk field from schema")
return nil, err
}
for _, seg := range t.plan.GetSegmentBinlogs() {
del, exp, err := t.writeSegment(ctx, seg, mWriter, pkField)
if err != nil {
mWriter.Close()
return nil, err
}
deletedRowCount += del
expiredRowCount += exp
}
if err := mWriter.Close(); err != nil {
log.Warn("compact wrong, failed to finish writer", zap.Error(err))
return nil, err
}
res := mWriter.GetCompactionSegments()
if len(res) == 0 {
// append an empty segment
id, err := segIDAlloc.AllocOne()
if err != nil {
return nil, err
}
res = append(res, &datapb.CompactionSegment{
SegmentID: id,
NumOfRows: 0,
Channel: t.GetChannelName(),
})
}
totalElapse := t.tr.RecordSpan()
log.Info("compact mergeSplit end",
zap.Int64("deleted row count", deletedRowCount),
zap.Int64("expired entities", expiredRowCount),
zap.Duration("total elapse", totalElapse))
return res, nil
}
func (t *mixCompactionTask) writeSegment(ctx context.Context,
seg *datapb.CompactionSegmentBinlogs,
mWriter *MultiSegmentWriter, pkField *schemapb.FieldSchema,
) (deletedRowCount, expiredRowCount int64, err error) {
deltaPaths := make([]string, 0)
for _, fieldBinlog := range seg.GetDeltalogs() {
for _, binlog := range fieldBinlog.GetBinlogs() {
deltaPaths = append(deltaPaths, binlog.GetLogPath())
}
}
delta, err := compaction.ComposeDeleteFromDeltalogs(ctx, t.binlogIO, deltaPaths)
if err != nil {
log.Warn("compact wrong, fail to merge deltalogs", zap.Error(err))
return
}
entityFilter := compaction.NewEntityFilter(delta, t.plan.GetCollectionTtl(), t.currentTime)
var reader storage.RecordReader
if seg.GetManifest() != "" {
reader, err = storage.NewManifestRecordReader(ctx,
seg.GetManifest(),
t.plan.GetSchema(),
storage.WithCollectionID(t.collectionID),
storage.WithDownloader(t.binlogIO.Download),
storage.WithVersion(seg.GetStorageVersion()),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
)
} else {
reader, err = storage.NewBinlogRecordReader(ctx,
seg.GetFieldBinlogs(),
t.plan.GetSchema(),
storage.WithCollectionID(t.collectionID),
storage.WithDownloader(t.binlogIO.Download),
storage.WithVersion(seg.GetStorageVersion()),
storage.WithStorageConfig(t.compactionParams.StorageConfig),
)
}
if err != nil {
log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err))
return
}
defer reader.Close()
for {
var r storage.Record
r, err = reader.Next()
if err != nil {
if err == sio.EOF {
err = nil
break
} else {
log.Warn("compact wrong, failed to iter through data", zap.Error(err))
return
}
}
var (
pkArray = r.Column(pkField.FieldID)
tsArray = r.Column(common.TimeStampField).(*array.Int64)
sliceStart = -1
rb *storage.RecordBuilder
)
for i := range r.Len() {
// Filtering deleted entities
var pk any
switch pkField.DataType {
case schemapb.DataType_Int64:
pk = pkArray.(*array.Int64).Value(i)
case schemapb.DataType_VarChar:
pk = pkArray.(*array.String).Value(i)
default:
panic("invalid data type")
}
ts := typeutil.Timestamp(tsArray.Value(i))
if entityFilter.Filtered(pk, ts) {
if rb == nil {
rb = storage.NewRecordBuilder(t.plan.GetSchema())
}
if sliceStart != -1 {
rb.Append(r, sliceStart, i)
}
sliceStart = -1
continue
}
if sliceStart == -1 {
sliceStart = i
}
}
if rb != nil {
if sliceStart != -1 {
rb.Append(r, sliceStart, r.Len())
}
if rb.GetRowNum() > 0 {
err := func() error {
rec := rb.Build()
defer rec.Release()
return mWriter.Write(rec)
}()
if err != nil {
return 0, 0, err
}
}
} else {
err := mWriter.Write(r)
if err != nil {
return 0, 0, err
}
}
}
deltalogDeleteEntriesCount := len(delta)
deletedRowCount = int64(entityFilter.GetDeletedCount())
expiredRowCount = int64(entityFilter.GetExpiredCount())
metrics.DataNodeCompactionDeleteCount.WithLabelValues(fmt.Sprint(t.collectionID)).Add(float64(deltalogDeleteEntriesCount))
metrics.DataNodeCompactionMissingDeleteCount.WithLabelValues(fmt.Sprint(t.collectionID)).Add(float64(entityFilter.GetMissingDeleteCount()))
return
}
func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
durInQueue := t.tr.RecordSpan()
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("MixCompact-%d", t.GetPlanID()))
defer span.End()
compactStart := time.Now()
if err := t.preCompact(); err != nil {
log.Warn("compact wrong, failed to preCompact", zap.Error(err))
return nil, err
}
log := log.Ctx(ctx).With(zap.Int64("planID", t.GetPlanID()),
zap.Int64("collectionID", t.collectionID),
zap.Int64("partitionID", t.partitionID))
ctxTimeout, cancelAll := context.WithCancel(ctx)
defer cancelAll()
log.Info("compact start")
// Decompress compaction binlogs first
if err := binlog.DecompressCompactionBinlogsWithRootPath(t.compactionParams.StorageConfig.GetRootPath(), t.plan.SegmentBinlogs); err != nil {
log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err))
return nil, err
}
// Unable to deal with all empty segments cases, so return error
isEmpty := lo.EveryBy(lo.FlatMap(t.plan.GetSegmentBinlogs(), func(seg *datapb.CompactionSegmentBinlogs, _ int) []*datapb.FieldBinlog {
return seg.GetFieldBinlogs()
}), func(field *datapb.FieldBinlog) bool {
return len(field.GetBinlogs()) == 0
})
if isEmpty {
log.Warn("compact wrong, all segments' binlogs are empty")
return nil, errors.New("illegal compaction plan")
}
sortMergeAppicable := t.compactionParams.UseMergeSort
if sortMergeAppicable {
for _, segment := range t.plan.GetSegmentBinlogs() {
if !segment.GetIsSorted() {
sortMergeAppicable = false
break
}
}
if len(t.plan.GetSegmentBinlogs()) > t.compactionParams.MaxSegmentMergeSort {
// sort merge is not applicable if there is only one segment or too many segments
sortMergeAppicable = false
}
}
var res []*datapb.CompactionSegment
var err error
if sortMergeAppicable {
log.Info("compact by merge sort")
res, err = mergeSortMultipleSegments(ctxTimeout, t.plan, t.collectionID, t.partitionID, t.maxRows, t.binlogIO,
t.plan.GetSegmentBinlogs(), t.tr, t.currentTime, t.plan.GetCollectionTtl(), t.compactionParams, t.sortByFieldIDs)
if err != nil {
log.Warn("compact wrong, fail to merge sort segments", zap.Error(err))
return nil, err
}
} else {
res, err = t.mergeSplit(ctxTimeout)
if err != nil {
log.Warn("compact wrong, failed to mergeSplit", zap.Error(err))
return nil, err
}
}
log.Info("compact done", zap.Duration("compact elapse", time.Since(compactStart)), zap.Any("res", res))
metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.plan.GetType().String()).Observe(float64(t.tr.ElapseSpan().Milliseconds()))
metrics.DataNodeCompactionLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(durInQueue.Milliseconds()))
planResult := &datapb.CompactionPlanResult{
State: datapb.CompactionTaskState_completed,
PlanID: t.GetPlanID(),
Channel: t.GetChannelName(),
Segments: res,
Type: t.plan.GetType(),
}
return planResult, nil
}
func (t *mixCompactionTask) Complete() {
t.done <- struct{}{}
}
func (t *mixCompactionTask) Stop() {
t.cancel()
<-t.done
}
func (t *mixCompactionTask) GetPlanID() typeutil.UniqueID {
return t.plan.GetPlanID()
}
func (t *mixCompactionTask) GetChannelName() string {
return t.plan.GetChannel()
}
func (t *mixCompactionTask) GetCompactionType() datapb.CompactionType {
return t.plan.GetType()
}
func (t *mixCompactionTask) GetCollection() typeutil.UniqueID {
return t.plan.GetSegmentBinlogs()[0].GetCollectionID()
}
func (t *mixCompactionTask) GetSlotUsage() int64 {
return t.plan.GetSlotUsage()
}
func GetBM25FieldIDs(coll *schemapb.CollectionSchema) []int64 {
return lo.FilterMap(coll.GetFunctions(), func(function *schemapb.FunctionSchema, _ int) (int64, bool) {
if function.GetType() == schemapb.FunctionType_BM25 {
return function.GetOutputFieldIds()[0], true
}
return 0, false
})
}