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

View File

@ -32,7 +32,7 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_NormalMix() {
NodeID: 1, NodeID: 1,
State: datapb.CompactionTaskState_executing, State: datapb.CompactionTaskState_executing,
InputSegments: []int64{200, 201}, InputSegments: []int64{200, 201},
ResultSegments: []int64{100}, ResultSegments: []int64{100, 200},
}, },
// plan: plan, // plan: plan,
meta: s.mockMeta, meta: s.mockMeta,
@ -67,12 +67,12 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_MixSegmentNotFound() {
State: datapb.CompactionTaskState_executing, State: datapb.CompactionTaskState_executing,
NodeID: 1, NodeID: 1,
InputSegments: []int64{200, 201}, InputSegments: []int64{200, 201},
ResultSegments: []int64{100}, ResultSegments: []int64{100, 200},
}, },
meta: s.mockMeta, meta: s.mockMeta,
} }
alloc := allocator.NewMockAllocator(s.T()) 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 task.allocator = alloc
_, err := task.BuildCompactionRequest() _, err := task.BuildCompactionRequest()
s.Error(err) s.Error(err)

View File

@ -355,26 +355,26 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
return err return err
} }
plans := t.generatePlans(group.segments, signal, ct) expectedSize := getExpectedSegmentSize(t.meta, coll)
currentID, _, err := t.allocator.AllocN(int64(len(plans) * 2)) plans := t.generatePlans(group.segments, signal, ct, expectedSize)
if err != nil {
return err
}
for _, plan := range plans { for _, plan := range plans {
totalRows := plan.A
segIDs := plan.B
if !signal.isForce && t.compactionHandler.isFull() { 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 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() start := time.Now()
planID := currentID
currentID++
targetSegmentID := currentID
currentID++
pts, _ := tsoutil.ParseTS(ct.startTime) pts, _ := tsoutil.ParseTS(ct.startTime)
task := &datapb.CompactionTask{ task := &datapb.CompactionTask{
PlanID: planID, PlanID: startID,
TriggerID: signal.id, TriggerID: signal.id,
State: datapb.CompactionTaskState_pipelining, State: datapb.CompactionTaskState_pipelining,
StartTime: pts.Unix(), StartTime: pts.Unix(),
@ -384,22 +384,26 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
CollectionID: group.collectionID, CollectionID: group.collectionID,
PartitionID: group.partitionID, PartitionID: group.partitionID,
Channel: group.channelName, Channel: group.channelName,
InputSegments: segIDs, InputSegments: inputSegmentIDs,
ResultSegments: []int64{targetSegmentID}, // pre-allocated target segment ResultSegments: []int64{startID + 1, endID}, // pre-allocated target segment
TotalRows: totalRows, TotalRows: totalRows,
Schema: coll.Schema, Schema: coll.Schema,
MaxSize: getExpandedSize(expectedSize),
} }
err := t.compactionHandler.enqueueCompaction(task) err = t.compactionHandler.enqueueCompaction(task)
if err != nil { if err != nil {
log.Warn("failed to execute compaction task", 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)) zap.Error(err))
continue continue
} }
log.Info("time cost of generating global compaction", log.Info("time cost of generating global compaction",
zap.Int64("time cost", time.Since(start).Milliseconds()), zap.Int64("time cost", time.Since(start).Milliseconds()),
zap.Int64s("segmentIDs", segIDs)) zap.Int64s("segmentIDs", inputSegmentIDs))
} }
} }
return nil return nil
@ -457,27 +461,26 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
return return
} }
plans := t.generatePlans(segments, signal, ct) expectedSize := getExpectedSegmentSize(t.meta, coll)
currentID, _, err := t.allocator.AllocN(int64(len(plans) * 2)) plans := t.generatePlans(segments, signal, ct, expectedSize)
if err != nil {
log.Warn("fail to allocate id", zap.Error(err))
return
}
for _, plan := range plans { for _, plan := range plans {
if t.compactionHandler.isFull() { if t.compactionHandler.isFull() {
log.Warn("compaction plan skipped due to handler full", zap.Int64("collection", signal.collectionID)) log.Warn("compaction plan skipped due to handler full", zap.Int64("collection", signal.collectionID))
break 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() start := time.Now()
planID := currentID
currentID++
targetSegmentID := currentID
currentID++
pts, _ := tsoutil.ParseTS(ct.startTime) pts, _ := tsoutil.ParseTS(ct.startTime)
if err := t.compactionHandler.enqueueCompaction(&datapb.CompactionTask{ task := &datapb.CompactionTask{
PlanID: planID, PlanID: startID,
TriggerID: signal.id, TriggerID: signal.id,
State: datapb.CompactionTaskState_pipelining, State: datapb.CompactionTaskState_pipelining,
StartTime: pts.Unix(), StartTime: pts.Unix(),
@ -487,29 +490,32 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
CollectionID: collectionID, CollectionID: collectionID,
PartitionID: partitionID, PartitionID: partitionID,
Channel: channel, Channel: channel,
InputSegments: segmentIDS, InputSegments: inputSegmentIDs,
ResultSegments: []int64{targetSegmentID}, // pre-allocated target segment ResultSegments: []int64{startID + 1, endID}, // pre-allocated target segment
TotalRows: totalRows, TotalRows: totalRows,
Schema: coll.Schema, Schema: coll.Schema,
}); err != nil { MaxSize: getExpandedSize(expectedSize),
}
if err := t.compactionHandler.enqueueCompaction(task); err != nil {
log.Warn("failed to execute compaction task", log.Warn("failed to execute compaction task",
zap.Int64("collection", collectionID), zap.Int64("collection", collectionID),
zap.Int64("planID", planID), zap.Int64("triggerID", signal.id),
zap.Int64s("segmentIDs", segmentIDS), zap.Int64("planID", task.GetPlanID()),
zap.Int64s("inputSegments", inputSegmentIDs),
zap.Error(err)) zap.Error(err))
continue continue
} }
log.Info("time cost of generating compaction", 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("time cost", time.Since(start).Milliseconds()),
zap.Int64("collectionID", signal.collectionID), zap.Int64("collectionID", signal.collectionID),
zap.String("channel", channel), zap.String("channel", channel),
zap.Int64("partitionID", partitionID), 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 { if len(segments) == 0 {
log.Warn("the number of candidate segments is 0, skip to generate compaction plan") log.Warn("the number of candidate segments is 0, skip to generate compaction plan")
return []*typeutil.Pair[int64, []int64]{} return []*typeutil.Pair[int64, []int64]{}
@ -521,8 +527,6 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, signal *compa
var smallCandidates []*SegmentInfo var smallCandidates []*SegmentInfo
var nonPlannedSegments []*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 // 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 { for _, segment := range segments {
segment := segment.ShadowClone() segment := segment.ShadowClone()
@ -853,3 +857,7 @@ func (t *compactionTrigger) squeezeSmallSegmentsToBuckets(small []*SegmentInfo,
return small 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) alloc := newMock0Allocator(h.t)
t.allocator = alloc t.allocator = alloc
t.ResultSegments = []int64{100} t.ResultSegments = []int64{100, 200}
plan, err := t.BuildCompactionRequest() plan, err := t.BuildCompactionRequest()
h.spyChan <- plan h.spyChan <- plan
return err return err
@ -486,8 +486,9 @@ func Test_compactionTrigger_force(t *testing.T) {
Channel: "ch1", Channel: "ch1",
TotalRows: 200, TotalRows: 200,
Schema: schema, Schema: schema,
PreAllocatedSegments: &datapb.IDRange{Begin: 100}, PreAllocatedSegments: &datapb.IDRange{Begin: 100, End: 200},
SlotUsage: 8, SlotUsage: 8,
MaxSize: 1342177280,
}, },
}, },
}, },
@ -780,6 +781,7 @@ func Test_compactionTrigger_force_maxSegmentLimit(t *testing.T) {
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(), TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(),
Type: datapb.CompactionType_MixCompaction, Type: datapb.CompactionType_MixCompaction,
Channel: "ch1", Channel: "ch1",
MaxSize: 1342177280,
}, },
}, },
}, },

