enhance: Enable to write multiple segments in mix compactor (#35705)

Prevent segments to be written larger than maxSize * expansionRate

See also: #35584

Signed-off-by: yangxuan <xuan.yang@zilliz.com>

---------

Signed-off-by: yangxuan <xuan.yang@zilliz.com>
pull/35865/head
XuanYang-cn 2024-08-30 11:29:01 +08:00 committed by GitHub
parent 1c3abd1aea
commit 323400c190
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 480 additions and 405 deletions

View File

@ -6,6 +6,7 @@ import (
"time"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.opentelemetry.io/otel/trace"
"go.uber.org/zap"
@ -23,11 +24,11 @@ type mixCompactionTask struct {
plan *datapb.CompactionPlan
result *datapb.CompactionPlanResult
span trace.Span
allocator allocator.Allocator
sessions session.DataNodeManager
meta CompactionMeta
newSegment *SegmentInfo
span trace.Span
allocator allocator.Allocator
sessions session.DataNodeManager
meta CompactionMeta
newSegmentIDs []int64
}
func (t *mixCompactionTask) processPipelining() bool {
@ -90,7 +91,7 @@ func (t *mixCompactionTask) processExecuting() bool {
}
case datapb.CompactionTaskState_completed:
t.result = result
if len(result.GetSegments()) == 0 || len(result.GetSegments()) > 1 {
if len(result.GetSegments()) == 0 {
log.Info("illegal compaction results")
err := t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_failed))
if err != nil {
@ -111,8 +112,7 @@ func (t *mixCompactionTask) processExecuting() bool {
}
return false
}
segments := []UniqueID{t.newSegment.GetID()}
err = t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_meta_saved), setResultSegments(segments))
err = t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_meta_saved), setResultSegments(t.newSegmentIDs))
if err != nil {
log.Warn("mixCompaction failed to setState meta saved", zap.Error(err))
return false
@ -144,7 +144,7 @@ func (t *mixCompactionTask) saveSegmentMeta() error {
return err
}
// Apply metrics after successful meta update.
t.newSegment = newSegments[0]
t.newSegmentIDs = lo.Map(newSegments, func(s *SegmentInfo, _ int) UniqueID { return s.GetID() })
metricMutation.commit()
log.Info("mixCompactionTask success to save segment meta")
return nil
@ -236,6 +236,7 @@ func (t *mixCompactionTask) ShadowClone(opts ...compactionTaskOpt) *datapb.Compa
FailReason: t.GetFailReason(),
RetryTimes: t.GetRetryTimes(),
Pos: t.GetPos(),
MaxSize: t.GetMaxSize(),
}
for _, opt := range opts {
opt(taskClone)
@ -348,8 +349,10 @@ func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, er
BeginLogID: beginLogID,
PreAllocatedSegments: &datapb.IDRange{
Begin: t.GetResultSegments()[0],
End: t.GetResultSegments()[1],
},
SlotUsage: Params.DataCoordCfg.MixCompactionSlotUsage.GetAsInt64(),
MaxSize: t.GetMaxSize(),
}
log := log.With(zap.Int64("taskID", t.GetTriggerID()), zap.Int64("planID", plan.GetPlanID()))
@ -371,6 +374,6 @@ func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, er
})
segIDMap[segID] = segInfo.GetDeltalogs()
}
log.Info("Compaction handler refreshed mix compaction plan", zap.Any("segID2DeltaLogs", segIDMap))
log.Info("Compaction handler refreshed mix compaction plan", zap.Int64("maxSize", plan.GetMaxSize()), zap.Any("segID2DeltaLogs", segIDMap))
return plan, nil
}

View File

@ -32,7 +32,7 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_NormalMix() {
NodeID: 1,
State: datapb.CompactionTaskState_executing,
InputSegments: []int64{200, 201},
ResultSegments: []int64{100},
ResultSegments: []int64{100, 200},
},
// plan: plan,
meta: s.mockMeta,
@ -67,12 +67,12 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_MixSegmentNotFound() {
State: datapb.CompactionTaskState_executing,
NodeID: 1,
InputSegments: []int64{200, 201},
ResultSegments: []int64{100},
ResultSegments: []int64{100, 200},
},
meta: s.mockMeta,
}
alloc := allocator.NewMockAllocator(s.T())
alloc.EXPECT().AllocN(mock.Anything).Return(100, 200, nil)
alloc.EXPECT().AllocN(int64(1)).Return(19530, 99999, nil)
task.allocator = alloc
_, err := task.BuildCompactionRequest()
s.Error(err)

View File

