2021-04-19 13:47:10 +08:00
|
|
|
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
|
|
|
//
|
|
|
|
// Licensed 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-06-22 16:44:09 +08:00
|
|
|
package querycoord
|
2021-01-15 15:28:54 +08:00
|
|
|
|
2021-01-16 15:31:10 +08:00
|
|
|
import (
|
2021-01-22 14:28:06 +08:00
|
|
|
"context"
|
2021-10-11 19:00:46 +08:00
|
|
|
"errors"
|
2021-10-22 19:07:15 +08:00
|
|
|
|
2021-10-24 22:39:09 +08:00
|
|
|
"fmt"
|
2021-03-08 15:25:55 +08:00
|
|
|
"math/rand"
|
2021-06-03 14:58:34 +08:00
|
|
|
"strconv"
|
2021-06-19 11:45:09 +08:00
|
|
|
"sync"
|
2021-01-22 14:28:06 +08:00
|
|
|
"sync/atomic"
|
2021-02-20 09:20:51 +08:00
|
|
|
"time"
|
2021-01-22 14:28:06 +08:00
|
|
|
|
2021-10-24 22:39:09 +08:00
|
|
|
"github.com/golang/protobuf/proto"
|
|
|
|
"go.etcd.io/etcd/api/v3/mvccpb"
|
2021-06-04 16:29:35 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2021-10-22 19:07:15 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/allocator"
|
2021-06-15 12:41:40 +08:00
|
|
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/msgstream"
|
2021-06-19 11:45:09 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
2021-06-19 11:45:09 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
2021-09-15 20:40:07 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
2021-06-19 11:45:09 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/retry"
|
2021-05-21 19:28:52 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
2021-10-22 19:07:15 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
2021-01-16 15:31:10 +08:00
|
|
|
)
|
2021-01-15 15:28:54 +08:00
|
|
|
|
2021-10-24 22:39:09 +08:00
|
|
|
const (
|
|
|
|
handoffSegmentPrefix = "querycoord-handoff"
|
|
|
|
)
|
|
|
|
|
2021-10-04 17:14:07 +08:00
|
|
|
// Timestamp is an alias for the Int64 type
|
2021-04-15 15:15:46 +08:00
|
|
|
type Timestamp = typeutil.Timestamp
|
|
|
|
|
2021-02-18 16:26:02 +08:00
|
|
|
type queryChannelInfo struct {
|
|
|
|
requestChannel string
|
|
|
|
responseChannel string
|
|
|
|
}
|
|
|
|
|
2021-10-05 15:34:26 +08:00
|
|
|
// QueryCoord is the coordinator of queryNodes
|
2021-06-22 16:44:09 +08:00
|
|
|
type QueryCoord struct {
|
2021-01-22 14:28:06 +08:00
|
|
|
loopCtx context.Context
|
|
|
|
loopCancel context.CancelFunc
|
2021-06-19 11:45:09 +08:00
|
|
|
loopWg sync.WaitGroup
|
|
|
|
kvClient *etcdkv.EtcdKV
|
2021-01-22 14:28:06 +08:00
|
|
|
|
2021-09-23 21:56:02 +08:00
|
|
|
initOnce sync.Once
|
|
|
|
|
2021-06-22 16:44:09 +08:00
|
|
|
queryCoordID uint64
|
2021-08-02 22:39:25 +08:00
|
|
|
meta Meta
|
2021-09-15 20:40:07 +08:00
|
|
|
cluster Cluster
|
2021-08-26 14:17:54 +08:00
|
|
|
newNodeFn newQueryNodeFn
|
2021-06-22 16:44:09 +08:00
|
|
|
scheduler *TaskScheduler
|
2021-10-22 19:07:15 +08:00
|
|
|
idAllocator func() (UniqueID, error)
|
2021-01-22 14:28:06 +08:00
|
|
|
|
2021-09-03 17:15:26 +08:00
|
|
|
metricsCacheManager *metricsinfo.MetricsCacheManager
|
|
|
|
|
2021-06-21 18:22:13 +08:00
|
|
|
dataCoordClient types.DataCoord
|
|
|
|
rootCoordClient types.RootCoord
|
2021-01-22 14:28:06 +08:00
|
|
|
|
2021-06-19 11:45:09 +08:00
|
|
|
session *sessionutil.Session
|
|
|
|
eventChan <-chan *sessionutil.SessionEvent
|
2021-05-21 19:28:52 +08:00
|
|
|
|
2021-01-26 15:13:20 +08:00
|
|
|
stateCode atomic.Value
|
|
|
|
enableGrpc bool
|
2021-02-08 14:30:54 +08:00
|
|
|
|
|
|
|
msFactory msgstream.Factory
|
2021-01-22 14:28:06 +08:00
|
|
|
}
|
|
|
|
|
2021-05-25 15:06:05 +08:00
|
|
|
// Register register query service at etcd
|
2021-06-22 16:44:09 +08:00
|
|
|
func (qc *QueryCoord) Register() error {
|
2021-07-13 14:16:00 +08:00
|
|
|
log.Debug("query coord session info", zap.String("metaPath", Params.MetaRootPath), zap.Strings("etcdEndPoints", Params.EtcdEndpoints), zap.String("address", Params.Address))
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.session = sessionutil.NewSession(qc.loopCtx, Params.MetaRootPath, Params.EtcdEndpoints)
|
2021-10-14 16:40:35 +08:00
|
|
|
qc.session.Init(typeutil.QueryCoordRole, Params.Address, true)
|
2021-06-22 16:44:09 +08:00
|
|
|
Params.NodeID = uint64(qc.session.ServerID)
|
2021-10-01 08:52:50 +08:00
|
|
|
Params.SetLogger(typeutil.UniqueID(-1))
|
2021-05-25 15:06:05 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-10-05 15:36:17 +08:00
|
|
|
// Init function initializes the queryCoord's meta, cluster, etcdKV and task scheduler
|
2021-06-22 16:44:09 +08:00
|
|
|
func (qc *QueryCoord) Init() error {
|
2021-10-11 09:54:37 +08:00
|
|
|
log.Debug("query coordinator start init")
|
|
|
|
//connect etcd
|
2021-06-19 11:45:09 +08:00
|
|
|
connectEtcdFn := func() error {
|
2021-08-13 11:04:09 +08:00
|
|
|
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
2021-06-19 11:45:09 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.kvClient = etcdKV
|
2021-08-03 22:03:25 +08:00
|
|
|
return nil
|
2021-06-19 11:45:09 +08:00
|
|
|
}
|
2021-09-23 21:56:02 +08:00
|
|
|
var initError error = nil
|
|
|
|
qc.initOnce.Do(func() {
|
|
|
|
log.Debug("query coordinator try to connect etcd")
|
|
|
|
initError = retry.Do(qc.loopCtx, connectEtcdFn, retry.Attempts(300))
|
|
|
|
if initError != nil {
|
|
|
|
log.Debug("query coordinator try to connect etcd failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
|
|
|
log.Debug("query coordinator try to connect etcd success")
|
2021-10-22 19:07:15 +08:00
|
|
|
|
|
|
|
// init id allocator
|
|
|
|
var idAllocatorKV *etcdkv.EtcdKV
|
|
|
|
idAllocatorKV, initError = tsoutil.NewTSOKVBase(Params.EtcdEndpoints, Params.KvRootPath, "queryCoordTaskID")
|
|
|
|
if initError != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
|
|
|
|
initError = idAllocator.Initialize()
|
|
|
|
if initError != nil {
|
|
|
|
log.Debug("query coordinator idAllocator initialize failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
|
|
|
qc.idAllocator = func() (UniqueID, error) {
|
|
|
|
return idAllocator.AllocOne()
|
|
|
|
}
|
|
|
|
|
|
|
|
// init meta
|
|
|
|
qc.meta, initError = newMeta(qc.loopCtx, qc.kvClient, qc.msFactory, qc.idAllocator)
|
2021-09-23 21:56:02 +08:00
|
|
|
if initError != nil {
|
|
|
|
log.Error("query coordinator init meta failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
2021-08-03 22:03:25 +08:00
|
|
|
|
2021-10-22 19:07:15 +08:00
|
|
|
// init cluster
|
2021-09-23 21:56:02 +08:00
|
|
|
qc.cluster, initError = newQueryNodeCluster(qc.loopCtx, qc.meta, qc.kvClient, qc.newNodeFn, qc.session)
|
|
|
|
if initError != nil {
|
|
|
|
log.Error("query coordinator init cluster failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
2021-08-03 22:03:25 +08:00
|
|
|
|
2021-10-22 19:07:15 +08:00
|
|
|
// init task scheduler
|
|
|
|
qc.scheduler, initError = NewTaskScheduler(qc.loopCtx, qc.meta, qc.cluster, qc.kvClient, qc.rootCoordClient, qc.dataCoordClient, qc.idAllocator)
|
2021-09-23 21:56:02 +08:00
|
|
|
if initError != nil {
|
|
|
|
log.Error("query coordinator init task scheduler failed", zap.Error(initError))
|
|
|
|
return
|
|
|
|
}
|
2021-08-03 22:03:25 +08:00
|
|
|
|
2021-09-23 21:56:02 +08:00
|
|
|
qc.metricsCacheManager = metricsinfo.NewMetricsCacheManager()
|
|
|
|
})
|
2021-09-03 17:15:26 +08:00
|
|
|
|
2021-09-23 21:56:02 +08:00
|
|
|
return initError
|
2021-01-22 14:28:06 +08:00
|
|
|
}
|
|
|
|
|
2021-10-05 15:38:03 +08:00
|
|
|
// Start function starts the goroutines to watch the meta and node updates
|
2021-06-22 16:44:09 +08:00
|
|
|
func (qc *QueryCoord) Start() error {
|
2021-10-22 19:07:15 +08:00
|
|
|
m := map[string]interface{}{
|
|
|
|
"PulsarAddress": Params.PulsarAddress,
|
|
|
|
"ReceiveBufSize": 1024,
|
|
|
|
"PulsarBufSize": 1024}
|
|
|
|
err := qc.msFactory.SetParams(m)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.scheduler.Start()
|
2021-04-15 15:15:46 +08:00
|
|
|
log.Debug("start scheduler ...")
|
2021-10-12 19:39:24 +08:00
|
|
|
|
|
|
|
Params.CreatedTime = time.Now()
|
|
|
|
Params.UpdatedTime = time.Now()
|
|
|
|
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.UpdateStateCode(internalpb.StateCode_Healthy)
|
2021-06-19 11:45:09 +08:00
|
|
|
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.loopWg.Add(1)
|
|
|
|
go qc.watchNodeLoop()
|
2021-06-19 11:45:09 +08:00
|
|
|
|
2021-10-24 22:39:09 +08:00
|
|
|
qc.loopWg.Add(1)
|
|
|
|
go qc.watchHandoffSegmentLoop()
|
|
|
|
|
2021-10-14 16:40:35 +08:00
|
|
|
go qc.session.LivenessCheck(qc.loopCtx, func() {
|
2021-10-30 10:24:38 +08:00
|
|
|
log.Error("Query Coord disconnected from etcd, process will exit", zap.Int64("Server Id", qc.session.ServerID))
|
|
|
|
if err := qc.Stop(); err != nil {
|
|
|
|
log.Fatal("failed to stop server", zap.Error(err))
|
|
|
|
}
|
2021-09-23 18:29:55 +08:00
|
|
|
})
|
|
|
|
|
2021-01-22 14:28:06 +08:00
|
|
|
return nil
|
2021-01-16 15:31:10 +08:00
|
|
|
}
|
|
|
|
|
2021-10-06 14:00:47 +08:00
|
|
|
// Stop function stops watching the meta and node updates
|
2021-06-22 16:44:09 +08:00
|
|
|
func (qc *QueryCoord) Stop() error {
|
|
|
|
qc.scheduler.Close()
|
2021-04-15 15:15:46 +08:00
|
|
|
log.Debug("close scheduler ...")
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.loopCancel()
|
|
|
|
qc.UpdateStateCode(internalpb.StateCode_Abnormal)
|
2021-06-19 11:45:09 +08:00
|
|
|
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.loopWg.Wait()
|
2021-01-22 14:28:06 +08:00
|
|
|
return nil
|
2021-01-16 15:31:10 +08:00
|
|
|
}
|
|
|
|
|
2021-10-06 14:02:34 +08:00
|
|
|
// UpdateStateCode updates the status of the coord, including healthy, unhealthy
|
2021-06-22 16:44:09 +08:00
|
|
|
func (qc *QueryCoord) UpdateStateCode(code internalpb.StateCode) {
|
|
|
|
qc.stateCode.Store(code)
|
2021-02-23 11:40:30 +08:00
|
|
|
}
|
|
|
|
|
2021-10-09 20:53:10 +08:00
|
|
|
// NewQueryCoord creates a QueryCoord object.
|
2021-06-22 16:44:09 +08:00
|
|
|
func NewQueryCoord(ctx context.Context, factory msgstream.Factory) (*QueryCoord, error) {
|
2021-03-08 15:25:55 +08:00
|
|
|
rand.Seed(time.Now().UnixNano())
|
2021-02-18 16:26:02 +08:00
|
|
|
queryChannels := make([]*queryChannelInfo, 0)
|
2021-06-03 14:58:34 +08:00
|
|
|
channelID := len(queryChannels)
|
|
|
|
searchPrefix := Params.SearchChannelPrefix
|
|
|
|
searchResultPrefix := Params.SearchResultChannelPrefix
|
|
|
|
allocatedQueryChannel := searchPrefix + "-" + strconv.FormatInt(int64(channelID), 10)
|
|
|
|
allocatedQueryResultChannel := searchResultPrefix + "-" + strconv.FormatInt(int64(channelID), 10)
|
|
|
|
|
|
|
|
queryChannels = append(queryChannels, &queryChannelInfo{
|
|
|
|
requestChannel: allocatedQueryChannel,
|
|
|
|
responseChannel: allocatedQueryResultChannel,
|
|
|
|
})
|
|
|
|
|
2021-01-22 14:28:06 +08:00
|
|
|
ctx1, cancel := context.WithCancel(ctx)
|
2021-06-22 16:44:09 +08:00
|
|
|
service := &QueryCoord{
|
2021-06-15 12:41:40 +08:00
|
|
|
loopCtx: ctx1,
|
|
|
|
loopCancel: cancel,
|
|
|
|
msFactory: factory,
|
2021-08-26 14:17:54 +08:00
|
|
|
newNodeFn: newQueryNode,
|
2021-01-22 14:28:06 +08:00
|
|
|
}
|
2021-02-24 09:48:17 +08:00
|
|
|
|
2021-03-12 14:22:09 +08:00
|
|
|
service.UpdateStateCode(internalpb.StateCode_Abnormal)
|
2021-06-22 16:44:09 +08:00
|
|
|
log.Debug("query coordinator", zap.Any("queryChannels", queryChannels))
|
2021-01-22 14:28:06 +08:00
|
|
|
return service, nil
|
2021-01-15 15:28:54 +08:00
|
|
|
}
|
2021-01-26 15:13:20 +08:00
|
|
|
|
2021-10-06 14:04:25 +08:00
|
|
|
// SetRootCoord sets root coordinator's client
|
2021-10-11 19:00:46 +08:00
|
|
|
func (qc *QueryCoord) SetRootCoord(rootCoord types.RootCoord) error {
|
|
|
|
if rootCoord == nil {
|
|
|
|
return errors.New("null root coordinator interface")
|
|
|
|
}
|
|
|
|
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.rootCoordClient = rootCoord
|
2021-10-11 19:00:46 +08:00
|
|
|
return nil
|
2021-01-26 15:13:20 +08:00
|
|
|
}
|
|
|
|
|
2021-10-07 19:52:46 +08:00
|
|
|
// SetDataCoord sets data coordinator's client
|
2021-10-11 19:00:46 +08:00
|
|
|
func (qc *QueryCoord) SetDataCoord(dataCoord types.DataCoord) error {
|
|
|
|
if dataCoord == nil {
|
|
|
|
return errors.New("null data coordinator interface")
|
|
|
|
}
|
|
|
|
|
2021-06-22 16:44:09 +08:00
|
|
|
qc.dataCoordClient = dataCoord
|
2021-10-11 19:00:46 +08:00
|
|
|
return nil
|
2021-01-26 15:13:20 +08:00
|
|
|
}
|
2021-06-19 11:45:09 +08:00
|
|
|
|
2021-06-22 16:44:09 +08:00
|
|
|
func (qc *QueryCoord) watchNodeLoop() {
|
|
|
|
ctx, cancel := context.WithCancel(qc.loopCtx)
|
2021-06-19 11:45:09 +08:00
|
|
|
defer cancel()
|
2021-06-22 16:44:09 +08:00
|
|
|
defer qc.loopWg.Done()
|
|
|
|
log.Debug("query coordinator start watch node loop")
|
2021-06-19 11:45:09 +08:00
|
|
|
|
2021-09-15 20:40:07 +08:00
|
|
|
offlineNodes, err := qc.cluster.offlineNodes()
|
|
|
|
if err == nil {
|
|
|
|
offlineNodeIDs := make([]int64, 0)
|
|
|
|
for id := range offlineNodes {
|
|
|
|
offlineNodeIDs = append(offlineNodeIDs, id)
|
|
|
|
}
|
|
|
|
loadBalanceSegment := &querypb.LoadBalanceRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_LoadBalanceSegments,
|
|
|
|
SourceID: qc.session.ServerID,
|
|
|
|
},
|
|
|
|
SourceNodeIDs: offlineNodeIDs,
|
2021-06-19 11:45:09 +08:00
|
|
|
}
|
|
|
|
|
2021-10-11 09:54:37 +08:00
|
|
|
baseTask := newBaseTask(qc.loopCtx, querypb.TriggerCondition_nodeDown)
|
2021-10-18 21:34:47 +08:00
|
|
|
loadBalanceTask := &loadBalanceTask{
|
|
|
|
baseTask: baseTask,
|
2021-09-15 20:40:07 +08:00
|
|
|
LoadBalanceRequest: loadBalanceSegment,
|
|
|
|
rootCoord: qc.rootCoordClient,
|
|
|
|
dataCoord: qc.dataCoordClient,
|
|
|
|
cluster: qc.cluster,
|
|
|
|
meta: qc.meta,
|
2021-06-19 11:45:09 +08:00
|
|
|
}
|
2021-10-11 09:54:37 +08:00
|
|
|
//TODO::deal enqueue error
|
|
|
|
qc.scheduler.Enqueue(loadBalanceTask)
|
2021-09-15 20:40:07 +08:00
|
|
|
log.Debug("start a loadBalance task", zap.Any("task", loadBalanceTask))
|
2021-06-19 11:45:09 +08:00
|
|
|
}
|
|
|
|
|
2021-09-15 20:40:07 +08:00
|
|
|
qc.eventChan = qc.session.WatchServices(typeutil.QueryNodeRole, qc.cluster.getSessionVersion()+1)
|
2021-06-19 11:45:09 +08:00
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
2021-10-14 19:20:35 +08:00
|
|
|
case event, ok := <-qc.eventChan:
|
|
|
|
if !ok {
|
|
|
|
return
|
|
|
|
}
|
2021-06-19 11:45:09 +08:00
|
|
|
switch event.EventType {
|
|
|
|
case sessionutil.SessionAddEvent:
|
|
|
|
serverID := event.Session.ServerID
|
2021-07-13 14:16:00 +08:00
|
|
|
log.Debug("start add a queryNode to cluster", zap.Any("nodeID", serverID))
|
2021-09-15 20:40:07 +08:00
|
|
|
err := qc.cluster.registerNode(ctx, event.Session, serverID, disConnect)
|
2021-07-13 14:16:00 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Error("query node failed to register", zap.Int64("nodeID", serverID), zap.String("error info", err.Error()))
|
|
|
|
}
|
2021-09-03 17:15:26 +08:00
|
|
|
qc.metricsCacheManager.InvalidateSystemInfoMetrics()
|
2021-06-19 11:45:09 +08:00
|
|
|
case sessionutil.SessionDelEvent:
|
|
|
|
serverID := event.Session.ServerID
|
2021-07-13 14:16:00 +08:00
|
|
|
log.Debug("get a del event after queryNode down", zap.Int64("nodeID", serverID))
|
2021-11-05 16:00:55 +08:00
|
|
|
nodeExist := qc.cluster.hasNode(serverID)
|
|
|
|
if !nodeExist {
|
2021-07-13 14:16:00 +08:00
|
|
|
log.Error("queryNode not exist", zap.Int64("nodeID", serverID))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
|
|
|
qc.cluster.stopNode(serverID)
|
|
|
|
loadBalanceSegment := &querypb.LoadBalanceRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_LoadBalanceSegments,
|
|
|
|
SourceID: qc.session.ServerID,
|
|
|
|
},
|
|
|
|
SourceNodeIDs: []int64{serverID},
|
|
|
|
BalanceReason: querypb.TriggerCondition_nodeDown,
|
|
|
|
}
|
|
|
|
|
2021-10-11 09:54:37 +08:00
|
|
|
baseTask := newBaseTask(qc.loopCtx, querypb.TriggerCondition_nodeDown)
|
2021-10-18 21:34:47 +08:00
|
|
|
loadBalanceTask := &loadBalanceTask{
|
|
|
|
baseTask: baseTask,
|
2021-07-13 14:16:00 +08:00
|
|
|
LoadBalanceRequest: loadBalanceSegment,
|
|
|
|
rootCoord: qc.rootCoordClient,
|
|
|
|
dataCoord: qc.dataCoordClient,
|
|
|
|
cluster: qc.cluster,
|
|
|
|
meta: qc.meta,
|
2021-06-27 12:16:09 +08:00
|
|
|
}
|
2021-09-03 17:15:26 +08:00
|
|
|
qc.metricsCacheManager.InvalidateSystemInfoMetrics()
|
2021-10-11 09:54:37 +08:00
|
|
|
//TODO:: deal enqueue error
|
|
|
|
qc.scheduler.Enqueue(loadBalanceTask)
|
|
|
|
log.Debug("start a loadBalance task", zap.Any("task", loadBalanceTask))
|
2021-06-19 11:45:09 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-10-24 22:39:09 +08:00
|
|
|
|
|
|
|
func (qc *QueryCoord) watchHandoffSegmentLoop() {
|
|
|
|
ctx, cancel := context.WithCancel(qc.loopCtx)
|
|
|
|
|
|
|
|
defer cancel()
|
|
|
|
defer qc.loopWg.Done()
|
|
|
|
log.Debug("query coordinator start watch segment loop")
|
|
|
|
|
|
|
|
// TODO:: recover handoff task when coord down
|
|
|
|
watchChan := qc.kvClient.WatchWithPrefix(handoffSegmentPrefix)
|
|
|
|
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
|
|
|
case resp := <-watchChan:
|
|
|
|
for _, event := range resp.Events {
|
|
|
|
segmentInfo := &querypb.SegmentInfo{}
|
|
|
|
err := proto.Unmarshal(event.Kv.Value, segmentInfo)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watchHandoffSegmentLoop: unmarshal failed", zap.Any("error", err.Error()))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
switch event.Type {
|
|
|
|
case mvccpb.PUT:
|
|
|
|
collectionID := segmentInfo.CollectionID
|
|
|
|
partitionID := segmentInfo.PartitionID
|
|
|
|
segmentID := segmentInfo.SegmentID
|
|
|
|
if Params.AutoHandoff {
|
|
|
|
log.Debug("watchHandoffSegmentLoop: handoff segment received",
|
2021-11-08 21:00:02 +08:00
|
|
|
zap.Int64("collectionID", collectionID),
|
|
|
|
zap.Int64("partitionID", partitionID),
|
|
|
|
zap.Int64("segmentID", segmentID),
|
|
|
|
zap.Any("segmentInfo", segmentInfo),
|
2021-10-24 22:39:09 +08:00
|
|
|
)
|
|
|
|
baseTask := newBaseTask(qc.loopCtx, querypb.TriggerCondition_handoff)
|
|
|
|
handoffReq := &querypb.HandoffSegmentsRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_HandoffSegments,
|
|
|
|
},
|
|
|
|
SegmentInfos: []*querypb.SegmentInfo{segmentInfo},
|
|
|
|
}
|
|
|
|
handoffTask := &handoffTask{
|
|
|
|
baseTask: baseTask,
|
|
|
|
HandoffSegmentsRequest: handoffReq,
|
|
|
|
dataCoord: qc.dataCoordClient,
|
|
|
|
cluster: qc.cluster,
|
|
|
|
meta: qc.meta,
|
|
|
|
}
|
|
|
|
err = qc.scheduler.Enqueue(handoffTask)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watchHandoffSegmentLoop: handoffTask enqueue failed", zap.Error(err))
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
go func() {
|
|
|
|
err := handoffTask.waitToFinish()
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watchHandoffSegmentLoop: handoffTask failed", zap.Error(err))
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
log.Debug("watchHandoffSegmentLoop: handoffTask completed",
|
|
|
|
zap.Any("collectionID", collectionID),
|
|
|
|
zap.Any("partitionID", partitionID),
|
|
|
|
zap.Any("segmentID", segmentID),
|
|
|
|
zap.Any("channel", segmentInfo.ChannelID),
|
|
|
|
)
|
|
|
|
}
|
|
|
|
|
|
|
|
buildQuerySegmentPath := fmt.Sprintf("%s/%d/%d/%d", handoffSegmentPrefix, collectionID, partitionID, segmentID)
|
|
|
|
err = qc.kvClient.Remove(buildQuerySegmentPath)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("watchHandoffSegmentLoop: remove handoff segment from etcd failed", zap.Error(err))
|
|
|
|
}
|
|
|
|
default:
|
|
|
|
// do nothing
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|