View File

@ -24,15 +24,11 @@ import (
"github.com/samber/lo" "github.com/samber/lo"
"go.uber.org/zap" "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/datacoord/allocator"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/log" "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/lock"
"github.com/milvus-io/milvus/pkg/util/logutil" "github.com/milvus-io/milvus/pkg/util/logutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
) )
type CompactionTriggerType int8 type CompactionTriggerType int8
@ -363,11 +359,14 @@ func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.C
func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Context, view CompactionView) { func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Context, view CompactionView) {
log := log.With(zap.String("view", view.String())) 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 { 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 return
} }
collection, err := m.handler.GetCollection(ctx, view.GetGroupLabel().CollectionID) collection, err := m.handler.GetCollection(ctx, view.GetGroupLabel().CollectionID)
if err != nil { if err != nil {
log.Warn("Failed to submit compaction view to scheduler because get collection fail", zap.Error(err)) 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() { for _, s := range view.GetSegmentsView() {
totalRows += s.NumOfRows totalRows += s.NumOfRows
} }
expectedSize := getExpectedSegmentSize(m.meta, collection)
task := &datapb.CompactionTask{ task := &datapb.CompactionTask{
PlanID: taskID, PlanID: startID,
TriggerID: view.(*MixSegmentView).triggerID, TriggerID: view.(*MixSegmentView).triggerID,
State: datapb.CompactionTaskState_pipelining, State: datapb.CompactionTaskState_pipelining,
StartTime: time.Now().Unix(), StartTime: time.Now().Unix(),
@ -390,9 +391,10 @@ func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Conte
Channel: view.GetGroupLabel().Channel, Channel: view.GetGroupLabel().Channel,
Schema: collection.Schema, Schema: collection.Schema,
InputSegments: lo.Map(view.GetSegmentsView(), func(segmentView *SegmentView, _ int) int64 { return segmentView.ID }), InputSegments: lo.Map(view.GetSegmentsView(), func(segmentView *SegmentView, _ int) int64 { return segmentView.ID }),
ResultSegments: []int64{taskID + 1}, ResultSegments: []int64{startID + 1, endID},
TotalRows: totalRows, TotalRows: totalRows,
LastStateStartTime: time.Now().Unix(), LastStateStartTime: time.Now().Unix(),
MaxSize: getExpandedSize(expectedSize),
} }
err = m.compactionHandler.enqueueCompaction(task) err = m.compactionHandler.enqueueCompaction(task)
if err != nil { if err != nil {
@ -409,21 +411,8 @@ func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Conte
) )
} }
func getExpectedSegmentSize(meta *meta, collection *collectionInfo) int64 { func getExpectedSegmentSize(meta *meta, collInfo *collectionInfo) int64 {
indexInfos := meta.indexMeta.GetIndexesForCollection(collection.ID, "") allDiskIndex := meta.indexMeta.AreAllDiskIndex(collInfo.ID, collInfo.Schema)
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)
if allDiskIndex { if allDiskIndex {
// Only if all vector fields index type are DiskANN, recalc segment max size here. // Only if all vector fields index type are DiskANN, recalc segment max size here.
return Params.DataCoordCfg.DiskSegmentMaxSize.GetAsInt64() * 1024 * 1024 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) updateSegStateAndPrepareMetrics(cloned, commonpb.SegmentState_Dropped, metricMutation)
} }
// MixCompaction / MergeCompaction will generates one and only one segment log = log.With(zap.Int64s("compactFrom", compactFromSegIDs))
compactToSegment := result.GetSegments()[0]
compactToSegmentInfo := NewSegmentInfo( compactToSegments := make([]*SegmentInfo, 0)
&datapb.SegmentInfo{ for _, compactToSegment := range result.GetSegments() {
ID: compactToSegment.GetSegmentID(), compactToSegmentInfo := NewSegmentInfo(
CollectionID: compactFromSegInfos[0].CollectionID, &datapb.SegmentInfo{
PartitionID: compactFromSegInfos[0].PartitionID, ID: compactToSegment.GetSegmentID(),
InsertChannel: t.GetChannel(), CollectionID: compactFromSegInfos[0].CollectionID,
NumOfRows: compactToSegment.NumOfRows, PartitionID: compactFromSegInfos[0].PartitionID,
State: commonpb.SegmentState_Flushed, InsertChannel: t.GetChannel(),
MaxRowNum: compactFromSegInfos[0].MaxRowNum, NumOfRows: compactToSegment.NumOfRows,
Binlogs: compactToSegment.GetInsertLogs(), State: commonpb.SegmentState_Flushed,
Statslogs: compactToSegment.GetField2StatslogPaths(), MaxRowNum: compactFromSegInfos[0].MaxRowNum,
Deltalogs: compactToSegment.GetDeltalogs(), Binlogs: compactToSegment.GetInsertLogs(),
Statslogs: compactToSegment.GetField2StatslogPaths(),
Deltalogs: compactToSegment.GetDeltalogs(),
CreatedByCompaction: true, CreatedByCompaction: true,
CompactionFrom: compactFromSegIDs, CompactionFrom: compactFromSegIDs,
LastExpireTime: tsoutil.ComposeTSByTime(time.Unix(t.GetStartTime(), 0), 0), LastExpireTime: tsoutil.ComposeTSByTime(time.Unix(t.GetStartTime(), 0), 0),
Level: datapb.SegmentLevel_L1, Level: datapb.SegmentLevel_L1,
StartPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition { StartPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetStartPosition() return info.GetStartPosition()
})), })),
DmlPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition { DmlPosition: getMinPosition(lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetDmlPosition() return info.GetDmlPosition()
})), })),
}) })
// L1 segment with NumRows=0 will be discarded, so no need to change the metric // L1 segment with NumRows=0 will be discarded, so no need to change the metric
if compactToSegmentInfo.GetNumOfRows() > 0 { if compactToSegmentInfo.GetNumOfRows() > 0 {
// metrics mutation for compactTo segments // metrics mutation for compactTo segments
metricMutation.addNewSeg(compactToSegmentInfo.GetState(), compactToSegmentInfo.GetLevel(), compactToSegmentInfo.GetNumOfRows()) metricMutation.addNewSeg(compactToSegmentInfo.GetState(), compactToSegmentInfo.GetLevel(), compactToSegmentInfo.GetNumOfRows())
} else { } else {
compactToSegmentInfo.State = commonpb.SegmentState_Dropped 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") log.Debug("meta update: prepare for meta mutation - complete")
compactFromInfos := lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *datapb.SegmentInfo { compactFromInfos := lo.Map(compactFromSegInfos, func(info *SegmentInfo, _ int) *datapb.SegmentInfo {
return info.SegmentInfo return info.SegmentInfo
}) })
log.Debug("meta update: alter meta store for compaction updates", compactToInfos := lo.Map(compactToSegments, func(info *SegmentInfo, _ int) *datapb.SegmentInfo {
zap.Int("binlog count", len(compactToSegmentInfo.GetBinlogs())), return info.SegmentInfo
zap.Int("statslog count", len(compactToSegmentInfo.GetStatslogs())), })
zap.Int("deltalog count", len(compactToSegmentInfo.GetDeltalogs())),
) binlogs := make([]metastore.BinlogsIncrement, 0)
if err := m.catalog.AlterSegments(m.ctx, []*datapb.SegmentInfo{compactToSegmentInfo.SegmentInfo}, for _, seg := range compactToInfos {
metastore.BinlogsIncrement{Segment: compactToSegmentInfo.SegmentInfo}, binlogs = append(binlogs, metastore.BinlogsIncrement{Segment: seg})
); err != nil { }
// 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)) log.Warn("fail to alter compactTo segments", zap.Error(err))
return nil, nil, 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)) log.Warn("fail to alter compactFrom segments", zap.Error(err))
return nil, nil, err return nil, nil, err
} }
lo.ForEach(compactFromSegInfos, func(info *SegmentInfo, _ int) { lo.ForEach(compactFromSegInfos, func(info *SegmentInfo, _ int) {
m.segments.SetSegment(info.GetID(), info) 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") 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) { 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) defer t.cleanUp(ctx)
// 1, download delta logs to build deltaMap // 1, download delta logs to build deltaMap
deltaBlobs, _, err := loadDeltaMap(t.plan.GetSegmentBinlogs()) deltaBlobs, _, err := composePaths(t.plan.GetSegmentBinlogs())
if err != nil { if err != nil {
return nil, err return nil, err
} }

View File

@ -104,7 +104,7 @@ func mergeDeltalogs(ctx context.Context, io io.BinlogIO, dpaths map[typeutil.Uni
return pk2ts, nil 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 { if err := binlog.DecompressCompactionBinlogs(segments); err != nil {
log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err)) log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err))
return nil, nil, 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) { 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") ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "statslog serializeWrite")
defer span.End() defer span.End()
sblob, err := writer.Finish(writer.GetRowNum()) sblob, err := writer.Finish()
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -220,3 +220,12 @@ func uploadStatsBlobs(ctx context.Context, collectionID, partitionID, segmentID,
return statFieldLog, nil 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" "github.com/milvus-io/milvus/pkg/util/typeutil"
) )
// for MixCompaction only
type mixCompactionTask struct { type mixCompactionTask struct {
binlogIO io.BinlogIO binlogIO io.BinlogIO
allocator allocator.Interface
currentTs typeutil.Timestamp currentTs typeutil.Timestamp
plan *datapb.CompactionPlan plan *datapb.CompactionPlan
@ -52,11 +50,16 @@ type mixCompactionTask struct {
ctx context.Context ctx context.Context
cancel context.CancelFunc cancel context.CancelFunc
collectionID int64
partitionID int64
targetSize int64
maxRows int64
pkID int64
done chan struct{} done chan struct{}
tr *timerecord.TimeRecorder tr *timerecord.TimeRecorder
} }
// make sure compactionTask implements compactor interface
var _ Compactor = (*mixCompactionTask)(nil) var _ Compactor = (*mixCompactionTask)(nil)
func NewMixCompactionTask( func NewMixCompactionTask(
@ -65,76 +68,77 @@ func NewMixCompactionTask(
plan *datapb.CompactionPlan, plan *datapb.CompactionPlan,
) *mixCompactionTask { ) *mixCompactionTask {
ctx1, cancel := context.WithCancel(ctx) ctx1, cancel := context.WithCancel(ctx)
alloc := allocator.NewLocalAllocator(plan.GetBeginLogID(), math.MaxInt64)
return &mixCompactionTask{ return &mixCompactionTask{
ctx: ctx1, ctx: ctx1,
cancel: cancel, cancel: cancel,
binlogIO: binlogIO, binlogIO: binlogIO,
allocator: alloc,
plan: plan, plan: plan,
tr: timerecord.NewTimeRecorder("mix compaction"), tr: timerecord.NewTimeRecorder("mergeSplit compaction"),
currentTs: tsoutil.GetCurrentTime(), currentTs: tsoutil.GetCurrentTime(),
done: make(chan struct{}, 1), done: make(chan struct{}, 1),
} }
} }
func (t *mixCompactionTask) Complete() { // preCompact exams whether its a valid compaction plan, and init the collectionID and partitionID
t.done <- struct{}{} func (t *mixCompactionTask) preCompact() error {
} if ok := funcutil.CheckCtxValid(t.ctx); !ok {
return t.ctx.Err()
}
func (t *mixCompactionTask) Stop() { if len(t.plan.GetSegmentBinlogs()) < 1 {
t.cancel() return errors.Newf("compaction plan is illegal, there's no segments in compaction plan, planID = %d", t.GetPlanID())
<-t.done }
}
func (t *mixCompactionTask) GetPlanID() typeutil.UniqueID { if t.plan.GetMaxSize() == 0 {
return t.plan.GetPlanID() return errors.Newf("compaction plan is illegal, empty maxSize, planID = %d", t.GetPlanID())
} }
func (t *mixCompactionTask) GetChannelName() string { t.collectionID = t.plan.GetSegmentBinlogs()[0].GetCollectionID()
return t.plan.GetChannel() t.partitionID = t.plan.GetSegmentBinlogs()[0].GetPartitionID()
} t.targetSize = t.plan.GetMaxSize()
func (t *mixCompactionTask) GetCompactionType() datapb.CompactionType { currSize := int64(0)
return t.plan.GetType() 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 // MemorySize might be incorrectly
func (t *mixCompactionTask) getNumRows() int64 { currSize += binlog.GetMemorySize()
numRows := int64(0)
for _, binlog := range t.plan.SegmentBinlogs {
if len(binlog.GetFieldBinlogs()) > 0 {
for _, ct := range binlog.GetFieldBinlogs()[0].GetBinlogs() {
numRows += ct.GetEntriesNum()
} }
} }
} }
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, ctx context.Context,
binlogPaths [][]string, binlogPaths [][]string,
delta map[interface{}]typeutil.Timestamp, delta map[interface{}]typeutil.Timestamp,
writer *SegmentWriter, ) ([]*datapb.CompactionSegment, error) {
) (*datapb.CompactionSegment, error) {
_ = t.tr.RecordSpan() _ = t.tr.RecordSpan()
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "CompactMerge") ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "MergeSplit")
defer span.End() 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 ( segIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedSegments().GetBegin(), t.plan.GetPreAllocatedSegments().GetEnd())
syncBatchCount int // binlog batch count logIDAlloc := allocator.NewLocalAllocator(t.plan.GetBeginLogID(), math.MaxInt64)
remainingRowCount int64 // the number of remaining entities compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
expiredRowCount int64 // the number of expired entities mWriter := NewMultiSegmentWriter(t.binlogIO, compAlloc, t.plan, t.maxRows, t.partitionID, t.collectionID)
deletedRowCount int64 = 0
unflushedRowCount int64 = 0
// All binlog meta of a segment
allBinlogs = make(map[typeutil.UniqueID]*datapb.FieldBinlog)
)
isValueDeleted := func(v *storage.Value) bool { isValueDeleted := func(v *storage.Value) bool {
ts, ok := delta[v.PK.GetValue()] ts, ok := delta[v.PK.GetValue()]
@ -147,25 +151,27 @@ func (t *mixCompactionTask) merge(
return false return false
} }
downloadTimeCost := time.Duration(0) deletedRowCount := int64(0)
serWriteTimeCost := time.Duration(0) expiredRowCount := int64(0)
uploadTimeCost := time.Duration(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 { for _, paths := range binlogPaths {
log := log.With(zap.Strings("paths", paths)) log := log.With(zap.Strings("paths", paths))
downloadStart := time.Now()
allValues, err := t.binlogIO.Download(ctx, paths) allValues, err := t.binlogIO.Download(ctx, paths)
if err != nil { if err != nil {
log.Warn("compact wrong, fail to download insertLogs", zap.Error(err)) log.Warn("compact wrong, fail to download insertLogs", zap.Error(err))
return nil, err return nil, err
} }
downloadTimeCost += time.Since(downloadStart)
blobs := lo.Map(allValues, func(v []byte, i int) *storage.Blob { blobs := lo.Map(allValues, func(v []byte, i int) *storage.Blob {
return &storage.Blob{Key: paths[i], Value: v} 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 { if err != nil {
log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err)) log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err))
return nil, err return nil, err
@ -193,148 +199,56 @@ func (t *mixCompactionTask) merge(
continue continue
} }
err = writer.Write(v) err = mWriter.Write(v)
if err != nil { if err != nil {
log.Warn("compact wrong, failed to writer row", zap.Error(err)) log.Warn("compact wrong, failed to writer row", zap.Error(err))
return nil, 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
}
} }
} }
res, err := mWriter.Finish()
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)
if err != nil { if err != nil {
log.Warn("compact wrong, failed to serialize write segment stats", log.Warn("compact wrong, failed to finish writer", zap.Error(err))
zap.Int64("remaining row count", remainingRowCount), zap.Error(err))
return nil, 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() totalElapse := t.tr.RecordSpan()
log.Info("compact mergeSplit end",
log.Info("compact merge end", zap.Int64s("mergeSplit to segments", lo.Keys(mWriter.cachedMeta)),
zap.Int64("remaining row count", remainingRowCount),
zap.Int64("deleted row count", deletedRowCount), zap.Int64("deleted row count", deletedRowCount),
zap.Int64("expired entities", expiredRowCount), 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)) zap.Duration("total elapse", totalElapse))
return pack, nil return res, 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()...)
}
} }
func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) { func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
durInQueue := t.tr.RecordSpan() durInQueue := t.tr.RecordSpan()
compactStart := time.Now()
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("MixCompact-%d", t.GetPlanID())) ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("MixCompact-%d", t.GetPlanID()))
defer span.End() defer span.End()
compactStart := time.Now()
if len(t.plan.GetSegmentBinlogs()) < 1 { if err := t.preCompact(); err != nil {
log.Warn("compact wrong, there's no segments in segment binlogs", zap.Int64("planID", t.plan.GetPlanID())) log.Warn("compact wrong, failed to preCompact", zap.Error(err))
return nil, errors.New("compaction plan is illegal") return nil, err
} }
collectionID := t.plan.GetSegmentBinlogs()[0].GetCollectionID() log := log.Ctx(ctx).With(zap.Int64("planID", t.GetPlanID()),
partitionID := t.plan.GetSegmentBinlogs()[0].GetPartitionID() zap.Int64("collectionID", t.collectionID),
zap.Int64("partitionID", t.partitionID),
log := log.Ctx(ctx).With(zap.Int64("planID", t.plan.GetPlanID()),
zap.Int64("collectionID", collectionID),
zap.Int64("partitionID", partitionID),
zap.Int32("timeout in seconds", t.plan.GetTimeoutInSeconds())) 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) ctxTimeout, cancelAll := context.WithTimeout(ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
defer cancelAll() defer cancelAll()
log.Info("compact start") log.Info("compact start")
deltaPaths, allBatchPaths, err := composePaths(t.plan.GetSegmentBinlogs())
targetSegID := t.plan.GetPreAllocatedSegments().GetBegin()
previousRowCount := t.getNumRows()
writer, err := NewSegmentWriter(t.plan.GetSchema(), previousRowCount, targetSegID, partitionID, collectionID)
if err != nil { 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 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 // 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") log.Warn("compact wrong, all segments' binlogs are empty")
return nil, errors.New("illegal compaction plan") return nil, errors.New("illegal compaction plan")
} }
@ -345,20 +259,13 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
return nil, err return nil, err
} }
compactToSeg, err := t.merge(ctxTimeout, allPath, deltaPk2Ts, writer) res, err := t.mergeSplit(ctxTimeout, allBatchPaths, deltaPk2Ts)
if err != nil { 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 return nil, err
} }
log.Info("compact done", log.Info("compact done", zap.Duration("compact elapse", time.Since(compactStart)))
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)),
)
metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.plan.GetType().String()).Observe(float64(t.tr.ElapseSpan().Milliseconds())) 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())) 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, State: datapb.CompactionTaskState_completed,
PlanID: t.GetPlanID(), PlanID: t.GetPlanID(),
Channel: t.GetChannelName(), Channel: t.GetChannelName(),
Segments: []*datapb.CompactionSegment{compactToSeg}, Segments: res,
Type: t.plan.GetType(), Type: t.plan.GetType(),
} }
return planResult, nil return planResult, nil
} }
func (t *mixCompactionTask) GetCollection() typeutil.UniqueID { func (t *mixCompactionTask) Complete() {
// The length of SegmentBinlogs is checked before task enqueueing. t.done <- struct{}{}
return t.plan.GetSegmentBinlogs()[0].GetCollectionID()
} }
func (t *mixCompactionTask) isExpiredEntity(ts typeutil.Timestamp) bool { func (t *mixCompactionTask) Stop() {
now := t.currentTs t.cancel()
<-t.done
}
// entity expire is not enabled if duration <= 0 func (t *mixCompactionTask) GetPlanID() typeutil.UniqueID {
if t.plan.GetCollectionTtl() <= 0 { return t.plan.GetPlanID()
return false }
}
entityT, _ := tsoutil.ParseTS(ts) func (t *mixCompactionTask) GetChannelName() string {
nowT, _ := tsoutil.ParseTS(now) 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 { func (t *mixCompactionTask) GetSlotUsage() int64 {

View File

@ -83,7 +83,8 @@ func (s *MixCompactionTaskSuite) SetupTest() {
Type: datapb.CompactionType_MixCompaction, Type: datapb.CompactionType_MixCompaction,
Schema: s.meta.GetSchema(), Schema: s.meta.GetSchema(),
BeginLogID: 19530, 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) s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, s.plan)
@ -132,13 +133,6 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
s.segWriter.writer.Flush() s.segWriter.writer.Flush()
s.Require().NoError(err) 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) kvs, fBinlogs, err := serializeWrite(context.TODO(), alloc, s.segWriter)
s.Require().NoError(err) s.Require().NoError(err)
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool { 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 len(left) == 0 && len(right) == 0
})).Return(lo.Values(kvs), nil).Once() })).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{ s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
SegmentID: segID, SegmentID: segID,
FieldBinlogs: lo.Values(fBinlogs), FieldBinlogs: lo.Values(fBinlogs),
@ -169,7 +156,7 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
s.Equal(1, len(result.GetSegments())) s.Equal(1, len(result.GetSegments()))
segment := result.GetSegments()[0] segment := result.GetSegments()[0]
s.EqualValues(19530, segment.GetSegmentID()) s.EqualValues(19531, segment.GetSegmentID())
s.EqualValues(3, segment.GetNumOfRows()) s.EqualValues(3, segment.GetNumOfRows())
s.NotEmpty(segment.InsertLogs) s.NotEmpty(segment.InsertLogs)
s.NotEmpty(segment.Field2StatslogPaths) s.NotEmpty(segment.Field2StatslogPaths)
@ -183,12 +170,6 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0) s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0)
for _, segID := range segments { for _, segID := range segments {
s.initSegBuffer(segID) 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) kvs, fBinlogs, err := serializeWrite(context.TODO(), alloc, s.segWriter)
s.Require().NoError(err) s.Require().NoError(err)
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool { 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 len(left) == 0 && len(right) == 0
})).Return(lo.Values(kvs), nil).Once() })).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{ s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
SegmentID: segID, SegmentID: segID,
FieldBinlogs: lo.Values(fBinlogs), FieldBinlogs: lo.Values(fBinlogs),
@ -229,49 +203,15 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
s.Equal(1, len(result.GetSegments())) s.Equal(1, len(result.GetSegments()))
segment := result.GetSegments()[0] segment := result.GetSegments()[0]
s.EqualValues(19530, segment.GetSegmentID()) s.EqualValues(19531, segment.GetSegmentID())
s.EqualValues(3, segment.GetNumOfRows()) s.EqualValues(3, segment.GetNumOfRows())
s.NotEmpty(segment.InsertLogs) s.NotEmpty(segment.InsertLogs)
s.NotEmpty(segment.Field2StatslogPaths) s.NotEmpty(segment.Field2StatslogPaths)
s.Empty(segment.Deltalogs) s.Empty(segment.Deltalogs)
} }
func (s *MixCompactionTaskSuite) TestMergeBufferFull() { func (s *MixCompactionTaskSuite) TestSplitMergeEntityExpired() {
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() {
s.initSegBuffer(3) s.initSegBuffer(3)
// entityTs == tsoutil.ComposeTSByTime(milvusBirthday, 0)
collTTL := 864000 // 10 days collTTL := 864000 // 10 days
currTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(time.Second*(time.Duration(collTTL)+1)), 0) currTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(time.Second*(time.Duration(collTTL)+1)), 0)
s.task.currentTs = currTs s.task.currentTs = currTs
@ -287,22 +227,22 @@ func (s *MixCompactionTaskSuite) TestMergeEntityExpired() {
}) })
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe() s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe()
segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, 19530, PartitionID, CollectionID) s.task.collectionID = CollectionID
s.Require().NoError(err) 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.NoError(err)
s.NotNil(compactionSegment) s.Equal(0, len(compactionSegments))
s.EqualValues(0, compactionSegment.GetNumOfRows())
} }
func (s *MixCompactionTaskSuite) TestMergeNoExpiration() { func (s *MixCompactionTaskSuite) TestMergeNoExpiration() {
s.initSegBuffer(4) s.initSegBuffer(4)
deleteTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(10*time.Second), 0) deleteTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(10*time.Second), 0)
tests := []struct { tests := []struct {
description string description string
deletions map[interface{}]uint64 deletions map[interface{}]uint64
expectedRowCount int expectedRes int
}{ }{
{"no deletion", nil, 1}, {"no deletion", nil, 1},
{"mismatch deletion", map[interface{}]uint64{int64(1): deleteTs}, 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() s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe()
segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, 19530, PartitionID, CollectionID) s.task.collectionID = CollectionID
s.Require().NoError(err) s.task.partitionID = PartitionID
s.task.maxRows = 1000
compactionSegment, err := s.task.merge(s.task.ctx, [][]string{lo.Keys(kvs)}, test.deletions, segWriter) res, err := s.task.mergeSplit(s.task.ctx, [][]string{lo.Keys(kvs)}, test.deletions)
s.NoError(err) s.NoError(err)
s.NotNil(compactionSegment) s.EqualValues(test.expectedRes, len(res))
s.EqualValues(test.expectedRowCount, compactionSegment.GetNumOfRows()) if test.expectedRes > 0 {
s.EqualValues(1, res[0].GetNumOfRows())
}
}) })
} }
} }
@ -481,6 +423,12 @@ func (s *MixCompactionTaskSuite) TestCompactFail() {
_, err := s.task.Compact() _, err := s.task.Compact()
s.Error(err) 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() { func (s *MixCompactionTaskSuite) TestIsExpiredEntity() {

View File

@ -5,20 +5,211 @@
package compaction package compaction
import ( import (
"fmt" "context"
"math" "math"
"go.uber.org/atomic" "go.uber.org/atomic"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "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/flushcommon/writebuffer"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/typeutil" "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 { func NewSegmentDeltaWriter(segmentID, partitionID, collectionID int64) *SegmentDeltaWriter {
return &SegmentDeltaWriter{ return &SegmentDeltaWriter{
deleteData: &storage.DeleteData{}, deleteData: &storage.DeleteData{},
@ -103,6 +294,7 @@ type SegmentWriter struct {
collectionID int64 collectionID int64
sch *schemapb.CollectionSchema sch *schemapb.CollectionSchema
rowCount *atomic.Int64 rowCount *atomic.Int64
syncedSize *atomic.Int64
} }
func (w *SegmentWriter) GetRowNum() int64 { func (w *SegmentWriter) GetRowNum() int64 {
@ -143,10 +335,10 @@ func (w *SegmentWriter) Write(v *storage.Value) error {
return w.writer.Write(v) return w.writer.Write(v)
} }
func (w *SegmentWriter) Finish(actualRowCount int64) (*storage.Blob, error) { func (w *SegmentWriter) Finish() (*storage.Blob, error) {
w.writer.Flush() w.writer.Flush()
codec := storage.NewInsertCodecWithSchema(&etcdpb.CollectionMeta{ID: w.collectionID, Schema: w.sch}) 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 { func (w *SegmentWriter) IsFull() bool {
@ -190,7 +382,13 @@ func (w *SegmentWriter) SerializeYield() ([]*storage.Blob, *writebuffer.TimeRang
return fieldData, tr, nil return fieldData, tr, nil
} }
func (w *SegmentWriter) GetTotalSize() int64 {
return w.syncedSize.Load() + int64(w.writer.WrittenMemorySize())
}
func (w *SegmentWriter) clear() { func (w *SegmentWriter) clear() {
w.syncedSize.Add(int64(w.writer.WrittenMemorySize()))
writer, closers, _ := newBinlogWriter(w.collectionID, w.partitionID, w.segmentID, w.sch) writer, closers, _ := newBinlogWriter(w.collectionID, w.partitionID, w.segmentID, w.sch)
w.writer = writer w.writer = writer
w.closers = closers w.closers = closers
@ -204,15 +402,10 @@ func NewSegmentWriter(sch *schemapb.CollectionSchema, maxCount int64, segID, par
return nil, err return nil, err
} }
var pkField *schemapb.FieldSchema pkField, err := typeutil.GetPrimaryFieldSchema(sch)
for _, fs := range sch.GetFields() { if err != nil {
if fs.GetIsPrimaryKey() && fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) {
pkField = fs
}
}
if pkField == nil {
log.Warn("failed to get pk field from schema") 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) 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, partitionID: partID,
collectionID: collID, collectionID: collID,
rowCount: atomic.NewInt64(0), rowCount: atomic.NewInt64(0),
syncedSize: atomic.NewInt64(0),
} }
return &segWriter, nil return &segWriter, nil
@ -244,6 +438,6 @@ func newBinlogWriter(collID, partID, segID int64, schema *schemapb.CollectionSch
for _, w := range fieldWriters { for _, w := range fieldWriters {
closers = append(closers, w.Finalize) closers = append(closers, w.Finalize)
} }
writer, err = storage.NewBinlogSerializeWriter(schema, partID, segID, fieldWriters, 1024) writer, err = storage.NewBinlogSerializeWriter(schema, partID, segID, fieldWriters, 100)
return return
} }

View File

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