@ -355,26 +355,26 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
return err
}
plans := t.generatePlans(group.segments, signal, ct)
currentID, _, err := t.allocator.AllocN(int64(len(plans) * 2))
if err != nil {
return err
}
expectedSize := getExpectedSegmentSize(t.meta, coll)
plans := t.generatePlans(group.segments, signal, ct, expectedSize)
for _, plan := range plans {
totalRows := plan.A
segIDs := plan.B
if !signal.isForce && t.compactionHandler.isFull() {
log.Warn("compaction plan skipped due to handler full", zap.Int64s("segmentIDs", segIDs))
log.Warn("compaction plan skipped due to handler full")
break
}
totalRows, inputSegmentIDs := plan.A, plan.B
// TODO[GOOSE], 11 = 1 planID + 10 segmentID, this is a hack need to be removed.
// Any plan that output segment number greater than 10 will be marked as invalid plan for now.
startID, endID, err := t.allocator.AllocN(11)
if err != nil {
log.Warn("fail to allocate id", zap.Error(err))
return err
}
start := time.Now()
planID := currentID
currentID++
targetSegmentID := currentID
currentID++
pts, _ := tsoutil.ParseTS(ct.startTime)
task := &datapb.CompactionTask{
PlanID: planID,
PlanID: startID,
TriggerID: signal.id,
State: datapb.CompactionTaskState_pipelining,
StartTime: pts.Unix(),
@ -384,22 +384,26 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
CollectionID: group.collectionID,
PartitionID: group.partitionID,
Channel: group.channelName,
InputSegments: segIDs,
ResultSegments: []int64{targetSegmentID}, // pre-allocated target segment
InputSegments: inputSegmentIDs,
ResultSegments: []int64{startID + 1, endID}, // pre-allocated target segment
TotalRows: totalRows,
Schema: coll.Schema,
MaxSize: getExpandedSize(expectedSize),
}
err := t.compactionHandler.enqueueCompaction(task)
err = t.compactionHandler.enqueueCompaction(task)
if err != nil {
log.Warn("failed to execute compaction task",
zap.Int64s("segmentIDs", segIDs),
zap.Int64("collection", group.collectionID),
zap.Int64("triggerID", signal.id),
zap.Int64("planID", task.GetPlanID()),
zap.Int64s("inputSegments", inputSegmentIDs),
zap.Error(err))
continue
}
log.Info("time cost of generating global compaction",
zap.Int64("time cost", time.Since(start).Milliseconds()),
zap.Int64s("segmentIDs", segIDs))
zap.Int64s("segmentIDs", inputSegmentIDs))
}
}
return nil
@ -457,27 +461,26 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
return
}
plans := t.generatePlans(segments, signal, ct)
currentID, _, err := t.allocator.AllocN(int64(len(plans) * 2))
if err != nil {
log.Warn("fail to allocate id", zap.Error(err))
return
}
expectedSize := getExpectedSegmentSize(t.meta, coll)
plans := t.generatePlans(segments, signal, ct, expectedSize)
for _, plan := range plans {
if t.compactionHandler.isFull() {
log.Warn("compaction plan skipped due to handler full", zap.Int64("collection", signal.collectionID))
break
}
totalRows := plan.A
segmentIDS := plan.B
// TODO[GOOSE], 11 = 1 planID + 10 segmentID, this is a hack need to be removed.
// Any plan that output segment number greater than 10 will be marked as invalid plan for now.
startID, endID, err := t.allocator.AllocN(11)
if err != nil {
log.Warn("fail to allocate id", zap.Error(err))
return
}
totalRows, inputSegmentIDs := plan.A, plan.B
start := time.Now()
planID := currentID
currentID++
targetSegmentID := currentID
currentID++
pts, _ := tsoutil.ParseTS(ct.startTime)
if err := t.compactionHandler.enqueueCompaction(&datapb.CompactionTask{
PlanID: planID,
task := &datapb.CompactionTask{
PlanID: startID,
TriggerID: signal.id,
State: datapb.CompactionTaskState_pipelining,
StartTime: pts.Unix(),
@ -487,29 +490,32 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
CollectionID: collectionID,
PartitionID: partitionID,
Channel: channel,
InputSegments: segmentIDS,
ResultSegments: []int64{targetSegmentID}, // pre-allocated target segment
InputSegments: inputSegmentIDs,
ResultSegments: []int64{startID + 1, endID}, // pre-allocated target segment
TotalRows: totalRows,
Schema: coll.Schema,
}); err != nil {
MaxSize: getExpandedSize(expectedSize),
}
if err := t.compactionHandler.enqueueCompaction(task); err != nil {
log.Warn("failed to execute compaction task",
zap.Int64("collection", collectionID),
zap.Int64("planID", planID),
zap.Int64s("segmentIDs", segmentIDS),
zap.Int64("triggerID", signal.id),
zap.Int64("planID", task.GetPlanID()),
zap.Int64s("inputSegments", inputSegmentIDs),
zap.Error(err))
continue
}
log.Info("time cost of generating compaction",
zap.Int64("planID", planID),
zap.Int64("planID", task.GetPlanID()),
zap.Int64("time cost", time.Since(start).Milliseconds()),
zap.Int64("collectionID", signal.collectionID),
zap.String("channel", channel),
zap.Int64("partitionID", partitionID),
zap.Int64s("segmentIDs", segmentIDS))
zap.Int64s("inputSegmentIDs", inputSegmentIDs))
}
}
func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, signal *compactionSignal, compactTime *compactTime) []*typeutil.Pair[int64, []int64] {
func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, signal *compactionSignal, compactTime *compactTime, expectedSize int64) []*typeutil.Pair[int64, []int64] {
if len(segments) == 0 {
log.Warn("the number of candidate segments is 0, skip to generate compaction plan")
return []*typeutil.Pair[int64, []int64]{}
@ -521,8 +527,6 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, signal *compa
var smallCandidates []*SegmentInfo
var nonPlannedSegments []*SegmentInfo
expectedSize := t.getExpectedSegmentSize(segments[0].CollectionID)
// TODO, currently we lack of the measurement of data distribution, there should be another compaction help on redistributing segment based on scalar/vector field distribution
for _, segment := range segments {
segment := segment.ShadowClone()
@ -853,3 +857,7 @@ func (t *compactionTrigger) squeezeSmallSegmentsToBuckets(small []*SegmentInfo,
return small
}
func getExpandedSize(size int64) int64 {
return int64(float64(size) * Params.DataCoordCfg.SegmentExpansionRate.GetAsFloat())
}

View File

@ -70,7 +70,7 @@ func (h *spyCompactionHandler) enqueueCompaction(task *datapb.CompactionTask) er
}
alloc := newMock0Allocator(h.t)
t.allocator = alloc
t.ResultSegments = []int64{100}
t.ResultSegments = []int64{100, 200}
plan, err := t.BuildCompactionRequest()
h.spyChan <- plan
return err
@ -486,8 +486,9 @@ func Test_compactionTrigger_force(t *testing.T) {
Channel: "ch1",
TotalRows: 200,
Schema: schema,
PreAllocatedSegments: &datapb.IDRange{Begin: 100},
PreAllocatedSegments: &datapb.IDRange{Begin: 100, End: 200},
SlotUsage: 8,
MaxSize: 1342177280,
},
},
},
@ -780,6 +781,7 @@ func Test_compactionTrigger_force_maxSegmentLimit(t *testing.T) {
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(),
Type: datapb.CompactionType_MixCompaction,
Channel: "ch1",
MaxSize: 1342177280,
},
},
},

View File

