mirror of https://github.com/milvus-io/milvus.git
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
parent
1c3abd1aea
commit
323400c190
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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())
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
},
|
||||
},
|
||||
},
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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()...)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
Loading…
Reference in New Issue