mirror of https://github.com/milvus-io/milvus.git
Make each dataNode only accept one bulk load task simultaneously (#16820)
issue: #16674 /kind improvement Signed-off-by: Yuchen Gao <yuchen.gao@zilliz.com>pull/16836/head
parent
92b6293be4
commit
6b2d3c3ec1
|
@ -2412,7 +2412,7 @@ func TestImport(t *testing.T) {
|
||||||
WorkingNodes: []int64{0},
|
WorkingNodes: []int64{0},
|
||||||
})
|
})
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.Status.GetErrorCode())
|
assert.EqualValues(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode())
|
||||||
etcd.StopEtcdServer()
|
etcd.StopEtcdServer()
|
||||||
})
|
})
|
||||||
|
|
||||||
|
|
|
@ -998,9 +998,10 @@ func (s *Server) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateR
|
||||||
return resp, nil
|
return resp, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Import data files(json, numpy, etc.) on MinIO/S3 storage, read and parse them into sealed segments
|
// Import distributes the import tasks to dataNodes.
|
||||||
|
// It returns a failed status if no dataNode is available or if any error occurs.
|
||||||
func (s *Server) Import(ctx context.Context, itr *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
func (s *Server) Import(ctx context.Context, itr *datapb.ImportTaskRequest) (*datapb.ImportTaskResponse, error) {
|
||||||
log.Info("receive import request", zap.Any("import task request", itr))
|
log.Info("received import request", zap.Any("import task request", itr))
|
||||||
resp := &datapb.ImportTaskResponse{
|
resp := &datapb.ImportTaskResponse{
|
||||||
Status: &commonpb.Status{
|
Status: &commonpb.Status{
|
||||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||||
|
@ -1008,18 +1009,17 @@ func (s *Server) Import(ctx context.Context, itr *datapb.ImportTaskRequest) (*da
|
||||||
}
|
}
|
||||||
|
|
||||||
if s.isClosed() {
|
if s.isClosed() {
|
||||||
log.Warn("failed to import because of closed server", zap.Any("import task request", itr))
|
log.Error("failed to import for closed dataCoord service")
|
||||||
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.GetNodeID())
|
resp.Status.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.GetNodeID())
|
||||||
return resp, nil
|
return resp, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
workingNodes := itr.WorkingNodes
|
|
||||||
nodes := s.channelManager.store.GetNodes()
|
nodes := s.channelManager.store.GetNodes()
|
||||||
if len(nodes) == 0 {
|
if len(nodes) == 0 {
|
||||||
log.Error("import failed as all dataNodes are offline", zap.Any("import task request", itr))
|
log.Error("import failed as all dataNodes are offline")
|
||||||
return resp, nil
|
return resp, nil
|
||||||
}
|
}
|
||||||
avaNodes := getDiff(nodes, workingNodes)
|
avaNodes := getDiff(nodes, itr.GetWorkingNodes())
|
||||||
if len(avaNodes) > 0 {
|
if len(avaNodes) > 0 {
|
||||||
// If there exists available DataNodes, pick one at random.
|
// If there exists available DataNodes, pick one at random.
|
||||||
dnID := avaNodes[rand.Intn(len(avaNodes))]
|
dnID := avaNodes[rand.Intn(len(avaNodes))]
|
||||||
|
@ -1028,12 +1028,11 @@ func (s *Server) Import(ctx context.Context, itr *datapb.ImportTaskRequest) (*da
|
||||||
zap.Int64("picking free dataNode with ID", dnID))
|
zap.Int64("picking free dataNode with ID", dnID))
|
||||||
s.cluster.Import(s.ctx, dnID, itr)
|
s.cluster.Import(s.ctx, dnID, itr)
|
||||||
} else {
|
} else {
|
||||||
// No DataNodes are available, choose a still working DataNode randomly.
|
// No dataNode is available, reject the import request.
|
||||||
dnID := nodes[rand.Intn(len(nodes))]
|
errMsg := "all dataNodes are busy working on data import, please try again later or add new dataNode instances"
|
||||||
log.Info("all dataNodes are busy, picking a random dataNode still",
|
log.Error(errMsg, zap.Int64("task ID", itr.GetImportTask().GetTaskId()))
|
||||||
zap.Any("all dataNodes", nodes),
|
resp.Status.Reason = errMsg
|
||||||
zap.Int64("picking dataNode with ID", dnID))
|
return resp, nil
|
||||||
s.cluster.Import(s.ctx, dnID, itr)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
resp.Status.ErrorCode = commonpb.ErrorCode_Success
|
resp.Status.ErrorCode = commonpb.ErrorCode_Success
|
||||||
|
|
|
@ -45,6 +45,14 @@ const (
|
||||||
taskExpiredMsgPrefix = "task has expired after "
|
taskExpiredMsgPrefix = "task has expired after "
|
||||||
)
|
)
|
||||||
|
|
||||||
|
// 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.
|
||||||
|
// default 10*60*1000 milliseconds (10 minutes)
|
||||||
|
var expireOldTasksInterval = 10 * 60 * 1000
|
||||||
|
|
||||||
// import task state
|
// import task state
|
||||||
type importTaskState struct {
|
type importTaskState struct {
|
||||||
stateCode commonpb.ImportState // state code
|
stateCode commonpb.ImportState // state code
|
||||||
|
@ -103,6 +111,41 @@ func (m *importManager) init(ctx context.Context) {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// 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:
|
||||||
|
log.Debug("sending out tasks")
|
||||||
|
m.sendOutTasks(m.ctx)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// expireOldTasksLoop starts a loop that checks and expires old tasks every `ImportTaskExpiration` seconds.
|
||||||
|
func (m *importManager) expireOldTasksLoop(wg *sync.WaitGroup) {
|
||||||
|
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))
|
||||||
|
m.expireOldTasks()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// sendOutTasks pushes all pending tasks to DataCoord, gets DataCoord response and re-add these tasks as working tasks.
|
// sendOutTasks pushes all pending tasks to DataCoord, gets DataCoord response and re-add these tasks as working tasks.
|
||||||
func (m *importManager) sendOutTasks(ctx context.Context) error {
|
func (m *importManager) sendOutTasks(ctx context.Context) error {
|
||||||
m.pendingLock.Lock()
|
m.pendingLock.Lock()
|
||||||
|
@ -113,10 +156,6 @@ func (m *importManager) sendOutTasks(ctx context.Context) error {
|
||||||
// Trigger Import() action to DataCoord.
|
// Trigger Import() action to DataCoord.
|
||||||
for len(m.pendingTasks) > 0 {
|
for len(m.pendingTasks) > 0 {
|
||||||
task := m.pendingTasks[0]
|
task := m.pendingTasks[0]
|
||||||
// Skip failed (mostly like expired) tasks.
|
|
||||||
if task.GetState().GetStateCode() == commonpb.ImportState_ImportFailed {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
// TODO: Use ImportTaskInfo directly.
|
// TODO: Use ImportTaskInfo directly.
|
||||||
it := &datapb.ImportTask{
|
it := &datapb.ImportTask{
|
||||||
CollectionId: task.GetCollectionId(),
|
CollectionId: task.GetCollectionId(),
|
||||||
|
@ -133,32 +172,33 @@ func (m *importManager) sendOutTasks(ctx context.Context) error {
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
log.Debug("sending import task to DataCoord", zap.Int64("taskID", task.GetId()))
|
|
||||||
// Get all busy dataNodes for reference.
|
// Get all busy dataNodes for reference.
|
||||||
var busyNodeList []int64
|
var busyNodeList []int64
|
||||||
for k := range m.busyNodes {
|
for k := range m.busyNodes {
|
||||||
busyNodeList = append(busyNodeList, k)
|
busyNodeList = append(busyNodeList, k)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Call DataCoord.Import().
|
// Send import task to dataCoord, which will then distribute the import task to dataNode.
|
||||||
resp := m.callImportService(ctx, &datapb.ImportTaskRequest{
|
resp := m.callImportService(ctx, &datapb.ImportTaskRequest{
|
||||||
ImportTask: it,
|
ImportTask: it,
|
||||||
WorkingNodes: busyNodeList,
|
WorkingNodes: busyNodeList,
|
||||||
})
|
})
|
||||||
if resp.Status.ErrorCode == commonpb.ErrorCode_UnexpectedError {
|
if resp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
||||||
log.Debug("import task is rejected", zap.Int64("task ID", it.GetTaskId()))
|
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()))
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Successfully assigned dataNode for the import task. Add task to working task list and update task store.
|
||||||
task.DatanodeId = resp.GetDatanodeId()
|
task.DatanodeId = resp.GetDatanodeId()
|
||||||
log.Debug("import task successfully assigned to DataNode",
|
log.Debug("import task successfully assigned to dataNode",
|
||||||
zap.Int64("task ID", it.GetTaskId()),
|
zap.Int64("task ID", it.GetTaskId()),
|
||||||
zap.Int64("dataNode ID", task.GetDatanodeId()))
|
zap.Int64("dataNode ID", task.GetDatanodeId()))
|
||||||
// Add new working dataNode to busyNodes.
|
// Add new working dataNode to busyNodes.
|
||||||
m.busyNodes[resp.GetDatanodeId()] = true
|
m.busyNodes[resp.GetDatanodeId()] = true
|
||||||
|
|
||||||
// erase this task from head of pending list if the callImportService succeed
|
|
||||||
m.pendingTasks = append(m.pendingTasks[:0], m.pendingTasks[1:]...)
|
|
||||||
|
|
||||||
func() {
|
func() {
|
||||||
m.workingLock.Lock()
|
m.workingLock.Lock()
|
||||||
defer m.workingLock.Unlock()
|
defer m.workingLock.Unlock()
|
||||||
|
@ -168,6 +208,9 @@ func (m *importManager) sendOutTasks(ctx context.Context) error {
|
||||||
m.workingTasks[task.GetId()] = task
|
m.workingTasks[task.GetId()] = task
|
||||||
m.updateImportTaskStore(task)
|
m.updateImportTaskStore(task)
|
||||||
}()
|
}()
|
||||||
|
|
||||||
|
// Erase this task from head of pending list.
|
||||||
|
m.pendingTasks = append(m.pendingTasks[:0], m.pendingTasks[1:]...)
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
|
@ -503,25 +546,6 @@ func (m *importManager) updateImportTaskStore(ti *datapb.ImportTaskInfo) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// expireOldTasksLoop starts a loop that checks and expires old tasks every `ImportTaskExpiration` seconds.
|
|
||||||
func (m *importManager) expireOldTasksLoop(wg *sync.WaitGroup) {
|
|
||||||
defer wg.Done()
|
|
||||||
ticker := time.NewTicker(time.Duration(Params.RootCoordCfg.ImportTaskExpiration*1000) * 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.Info("(in loop) starting expiring old tasks...",
|
|
||||||
zap.Duration("cleaning up interval",
|
|
||||||
time.Duration(Params.RootCoordCfg.ImportTaskExpiration*1000)*time.Millisecond))
|
|
||||||
m.expireOldTasks()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// expireOldTasks marks expires tasks as failed.
|
// expireOldTasks marks expires tasks as failed.
|
||||||
func (m *importManager) expireOldTasks() {
|
func (m *importManager) expireOldTasks() {
|
||||||
// Expire old pending tasks, if any.
|
// Expire old pending tasks, if any.
|
||||||
|
|
|
@ -48,7 +48,9 @@ func TestImportManager_NewImportManager(t *testing.T) {
|
||||||
return globalCount, 0, nil
|
return globalCount, 0, nil
|
||||||
}
|
}
|
||||||
Params.RootCoordCfg.ImportTaskSubPath = "test_import_task"
|
Params.RootCoordCfg.ImportTaskSubPath = "test_import_task"
|
||||||
Params.RootCoordCfg.ImportTaskExpiration = 1
|
Params.RootCoordCfg.ImportTaskExpiration = 100
|
||||||
|
checkPendingTasksInterval = 100
|
||||||
|
expireOldTasksInterval = 100
|
||||||
mockKv := &kv.MockMetaKV{}
|
mockKv := &kv.MockMetaKV{}
|
||||||
mockKv.InMemKv = make(map[string]string)
|
mockKv.InMemKv = make(map[string]string)
|
||||||
ti1 := &datapb.ImportTaskInfo{
|
ti1 := &datapb.ImportTaskInfo{
|
||||||
|
@ -90,8 +92,9 @@ func TestImportManager_NewImportManager(t *testing.T) {
|
||||||
assert.NotNil(t, mgr)
|
assert.NotNil(t, mgr)
|
||||||
mgr.init(ctx)
|
mgr.init(ctx)
|
||||||
var wgLoop sync.WaitGroup
|
var wgLoop sync.WaitGroup
|
||||||
wgLoop.Add(1)
|
wgLoop.Add(2)
|
||||||
mgr.expireOldTasksLoop(&wgLoop)
|
mgr.expireOldTasksLoop(&wgLoop)
|
||||||
|
mgr.sendOutTasksLoop(&wgLoop)
|
||||||
wgLoop.Wait()
|
wgLoop.Wait()
|
||||||
})
|
})
|
||||||
|
|
||||||
|
@ -104,8 +107,9 @@ func TestImportManager_NewImportManager(t *testing.T) {
|
||||||
assert.NotNil(t, mgr)
|
assert.NotNil(t, mgr)
|
||||||
mgr.init(context.TODO())
|
mgr.init(context.TODO())
|
||||||
var wgLoop sync.WaitGroup
|
var wgLoop sync.WaitGroup
|
||||||
wgLoop.Add(1)
|
wgLoop.Add(2)
|
||||||
mgr.expireOldTasksLoop(&wgLoop)
|
mgr.expireOldTasksLoop(&wgLoop)
|
||||||
|
mgr.sendOutTasksLoop(&wgLoop)
|
||||||
wgLoop.Wait()
|
wgLoop.Wait()
|
||||||
})
|
})
|
||||||
|
|
||||||
|
@ -125,8 +129,25 @@ func TestImportManager_NewImportManager(t *testing.T) {
|
||||||
})
|
})
|
||||||
mgr.loadFromTaskStore()
|
mgr.loadFromTaskStore()
|
||||||
var wgLoop sync.WaitGroup
|
var wgLoop sync.WaitGroup
|
||||||
wgLoop.Add(1)
|
wgLoop.Add(2)
|
||||||
mgr.expireOldTasksLoop(&wgLoop)
|
mgr.expireOldTasksLoop(&wgLoop)
|
||||||
|
mgr.sendOutTasksLoop(&wgLoop)
|
||||||
|
wgLoop.Wait()
|
||||||
|
})
|
||||||
|
|
||||||
|
wg.Add(1)
|
||||||
|
t.Run("check init", func(t *testing.T) {
|
||||||
|
defer wg.Done()
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
|
||||||
|
defer cancel()
|
||||||
|
mgr := newImportManager(ctx, mockKv, idAlloc, fn)
|
||||||
|
assert.NotNil(t, mgr)
|
||||||
|
mgr.init(ctx)
|
||||||
|
var wgLoop sync.WaitGroup
|
||||||
|
wgLoop.Add(2)
|
||||||
|
mgr.expireOldTasksLoop(&wgLoop)
|
||||||
|
mgr.sendOutTasksLoop(&wgLoop)
|
||||||
|
time.Sleep(500 * time.Millisecond)
|
||||||
wgLoop.Wait()
|
wgLoop.Wait()
|
||||||
})
|
})
|
||||||
|
|
||||||
|
@ -233,6 +254,88 @@ func TestImportManager_ImportJob(t *testing.T) {
|
||||||
assert.Equal(t, 2, len(mgr.workingTasks))
|
assert.Equal(t, 2, len(mgr.workingTasks))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestImportManager_AllDataNodesBusy(t *testing.T) {
|
||||||
|
var countLock sync.RWMutex
|
||||||
|
var globalCount = typeutil.UniqueID(0)
|
||||||
|
|
||||||
|
var idAlloc = func(count uint32) (typeutil.UniqueID, typeutil.UniqueID, error) {
|
||||||
|
countLock.Lock()
|
||||||
|
defer countLock.Unlock()
|
||||||
|
globalCount++
|
||||||
|
return globalCount, 0, nil
|
||||||
|
}
|
||||||
|
Params.RootCoordCfg.ImportTaskSubPath = "test_import_task"
|
||||||
|
colID := int64(100)
|
||||||
|
mockKv := &kv.MockMetaKV{}
|
||||||
|
mockKv.InMemKv = make(map[string]string)
|
||||||
|
rowReq := &milvuspb.ImportRequest{
|
||||||
|
CollectionName: "c1",
|
||||||
|
PartitionName: "p1",
|
||||||
|
RowBased: true,
|
||||||
|
Files: []string{"f1", "f2", "f3"},
|
||||||
|
}
|
||||||
|
colReq := &milvuspb.ImportRequest{
|
||||||
|
CollectionName: "c1",
|
||||||
|
PartitionName: "p1",
|
||||||
|
RowBased: false,
|
||||||
|
Files: []string{"f1", "f2"},
|
||||||
|
Options: []*commonpb.KeyValuePair{
|
||||||
|
{
|
||||||
|
Key: Bucket,
|
||||||
|
Value: "mybucket",
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
dnList := []int64{1, 2, 3}
|
||||||
|
count := 0
|
||||||
|
fn := func(ctx context.Context, req *datapb.ImportTaskRequest) *datapb.ImportTaskResponse {
|
||||||
|
if count < len(dnList) {
|
||||||
|
count++
|
||||||
|
return &datapb.ImportTaskResponse{
|
||||||
|
Status: &commonpb.Status{
|
||||||
|
ErrorCode: commonpb.ErrorCode_Success,
|
||||||
|
},
|
||||||
|
DatanodeId: dnList[count-1],
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return &datapb.ImportTaskResponse{
|
||||||
|
Status: &commonpb.Status{
|
||||||
|
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
mgr := newImportManager(context.TODO(), mockKv, idAlloc, fn)
|
||||||
|
mgr.importJob(context.TODO(), rowReq, colID, 0)
|
||||||
|
assert.Equal(t, 0, len(mgr.pendingTasks))
|
||||||
|
assert.Equal(t, len(rowReq.Files), len(mgr.workingTasks))
|
||||||
|
|
||||||
|
mgr = newImportManager(context.TODO(), mockKv, idAlloc, fn)
|
||||||
|
mgr.importJob(context.TODO(), rowReq, colID, 0)
|
||||||
|
assert.Equal(t, len(rowReq.Files), len(mgr.pendingTasks))
|
||||||
|
assert.Equal(t, 0, len(mgr.workingTasks))
|
||||||
|
|
||||||
|
// Reset count.
|
||||||
|
count = 0
|
||||||
|
mgr = newImportManager(context.TODO(), mockKv, idAlloc, fn)
|
||||||
|
mgr.importJob(context.TODO(), colReq, colID, 0)
|
||||||
|
assert.Equal(t, 0, len(mgr.pendingTasks))
|
||||||
|
assert.Equal(t, 1, len(mgr.workingTasks))
|
||||||
|
|
||||||
|
mgr.importJob(context.TODO(), colReq, colID, 0)
|
||||||
|
assert.Equal(t, 0, len(mgr.pendingTasks))
|
||||||
|
assert.Equal(t, 2, len(mgr.workingTasks))
|
||||||
|
|
||||||
|
mgr.importJob(context.TODO(), colReq, colID, 0)
|
||||||
|
assert.Equal(t, 0, len(mgr.pendingTasks))
|
||||||
|
assert.Equal(t, 3, len(mgr.workingTasks))
|
||||||
|
|
||||||
|
mgr.importJob(context.TODO(), colReq, colID, 0)
|
||||||
|
assert.Equal(t, 1, len(mgr.pendingTasks))
|
||||||
|
assert.Equal(t, 3, len(mgr.workingTasks))
|
||||||
|
}
|
||||||
|
|
||||||
func TestImportManager_TaskState(t *testing.T) {
|
func TestImportManager_TaskState(t *testing.T) {
|
||||||
var countLock sync.RWMutex
|
var countLock sync.RWMutex
|
||||||
var globalCount = typeutil.UniqueID(0)
|
var globalCount = typeutil.UniqueID(0)
|
||||||
|
|
|
@ -707,19 +707,13 @@ func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error {
|
||||||
ErrorCode: commonpb.ErrorCode_Success,
|
ErrorCode: commonpb.ErrorCode_Success,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
defer func() {
|
defer func() {
|
||||||
if err := recover(); err != nil {
|
if err := recover(); err != nil {
|
||||||
resp.Status.ErrorCode = commonpb.ErrorCode_UnexpectedError
|
resp.Status.ErrorCode = commonpb.ErrorCode_UnexpectedError
|
||||||
resp.Status.Reason = "assign import task to data coord panic"
|
resp.Status.Reason = "assign import task to data coord panic"
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
resp, _ = s.Import(ctx, req)
|
resp, _ = s.Import(ctx, req)
|
||||||
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
|
|
||||||
return resp
|
|
||||||
}
|
|
||||||
|
|
||||||
return resp
|
return resp
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1296,12 +1290,13 @@ func (c *Core) Start() error {
|
||||||
log.Fatal("RootCoord Start reSendDdMsg failed", zap.Error(err))
|
log.Fatal("RootCoord Start reSendDdMsg failed", zap.Error(err))
|
||||||
panic(err)
|
panic(err)
|
||||||
}
|
}
|
||||||
c.wg.Add(5)
|
c.wg.Add(6)
|
||||||
go c.startTimeTickLoop()
|
go c.startTimeTickLoop()
|
||||||
go c.tsLoop()
|
go c.tsLoop()
|
||||||
go c.chanTimeTick.startWatch(&c.wg)
|
go c.chanTimeTick.startWatch(&c.wg)
|
||||||
go c.checkFlushedSegmentsLoop()
|
go c.checkFlushedSegmentsLoop()
|
||||||
go c.importManager.expireOldTasksLoop(&c.wg)
|
go c.importManager.expireOldTasksLoop(&c.wg)
|
||||||
|
go c.importManager.sendOutTasksLoop(&c.wg)
|
||||||
Params.RootCoordCfg.CreatedTime = time.Now()
|
Params.RootCoordCfg.CreatedTime = time.Now()
|
||||||
Params.RootCoordCfg.UpdatedTime = time.Now()
|
Params.RootCoordCfg.UpdatedTime = time.Now()
|
||||||
})
|
})
|
||||||
|
|
Loading…
Reference in New Issue