@ -24,15 +24,11 @@ import (
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
"github.com/milvus-io/milvus/pkg/util/lock"
"github.com/milvus-io/milvus/pkg/util/logutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type CompactionTriggerType int8
@ -363,11 +359,14 @@ func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.C
func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Context, view CompactionView) {
log := log.With(zap.String("view", view.String()))
taskID, _, err := m.allocator.AllocN(2)
// TODO[GOOSE], 11 = 1 planID + 10 segmentID, this is a hack need to be removed.
// Any plan that output segment number greater than 10 will be marked as invalid plan for now.
startID, endID, err := m.allocator.AllocN(11)
if err != nil {
log.Warn("Failed to submit compaction view to scheduler because allocate id fail", zap.Error(err))
log.Warn("fFailed to submit compaction view to scheduler because allocate id fail", zap.Error(err))
return
}
collection, err := m.handler.GetCollection(ctx, view.GetGroupLabel().CollectionID)
if err != nil {
log.Warn("Failed to submit compaction view to scheduler because get collection fail", zap.Error(err))
@ -377,8 +376,10 @@ func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Conte
for _, s := range view.GetSegmentsView() {
totalRows += s.NumOfRows
}
expectedSize := getExpectedSegmentSize(m.meta, collection)
task := &datapb.CompactionTask{
PlanID: taskID,
PlanID: startID,
TriggerID: view.(*MixSegmentView).triggerID,
State: datapb.CompactionTaskState_pipelining,
StartTime: time.Now().Unix(),
@ -390,9 +391,10 @@ func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Conte
Channel: view.GetGroupLabel().Channel,
Schema: collection.Schema,
InputSegments: lo.Map(view.GetSegmentsView(), func(segmentView *SegmentView, _ int) int64 { return segmentView.ID }),
ResultSegments: []int64{taskID + 1},
ResultSegments: []int64{startID + 1, endID},
TotalRows: totalRows,
LastStateStartTime: time.Now().Unix(),
MaxSize: getExpandedSize(expectedSize),
}
err = m.compactionHandler.enqueueCompaction(task)
if err != nil {
@ -409,21 +411,8 @@ func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Conte
)
}
func getExpectedSegmentSize(meta *meta, collection *collectionInfo) int64 {
indexInfos := meta.indexMeta.GetIndexesForCollection(collection.ID, "")
vectorFields := typeutil.GetVectorFieldSchemas(collection.Schema)
fieldIndexTypes := lo.SliceToMap(indexInfos, func(t *model.Index) (int64, indexparamcheck.IndexType) {
return t.FieldID, GetIndexType(t.IndexParams)
})
vectorFieldsWithDiskIndex := lo.Filter(vectorFields, func(field *schemapb.FieldSchema, _ int) bool {
if indexType, ok := fieldIndexTypes[field.FieldID]; ok {
return indexparamcheck.IsDiskIndex(indexType)
}
return false
})
allDiskIndex := len(vectorFields) == len(vectorFieldsWithDiskIndex)
func getExpectedSegmentSize(meta *meta, collInfo *collectionInfo) int64 {
allDiskIndex := meta.indexMeta.AreAllDiskIndex(collInfo.ID, collInfo.Schema)
if allDiskIndex {
// Only if all vector fields index type are DiskANN, recalc segment max size here.
return Params.DataCoordCfg.DiskSegmentMaxSize.GetAsInt64() * 1024 * 1024

View File

@ -1505,62 +1505,69 @@ func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *d
updateSegStateAndPrepareMetrics(cloned, commonpb.SegmentState_Dropped, metricMutation)
}
// MixCompaction / MergeCompaction will generates one and only one segment
compactToSegment := result.GetSegments()[0]
log = log.With(zap.Int64s("compactFrom", compactFromSegIDs))
compactToSegmentInfo := NewSegmentInfo(
&datapb.SegmentInfo{
ID: compactToSegment.GetSegmentID(),
CollectionID: compactFromSegInfos[0].CollectionID,
PartitionID: compactFromSegInfos[0].PartitionID,
InsertChannel: t.GetChannel(),
NumOfRows: compactToSegment.NumOfRows,
State: commonpb.SegmentState_Flushed,
MaxRowNum: compactFromSegInfos[0].MaxRowNum,
Binlogs: compactToSegment.GetInsertLogs(),
Statslogs: compactToSegment.GetField2StatslogPaths(),
Deltalogs: compactToSegment.GetDeltalogs(),
compactToSegments := make([]*SegmentInfo, 0)
for _, compactToSegment := range result.GetSegments() {
compactToSegmentInfo := NewSegmentInfo(
&datapb.SegmentInfo{
ID: compactToSegment.GetSegmentID(),
CollectionID: compactFromSegInfos[0].CollectionID,
PartitionID: compactFromSegInfos[0].PartitionID,
InsertChannel: t.GetChannel(),
NumOfRows: compactToSegment.NumOfRows,
State: commonpb.SegmentState_Flushed,
MaxRowNum: compactFromSegInfos[0].MaxRowNum,
Binlogs: compactToSegment.GetInsertLogs(),
Statslogs: compactToSegment.GetField2StatslogPaths(),
Deltalogs: compactToSegment.GetDeltalogs(),
CreatedByCompaction: true,
CompactionFrom: compactFromSegIDs,
LastExpireTime: tsoutil.ComposeTSByTime(time.Unix(t.GetStartTime(), 0), 0),
Level: datapb.SegmentLevel_L1,
CreatedByCompaction: true,
CompactionFrom: compactFromSegIDs,
LastExpireTime: tsoutil.ComposeTSByTime(time.Unix(t.GetStartTime(), 0), 0),
Level: datapb.SegmentLevel_L1,
StartPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetStartPosition()
})),
DmlPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetDmlPosition()
})),
})
StartPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetStartPosition()
})),
DmlPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetDmlPosition()
})),
})
// L1 segment with NumRows=0 will be discarded, so no need to change the metric
if compactToSegmentInfo.GetNumOfRows() > 0 {
// metrics mutation for compactTo segments
metricMutation.addNewSeg(compactToSegmentInfo.GetState(), compactToSegmentInfo.GetLevel(), compactToSegmentInfo.GetNumOfRows())
} else {
compactToSegmentInfo.State = commonpb.SegmentState_Dropped
// L1 segment with NumRows=0 will be discarded, so no need to change the metric
if compactToSegmentInfo.GetNumOfRows() > 0 {
// metrics mutation for compactTo segments
metricMutation.addNewSeg(compactToSegmentInfo.GetState(), compactToSegmentInfo.GetLevel(), compactToSegmentInfo.GetNumOfRows())
} else {
compactToSegmentInfo.State = commonpb.SegmentState_Dropped
}
log.Info("Add a new compactTo segment",
zap.Int64("compactTo", compactToSegmentInfo.GetID()),
zap.Int64("compactTo segment numRows", compactToSegmentInfo.GetNumOfRows()),
zap.Int("binlog count", len(compactToSegmentInfo.GetBinlogs())),
zap.Int("statslog count", len(compactToSegmentInfo.GetStatslogs())),
zap.Int("deltalog count", len(compactToSegmentInfo.GetDeltalogs())),
)
compactToSegments = append(compactToSegments, compactToSegmentInfo)
}
log = log.With(
zap.Int64s("compactFrom", compactFromSegIDs),
zap.Int64("compactTo", compactToSegmentInfo.GetID()),
zap.Int64("compactTo segment numRows", compactToSegmentInfo.GetNumOfRows()),
)
log.Debug("meta update: prepare for meta mutation - complete")
compactFromInfos := lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *datapb.SegmentInfo {
return info.SegmentInfo
})
log.Debug("meta update: alter meta store for compaction updates",
zap.Int("binlog count", len(compactToSegmentInfo.GetBinlogs())),
zap.Int("statslog count", len(compactToSegmentInfo.GetStatslogs())),
zap.Int("deltalog count", len(compactToSegmentInfo.GetDeltalogs())),
)
if err := m.catalog.AlterSegments(m.ctx, []*datapb.SegmentInfo{compactToSegmentInfo.SegmentInfo},
metastore.BinlogsIncrement{Segment: compactToSegmentInfo.SegmentInfo},
); err != nil {
compactToInfos := lo.Map(compactToSegments, func(info *SegmentInfo, _ int) *datapb.SegmentInfo {
return info.SegmentInfo
})
binlogs := make([]metastore.BinlogsIncrement, 0)
for _, seg := range compactToInfos {
binlogs = append(binlogs, metastore.BinlogsIncrement{Segment: seg})
}
// alter compactTo before compactFrom segments to avoid data lost if service crash during AlterSegments
if err := m.catalog.AlterSegments(m.ctx, compactToInfos, binlogs...); err != nil {
log.Warn("fail to alter compactTo segments", zap.Error(err))
return nil, nil, err
}
@ -1568,14 +1575,15 @@ func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *d
log.Warn("fail to alter compactFrom segments", zap.Error(err))
return nil, nil, err
}
lo.ForEach(compactFromSegInfos, func(info *SegmentInfo, _ int) {
m.segments.SetSegment(info.GetID(), info)
})
m.segments.SetSegment(compactToSegmentInfo.GetID(), compactToSegmentInfo)
lo.ForEach(compactToSegments, func(info *SegmentInfo, _ int) {
m.segments.SetSegment(info.GetID(), info)
})
log.Info("meta update: alter in memory meta after compaction - complete")
return []*SegmentInfo{compactToSegmentInfo}, metricMutation, nil
return compactToSegments, metricMutation, nil
}
func (m *meta) CompleteCompactionMutation(t *datapb.CompactionTask, result *datapb.CompactionPlanResult) ([]*SegmentInfo, *segMetricMutation, error) {

View File

@ -235,7 +235,7 @@ func (t *clusteringCompactionTask) Compact() (*datapb.CompactionPlanResult, erro
defer t.cleanUp(ctx)
// 1, download delta logs to build deltaMap
deltaBlobs, _, err := loadDeltaMap(t.plan.GetSegmentBinlogs())
deltaBlobs, _, err := composePaths(t.plan.GetSegmentBinlogs())
if err != nil {
return nil, err
}

View File

@ -104,7 +104,7 @@ func mergeDeltalogs(ctx context.Context, io io.BinlogIO, dpaths map[typeutil.Uni
return pk2ts, nil
}
func loadDeltaMap(segments []*datapb.CompactionSegmentBinlogs) (map[typeutil.UniqueID][]string, [][]string, error) {
func composePaths(segments []*datapb.CompactionSegmentBinlogs) (map[typeutil.UniqueID][]string, [][]string, error) {
if err := binlog.DecompressCompactionBinlogs(segments); err != nil {
log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err))
return nil, nil, err
@ -184,7 +184,7 @@ func serializeWrite(ctx context.Context, allocator allocator.Interface, writer *
func statSerializeWrite(ctx context.Context, io io.BinlogIO, allocator allocator.Interface, writer *SegmentWriter) (*datapb.FieldBinlog, error) {
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "statslog serializeWrite")
defer span.End()
sblob, err := writer.Finish(writer.GetRowNum())
sblob, err := writer.Finish()
if err != nil {
return nil, err
}
@ -220,3 +220,12 @@ func uploadStatsBlobs(ctx context.Context, collectionID, partitionID, segmentID,
return statFieldLog, nil
}
func mergeFieldBinlogs(base, paths map[typeutil.UniqueID]*datapb.FieldBinlog) {
for fID, fpath := range paths {
if _, ok := base[fID]; !ok {
base[fID] = &datapb.FieldBinlog{FieldID: fID, Binlogs: make([]*datapb.Binlog, 0)}
}
base[fID].Binlogs = append(base[fID].Binlogs, fpath.GetBinlogs()...)
}
}

View File

@ -41,10 +41,8 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
// for MixCompaction only
type mixCompactionTask struct {
binlogIO io.BinlogIO
allocator allocator.Interface
currentTs typeutil.Timestamp
plan *datapb.CompactionPlan
@ -52,11 +50,16 @@ type mixCompactionTask struct {
ctx context.Context
cancel context.CancelFunc
collectionID int64
partitionID int64
targetSize int64
maxRows int64
pkID int64
done chan struct{}
tr *timerecord.TimeRecorder
}
// make sure compactionTask implements compactor interface
var _ Compactor = (*mixCompactionTask)(nil)
func NewMixCompactionTask(
@ -65,76 +68,77 @@ func NewMixCompactionTask(
plan *datapb.CompactionPlan,
) *mixCompactionTask {
ctx1, cancel := context.WithCancel(ctx)
alloc := allocator.NewLocalAllocator(plan.GetBeginLogID(), math.MaxInt64)
return &mixCompactionTask{
ctx: ctx1,
cancel: cancel,
binlogIO: binlogIO,
allocator: alloc,
plan: plan,
tr: timerecord.NewTimeRecorder("mix compaction"),
tr: timerecord.NewTimeRecorder("mergeSplit compaction"),
currentTs: tsoutil.GetCurrentTime(),
done: make(chan struct{}, 1),
}
}
func (t *mixCompactionTask) Complete() {
t.done <- struct{}{}
}
// 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()
}
func (t *mixCompactionTask) Stop() {
t.cancel()
<-t.done
}
if len(t.plan.GetSegmentBinlogs()) < 1 {
return errors.Newf("compaction plan is illegal, there's no segments in compaction plan, planID = %d", t.GetPlanID())
}
func (t *mixCompactionTask) GetPlanID() typeutil.UniqueID {
return t.plan.GetPlanID()
}
if t.plan.GetMaxSize() == 0 {
return errors.Newf("compaction plan is illegal, empty maxSize, planID = %d", t.GetPlanID())
}
func (t *mixCompactionTask) GetChannelName() string {
return t.plan.GetChannel()
}
t.collectionID = t.plan.GetSegmentBinlogs()[0].GetCollectionID()
t.partitionID = t.plan.GetSegmentBinlogs()[0].GetPartitionID()
t.targetSize = t.plan.GetMaxSize()
func (t *mixCompactionTask) GetCompactionType() datapb.CompactionType {
return t.plan.GetType()
}
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()
}
// return num rows of all segment compaction from
func (t *mixCompactionTask) getNumRows() int64 {
numRows := int64(0)
for _, binlog := range t.plan.SegmentBinlogs {
if len(binlog.GetFieldBinlogs()) > 0 {
for _, ct := range binlog.GetFieldBinlogs()[0].GetBinlogs() {
numRows += ct.GetEntriesNum()
// MemorySize might be incorrectly
currSize += binlog.GetMemorySize()
}
}
}
return numRows
outputSegmentCount := int64(math.Ceil(float64(currSize) / float64(t.targetSize)))
log.Info("preCompaction analyze",
zap.Int64("planID", t.GetPlanID()),
zap.Int64("currSize", currSize),
zap.Int64("targetSize", t.targetSize),
zap.Int64("estimatedSegmentCount", outputSegmentCount),
)
return nil
}
func (t *mixCompactionTask) merge(
func (t *mixCompactionTask) mergeSplit(
ctx context.Context,
binlogPaths [][]string,
delta map[interface{}]typeutil.Timestamp,
writer *SegmentWriter,
) (*datapb.CompactionSegment, error) {
) ([]*datapb.CompactionSegment, error) {
_ = t.tr.RecordSpan()
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "CompactMerge")
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "MergeSplit")
defer span.End()
log := log.With(zap.Int64("planID", t.GetPlanID()), zap.Int64("compactTo segment", writer.GetSegmentID()))
log := log.With(zap.Int64("planID", t.GetPlanID()))
var (
syncBatchCount int // binlog batch count
remainingRowCount int64 // the number of remaining entities
expiredRowCount int64 // the number of expired entities
deletedRowCount int64 = 0
unflushedRowCount int64 = 0
// All binlog meta of a segment
allBinlogs = make(map[typeutil.UniqueID]*datapb.FieldBinlog)
)
segIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedSegments().GetBegin(), t.plan.GetPreAllocatedSegments().GetEnd())
logIDAlloc := allocator.NewLocalAllocator(t.plan.GetBeginLogID(), math.MaxInt64)
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
mWriter := NewMultiSegmentWriter(t.binlogIO, compAlloc, t.plan, t.maxRows, t.partitionID, t.collectionID)
isValueDeleted := func(v *storage.Value) bool {
ts, ok := delta[v.PK.GetValue()]
@ -147,25 +151,27 @@ func (t *mixCompactionTask) merge(
return false
}
downloadTimeCost := time.Duration(0)
serWriteTimeCost := time.Duration(0)
uploadTimeCost := time.Duration(0)
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 _, paths := range binlogPaths {
log := log.With(zap.Strings("paths", paths))
downloadStart := time.Now()
allValues, err := t.binlogIO.Download(ctx, paths)
if err != nil {
log.Warn("compact wrong, fail to download insertLogs", zap.Error(err))
return nil, err
}
downloadTimeCost += time.Since(downloadStart)
blobs := lo.Map(allValues, func(v []byte, i int) *storage.Blob {
return &storage.Blob{Key: paths[i], Value: v}
})
iter, err := storage.NewBinlogDeserializeReader(blobs, writer.GetPkID())
iter, err := storage.NewBinlogDeserializeReader(blobs, pkField.GetFieldID())
if err != nil {
log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err))
return nil, err
@ -193,148 +199,56 @@ func (t *mixCompactionTask) merge(
continue
}
err = writer.Write(v)
err = mWriter.Write(v)
if err != nil {
log.Warn("compact wrong, failed to writer row", zap.Error(err))
return nil, err
}
unflushedRowCount++
remainingRowCount++
if (unflushedRowCount+1)%100 == 0 && writer.FlushAndIsFull() {
serWriteStart := time.Now()
kvs, partialBinlogs, err := serializeWrite(ctx, t.allocator, writer)
if err != nil {
log.Warn("compact wrong, failed to serialize writer", zap.Error(err))
return nil, err
}
serWriteTimeCost += time.Since(serWriteStart)
uploadStart := time.Now()
if err := t.binlogIO.Upload(ctx, kvs); err != nil {
log.Warn("compact wrong, failed to upload kvs", zap.Error(err))
return nil, err
}
uploadTimeCost += time.Since(uploadStart)
mergeFieldBinlogs(allBinlogs, partialBinlogs)
syncBatchCount++
unflushedRowCount = 0
}
}
}
if !writer.FlushAndIsEmpty() {
serWriteStart := time.Now()
kvs, partialBinlogs, err := serializeWrite(ctx, t.allocator, writer)
if err != nil {
log.Warn("compact wrong, failed to serialize writer", zap.Error(err))
return nil, err
}
serWriteTimeCost += time.Since(serWriteStart)
uploadStart := time.Now()
if err := t.binlogIO.Upload(ctx, kvs); err != nil {
log.Warn("compact wrong, failed to upload kvs", zap.Error(err))
return nil, err
}
uploadTimeCost += time.Since(uploadStart)
mergeFieldBinlogs(allBinlogs, partialBinlogs)
syncBatchCount++
}
serWriteStart := time.Now()
sPath, err := statSerializeWrite(ctx, t.binlogIO, t.allocator, writer)
res, err := mWriter.Finish()
if err != nil {
log.Warn("compact wrong, failed to serialize write segment stats",
zap.Int64("remaining row count", remainingRowCount), zap.Error(err))
log.Warn("compact wrong, failed to finish writer", zap.Error(err))
return nil, err
}
serWriteTimeCost += time.Since(serWriteStart)
pack := &datapb.CompactionSegment{
SegmentID: writer.GetSegmentID(),
InsertLogs: lo.Values(allBinlogs),
Field2StatslogPaths: []*datapb.FieldBinlog{sPath},
NumOfRows: remainingRowCount,
Channel: t.plan.GetChannel(),
}
totalElapse := t.tr.RecordSpan()
log.Info("compact merge end",
zap.Int64("remaining row count", remainingRowCount),
log.Info("compact mergeSplit end",
zap.Int64s("mergeSplit to segments", lo.Keys(mWriter.cachedMeta)),
zap.Int64("deleted row count", deletedRowCount),
zap.Int64("expired entities", expiredRowCount),
zap.Int("binlog batch count", syncBatchCount),
zap.Duration("download binlogs elapse", downloadTimeCost),
zap.Duration("upload binlogs elapse", uploadTimeCost),
zap.Duration("serWrite elapse", serWriteTimeCost),
zap.Duration("deRead elapse", totalElapse-serWriteTimeCost-downloadTimeCost-uploadTimeCost),
zap.Duration("total elapse", totalElapse))
return pack, nil
}
func mergeFieldBinlogs(base, paths map[typeutil.UniqueID]*datapb.FieldBinlog) {
for fID, fpath := range paths {
if _, ok := base[fID]; !ok {
base[fID] = &datapb.FieldBinlog{FieldID: fID, Binlogs: make([]*datapb.Binlog, 0)}
}
base[fID].Binlogs = append(base[fID].Binlogs, fpath.GetBinlogs()...)
}
return res, nil
}
func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
durInQueue := t.tr.RecordSpan()
compactStart := time.Now()
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("MixCompact-%d", t.GetPlanID()))
defer span.End()
compactStart := time.Now()
if len(t.plan.GetSegmentBinlogs()) < 1 {
log.Warn("compact wrong, there's no segments in segment binlogs", zap.Int64("planID", t.plan.GetPlanID()))
return nil, errors.New("compaction plan is illegal")
if err := t.preCompact(); err != nil {
log.Warn("compact wrong, failed to preCompact", zap.Error(err))
return nil, err
}
collectionID := t.plan.GetSegmentBinlogs()[0].GetCollectionID()
partitionID := t.plan.GetSegmentBinlogs()[0].GetPartitionID()
log := log.Ctx(ctx).With(zap.Int64("planID", t.plan.GetPlanID()),
zap.Int64("collectionID", collectionID),
zap.Int64("partitionID", partitionID),
log := log.Ctx(ctx).With(zap.Int64("planID", t.GetPlanID()),
zap.Int64("collectionID", t.collectionID),
zap.Int64("partitionID", t.partitionID),
zap.Int32("timeout in seconds", t.plan.GetTimeoutInSeconds()))
if ok := funcutil.CheckCtxValid(ctx); !ok {
log.Warn("compact wrong, task context done or timeout")
return nil, ctx.Err()
}
ctxTimeout, cancelAll := context.WithTimeout(ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
defer cancelAll()
log.Info("compact start")
targetSegID := t.plan.GetPreAllocatedSegments().GetBegin()
previousRowCount := t.getNumRows()
writer, err := NewSegmentWriter(t.plan.GetSchema(), previousRowCount, targetSegID, partitionID, collectionID)
deltaPaths, allBatchPaths, err := composePaths(t.plan.GetSegmentBinlogs())
if err != nil {
log.Warn("compact wrong, unable to init segment writer", zap.Error(err))
log.Warn("compact wrong, failed to composePaths", zap.Error(err))
return nil, err
}
segIDs := lo.Map(t.plan.GetSegmentBinlogs(), func(binlogs *datapb.CompactionSegmentBinlogs, _ int) int64 {
return binlogs.GetSegmentID()
})
deltaPaths, allPath, err := loadDeltaMap(t.plan.GetSegmentBinlogs())
if err != nil {
log.Warn("fail to merge deltalogs", zap.Error(err))
return nil, err
}
// Unable to deal with all empty segments cases, so return error
if len(allPath) == 0 {
if len(allBatchPaths) == 0 {
log.Warn("compact wrong, all segments' binlogs are empty")
return nil, errors.New("illegal compaction plan")
}
@ -345,20 +259,13 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
return nil, err
}
compactToSeg, err := t.merge(ctxTimeout, allPath, deltaPk2Ts, writer)
res, err := t.mergeSplit(ctxTimeout, allBatchPaths, deltaPk2Ts)
if err != nil {
log.Warn("compact wrong, fail to merge", zap.Error(err))
log.Warn("compact wrong, failed to mergeSplit", zap.Error(err))
return nil, err
}
log.Info("compact done",
zap.Int64("compact to segment", targetSegID),
zap.Int64s("compact from segments", segIDs),
zap.Int("num of binlog paths", len(compactToSeg.GetInsertLogs())),
zap.Int("num of stats paths", 1),
zap.Int("num of delta paths", len(compactToSeg.GetDeltalogs())),
zap.Duration("compact elapse", time.Since(compactStart)),
)
log.Info("compact done", zap.Duration("compact elapse", time.Since(compactStart)))
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()))
@ -367,30 +274,35 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
State: datapb.CompactionTaskState_completed,
PlanID: t.GetPlanID(),
Channel: t.GetChannelName(),
Segments: []*datapb.CompactionSegment{compactToSeg},
Segments: res,
Type: t.plan.GetType(),
}
return planResult, nil
}
func (t *mixCompactionTask) GetCollection() typeutil.UniqueID {
// The length of SegmentBinlogs is checked before task enqueueing.
return t.plan.GetSegmentBinlogs()[0].GetCollectionID()
func (t *mixCompactionTask) Complete() {
t.done <- struct{}{}
}
func (t *mixCompactionTask) isExpiredEntity(ts typeutil.Timestamp) bool {
now := t.currentTs
func (t *mixCompactionTask) Stop() {
t.cancel()
<-t.done
}
// entity expire is not enabled if duration <= 0
if t.plan.GetCollectionTtl() <= 0 {
return false
}
func (t *mixCompactionTask) GetPlanID() typeutil.UniqueID {
return t.plan.GetPlanID()
}
entityT, _ := tsoutil.ParseTS(ts)
nowT, _ := tsoutil.ParseTS(now)
func (t *mixCompactionTask) GetChannelName() string {
return t.plan.GetChannel()
}
return entityT.Add(time.Duration(t.plan.GetCollectionTtl())).Before(nowT)
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 {

View File

@ -83,7 +83,8 @@ func (s *MixCompactionTaskSuite) SetupTest() {
Type: datapb.CompactionType_MixCompaction,
Schema: s.meta.GetSchema(),
BeginLogID: 19530,
PreAllocatedSegments: &datapb.IDRange{Begin: 19530},
PreAllocatedSegments: &datapb.IDRange{Begin: 19531, End: math.MaxInt64},
MaxSize: 64 * 1024 * 1024,
}
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, s.plan)
@ -132,13 +133,6 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
s.segWriter.writer.Flush()
s.Require().NoError(err)
//statistic := &storage.PkStatistics{
// PkFilter: s.segWriter.pkstats.BF,
// MinPK: s.segWriter.pkstats.MinPk,
// MaxPK: s.segWriter.pkstats.MaxPk,
//}
//bfs := metacache.NewBloomFilterSet(statistic)
kvs, fBinlogs, err := serializeWrite(context.TODO(), alloc, s.segWriter)
s.Require().NoError(err)
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool {
@ -146,13 +140,6 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
return len(left) == 0 && len(right) == 0
})).Return(lo.Values(kvs), nil).Once()
//seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{
// CollectionID: CollectionID,
// PartitionID: PartitionID,
// ID: segID,
// NumOfRows: 1,
//}, bfs)
s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
SegmentID: segID,
FieldBinlogs: lo.Values(fBinlogs),
@ -169,7 +156,7 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
s.Equal(1, len(result.GetSegments()))
segment := result.GetSegments()[0]
s.EqualValues(19530, segment.GetSegmentID())
s.EqualValues(19531, segment.GetSegmentID())
s.EqualValues(3, segment.GetNumOfRows())
s.NotEmpty(segment.InsertLogs)
s.NotEmpty(segment.Field2StatslogPaths)
@ -183,12 +170,6 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0)
for _, segID := range segments {
s.initSegBuffer(segID)
//statistic := &storage.PkStatistics{
// PkFilter: s.segWriter.pkstats.BF,
// MinPK: s.segWriter.pkstats.MinPk,
// MaxPK: s.segWriter.pkstats.MaxPk,
//}
//bfs := metacache.NewBloomFilterSet(statistic)
kvs, fBinlogs, err := serializeWrite(context.TODO(), alloc, s.segWriter)
s.Require().NoError(err)
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool {
@ -196,13 +177,6 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
return len(left) == 0 && len(right) == 0
})).Return(lo.Values(kvs), nil).Once()
//seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{
// CollectionID: CollectionID,
// PartitionID: PartitionID,
// ID: segID,
// NumOfRows: 1,
//}, bfs)
s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
SegmentID: segID,
FieldBinlogs: lo.Values(fBinlogs),
@ -229,49 +203,15 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
s.Equal(1, len(result.GetSegments()))
segment := result.GetSegments()[0]
s.EqualValues(19530, segment.GetSegmentID())
s.EqualValues(19531, segment.GetSegmentID())
s.EqualValues(3, segment.GetNumOfRows())
s.NotEmpty(segment.InsertLogs)
s.NotEmpty(segment.Field2StatslogPaths)
s.Empty(segment.Deltalogs)
}
func (s *MixCompactionTaskSuite) TestMergeBufferFull() {
paramtable.Get().Save(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key, "1")
defer paramtable.Get().Reset(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key)
s.initSegBuffer(5)
v := storage.Value{
PK: storage.NewInt64PrimaryKey(100),
Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)),
Value: getRow(100),
}
err := s.segWriter.Write(&v)
s.Require().NoError(err)
alloc := allocator.NewLocalAllocator(888888, math.MaxInt64)
kvs, _, err := serializeWrite(context.TODO(), alloc, s.segWriter)
s.Require().NoError(err)
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, paths []string) ([][]byte, error) {
s.Require().Equal(len(paths), len(kvs))
return lo.Values(kvs), nil
})
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe()
segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, 19530, PartitionID, CollectionID)
s.Require().NoError(err)
compactionSegment, err := s.task.merge(s.task.ctx, [][]string{lo.Keys(kvs)}, nil, segWriter)
s.NoError(err)
s.NotNil(compactionSegment)
s.EqualValues(2, compactionSegment.GetNumOfRows())
}
func (s *MixCompactionTaskSuite) TestMergeEntityExpired() {
func (s *MixCompactionTaskSuite) TestSplitMergeEntityExpired() {
s.initSegBuffer(3)
// entityTs == tsoutil.ComposeTSByTime(milvusBirthday, 0)
collTTL := 864000 // 10 days
currTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(time.Second*(time.Duration(collTTL)+1)), 0)
s.task.currentTs = currTs
@ -287,22 +227,22 @@ func (s *MixCompactionTaskSuite) TestMergeEntityExpired() {
})
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe()
segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, 19530, PartitionID, CollectionID)
s.Require().NoError(err)
s.task.collectionID = CollectionID
s.task.partitionID = PartitionID
s.task.maxRows = 1000
compactionSegment, err := s.task.merge(s.task.ctx, [][]string{lo.Keys(kvs)}, nil, segWriter)
compactionSegments, err := s.task.mergeSplit(s.task.ctx, [][]string{lo.Keys(kvs)}, nil)
s.NoError(err)
s.NotNil(compactionSegment)
s.EqualValues(0, compactionSegment.GetNumOfRows())
s.Equal(0, len(compactionSegments))
}
func (s *MixCompactionTaskSuite) TestMergeNoExpiration() {
s.initSegBuffer(4)
deleteTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(10*time.Second), 0)
tests := []struct {
description string
deletions map[interface{}]uint64
expectedRowCount int
description string
deletions map[interface{}]uint64
expectedRes int
}{
{"no deletion", nil, 1},
{"mismatch deletion", map[interface{}]uint64{int64(1): deleteTs}, 1},
@ -321,13 +261,15 @@ func (s *MixCompactionTaskSuite) TestMergeNoExpiration() {
})
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe()
segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, 19530, PartitionID, CollectionID)
s.Require().NoError(err)
compactionSegment, err := s.task.merge(s.task.ctx, [][]string{lo.Keys(kvs)}, test.deletions, segWriter)
s.task.collectionID = CollectionID
s.task.partitionID = PartitionID
s.task.maxRows = 1000
res, err := s.task.mergeSplit(s.task.ctx, [][]string{lo.Keys(kvs)}, test.deletions)
s.NoError(err)
s.NotNil(compactionSegment)
s.EqualValues(test.expectedRowCount, compactionSegment.GetNumOfRows())
s.EqualValues(test.expectedRes, len(res))
if test.expectedRes > 0 {
s.EqualValues(1, res[0].GetNumOfRows())
}
})
}
}
@ -481,6 +423,12 @@ func (s *MixCompactionTaskSuite) TestCompactFail() {
_, err := s.task.Compact()
s.Error(err)
})
s.Run("Test compact failed maxSize zero", func() {
s.plan.MaxSize = 0
_, err := s.task.Compact()
s.Error(err)
})
}
func (s *MixCompactionTaskSuite) TestIsExpiredEntity() {

View File

@ -5,20 +5,211 @@
package compaction
import (
"fmt"
"context"
"math"
"go.uber.org/atomic"
"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/io"
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/typeutil"
"github.com/samber/lo"
)
// Not concurrent safe.
type MultiSegmentWriter struct {
binlogIO io.BinlogIO
allocator *compactionAlloactor
writers []*SegmentWriter
current int
maxRows int64
segmentSize int64
// segmentSize in Bytes
// segmentSize might be changed dynamicly. To make sure a compaction plan is static,
// The target segmentSize is defined when creating the compaction plan.
schema *schemapb.CollectionSchema
partitionID int64
collectionID int64
channel string
cachedMeta map[typeutil.UniqueID]map[typeutil.UniqueID]*datapb.FieldBinlog
// segID -> fieldID -> binlogs
res []*datapb.CompactionSegment
}
type compactionAlloactor struct {
segmentAlloc allocator.Interface
logIDAlloc allocator.Interface
}
func NewCompactionAllocator(segmentAlloc, logIDAlloc allocator.Interface) *compactionAlloactor {
return &compactionAlloactor{
segmentAlloc: segmentAlloc,
logIDAlloc: logIDAlloc,
}
}
func (alloc *compactionAlloactor) allocSegmentID() (typeutil.UniqueID, error) {
return alloc.segmentAlloc.AllocOne()
}
func (alloc *compactionAlloactor) getLogIDAllocator() allocator.Interface {
return alloc.logIDAlloc
}
func NewMultiSegmentWriter(binlogIO io.BinlogIO, allocator *compactionAlloactor, plan *datapb.CompactionPlan, maxRows int64, partitionID, collectionID int64) *MultiSegmentWriter {
return &MultiSegmentWriter{
binlogIO: binlogIO,
allocator: allocator,
writers: make([]*SegmentWriter, 0),
current: -1,
maxRows: maxRows, // For bloomfilter only
segmentSize: plan.GetMaxSize(),
schema: plan.GetSchema(),
partitionID: partitionID,
collectionID: collectionID,
channel: plan.GetChannel(),
cachedMeta: make(map[typeutil.UniqueID]map[typeutil.UniqueID]*datapb.FieldBinlog),
res: make([]*datapb.CompactionSegment, 0),
}
}
func (w *MultiSegmentWriter) finishCurrent() error {
writer := w.writers[w.current]
allBinlogs, ok := w.cachedMeta[writer.segmentID]
if !ok {
allBinlogs = make(map[typeutil.UniqueID]*datapb.FieldBinlog)
}
if !writer.FlushAndIsEmpty() {
kvs, partialBinlogs, err := serializeWrite(context.TODO(), w.allocator.getLogIDAllocator(), writer)
if err != nil {
return err
}
if err := w.binlogIO.Upload(context.TODO(), kvs); err != nil {
return err
}
mergeFieldBinlogs(allBinlogs, partialBinlogs)
}
sPath, err := statSerializeWrite(context.TODO(), w.binlogIO, w.allocator.getLogIDAllocator(), writer)
if err != nil {
return err
}
w.res = append(w.res, &datapb.CompactionSegment{
SegmentID: writer.GetSegmentID(),
InsertLogs: lo.Values(allBinlogs),
Field2StatslogPaths: []*datapb.FieldBinlog{sPath},
NumOfRows: writer.GetRowNum(),
Channel: w.channel,
})
log.Info("Segment writer flushed a segment",
zap.Int64("segmentID", writer.GetSegmentID()),
zap.String("channel", w.channel),
zap.Int64("totalRows", writer.GetRowNum()),
zap.Int64("totalSize", writer.GetTotalSize()))
w.cachedMeta[writer.segmentID] = nil
return nil
}
func (w *MultiSegmentWriter) addNewWriter() error {
newSegmentID, err := w.allocator.allocSegmentID()
if err != nil {
return err
}
writer, err := NewSegmentWriter(w.schema, w.maxRows, newSegmentID, w.partitionID, w.collectionID)
if err != nil {
return err
}
w.writers = append(w.writers, writer)
w.current++
return nil
}
func (w *MultiSegmentWriter) getWriter() (*SegmentWriter, error) {
if len(w.writers) == 0 {
if err := w.addNewWriter(); err != nil {
return nil, err
}
return w.writers[w.current], nil
}
if w.writers[w.current].GetTotalSize() > w.segmentSize {
if err := w.finishCurrent(); err != nil {
return nil, err
}
if err := w.addNewWriter(); err != nil {
return nil, err
}
}
return w.writers[w.current], nil
}
func (w *MultiSegmentWriter) Write(v *storage.Value) error {
writer, err := w.getWriter()
if err != nil {
return err
}
if writer.IsFull() {
// init segment fieldBinlogs if it is not exist
if _, ok := w.cachedMeta[writer.segmentID]; !ok {
w.cachedMeta[writer.segmentID] = make(map[typeutil.UniqueID]*datapb.FieldBinlog)
}
kvs, partialBinlogs, err := serializeWrite(context.TODO(), w.allocator.getLogIDAllocator(), writer)
if err != nil {
return err
}
if err := w.binlogIO.Upload(context.TODO(), kvs); err != nil {
return err
}
mergeFieldBinlogs(w.cachedMeta[writer.segmentID], partialBinlogs)
}
return writer.Write(v)
}
// Could return an empty list if every insert of the segment is deleted
func (w *MultiSegmentWriter) Finish() ([]*datapb.CompactionSegment, error) {
if w.current == -1 {
return w.res, nil
}
if !w.writers[w.current].FlushAndIsEmpty() {
if err := w.finishCurrent(); err != nil {
return nil, err
}
}
return w.res, nil
}
func NewSegmentDeltaWriter(segmentID, partitionID, collectionID int64) *SegmentDeltaWriter {
return &SegmentDeltaWriter{
deleteData: &storage.DeleteData{},
@ -103,6 +294,7 @@ type SegmentWriter struct {
collectionID int64
sch *schemapb.CollectionSchema
rowCount *atomic.Int64
syncedSize *atomic.Int64
}
func (w *SegmentWriter) GetRowNum() int64 {
@ -143,10 +335,10 @@ func (w *SegmentWriter) Write(v *storage.Value) error {
return w.writer.Write(v)
}
func (w *SegmentWriter) Finish(actualRowCount int64) (*storage.Blob, error) {
func (w *SegmentWriter) Finish() (*storage.Blob, error) {
w.writer.Flush()
codec := storage.NewInsertCodecWithSchema(&etcdpb.CollectionMeta{ID: w.collectionID, Schema: w.sch})
return codec.SerializePkStats(w.pkstats, actualRowCount)
return codec.SerializePkStats(w.pkstats, w.GetRowNum())
}
func (w *SegmentWriter) IsFull() bool {
@ -190,7 +382,13 @@ func (w *SegmentWriter) SerializeYield() ([]*storage.Blob, *writebuffer.TimeRang
return fieldData, tr, nil
}
func (w *SegmentWriter) GetTotalSize() int64 {
return w.syncedSize.Load() + int64(w.writer.WrittenMemorySize())
}
func (w *SegmentWriter) clear() {
w.syncedSize.Add(int64(w.writer.WrittenMemorySize()))
writer, closers, _ := newBinlogWriter(w.collectionID, w.partitionID, w.segmentID, w.sch)
w.writer = writer
w.closers = closers
@ -204,15 +402,10 @@ func NewSegmentWriter(sch *schemapb.CollectionSchema, maxCount int64, segID, par
return nil, err
}
var pkField *schemapb.FieldSchema
for _, fs := range sch.GetFields() {
if fs.GetIsPrimaryKey() && fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) {
pkField = fs
}
}
if pkField == nil {
pkField, err := typeutil.GetPrimaryFieldSchema(sch)
if err != nil {
log.Warn("failed to get pk field from schema")
return nil, fmt.Errorf("no pk field in schema")
return nil, err
}
stats, err := storage.NewPrimaryKeyStats(pkField.GetFieldID(), int64(pkField.GetDataType()), maxCount)
@ -232,6 +425,7 @@ func NewSegmentWriter(sch *schemapb.CollectionSchema, maxCount int64, segID, par
partitionID: partID,
collectionID: collID,
rowCount: atomic.NewInt64(0),
syncedSize: atomic.NewInt64(0),
}
return &segWriter, nil
@ -244,6 +438,6 @@ func newBinlogWriter(collID, partID, segID int64, schema *schemapb.CollectionSch
for _, w := range fieldWriters {
closers = append(closers, w.Finalize)
}
writer, err = storage.NewBinlogSerializeWriter(schema, partID, segID, fieldWriters, 1024)
writer, err = storage.NewBinlogSerializeWriter(schema, partID, segID, fieldWriters, 100)
return
}

View File

@ -592,6 +592,7 @@ message CompactionPlan {
int64 begin_logID = 17;
IDRange pre_allocated_segments = 18; // only for clustering compaction
int64 slot_usage = 19;
int64 max_size = 20;
}
message CompactionSegment {
@ -946,6 +947,7 @@ message CompactionTask{
int64 analyzeTaskID = 23;
int64 analyzeVersion = 24;
int64 lastStateStartTime = 25;
int64 max_size = 26;
}
message PartitionStatsInfo {