milvus/internal/datanode/compactor/mix_compactor.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

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
})
}