2022-10-11 11:39:22 +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.
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
package querycoordv2
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"errors"
|
|
|
|
"fmt"
|
|
|
|
"os"
|
|
|
|
"sort"
|
|
|
|
"sync"
|
|
|
|
"sync/atomic"
|
|
|
|
"syscall"
|
|
|
|
"time"
|
|
|
|
|
2022-11-10 15:01:04 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/metrics"
|
2022-11-08 20:13:03 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/timerecord"
|
|
|
|
|
2022-10-16 20:49:27 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/commonpb"
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
|
2022-09-15 18:48:32 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/allocator"
|
|
|
|
"github.com/milvus-io/milvus/internal/common"
|
|
|
|
"github.com/milvus-io/milvus/internal/kv"
|
|
|
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/balance"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/checkers"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/dist"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/job"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/observers"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/params"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/session"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/task"
|
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
2022-11-07 19:37:04 +08:00
|
|
|
clientv3 "go.etcd.io/etcd/client/v3"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
"golang.org/x/sync/errgroup"
|
2022-09-15 18:48:32 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
var (
|
|
|
|
// Only for re-export
|
2022-11-04 14:25:38 +08:00
|
|
|
Params = params.Params
|
2022-09-15 18:48:32 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
type Server struct {
|
|
|
|
ctx context.Context
|
|
|
|
cancel context.CancelFunc
|
|
|
|
wg sync.WaitGroup
|
|
|
|
status atomic.Value
|
|
|
|
etcdCli *clientv3.Client
|
2022-11-04 14:25:38 +08:00
|
|
|
address string
|
2022-09-15 18:48:32 +08:00
|
|
|
session *sessionutil.Session
|
|
|
|
kv kv.MetaKv
|
|
|
|
idAllocator func() (int64, error)
|
|
|
|
metricsCacheManager *metricsinfo.MetricsCacheManager
|
|
|
|
|
|
|
|
// Coordinators
|
2023-01-06 14:21:37 +08:00
|
|
|
dataCoord types.DataCoord
|
|
|
|
rootCoord types.RootCoord
|
2022-09-15 18:48:32 +08:00
|
|
|
|
|
|
|
// Meta
|
|
|
|
store meta.Store
|
|
|
|
meta *meta.Meta
|
|
|
|
dist *meta.DistributionManager
|
|
|
|
targetMgr *meta.TargetManager
|
|
|
|
broker meta.Broker
|
|
|
|
|
|
|
|
// Session
|
2023-01-12 19:49:40 +08:00
|
|
|
cluster session.Cluster
|
|
|
|
nodeMgr *session.NodeManager
|
|
|
|
queryNodeCreator session.QueryNodeCreator
|
2022-09-15 18:48:32 +08:00
|
|
|
|
|
|
|
// Schedulers
|
|
|
|
jobScheduler *job.Scheduler
|
|
|
|
taskScheduler task.Scheduler
|
|
|
|
|
|
|
|
// HeartBeat
|
|
|
|
distController *dist.Controller
|
|
|
|
|
|
|
|
// Checkers
|
|
|
|
checkerController *checkers.CheckerController
|
|
|
|
|
|
|
|
// Observers
|
|
|
|
collectionObserver *observers.CollectionObserver
|
|
|
|
leaderObserver *observers.LeaderObserver
|
2022-11-07 19:37:04 +08:00
|
|
|
targetObserver *observers.TargetObserver
|
2023-01-30 10:19:48 +08:00
|
|
|
replicaObserver *observers.ReplicaObserver
|
|
|
|
resourceObserver *observers.ResourceObserver
|
2022-09-15 18:48:32 +08:00
|
|
|
|
|
|
|
balancer balance.Balance
|
2022-10-13 16:51:24 +08:00
|
|
|
|
|
|
|
// Active-standby
|
|
|
|
enableActiveStandBy bool
|
2023-02-09 15:24:31 +08:00
|
|
|
activateFunc func() error
|
2022-09-15 18:48:32 +08:00
|
|
|
}
|
|
|
|
|
2023-01-12 15:59:39 +08:00
|
|
|
func NewQueryCoord(ctx context.Context) (*Server, error) {
|
2022-09-15 18:48:32 +08:00
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
|
|
|
server := &Server{
|
2023-01-12 15:59:39 +08:00
|
|
|
ctx: ctx,
|
|
|
|
cancel: cancel,
|
2022-09-15 18:48:32 +08:00
|
|
|
}
|
2022-10-10 15:55:22 +08:00
|
|
|
server.UpdateStateCode(commonpb.StateCode_Abnormal)
|
2023-01-12 19:49:40 +08:00
|
|
|
server.queryNodeCreator = session.DefaultQueryNodeCreator
|
2022-09-15 18:48:32 +08:00
|
|
|
return server, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) Register() error {
|
|
|
|
s.session.Register()
|
2022-10-13 16:51:24 +08:00
|
|
|
if s.enableActiveStandBy {
|
2023-02-09 15:24:31 +08:00
|
|
|
if err := s.session.ProcessActiveStandBy(s.activateFunc); err != nil {
|
|
|
|
log.Error("failed to activate standby server", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
2022-10-13 16:51:24 +08:00
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
go s.session.LivenessCheck(s.ctx, func() {
|
|
|
|
log.Error("QueryCoord disconnected from etcd, process will exit", zap.Int64("serverID", s.session.ServerID))
|
|
|
|
if err := s.Stop(); err != nil {
|
|
|
|
log.Fatal("failed to stop server", zap.Error(err))
|
|
|
|
}
|
|
|
|
// manually send signal to starter goroutine
|
|
|
|
if s.session.TriggerKill {
|
|
|
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
|
|
|
p.Signal(syscall.SIGINT)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
})
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2023-01-30 11:11:50 +08:00
|
|
|
func (s *Server) initSession() error {
|
2022-09-15 18:48:32 +08:00
|
|
|
// Init QueryCoord session
|
2022-11-17 18:59:09 +08:00
|
|
|
s.session = sessionutil.NewSession(s.ctx, Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
|
2022-09-15 18:48:32 +08:00
|
|
|
if s.session == nil {
|
|
|
|
return fmt.Errorf("failed to create session")
|
|
|
|
}
|
2022-11-04 14:25:38 +08:00
|
|
|
s.session.Init(typeutil.QueryCoordRole, s.address, true, true)
|
2022-12-07 18:01:19 +08:00
|
|
|
s.enableActiveStandBy = Params.QueryCoordCfg.EnableActiveStandby.GetAsBool()
|
2022-10-13 16:51:24 +08:00
|
|
|
s.session.SetEnableActiveStandBy(s.enableActiveStandBy)
|
2023-01-30 11:11:50 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) Init() error {
|
|
|
|
log.Info("QueryCoord start init",
|
|
|
|
zap.String("meta-root-path", Params.EtcdCfg.MetaRootPath.GetValue()),
|
|
|
|
zap.String("address", s.address))
|
|
|
|
|
|
|
|
if err := s.initSession(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
if s.enableActiveStandBy {
|
2023-02-09 15:24:31 +08:00
|
|
|
s.activateFunc = func() error {
|
2023-01-30 11:11:50 +08:00
|
|
|
log.Info("QueryCoord switch from standby to active, activating")
|
|
|
|
if err := s.initQueryCoord(); err != nil {
|
2023-02-09 15:24:31 +08:00
|
|
|
log.Error("QueryCoord init failed", zap.Error(err))
|
|
|
|
return err
|
2023-01-30 11:11:50 +08:00
|
|
|
}
|
|
|
|
if err := s.startQueryCoord(); err != nil {
|
2023-02-09 15:24:31 +08:00
|
|
|
log.Error("QueryCoord init failed", zap.Error(err))
|
|
|
|
return err
|
2023-01-30 11:11:50 +08:00
|
|
|
}
|
|
|
|
s.UpdateStateCode(commonpb.StateCode_Healthy)
|
|
|
|
log.Info("QueryCoord startup success")
|
2023-02-09 15:24:31 +08:00
|
|
|
return nil
|
2023-01-30 11:11:50 +08:00
|
|
|
}
|
|
|
|
s.UpdateStateCode(commonpb.StateCode_StandBy)
|
|
|
|
log.Info("QueryCoord enter standby mode successfully")
|
|
|
|
return nil
|
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
|
2023-01-30 11:11:50 +08:00
|
|
|
return s.initQueryCoord()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) initQueryCoord() error {
|
2022-09-15 18:48:32 +08:00
|
|
|
// Init KV
|
2022-11-17 18:59:09 +08:00
|
|
|
etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
|
2022-09-15 18:48:32 +08:00
|
|
|
s.kv = etcdKV
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("query coordinator try to connect etcd success")
|
2022-09-15 18:48:32 +08:00
|
|
|
|
|
|
|
// Init ID allocator
|
2022-11-17 18:59:09 +08:00
|
|
|
idAllocatorKV := tsoutil.NewTSOKVBase(s.etcdCli, Params.EtcdCfg.KvRootPath.GetValue(), "querycoord-id-allocator")
|
2022-09-15 18:48:32 +08:00
|
|
|
idAllocator := allocator.NewGlobalIDAllocator("idTimestamp", idAllocatorKV)
|
|
|
|
err := idAllocator.Initialize()
|
|
|
|
if err != nil {
|
|
|
|
log.Error("query coordinator id allocator initialize failed", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
s.idAllocator = func() (int64, error) {
|
|
|
|
return idAllocator.AllocOne()
|
|
|
|
}
|
|
|
|
|
|
|
|
// Init metrics cache manager
|
|
|
|
s.metricsCacheManager = metricsinfo.NewMetricsCacheManager()
|
|
|
|
|
|
|
|
// Init meta
|
2023-01-30 10:19:48 +08:00
|
|
|
s.nodeMgr = session.NewNodeManager()
|
2022-09-15 18:48:32 +08:00
|
|
|
err = s.initMeta()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
// Init session
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("init session")
|
2023-01-12 19:49:40 +08:00
|
|
|
s.cluster = session.NewCluster(s.nodeMgr, s.queryNodeCreator)
|
2022-09-15 18:48:32 +08:00
|
|
|
|
|
|
|
// Init schedulers
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("init schedulers")
|
2022-09-15 18:48:32 +08:00
|
|
|
s.jobScheduler = job.NewScheduler()
|
|
|
|
s.taskScheduler = task.NewScheduler(
|
|
|
|
s.ctx,
|
|
|
|
s.meta,
|
|
|
|
s.dist,
|
|
|
|
s.targetMgr,
|
|
|
|
s.broker,
|
|
|
|
s.cluster,
|
|
|
|
s.nodeMgr,
|
|
|
|
)
|
|
|
|
|
|
|
|
// Init heartbeat
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("init dist controller")
|
2022-09-15 18:48:32 +08:00
|
|
|
s.distController = dist.NewDistController(
|
|
|
|
s.cluster,
|
|
|
|
s.nodeMgr,
|
|
|
|
s.dist,
|
|
|
|
s.targetMgr,
|
|
|
|
s.taskScheduler,
|
|
|
|
)
|
|
|
|
|
|
|
|
// Init balancer
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("init balancer")
|
2022-09-15 18:48:32 +08:00
|
|
|
s.balancer = balance.NewRowCountBasedBalancer(
|
|
|
|
s.taskScheduler,
|
|
|
|
s.nodeMgr,
|
|
|
|
s.dist,
|
|
|
|
s.meta,
|
2022-11-21 16:21:11 +08:00
|
|
|
s.targetMgr,
|
2022-09-15 18:48:32 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
// Init checker controller
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("init checker controller")
|
2022-09-15 18:48:32 +08:00
|
|
|
s.checkerController = checkers.NewCheckerController(
|
|
|
|
s.meta,
|
|
|
|
s.dist,
|
|
|
|
s.targetMgr,
|
|
|
|
s.balancer,
|
|
|
|
s.taskScheduler,
|
|
|
|
)
|
|
|
|
|
|
|
|
// Init observers
|
|
|
|
s.initObserver()
|
|
|
|
|
2023-01-10 20:35:39 +08:00
|
|
|
// Init load status cache
|
|
|
|
meta.GlobalFailedLoadCache = meta.NewFailedLoadCache()
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
log.Info("QueryCoord init success")
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) initMeta() error {
|
2022-11-08 20:13:03 +08:00
|
|
|
record := timerecord.NewTimeRecorder("querycoord")
|
|
|
|
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("init meta")
|
2022-09-15 18:48:32 +08:00
|
|
|
s.store = meta.NewMetaStore(s.kv)
|
2023-01-30 10:19:48 +08:00
|
|
|
s.meta = meta.NewMeta(s.idAllocator, s.store, s.nodeMgr)
|
2022-09-15 18:48:32 +08:00
|
|
|
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("recover meta...")
|
2022-09-15 18:48:32 +08:00
|
|
|
err := s.meta.CollectionManager.Recover()
|
|
|
|
if err != nil {
|
|
|
|
log.Error("failed to recover collections")
|
|
|
|
return err
|
|
|
|
}
|
2022-12-20 20:33:27 +08:00
|
|
|
collections := s.meta.GetAll()
|
|
|
|
log.Info("recovering collections...", zap.Int64s("collections", collections))
|
|
|
|
metrics.QueryCoordNumCollections.WithLabelValues().Set(float64(len(collections)))
|
2022-11-10 15:01:04 +08:00
|
|
|
|
2022-12-20 20:33:27 +08:00
|
|
|
err = s.meta.ReplicaManager.Recover(collections)
|
2022-09-15 18:48:32 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Error("failed to recover replicas")
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2023-01-30 10:19:48 +08:00
|
|
|
err = s.meta.ResourceManager.Recover()
|
|
|
|
if err != nil {
|
|
|
|
log.Error("failed to recover resource groups")
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
s.dist = &meta.DistributionManager{
|
|
|
|
SegmentDistManager: meta.NewSegmentDistManager(),
|
|
|
|
ChannelDistManager: meta.NewChannelDistManager(),
|
|
|
|
LeaderViewManager: meta.NewLeaderViewManager(),
|
|
|
|
}
|
|
|
|
s.broker = meta.NewCoordinatorBroker(
|
|
|
|
s.dataCoord,
|
|
|
|
s.rootCoord,
|
|
|
|
)
|
2022-11-07 19:37:04 +08:00
|
|
|
s.targetMgr = meta.NewTargetManager(s.broker, s.meta)
|
|
|
|
|
2022-11-08 20:13:03 +08:00
|
|
|
record.Record("Server initMeta")
|
2022-09-15 18:48:32 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) initObserver() {
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("init observers")
|
2022-09-15 18:48:32 +08:00
|
|
|
s.leaderObserver = observers.NewLeaderObserver(
|
|
|
|
s.dist,
|
|
|
|
s.meta,
|
|
|
|
s.targetMgr,
|
|
|
|
s.cluster,
|
|
|
|
)
|
2022-11-07 19:37:04 +08:00
|
|
|
s.targetObserver = observers.NewTargetObserver(
|
2022-09-15 18:48:32 +08:00
|
|
|
s.meta,
|
|
|
|
s.targetMgr,
|
2022-11-07 19:37:04 +08:00
|
|
|
s.dist,
|
2022-11-07 17:05:06 +08:00
|
|
|
s.broker,
|
2022-09-15 18:48:32 +08:00
|
|
|
)
|
2023-01-17 11:41:51 +08:00
|
|
|
s.collectionObserver = observers.NewCollectionObserver(
|
|
|
|
s.dist,
|
|
|
|
s.meta,
|
|
|
|
s.targetMgr,
|
|
|
|
s.targetObserver,
|
|
|
|
)
|
2023-01-30 10:19:48 +08:00
|
|
|
|
|
|
|
s.replicaObserver = observers.NewReplicaObserver(
|
|
|
|
s.meta,
|
|
|
|
s.dist,
|
|
|
|
)
|
|
|
|
|
|
|
|
s.resourceObserver = observers.NewResourceObserver(s.meta)
|
2022-09-15 18:48:32 +08:00
|
|
|
}
|
|
|
|
|
2022-11-09 17:45:04 +08:00
|
|
|
func (s *Server) afterStart() {
|
|
|
|
}
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
func (s *Server) Start() error {
|
2023-01-30 11:11:50 +08:00
|
|
|
if !s.enableActiveStandBy {
|
|
|
|
if err := s.startQueryCoord(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
s.UpdateStateCode(commonpb.StateCode_Healthy)
|
|
|
|
log.Info("QueryCoord started")
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) startQueryCoord() error {
|
2022-09-15 18:48:32 +08:00
|
|
|
log.Info("start watcher...")
|
|
|
|
sessions, revision, err := s.session.GetSessions(typeutil.QueryNodeRole)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
for _, node := range sessions {
|
|
|
|
s.nodeMgr.Add(session.NewNodeInfo(node.ServerID, node.Address))
|
2022-11-30 13:57:15 +08:00
|
|
|
s.taskScheduler.AddExecutor(node.ServerID)
|
2022-09-15 18:48:32 +08:00
|
|
|
}
|
|
|
|
s.checkReplicas()
|
|
|
|
for _, node := range sessions {
|
|
|
|
s.handleNodeUp(node.ServerID)
|
|
|
|
}
|
|
|
|
s.wg.Add(1)
|
|
|
|
go s.watchNodes(revision)
|
|
|
|
|
|
|
|
log.Info("start recovering dist and target")
|
|
|
|
err = s.recover()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2023-01-30 11:11:50 +08:00
|
|
|
s.startServerLoop()
|
2022-12-28 10:49:29 +08:00
|
|
|
s.afterStart()
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) startServerLoop() {
|
2022-09-15 18:48:32 +08:00
|
|
|
log.Info("start cluster...")
|
|
|
|
s.cluster.Start(s.ctx)
|
|
|
|
|
|
|
|
log.Info("start job scheduler...")
|
|
|
|
s.jobScheduler.Start(s.ctx)
|
|
|
|
|
2022-09-21 14:42:51 +08:00
|
|
|
log.Info("start task scheduler...")
|
|
|
|
s.taskScheduler.Start(s.ctx)
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
log.Info("start checker controller...")
|
|
|
|
s.checkerController.Start(s.ctx)
|
|
|
|
|
|
|
|
log.Info("start observers...")
|
|
|
|
s.collectionObserver.Start(s.ctx)
|
|
|
|
s.leaderObserver.Start(s.ctx)
|
2022-11-07 19:37:04 +08:00
|
|
|
s.targetObserver.Start(s.ctx)
|
2023-01-30 10:19:48 +08:00
|
|
|
s.replicaObserver.Start(s.ctx)
|
|
|
|
s.resourceObserver.Start(s.ctx)
|
2022-09-15 18:48:32 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) Stop() error {
|
|
|
|
s.cancel()
|
2022-10-31 12:05:34 +08:00
|
|
|
if s.session != nil {
|
|
|
|
s.session.Revoke(time.Second)
|
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
|
2022-10-31 12:05:34 +08:00
|
|
|
if s.session != nil {
|
|
|
|
log.Info("stop cluster...")
|
|
|
|
s.cluster.Stop()
|
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
|
2022-10-31 12:05:34 +08:00
|
|
|
if s.distController != nil {
|
|
|
|
log.Info("stop dist controller...")
|
|
|
|
s.distController.Stop()
|
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
|
2022-10-31 12:05:34 +08:00
|
|
|
if s.checkerController != nil {
|
|
|
|
log.Info("stop checker controller...")
|
|
|
|
s.checkerController.Stop()
|
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
|
2022-10-31 12:05:34 +08:00
|
|
|
if s.taskScheduler != nil {
|
|
|
|
log.Info("stop task scheduler...")
|
|
|
|
s.taskScheduler.Stop()
|
|
|
|
}
|
2022-09-21 14:42:51 +08:00
|
|
|
|
2022-10-31 12:05:34 +08:00
|
|
|
if s.jobScheduler != nil {
|
|
|
|
log.Info("stop job scheduler...")
|
|
|
|
s.jobScheduler.Stop()
|
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
|
|
|
|
log.Info("stop observers...")
|
2022-10-31 12:05:34 +08:00
|
|
|
if s.collectionObserver != nil {
|
|
|
|
s.collectionObserver.Stop()
|
|
|
|
}
|
|
|
|
if s.leaderObserver != nil {
|
|
|
|
s.leaderObserver.Stop()
|
|
|
|
}
|
2022-11-07 19:37:04 +08:00
|
|
|
if s.targetObserver != nil {
|
|
|
|
s.targetObserver.Stop()
|
2022-10-31 12:05:34 +08:00
|
|
|
}
|
2023-01-30 10:19:48 +08:00
|
|
|
if s.replicaObserver != nil {
|
|
|
|
s.replicaObserver.Stop()
|
|
|
|
}
|
|
|
|
if s.resourceObserver != nil {
|
|
|
|
s.resourceObserver.Stop()
|
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
|
|
|
|
s.wg.Wait()
|
2022-10-31 12:05:34 +08:00
|
|
|
log.Info("QueryCoord stop successfully")
|
2022-09-15 18:48:32 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// UpdateStateCode updates the status of the coord, including healthy, unhealthy
|
2022-10-10 15:55:22 +08:00
|
|
|
func (s *Server) UpdateStateCode(code commonpb.StateCode) {
|
2022-09-15 18:48:32 +08:00
|
|
|
s.status.Store(code)
|
|
|
|
}
|
|
|
|
|
2022-10-10 15:55:22 +08:00
|
|
|
func (s *Server) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
|
2022-09-15 18:48:32 +08:00
|
|
|
nodeID := common.NotRegisteredID
|
|
|
|
if s.session != nil && s.session.Registered() {
|
|
|
|
nodeID = s.session.ServerID
|
|
|
|
}
|
2022-10-10 15:55:22 +08:00
|
|
|
serviceComponentInfo := &milvuspb.ComponentInfo{
|
2022-09-15 18:48:32 +08:00
|
|
|
// NodeID: Params.QueryCoordID, // will race with QueryCoord.Register()
|
|
|
|
NodeID: nodeID,
|
2022-10-10 15:55:22 +08:00
|
|
|
StateCode: s.status.Load().(commonpb.StateCode),
|
2022-09-15 18:48:32 +08:00
|
|
|
}
|
|
|
|
|
2022-10-10 15:55:22 +08:00
|
|
|
return &milvuspb.ComponentStates{
|
2022-09-15 18:48:32 +08:00
|
|
|
Status: &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_Success,
|
|
|
|
},
|
|
|
|
State: serviceComponentInfo,
|
|
|
|
//SubcomponentStates: subComponentInfos,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
|
|
|
|
return &milvuspb.StringResponse{
|
|
|
|
Status: &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_Success,
|
|
|
|
Reason: "",
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) GetTimeTickChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
|
|
|
|
return &milvuspb.StringResponse{
|
|
|
|
Status: &commonpb.Status{
|
|
|
|
ErrorCode: commonpb.ErrorCode_Success,
|
|
|
|
Reason: "",
|
|
|
|
},
|
2022-12-07 18:01:19 +08:00
|
|
|
Value: Params.CommonCfg.QueryCoordTimeTick.GetValue(),
|
2022-09-15 18:48:32 +08:00
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
func (s *Server) SetAddress(address string) {
|
|
|
|
s.address = address
|
|
|
|
}
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
// SetEtcdClient sets etcd's client
|
|
|
|
func (s *Server) SetEtcdClient(etcdClient *clientv3.Client) {
|
|
|
|
s.etcdCli = etcdClient
|
|
|
|
}
|
|
|
|
|
|
|
|
// SetRootCoord sets root coordinator's client
|
|
|
|
func (s *Server) SetRootCoord(rootCoord types.RootCoord) error {
|
|
|
|
if rootCoord == nil {
|
|
|
|
return errors.New("null RootCoord interface")
|
|
|
|
}
|
|
|
|
|
|
|
|
s.rootCoord = rootCoord
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// SetDataCoord sets data coordinator's client
|
|
|
|
func (s *Server) SetDataCoord(dataCoord types.DataCoord) error {
|
|
|
|
if dataCoord == nil {
|
|
|
|
return errors.New("null DataCoord interface")
|
|
|
|
}
|
|
|
|
|
|
|
|
s.dataCoord = dataCoord
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2023-01-12 19:49:40 +08:00
|
|
|
func (s *Server) SetQueryNodeCreator(f func(ctx context.Context, addr string) (types.QueryNode, error)) {
|
|
|
|
s.queryNodeCreator = f
|
|
|
|
}
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
func (s *Server) recover() error {
|
|
|
|
// Recover target managers
|
|
|
|
group, ctx := errgroup.WithContext(s.ctx)
|
|
|
|
for _, collection := range s.meta.GetAll() {
|
|
|
|
collection := collection
|
|
|
|
group.Go(func() error {
|
|
|
|
return s.recoverCollectionTargets(ctx, collection)
|
|
|
|
})
|
|
|
|
}
|
|
|
|
err := group.Wait()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Recover dist
|
|
|
|
s.distController.SyncAll(s.ctx)
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) recoverCollectionTargets(ctx context.Context, collection int64) error {
|
2022-11-07 19:37:04 +08:00
|
|
|
err := s.targetMgr.UpdateCollectionNextTarget(collection)
|
2022-09-15 18:48:32 +08:00
|
|
|
if err != nil {
|
2022-11-14 14:51:06 +08:00
|
|
|
s.meta.CollectionManager.RemoveCollection(collection)
|
|
|
|
s.meta.ReplicaManager.RemoveCollection(collection)
|
|
|
|
log.Error("failed to recover collection due to update next target failed",
|
|
|
|
zap.Int64("collectionID", collection),
|
|
|
|
zap.Error(err),
|
|
|
|
)
|
2022-09-15 18:48:32 +08:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) watchNodes(revision int64) {
|
|
|
|
defer s.wg.Done()
|
|
|
|
|
|
|
|
eventChan := s.session.WatchServices(typeutil.QueryNodeRole, revision+1, nil)
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-s.ctx.Done():
|
|
|
|
log.Info("stop watching nodes, QueryCoord stopped")
|
|
|
|
return
|
|
|
|
|
|
|
|
case event, ok := <-eventChan:
|
|
|
|
if !ok {
|
|
|
|
// ErrCompacted is handled inside SessionWatcher
|
|
|
|
log.Error("Session Watcher channel closed", zap.Int64("serverID", s.session.ServerID))
|
|
|
|
go s.Stop()
|
|
|
|
if s.session.TriggerKill {
|
|
|
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
|
|
|
p.Signal(syscall.SIGINT)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
switch event.EventType {
|
|
|
|
case sessionutil.SessionAddEvent:
|
|
|
|
nodeID := event.Session.ServerID
|
|
|
|
addr := event.Session.Address
|
|
|
|
log.Info("add node to NodeManager",
|
|
|
|
zap.Int64("nodeID", nodeID),
|
|
|
|
zap.String("nodeAddr", addr),
|
|
|
|
)
|
|
|
|
s.nodeMgr.Add(session.NewNodeInfo(nodeID, addr))
|
|
|
|
s.handleNodeUp(nodeID)
|
|
|
|
s.metricsCacheManager.InvalidateSystemInfoMetrics()
|
|
|
|
|
2022-12-06 22:59:19 +08:00
|
|
|
case sessionutil.SessionUpdateEvent:
|
|
|
|
nodeID := event.Session.ServerID
|
|
|
|
addr := event.Session.Address
|
|
|
|
log.Info("stopping the node",
|
|
|
|
zap.Int64("nodeID", nodeID),
|
|
|
|
zap.String("nodeAddr", addr),
|
|
|
|
)
|
|
|
|
s.nodeMgr.Stopping(nodeID)
|
|
|
|
s.checkerController.Check()
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
case sessionutil.SessionDelEvent:
|
|
|
|
nodeID := event.Session.ServerID
|
|
|
|
log.Info("a node down, remove it", zap.Int64("nodeID", nodeID))
|
|
|
|
s.nodeMgr.Remove(nodeID)
|
|
|
|
s.handleNodeDown(nodeID)
|
|
|
|
s.metricsCacheManager.InvalidateSystemInfoMetrics()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) handleNodeUp(node int64) {
|
|
|
|
log := log.With(zap.Int64("nodeID", node))
|
2022-11-30 13:57:15 +08:00
|
|
|
s.taskScheduler.AddExecutor(node)
|
2022-09-15 18:48:32 +08:00
|
|
|
s.distController.StartDistInstance(s.ctx, node)
|
|
|
|
|
2023-01-30 10:19:48 +08:00
|
|
|
// need assign to new rg and replica
|
|
|
|
rgName, err := s.meta.ResourceManager.HandleNodeUp(node)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("HandleNodeUp: failed to assign node to resource group",
|
|
|
|
zap.Error(err),
|
|
|
|
)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Info("HandleNodeUp: assign node to resource group",
|
|
|
|
zap.String("resourceGroup", rgName),
|
|
|
|
)
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
for _, collection := range s.meta.CollectionManager.GetAll() {
|
|
|
|
log := log.With(zap.Int64("collectionID", collection))
|
|
|
|
replica := s.meta.ReplicaManager.GetByCollectionAndNode(collection, node)
|
|
|
|
if replica == nil {
|
2023-01-30 10:19:48 +08:00
|
|
|
replicas := s.meta.ReplicaManager.GetByCollectionAndRG(collection, rgName)
|
|
|
|
if len(replicas) == 0 {
|
|
|
|
continue
|
|
|
|
}
|
2022-09-15 18:48:32 +08:00
|
|
|
sort.Slice(replicas, func(i, j int) bool {
|
2023-01-30 10:19:48 +08:00
|
|
|
return replicas[i].Len() < replicas[j].Len()
|
2022-09-15 18:48:32 +08:00
|
|
|
})
|
|
|
|
replica := replicas[0]
|
|
|
|
// TODO(yah01): this may fail, need a component to check whether a node is assigned
|
2023-01-30 10:19:48 +08:00
|
|
|
err = s.meta.ReplicaManager.AddNode(replica.GetID(), node)
|
2022-09-15 18:48:32 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to assign node to replicas",
|
|
|
|
zap.Int64("replicaID", replica.GetID()),
|
|
|
|
zap.Error(err),
|
|
|
|
)
|
|
|
|
}
|
|
|
|
log.Info("assign node to replica",
|
|
|
|
zap.Int64("replicaID", replica.GetID()))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) handleNodeDown(node int64) {
|
|
|
|
log := log.With(zap.Int64("nodeID", node))
|
2022-11-30 13:57:15 +08:00
|
|
|
s.taskScheduler.RemoveExecutor(node)
|
2022-09-15 18:48:32 +08:00
|
|
|
s.distController.Remove(node)
|
|
|
|
|
|
|
|
// Clear dist
|
|
|
|
s.dist.LeaderViewManager.Update(node)
|
|
|
|
s.dist.ChannelDistManager.Update(node)
|
|
|
|
s.dist.SegmentDistManager.Update(node)
|
|
|
|
|
|
|
|
// Clear meta
|
|
|
|
for _, collection := range s.meta.CollectionManager.GetAll() {
|
|
|
|
log := log.With(zap.Int64("collectionID", collection))
|
|
|
|
replica := s.meta.ReplicaManager.GetByCollectionAndNode(collection, node)
|
|
|
|
if replica == nil {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
err := s.meta.ReplicaManager.RemoveNode(replica.GetID(), node)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to remove node from collection's replicas",
|
|
|
|
zap.Int64("replicaID", replica.GetID()),
|
|
|
|
zap.Error(err),
|
|
|
|
)
|
|
|
|
}
|
|
|
|
log.Info("remove node from replica",
|
|
|
|
zap.Int64("replicaID", replica.GetID()))
|
|
|
|
}
|
|
|
|
|
|
|
|
// Clear tasks
|
|
|
|
s.taskScheduler.RemoveByNode(node)
|
2023-01-30 10:19:48 +08:00
|
|
|
|
|
|
|
rgName, err := s.meta.ResourceManager.HandleNodeDown(node)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("HandleNodeDown: failed to remove node from resource group",
|
|
|
|
zap.String("resourceGroup", rgName),
|
|
|
|
zap.Error(err),
|
|
|
|
)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Info("HandleNodeDown: remove node from resource group",
|
|
|
|
zap.String("resourceGroup", rgName),
|
|
|
|
)
|
2022-09-15 18:48:32 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// checkReplicas checks whether replica contains offline node, and remove those nodes
|
|
|
|
func (s *Server) checkReplicas() {
|
|
|
|
for _, collection := range s.meta.CollectionManager.GetAll() {
|
|
|
|
log := log.With(zap.Int64("collectionID", collection))
|
|
|
|
replicas := s.meta.ReplicaManager.GetByCollection(collection)
|
|
|
|
for _, replica := range replicas {
|
|
|
|
replica := replica.Clone()
|
|
|
|
toRemove := make([]int64, 0)
|
2023-01-30 10:19:48 +08:00
|
|
|
for _, node := range replica.GetNodes() {
|
2022-09-15 18:48:32 +08:00
|
|
|
if s.nodeMgr.Get(node) == nil {
|
|
|
|
toRemove = append(toRemove, node)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(toRemove) > 0 {
|
2022-09-29 18:02:54 +08:00
|
|
|
log := log.With(
|
|
|
|
zap.Int64("replicaID", replica.GetID()),
|
|
|
|
zap.Int64s("offlineNodes", toRemove),
|
|
|
|
)
|
2022-10-25 19:29:36 +08:00
|
|
|
log.Info("some nodes are offline, remove them from replica", zap.Any("toRemove", toRemove))
|
2022-09-15 18:48:32 +08:00
|
|
|
replica.RemoveNode(toRemove...)
|
|
|
|
err := s.meta.ReplicaManager.Put(replica)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to remove offline nodes from replica")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|