mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
enhance: Valid row count result and avoid misleading warn log for sort compaction (#46746)
issue: https://github.com/milvus-io/milvus/issues/46743 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> ## Summary **Core Invariant:** Sort compaction tasks must not be created concurrently for the same segment. The system relies on atomic check-and-set semantics to prevent duplicate task creation. **What Logic is Improved:** The code now guards sort compaction task creation with an explicit `CheckAndSetSegmentsCompacting` check before calling `createSortCompactionTask`. Previously, tasks could be attempted for segments already undergoing compaction, triggering warning logs that incorrectly suggested task creation failures. The fix skips task creation when a segment is already compacting, avoiding these misleading warnings entirely. **Why No Data Loss or Regression:** - The `CheckAndSetSegmentsCompacting` method atomically checks whether a segment is already being compacted and only proceeds if it's not; this is the correct guard pattern for preventing concurrent compactions - When a segment is already compacting (`isCompacting == true`), the code correctly increments the done counter and skips to the next segment, which is the intended behavior (no wasted task creation attempts) - The function signature change to `createSortCompactionTask` adds only an internal parameter (the current task context for logging); no public APIs are affected - Logging refactoring maintains semantic equivalence while providing task-scoped context **Concrete Fix:** The misleading warning during sort compaction is eliminated by preventing task creation attempts for already-compacting segments through the mutex-protected `CheckAndSetSegmentsCompacting` guard, rather than attempting creation and failing downstream. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
This commit is contained in:
parent
ee216877bb
commit
fc2db1a674
@ -355,7 +355,7 @@ func (c *importChecker) checkSortingJob(job ImportJob) {
|
||||
sortSegmentIDs := task.(*importTask).GetSortedSegmentIDs()
|
||||
taskCnt += len(originSegmentIDs)
|
||||
for i, originSegmentID := range originSegmentIDs {
|
||||
taskLogFields := WrapTaskLog(task, zap.Int64("origin", originSegmentID), zap.Int64("target", sortSegmentIDs[i]))
|
||||
logger := log.With(WrapTaskLog(task, zap.Int64("origin", originSegmentID), zap.Int64("target", sortSegmentIDs[i]))...)
|
||||
originSegment := c.meta.GetHealthySegment(c.ctx, originSegmentID)
|
||||
targetSegment := c.meta.GetHealthySegment(c.ctx, sortSegmentIDs[i])
|
||||
if originSegment == nil {
|
||||
@ -363,26 +363,31 @@ func (c *importChecker) checkSortingJob(job ImportJob) {
|
||||
doneCnt++
|
||||
continue
|
||||
}
|
||||
if targetSegment == nil {
|
||||
compactionTask, err := createSortCompactionTask(c.ctx, originSegment, sortSegmentIDs[i], c.meta, c.handler, c.alloc)
|
||||
if targetSegment != nil {
|
||||
// sort compaction is already done
|
||||
doneCnt++
|
||||
continue
|
||||
}
|
||||
// if not compacting, trigger sort compaction task
|
||||
isCompacting := c.meta.IsSegmentCompacting(originSegmentID)
|
||||
if !isCompacting {
|
||||
compactionTask, err := createSortCompactionTask(c.ctx, task, originSegment, sortSegmentIDs[i], c.meta, c.handler, c.alloc)
|
||||
if err != nil {
|
||||
log.Warn("create sort compaction task failed", zap.Int64("segmentID", originSegmentID), zap.Error(err))
|
||||
logger.Warn("create sort compaction task failed", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
if compactionTask == nil {
|
||||
log.Info("maybe it no need to create sort compaction task", zap.Int64("segmentID", originSegmentID))
|
||||
logger.Info("maybe it no need to create sort compaction task")
|
||||
doneCnt++
|
||||
continue
|
||||
}
|
||||
log.Info("create sort compaction task success", taskLogFields...)
|
||||
err = c.ci.enqueueCompaction(compactionTask)
|
||||
if err != nil {
|
||||
log.Warn("sort compaction task enqueue failed", zap.Error(err))
|
||||
logger.Warn("sort compaction task enqueue failed", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
continue
|
||||
logger.Info("create sort compaction task and enqueue success")
|
||||
}
|
||||
doneCnt++
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -205,6 +205,7 @@ func (t *importTask) QueryTaskOnWorker(cluster session.Cluster) {
|
||||
}
|
||||
}
|
||||
if resp.GetState() == datapb.ImportTaskStateV2_Completed {
|
||||
totalRows := int64(0)
|
||||
for _, info := range resp.GetImportSegmentsInfo() {
|
||||
// try to parse path and fill logID
|
||||
err = binlog.CompressBinLogs(info.GetBinlogs(), info.GetDeltalogs(), info.GetStatslogs(), info.GetBm25Logs())
|
||||
@ -226,6 +227,7 @@ func (t *importTask) QueryTaskOnWorker(cluster session.Cluster) {
|
||||
return
|
||||
}
|
||||
log.Info("update import segment info done", WrapTaskLog(t, zap.Int64("segmentID", info.GetSegmentID()), zap.Any("segmentInfo", info))...)
|
||||
totalRows += info.GetImportedRows()
|
||||
}
|
||||
completeTime := time.Now().Format("2006-01-02T15:04:05Z07:00")
|
||||
err = t.importMeta.UpdateTask(context.TODO(), t.GetTaskID(), UpdateState(datapb.ImportTaskStateV2_Completed), UpdateCompleteTime(completeTime))
|
||||
@ -235,7 +237,7 @@ func (t *importTask) QueryTaskOnWorker(cluster session.Cluster) {
|
||||
}
|
||||
importDuration := t.GetTR().RecordSpan()
|
||||
metrics.ImportTaskLatency.WithLabelValues(metrics.ImportStageImport).Observe(float64(importDuration.Milliseconds()))
|
||||
log.Info("import done", WrapTaskLog(t, zap.Duration("taskTimeCost/import", importDuration))...)
|
||||
log.Info("import done", WrapTaskLog(t, zap.Int64("totalRows", totalRows), zap.Duration("taskTimeCost/import", importDuration))...)
|
||||
}
|
||||
log.Info("query import", WrapTaskLog(t, zap.String("respState", resp.GetState().String()),
|
||||
zap.String("reason", resp.GetReason()))...)
|
||||
|
||||
@ -847,17 +847,19 @@ func CalculateTaskSlot(task ImportTask, importMeta ImportMeta) int {
|
||||
}
|
||||
|
||||
func createSortCompactionTask(ctx context.Context,
|
||||
t ImportTask,
|
||||
originSegment *SegmentInfo,
|
||||
targetSegmentID int64,
|
||||
meta *meta,
|
||||
handler Handler,
|
||||
alloc allocator.Allocator,
|
||||
) (*datapb.CompactionTask, error) {
|
||||
log := log.Ctx(ctx).With(WrapTaskLog(t)...)
|
||||
if originSegment.GetNumOfRows() == 0 {
|
||||
operator := UpdateStatusOperator(originSegment.GetID(), commonpb.SegmentState_Dropped)
|
||||
err := meta.UpdateSegmentsInfo(ctx, operator)
|
||||
if err != nil {
|
||||
log.Ctx(ctx).Warn("import zero num row segment, but mark it dropped failed", zap.Error(err))
|
||||
log.Warn("import zero num row segment, but mark it dropped failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
return nil, nil
|
||||
@ -870,13 +872,13 @@ func createSortCompactionTask(ctx context.Context,
|
||||
|
||||
collectionTTL, err := common.GetCollectionTTLFromMap(collection.Properties)
|
||||
if err != nil {
|
||||
log.Warn("failed to apply triggerSegmentSortCompaction, get collection ttl failed")
|
||||
log.Warn("Failed to create sort compaction task because get collection ttl failed")
|
||||
return nil, err
|
||||
}
|
||||
|
||||
startID, _, err := alloc.AllocN(2)
|
||||
if err != nil {
|
||||
log.Warn("fFailed to submit compaction view to scheduler because allocate id fail", zap.Error(err))
|
||||
log.Warn("Failed to create sort compaction task because allocate id fail", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
@ -903,7 +905,7 @@ func createSortCompactionTask(ctx context.Context,
|
||||
},
|
||||
}
|
||||
|
||||
log.Ctx(ctx).Info("create sort compaction task success", zap.Int64("segmentID", originSegment.GetID()),
|
||||
log.Info("create sort compaction task success", zap.Int64("segmentID", originSegment.GetID()),
|
||||
zap.Int64("targetSegmentID", targetSegmentID), zap.Int64("num rows", originSegment.GetNumOfRows()))
|
||||
return task, nil
|
||||
}
|
||||
|
||||
@ -1624,6 +1624,17 @@ func (m *meta) SetSegmentCompacting(segmentID UniqueID, compacting bool) {
|
||||
m.segments.SetIsCompacting(segmentID, compacting)
|
||||
}
|
||||
|
||||
// IsSegmentCompacting check if segment is compacting
|
||||
func (m *meta) IsSegmentCompacting(segmentID UniqueID) bool {
|
||||
m.segMu.RLock()
|
||||
defer m.segMu.RUnlock()
|
||||
seg := m.segments.GetSegment(segmentID)
|
||||
if seg == nil {
|
||||
return false
|
||||
}
|
||||
return seg.isCompacting
|
||||
}
|
||||
|
||||
// CheckAndSetSegmentsCompacting check all segments are not compacting
|
||||
// if true, set them compacting and return true
|
||||
// if false, skip setting and
|
||||
|
||||
@ -45,6 +45,7 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/timerecord"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
@ -286,6 +287,17 @@ func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.Compactio
|
||||
}
|
||||
|
||||
debug.FreeOSMemory()
|
||||
|
||||
if numValidRows != int(numRows)-entityFilter.GetDeletedCount()-entityFilter.GetExpiredCount() {
|
||||
log.Warn("unexpected row count after sort compaction",
|
||||
zap.Int64("target segmentID", targetSegmentID),
|
||||
zap.Int64("old rows", numRows),
|
||||
zap.Int("valid rows", numValidRows),
|
||||
zap.Int("deleted rows", entityFilter.GetDeletedCount()),
|
||||
zap.Int("expired rows", entityFilter.GetExpiredCount()))
|
||||
return nil, merr.WrapErrServiceInternal("unexpected row count")
|
||||
}
|
||||
|
||||
log.Info("sort segment end",
|
||||
zap.Int64("target segmentID", targetSegmentID),
|
||||
zap.Int64("old rows", numRows),
|
||||
|
||||
@ -248,7 +248,7 @@ func (s *SortCompactionTaskSuite) setupBM25Test() {
|
||||
PreAllocatedLogIDs: &datapb.IDRange{Begin: 9530, End: 19530},
|
||||
MaxSize: 64 * 1024 * 1024,
|
||||
JsonParams: params,
|
||||
TotalRows: 3,
|
||||
TotalRows: 1,
|
||||
}
|
||||
|
||||
pk, err := typeutil.GetPrimaryFieldSchema(plan.GetSchema())
|
||||
@ -285,7 +285,7 @@ func (s *SortCompactionTaskSuite) prepareSortCompactionWithBM25Task() {
|
||||
func (s *SortCompactionTaskSuite) TestSortCompactionWithExpiredData() {
|
||||
segmentID := int64(1001)
|
||||
|
||||
s.initSegBuffer(1, segmentID)
|
||||
s.initSegBuffer(1000, segmentID)
|
||||
collTTL := 864000 // 10 days
|
||||
s.task.currentTime = getMilvusBirthday().Add(time.Second * (time.Duration(collTTL) + 1))
|
||||
s.task.plan.CollectionTtl = int64(collTTL)
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user