2021-11-01 22:51:41 +08: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
|
2021-04-19 13:50:12 +08:00
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
2021-11-01 22:51:41 +08:00
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
2021-04-19 13:50:12 +08:00
|
|
|
//
|
2021-11-01 22:51:41 +08:00
|
|
|
// 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-04-19 13:50:12 +08:00
|
|
|
|
2021-01-15 14:38:36 +08:00
|
|
|
package indexnode
|
2020-12-10 17:55:55 +08:00
|
|
|
|
|
|
|
import (
|
|
|
|
"container/list"
|
|
|
|
"context"
|
2023-06-26 17:52:44 +08:00
|
|
|
"fmt"
|
2022-08-25 15:48:54 +08:00
|
|
|
"runtime/debug"
|
2020-12-10 17:55:55 +08:00
|
|
|
"sync"
|
|
|
|
|
2023-02-26 11:31:49 +08:00
|
|
|
"github.com/cockroachdb/errors"
|
2021-03-10 09:56:09 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2024-06-06 17:37:51 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
2023-06-26 17:52:44 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
2023-10-11 21:01:35 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
2023-06-26 17:52:44 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
2020-12-10 17:55:55 +08:00
|
|
|
)
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// TaskQueue is a queue used to store tasks.
|
2020-12-10 17:55:55 +08:00
|
|
|
type TaskQueue interface {
|
|
|
|
utChan() <-chan int
|
|
|
|
utEmpty() bool
|
|
|
|
utFull() bool
|
|
|
|
addUnissuedTask(t task) error
|
|
|
|
PopUnissuedTask() task
|
|
|
|
AddActiveTask(t task)
|
2022-08-25 15:48:54 +08:00
|
|
|
PopActiveTask(tName string) task
|
2020-12-10 17:55:55 +08:00
|
|
|
Enqueue(t task) error
|
2022-08-25 15:48:54 +08:00
|
|
|
GetTaskNum() (int, int)
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// BaseTaskQueue is a basic instance of TaskQueue.
|
2022-08-25 15:48:54 +08:00
|
|
|
type IndexTaskQueue struct {
|
2020-12-10 17:55:55 +08:00
|
|
|
unissuedTasks *list.List
|
2022-08-25 15:48:54 +08:00
|
|
|
activeTasks map[string]task
|
2020-12-10 17:55:55 +08:00
|
|
|
utLock sync.Mutex
|
|
|
|
atLock sync.Mutex
|
|
|
|
|
|
|
|
// maxTaskNum should keep still
|
|
|
|
maxTaskNum int64
|
|
|
|
|
|
|
|
utBufChan chan int // to block scheduler
|
|
|
|
|
|
|
|
sched *TaskScheduler
|
|
|
|
}
|
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) utChan() <-chan int {
|
2020-12-10 17:55:55 +08:00
|
|
|
return queue.utBufChan
|
|
|
|
}
|
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) utEmpty() bool {
|
2020-12-10 17:55:55 +08:00
|
|
|
return queue.unissuedTasks.Len() == 0
|
|
|
|
}
|
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) utFull() bool {
|
2020-12-10 17:55:55 +08:00
|
|
|
return int64(queue.unissuedTasks.Len()) >= queue.maxTaskNum
|
|
|
|
}
|
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) addUnissuedTask(t task) error {
|
2020-12-10 17:55:55 +08:00
|
|
|
queue.utLock.Lock()
|
|
|
|
defer queue.utLock.Unlock()
|
|
|
|
|
|
|
|
if queue.utFull() {
|
2021-06-06 09:41:35 +08:00
|
|
|
return errors.New("IndexNode task queue is full")
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
queue.unissuedTasks.PushBack(t)
|
|
|
|
queue.utBufChan <- 1
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// PopUnissuedTask pops a task from tasks queue.
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) PopUnissuedTask() task {
|
2020-12-10 17:55:55 +08:00
|
|
|
queue.utLock.Lock()
|
|
|
|
defer queue.utLock.Unlock()
|
|
|
|
|
|
|
|
if queue.unissuedTasks.Len() <= 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
ft := queue.unissuedTasks.Front()
|
|
|
|
queue.unissuedTasks.Remove(ft)
|
|
|
|
|
|
|
|
return ft.Value.(task)
|
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// AddActiveTask adds a task to activeTasks.
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) AddActiveTask(t task) {
|
2020-12-10 17:55:55 +08:00
|
|
|
queue.atLock.Lock()
|
|
|
|
defer queue.atLock.Unlock()
|
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
tName := t.Name()
|
|
|
|
_, ok := queue.activeTasks[tName]
|
2020-12-10 17:55:55 +08:00
|
|
|
if ok {
|
2024-01-05 16:12:48 +08:00
|
|
|
log.Debug("IndexNode task already in active task list", zap.String("TaskID", tName))
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
queue.activeTasks[tName] = t
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
|
2021-12-28 19:50:25 +08:00
|
|
|
// PopActiveTask pops a task from activateTask and the task will be executed.
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) PopActiveTask(tName string) task {
|
2020-12-10 17:55:55 +08:00
|
|
|
queue.atLock.Lock()
|
|
|
|
defer queue.atLock.Unlock()
|
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
t, ok := queue.activeTasks[tName]
|
2020-12-10 17:55:55 +08:00
|
|
|
if ok {
|
2022-08-25 15:48:54 +08:00
|
|
|
delete(queue.activeTasks, tName)
|
2020-12-10 17:55:55 +08:00
|
|
|
return t
|
|
|
|
}
|
2022-08-25 15:48:54 +08:00
|
|
|
log.Debug("IndexNode task was not found in the active task list", zap.String("TaskName", tName))
|
2020-12-10 17:55:55 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// Enqueue adds a task to TaskQueue.
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) Enqueue(t task) error {
|
|
|
|
err := t.OnEnqueue(t.Ctx())
|
2020-12-13 06:48:05 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2020-12-10 17:55:55 +08:00
|
|
|
return queue.addUnissuedTask(t)
|
|
|
|
}
|
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
func (queue *IndexTaskQueue) GetTaskNum() (int, int) {
|
2022-07-07 14:44:21 +08:00
|
|
|
queue.utLock.Lock()
|
2022-08-25 15:48:54 +08:00
|
|
|
defer queue.utLock.Unlock()
|
2022-07-07 14:44:21 +08:00
|
|
|
queue.atLock.Lock()
|
2022-08-25 15:48:54 +08:00
|
|
|
defer queue.atLock.Unlock()
|
2022-07-07 14:44:21 +08:00
|
|
|
|
2022-08-25 15:48:54 +08:00
|
|
|
utNum := queue.unissuedTasks.Len()
|
2022-09-23 09:40:52 +08:00
|
|
|
atNum := 0
|
|
|
|
// remove the finished task
|
|
|
|
for _, task := range queue.activeTasks {
|
2024-06-06 17:37:51 +08:00
|
|
|
if task.GetState() != indexpb.JobState_JobStateFinished && task.GetState() != indexpb.JobState_JobStateFailed {
|
2022-09-23 09:40:52 +08:00
|
|
|
atNum++
|
|
|
|
}
|
|
|
|
}
|
2022-08-25 15:48:54 +08:00
|
|
|
return utNum, atNum
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// NewIndexBuildTaskQueue creates a new IndexBuildTaskQueue.
|
2022-08-25 15:48:54 +08:00
|
|
|
func NewIndexBuildTaskQueue(sched *TaskScheduler) *IndexTaskQueue {
|
|
|
|
return &IndexTaskQueue{
|
|
|
|
unissuedTasks: list.New(),
|
|
|
|
activeTasks: make(map[string]task),
|
|
|
|
maxTaskNum: 1024,
|
2023-12-13 17:24:38 +08:00
|
|
|
|
|
|
|
utBufChan: make(chan int, 1024),
|
|
|
|
sched: sched,
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// TaskScheduler is a scheduler of indexing tasks.
|
2020-12-10 17:55:55 +08:00
|
|
|
type TaskScheduler struct {
|
2024-06-06 17:37:51 +08:00
|
|
|
TaskQueue TaskQueue
|
2020-12-10 17:55:55 +08:00
|
|
|
|
2021-01-26 09:38:40 +08:00
|
|
|
buildParallel int
|
|
|
|
wg sync.WaitGroup
|
|
|
|
ctx context.Context
|
|
|
|
cancel context.CancelFunc
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// NewTaskScheduler creates a new task scheduler of indexing tasks.
|
2023-01-12 19:49:40 +08:00
|
|
|
func NewTaskScheduler(ctx context.Context) *TaskScheduler {
|
2020-12-10 17:55:55 +08:00
|
|
|
ctx1, cancel := context.WithCancel(ctx)
|
|
|
|
s := &TaskScheduler{
|
2021-01-26 09:38:40 +08:00
|
|
|
ctx: ctx1,
|
|
|
|
cancel: cancel,
|
2022-12-07 18:01:19 +08:00
|
|
|
buildParallel: Params.IndexNodeCfg.BuildParallel.GetAsInt(),
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
2024-06-06 17:37:51 +08:00
|
|
|
s.TaskQueue = NewIndexBuildTaskQueue(s)
|
2020-12-10 17:55:55 +08:00
|
|
|
|
2023-01-12 19:49:40 +08:00
|
|
|
return s
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
|
2021-01-26 09:38:40 +08:00
|
|
|
func (sched *TaskScheduler) scheduleIndexBuildTask() []task {
|
|
|
|
ret := make([]task, 0)
|
|
|
|
for i := 0; i < sched.buildParallel; i++ {
|
2024-06-06 17:37:51 +08:00
|
|
|
t := sched.TaskQueue.PopUnissuedTask()
|
2021-01-26 09:38:40 +08:00
|
|
|
if t == nil {
|
|
|
|
return ret
|
|
|
|
}
|
|
|
|
ret = append(ret, t)
|
|
|
|
}
|
|
|
|
return ret
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
|
2024-06-06 17:37:51 +08:00
|
|
|
func getStateFromError(err error) indexpb.JobState {
|
2024-04-09 11:51:18 +08:00
|
|
|
if errors.Is(err, errCancel) {
|
2024-06-06 17:37:51 +08:00
|
|
|
return indexpb.JobState_JobStateRetry
|
2024-04-09 11:51:18 +08:00
|
|
|
} else if errors.Is(err, merr.ErrIoKeyNotFound) || errors.Is(err, merr.ErrSegcoreUnsupported) {
|
|
|
|
// NoSuchKey or unsupported error
|
2024-06-06 17:37:51 +08:00
|
|
|
return indexpb.JobState_JobStateFailed
|
|
|
|
} else if errors.Is(err, merr.ErrSegcorePretendFinished) {
|
|
|
|
return indexpb.JobState_JobStateFinished
|
2024-04-09 11:51:18 +08:00
|
|
|
}
|
2024-06-06 17:37:51 +08:00
|
|
|
return indexpb.JobState_JobStateRetry
|
2024-04-09 11:51:18 +08:00
|
|
|
}
|
|
|
|
|
2020-12-10 17:55:55 +08:00
|
|
|
func (sched *TaskScheduler) processTask(t task, q TaskQueue) {
|
2022-08-25 15:48:54 +08:00
|
|
|
wrap := func(fn func(ctx context.Context) error) error {
|
|
|
|
select {
|
|
|
|
case <-t.Ctx().Done():
|
|
|
|
return errCancel
|
|
|
|
default:
|
|
|
|
return fn(t.Ctx())
|
|
|
|
}
|
2022-01-24 17:18:46 +08:00
|
|
|
}
|
|
|
|
|
2020-12-10 17:55:55 +08:00
|
|
|
defer func() {
|
2022-08-25 15:48:54 +08:00
|
|
|
t.Reset()
|
|
|
|
debug.FreeOSMemory()
|
2020-12-10 17:55:55 +08:00
|
|
|
}()
|
2024-06-06 17:37:51 +08:00
|
|
|
sched.TaskQueue.AddActiveTask(t)
|
|
|
|
defer sched.TaskQueue.PopActiveTask(t.Name())
|
2022-09-27 14:56:52 +08:00
|
|
|
log.Ctx(t.Ctx()).Debug("process task", zap.String("task", t.Name()))
|
2024-06-06 17:37:51 +08:00
|
|
|
pipelines := []func(context.Context) error{t.PreExecute, t.Execute, t.PostExecute}
|
2022-08-25 15:48:54 +08:00
|
|
|
for _, fn := range pipelines {
|
|
|
|
if err := wrap(fn); err != nil {
|
2024-04-09 11:51:18 +08:00
|
|
|
log.Ctx(t.Ctx()).Warn("process task failed", zap.Error(err))
|
|
|
|
t.SetState(getStateFromError(err), err.Error())
|
2022-08-25 15:48:54 +08:00
|
|
|
return
|
|
|
|
}
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
2024-06-06 17:37:51 +08:00
|
|
|
t.SetState(indexpb.JobState_JobStateFinished, "")
|
2023-06-26 17:52:44 +08:00
|
|
|
if indexBuildTask, ok := t.(*indexBuildTask); ok {
|
2023-09-13 17:21:18 +08:00
|
|
|
metrics.IndexNodeBuildIndexLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(indexBuildTask.tr.ElapseSpan().Seconds())
|
2023-06-26 17:52:44 +08:00
|
|
|
metrics.IndexNodeIndexTaskLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(indexBuildTask.queueDur.Milliseconds()))
|
|
|
|
}
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
|
2020-12-13 06:48:05 +08:00
|
|
|
func (sched *TaskScheduler) indexBuildLoop() {
|
2021-06-06 09:41:35 +08:00
|
|
|
log.Debug("IndexNode TaskScheduler start build loop ...")
|
2020-12-10 17:55:55 +08:00
|
|
|
defer sched.wg.Done()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-sched.ctx.Done():
|
|
|
|
return
|
2024-06-06 17:37:51 +08:00
|
|
|
case <-sched.TaskQueue.utChan():
|
2022-08-25 15:48:54 +08:00
|
|
|
tasks := sched.scheduleIndexBuildTask()
|
|
|
|
var wg sync.WaitGroup
|
|
|
|
for _, t := range tasks {
|
|
|
|
wg.Add(1)
|
|
|
|
go func(group *sync.WaitGroup, t task) {
|
|
|
|
defer group.Done()
|
2024-06-06 17:37:51 +08:00
|
|
|
sched.processTask(t, sched.TaskQueue)
|
2022-08-25 15:48:54 +08:00
|
|
|
}(&wg, t)
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
2022-08-25 15:48:54 +08:00
|
|
|
wg.Wait()
|
2020-12-10 17:55:55 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// Start stats the task scheduler of indexing tasks.
|
2020-12-10 17:55:55 +08:00
|
|
|
func (sched *TaskScheduler) Start() error {
|
2020-12-13 06:48:05 +08:00
|
|
|
sched.wg.Add(1)
|
|
|
|
go sched.indexBuildLoop()
|
2020-12-10 17:55:55 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-03 10:15:56 +08:00
|
|
|
// Close closes the task scheduler of indexing tasks.
|
2020-12-10 17:55:55 +08:00
|
|
|
func (sched *TaskScheduler) Close() {
|
|
|
|
sched.cancel()
|
|
|
|
sched.wg.Wait()
|
|
|
|
}
|