2024-03-01 10:31:02 +00:00
// 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 (
2024-11-26 11:46:34 +00:00
"context"
2024-03-01 10:31:02 +00:00
"sort"
2024-03-19 07:23:06 +00:00
"strconv"
2024-03-01 10:31:02 +00:00
"sync"
"time"
"github.com/samber/lo"
"go.uber.org/zap"
2024-04-01 12:09:13 +00:00
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
2024-08-22 02:06:56 +00:00
"github.com/milvus-io/milvus/internal/datacoord/allocator"
2024-08-22 08:02:56 +00:00
"github.com/milvus-io/milvus/internal/datacoord/session"
2024-03-12 02:13:03 +00:00
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
2024-03-01 10:31:02 +00:00
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/pkg/log"
2024-03-19 07:23:06 +00:00
"github.com/milvus-io/milvus/pkg/metrics"
2024-05-15 08:33:34 +00:00
"github.com/milvus-io/milvus/pkg/util/lock"
2024-03-01 10:31:02 +00:00
)
const (
NullNodeID = - 1
)
type ImportScheduler interface {
Start ( )
Close ( )
}
type importScheduler struct {
meta * meta
cluster Cluster
2024-08-22 02:06:56 +00:00
alloc allocator . Allocator
2024-03-01 10:31:02 +00:00
imeta ImportMeta
closeOnce sync . Once
closeChan chan struct { }
}
func NewImportScheduler ( meta * meta ,
cluster Cluster ,
2024-08-22 02:06:56 +00:00
alloc allocator . Allocator ,
2024-03-01 10:31:02 +00:00
imeta ImportMeta ,
) ImportScheduler {
return & importScheduler {
2024-09-12 12:37:08 +00:00
meta : meta ,
cluster : cluster ,
alloc : alloc ,
imeta : imeta ,
closeChan : make ( chan struct { } ) ,
2024-03-01 10:31:02 +00:00
}
}
func ( s * importScheduler ) Start ( ) {
log . Info ( "start import scheduler" )
ticker := time . NewTicker ( Params . DataCoordCfg . ImportScheduleInterval . GetAsDuration ( time . Second ) )
defer ticker . Stop ( )
for {
select {
case <- s . closeChan :
log . Info ( "import scheduler exited" )
return
case <- ticker . C :
s . process ( )
}
}
}
func ( s * importScheduler ) Close ( ) {
s . closeOnce . Do ( func ( ) {
close ( s . closeChan )
} )
}
func ( s * importScheduler ) process ( ) {
2024-11-26 11:46:34 +00:00
jobs := s . imeta . GetJobBy ( context . TODO ( ) )
2024-03-01 10:31:02 +00:00
sort . Slice ( jobs , func ( i , j int ) bool {
return jobs [ i ] . GetJobID ( ) < jobs [ j ] . GetJobID ( )
} )
nodeSlots := s . peekSlots ( )
for _ , job := range jobs {
2024-11-26 11:46:34 +00:00
tasks := s . imeta . GetTaskBy ( context . TODO ( ) , WithJob ( job . GetJobID ( ) ) )
2024-03-01 10:31:02 +00:00
for _ , task := range tasks {
switch task . GetState ( ) {
case datapb . ImportTaskStateV2_Pending :
2024-10-09 06:41:20 +00:00
nodeID := s . getNodeID ( task , nodeSlots )
2024-03-01 10:31:02 +00:00
switch task . GetType ( ) {
case PreImportTaskType :
s . processPendingPreImport ( task , nodeID )
case ImportTaskType :
s . processPendingImport ( task , nodeID )
}
case datapb . ImportTaskStateV2_InProgress :
switch task . GetType ( ) {
case PreImportTaskType :
s . processInProgressPreImport ( task )
case ImportTaskType :
s . processInProgressImport ( task )
}
case datapb . ImportTaskStateV2_Completed :
s . processCompleted ( task )
case datapb . ImportTaskStateV2_Failed :
s . processFailed ( task )
}
}
}
}
func ( s * importScheduler ) peekSlots ( ) map [ int64 ] int64 {
2024-08-22 08:02:56 +00:00
nodeIDs := lo . Map ( s . cluster . GetSessions ( ) , func ( s * session . Session , _ int ) int64 {
return s . NodeID ( )
2024-03-01 10:31:02 +00:00
} )
nodeSlots := make ( map [ int64 ] int64 )
2024-05-15 08:33:34 +00:00
mu := & lock . Mutex { }
2024-03-01 10:31:02 +00:00
wg := & sync . WaitGroup { }
for _ , nodeID := range nodeIDs {
wg . Add ( 1 )
go func ( nodeID int64 ) {
defer wg . Done ( )
resp , err := s . cluster . QueryImport ( nodeID , & datapb . QueryImportRequest { QuerySlot : true } )
if err != nil {
log . Warn ( "query import failed" , zap . Error ( err ) )
return
}
mu . Lock ( )
defer mu . Unlock ( )
nodeSlots [ nodeID ] = resp . GetSlots ( )
} ( nodeID )
}
wg . Wait ( )
2024-12-14 14:36:44 +00:00
log . Ctx ( context . TODO ( ) ) . Debug ( "peek slots done" , zap . Any ( "nodeSlots" , nodeSlots ) )
2024-03-01 10:31:02 +00:00
return nodeSlots
}
2024-10-09 06:41:20 +00:00
func ( s * importScheduler ) getNodeID ( task ImportTask , nodeSlots map [ int64 ] int64 ) int64 {
var (
nodeID int64 = NullNodeID
maxSlots int64 = - 1
)
require := task . GetSlots ( )
for id , slots := range nodeSlots {
// find the most idle datanode
if slots > 0 && slots >= require && slots > maxSlots {
nodeID = id
maxSlots = slots
}
}
if nodeID != NullNodeID {
nodeSlots [ nodeID ] -= require
}
return nodeID
}
2024-03-01 10:31:02 +00:00
func ( s * importScheduler ) processPendingPreImport ( task ImportTask , nodeID int64 ) {
if nodeID == NullNodeID {
return
}
log . Info ( "processing pending preimport task..." , WrapTaskLog ( task ) ... )
2024-11-26 11:46:34 +00:00
job := s . imeta . GetJob ( context . TODO ( ) , task . GetJobID ( ) )
2024-03-01 10:31:02 +00:00
req := AssemblePreImportRequest ( task , job )
err := s . cluster . PreImport ( nodeID , req )
if err != nil {
log . Warn ( "preimport failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
2024-11-26 11:46:34 +00:00
err = s . imeta . UpdateTask ( context . TODO ( ) , task . GetTaskID ( ) ,
2024-03-01 10:31:02 +00:00
UpdateState ( datapb . ImportTaskStateV2_InProgress ) ,
UpdateNodeID ( nodeID ) )
if err != nil {
log . Warn ( "update import task failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
2024-10-09 06:41:20 +00:00
pendingDuration := task . GetTR ( ) . RecordSpan ( )
metrics . ImportTaskLatency . WithLabelValues ( metrics . ImportStagePending ) . Observe ( float64 ( pendingDuration . Milliseconds ( ) ) )
log . Info ( "preimport task start to execute" , WrapTaskLog ( task , zap . Int64 ( "scheduledNodeID" , nodeID ) , zap . Duration ( "taskTimeCost/pending" , pendingDuration ) ) ... )
2024-03-01 10:31:02 +00:00
}
func ( s * importScheduler ) processPendingImport ( task ImportTask , nodeID int64 ) {
if nodeID == NullNodeID {
return
}
log . Info ( "processing pending import task..." , WrapTaskLog ( task ) ... )
2024-11-26 11:46:34 +00:00
job := s . imeta . GetJob ( context . TODO ( ) , task . GetJobID ( ) )
2024-03-01 10:31:02 +00:00
req , err := AssembleImportRequest ( task , job , s . meta , s . alloc )
if err != nil {
log . Warn ( "assemble import request failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
err = s . cluster . ImportV2 ( nodeID , req )
if err != nil {
log . Warn ( "import failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
2024-11-26 11:46:34 +00:00
err = s . imeta . UpdateTask ( context . TODO ( ) , task . GetTaskID ( ) ,
2024-03-01 10:31:02 +00:00
UpdateState ( datapb . ImportTaskStateV2_InProgress ) ,
UpdateNodeID ( nodeID ) )
if err != nil {
log . Warn ( "update import task failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
2024-10-09 06:41:20 +00:00
pendingDuration := task . GetTR ( ) . RecordSpan ( )
metrics . ImportTaskLatency . WithLabelValues ( metrics . ImportStagePending ) . Observe ( float64 ( pendingDuration . Milliseconds ( ) ) )
log . Info ( "import task start to execute" , WrapTaskLog ( task , zap . Int64 ( "scheduledNodeID" , nodeID ) , zap . Duration ( "taskTimeCost/pending" , pendingDuration ) ) ... )
2024-03-01 10:31:02 +00:00
}
func ( s * importScheduler ) processInProgressPreImport ( task ImportTask ) {
req := & datapb . QueryPreImportRequest {
JobID : task . GetJobID ( ) ,
TaskID : task . GetTaskID ( ) ,
}
resp , err := s . cluster . QueryPreImport ( task . GetNodeID ( ) , req )
if err != nil {
2024-11-26 11:46:34 +00:00
updateErr := s . imeta . UpdateTask ( context . TODO ( ) , task . GetTaskID ( ) , UpdateState ( datapb . ImportTaskStateV2_Pending ) )
2024-03-15 03:05:04 +00:00
if updateErr != nil {
log . Warn ( "failed to update preimport task state to pending" , WrapTaskLog ( task , zap . Error ( updateErr ) ) ... )
}
log . Info ( "reset preimport task state to pending due to error occurs" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
2024-03-01 10:31:02 +00:00
return
}
if resp . GetState ( ) == datapb . ImportTaskStateV2_Failed {
2024-11-26 11:46:34 +00:00
err = s . imeta . UpdateJob ( context . TODO ( ) , task . GetJobID ( ) , UpdateJobState ( internalpb . ImportJobState_Failed ) ,
2024-03-01 10:31:02 +00:00
UpdateJobReason ( resp . GetReason ( ) ) )
if err != nil {
log . Warn ( "failed to update job state to Failed" , zap . Int64 ( "jobID" , task . GetJobID ( ) ) , zap . Error ( err ) )
}
log . Warn ( "preimport failed" , WrapTaskLog ( task , zap . String ( "reason" , resp . GetReason ( ) ) ) ... )
return
}
actions := [ ] UpdateAction { UpdateFileStats ( resp . GetFileStats ( ) ) }
if resp . GetState ( ) == datapb . ImportTaskStateV2_Completed {
actions = append ( actions , UpdateState ( datapb . ImportTaskStateV2_Completed ) )
}
2024-11-26 11:46:34 +00:00
err = s . imeta . UpdateTask ( context . TODO ( ) , task . GetTaskID ( ) , actions ... )
2024-03-01 10:31:02 +00:00
if err != nil {
log . Warn ( "update preimport task failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
log . Info ( "query preimport" , WrapTaskLog ( task , zap . String ( "state" , resp . GetState ( ) . String ( ) ) ,
zap . Any ( "fileStats" , resp . GetFileStats ( ) ) ) ... )
2024-10-09 06:41:20 +00:00
if resp . GetState ( ) == datapb . ImportTaskStateV2_Completed {
preimportDuration := task . GetTR ( ) . RecordSpan ( )
metrics . ImportTaskLatency . WithLabelValues ( metrics . ImportStagePreImport ) . Observe ( float64 ( preimportDuration . Milliseconds ( ) ) )
log . Info ( "preimport done" , WrapTaskLog ( task , zap . Duration ( "taskTimeCost/preimport" , preimportDuration ) ) ... )
}
2024-03-01 10:31:02 +00:00
}
func ( s * importScheduler ) processInProgressImport ( task ImportTask ) {
req := & datapb . QueryImportRequest {
JobID : task . GetJobID ( ) ,
TaskID : task . GetTaskID ( ) ,
}
resp , err := s . cluster . QueryImport ( task . GetNodeID ( ) , req )
if err != nil {
2024-11-26 11:46:34 +00:00
updateErr := s . imeta . UpdateTask ( context . TODO ( ) , task . GetTaskID ( ) , UpdateState ( datapb . ImportTaskStateV2_Pending ) )
2024-03-15 03:05:04 +00:00
if updateErr != nil {
log . Warn ( "failed to update import task state to pending" , WrapTaskLog ( task , zap . Error ( updateErr ) ) ... )
}
log . Info ( "reset import task state to pending due to error occurs" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
2024-03-01 10:31:02 +00:00
return
}
if resp . GetState ( ) == datapb . ImportTaskStateV2_Failed {
2024-11-26 11:46:34 +00:00
err = s . imeta . UpdateJob ( context . TODO ( ) , task . GetJobID ( ) , UpdateJobState ( internalpb . ImportJobState_Failed ) ,
2024-03-01 10:31:02 +00:00
UpdateJobReason ( resp . GetReason ( ) ) )
if err != nil {
log . Warn ( "failed to update job state to Failed" , zap . Int64 ( "jobID" , task . GetJobID ( ) ) , zap . Error ( err ) )
}
log . Warn ( "import failed" , WrapTaskLog ( task , zap . String ( "reason" , resp . GetReason ( ) ) ) ... )
return
}
2024-04-26 07:53:29 +00:00
collInfo := s . meta . GetCollection ( task . GetCollectionID ( ) )
dbName := ""
if collInfo != nil {
dbName = collInfo . DatabaseName
}
2024-03-01 10:31:02 +00:00
for _ , info := range resp . GetImportSegmentsInfo ( ) {
2024-11-26 11:46:34 +00:00
segment := s . meta . GetSegment ( context . TODO ( ) , info . GetSegmentID ( ) )
2024-03-01 10:31:02 +00:00
if info . GetImportedRows ( ) <= segment . GetNumOfRows ( ) {
continue // rows not changed, no need to update
}
2024-03-19 07:23:06 +00:00
diff := info . GetImportedRows ( ) - segment . GetNumOfRows ( )
2024-03-01 10:31:02 +00:00
op := UpdateImportedRows ( info . GetSegmentID ( ) , info . GetImportedRows ( ) )
2024-11-26 11:46:34 +00:00
err = s . meta . UpdateSegmentsInfo ( context . TODO ( ) , op )
2024-03-01 10:31:02 +00:00
if err != nil {
log . Warn ( "update import segment rows failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
2024-04-26 07:53:29 +00:00
2024-03-19 07:23:06 +00:00
metrics . DataCoordBulkVectors . WithLabelValues (
2024-04-26 07:53:29 +00:00
dbName ,
2024-03-19 07:23:06 +00:00
strconv . FormatInt ( task . GetCollectionID ( ) , 10 ) ,
) . Add ( float64 ( diff ) )
2024-03-01 10:31:02 +00:00
}
if resp . GetState ( ) == datapb . ImportTaskStateV2_Completed {
for _ , info := range resp . GetImportSegmentsInfo ( ) {
2024-03-12 02:13:03 +00:00
// try to parse path and fill logID
2024-06-07 06:17:20 +00:00
err = binlog . CompressBinLogs ( info . GetBinlogs ( ) , info . GetDeltalogs ( ) , info . GetStatslogs ( ) )
2024-03-12 02:13:03 +00:00
if err != nil {
2024-06-07 06:17:20 +00:00
log . Warn ( "fail to CompressBinLogs for import binlogs" ,
2024-03-12 02:13:03 +00:00
WrapTaskLog ( task , zap . Int64 ( "segmentID" , info . GetSegmentID ( ) ) , zap . Error ( err ) ) ... )
return
}
2024-06-07 06:17:20 +00:00
op1 := UpdateBinlogsOperator ( info . GetSegmentID ( ) , info . GetBinlogs ( ) , info . GetStatslogs ( ) , info . GetDeltalogs ( ) )
2024-04-01 12:09:13 +00:00
op2 := UpdateStatusOperator ( info . GetSegmentID ( ) , commonpb . SegmentState_Flushed )
2024-11-26 11:46:34 +00:00
err = s . meta . UpdateSegmentsInfo ( context . TODO ( ) , op1 , op2 )
2024-03-01 10:31:02 +00:00
if err != nil {
log . Warn ( "update import segment binlogs failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
}
2024-03-13 11:51:03 +00:00
completeTime := time . Now ( ) . Format ( "2006-01-02T15:04:05Z07:00" )
2024-11-26 11:46:34 +00:00
err = s . imeta . UpdateTask ( context . TODO ( ) , task . GetTaskID ( ) , UpdateState ( datapb . ImportTaskStateV2_Completed ) , UpdateCompleteTime ( completeTime ) )
2024-03-01 10:31:02 +00:00
if err != nil {
log . Warn ( "update import task failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
return
}
2024-10-09 06:41:20 +00:00
importDuration := task . GetTR ( ) . RecordSpan ( )
metrics . ImportTaskLatency . WithLabelValues ( metrics . ImportStageImport ) . Observe ( float64 ( importDuration . Milliseconds ( ) ) )
log . Info ( "import done" , WrapTaskLog ( task , zap . Duration ( "taskTimeCost/import" , importDuration ) ) ... )
2024-03-01 10:31:02 +00:00
}
log . Info ( "query import" , WrapTaskLog ( task , zap . String ( "state" , resp . GetState ( ) . String ( ) ) ,
zap . String ( "reason" , resp . GetReason ( ) ) ) ... )
}
func ( s * importScheduler ) processCompleted ( task ImportTask ) {
err := DropImportTask ( task , s . cluster , s . imeta )
if err != nil {
log . Warn ( "drop import failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
}
}
func ( s * importScheduler ) processFailed ( task ImportTask ) {
if task . GetType ( ) == ImportTaskType {
2024-09-15 07:17:08 +00:00
originSegmentIDs := task . ( * importTask ) . GetSegmentIDs ( )
statsSegmentIDs := task . ( * importTask ) . GetStatsSegmentIDs ( )
segments := append ( originSegmentIDs , statsSegmentIDs ... )
2024-03-01 10:31:02 +00:00
for _ , segment := range segments {
2024-06-25 08:58:02 +00:00
op := UpdateStatusOperator ( segment , commonpb . SegmentState_Dropped )
2024-11-26 11:46:34 +00:00
err := s . meta . UpdateSegmentsInfo ( context . TODO ( ) , op )
2024-03-01 10:31:02 +00:00
if err != nil {
2024-06-25 08:58:02 +00:00
log . Warn ( "drop import segment failed" , WrapTaskLog ( task , zap . Int64 ( "segment" , segment ) , zap . Error ( err ) ) ... )
2024-03-01 10:31:02 +00:00
return
}
}
2024-06-03 06:15:53 +00:00
if len ( segments ) > 0 {
2024-11-26 11:46:34 +00:00
err := s . imeta . UpdateTask ( context . TODO ( ) , task . GetTaskID ( ) , UpdateSegmentIDs ( nil ) , UpdateStatsSegmentIDs ( nil ) )
2024-06-03 06:15:53 +00:00
if err != nil {
log . Warn ( "update import task segments failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
}
2024-03-01 10:31:02 +00:00
}
}
err := DropImportTask ( task , s . cluster , s . imeta )
if err != nil {
log . Warn ( "drop import failed" , WrapTaskLog ( task , zap . Error ( err ) ) ... )
}
}