mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-01 11:29:48 +08:00
c924f73105
Co-authored-by: Zach41 <zongmei.zhang@zilliz.com> Signed-off-by: cai.zhang <cai.zhang@zilliz.com> Signed-off-by: cai.zhang <cai.zhang@zilliz.com> Co-authored-by: Zach41 <zongmei.zhang@zilliz.com>
304 lines
7.5 KiB
Go
304 lines
7.5 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 indexcoord
|
|
|
|
import (
|
|
"container/list"
|
|
"context"
|
|
"errors"
|
|
"sync"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
"github.com/milvus-io/milvus/internal/storage"
|
|
"github.com/milvus-io/milvus/internal/types"
|
|
"github.com/milvus-io/milvus/internal/util/trace"
|
|
"github.com/opentracing/opentracing-go"
|
|
oplog "github.com/opentracing/opentracing-go/log"
|
|
)
|
|
|
|
// TaskQueue is a queue used to store tasks.
|
|
type TaskQueue interface {
|
|
utChan() <-chan int
|
|
utEmpty() bool
|
|
utFull() bool
|
|
addUnissuedTask(t task) error
|
|
//FrontUnissuedTask() task
|
|
PopUnissuedTask() task
|
|
AddActiveTask(t task)
|
|
PopActiveTask(tID UniqueID) task
|
|
Enqueue(t task) error
|
|
tryToRemoveUselessIndexAddTask(indexID UniqueID) []UniqueID
|
|
}
|
|
|
|
// BaseTaskQueue is a basic instance of TaskQueue.
|
|
type BaseTaskQueue struct {
|
|
unissuedTasks *list.List
|
|
activeTasks map[UniqueID]task
|
|
utLock sync.Mutex
|
|
atLock sync.Mutex
|
|
|
|
// maxTaskNum should keep still
|
|
maxTaskNum int64
|
|
|
|
utBufChan chan int // to block scheduler
|
|
|
|
sched *TaskScheduler
|
|
}
|
|
|
|
func (queue *BaseTaskQueue) utChan() <-chan int {
|
|
return queue.utBufChan
|
|
}
|
|
|
|
func (queue *BaseTaskQueue) utEmpty() bool {
|
|
return queue.unissuedTasks.Len() == 0
|
|
}
|
|
|
|
func (queue *BaseTaskQueue) utFull() bool {
|
|
return int64(queue.unissuedTasks.Len()) >= queue.maxTaskNum
|
|
}
|
|
|
|
func (queue *BaseTaskQueue) addUnissuedTask(t task) error {
|
|
queue.utLock.Lock()
|
|
defer queue.utLock.Unlock()
|
|
|
|
if queue.utFull() {
|
|
return errors.New("task queue is full")
|
|
}
|
|
queue.unissuedTasks.PushBack(t)
|
|
queue.utBufChan <- 1
|
|
return nil
|
|
}
|
|
|
|
//func (queue *BaseTaskQueue) FrontUnissuedTask() task {
|
|
// queue.utLock.Lock()
|
|
// defer queue.utLock.Unlock()
|
|
//
|
|
// if queue.unissuedTasks.Len() <= 0 {
|
|
// log.Warn("sorry, but the unissued task list is empty!")
|
|
// return nil
|
|
// }
|
|
//
|
|
// return queue.unissuedTasks.Front().Value.(task)
|
|
//}
|
|
|
|
// PopUnissuedTask pops a task from tasks queue.
|
|
func (queue *BaseTaskQueue) PopUnissuedTask() task {
|
|
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)
|
|
}
|
|
|
|
// AddActiveTask adds a task to activeTasks.
|
|
func (queue *BaseTaskQueue) AddActiveTask(t task) {
|
|
queue.atLock.Lock()
|
|
defer queue.atLock.Unlock()
|
|
|
|
tID := t.ID()
|
|
_, ok := queue.activeTasks[tID]
|
|
if ok {
|
|
log.Warn("indexcoord", zap.Int64("task with ID already in active task list!", tID))
|
|
}
|
|
|
|
queue.activeTasks[tID] = t
|
|
}
|
|
|
|
// PopActiveTask tasks out a task from activateTask and the task will be executed.
|
|
func (queue *BaseTaskQueue) PopActiveTask(tID UniqueID) task {
|
|
queue.atLock.Lock()
|
|
defer queue.atLock.Unlock()
|
|
|
|
t, ok := queue.activeTasks[tID]
|
|
if ok {
|
|
delete(queue.activeTasks, tID)
|
|
return t
|
|
}
|
|
log.Debug("indexcoord", zap.Int64("sorry, but the ID was not found in the active task list!", tID))
|
|
return nil
|
|
}
|
|
|
|
// Enqueue adds a task to TaskQueue.
|
|
func (queue *BaseTaskQueue) Enqueue(t task) error {
|
|
err := t.OnEnqueue()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
return queue.addUnissuedTask(t)
|
|
}
|
|
|
|
// IndexAddTaskQueue is a task queue used to store building index tasks.
|
|
type IndexAddTaskQueue struct {
|
|
BaseTaskQueue
|
|
lock sync.Mutex
|
|
}
|
|
|
|
// Enqueue adds a building index task to IndexAddTaskQueue.
|
|
func (queue *IndexAddTaskQueue) Enqueue(t task) error {
|
|
queue.lock.Lock()
|
|
defer queue.lock.Unlock()
|
|
return queue.BaseTaskQueue.Enqueue(t)
|
|
}
|
|
|
|
// Note: tryToRemoveUselessIndexAddTask must be called by DropIndex
|
|
func (queue *IndexAddTaskQueue) tryToRemoveUselessIndexAddTask(indexID UniqueID) []UniqueID {
|
|
queue.lock.Lock()
|
|
defer queue.lock.Unlock()
|
|
|
|
var indexBuildIDs []UniqueID
|
|
var next *list.Element
|
|
for e := queue.unissuedTasks.Front(); e != nil; e = next {
|
|
next = e.Next()
|
|
indexAddTask, ok := e.Value.(*IndexAddTask)
|
|
if !ok {
|
|
continue
|
|
}
|
|
if indexAddTask.segmentIndex.IndexID == indexID {
|
|
queue.unissuedTasks.Remove(e)
|
|
indexAddTask.Notify(nil)
|
|
indexBuildIDs = append(indexBuildIDs, indexAddTask.segmentIndex.BuildID)
|
|
}
|
|
}
|
|
|
|
return indexBuildIDs
|
|
}
|
|
|
|
// NewIndexAddTaskQueue creates a new IndexAddTaskQueue.
|
|
func NewIndexAddTaskQueue(sched *TaskScheduler) *IndexAddTaskQueue {
|
|
return &IndexAddTaskQueue{
|
|
BaseTaskQueue: BaseTaskQueue{
|
|
unissuedTasks: list.New(),
|
|
activeTasks: make(map[UniqueID]task),
|
|
maxTaskNum: 1024,
|
|
utBufChan: make(chan int, 1024),
|
|
sched: sched,
|
|
},
|
|
}
|
|
}
|
|
|
|
// TaskScheduler is a scheduler of indexing tasks.
|
|
type TaskScheduler struct {
|
|
IndexAddQueue TaskQueue
|
|
|
|
rootcoordClient types.RootCoord
|
|
metaTable *metaTable
|
|
cm storage.ChunkManager
|
|
|
|
wg sync.WaitGroup
|
|
ctx context.Context
|
|
cancel context.CancelFunc
|
|
}
|
|
|
|
// NewTaskScheduler creates a new task scheduler of indexing tasks.
|
|
func NewTaskScheduler(ctx context.Context,
|
|
client types.RootCoord,
|
|
cm storage.ChunkManager,
|
|
table *metaTable) (*TaskScheduler, error) {
|
|
ctx1, cancel := context.WithCancel(ctx)
|
|
s := &TaskScheduler{
|
|
metaTable: table,
|
|
rootcoordClient: client,
|
|
cm: cm,
|
|
ctx: ctx1,
|
|
cancel: cancel,
|
|
}
|
|
s.IndexAddQueue = NewIndexAddTaskQueue(s)
|
|
return s, nil
|
|
}
|
|
|
|
func (sched *TaskScheduler) scheduleIndexAddTask() task {
|
|
return sched.IndexAddQueue.PopUnissuedTask()
|
|
}
|
|
|
|
//func (sched *TaskScheduler) scheduleIndexBuildClient() indexnode.Interface {
|
|
// return sched.IndexAddQueue.PopUnissuedTask()
|
|
//}
|
|
|
|
func (sched *TaskScheduler) processTask(t task, q TaskQueue) {
|
|
span, ctx := trace.StartSpanFromContext(t.Ctx(),
|
|
opentracing.Tags{
|
|
"Type": t.Name(),
|
|
})
|
|
defer span.Finish()
|
|
span.LogFields(oplog.String("scheduler process PreExecute", t.Name()))
|
|
err := t.PreExecute(ctx)
|
|
|
|
defer func() {
|
|
t.Notify(err)
|
|
}()
|
|
if err != nil {
|
|
trace.LogError(span, err)
|
|
return
|
|
}
|
|
|
|
span.LogFields(oplog.String("scheduler process AddActiveTask", t.Name()))
|
|
q.AddActiveTask(t)
|
|
defer func() {
|
|
span.LogFields(oplog.String("scheduler process PopActiveTask", t.Name()))
|
|
q.PopActiveTask(t.ID())
|
|
}()
|
|
|
|
span.LogFields(oplog.String("scheduler process Execute", t.Name()))
|
|
err = t.Execute(ctx)
|
|
if err != nil {
|
|
trace.LogError(span, err)
|
|
return
|
|
}
|
|
span.LogFields(oplog.String("scheduler process PostExecute", t.Name()))
|
|
err = t.PostExecute(ctx)
|
|
}
|
|
|
|
func (sched *TaskScheduler) indexAddLoop() {
|
|
defer sched.wg.Done()
|
|
for {
|
|
select {
|
|
case <-sched.ctx.Done():
|
|
return
|
|
case <-sched.IndexAddQueue.utChan():
|
|
if !sched.IndexAddQueue.utEmpty() {
|
|
t := sched.scheduleIndexAddTask()
|
|
go sched.processTask(t, sched.IndexAddQueue)
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// Start stats the task scheduler of indexing tasks.
|
|
func (sched *TaskScheduler) Start() error {
|
|
|
|
sched.wg.Add(1)
|
|
go sched.indexAddLoop()
|
|
|
|
return nil
|
|
}
|
|
|
|
// Close closes the task scheduler of indexing tasks.
|
|
func (sched *TaskScheduler) Close() {
|
|
if sched.cancel != nil {
|
|
sched.cancel()
|
|
}
|
|
sched.wg.Wait()
|
|
}
|