mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-28 14:35:27 +08:00
issue: #46576 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: During meta load, only tasks that are truly terminal-cleaned (states cleaned or unknown) should be dropped; all other non-terminal tasks (including timeout and completed) must be restored so the inspector can reattach them to executing/cleaning queues and finish their cleanup lifecycle. - Removed/simplified logic: loadMeta no longer uses the broad isCompactionTaskFinished predicate (which treated timeout, completed, cleaned, unknown as terminal). It now uses the new isCompactionTaskCleaned predicate that only treats cleaned/unknown as terminal. This removes the redundant exclusion of timeout/completed tasks and simplifies the guard to drop only cleaned/unknown tasks. - Bug fix (root cause & exact change): Fixes issue #46576 — the previous isCompactionTaskFinished caused timeout/completed tasks to be skipped during meta load and thus not passed into restoreTask(). The PR adds isCompactionTaskCleaned and replaces the finished check so timeout and completed tasks are included in restoreTask() and re-attached to the inspector’s existing executing/cleaning queues. - No data loss or regression: Tasks in cleaned/unknown remain dropped (isCompactionTaskCleaned still returns true for cleaned/unknown). Non-terminal timeout/completed tasks now follow the same restoreTask() control path used previously for restored tasks — they are enqueued into the inspector’s queue/executing/cleaning flows rather than being discarded. No exported signatures changed and all restored tasks flow into existing handlers, avoiding behavior regression or data loss. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>
139 lines
4.8 KiB
Go
139 lines
4.8 KiB
Go
// Licensed to the LF AI & Data foundation under one
|
|
// or more contributor license agreements. See the NOTICE file
|
|
// distributed with this work for additional information
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
// to you under the Apache License, Version 2.0 (the
|
|
// "License"); you may not use this file except in compliance
|
|
// with the License. You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package datacoord
|
|
|
|
import (
|
|
"google.golang.org/protobuf/proto"
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
|
"github.com/milvus-io/milvus/internal/datacoord/allocator"
|
|
"github.com/milvus-io/milvus/internal/util/hookutil"
|
|
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
|
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
|
"github.com/milvus-io/milvus/pkg/v2/proto/workerpb"
|
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
|
)
|
|
|
|
// PreAllocateBinlogIDs pre-allocates binlog IDs based on the total number of binlogs from
|
|
// the segments for compaction, multiplied by an expansion factor.
|
|
func PreAllocateBinlogIDs(allocator allocator.Allocator, segmentInfos []*SegmentInfo) (*datapb.IDRange, error) {
|
|
binlogNum := 0
|
|
for _, s := range segmentInfos {
|
|
for _, l := range s.GetBinlogs() {
|
|
binlogNum += len(l.GetBinlogs())
|
|
}
|
|
for _, l := range s.GetDeltalogs() {
|
|
binlogNum += len(l.GetBinlogs())
|
|
}
|
|
for _, l := range s.GetStatslogs() {
|
|
binlogNum += len(l.GetBinlogs())
|
|
}
|
|
for _, l := range s.GetBm25Statslogs() {
|
|
binlogNum += len(l.GetBinlogs())
|
|
}
|
|
}
|
|
n := binlogNum * paramtable.Get().DataCoordCfg.CompactionPreAllocateIDExpansionFactor.GetAsInt()
|
|
begin, end, err := allocator.AllocN(int64(n))
|
|
return &datapb.IDRange{Begin: begin, End: end}, err
|
|
}
|
|
|
|
func WrapPluginContextWithImport(collectionID int64, dbProperties []*commonpb.KeyValuePair, options importutilv2.Options, msg proto.Message) {
|
|
pluginContext := make([]*commonpb.KeyValuePair, 0)
|
|
|
|
importEzk, _ := importutilv2.GetEZK(options)
|
|
readPluginContext := hookutil.GetReadStoragePluginContext(importEzk)
|
|
if readPluginContext != nil {
|
|
pluginContext = append(pluginContext, readPluginContext...)
|
|
}
|
|
|
|
writePluginContext := hookutil.GetStoragePluginContext(dbProperties, collectionID)
|
|
if writePluginContext != nil {
|
|
pluginContext = append(pluginContext, writePluginContext...)
|
|
}
|
|
|
|
if len(pluginContext) == 0 {
|
|
return
|
|
}
|
|
|
|
switch msg.(type) {
|
|
case *datapb.ImportRequest:
|
|
job := msg.(*datapb.ImportRequest)
|
|
job.PluginContext = append(job.PluginContext, pluginContext...)
|
|
case *datapb.PreImportRequest:
|
|
job := msg.(*datapb.PreImportRequest)
|
|
job.PluginContext = append(job.PluginContext, pluginContext...)
|
|
default:
|
|
return
|
|
}
|
|
}
|
|
|
|
func WrapPluginContext(collectionID int64, properties []*commonpb.KeyValuePair, msg proto.Message) {
|
|
pluginContext := hookutil.GetStoragePluginContext(properties, collectionID)
|
|
if pluginContext == nil {
|
|
return
|
|
}
|
|
|
|
switch msg.(type) {
|
|
case *datapb.CompactionPlan:
|
|
plan := msg.(*datapb.CompactionPlan)
|
|
plan.PluginContext = append(plan.PluginContext, pluginContext...)
|
|
case *workerpb.CreateJobRequest:
|
|
job := msg.(*workerpb.CreateJobRequest)
|
|
job.PluginContext = append(job.PluginContext, pluginContext...)
|
|
case *workerpb.AnalyzeRequest:
|
|
job := msg.(*workerpb.AnalyzeRequest)
|
|
job.PluginContext = append(job.PluginContext, pluginContext...)
|
|
case *workerpb.CreateStatsRequest:
|
|
job := msg.(*workerpb.CreateStatsRequest)
|
|
job.PluginContext = append(job.PluginContext, pluginContext...)
|
|
case *datapb.ImportRequest:
|
|
job := msg.(*datapb.ImportRequest)
|
|
job.PluginContext = append(job.PluginContext, pluginContext...)
|
|
case *datapb.PreImportRequest:
|
|
job := msg.(*datapb.PreImportRequest)
|
|
job.PluginContext = append(job.PluginContext, pluginContext...)
|
|
default:
|
|
return
|
|
}
|
|
}
|
|
|
|
// isCompactionTaskFinished returns true if the task has reached a terminal state
|
|
// (timeout, completed, cleaned, or unknown) and requires no further processing.
|
|
func isCompactionTaskFinished(t *datapb.CompactionTask) bool {
|
|
switch t.GetState() {
|
|
case datapb.CompactionTaskState_timeout,
|
|
datapb.CompactionTaskState_completed,
|
|
datapb.CompactionTaskState_cleaned,
|
|
datapb.CompactionTaskState_unknown:
|
|
return true
|
|
default:
|
|
return false
|
|
}
|
|
}
|
|
|
|
// isCompactionTaskCleaned returns true if the task has been cleaned
|
|
// (cleaned, or unknown) and requires no further processing.
|
|
func isCompactionTaskCleaned(t *datapb.CompactionTask) bool {
|
|
switch t.GetState() {
|
|
case datapb.CompactionTaskState_cleaned,
|
|
datapb.CompactionTaskState_unknown:
|
|
return true
|
|
default:
|
|
return false
|
|
}
|
|
}
|