2021-12-01 11:33:32 +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.
2021-11-05 14:25:00 +00:00
package datacoord
import (
"context"
"fmt"
2024-06-05 02:17:50 +00:00
"sort"
2021-11-05 14:25:00 +00:00
"sync"
"time"
2023-02-26 03:31:49 +00:00
"github.com/cockroachdb/errors"
2024-01-23 02:37:00 +00:00
"go.opentelemetry.io/otel"
2023-04-06 11:14:32 +00:00
"go.uber.org/zap"
2023-02-26 03:31:49 +00:00
2023-06-08 17:28:37 +00:00
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
2024-06-05 02:17:50 +00:00
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
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"
2021-11-05 14:25:00 +00:00
"github.com/milvus-io/milvus/internal/proto/datapb"
2023-04-06 11:14:32 +00:00
"github.com/milvus-io/milvus/pkg/log"
2024-07-11 09:45:37 +00:00
"github.com/milvus-io/milvus/pkg/metrics"
2023-11-06 19:18:18 +00:00
"github.com/milvus-io/milvus/pkg/util/conc"
2024-05-15 08:33:34 +00:00
"github.com/milvus-io/milvus/pkg/util/lock"
2024-03-06 13:36:59 +00:00
"github.com/milvus-io/milvus/pkg/util/merr"
2024-10-28 06:33:29 +00:00
"github.com/milvus-io/milvus/pkg/util/paramtable"
2024-01-23 02:37:00 +00:00
"github.com/milvus-io/milvus/pkg/util/typeutil"
2021-11-05 14:25:00 +00:00
)
2024-10-28 06:33:29 +00:00
// TODO: we just warn about the long executing/queuing tasks
// need to get rid of long queuing tasks because the compaction tasks are local optimum.
2024-09-08 09:23:05 +00:00
var maxCompactionTaskExecutionDuration = map [ datapb . CompactionType ] time . Duration {
datapb . CompactionType_MixCompaction : 30 * time . Minute ,
datapb . CompactionType_Level0DeleteCompaction : 30 * time . Minute ,
datapb . CompactionType_ClusteringCompaction : 60 * time . Minute ,
}
2021-11-05 14:25:00 +00:00
type compactionPlanContext interface {
start ( )
stop ( )
2024-06-05 02:17:50 +00:00
// enqueueCompaction start to enqueue compaction task and return immediately
enqueueCompaction ( task * datapb . CompactionTask ) error
2021-11-05 14:25:00 +00:00
// isFull return true if the task pool is full
isFull ( ) bool
2021-11-09 06:47:02 +00:00
// get compaction tasks by signal id
2024-06-05 02:17:50 +00:00
getCompactionTasksNumBySignalID ( signalID int64 ) int
2024-11-26 11:46:34 +00:00
getCompactionInfo ( ctx context . Context , signalID int64 ) * compactionInfo
2023-11-29 02:50:29 +00:00
removeTasksByChannel ( channel string )
2021-11-05 14:25:00 +00:00
}
var (
errChannelNotWatched = errors . New ( "channel is not watched" )
errChannelInBuffer = errors . New ( "channel is in buffer" )
)
2024-06-05 02:17:50 +00:00
var _ compactionPlanContext = ( * compactionPlanHandler ) ( nil )
2023-12-05 10:44:37 +00:00
2024-06-05 02:17:50 +00:00
type compactionInfo struct {
state commonpb . CompactionState
executingCnt int
completedCnt int
failedCnt int
timeoutCnt int
mergeInfos map [ int64 ] * milvuspb . CompactionMergeInfo
2021-11-05 14:25:00 +00:00
}
type compactionPlanHandler struct {
2024-10-09 11:11:20 +00:00
queueTasks CompactionQueue
2024-06-05 02:17:50 +00:00
2024-07-11 09:45:37 +00:00
executingGuard lock . RWMutex
2024-06-05 02:17:50 +00:00
executingTasks map [ int64 ] CompactionTask // planID -> task
2023-11-29 02:50:29 +00:00
2024-12-19 04:36:47 +00:00
cleaningGuard lock . RWMutex
cleaningTasks map [ int64 ] CompactionTask // planID -> task
2024-06-10 13:34:08 +00:00
meta CompactionMeta
2024-08-22 02:06:56 +00:00
allocator allocator . Allocator
2024-08-22 08:02:56 +00:00
sessions session . DataNodeManager
2024-06-10 13:34:08 +00:00
cluster Cluster
analyzeScheduler * taskScheduler
handler Handler
2023-11-29 02:50:29 +00:00
stopCh chan struct { }
stopOnce sync . Once
stopWg sync . WaitGroup
2024-06-05 02:17:50 +00:00
}
2024-11-26 11:46:34 +00:00
func ( c * compactionPlanHandler ) getCompactionInfo ( ctx context . Context , triggerID int64 ) * compactionInfo {
tasks := c . meta . GetCompactionTasksByTriggerID ( ctx , triggerID )
2024-06-10 13:34:08 +00:00
return summaryCompactionState ( tasks )
}
2024-06-05 02:17:50 +00:00
2024-06-10 13:34:08 +00:00
func summaryCompactionState ( tasks [ ] * datapb . CompactionTask ) * compactionInfo {
ret := & compactionInfo { }
2024-09-02 06:19:03 +00:00
var executingCnt , pipeliningCnt , completedCnt , failedCnt , timeoutCnt , analyzingCnt , indexingCnt , cleanedCnt , metaSavedCnt , stats int
2024-06-05 02:17:50 +00:00
mergeInfos := make ( map [ int64 ] * milvuspb . CompactionMergeInfo )
2024-06-10 13:34:08 +00:00
for _ , task := range tasks {
if task == nil {
continue
}
switch task . GetState ( ) {
case datapb . CompactionTaskState_executing :
2024-06-05 02:17:50 +00:00
executingCnt ++
2024-06-10 13:34:08 +00:00
case datapb . CompactionTaskState_pipelining :
pipeliningCnt ++
2024-06-05 02:17:50 +00:00
case datapb . CompactionTaskState_completed :
completedCnt ++
case datapb . CompactionTaskState_failed :
failedCnt ++
case datapb . CompactionTaskState_timeout :
timeoutCnt ++
2024-06-10 13:34:08 +00:00
case datapb . CompactionTaskState_analyzing :
analyzingCnt ++
case datapb . CompactionTaskState_indexing :
indexingCnt ++
case datapb . CompactionTaskState_cleaned :
cleanedCnt ++
case datapb . CompactionTaskState_meta_saved :
metaSavedCnt ++
2024-09-02 06:19:03 +00:00
case datapb . CompactionTaskState_statistic :
stats ++
2024-06-10 13:34:08 +00:00
default :
2024-06-05 02:17:50 +00:00
}
2024-06-10 13:34:08 +00:00
mergeInfos [ task . GetPlanID ( ) ] = getCompactionMergeInfo ( task )
2024-06-05 02:17:50 +00:00
}
2024-09-02 06:19:03 +00:00
ret . executingCnt = executingCnt + pipeliningCnt + analyzingCnt + indexingCnt + metaSavedCnt + stats
2024-06-05 02:17:50 +00:00
ret . completedCnt = completedCnt
ret . timeoutCnt = timeoutCnt
ret . failedCnt = failedCnt
ret . mergeInfos = mergeInfos
2024-06-10 13:34:08 +00:00
if ret . executingCnt != 0 {
2024-06-05 02:17:50 +00:00
ret . state = commonpb . CompactionState_Executing
} else {
ret . state = commonpb . CompactionState_Completed
}
2024-06-10 13:34:08 +00:00
log . Info ( "compaction states" ,
zap . String ( "state" , ret . state . String ( ) ) ,
zap . Int ( "executingCnt" , executingCnt ) ,
zap . Int ( "pipeliningCnt" , pipeliningCnt ) ,
zap . Int ( "completedCnt" , completedCnt ) ,
zap . Int ( "failedCnt" , failedCnt ) ,
zap . Int ( "timeoutCnt" , timeoutCnt ) ,
zap . Int ( "analyzingCnt" , analyzingCnt ) ,
zap . Int ( "indexingCnt" , indexingCnt ) ,
zap . Int ( "cleanedCnt" , cleanedCnt ) ,
zap . Int ( "metaSavedCnt" , metaSavedCnt ) )
2024-06-05 02:17:50 +00:00
return ret
}
func ( c * compactionPlanHandler ) getCompactionTasksNumBySignalID ( triggerID int64 ) int {
cnt := 0
2024-10-09 11:11:20 +00:00
c . queueTasks . ForEach ( func ( ct CompactionTask ) {
2024-10-24 01:55:28 +00:00
if ct . GetTaskProto ( ) . GetTriggerID ( ) == triggerID {
2024-06-05 02:17:50 +00:00
cnt += 1
}
2024-10-09 11:11:20 +00:00
} )
2024-07-11 09:45:37 +00:00
c . executingGuard . RLock ( )
2024-06-05 02:17:50 +00:00
for _ , t := range c . executingTasks {
2024-10-24 01:55:28 +00:00
if t . GetTaskProto ( ) . GetTriggerID ( ) == triggerID {
2024-06-05 02:17:50 +00:00
cnt += 1
}
}
2024-07-11 09:45:37 +00:00
c . executingGuard . RUnlock ( )
2024-06-05 02:17:50 +00:00
return cnt
2021-11-05 14:25:00 +00:00
}
2024-11-07 02:26:26 +00:00
func newCompactionPlanHandler ( cluster Cluster , sessions session . DataNodeManager , meta CompactionMeta ,
2024-10-09 11:11:20 +00:00
allocator allocator . Allocator , analyzeScheduler * taskScheduler , handler Handler ,
2023-09-21 01:45:27 +00:00
) * compactionPlanHandler {
2024-10-28 06:33:29 +00:00
// Higher capacity will have better ordering in priority, but consumes more memory.
// TODO[GOOSE]: Higher capacity makes tasks waiting longer, which need to be get rid of.
capacity := paramtable . Get ( ) . DataCoordCfg . CompactionTaskQueueCapacity . GetAsInt ( )
2021-11-05 14:25:00 +00:00
return & compactionPlanHandler {
2024-10-28 06:33:29 +00:00
queueTasks : * NewCompactionQueue ( capacity , getPrioritizer ( ) ) ,
2024-06-10 13:34:08 +00:00
meta : meta ,
sessions : sessions ,
allocator : allocator ,
stopCh : make ( chan struct { } ) ,
cluster : cluster ,
executingTasks : make ( map [ int64 ] CompactionTask ) ,
2024-12-19 04:36:47 +00:00
cleaningTasks : make ( map [ int64 ] CompactionTask ) ,
2024-06-10 13:34:08 +00:00
analyzeScheduler : analyzeScheduler ,
handler : handler ,
2021-11-05 14:25:00 +00:00
}
}
2024-06-05 02:17:50 +00:00
func ( c * compactionPlanHandler ) schedule ( ) [ ] CompactionTask {
2024-11-13 07:12:30 +00:00
selected := make ( [ ] CompactionTask , 0 )
if c . queueTasks . Len ( ) == 0 {
return selected
}
var (
parallelism = Params . DataCoordCfg . CompactionMaxParallelTasks . GetAsInt ( )
slots map [ int64 ] int64
)
2024-06-05 02:17:50 +00:00
l0ChannelExcludes := typeutil . NewSet [ string ] ( )
mixChannelExcludes := typeutil . NewSet [ string ] ( )
2024-06-10 13:34:08 +00:00
clusterChannelExcludes := typeutil . NewSet [ string ] ( )
2024-06-05 02:17:50 +00:00
mixLabelExcludes := typeutil . NewSet [ string ] ( )
2024-06-10 13:34:08 +00:00
clusterLabelExcludes := typeutil . NewSet [ string ] ( )
2024-06-05 02:17:50 +00:00
2024-07-11 09:45:37 +00:00
c . executingGuard . RLock ( )
2024-12-30 14:44:49 +00:00
if len ( c . executingTasks ) >= parallelism {
c . executingGuard . RUnlock ( )
return selected
}
2024-06-05 02:17:50 +00:00
for _ , t := range c . executingTasks {
2024-10-24 01:55:28 +00:00
switch t . GetTaskProto ( ) . GetType ( ) {
2024-06-05 02:17:50 +00:00
case datapb . CompactionType_Level0DeleteCompaction :
2024-10-24 01:55:28 +00:00
l0ChannelExcludes . Insert ( t . GetTaskProto ( ) . GetChannel ( ) )
2024-06-05 02:17:50 +00:00
case datapb . CompactionType_MixCompaction :
2024-10-24 01:55:28 +00:00
mixChannelExcludes . Insert ( t . GetTaskProto ( ) . GetChannel ( ) )
2024-06-05 02:17:50 +00:00
mixLabelExcludes . Insert ( t . GetLabel ( ) )
2024-06-10 13:34:08 +00:00
case datapb . CompactionType_ClusteringCompaction :
2024-10-24 01:55:28 +00:00
clusterChannelExcludes . Insert ( t . GetTaskProto ( ) . GetChannel ( ) )
2024-06-10 13:34:08 +00:00
clusterLabelExcludes . Insert ( t . GetLabel ( ) )
2024-06-05 02:17:50 +00:00
}
2024-03-18 17:01:36 +00:00
}
2024-07-11 09:45:37 +00:00
c . executingGuard . RUnlock ( )
2023-12-22 04:00:43 +00:00
2024-10-09 11:11:20 +00:00
excluded := make ( [ ] CompactionTask , 0 )
defer func ( ) {
// Add back the excluded tasks
for _ , t := range excluded {
c . queueTasks . Enqueue ( t )
}
} ( )
p := getPrioritizer ( )
if & c . queueTasks . prioritizer != & p {
c . queueTasks . UpdatePrioritizer ( p )
}
2024-11-13 07:12:30 +00:00
// The schedule loop will stop if either:
// 1. no more task to schedule (the task queue is empty)
// 2. the parallelism of running tasks is reached
// 3. no avaiable slots
for {
2024-10-09 11:11:20 +00:00
t , err := c . queueTasks . Dequeue ( )
if err != nil {
2024-11-13 07:12:30 +00:00
break // 1. no more task to schedule
2024-10-09 11:11:20 +00:00
}
2024-10-24 01:55:28 +00:00
switch t . GetTaskProto ( ) . GetType ( ) {
2024-06-05 02:17:50 +00:00
case datapb . CompactionType_Level0DeleteCompaction :
2024-10-28 22:28:24 +00:00
if mixChannelExcludes . Contain ( t . GetTaskProto ( ) . GetChannel ( ) ) ||
clusterChannelExcludes . Contain ( t . GetTaskProto ( ) . GetChannel ( ) ) {
2024-10-09 11:11:20 +00:00
excluded = append ( excluded , t )
2024-06-05 02:17:50 +00:00
continue
}
2024-10-24 01:55:28 +00:00
l0ChannelExcludes . Insert ( t . GetTaskProto ( ) . GetChannel ( ) )
2024-10-09 11:11:20 +00:00
selected = append ( selected , t )
2024-06-05 02:17:50 +00:00
case datapb . CompactionType_MixCompaction :
2024-10-24 01:55:28 +00:00
if l0ChannelExcludes . Contain ( t . GetTaskProto ( ) . GetChannel ( ) ) {
2024-10-09 11:11:20 +00:00
excluded = append ( excluded , t )
2024-06-05 02:17:50 +00:00
continue
}
2024-10-24 01:55:28 +00:00
mixChannelExcludes . Insert ( t . GetTaskProto ( ) . GetChannel ( ) )
2024-06-05 02:17:50 +00:00
mixLabelExcludes . Insert ( t . GetLabel ( ) )
2024-10-09 11:11:20 +00:00
selected = append ( selected , t )
2024-06-10 13:34:08 +00:00
case datapb . CompactionType_ClusteringCompaction :
2024-10-24 01:55:28 +00:00
if l0ChannelExcludes . Contain ( t . GetTaskProto ( ) . GetChannel ( ) ) ||
2024-06-10 13:34:08 +00:00
mixLabelExcludes . Contain ( t . GetLabel ( ) ) ||
clusterLabelExcludes . Contain ( t . GetLabel ( ) ) {
2024-10-09 11:11:20 +00:00
excluded = append ( excluded , t )
2024-06-10 13:34:08 +00:00
continue
}
2024-10-24 01:55:28 +00:00
clusterChannelExcludes . Insert ( t . GetTaskProto ( ) . GetChannel ( ) )
2024-06-10 13:34:08 +00:00
clusterLabelExcludes . Insert ( t . GetLabel ( ) )
2024-10-09 11:11:20 +00:00
selected = append ( selected , t )
2024-06-05 02:17:50 +00:00
}
2024-10-09 11:11:20 +00:00
2024-11-13 07:12:30 +00:00
if t . NeedReAssignNodeID ( ) {
if slots == nil {
slots = c . cluster . QuerySlots ( )
}
id := assignNodeID ( slots , t )
if id == NullNodeID {
log . RatedWarn ( 10 , "not enough slots for compaction task" , zap . Int64 ( "planID" , t . GetTaskProto ( ) . GetPlanID ( ) ) )
selected = selected [ : len ( selected ) - 1 ]
excluded = append ( excluded , t )
break // 3. no avaiable slots
}
}
2024-10-09 11:11:20 +00:00
c . executingGuard . Lock ( )
2024-10-24 01:55:28 +00:00
c . executingTasks [ t . GetTaskProto ( ) . GetPlanID ( ) ] = t
2024-11-13 07:12:30 +00:00
if len ( c . executingTasks ) >= parallelism {
2024-11-21 09:26:46 +00:00
c . executingGuard . Unlock ( )
2024-11-13 07:12:30 +00:00
break // 2. the parallelism of running tasks is reached
}
2024-10-09 11:11:20 +00:00
c . executingGuard . Unlock ( )
2024-10-24 01:55:28 +00:00
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , NullNodeID ) , t . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Pending ) . Dec ( )
2024-11-13 07:12:30 +00:00
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , t . GetTaskProto ( ) . GetNodeID ( ) ) , t . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Executing ) . Inc ( )
2023-12-05 10:44:37 +00:00
}
2024-10-09 11:11:20 +00:00
return selected
2024-06-05 02:17:50 +00:00
}
func ( c * compactionPlanHandler ) start ( ) {
2024-06-10 13:34:08 +00:00
c . loadMeta ( )
2024-06-05 02:17:50 +00:00
c . stopWg . Add ( 3 )
go c . loopSchedule ( )
go c . loopCheck ( )
go c . loopClean ( )
2023-12-05 10:44:37 +00:00
}
2024-06-10 13:34:08 +00:00
func ( c * compactionPlanHandler ) loadMeta ( ) {
2024-07-11 09:45:37 +00:00
// TODO: make it compatible to all types of compaction with persist meta
2024-11-26 11:46:34 +00:00
triggers := c . meta . GetCompactionTasks ( context . TODO ( ) )
2024-06-10 13:34:08 +00:00
for _ , tasks := range triggers {
for _ , task := range tasks {
2024-06-25 02:08:03 +00:00
state := task . GetState ( )
if state == datapb . CompactionTaskState_completed ||
state == datapb . CompactionTaskState_cleaned ||
2024-10-28 06:33:29 +00:00
state == datapb . CompactionTaskState_timeout ||
2024-06-25 02:08:03 +00:00
state == datapb . CompactionTaskState_unknown {
log . Info ( "compactionPlanHandler loadMeta abandon compactionTask" ,
zap . Int64 ( "planID" , task . GetPlanID ( ) ) ,
2024-07-11 09:45:37 +00:00
zap . String ( "type" , task . GetType ( ) . String ( ) ) ,
zap . String ( "state" , task . GetState ( ) . String ( ) ) )
2024-06-25 02:08:03 +00:00
continue
} else {
t , err := c . createCompactTask ( task )
if err != nil {
2024-11-21 10:48:32 +00:00
log . Info ( "compactionPlanHandler loadMeta create compactionTask failed, try to clean it" ,
2024-06-25 02:08:03 +00:00
zap . Int64 ( "planID" , task . GetPlanID ( ) ) ,
2024-07-11 09:45:37 +00:00
zap . String ( "type" , task . GetType ( ) . String ( ) ) ,
zap . String ( "state" , task . GetState ( ) . String ( ) ) ,
zap . Error ( err ) ,
)
2024-11-21 10:48:32 +00:00
// ignore the drop error
2024-11-26 11:46:34 +00:00
c . meta . DropCompactionTask ( context . TODO ( ) , task )
2024-06-25 02:08:03 +00:00
continue
}
if t . NeedReAssignNodeID ( ) {
2024-11-29 03:10:36 +00:00
if err = c . submitTask ( t ) ; err != nil {
log . Info ( "compactionPlanHandler loadMeta submit task failed, try to clean it" ,
zap . Int64 ( "planID" , task . GetPlanID ( ) ) ,
zap . String ( "type" , task . GetType ( ) . String ( ) ) ,
zap . String ( "state" , task . GetState ( ) . String ( ) ) ,
zap . Error ( err ) ,
)
// ignore the drop error
2024-11-29 04:00:36 +00:00
c . meta . DropCompactionTask ( context . Background ( ) , task )
2024-11-29 03:10:36 +00:00
continue
}
2024-06-25 02:08:03 +00:00
log . Info ( "compactionPlanHandler loadMeta submitTask" ,
2024-10-24 01:55:28 +00:00
zap . Int64 ( "planID" , t . GetTaskProto ( ) . GetPlanID ( ) ) ,
zap . Int64 ( "triggerID" , t . GetTaskProto ( ) . GetTriggerID ( ) ) ,
zap . Int64 ( "collectionID" , t . GetTaskProto ( ) . GetCollectionID ( ) ) ,
2024-07-11 09:45:37 +00:00
zap . String ( "type" , task . GetType ( ) . String ( ) ) ,
2024-10-24 01:55:28 +00:00
zap . String ( "state" , t . GetTaskProto ( ) . GetState ( ) . String ( ) ) )
2024-06-25 02:08:03 +00:00
} else {
c . restoreTask ( t )
log . Info ( "compactionPlanHandler loadMeta restoreTask" ,
2024-10-24 01:55:28 +00:00
zap . Int64 ( "planID" , t . GetTaskProto ( ) . GetPlanID ( ) ) ,
zap . Int64 ( "triggerID" , t . GetTaskProto ( ) . GetTriggerID ( ) ) ,
zap . Int64 ( "collectionID" , t . GetTaskProto ( ) . GetCollectionID ( ) ) ,
2024-07-11 09:45:37 +00:00
zap . String ( "type" , task . GetType ( ) . String ( ) ) ,
2024-10-24 01:55:28 +00:00
zap . String ( "state" , t . GetTaskProto ( ) . GetState ( ) . String ( ) ) )
2024-06-25 02:08:03 +00:00
}
2024-06-10 13:34:08 +00:00
}
}
}
}
2024-06-05 02:17:50 +00:00
func ( c * compactionPlanHandler ) loopSchedule ( ) {
log . Info ( "compactionPlanHandler start loop schedule" )
defer c . stopWg . Done ( )
2021-11-05 14:25:00 +00:00
2024-06-05 02:17:50 +00:00
scheduleTicker := time . NewTicker ( 3 * time . Second )
defer scheduleTicker . Stop ( )
for {
select {
case <- c . stopCh :
log . Info ( "compactionPlanHandler quit loop schedule" )
return
case <- scheduleTicker . C :
2024-10-09 11:11:20 +00:00
c . schedule ( )
2023-11-29 02:50:29 +00:00
}
2024-06-05 02:17:50 +00:00
}
}
func ( c * compactionPlanHandler ) loopCheck ( ) {
interval := Params . DataCoordCfg . CompactionCheckIntervalInSeconds . GetAsDuration ( time . Second )
log . Info ( "compactionPlanHandler start loop check" , zap . Any ( "check result interval" , interval ) )
defer c . stopWg . Done ( )
checkResultTicker := time . NewTicker ( interval )
2024-08-13 13:36:52 +00:00
defer checkResultTicker . Stop ( )
2024-06-05 02:17:50 +00:00
for {
select {
case <- c . stopCh :
log . Info ( "compactionPlanHandler quit loop check" )
return
case <- checkResultTicker . C :
err := c . checkCompaction ( )
if err != nil {
log . Info ( "fail to update compaction" , zap . Error ( err ) )
2021-11-05 14:25:00 +00:00
}
2024-12-19 04:36:47 +00:00
c . cleanFailedTasks ( )
2021-11-05 14:25:00 +00:00
}
2024-06-05 02:17:50 +00:00
}
}
func ( c * compactionPlanHandler ) loopClean ( ) {
2024-07-30 12:21:56 +00:00
interval := Params . DataCoordCfg . CompactionGCIntervalInSeconds . GetAsDuration ( time . Second )
log . Info ( "compactionPlanHandler start clean check loop" , zap . Any ( "gc interval" , interval ) )
2024-06-05 02:17:50 +00:00
defer c . stopWg . Done ( )
2024-07-30 12:21:56 +00:00
cleanTicker := time . NewTicker ( interval )
2024-06-05 02:17:50 +00:00
defer cleanTicker . Stop ( )
for {
select {
case <- c . stopCh :
log . Info ( "Compaction handler quit loopClean" )
return
case <- cleanTicker . C :
c . Clean ( )
2023-12-22 04:00:43 +00:00
}
2024-06-05 02:17:50 +00:00
}
2023-12-22 04:00:43 +00:00
}
func ( c * compactionPlanHandler ) Clean ( ) {
2024-06-05 02:17:50 +00:00
c . cleanCompactionTaskMeta ( )
2024-06-10 13:34:08 +00:00
c . cleanPartitionStats ( )
2024-06-05 02:17:50 +00:00
}
func ( c * compactionPlanHandler ) cleanCompactionTaskMeta ( ) {
// gc clustering compaction tasks
2024-11-26 11:46:34 +00:00
triggers := c . meta . GetCompactionTasks ( context . TODO ( ) )
2024-06-05 02:17:50 +00:00
for _ , tasks := range triggers {
for _ , task := range tasks {
2024-12-19 04:36:47 +00:00
if task . State == datapb . CompactionTaskState_cleaned {
2024-08-02 02:30:19 +00:00
duration := time . Since ( time . Unix ( task . StartTime , 0 ) ) . Seconds ( )
2024-07-30 12:21:56 +00:00
if duration > float64 ( Params . DataCoordCfg . CompactionDropToleranceInSeconds . GetAsDuration ( time . Second ) . Seconds ( ) ) {
2024-06-05 02:17:50 +00:00
// try best to delete meta
2024-11-26 11:46:34 +00:00
err := c . meta . DropCompactionTask ( context . TODO ( ) , task )
2024-12-14 14:36:44 +00:00
log . Ctx ( context . TODO ( ) ) . Debug ( "drop compaction task meta" , zap . Int64 ( "planID" , task . PlanID ) )
2024-06-05 02:17:50 +00:00
if err != nil {
2024-12-14 14:36:44 +00:00
log . Ctx ( context . TODO ( ) ) . Warn ( "fail to drop task" , zap . Int64 ( "planID" , task . PlanID ) , zap . Error ( err ) )
2024-06-05 02:17:50 +00:00
}
}
}
2023-12-22 04:00:43 +00:00
}
}
2021-11-05 14:25:00 +00:00
}
2024-06-10 13:34:08 +00:00
func ( c * compactionPlanHandler ) cleanPartitionStats ( ) error {
2024-12-14 14:36:44 +00:00
log := log . Ctx ( context . TODO ( ) )
2024-06-10 13:34:08 +00:00
log . Debug ( "start gc partitionStats meta and files" )
// gc partition stats
channelPartitionStatsInfos := make ( map [ string ] [ ] * datapb . PartitionStatsInfo )
unusedPartStats := make ( [ ] * datapb . PartitionStatsInfo , 0 )
if c . meta . GetPartitionStatsMeta ( ) == nil {
return nil
}
infos := c . meta . GetPartitionStatsMeta ( ) . ListAllPartitionStatsInfos ( )
for _ , info := range infos {
collInfo := c . meta . ( * meta ) . GetCollection ( info . GetCollectionID ( ) )
if collInfo == nil {
unusedPartStats = append ( unusedPartStats , info )
continue
}
channel := fmt . Sprintf ( "%d/%d/%s" , info . CollectionID , info . PartitionID , info . VChannel )
if _ , ok := channelPartitionStatsInfos [ channel ] ; ! ok {
channelPartitionStatsInfos [ channel ] = make ( [ ] * datapb . PartitionStatsInfo , 0 )
}
channelPartitionStatsInfos [ channel ] = append ( channelPartitionStatsInfos [ channel ] , info )
}
log . Debug ( "channels with PartitionStats meta" , zap . Int ( "len" , len ( channelPartitionStatsInfos ) ) )
for _ , info := range unusedPartStats {
log . Debug ( "collection has been dropped, remove partition stats" ,
zap . Int64 ( "collID" , info . GetCollectionID ( ) ) )
2024-11-26 11:46:34 +00:00
if err := c . meta . CleanPartitionStatsInfo ( context . TODO ( ) , info ) ; err != nil {
2024-06-10 13:34:08 +00:00
log . Warn ( "gcPartitionStatsInfo fail" , zap . Error ( err ) )
return err
}
}
for channel , infos := range channelPartitionStatsInfos {
sort . Slice ( infos , func ( i , j int ) bool {
return infos [ i ] . Version > infos [ j ] . Version
} )
log . Debug ( "PartitionStats in channel" , zap . String ( "channel" , channel ) , zap . Int ( "len" , len ( infos ) ) )
if len ( infos ) > 2 {
for i := 2 ; i < len ( infos ) ; i ++ {
info := infos [ i ]
2024-11-26 11:46:34 +00:00
if err := c . meta . CleanPartitionStatsInfo ( context . TODO ( ) , info ) ; err != nil {
2024-06-10 13:34:08 +00:00
log . Warn ( "gcPartitionStatsInfo fail" , zap . Error ( err ) )
return err
}
}
}
}
return nil
}
2021-11-05 14:25:00 +00:00
func ( c * compactionPlanHandler ) stop ( ) {
2023-11-29 02:50:29 +00:00
c . stopOnce . Do ( func ( ) {
close ( c . stopCh )
} )
c . stopWg . Wait ( )
}
func ( c * compactionPlanHandler ) removeTasksByChannel ( channel string ) {
2024-12-14 14:36:44 +00:00
log := log . Ctx ( context . TODO ( ) )
2024-10-09 11:11:20 +00:00
log . Info ( "removing tasks by channel" , zap . String ( "channel" , channel ) )
c . queueTasks . RemoveAll ( func ( task CompactionTask ) bool {
2024-10-24 01:55:28 +00:00
if task . GetTaskProto ( ) . GetChannel ( ) == channel {
2023-12-05 10:44:37 +00:00
log . Info ( "Compaction handler removing tasks by channel" ,
zap . String ( "channel" , channel ) ,
2024-10-24 01:55:28 +00:00
zap . Int64 ( "planID" , task . GetTaskProto ( ) . GetPlanID ( ) ) ,
zap . Int64 ( "node" , task . GetTaskProto ( ) . GetNodeID ( ) ) ,
2023-12-05 10:44:37 +00:00
)
2024-10-24 01:55:28 +00:00
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , task . GetTaskProto ( ) . GetNodeID ( ) ) , task . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Pending ) . Dec ( )
2024-10-09 11:11:20 +00:00
return true
2023-11-29 02:50:29 +00:00
}
2024-10-09 11:11:20 +00:00
return false
} )
2024-07-11 09:45:37 +00:00
c . executingGuard . Lock ( )
2024-06-05 02:17:50 +00:00
for id , task := range c . executingTasks {
log . Info ( "Compaction handler removing tasks by channel" ,
2024-10-24 01:55:28 +00:00
zap . String ( "channel" , channel ) , zap . Int64 ( "planID" , id ) , zap . Any ( "task_channel" , task . GetTaskProto ( ) . GetChannel ( ) ) )
if task . GetTaskProto ( ) . GetChannel ( ) == channel {
2024-06-05 02:17:50 +00:00
log . Info ( "Compaction handler removing tasks by channel" ,
zap . String ( "channel" , channel ) ,
2024-10-24 01:55:28 +00:00
zap . Int64 ( "planID" , task . GetTaskProto ( ) . GetPlanID ( ) ) ,
zap . Int64 ( "node" , task . GetTaskProto ( ) . GetNodeID ( ) ) ,
2024-06-05 02:17:50 +00:00
)
delete ( c . executingTasks , id )
2024-10-24 01:55:28 +00:00
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , task . GetTaskProto ( ) . GetNodeID ( ) ) , task . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Executing ) . Dec ( )
2024-06-05 02:17:50 +00:00
}
2023-11-29 10:54:27 +00:00
}
2024-07-11 09:45:37 +00:00
c . executingGuard . Unlock ( )
2023-02-15 08:00:33 +00:00
}
2024-11-29 03:10:36 +00:00
func ( c * compactionPlanHandler ) submitTask ( t CompactionTask ) error {
2024-10-24 01:55:28 +00:00
_ , span := otel . Tracer ( typeutil . DataCoordRole ) . Start ( context . Background ( ) , fmt . Sprintf ( "Compaction-%s" , t . GetTaskProto ( ) . GetType ( ) ) )
2024-06-25 02:08:03 +00:00
t . SetSpan ( span )
2024-11-29 03:10:36 +00:00
if err := c . queueTasks . Enqueue ( t ) ; err != nil {
return err
}
2024-10-24 01:55:28 +00:00
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , NullNodeID ) , t . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Pending ) . Inc ( )
2024-11-29 03:10:36 +00:00
return nil
2021-11-05 14:25:00 +00:00
}
2024-06-05 02:17:50 +00:00
// restoreTask used to restore Task from etcd
func ( c * compactionPlanHandler ) restoreTask ( t CompactionTask ) {
2024-10-24 01:55:28 +00:00
_ , span := otel . Tracer ( typeutil . DataCoordRole ) . Start ( context . Background ( ) , fmt . Sprintf ( "Compaction-%s" , t . GetTaskProto ( ) . GetType ( ) ) )
2024-06-25 02:08:03 +00:00
t . SetSpan ( span )
2024-07-11 09:45:37 +00:00
c . executingGuard . Lock ( )
2024-10-24 01:55:28 +00:00
c . executingTasks [ t . GetTaskProto ( ) . GetPlanID ( ) ] = t
2024-07-11 09:45:37 +00:00
c . executingGuard . Unlock ( )
2024-10-24 01:55:28 +00:00
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , t . GetTaskProto ( ) . GetNodeID ( ) ) , t . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Executing ) . Inc ( )
2024-06-05 02:17:50 +00:00
}
2023-11-23 09:30:25 +00:00
2024-06-05 02:17:50 +00:00
// getCompactionTask return compaction
func ( c * compactionPlanHandler ) getCompactionTask ( planID int64 ) CompactionTask {
2024-10-09 11:11:20 +00:00
var t CompactionTask = nil
c . queueTasks . ForEach ( func ( task CompactionTask ) {
2024-10-24 01:55:28 +00:00
if task . GetTaskProto ( ) . GetPlanID ( ) == planID {
2024-10-09 11:11:20 +00:00
t = task
}
} )
if t != nil {
2024-06-05 02:17:50 +00:00
return t
2023-11-23 09:30:25 +00:00
}
2024-10-09 11:11:20 +00:00
2024-07-11 09:45:37 +00:00
c . executingGuard . RLock ( )
2024-10-09 11:11:20 +00:00
defer c . executingGuard . RUnlock ( )
t = c . executingTasks [ planID ]
2024-06-05 02:17:50 +00:00
return t
2023-11-23 09:30:25 +00:00
}
2024-06-05 02:17:50 +00:00
func ( c * compactionPlanHandler ) enqueueCompaction ( task * datapb . CompactionTask ) error {
2024-12-14 14:36:44 +00:00
log := log . Ctx ( context . TODO ( ) ) . With ( zap . Int64 ( "planID" , task . GetPlanID ( ) ) , zap . Int64 ( "triggerID" , task . GetTriggerID ( ) ) , zap . Int64 ( "collectionID" , task . GetCollectionID ( ) ) , zap . String ( "type" , task . GetType ( ) . String ( ) ) )
2024-06-25 02:08:03 +00:00
t , err := c . createCompactTask ( task )
if err != nil {
2024-07-16 07:57:42 +00:00
// Conflict is normal
if errors . Is ( err , merr . ErrCompactionPlanConflict ) {
log . RatedInfo ( 60 , "Failed to create compaction task, compaction plan conflict" , zap . Error ( err ) )
} else {
log . Warn ( "Failed to create compaction task, unable to create compaction task" , zap . Error ( err ) )
}
2024-06-25 02:08:03 +00:00
return err
2024-06-05 02:17:50 +00:00
}
2024-07-16 07:57:42 +00:00
2024-06-25 02:08:03 +00:00
t . SetTask ( t . ShadowClone ( setStartTime ( time . Now ( ) . Unix ( ) ) ) )
err = t . SaveTaskMeta ( )
if err != nil {
2024-11-26 11:46:34 +00:00
c . meta . SetSegmentsCompacting ( context . TODO ( ) , t . GetTaskProto ( ) . GetInputSegments ( ) , false )
2024-07-16 07:57:42 +00:00
log . Warn ( "Failed to enqueue compaction task, unable to save task meta" , zap . Error ( err ) )
2024-06-25 02:08:03 +00:00
return err
2021-11-05 14:25:00 +00:00
}
2024-11-29 03:10:36 +00:00
if err = c . submitTask ( t ) ; err != nil {
log . Warn ( "submit compaction task failed" , zap . Error ( err ) )
2024-11-29 04:00:36 +00:00
c . meta . SetSegmentsCompacting ( context . Background ( ) , t . GetTaskProto ( ) . GetInputSegments ( ) , false )
2024-11-29 03:10:36 +00:00
return err
}
2024-06-05 02:17:50 +00:00
log . Info ( "Compaction plan submitted" )
return nil
}
2021-11-05 14:25:00 +00:00
2024-06-05 02:17:50 +00:00
// set segments compacting, one segment can only participate one compactionTask
2024-06-25 02:08:03 +00:00
func ( c * compactionPlanHandler ) createCompactTask ( t * datapb . CompactionTask ) ( CompactionTask , error ) {
2024-06-05 02:17:50 +00:00
var task CompactionTask
switch t . GetType ( ) {
case datapb . CompactionType_MixCompaction :
2024-09-26 12:19:14 +00:00
task = newMixCompactionTask ( t , c . allocator , c . meta , c . sessions )
2023-12-05 10:44:37 +00:00
case datapb . CompactionType_Level0DeleteCompaction :
2024-09-26 12:19:14 +00:00
task = newL0CompactionTask ( t , c . allocator , c . meta , c . sessions )
2024-06-10 13:34:08 +00:00
case datapb . CompactionType_ClusteringCompaction :
2024-07-30 12:37:57 +00:00
task = newClusteringCompactionTask ( t , c . allocator , c . meta , c . sessions , c . handler , c . analyzeScheduler )
2024-06-25 02:08:03 +00:00
default :
return nil , merr . WrapErrIllegalCompactionPlan ( "illegal compaction type" )
2021-11-05 14:25:00 +00:00
}
2024-11-26 11:46:34 +00:00
exist , succeed := c . meta . CheckAndSetSegmentsCompacting ( context . TODO ( ) , t . GetInputSegments ( ) )
2024-06-25 02:08:03 +00:00
if ! exist {
return nil , merr . WrapErrIllegalCompactionPlan ( "segment not exist" )
2023-12-05 10:44:37 +00:00
}
2024-06-25 02:08:03 +00:00
if ! succeed {
return nil , merr . WrapErrCompactionPlanConflict ( "segment is compacting" )
}
return task , nil
2023-12-05 10:44:37 +00:00
}
2024-11-13 07:12:30 +00:00
func assignNodeID ( slots map [ int64 ] int64 , t CompactionTask ) int64 {
2024-06-05 02:17:50 +00:00
if len ( slots ) == 0 {
2024-11-13 07:12:30 +00:00
return NullNodeID
2023-11-29 02:50:29 +00:00
}
2022-09-27 08:02:53 +00:00
2024-12-14 14:36:44 +00:00
log := log . Ctx ( context . TODO ( ) )
2024-11-13 07:12:30 +00:00
nodeID , useSlot := pickAnyNode ( slots , t )
if nodeID == NullNodeID {
log . Info ( "compactionHandler cannot find datanode for compaction task" ,
zap . Int64 ( "planID" , t . GetTaskProto ( ) . GetPlanID ( ) ) , zap . String ( "type" , t . GetTaskProto ( ) . GetType ( ) . String ( ) ) , zap . String ( "vchannel" , t . GetTaskProto ( ) . GetChannel ( ) ) )
return NullNodeID
}
err := t . SetNodeID ( nodeID )
if err != nil {
log . Info ( "compactionHandler assignNodeID failed" ,
zap . Int64 ( "planID" , t . GetTaskProto ( ) . GetPlanID ( ) ) , zap . String ( "vchannel" , t . GetTaskProto ( ) . GetChannel ( ) ) , zap . Error ( err ) )
return NullNodeID
2022-09-27 08:02:53 +00:00
}
2024-11-13 07:12:30 +00:00
// update the input nodeSlots
slots [ nodeID ] = slots [ nodeID ] - useSlot
log . Info ( "compactionHandler assignNodeID success" ,
zap . Int64 ( "planID" , t . GetTaskProto ( ) . GetPlanID ( ) ) , zap . String ( "vchannel" , t . GetTaskProto ( ) . GetChannel ( ) ) , zap . Any ( "nodeID" , nodeID ) )
return nodeID
2021-11-05 14:25:00 +00:00
}
2024-12-19 04:36:47 +00:00
// checkCompaction retrieves executing tasks and calls each task's Process() method
// to evaluate its state and progress through the state machine.
// Completed tasks are removed from executingTasks.
// Tasks that fail or timeout are moved from executingTasks to cleaningTasks,
// where task-specific clean logic is performed asynchronously.
2024-06-05 02:17:50 +00:00
func ( c * compactionPlanHandler ) checkCompaction ( ) error {
2023-08-30 03:12:27 +00:00
// Get executing executingTasks before GetCompactionState from DataNode to prevent false failure,
2023-08-01 00:55:04 +00:00
// for DC might add new task while GetCompactionState.
2024-03-01 03:31:00 +00:00
2024-11-13 07:12:30 +00:00
// Assign node id if needed
var slots map [ int64 ] int64
2024-07-11 09:45:37 +00:00
c . executingGuard . RLock ( )
2024-06-05 02:17:50 +00:00
for _ , t := range c . executingTasks {
if t . NeedReAssignNodeID ( ) {
2024-11-13 07:12:30 +00:00
if slots == nil {
slots = c . cluster . QuerySlots ( )
}
id := assignNodeID ( slots , t )
if id == NullNodeID {
break
}
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , NullNodeID ) , t . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Executing ) . Dec ( )
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , t . GetTaskProto ( ) . GetNodeID ( ) ) , t . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Executing ) . Inc ( )
2021-11-05 14:25:00 +00:00
}
}
2024-07-11 09:45:37 +00:00
c . executingGuard . RUnlock ( )
2024-03-01 03:31:00 +00:00
2024-06-05 02:17:50 +00:00
var finishedTasks [ ] CompactionTask
2024-07-11 09:45:37 +00:00
c . executingGuard . RLock ( )
2024-06-05 02:17:50 +00:00
for _ , t := range c . executingTasks {
2024-09-08 09:23:05 +00:00
c . checkDelay ( t )
2024-06-05 02:17:50 +00:00
finished := t . Process ( )
if finished {
finishedTasks = append ( finishedTasks , t )
2023-08-30 03:12:27 +00:00
}
}
2024-07-11 09:45:37 +00:00
c . executingGuard . RUnlock ( )
2024-03-01 03:31:00 +00:00
2024-06-05 02:17:50 +00:00
// delete all finished
2024-07-11 09:45:37 +00:00
c . executingGuard . Lock ( )
2024-06-05 02:17:50 +00:00
for _ , t := range finishedTasks {
2024-10-24 01:55:28 +00:00
delete ( c . executingTasks , t . GetTaskProto ( ) . GetPlanID ( ) )
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , t . GetTaskProto ( ) . GetNodeID ( ) ) , t . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Executing ) . Dec ( )
metrics . DataCoordCompactionTaskNum . WithLabelValues ( fmt . Sprintf ( "%d" , t . GetTaskProto ( ) . GetNodeID ( ) ) , t . GetTaskProto ( ) . GetType ( ) . String ( ) , metrics . Done ) . Inc ( )
2021-11-05 14:25:00 +00:00
}
2024-07-11 09:45:37 +00:00
c . executingGuard . Unlock ( )
2024-12-19 04:36:47 +00:00
// insert task need to clean
c . cleaningGuard . Lock ( )
for _ , t := range finishedTasks {
if t . GetTaskProto ( ) . GetState ( ) == datapb . CompactionTaskState_failed ||
t . GetTaskProto ( ) . GetState ( ) == datapb . CompactionTaskState_timeout ||
t . GetTaskProto ( ) . GetState ( ) == datapb . CompactionTaskState_completed {
log . Ctx ( context . TODO ( ) ) . Info ( "task need to clean" ,
zap . Int64 ( "collectionID" , t . GetTaskProto ( ) . GetCollectionID ( ) ) ,
zap . Int64 ( "planID" , t . GetTaskProto ( ) . GetPlanID ( ) ) ,
zap . String ( "state" , t . GetTaskProto ( ) . GetState ( ) . String ( ) ) )
c . cleaningTasks [ t . GetTaskProto ( ) . GetPlanID ( ) ] = t
}
}
c . cleaningGuard . Unlock ( )
2024-06-05 02:17:50 +00:00
return nil
2021-11-05 14:25:00 +00:00
}
2024-12-19 04:36:47 +00:00
// cleanFailedTasks performs task define Clean logic
// while compactionPlanHandler.Clean is to do garbage collection for cleaned tasks
func ( c * compactionPlanHandler ) cleanFailedTasks ( ) {
c . cleaningGuard . RLock ( )
cleanedTasks := make ( [ ] CompactionTask , 0 )
for _ , t := range c . cleaningTasks {
clean := t . Clean ( )
if clean {
cleanedTasks = append ( cleanedTasks , t )
}
}
c . cleaningGuard . RUnlock ( )
c . cleaningGuard . Lock ( )
for _ , t := range cleanedTasks {
delete ( c . cleaningTasks , t . GetTaskProto ( ) . GetPlanID ( ) )
}
c . cleaningGuard . Unlock ( )
}
2024-11-13 07:12:30 +00:00
func pickAnyNode ( nodeSlots map [ int64 ] int64 , task CompactionTask ) ( nodeID int64 , useSlot int64 ) {
2024-07-18 02:27:41 +00:00
nodeID = NullNodeID
var maxSlots int64 = - 1
2024-09-26 12:19:14 +00:00
useSlot = task . GetSlotUsage ( )
if useSlot <= 0 {
2024-12-14 14:36:44 +00:00
log . Ctx ( context . TODO ( ) ) . Warn ( "task slot should not be 0" , zap . Int64 ( "planID" , task . GetTaskProto ( ) . GetPlanID ( ) ) , zap . String ( "type" , task . GetTaskProto ( ) . GetType ( ) . String ( ) ) )
2024-09-26 12:19:14 +00:00
return NullNodeID , useSlot
2024-07-18 02:27:41 +00:00
}
2024-06-05 02:17:50 +00:00
for id , slots := range nodeSlots {
2024-07-18 02:27:41 +00:00
if slots >= useSlot && slots > maxSlots {
2024-06-05 02:17:50 +00:00
nodeID = id
maxSlots = slots
2021-11-05 14:25:00 +00:00
}
}
2024-07-18 02:27:41 +00:00
return nodeID , useSlot
2021-11-05 14:25:00 +00:00
}
2024-06-05 02:17:50 +00:00
// isFull return true if the task pool is full
func ( c * compactionPlanHandler ) isFull ( ) bool {
2024-10-09 11:11:20 +00:00
return c . queueTasks . Len ( ) >= c . queueTasks . capacity
2022-08-23 07:50:52 +00:00
}
2023-11-06 19:18:18 +00:00
2024-09-08 09:23:05 +00:00
func ( c * compactionPlanHandler ) checkDelay ( t CompactionTask ) {
log := log . Ctx ( context . TODO ( ) ) . WithRateGroup ( "compactionPlanHandler.checkDelay" , 1.0 , 60.0 )
2024-10-24 01:55:28 +00:00
maxExecDuration := maxCompactionTaskExecutionDuration [ t . GetTaskProto ( ) . GetType ( ) ]
startTime := time . Unix ( t . GetTaskProto ( ) . GetStartTime ( ) , 0 )
2024-09-08 09:23:05 +00:00
execDuration := time . Since ( startTime )
if execDuration >= maxExecDuration {
log . RatedWarn ( 60 , "compaction task is delay" ,
2024-10-24 01:55:28 +00:00
zap . Int64 ( "planID" , t . GetTaskProto ( ) . GetPlanID ( ) ) ,
zap . String ( "type" , t . GetTaskProto ( ) . GetType ( ) . String ( ) ) ,
zap . String ( "state" , t . GetTaskProto ( ) . GetState ( ) . String ( ) ) ,
zap . String ( "vchannel" , t . GetTaskProto ( ) . GetChannel ( ) ) ,
zap . Int64 ( "nodeID" , t . GetTaskProto ( ) . GetNodeID ( ) ) ,
2024-09-08 09:23:05 +00:00
zap . Time ( "startTime" , startTime ) ,
zap . Duration ( "execDuration" , execDuration ) )
}
}
2023-11-06 19:18:18 +00:00
var (
ioPool * conc . Pool [ any ]
ioPoolInitOnce sync . Once
)
func initIOPool ( ) {
capacity := Params . DataNodeCfg . IOConcurrency . GetAsInt ( )
if capacity > 32 {
capacity = 32
}
// error only happens with negative expiry duration or with negative pre-alloc size.
ioPool = conc . NewPool [ any ] ( capacity )
}
func getOrCreateIOPool ( ) * conc . Pool [ any ] {
ioPoolInitOnce . Do ( initIOPool )
return ioPool
}