2022-03-21 15:47:23 +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
// 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 rootcoord
import (
"context"
"errors"
2022-03-25 11:03:25 +08:00
"fmt"
2022-06-14 16:18:09 +08:00
"sort"
2022-03-21 15:47:23 +08:00
"strconv"
2022-04-20 14:03:40 +08:00
"strings"
2022-03-21 15:47:23 +08:00
"sync"
"time"
2022-03-25 11:03:25 +08:00
"github.com/golang/protobuf/proto"
"github.com/milvus-io/milvus/internal/kv"
2022-03-21 15:47:23 +08:00
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/milvuspb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
2022-06-15 12:20:10 +08:00
"github.com/milvus-io/milvus/internal/util/retry"
2022-04-24 11:29:45 +08:00
"github.com/milvus-io/milvus/internal/util/typeutil"
2022-03-21 15:47:23 +08:00
"go.uber.org/zap"
)
const (
2022-04-03 11:37:29 +08:00
Bucket = "bucket"
FailedReason = "failed_reason"
2022-04-20 14:03:40 +08:00
Files = "files"
2022-06-14 16:18:09 +08:00
CollectionName = "collection"
PartitionName = "partition"
2022-04-03 11:37:29 +08:00
MaxPendingCount = 32
delimiter = "/"
taskExpiredMsgPrefix = "task has expired after "
2022-03-21 15:47:23 +08:00
)
2022-05-07 14:05:52 +08:00
// CheckPendingTasksInterval is the default interval to check and send out pending tasks,
// default 60*1000 milliseconds (1 minute).
var checkPendingTasksInterval = 60 * 1000
// ExpireOldTasksInterval is the default interval to loop through all in memory tasks and expire old ones.
2022-06-15 12:20:10 +08:00
// default 2*60*1000 milliseconds (2 minutes)
var expireOldTasksInterval = 2 * 60 * 1000
2022-05-07 14:05:52 +08:00
2022-03-21 15:47:23 +08:00
// import task state
type importTaskState struct {
stateCode commonpb . ImportState // state code
2022-03-25 11:03:25 +08:00
segments [ ] int64 // ID list of generated segments
rowIDs [ ] int64 // ID list of auto-generated is for auto-id primary key
2022-03-21 15:47:23 +08:00
rowCount int64 // how many rows imported
failedReason string // failed reason
}
// importManager manager for import tasks
type importManager struct {
2022-04-03 11:37:29 +08:00
ctx context . Context // reserved
taskStore kv . MetaKv // Persistent task info storage.
busyNodes map [ int64 ] bool // Set of all current working DataNodes.
2022-03-21 15:47:23 +08:00
2022-03-31 13:51:28 +08:00
// TODO: Make pendingTask a map to improve look up performance.
2022-04-01 11:33:28 +08:00
pendingTasks [ ] * datapb . ImportTaskInfo // pending tasks
workingTasks map [ int64 ] * datapb . ImportTaskInfo // in-progress tasks
pendingLock sync . RWMutex // lock pending task list
workingLock sync . RWMutex // lock working task map
busyNodesLock sync . RWMutex // lock for working nodes.
lastReqID int64 // for generating a unique ID for import request
2022-04-03 11:37:29 +08:00
startOnce sync . Once
2022-04-24 11:29:45 +08:00
idAllocator func ( count uint32 ) ( typeutil . UniqueID , typeutil . UniqueID , error )
2022-04-01 11:33:28 +08:00
callImportService func ( ctx context . Context , req * datapb . ImportTaskRequest ) * datapb . ImportTaskResponse
2022-06-14 16:18:09 +08:00
getCollectionName func ( collID , partitionID typeutil . UniqueID ) ( string , string , error )
2022-03-21 15:47:23 +08:00
}
// newImportManager helper function to create a importManager
2022-04-03 11:37:29 +08:00
func newImportManager ( ctx context . Context , client kv . MetaKv ,
2022-04-24 11:29:45 +08:00
idAlloc func ( count uint32 ) ( typeutil . UniqueID , typeutil . UniqueID , error ) ,
2022-06-14 16:18:09 +08:00
importService func ( ctx context . Context , req * datapb . ImportTaskRequest ) * datapb . ImportTaskResponse ,
getCollectionName func ( collID , partitionID typeutil . UniqueID ) ( string , string , error ) ) * importManager {
2022-03-21 15:47:23 +08:00
mgr := & importManager {
ctx : ctx ,
2022-03-25 11:03:25 +08:00
taskStore : client ,
pendingTasks : make ( [ ] * datapb . ImportTaskInfo , 0 , MaxPendingCount ) , // currently task queue max size is 32
workingTasks : make ( map [ int64 ] * datapb . ImportTaskInfo ) ,
2022-04-01 11:33:28 +08:00
busyNodes : make ( map [ int64 ] bool ) ,
2022-03-25 11:03:25 +08:00
pendingLock : sync . RWMutex { } ,
workingLock : sync . RWMutex { } ,
2022-04-01 11:33:28 +08:00
busyNodesLock : sync . RWMutex { } ,
2022-03-22 15:11:24 +08:00
lastReqID : 0 ,
2022-04-24 11:29:45 +08:00
idAllocator : idAlloc ,
2022-03-21 15:47:23 +08:00
callImportService : importService ,
2022-06-14 16:18:09 +08:00
getCollectionName : getCollectionName ,
2022-03-21 15:47:23 +08:00
}
return mgr
}
2022-04-03 11:37:29 +08:00
func ( m * importManager ) init ( ctx context . Context ) {
m . startOnce . Do ( func ( ) {
// Read tasks from Etcd and save them as pending tasks or working tasks.
m . loadFromTaskStore ( )
// Send out tasks to dataCoord.
m . sendOutTasks ( ctx )
} )
2022-03-21 15:47:23 +08:00
}
2022-05-07 14:05:52 +08:00
// sendOutTasksLoop periodically calls `sendOutTasks` to process left over pending tasks.
func ( m * importManager ) sendOutTasksLoop ( wg * sync . WaitGroup ) {
defer wg . Done ( )
ticker := time . NewTicker ( time . Duration ( checkPendingTasksInterval ) * time . Millisecond )
defer ticker . Stop ( )
for {
select {
case <- m . ctx . Done ( ) :
log . Debug ( "import manager context done, exit check sendOutTasksLoop" )
return
case <- ticker . C :
m . sendOutTasks ( m . ctx )
}
}
}
// expireOldTasksLoop starts a loop that checks and expires old tasks every `ImportTaskExpiration` seconds.
2022-06-15 12:20:10 +08:00
func ( m * importManager ) expireOldTasksLoop ( wg * sync . WaitGroup , releaseLockFunc func ( context . Context , [ ] int64 ) error ) {
2022-05-07 14:05:52 +08:00
defer wg . Done ( )
ticker := time . NewTicker ( time . Duration ( expireOldTasksInterval ) * time . Millisecond )
defer ticker . Stop ( )
for {
select {
case <- m . ctx . Done ( ) :
log . Info ( "(in loop) import manager context done, exit expireOldTasksLoop" )
return
case <- ticker . C :
log . Debug ( "(in loop) starting expiring old tasks..." ,
zap . Duration ( "cleaning up interval" , time . Duration ( expireOldTasksInterval ) * time . Millisecond ) )
2022-06-15 12:20:10 +08:00
m . expireOldTasks ( releaseLockFunc )
2022-05-07 14:05:52 +08:00
}
}
}
2022-03-31 13:51:28 +08:00
// sendOutTasks pushes all pending tasks to DataCoord, gets DataCoord response and re-add these tasks as working tasks.
2022-04-01 11:33:28 +08:00
func ( m * importManager ) sendOutTasks ( ctx context . Context ) error {
2022-03-21 15:47:23 +08:00
m . pendingLock . Lock ( )
2022-04-01 11:33:28 +08:00
m . busyNodesLock . Lock ( )
2022-03-21 15:47:23 +08:00
defer m . pendingLock . Unlock ( )
2022-04-01 11:33:28 +08:00
defer m . busyNodesLock . Unlock ( )
2022-03-21 15:47:23 +08:00
2022-03-31 13:51:28 +08:00
// Trigger Import() action to DataCoord.
for len ( m . pendingTasks ) > 0 {
2022-03-21 15:47:23 +08:00
task := m . pendingTasks [ 0 ]
2022-04-12 13:25:34 +08:00
// TODO: Use ImportTaskInfo directly.
2022-03-31 13:51:28 +08:00
it := & datapb . ImportTask {
CollectionId : task . GetCollectionId ( ) ,
PartitionId : task . GetPartitionId ( ) ,
2022-04-12 13:25:34 +08:00
ChannelNames : task . GetChannelNames ( ) ,
2022-03-31 13:51:28 +08:00
RowBased : task . GetRowBased ( ) ,
TaskId : task . GetId ( ) ,
Files : task . GetFiles ( ) ,
2022-03-21 15:47:23 +08:00
Infos : [ ] * commonpb . KeyValuePair {
{
Key : Bucket ,
2022-03-25 11:03:25 +08:00
Value : task . GetBucket ( ) ,
2022-03-21 15:47:23 +08:00
} ,
} ,
}
2022-04-01 11:33:28 +08:00
// Get all busy dataNodes for reference.
var busyNodeList [ ] int64
for k := range m . busyNodes {
busyNodeList = append ( busyNodeList , k )
}
2022-05-07 14:05:52 +08:00
// Send import task to dataCoord, which will then distribute the import task to dataNode.
2022-04-01 11:33:28 +08:00
resp := m . callImportService ( ctx , & datapb . ImportTaskRequest {
ImportTask : it ,
WorkingNodes : busyNodeList ,
} )
2022-05-07 14:05:52 +08:00
if resp . GetStatus ( ) . GetErrorCode ( ) != commonpb . ErrorCode_Success {
log . Warn ( "import task is rejected" ,
zap . Int64 ( "task ID" , it . GetTaskId ( ) ) ,
zap . Any ( "error code" , resp . GetStatus ( ) . GetErrorCode ( ) ) ,
zap . String ( "cause" , resp . GetStatus ( ) . GetReason ( ) ) )
2022-03-21 15:47:23 +08:00
break
}
2022-05-07 14:05:52 +08:00
// Successfully assigned dataNode for the import task. Add task to working task list and update task store.
2022-03-25 11:03:25 +08:00
task . DatanodeId = resp . GetDatanodeId ( )
2022-05-07 14:05:52 +08:00
log . Debug ( "import task successfully assigned to dataNode" ,
2022-03-31 13:51:28 +08:00
zap . Int64 ( "task ID" , it . GetTaskId ( ) ) ,
2022-04-12 13:25:34 +08:00
zap . Int64 ( "dataNode ID" , task . GetDatanodeId ( ) ) )
2022-04-01 11:33:28 +08:00
// Add new working dataNode to busyNodes.
m . busyNodes [ resp . GetDatanodeId ( ) ] = true
2022-03-21 15:47:23 +08:00
func ( ) {
m . workingLock . Lock ( )
defer m . workingLock . Unlock ( )
2022-03-31 13:51:28 +08:00
log . Debug ( "import task added as working task" , zap . Int64 ( "task ID" , it . TaskId ) )
2022-03-25 11:03:25 +08:00
task . State . StateCode = commonpb . ImportState_ImportPending
2022-03-31 13:51:28 +08:00
m . workingTasks [ task . GetId ( ) ] = task
2022-04-03 11:37:29 +08:00
m . updateImportTaskStore ( task )
2022-03-21 15:47:23 +08:00
} ( )
2022-05-07 14:05:52 +08:00
// Erase this task from head of pending list.
m . pendingTasks = append ( m . pendingTasks [ : 0 ] , m . pendingTasks [ 1 : ] ... )
2022-03-21 15:47:23 +08:00
}
return nil
}
2022-03-25 11:03:25 +08:00
// genReqID generates a unique id for import request, this method has no lock, should only be called by importJob()
2022-03-22 15:11:24 +08:00
func ( m * importManager ) genReqID ( ) int64 {
if m . lastReqID == 0 {
m . lastReqID = time . Now ( ) . Unix ( )
} else {
id := time . Now ( ) . Unix ( )
if id == m . lastReqID {
id ++
}
m . lastReqID = id
}
return m . lastReqID
}
2022-03-31 13:51:28 +08:00
// importJob processes the import request, generates import tasks, sends these tasks to DataCoord, and returns
// immediately.
2022-04-20 14:03:40 +08:00
func ( m * importManager ) importJob ( ctx context . Context , req * milvuspb . ImportRequest , cID int64 , pID int64 ) * milvuspb . ImportResponse {
2022-03-21 15:47:23 +08:00
if req == nil || len ( req . Files ) == 0 {
return & milvuspb . ImportResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
Reason : "import request is empty" ,
} ,
}
}
if m . callImportService == nil {
return & milvuspb . ImportResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
Reason : "import service is not available" ,
} ,
}
}
resp := & milvuspb . ImportResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
} ,
2022-04-20 14:03:40 +08:00
Tasks : make ( [ ] int64 , 0 ) ,
2022-03-21 15:47:23 +08:00
}
2022-03-31 13:51:28 +08:00
log . Debug ( "request received" ,
zap . String ( "collection name" , req . GetCollectionName ( ) ) ,
2022-04-20 14:03:40 +08:00
zap . Int64 ( "collection ID" , cID ) ,
zap . Int64 ( "partition ID" , pID ) )
2022-04-24 11:29:45 +08:00
err := func ( ) ( err error ) {
2022-03-21 15:47:23 +08:00
m . pendingLock . Lock ( )
defer m . pendingLock . Unlock ( )
capacity := cap ( m . pendingTasks )
length := len ( m . pendingTasks )
taskCount := 1
if req . RowBased {
taskCount = len ( req . Files )
}
// task queue size has a limit, return error if import request contains too many data files
if capacity - length < taskCount {
resp . Status = & commonpb . Status {
ErrorCode : commonpb . ErrorCode_IllegalArgument ,
Reason : "Import task queue max size is " + strconv . Itoa ( capacity ) + ", currently there are " + strconv . Itoa ( length ) + " tasks is pending. Not able to execute this request with " + strconv . Itoa ( taskCount ) + " tasks." ,
}
return
}
bucket := ""
for _ , kv := range req . Options {
if kv . Key == Bucket {
bucket = kv . Value
break
}
}
2022-03-22 15:11:24 +08:00
reqID := m . genReqID ( )
2022-03-21 15:47:23 +08:00
// convert import request to import tasks
if req . RowBased {
2022-03-31 13:51:28 +08:00
// For row-based importing, each file makes a task.
2022-03-22 15:11:24 +08:00
taskList := make ( [ ] int64 , len ( req . Files ) )
2022-03-21 15:47:23 +08:00
for i := 0 ; i < len ( req . Files ) ; i ++ {
2022-04-24 11:29:45 +08:00
tID , _ , err := m . idAllocator ( 1 )
if err != nil {
return err
}
2022-03-25 11:03:25 +08:00
newTask := & datapb . ImportTaskInfo {
2022-04-24 11:29:45 +08:00
Id : tID ,
2022-03-25 11:03:25 +08:00
RequestId : reqID ,
2022-03-31 13:51:28 +08:00
CollectionId : cID ,
2022-04-20 14:03:40 +08:00
PartitionId : pID ,
2022-04-12 13:25:34 +08:00
ChannelNames : req . ChannelNames ,
2022-03-25 11:03:25 +08:00
Bucket : bucket ,
RowBased : req . GetRowBased ( ) ,
Files : [ ] string { req . GetFiles ( ) [ i ] } ,
CreateTs : time . Now ( ) . Unix ( ) ,
State : & datapb . ImportTaskState {
StateCode : commonpb . ImportState_ImportPending ,
} ,
2022-06-15 12:20:10 +08:00
DataQueryable : false ,
DataIndexed : false ,
2022-03-21 15:47:23 +08:00
}
2022-04-20 14:03:40 +08:00
resp . Tasks = append ( resp . Tasks , newTask . GetId ( ) )
2022-03-25 11:03:25 +08:00
taskList [ i ] = newTask . GetId ( )
2022-03-31 13:51:28 +08:00
log . Info ( "new task created as pending task" , zap . Int64 ( "task ID" , newTask . GetId ( ) ) )
2022-03-21 15:47:23 +08:00
m . pendingTasks = append ( m . pendingTasks , newTask )
2022-04-03 11:37:29 +08:00
m . storeImportTask ( newTask )
2022-03-21 15:47:23 +08:00
}
2022-03-31 13:51:28 +08:00
log . Info ( "row-based import request processed" , zap . Int64 ( "reqID" , reqID ) , zap . Any ( "taskIDs" , taskList ) )
2022-03-21 15:47:23 +08:00
} else {
2022-03-31 13:51:28 +08:00
// TODO: Merge duplicated code :(
2022-03-21 15:47:23 +08:00
// for column-based, all files is a task
2022-04-24 11:29:45 +08:00
tID , _ , err := m . idAllocator ( 1 )
if err != nil {
return err
}
2022-03-25 11:03:25 +08:00
newTask := & datapb . ImportTaskInfo {
2022-04-24 11:29:45 +08:00
Id : tID ,
2022-03-25 11:03:25 +08:00
RequestId : reqID ,
2022-03-31 13:51:28 +08:00
CollectionId : cID ,
2022-04-20 14:03:40 +08:00
PartitionId : pID ,
2022-04-12 13:25:34 +08:00
ChannelNames : req . ChannelNames ,
2022-03-25 11:03:25 +08:00
Bucket : bucket ,
RowBased : req . GetRowBased ( ) ,
Files : req . GetFiles ( ) ,
CreateTs : time . Now ( ) . Unix ( ) ,
State : & datapb . ImportTaskState {
StateCode : commonpb . ImportState_ImportPending ,
} ,
2022-06-15 12:20:10 +08:00
DataQueryable : false ,
DataIndexed : false ,
2022-03-21 15:47:23 +08:00
}
2022-04-20 14:03:40 +08:00
resp . Tasks = append ( resp . Tasks , newTask . GetId ( ) )
2022-03-31 13:51:28 +08:00
log . Info ( "new task created as pending task" , zap . Int64 ( "task ID" , newTask . GetId ( ) ) )
2022-03-21 15:47:23 +08:00
m . pendingTasks = append ( m . pendingTasks , newTask )
2022-04-03 11:37:29 +08:00
m . storeImportTask ( newTask )
2022-03-31 13:51:28 +08:00
log . Info ( "column-based import request processed" , zap . Int64 ( "reqID" , reqID ) , zap . Int64 ( "taskID" , newTask . GetId ( ) ) )
2022-03-21 15:47:23 +08:00
}
2022-04-24 11:29:45 +08:00
return nil
2022-03-21 15:47:23 +08:00
} ( )
2022-04-24 11:29:45 +08:00
if err != nil {
return & milvuspb . ImportResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
Reason : err . Error ( ) ,
} ,
}
}
2022-04-01 11:33:28 +08:00
m . sendOutTasks ( ctx )
2022-03-21 15:47:23 +08:00
return resp
}
2022-05-12 19:23:53 +08:00
// setTaskDataQueryable sets task's DataQueryable flag to true.
func ( m * importManager ) setTaskDataQueryable ( taskID int64 ) {
2022-05-05 21:17:50 +08:00
m . workingLock . Lock ( )
defer m . workingLock . Unlock ( )
if v , ok := m . workingTasks [ taskID ] ; ok {
2022-06-15 12:20:10 +08:00
v . DataQueryable = true
2022-05-12 19:23:53 +08:00
} else {
log . Error ( "task ID not found" , zap . Int64 ( "task ID" , taskID ) )
}
}
// setTaskDataIndexed sets task's DataIndexed flag to true.
func ( m * importManager ) setTaskDataIndexed ( taskID int64 ) {
m . workingLock . Lock ( )
defer m . workingLock . Unlock ( )
if v , ok := m . workingTasks [ taskID ] ; ok {
2022-06-15 12:20:10 +08:00
v . DataIndexed = true
2022-05-05 21:17:50 +08:00
} else {
log . Error ( "task ID not found" , zap . Int64 ( "task ID" , taskID ) )
}
}
2022-04-03 11:37:29 +08:00
// updateTaskState updates the task's state in in-memory working tasks list and in task store, given ImportResult
// result. It returns the ImportTaskInfo of the given task.
2022-03-31 13:51:28 +08:00
func ( m * importManager ) updateTaskState ( ir * rootcoordpb . ImportResult ) ( * datapb . ImportTaskInfo , error ) {
if ir == nil {
return nil , errors . New ( "import result is nil" )
2022-03-21 15:47:23 +08:00
}
2022-03-31 13:51:28 +08:00
log . Debug ( "import manager update task import result" , zap . Int64 ( "taskID" , ir . GetTaskId ( ) ) )
2022-03-21 15:47:23 +08:00
found := false
2022-03-31 13:51:28 +08:00
var v * datapb . ImportTaskInfo
2022-04-03 11:37:29 +08:00
m . workingLock . Lock ( )
defer m . workingLock . Unlock ( )
ok := false
2022-05-05 21:17:50 +08:00
if v , ok = m . workingTasks [ ir . GetTaskId ( ) ] ; ok {
2022-04-03 11:37:29 +08:00
// If the task has already been marked failed. Prevent further state updating and return an error.
if v . GetState ( ) . GetStateCode ( ) == commonpb . ImportState_ImportFailed {
log . Warn ( "trying to update an already failed task which will end up being a no-op" )
return nil , errors . New ( "trying to update an already failed task " + strconv . FormatInt ( ir . GetTaskId ( ) , 10 ) )
}
found = true
v . State . StateCode = ir . GetState ( )
v . State . Segments = ir . GetSegments ( )
v . State . RowCount = ir . GetRowCount ( )
2022-04-21 21:37:42 +08:00
v . State . RowIds = ir . AutoIds
2022-04-03 11:37:29 +08:00
for _ , kv := range ir . GetInfos ( ) {
if kv . GetKey ( ) == FailedReason {
v . State . ErrorMessage = kv . GetValue ( )
break
2022-03-21 15:47:23 +08:00
}
}
2022-04-03 11:37:29 +08:00
// Update task in task store.
m . updateImportTaskStore ( v )
}
2022-03-21 15:47:23 +08:00
if ! found {
2022-04-24 11:29:45 +08:00
log . Debug ( "import manager update task import result failed" , zap . Int64 ( "task ID" , ir . GetTaskId ( ) ) )
2022-03-31 13:51:28 +08:00
return nil , errors . New ( "failed to update import task, ID not found: " + strconv . FormatInt ( ir . TaskId , 10 ) )
2022-03-21 15:47:23 +08:00
}
2022-03-31 13:51:28 +08:00
return v , nil
2022-03-21 15:47:23 +08:00
}
2022-06-14 16:18:09 +08:00
func ( m * importManager ) getCollectionPartitionName ( task * datapb . ImportTaskInfo , resp * milvuspb . GetImportStateResponse ) {
if m . getCollectionName != nil {
colName , partName , err := m . getCollectionName ( task . GetCollectionId ( ) , task . GetPartitionId ( ) )
if err == nil {
resp . Infos = append ( resp . Infos , & commonpb . KeyValuePair { Key : CollectionName , Value : colName } )
resp . Infos = append ( resp . Infos , & commonpb . KeyValuePair { Key : PartitionName , Value : partName } )
}
}
}
2022-06-15 12:20:10 +08:00
// appendTaskSegments updates the task's segment lists by adding `segIDs` to it.
func ( m * importManager ) appendTaskSegments ( taskID int64 , segIDs [ ] int64 ) error {
log . Debug ( "import manager appending task segments" ,
zap . Int64 ( "task ID" , taskID ) ,
zap . Int64s ( "segment ID" , segIDs ) )
var v * datapb . ImportTaskInfo
m . workingLock . Lock ( )
ok := false
if v , ok = m . workingTasks [ taskID ] ; ok {
v . State . Segments = append ( v . GetState ( ) . GetSegments ( ) , segIDs ... )
// Update task in task store.
m . updateImportTaskStore ( v )
}
m . workingLock . Unlock ( )
if ! ok {
log . Debug ( "import manager appending task segments failed" , zap . Int64 ( "task ID" , taskID ) )
return errors . New ( "failed to update import task, ID not found: " + strconv . FormatInt ( taskID , 10 ) )
}
return nil
}
2022-03-31 13:51:28 +08:00
// getTaskState looks for task with the given ID and returns its import state.
func ( m * importManager ) getTaskState ( tID int64 ) * milvuspb . GetImportStateResponse {
2022-03-21 15:47:23 +08:00
resp := & milvuspb . GetImportStateResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_UnexpectedError ,
Reason : "import task id doesn't exist" ,
} ,
2022-04-20 14:03:40 +08:00
Infos : make ( [ ] * commonpb . KeyValuePair , 0 ) ,
2022-03-21 15:47:23 +08:00
}
2022-03-31 13:51:28 +08:00
log . Debug ( "getting import task state" , zap . Int64 ( "taskID" , tID ) )
2022-03-21 15:47:23 +08:00
found := false
func ( ) {
m . pendingLock . Lock ( )
defer m . pendingLock . Unlock ( )
2022-04-28 17:21:47 +08:00
for _ , t := range m . pendingTasks {
if tID == t . Id {
2022-03-21 15:47:23 +08:00
resp . Status = & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
}
2022-04-28 17:21:47 +08:00
resp . Id = tID
2022-03-21 15:47:23 +08:00
resp . State = commonpb . ImportState_ImportPending
2022-04-28 17:21:47 +08:00
resp . Infos = append ( resp . Infos , & commonpb . KeyValuePair { Key : Files , Value : strings . Join ( t . GetFiles ( ) , "," ) } )
2022-06-15 12:20:10 +08:00
resp . DataQueryable = t . GetDataQueryable ( )
resp . DataIndexed = t . GetDataIndexed ( )
2022-06-14 16:18:09 +08:00
m . getCollectionPartitionName ( t , resp )
2022-03-21 15:47:23 +08:00
found = true
break
}
}
} ( )
if found {
return resp
}
func ( ) {
m . workingLock . Lock ( )
defer m . workingLock . Unlock ( )
2022-03-31 13:51:28 +08:00
if v , ok := m . workingTasks [ tID ] ; ok {
found = true
resp . Status = & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
2022-03-21 15:47:23 +08:00
}
2022-04-28 17:21:47 +08:00
resp . Id = tID
2022-03-31 13:51:28 +08:00
resp . State = v . GetState ( ) . GetStateCode ( )
resp . RowCount = v . GetState ( ) . GetRowCount ( )
resp . IdList = v . GetState ( ) . GetRowIds ( )
2022-04-20 14:03:40 +08:00
resp . Infos = append ( resp . Infos , & commonpb . KeyValuePair { Key : Files , Value : strings . Join ( v . GetFiles ( ) , "," ) } )
2022-03-31 13:51:28 +08:00
resp . Infos = append ( resp . Infos , & commonpb . KeyValuePair {
Key : FailedReason ,
Value : v . GetState ( ) . GetErrorMessage ( ) ,
} )
2022-06-15 12:20:10 +08:00
resp . DataQueryable = v . GetDataQueryable ( )
resp . DataIndexed = v . GetDataIndexed ( )
2022-06-14 16:18:09 +08:00
m . getCollectionPartitionName ( v , resp )
2022-03-21 15:47:23 +08:00
}
} ( )
2022-03-31 13:51:28 +08:00
if found {
return resp
2022-03-21 15:47:23 +08:00
}
2022-03-31 13:51:28 +08:00
log . Debug ( "get import task state failed" , zap . Int64 ( "taskID" , tID ) )
2022-03-21 15:47:23 +08:00
return resp
}
2022-03-25 11:03:25 +08:00
2022-04-03 11:37:29 +08:00
// loadFromTaskStore loads task info from task store when RootCoord (re)starts.
func ( m * importManager ) loadFromTaskStore ( ) error {
2022-03-31 13:51:28 +08:00
log . Info ( "import manager starts loading from Etcd" )
2022-03-25 11:03:25 +08:00
_ , v , err := m . taskStore . LoadWithPrefix ( Params . RootCoordCfg . ImportTaskSubPath )
if err != nil {
2022-03-31 13:51:28 +08:00
log . Error ( "import manager failed to load from Etcd" , zap . Error ( err ) )
2022-03-25 11:03:25 +08:00
return err
}
m . workingLock . Lock ( )
defer m . workingLock . Unlock ( )
m . pendingLock . Lock ( )
defer m . pendingLock . Unlock ( )
for i := range v {
ti := & datapb . ImportTaskInfo { }
if err := proto . Unmarshal ( [ ] byte ( v [ i ] ) , ti ) ; err != nil {
2022-03-31 13:51:28 +08:00
log . Error ( "failed to unmarshal proto" , zap . String ( "taskInfo" , v [ i ] ) , zap . Error ( err ) )
2022-03-25 11:03:25 +08:00
// Ignore bad protos.
continue
}
// Put tasks back to pending or working task list, given their import states.
if ti . GetState ( ) . GetStateCode ( ) == commonpb . ImportState_ImportPending {
2022-04-24 11:29:45 +08:00
log . Info ( "task has been reloaded as a pending task" , zap . Int64 ( "task ID" , ti . GetId ( ) ) )
2022-03-25 11:03:25 +08:00
m . pendingTasks = append ( m . pendingTasks , ti )
} else {
2022-04-24 11:29:45 +08:00
log . Info ( "task has been reloaded as a working tasks" , zap . Int64 ( "task ID" , ti . GetId ( ) ) )
2022-03-31 13:51:28 +08:00
m . workingTasks [ ti . GetId ( ) ] = ti
2022-03-25 11:03:25 +08:00
}
}
return nil
}
2022-04-03 11:37:29 +08:00
// storeImportTask signs a lease and saves import task info into Etcd with this lease.
func ( m * importManager ) storeImportTask ( task * datapb . ImportTaskInfo ) error {
log . Debug ( "saving import task to Etcd" , zap . Int64 ( "task ID" , task . GetId ( ) ) )
// Sign a lease. Tasks will be stored for at least `ImportTaskRetention` seconds.
leaseID , err := m . taskStore . Grant ( int64 ( Params . RootCoordCfg . ImportTaskRetention ) )
2022-03-25 11:03:25 +08:00
if err != nil {
2022-03-31 13:51:28 +08:00
log . Error ( "failed to grant lease from Etcd for data import" ,
2022-04-03 11:37:29 +08:00
zap . Int64 ( "task ID" , task . GetId ( ) ) ,
2022-03-31 13:51:28 +08:00
zap . Error ( err ) )
2022-03-25 11:03:25 +08:00
return err
}
2022-04-03 11:37:29 +08:00
log . Debug ( "lease granted for task" , zap . Int64 ( "task ID" , task . GetId ( ) ) )
2022-03-25 11:03:25 +08:00
var taskInfo [ ] byte
if taskInfo , err = proto . Marshal ( task ) ; err != nil {
2022-04-03 11:37:29 +08:00
log . Error ( "failed to marshall task proto" , zap . Int64 ( "task ID" , task . GetId ( ) ) , zap . Error ( err ) )
2022-03-25 11:03:25 +08:00
return err
2022-03-31 13:51:28 +08:00
} else if err = m . taskStore . SaveWithLease ( BuildImportTaskKey ( task . GetId ( ) ) , string ( taskInfo ) , leaseID ) ; err != nil {
log . Error ( "failed to save import task info into Etcd" ,
zap . Int64 ( "task ID" , task . GetId ( ) ) ,
zap . Error ( err ) )
2022-03-25 11:03:25 +08:00
return err
}
2022-04-03 11:37:29 +08:00
log . Debug ( "task info successfully saved" , zap . Int64 ( "task ID" , task . GetId ( ) ) )
2022-03-25 11:03:25 +08:00
return nil
}
2022-04-03 11:37:29 +08:00
// updateImportTaskStore updates the task info in Etcd according to task ID. It won't change the lease on the key.
func ( m * importManager ) updateImportTaskStore ( ti * datapb . ImportTaskInfo ) error {
2022-03-31 13:51:28 +08:00
log . Debug ( "updating import task info in Etcd" , zap . Int64 ( "Task ID" , ti . GetId ( ) ) )
if taskInfo , err := proto . Marshal ( ti ) ; err != nil {
log . Error ( "failed to marshall task info proto" , zap . Int64 ( "Task ID" , ti . GetId ( ) ) , zap . Error ( err ) )
2022-03-25 11:03:25 +08:00
return err
2022-03-31 13:51:28 +08:00
} else if err = m . taskStore . SaveWithIgnoreLease ( BuildImportTaskKey ( ti . GetId ( ) ) , string ( taskInfo ) ) ; err != nil {
log . Error ( "failed to update import task info info in Etcd" , zap . Int64 ( "Task ID" , ti . GetId ( ) ) , zap . Error ( err ) )
2022-03-25 11:03:25 +08:00
return err
}
2022-03-31 13:51:28 +08:00
log . Debug ( "task info successfully updated in Etcd" , zap . Int64 ( "Task ID" , ti . GetId ( ) ) )
2022-03-25 11:03:25 +08:00
return nil
}
2022-04-03 11:37:29 +08:00
// expireOldTasks marks expires tasks as failed.
2022-06-15 12:20:10 +08:00
func ( m * importManager ) expireOldTasks ( releaseLockFunc func ( context . Context , [ ] int64 ) error ) {
2022-04-03 11:37:29 +08:00
// Expire old pending tasks, if any.
func ( ) {
m . pendingLock . Lock ( )
defer m . pendingLock . Unlock ( )
for _ , t := range m . pendingTasks {
if taskExpired ( t ) {
2022-06-15 12:20:10 +08:00
// Mark this expired task as failed.
2022-04-06 15:33:32 +08:00
log . Info ( "a pending task has expired" , zap . Int64 ( "task ID" , t . GetId ( ) ) )
2022-04-03 11:37:29 +08:00
t . State . StateCode = commonpb . ImportState_ImportFailed
t . State . ErrorMessage = taskExpiredMsgPrefix +
( time . Duration ( Params . RootCoordCfg . ImportTaskExpiration * 1000 ) * time . Millisecond ) . String ( )
2022-06-15 12:20:10 +08:00
log . Info ( "releasing seg ref locks on expired import task" ,
zap . Int64s ( "segment IDs" , t . GetState ( ) . GetSegments ( ) ) )
err := retry . Do ( m . ctx , func ( ) error {
return releaseLockFunc ( m . ctx , t . GetState ( ) . GetSegments ( ) )
} , retry . Attempts ( 100 ) )
if err != nil {
log . Error ( "failed to release lock, about to panic!" )
panic ( err )
}
2022-04-03 11:37:29 +08:00
m . updateImportTaskStore ( t )
}
}
} ( )
// Expire old working tasks.
func ( ) {
m . workingLock . Lock ( )
defer m . workingLock . Unlock ( )
for _ , v := range m . workingTasks {
if taskExpired ( v ) {
2022-06-15 12:20:10 +08:00
// Mark this expired task as failed.
2022-04-06 15:33:32 +08:00
log . Info ( "a working task has expired" , zap . Int64 ( "task ID" , v . GetId ( ) ) )
2022-04-03 11:37:29 +08:00
v . State . StateCode = commonpb . ImportState_ImportFailed
v . State . ErrorMessage = taskExpiredMsgPrefix +
( time . Duration ( Params . RootCoordCfg . ImportTaskExpiration * 1000 ) * time . Millisecond ) . String ( )
2022-06-15 12:20:10 +08:00
log . Info ( "releasing seg ref locks on expired import task" ,
zap . Int64s ( "segment IDs" , v . GetState ( ) . GetSegments ( ) ) )
err := retry . Do ( m . ctx , func ( ) error {
return releaseLockFunc ( m . ctx , v . GetState ( ) . GetSegments ( ) )
} , retry . Attempts ( 100 ) )
if err != nil {
log . Error ( "failed to release lock, about to panic!" )
panic ( err )
}
2022-04-03 11:37:29 +08:00
m . updateImportTaskStore ( v )
}
}
} ( )
}
2022-06-14 16:18:09 +08:00
func rearrangeTasks ( tasks [ ] * milvuspb . GetImportStateResponse ) {
sort . Slice ( tasks , func ( i , j int ) bool {
return tasks [ i ] . GetId ( ) < tasks [ j ] . GetId ( )
} )
}
2022-04-28 17:21:47 +08:00
func ( m * importManager ) listAllTasks ( ) [ ] * milvuspb . GetImportStateResponse {
tasks := make ( [ ] * milvuspb . GetImportStateResponse , 0 )
2022-04-25 17:37:46 +08:00
func ( ) {
m . pendingLock . Lock ( )
defer m . pendingLock . Unlock ( )
for _ , t := range m . pendingTasks {
2022-04-28 17:21:47 +08:00
resp := & milvuspb . GetImportStateResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
} ,
2022-06-15 12:20:10 +08:00
Infos : make ( [ ] * commonpb . KeyValuePair , 0 ) ,
Id : t . GetId ( ) ,
State : commonpb . ImportState_ImportPending ,
DataQueryable : t . GetDataQueryable ( ) ,
DataIndexed : t . GetDataIndexed ( ) ,
2022-04-28 17:21:47 +08:00
}
resp . Infos = append ( resp . Infos , & commonpb . KeyValuePair { Key : Files , Value : strings . Join ( t . GetFiles ( ) , "," ) } )
2022-06-14 16:18:09 +08:00
m . getCollectionPartitionName ( t , resp )
2022-04-28 17:21:47 +08:00
tasks = append ( tasks , resp )
2022-04-25 17:37:46 +08:00
}
2022-04-28 17:21:47 +08:00
log . Info ( "tasks in pending list" , zap . Int ( "count" , len ( m . pendingTasks ) ) )
2022-04-25 17:37:46 +08:00
} ( )
func ( ) {
m . workingLock . Lock ( )
defer m . workingLock . Unlock ( )
for _ , v := range m . workingTasks {
2022-04-28 17:21:47 +08:00
resp := & milvuspb . GetImportStateResponse {
Status : & commonpb . Status {
ErrorCode : commonpb . ErrorCode_Success ,
} ,
2022-06-15 12:20:10 +08:00
Infos : make ( [ ] * commonpb . KeyValuePair , 0 ) ,
Id : v . GetId ( ) ,
State : v . GetState ( ) . GetStateCode ( ) ,
RowCount : v . GetState ( ) . GetRowCount ( ) ,
IdList : v . GetState ( ) . GetRowIds ( ) ,
DataQueryable : v . GetDataQueryable ( ) ,
DataIndexed : v . GetDataIndexed ( ) ,
2022-04-28 17:21:47 +08:00
}
resp . Infos = append ( resp . Infos , & commonpb . KeyValuePair { Key : Files , Value : strings . Join ( v . GetFiles ( ) , "," ) } )
resp . Infos = append ( resp . Infos , & commonpb . KeyValuePair {
Key : FailedReason ,
Value : v . GetState ( ) . GetErrorMessage ( ) ,
} )
2022-06-14 16:18:09 +08:00
m . getCollectionPartitionName ( v , resp )
2022-04-28 17:21:47 +08:00
tasks = append ( tasks , resp )
2022-04-25 17:37:46 +08:00
}
2022-04-28 17:21:47 +08:00
log . Info ( "tasks in working list" , zap . Int ( "count" , len ( m . workingTasks ) ) )
2022-04-25 17:37:46 +08:00
} ( )
2022-06-14 16:18:09 +08:00
rearrangeTasks ( tasks )
2022-04-25 17:37:46 +08:00
return tasks
}
2022-03-25 11:03:25 +08:00
// BuildImportTaskKey constructs and returns an Etcd key with given task ID.
func BuildImportTaskKey ( taskID int64 ) string {
return fmt . Sprintf ( "%s%s%d" , Params . RootCoordCfg . ImportTaskSubPath , delimiter , taskID )
}
2022-04-03 11:37:29 +08:00
2022-06-15 12:20:10 +08:00
// taskExpired returns true if the task is considered expired.
2022-04-03 11:37:29 +08:00
func taskExpired ( ti * datapb . ImportTaskInfo ) bool {
2022-06-15 12:20:10 +08:00
return ti . GetState ( ) . GetStateCode ( ) != commonpb . ImportState_ImportFailed &&
ti . GetState ( ) . GetStateCode ( ) != commonpb . ImportState_ImportPersisted &&
ti . GetState ( ) . GetStateCode ( ) != commonpb . ImportState_ImportCompleted &&
Params . RootCoordCfg . ImportTaskExpiration <= float64 ( time . Now ( ) . Unix ( ) - ti . GetCreateTs ( ) )
2022-04-03 11:37:29 +08:00
}