2021-10-25 19:44:37 +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
|
2021-04-19 11:35:38 +08:00
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
2021-10-25 19:44:37 +08:00
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
2021-04-19 11:35:38 +08:00
|
|
|
//
|
2021-10-25 19:44:37 +08:00
|
|
|
// 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-05-20 11:34:45 +08:00
|
|
|
|
2021-06-22 10:42:07 +08:00
|
|
|
package datacoord
|
2021-01-19 12:10:49 +08:00
|
|
|
|
2021-01-22 11:07:07 +08:00
|
|
|
import (
|
|
|
|
"context"
|
2021-08-19 13:00:12 +08:00
|
|
|
"errors"
|
2021-08-12 19:58:08 +08:00
|
|
|
"fmt"
|
2021-03-08 15:25:55 +08:00
|
|
|
"math/rand"
|
2022-03-17 17:17:22 +08:00
|
|
|
"os"
|
2021-01-23 20:22:59 +08:00
|
|
|
"sync"
|
2021-01-26 15:14:49 +08:00
|
|
|
"sync/atomic"
|
2021-11-22 16:23:17 +08:00
|
|
|
"syscall"
|
2021-01-26 15:14:49 +08:00
|
|
|
"time"
|
|
|
|
|
2022-09-02 10:34:59 +08:00
|
|
|
"github.com/blang/semver/v4"
|
2022-04-07 22:05:32 +08:00
|
|
|
clientv3 "go.etcd.io/etcd/client/v3"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
|
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"
|
2021-05-28 09:55:21 +08:00
|
|
|
datanodeclient "github.com/milvus-io/milvus/internal/distributed/datanode/client"
|
2021-06-18 21:30:08 +08:00
|
|
|
rootcoordclient "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
|
2021-04-22 14:45:57 +08:00
|
|
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
2022-03-15 21:51:21 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/metrics"
|
2022-03-03 21:57:56 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
|
|
|
"github.com/milvus-io/milvus/internal/mq/msgstream/mqwrapper"
|
2021-12-21 15:21:25 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2022-07-22 22:10:28 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/storage"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
2022-10-21 15:57:28 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/commonpbutil"
|
2022-04-07 22:05:32 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/dependency"
|
2022-06-30 17:26:19 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
2022-02-09 18:55:46 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/logutil"
|
2021-12-21 15:21:25 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
2021-12-23 18:39:11 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/paramtable"
|
2021-04-22 14:45:57 +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"
|
2022-01-05 14:04:46 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/timerecord"
|
2021-12-21 15:21:25 +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-22 11:07:07 +08:00
|
|
|
)
|
|
|
|
|
2021-11-13 08:45:09 +08:00
|
|
|
const (
|
2022-05-05 09:31:51 +08:00
|
|
|
connEtcdMaxRetryTime = 100
|
2021-11-13 08:45:09 +08:00
|
|
|
allPartitionID = 0 // paritionID means no filtering
|
|
|
|
)
|
2021-05-28 09:55:21 +08:00
|
|
|
|
2021-08-12 19:58:08 +08:00
|
|
|
var (
|
|
|
|
// TODO: sunby put to config
|
2021-10-20 15:02:36 +08:00
|
|
|
enableTtChecker = true
|
|
|
|
ttCheckerName = "dataTtChecker"
|
2022-01-27 11:25:41 +08:00
|
|
|
ttMaxInterval = 2 * time.Minute
|
|
|
|
ttCheckerWarnMsg = fmt.Sprintf("Datacoord haven't received tt for %f minutes", ttMaxInterval.Minutes())
|
2021-10-20 15:02:36 +08:00
|
|
|
segmentTimedFlushDuration = 10.0
|
2021-08-12 19:58:08 +08:00
|
|
|
)
|
|
|
|
|
2021-01-22 11:07:07 +08:00
|
|
|
type (
|
2021-09-08 11:35:59 +08:00
|
|
|
// UniqueID shortcut for typeutil.UniqueID
|
|
|
|
UniqueID = typeutil.UniqueID
|
|
|
|
// Timestamp shortcurt for typeutil.Timestamp
|
2021-01-22 19:43:27 +08:00
|
|
|
Timestamp = typeutil.Timestamp
|
2021-01-22 11:07:07 +08:00
|
|
|
)
|
2021-06-22 18:24:08 +08:00
|
|
|
|
2021-10-14 15:44:34 +08:00
|
|
|
type dataNodeCreatorFunc func(ctx context.Context, addr string) (types.DataNode, error)
|
2021-12-29 14:35:21 +08:00
|
|
|
type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdClient *clientv3.Client) (types.RootCoord, error)
|
2021-06-22 18:24:08 +08:00
|
|
|
|
2021-10-03 19:44:11 +08:00
|
|
|
// makes sure Server implements `DataCoord`
|
|
|
|
var _ types.DataCoord = (*Server)(nil)
|
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
var Params *paramtable.ComponentParam = paramtable.Get()
|
2021-12-23 18:39:11 +08:00
|
|
|
|
2022-04-06 15:33:32 +08:00
|
|
|
// Server implements `types.DataCoord`
|
|
|
|
// handles Data Coordinator related jobs
|
2021-03-05 20:41:34 +08:00
|
|
|
type Server struct {
|
2021-04-13 09:47:02 +08:00
|
|
|
ctx context.Context
|
|
|
|
serverLoopCtx context.Context
|
|
|
|
serverLoopCancel context.CancelFunc
|
|
|
|
serverLoopWg sync.WaitGroup
|
2021-11-22 16:23:17 +08:00
|
|
|
quitCh chan struct{}
|
2022-10-10 15:55:22 +08:00
|
|
|
stateCode atomic.Value
|
2021-08-16 11:00:09 +08:00
|
|
|
helper ServerHelper
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-12-29 14:35:21 +08:00
|
|
|
etcdCli *clientv3.Client
|
2022-11-04 14:25:38 +08:00
|
|
|
address string
|
2021-11-10 19:03:38 +08:00
|
|
|
kvClient *etcdkv.EtcdKV
|
|
|
|
meta *meta
|
|
|
|
segmentManager Manager
|
|
|
|
allocator allocator
|
|
|
|
cluster *Cluster
|
|
|
|
sessionManager *SessionManager
|
|
|
|
channelManager *ChannelManager
|
|
|
|
rootCoordClient types.RootCoord
|
|
|
|
garbageCollector *garbageCollector
|
|
|
|
gcOpt GcOption
|
2021-11-17 23:25:12 +08:00
|
|
|
handler Handler
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-11-05 22:25:00 +08:00
|
|
|
compactionTrigger trigger
|
|
|
|
compactionHandler compactionPlanContext
|
|
|
|
|
2021-09-03 17:15:26 +08:00
|
|
|
metricsCacheManager *metricsinfo.MetricsCacheManager
|
|
|
|
|
2022-04-07 22:05:32 +08:00
|
|
|
flushCh chan UniqueID
|
|
|
|
factory dependency.Factory
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2022-05-31 16:36:03 +08:00
|
|
|
session *sessionutil.Session
|
|
|
|
dnEventCh <-chan *sessionutil.SessionEvent
|
2022-07-07 14:44:21 +08:00
|
|
|
//icEventCh <-chan *sessionutil.SessionEvent
|
2022-05-31 16:36:03 +08:00
|
|
|
qcEventCh <-chan *sessionutil.SessionEvent
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2022-09-29 18:35:02 +08:00
|
|
|
enableActiveStandBy bool
|
|
|
|
activateFunc func()
|
|
|
|
|
2021-10-14 15:44:34 +08:00
|
|
|
dataNodeCreator dataNodeCreatorFunc
|
|
|
|
rootCoordClientCreator rootCoordCreatorFunc
|
2022-09-16 11:32:48 +08:00
|
|
|
indexCoord types.IndexCoord
|
2022-05-31 16:36:03 +08:00
|
|
|
|
|
|
|
segReferManager *SegmentReferenceManager
|
2021-03-05 20:41:34 +08:00
|
|
|
}
|
2021-01-22 11:07:07 +08:00
|
|
|
|
2021-09-06 17:02:41 +08:00
|
|
|
// ServerHelper datacoord server injection helper
|
2021-08-16 11:00:09 +08:00
|
|
|
type ServerHelper struct {
|
|
|
|
eventAfterHandleDataNodeTt func()
|
|
|
|
}
|
|
|
|
|
|
|
|
func defaultServerHelper() ServerHelper {
|
|
|
|
return ServerHelper{
|
|
|
|
eventAfterHandleDataNodeTt: func() {},
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-09-06 17:02:41 +08:00
|
|
|
// Option utility function signature to set DataCoord server attributes
|
2021-07-19 10:41:15 +08:00
|
|
|
type Option func(svr *Server)
|
|
|
|
|
2021-09-06 17:02:41 +08:00
|
|
|
// SetRootCoordCreator returns an `Option` setting RootCoord creator with provided parameter
|
2021-10-14 15:44:34 +08:00
|
|
|
func SetRootCoordCreator(creator rootCoordCreatorFunc) Option {
|
2021-07-19 10:41:15 +08:00
|
|
|
return func(svr *Server) {
|
|
|
|
svr.rootCoordClientCreator = creator
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-09-06 17:02:41 +08:00
|
|
|
// SetServerHelper returns an `Option` setting ServerHelp with provided parameter
|
2021-08-16 11:00:09 +08:00
|
|
|
func SetServerHelper(helper ServerHelper) Option {
|
|
|
|
return func(svr *Server) {
|
|
|
|
svr.helper = helper
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-09-06 17:02:41 +08:00
|
|
|
// SetCluster returns an `Option` setting Cluster with provided parameter
|
|
|
|
func SetCluster(cluster *Cluster) Option {
|
|
|
|
return func(svr *Server) {
|
|
|
|
svr.cluster = cluster
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-09-18 19:10:07 +08:00
|
|
|
// SetDataNodeCreator returns an `Option` setting DataNode create function
|
2021-10-14 15:44:34 +08:00
|
|
|
func SetDataNodeCreator(creator dataNodeCreatorFunc) Option {
|
2021-09-18 19:10:07 +08:00
|
|
|
return func(svr *Server) {
|
|
|
|
svr.dataNodeCreator = creator
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-12 00:22:42 +08:00
|
|
|
// SetSegmentManager returns an Option to set SegmentManager
|
|
|
|
func SetSegmentManager(manager Manager) Option {
|
|
|
|
return func(svr *Server) {
|
|
|
|
svr.segmentManager = manager
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-12-20 19:54:58 +08:00
|
|
|
// CreateServer creates a `Server` instance
|
2022-04-07 22:05:32 +08:00
|
|
|
func CreateServer(ctx context.Context, factory dependency.Factory, opts ...Option) *Server {
|
2021-03-08 15:25:55 +08:00
|
|
|
rand.Seed(time.Now().UnixNano())
|
2021-01-26 15:14:49 +08:00
|
|
|
s := &Server{
|
2021-06-22 18:24:08 +08:00
|
|
|
ctx: ctx,
|
2021-11-22 16:23:17 +08:00
|
|
|
quitCh: make(chan struct{}),
|
2022-04-07 22:05:32 +08:00
|
|
|
factory: factory,
|
2021-06-22 18:24:08 +08:00
|
|
|
flushCh: make(chan UniqueID, 1024),
|
2021-09-18 19:10:07 +08:00
|
|
|
dataNodeCreator: defaultDataNodeCreatorFunc,
|
2021-06-22 18:24:08 +08:00
|
|
|
rootCoordClientCreator: defaultRootCoordCreatorFunc,
|
2021-08-16 11:00:09 +08:00
|
|
|
helper: defaultServerHelper(),
|
2022-09-29 18:35:02 +08:00
|
|
|
metricsCacheManager: metricsinfo.NewMetricsCacheManager(),
|
|
|
|
enableActiveStandBy: Params.DataCoordCfg.EnableActiveStandby,
|
2021-01-26 15:14:49 +08:00
|
|
|
}
|
2021-07-19 10:41:15 +08:00
|
|
|
|
|
|
|
for _, opt := range opts {
|
|
|
|
opt(s)
|
|
|
|
}
|
2021-12-29 14:35:21 +08:00
|
|
|
return s
|
2021-01-26 15:14:49 +08:00
|
|
|
}
|
|
|
|
|
2021-06-24 19:05:06 +08:00
|
|
|
func defaultDataNodeCreatorFunc(ctx context.Context, addr string) (types.DataNode, error) {
|
|
|
|
return datanodeclient.NewClient(ctx, addr)
|
2021-06-22 18:24:08 +08:00
|
|
|
}
|
|
|
|
|
2021-12-29 14:35:21 +08:00
|
|
|
func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, client *clientv3.Client) (types.RootCoord, error) {
|
|
|
|
return rootcoordclient.NewClient(ctx, metaRootPath, client)
|
2021-06-22 18:24:08 +08:00
|
|
|
}
|
|
|
|
|
2021-11-22 16:23:17 +08:00
|
|
|
// QuitSignal returns signal when server quits
|
|
|
|
func (s *Server) QuitSignal() <-chan struct{} {
|
|
|
|
return s.quitCh
|
|
|
|
}
|
|
|
|
|
2021-12-20 19:54:58 +08:00
|
|
|
// Register registers data service at etcd
|
2021-05-25 15:06:05 +08:00
|
|
|
func (s *Server) Register() error {
|
2021-12-15 11:47:10 +08:00
|
|
|
s.session.Register()
|
2022-09-29 18:35:02 +08:00
|
|
|
if s.enableActiveStandBy {
|
|
|
|
s.session.ProcessActiveStandBy(s.activateFunc)
|
|
|
|
}
|
2021-12-15 11:47:10 +08:00
|
|
|
go s.session.LivenessCheck(s.serverLoopCtx, func() {
|
2021-12-28 19:42:19 +08:00
|
|
|
logutil.Logger(s.ctx).Error("disconnected from etcd and exited", zap.Int64("serverID", s.session.ServerID))
|
2021-12-15 11:47:10 +08:00
|
|
|
if err := s.Stop(); err != nil {
|
2021-12-28 19:42:19 +08:00
|
|
|
logutil.Logger(s.ctx).Fatal("failed to stop server", zap.Error(err))
|
2021-12-15 11:47:10 +08:00
|
|
|
}
|
|
|
|
// manually send signal to starter goroutine
|
2021-12-29 14:35:21 +08:00
|
|
|
if s.session.TriggerKill {
|
2022-03-17 17:17:22 +08:00
|
|
|
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
|
|
|
p.Signal(syscall.SIGINT)
|
|
|
|
}
|
2021-12-29 14:35:21 +08:00
|
|
|
}
|
2021-12-15 11:47:10 +08:00
|
|
|
})
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) initSession() error {
|
2022-02-07 10:09:45 +08:00
|
|
|
s.session = sessionutil.NewSession(s.ctx, Params.EtcdCfg.MetaRootPath, s.etcdCli)
|
2021-09-09 10:06:29 +08:00
|
|
|
if s.session == nil {
|
|
|
|
return errors.New("failed to initialize session")
|
|
|
|
}
|
2022-11-04 14:25:38 +08:00
|
|
|
s.session.Init(typeutil.DataCoordRole, s.address, true, true)
|
2022-09-29 18:35:02 +08:00
|
|
|
s.session.SetEnableActiveStandBy(s.enableActiveStandBy)
|
2021-05-25 15:06:05 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-06-29 10:46:13 +08:00
|
|
|
// Init change server state to Initializing
|
2021-05-25 15:06:05 +08:00
|
|
|
func (s *Server) Init() error {
|
2022-09-27 18:04:53 +08:00
|
|
|
var err error
|
2022-10-10 15:55:22 +08:00
|
|
|
s.stateCode.Store(commonpb.StateCode_Initializing)
|
2022-11-04 14:25:38 +08:00
|
|
|
s.factory.Init(Params)
|
2021-01-22 19:43:27 +08:00
|
|
|
|
2021-06-21 17:28:03 +08:00
|
|
|
if err = s.initRootCoordClient(); err != nil {
|
2021-05-28 09:55:21 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2022-09-25 15:56:51 +08:00
|
|
|
storageCli, err := s.newChunkManagerFactory()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
if err = s.initMeta(storageCli.RootPath()); err != nil {
|
2021-05-28 09:55:21 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-11-17 23:25:12 +08:00
|
|
|
s.handler = newServerHandler(s)
|
|
|
|
|
2021-05-28 09:55:21 +08:00
|
|
|
if err = s.initCluster(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-08-23 17:59:51 +08:00
|
|
|
s.allocator = newRootCoordAllocator(s.rootCoordClient)
|
2022-06-20 21:56:12 +08:00
|
|
|
|
2022-09-27 18:04:53 +08:00
|
|
|
if err = s.initSession(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2022-06-20 21:56:12 +08:00
|
|
|
if err = s.initServiceDiscovery(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2021-12-23 18:39:11 +08:00
|
|
|
if Params.DataCoordCfg.EnableCompaction {
|
2021-11-05 22:25:00 +08:00
|
|
|
s.createCompactionHandler()
|
|
|
|
s.createCompactionTrigger()
|
|
|
|
}
|
2022-09-27 18:04:53 +08:00
|
|
|
s.initSegmentManager()
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2022-09-25 15:56:51 +08:00
|
|
|
s.initGarbageCollection(storageCli)
|
2021-11-10 19:03:38 +08:00
|
|
|
|
2022-09-27 18:04:53 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Start initialize `Server` members and start loops, follow steps are taken:
|
2022-10-13 19:21:23 +08:00
|
|
|
// 1. initialize message factory parameters
|
|
|
|
// 2. initialize root coord client, meta, datanode cluster, segment info channel,
|
|
|
|
// allocator, segment manager
|
|
|
|
// 3. start service discovery and server loops, which includes message stream handler (segment statistics,datanode tt)
|
|
|
|
// datanodes etcd watch, etcd alive check and flush completed status check
|
|
|
|
// 4. set server state to Healthy
|
2022-09-27 18:04:53 +08:00
|
|
|
func (s *Server) Start() error {
|
2022-11-15 14:19:06 +08:00
|
|
|
if Params.DataCoordCfg.EnableCompaction {
|
|
|
|
s.compactionHandler.start()
|
|
|
|
s.compactionTrigger.start()
|
|
|
|
}
|
2022-09-27 18:04:53 +08:00
|
|
|
|
2022-09-29 18:35:02 +08:00
|
|
|
if s.enableActiveStandBy {
|
|
|
|
s.activateFunc = func() {
|
|
|
|
// todo complete the activateFunc
|
|
|
|
log.Info("datacoord switch from standby to active, activating")
|
|
|
|
s.startServerLoop()
|
2022-10-10 15:55:22 +08:00
|
|
|
s.stateCode.Store(commonpb.StateCode_Healthy)
|
2022-10-25 19:31:30 +08:00
|
|
|
logutil.Logger(s.ctx).Info("startup success")
|
2022-09-29 18:35:02 +08:00
|
|
|
}
|
2022-10-10 15:55:22 +08:00
|
|
|
s.stateCode.Store(commonpb.StateCode_StandBy)
|
2022-10-25 19:31:30 +08:00
|
|
|
logutil.Logger(s.ctx).Info("DataCoord enter standby mode successfully")
|
2022-09-29 18:35:02 +08:00
|
|
|
} else {
|
|
|
|
s.startServerLoop()
|
2022-10-10 15:55:22 +08:00
|
|
|
s.stateCode.Store(commonpb.StateCode_Healthy)
|
2022-10-25 19:31:30 +08:00
|
|
|
logutil.Logger(s.ctx).Info("DataCoord startup successfully")
|
2022-09-29 18:35:02 +08:00
|
|
|
}
|
|
|
|
|
2021-12-23 18:39:11 +08:00
|
|
|
Params.DataCoordCfg.CreatedTime = time.Now()
|
|
|
|
Params.DataCoordCfg.UpdatedTime = time.Now()
|
2021-09-26 17:50:07 +08:00
|
|
|
|
2022-05-25 14:34:00 +08:00
|
|
|
// DataCoord (re)starts successfully and starts to collection segment stats
|
|
|
|
// data from all DataNode.
|
|
|
|
// This will prevent DataCoord from missing out any important segment stats
|
|
|
|
// data while offline.
|
2022-08-19 19:50:50 +08:00
|
|
|
log.Info("DataCoord (re)starts successfully and re-collecting segment stats from DataNodes")
|
2022-05-25 14:34:00 +08:00
|
|
|
s.reCollectSegmentStats(s.ctx)
|
|
|
|
|
2021-05-28 09:55:21 +08:00
|
|
|
return nil
|
2021-05-26 19:06:56 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) initCluster() error {
|
2021-10-14 15:44:34 +08:00
|
|
|
if s.cluster != nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-07-12 11:03:52 +08:00
|
|
|
var err error
|
2022-04-07 22:05:32 +08:00
|
|
|
s.channelManager, err = NewChannelManager(s.kvClient, s.handler, withMsgstreamFactory(s.factory), withStateChecker())
|
2021-10-14 15:44:34 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
2021-09-06 17:02:41 +08:00
|
|
|
}
|
2021-11-05 22:25:00 +08:00
|
|
|
s.sessionManager = NewSessionManager(withSessionCreator(s.dataNodeCreator))
|
|
|
|
s.cluster = NewCluster(s.sessionManager, s.channelManager)
|
2021-10-14 15:44:34 +08:00
|
|
|
return nil
|
2021-05-26 19:06:56 +08:00
|
|
|
}
|
2021-02-08 14:30:54 +08:00
|
|
|
|
2022-11-04 14:25:38 +08:00
|
|
|
func (s *Server) SetAddress(address string) {
|
|
|
|
s.address = address
|
|
|
|
}
|
|
|
|
|
2021-12-29 14:35:21 +08:00
|
|
|
// SetEtcdClient sets etcd client for datacoord.
|
|
|
|
func (s *Server) SetEtcdClient(client *clientv3.Client) {
|
|
|
|
s.etcdCli = client
|
|
|
|
}
|
|
|
|
|
2022-09-16 11:32:48 +08:00
|
|
|
func (s *Server) SetIndexCoord(indexCoord types.IndexCoord) {
|
|
|
|
s.indexCoord = indexCoord
|
|
|
|
}
|
|
|
|
|
2021-11-05 22:25:00 +08:00
|
|
|
func (s *Server) createCompactionHandler() {
|
2022-06-20 21:56:12 +08:00
|
|
|
s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh, s.segReferManager)
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) stopCompactionHandler() {
|
|
|
|
s.compactionHandler.stop()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) createCompactionTrigger() {
|
2022-10-11 21:39:24 +08:00
|
|
|
s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator, s.segReferManager, s.indexCoord, s.handler)
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) stopCompactionTrigger() {
|
|
|
|
s.compactionTrigger.stop()
|
|
|
|
}
|
|
|
|
|
2022-09-25 15:56:51 +08:00
|
|
|
func (s *Server) newChunkManagerFactory() (storage.ChunkManager, error) {
|
2022-11-04 14:25:38 +08:00
|
|
|
chunkManagerFactory := storage.NewChunkManagerFactoryWithParam(Params)
|
2022-09-25 15:56:51 +08:00
|
|
|
cli, err := chunkManagerFactory.NewPersistentStorageChunkManager(s.ctx)
|
2022-09-23 14:40:51 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Error("chunk manager init failed", zap.Error(err))
|
2022-09-25 15:56:51 +08:00
|
|
|
return nil, err
|
2022-09-23 14:40:51 +08:00
|
|
|
}
|
2022-09-25 15:56:51 +08:00
|
|
|
return cli, err
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) initGarbageCollection(cli storage.ChunkManager) {
|
2022-10-11 21:39:24 +08:00
|
|
|
s.garbageCollector = newGarbageCollector(s.meta, s.handler, s.segReferManager, s.indexCoord, GcOption{
|
2022-09-23 14:40:51 +08:00
|
|
|
cli: cli,
|
|
|
|
enabled: Params.DataCoordCfg.EnableGarbageCollection,
|
2021-12-23 18:39:11 +08:00
|
|
|
checkInterval: Params.DataCoordCfg.GCInterval,
|
|
|
|
missingTolerance: Params.DataCoordCfg.GCMissingTolerance,
|
|
|
|
dropTolerance: Params.DataCoordCfg.GCDropTolerance,
|
2021-11-10 19:03:38 +08:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2021-05-26 19:06:56 +08:00
|
|
|
func (s *Server) initServiceDiscovery() error {
|
2022-09-02 10:34:59 +08:00
|
|
|
r := semver.MustParseRange(">=2.1.2")
|
|
|
|
sessions, rev, err := s.session.GetSessionsWithVersionRange(typeutil.DataNodeRole, r)
|
2021-05-26 19:06:56 +08:00
|
|
|
if err != nil {
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Warn("DataCoord failed to init service discovery", zap.Error(err))
|
2021-05-19 18:36:05 +08:00
|
|
|
return err
|
|
|
|
}
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Info("DataCoord success to get DataNode sessions", zap.Any("sessions", sessions))
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-07-12 11:03:52 +08:00
|
|
|
datanodes := make([]*NodeInfo, 0, len(sessions))
|
2021-05-26 19:06:56 +08:00
|
|
|
for _, session := range sessions {
|
2021-10-14 15:44:34 +08:00
|
|
|
info := &NodeInfo{
|
|
|
|
NodeID: session.ServerID,
|
|
|
|
Address: session.Address,
|
2021-07-12 11:03:52 +08:00
|
|
|
}
|
2021-10-14 15:44:34 +08:00
|
|
|
datanodes = append(datanodes, info)
|
2021-05-26 19:06:56 +08:00
|
|
|
}
|
2021-05-19 18:36:05 +08:00
|
|
|
|
2022-03-28 22:33:27 +08:00
|
|
|
s.cluster.Startup(s.ctx, datanodes)
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2022-02-15 15:07:48 +08:00
|
|
|
// TODO implement rewatch logic
|
2022-09-02 10:34:59 +08:00
|
|
|
s.dnEventCh = s.session.WatchServicesWithVersionRange(typeutil.DataNodeRole, r, rev+1, nil)
|
2022-05-31 16:36:03 +08:00
|
|
|
|
2022-07-07 14:44:21 +08:00
|
|
|
//icSessions, icRevision, err := s.session.GetSessions(typeutil.IndexCoordRole)
|
|
|
|
//if err != nil {
|
|
|
|
// log.Error("DataCoord get IndexCoord session failed", zap.Error(err))
|
|
|
|
// return err
|
|
|
|
//}
|
|
|
|
//serverIDs := make([]UniqueID, 0, len(icSessions))
|
|
|
|
//for _, session := range icSessions {
|
|
|
|
// serverIDs = append(serverIDs, session.ServerID)
|
|
|
|
//}
|
|
|
|
//s.icEventCh = s.session.WatchServices(typeutil.IndexCoordRole, icRevision+1, nil)
|
2022-05-31 16:36:03 +08:00
|
|
|
|
|
|
|
qcSessions, qcRevision, err := s.session.GetSessions(typeutil.QueryCoordRole)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("DataCoord get QueryCoord session failed", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
2022-07-07 14:44:21 +08:00
|
|
|
serverIDs := make([]UniqueID, 0, len(qcSessions))
|
2022-05-31 16:36:03 +08:00
|
|
|
for _, session := range qcSessions {
|
|
|
|
serverIDs = append(serverIDs, session.ServerID)
|
|
|
|
}
|
|
|
|
s.qcEventCh = s.session.WatchServices(typeutil.QueryCoordRole, qcRevision+1, nil)
|
|
|
|
|
|
|
|
s.segReferManager, err = NewSegmentReferenceManager(s.kvClient, serverIDs)
|
|
|
|
return err
|
2021-01-22 19:43:27 +08:00
|
|
|
}
|
|
|
|
|
2022-09-27 18:04:53 +08:00
|
|
|
func (s *Server) initSegmentManager() {
|
2021-11-12 00:22:42 +08:00
|
|
|
if s.segmentManager == nil {
|
2022-06-27 13:56:17 +08:00
|
|
|
s.segmentManager = newSegmentManager(s.meta, s.allocator, s.rootCoordClient)
|
2021-11-12 00:22:42 +08:00
|
|
|
}
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
2022-09-25 15:56:51 +08:00
|
|
|
func (s *Server) initMeta(chunkManagerRootPath string) error {
|
2022-02-07 10:09:45 +08:00
|
|
|
etcdKV := etcdkv.NewEtcdKV(s.etcdCli, Params.EtcdCfg.MetaRootPath)
|
2021-12-29 14:35:21 +08:00
|
|
|
s.kvClient = etcdKV
|
|
|
|
reloadEtcdFn := func() error {
|
|
|
|
var err error
|
2022-09-25 15:56:51 +08:00
|
|
|
s.meta, err = newMeta(s.ctx, s.kvClient, chunkManagerRootPath)
|
2021-02-26 15:17:47 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
2021-01-22 11:07:07 +08:00
|
|
|
}
|
2021-12-29 14:35:21 +08:00
|
|
|
return retry.Do(s.ctx, reloadEtcdFn, retry.Attempts(connEtcdMaxRetryTime))
|
2021-01-22 11:07:07 +08:00
|
|
|
}
|
|
|
|
|
2021-01-26 09:43:41 +08:00
|
|
|
func (s *Server) startServerLoop() {
|
|
|
|
s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx)
|
2021-12-15 10:53:16 +08:00
|
|
|
s.serverLoopWg.Add(3)
|
2021-10-29 21:30:49 +08:00
|
|
|
s.startDataNodeTtLoop(s.serverLoopCtx)
|
|
|
|
s.startWatchService(s.serverLoopCtx)
|
|
|
|
s.startFlushLoop(s.serverLoopCtx)
|
2021-11-10 19:03:38 +08:00
|
|
|
s.garbageCollector.start()
|
2021-01-26 09:43:41 +08:00
|
|
|
}
|
|
|
|
|
2021-11-05 23:01:02 +08:00
|
|
|
// startDataNodeTtLoop start a goroutine to recv data node tt msg from msgstream
|
|
|
|
// tt msg stands for the currently consumed timestamp for each channel
|
2021-05-25 15:35:37 +08:00
|
|
|
func (s *Server) startDataNodeTtLoop(ctx context.Context) {
|
2022-04-07 22:05:32 +08:00
|
|
|
ttMsgStream, err := s.factory.NewMsgStream(ctx)
|
2021-05-25 15:35:37 +08:00
|
|
|
if err != nil {
|
2021-12-20 11:18:49 +08:00
|
|
|
log.Error("DataCoord failed to create timetick channel", zap.Error(err))
|
2022-10-25 13:23:30 +08:00
|
|
|
panic(err)
|
2021-05-25 15:35:37 +08:00
|
|
|
}
|
2022-11-08 20:15:04 +08:00
|
|
|
subName := fmt.Sprintf("%s-%d-datanodeTl", Params.CommonCfg.DataCoordSubName, paramtable.GetNodeID())
|
2022-10-25 13:23:30 +08:00
|
|
|
ttMsgStream.AsConsumer([]string{Params.CommonCfg.DataCoordTimeTick},
|
2022-11-08 20:15:04 +08:00
|
|
|
subName, mqwrapper.SubscriptionPositionLatest)
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Info("DataCoord creates the timetick channel consumer",
|
2022-03-04 11:17:56 +08:00
|
|
|
zap.String("timeTickChannel", Params.CommonCfg.DataCoordTimeTick),
|
2022-11-08 20:15:04 +08:00
|
|
|
zap.String("subscription", subName))
|
2021-05-25 15:35:37 +08:00
|
|
|
ttMsgStream.Start()
|
2021-08-12 19:58:08 +08:00
|
|
|
|
2022-02-23 11:09:52 +08:00
|
|
|
go s.handleDataNodeTimetickMsgstream(ctx, ttMsgStream)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) handleDataNodeTimetickMsgstream(ctx context.Context, ttMsgStream msgstream.MsgStream) {
|
|
|
|
var checker *timerecord.LongTermChecker
|
|
|
|
if enableTtChecker {
|
|
|
|
checker = timerecord.NewLongTermChecker(ctx, ttCheckerName, ttMaxInterval, ttCheckerWarnMsg)
|
|
|
|
checker.Start()
|
|
|
|
defer checker.Stop()
|
|
|
|
}
|
2021-12-15 10:53:16 +08:00
|
|
|
|
2022-02-23 11:09:52 +08:00
|
|
|
defer logutil.LogPanic()
|
|
|
|
defer s.serverLoopWg.Done()
|
|
|
|
defer func() {
|
|
|
|
// https://github.com/milvus-io/milvus/issues/15659
|
|
|
|
// msgstream service closed before datacoord quits
|
|
|
|
defer func() {
|
|
|
|
if x := recover(); x != nil {
|
|
|
|
log.Error("Failed to close ttMessage", zap.Any("recovered", x))
|
2021-01-26 09:43:41 +08:00
|
|
|
}
|
2022-02-23 11:09:52 +08:00
|
|
|
}()
|
|
|
|
ttMsgStream.Close()
|
|
|
|
}()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Info("DataNode timetick loop shutdown")
|
2022-02-23 11:09:52 +08:00
|
|
|
return
|
2022-03-11 20:09:59 +08:00
|
|
|
case msgPack, ok := <-ttMsgStream.Chan():
|
|
|
|
if !ok || msgPack == nil || len(msgPack.Msgs) == 0 {
|
|
|
|
log.Info("receive nil timetick msg and shutdown timetick channel")
|
|
|
|
return
|
2021-08-12 19:58:08 +08:00
|
|
|
}
|
2021-05-25 15:35:37 +08:00
|
|
|
|
2022-03-11 20:09:59 +08:00
|
|
|
for _, msg := range msgPack.Msgs {
|
|
|
|
ttMsg, ok := msg.(*msgstream.DataNodeTtMsg)
|
|
|
|
if !ok {
|
|
|
|
log.Warn("receive unexpected msg type from tt channel")
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
if enableTtChecker {
|
|
|
|
checker.Check()
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := s.handleTimetickMessage(ctx, ttMsg); err != nil {
|
|
|
|
log.Error("failed to handle timetick message", zap.Error(err))
|
|
|
|
continue
|
|
|
|
}
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
2022-03-11 20:09:59 +08:00
|
|
|
s.helper.eventAfterHandleDataNodeTt()
|
2021-05-26 19:06:56 +08:00
|
|
|
}
|
2022-02-23 11:09:52 +08:00
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
}
|
|
|
|
|
2021-12-15 10:53:16 +08:00
|
|
|
func (s *Server) handleTimetickMessage(ctx context.Context, ttMsg *msgstream.DataNodeTtMsg) error {
|
|
|
|
ch := ttMsg.GetChannelName()
|
|
|
|
ts := ttMsg.GetTimestamp()
|
|
|
|
physical, _ := tsoutil.ParseTS(ts)
|
|
|
|
if time.Since(physical).Minutes() > 1 {
|
|
|
|
// if lag behind, log every 1 mins about
|
|
|
|
log.RatedWarn(60.0, "time tick lag behind for more than 1 minutes", zap.String("channel", ch), zap.Time("timetick", physical))
|
|
|
|
}
|
|
|
|
|
2022-11-07 10:15:02 +08:00
|
|
|
sub := tsoutil.SubByNow(ts)
|
2022-06-30 17:26:19 +08:00
|
|
|
pChannelName := funcutil.ToPhysicalChannel(ch)
|
2022-11-07 10:15:02 +08:00
|
|
|
metrics.DataCoordConsumeDataNodeTimeTickLag.
|
|
|
|
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), pChannelName).
|
|
|
|
Set(float64(sub))
|
2022-03-15 21:51:21 +08:00
|
|
|
|
2021-12-15 10:53:16 +08:00
|
|
|
s.updateSegmentStatistics(ttMsg.GetSegmentsStats())
|
|
|
|
|
|
|
|
if err := s.segmentManager.ExpireAllocations(ch, ts); err != nil {
|
|
|
|
return fmt.Errorf("expire allocations: %w", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
flushableIDs, err := s.segmentManager.GetFlushableSegments(ctx, ch, ts)
|
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("get flushable segments: %w", err)
|
|
|
|
}
|
|
|
|
flushableSegments := s.getFlushableSegmentsInfo(flushableIDs)
|
|
|
|
|
2022-11-10 22:13:04 +08:00
|
|
|
if len(flushableSegments) == 0 {
|
2021-12-15 10:53:16 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2022-04-25 11:07:47 +08:00
|
|
|
log.Info("start flushing segments",
|
2022-11-10 22:13:04 +08:00
|
|
|
zap.Int64s("segment IDs", flushableIDs))
|
2022-10-22 12:09:28 +08:00
|
|
|
// update segment last update triggered time
|
|
|
|
// it's ok to fail flushing, since next timetick after duration will re-trigger
|
2021-12-15 10:53:16 +08:00
|
|
|
s.setLastFlushTime(flushableSegments)
|
|
|
|
|
2022-11-10 22:13:04 +08:00
|
|
|
finfo := make([]*datapb.SegmentInfo, 0, len(flushableSegments))
|
2021-12-15 10:53:16 +08:00
|
|
|
for _, info := range flushableSegments {
|
|
|
|
finfo = append(finfo, info.SegmentInfo)
|
|
|
|
}
|
2022-11-10 22:13:04 +08:00
|
|
|
err = s.cluster.Flush(s.ctx, ttMsg.GetBase().GetSourceID(), ch, finfo)
|
2022-10-22 12:09:28 +08:00
|
|
|
if err != nil {
|
|
|
|
log.Warn("handle")
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2021-12-15 10:53:16 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) updateSegmentStatistics(stats []*datapb.SegmentStats) {
|
|
|
|
for _, stat := range stats {
|
2022-08-19 19:50:50 +08:00
|
|
|
// Log if # of rows is updated.
|
2022-10-15 15:13:24 +08:00
|
|
|
if s.meta.GetSegmentUnsafe(stat.GetSegmentID()) != nil &&
|
|
|
|
s.meta.GetSegmentUnsafe(stat.GetSegmentID()).GetNumOfRows() != stat.GetNumRows() {
|
2022-10-25 19:31:30 +08:00
|
|
|
log.Info("Updating segment number of rows",
|
2022-08-19 19:50:50 +08:00
|
|
|
zap.Int64("segment ID", stat.GetSegmentID()),
|
2022-10-15 15:13:24 +08:00
|
|
|
zap.Int64("old value", s.meta.GetSegmentUnsafe(stat.GetSegmentID()).GetNumOfRows()),
|
2022-08-19 19:50:50 +08:00
|
|
|
zap.Int64("new value", stat.GetNumRows()),
|
|
|
|
)
|
|
|
|
}
|
2021-12-15 10:53:16 +08:00
|
|
|
s.meta.SetCurrentRows(stat.GetSegmentID(), stat.GetNumRows())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) getFlushableSegmentsInfo(flushableIDs []int64) []*SegmentInfo {
|
|
|
|
res := make([]*SegmentInfo, 0, len(flushableIDs))
|
|
|
|
for _, id := range flushableIDs {
|
|
|
|
sinfo := s.meta.GetSegment(id)
|
|
|
|
if sinfo == nil {
|
|
|
|
log.Error("get segment from meta error", zap.Int64("id", id))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
res = append(res, sinfo)
|
|
|
|
}
|
|
|
|
return res
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) setLastFlushTime(segments []*SegmentInfo) {
|
|
|
|
for _, sinfo := range segments {
|
|
|
|
s.meta.SetLastFlushTime(sinfo.GetID(), time.Now())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-29 21:30:49 +08:00
|
|
|
// start a goroutine wto watch services
|
2021-05-26 19:06:56 +08:00
|
|
|
func (s *Server) startWatchService(ctx context.Context) {
|
2021-10-29 21:30:49 +08:00
|
|
|
go s.watchService(ctx)
|
|
|
|
}
|
|
|
|
|
2022-06-15 12:20:10 +08:00
|
|
|
func (s *Server) stopServiceWatch() {
|
|
|
|
// ErrCompacted is handled inside SessionWatcher, which means there is some other error occurred, closing server.
|
|
|
|
logutil.Logger(s.ctx).Error("watch service 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)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) processSessionEvent(ctx context.Context, role string, event *sessionutil.SessionEvent) {
|
|
|
|
switch event.EventType {
|
|
|
|
case sessionutil.SessionAddEvent:
|
|
|
|
log.Info("there is a new service online",
|
|
|
|
zap.String("server role", role),
|
|
|
|
zap.Int64("server ID", event.Session.ServerID))
|
|
|
|
|
|
|
|
case sessionutil.SessionDelEvent:
|
|
|
|
log.Warn("there is service offline",
|
|
|
|
zap.String("server role", role),
|
|
|
|
zap.Int64("server ID", event.Session.ServerID))
|
|
|
|
if err := retry.Do(ctx, func() error {
|
|
|
|
return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID)
|
|
|
|
}, retry.Attempts(100)); err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-02-22 13:15:51 +08:00
|
|
|
// watchService watches services.
|
2021-10-29 21:30:49 +08:00
|
|
|
func (s *Server) watchService(ctx context.Context) {
|
2021-05-28 09:55:21 +08:00
|
|
|
defer logutil.LogPanic()
|
2021-05-26 19:06:56 +08:00
|
|
|
defer s.serverLoopWg.Done()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Info("watch service shutdown")
|
2021-05-26 19:06:56 +08:00
|
|
|
return
|
2022-05-31 16:36:03 +08:00
|
|
|
case event, ok := <-s.dnEventCh:
|
2021-10-14 19:20:35 +08:00
|
|
|
if !ok {
|
2022-06-15 12:20:10 +08:00
|
|
|
s.stopServiceWatch()
|
2021-10-14 19:20:35 +08:00
|
|
|
return
|
|
|
|
}
|
2021-10-14 15:44:34 +08:00
|
|
|
if err := s.handleSessionEvent(ctx, event); err != nil {
|
|
|
|
go func() {
|
|
|
|
if err := s.Stop(); err != nil {
|
2022-06-15 12:20:10 +08:00
|
|
|
log.Warn("DataCoord server stop error", zap.Error(err))
|
2021-10-14 15:44:34 +08:00
|
|
|
}
|
|
|
|
}()
|
|
|
|
return
|
|
|
|
}
|
2022-07-07 14:44:21 +08:00
|
|
|
//case event, ok := <-s.icEventCh:
|
|
|
|
// if !ok {
|
|
|
|
// s.stopServiceWatch()
|
|
|
|
// return
|
|
|
|
// }
|
|
|
|
// s.processSessionEvent(ctx, "IndexCoord", event)
|
2022-05-31 16:36:03 +08:00
|
|
|
case event, ok := <-s.qcEventCh:
|
|
|
|
if !ok {
|
2022-06-15 12:20:10 +08:00
|
|
|
s.stopServiceWatch()
|
2022-05-31 16:36:03 +08:00
|
|
|
return
|
|
|
|
}
|
2022-06-15 12:20:10 +08:00
|
|
|
s.processSessionEvent(ctx, "QueryCoord", event)
|
2021-05-26 19:06:56 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-05-25 15:35:37 +08:00
|
|
|
|
2021-09-06 17:02:41 +08:00
|
|
|
// handles session events - DataNodes Add/Del
|
2021-10-14 15:44:34 +08:00
|
|
|
func (s *Server) handleSessionEvent(ctx context.Context, event *sessionutil.SessionEvent) error {
|
2021-09-06 17:02:41 +08:00
|
|
|
if event == nil {
|
2021-10-14 15:44:34 +08:00
|
|
|
return nil
|
2021-09-06 17:02:41 +08:00
|
|
|
}
|
|
|
|
info := &datapb.DataNodeInfo{
|
|
|
|
Address: event.Session.Address,
|
|
|
|
Version: event.Session.ServerID,
|
|
|
|
Channels: []*datapb.ChannelStatus{},
|
|
|
|
}
|
2021-10-14 15:44:34 +08:00
|
|
|
node := &NodeInfo{
|
|
|
|
NodeID: event.Session.ServerID,
|
|
|
|
Address: event.Session.Address,
|
|
|
|
}
|
2021-09-06 17:02:41 +08:00
|
|
|
switch event.EventType {
|
|
|
|
case sessionutil.SessionAddEvent:
|
|
|
|
log.Info("received datanode register",
|
|
|
|
zap.String("address", info.Address),
|
|
|
|
zap.Int64("serverID", info.Version))
|
2021-10-14 15:44:34 +08:00
|
|
|
if err := s.cluster.Register(node); err != nil {
|
2022-06-21 18:54:13 +08:00
|
|
|
log.Warn("failed to register node", zap.Int64("id", node.NodeID), zap.String("address", node.Address), zap.Error(err))
|
2021-10-14 15:44:34 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-09-06 17:02:41 +08:00
|
|
|
s.metricsCacheManager.InvalidateSystemInfoMetrics()
|
|
|
|
case sessionutil.SessionDelEvent:
|
|
|
|
log.Info("received datanode unregister",
|
|
|
|
zap.String("address", info.Address),
|
|
|
|
zap.Int64("serverID", info.Version))
|
2021-10-14 15:44:34 +08:00
|
|
|
if err := s.cluster.UnRegister(node); err != nil {
|
2022-06-21 18:54:13 +08:00
|
|
|
log.Warn("failed to deregister node", zap.Int64("id", node.NodeID), zap.String("address", node.Address), zap.Error(err))
|
2021-10-14 15:44:34 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-09-06 17:02:41 +08:00
|
|
|
s.metricsCacheManager.InvalidateSystemInfoMetrics()
|
|
|
|
default:
|
|
|
|
log.Warn("receive unknown service event type",
|
|
|
|
zap.Any("type", event.EventType))
|
|
|
|
}
|
2021-10-14 15:44:34 +08:00
|
|
|
return nil
|
2021-09-06 17:02:41 +08:00
|
|
|
}
|
|
|
|
|
2021-12-15 09:45:09 +08:00
|
|
|
// startFlushLoop starts a goroutine to handle post func process
|
|
|
|
// which is to notify `RootCoord` that this segment is flushed
|
2021-05-28 09:55:21 +08:00
|
|
|
func (s *Server) startFlushLoop(ctx context.Context) {
|
2021-10-29 21:30:49 +08:00
|
|
|
go func() {
|
|
|
|
defer logutil.LogPanic()
|
|
|
|
defer s.serverLoopWg.Done()
|
|
|
|
ctx2, cancel := context.WithCancel(ctx)
|
|
|
|
defer cancel()
|
|
|
|
// send `Flushing` segments
|
|
|
|
go s.handleFlushingSegments(ctx2)
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
2022-10-25 19:31:30 +08:00
|
|
|
logutil.Logger(s.ctx).Info("flush loop shutdown")
|
2021-10-29 21:30:49 +08:00
|
|
|
return
|
|
|
|
case segmentID := <-s.flushCh:
|
|
|
|
//Ignore return error
|
2022-10-27 15:15:32 +08:00
|
|
|
log.Info("flush successfully", zap.Any("segmentID", segmentID))
|
|
|
|
err := s.postFlush(ctx, segmentID)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to do post flush", zap.Any("segmentID", segmentID), zap.Error(err))
|
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
}
|
2021-05-28 09:55:21 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
}()
|
2021-05-28 09:55:21 +08:00
|
|
|
}
|
|
|
|
|
2021-09-06 17:02:41 +08:00
|
|
|
// post function after flush is done
|
|
|
|
// 1. check segment id is valid
|
|
|
|
// 2. notify RootCoord segment is flushed
|
|
|
|
// 3. change segment state to `Flushed` in meta
|
|
|
|
func (s *Server) postFlush(ctx context.Context, segmentID UniqueID) error {
|
|
|
|
segment := s.meta.GetSegment(segmentID)
|
|
|
|
if segment == nil {
|
2022-10-27 17:15:32 +08:00
|
|
|
return errors.New("segment not found, might be a faked segemnt, ignore post flush")
|
2021-09-06 17:02:41 +08:00
|
|
|
}
|
|
|
|
// set segment to SegmentState_Flushed
|
2022-08-25 15:48:54 +08:00
|
|
|
if err := s.meta.SetState(segmentID, commonpb.SegmentState_Flushed); err != nil {
|
2021-09-06 17:02:41 +08:00
|
|
|
log.Error("flush segment complete failed", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
2022-03-02 15:35:55 +08:00
|
|
|
log.Info("flush segment complete", zap.Int64("id", segmentID))
|
2021-09-06 17:02:41 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// recovery logic, fetch all Segment in `Flushing` state and do Flush notification logic
|
2021-05-28 09:55:21 +08:00
|
|
|
func (s *Server) handleFlushingSegments(ctx context.Context) {
|
|
|
|
segments := s.meta.GetFlushingSegments()
|
|
|
|
for _, segment := range segments {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
|
|
|
case s.flushCh <- segment.ID:
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-06-21 17:28:03 +08:00
|
|
|
func (s *Server) initRootCoordClient() error {
|
2021-05-28 09:55:21 +08:00
|
|
|
var err error
|
2022-02-07 10:09:45 +08:00
|
|
|
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.EtcdCfg.MetaRootPath, s.etcdCli); err != nil {
|
2021-05-28 09:55:21 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-06-21 17:28:03 +08:00
|
|
|
if err = s.rootCoordClient.Init(); err != nil {
|
2021-05-28 09:55:21 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-06-21 17:28:03 +08:00
|
|
|
return s.rootCoordClient.Start()
|
2021-05-28 09:55:21 +08:00
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-06-29 10:46:13 +08:00
|
|
|
// Stop do the Server finalize processes
|
|
|
|
// it checks the server status is healthy, if not, just quit
|
|
|
|
// if Server is healthy, set server state to stopped, release etcd session,
|
2022-10-13 19:21:23 +08:00
|
|
|
//
|
2021-06-29 10:46:13 +08:00
|
|
|
// stop message stream client and stop server loops
|
2021-01-22 11:07:07 +08:00
|
|
|
func (s *Server) Stop() error {
|
2022-10-10 15:55:22 +08:00
|
|
|
if !s.stateCode.CompareAndSwap(commonpb.StateCode_Healthy, commonpb.StateCode_Abnormal) {
|
2021-05-28 09:55:21 +08:00
|
|
|
return nil
|
|
|
|
}
|
2022-10-25 19:31:30 +08:00
|
|
|
logutil.Logger(s.ctx).Info("server shutdown")
|
2021-07-12 11:03:52 +08:00
|
|
|
s.cluster.Close()
|
2021-11-10 19:03:38 +08:00
|
|
|
s.garbageCollector.close()
|
2021-05-28 09:55:21 +08:00
|
|
|
s.stopServerLoop()
|
2021-11-16 22:31:14 +08:00
|
|
|
s.session.Revoke(time.Second)
|
2021-11-05 22:25:00 +08:00
|
|
|
|
2021-12-23 18:39:11 +08:00
|
|
|
if Params.DataCoordCfg.EnableCompaction {
|
2021-11-05 22:25:00 +08:00
|
|
|
s.stopCompactionTrigger()
|
|
|
|
s.stopCompactionHandler()
|
|
|
|
}
|
2021-01-22 11:07:07 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-04-16 16:30:55 +08:00
|
|
|
// CleanMeta only for test
|
|
|
|
func (s *Server) CleanMeta() error {
|
2021-05-26 19:06:56 +08:00
|
|
|
log.Debug("clean meta", zap.Any("kv", s.kvClient))
|
2021-04-24 11:29:15 +08:00
|
|
|
return s.kvClient.RemoveWithPrefix("")
|
2021-04-16 16:30:55 +08:00
|
|
|
}
|
|
|
|
|
2021-01-23 20:22:59 +08:00
|
|
|
func (s *Server) stopServerLoop() {
|
|
|
|
s.serverLoopCancel()
|
|
|
|
s.serverLoopWg.Wait()
|
|
|
|
}
|
|
|
|
|
2021-04-26 09:45:54 +08:00
|
|
|
//func (s *Server) validateAllocRequest(collID UniqueID, partID UniqueID, channelName string) error {
|
|
|
|
// if !s.meta.HasCollection(collID) {
|
|
|
|
// return fmt.Errorf("can not find collection %d", collID)
|
|
|
|
// }
|
|
|
|
// if !s.meta.HasPartition(collID, partID) {
|
|
|
|
// return fmt.Errorf("can not find partition %d", partID)
|
|
|
|
// }
|
|
|
|
// for _, name := range s.insertChannels {
|
|
|
|
// if name == channelName {
|
|
|
|
// return nil
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
// return fmt.Errorf("can not find channel %s", channelName)
|
|
|
|
//}
|
2021-04-12 16:35:51 +08:00
|
|
|
|
2022-02-22 13:15:51 +08:00
|
|
|
// loadCollectionFromRootCoord communicates with RootCoord and asks for collection information.
|
|
|
|
// collection information will be added to server meta info.
|
2021-06-21 17:28:03 +08:00
|
|
|
func (s *Server) loadCollectionFromRootCoord(ctx context.Context, collectionID int64) error {
|
|
|
|
resp, err := s.rootCoordClient.DescribeCollection(ctx, &milvuspb.DescribeCollectionRequest{
|
2022-10-21 15:57:28 +08:00
|
|
|
Base: commonpbutil.NewMsgBase(
|
|
|
|
commonpbutil.WithMsgType(commonpb.MsgType_DescribeCollection),
|
2022-11-04 14:25:38 +08:00
|
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
2022-10-21 15:57:28 +08:00
|
|
|
),
|
2021-02-26 09:23:39 +08:00
|
|
|
DbName: "",
|
|
|
|
CollectionID: collectionID,
|
|
|
|
})
|
|
|
|
if err = VerifyResponse(resp, err); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-06-21 17:28:03 +08:00
|
|
|
presp, err := s.rootCoordClient.ShowPartitions(ctx, &milvuspb.ShowPartitionsRequest{
|
2022-10-21 15:57:28 +08:00
|
|
|
Base: commonpbutil.NewMsgBase(
|
|
|
|
commonpbutil.WithMsgType(commonpb.MsgType_ShowPartitions),
|
|
|
|
commonpbutil.WithMsgID(0),
|
2022-11-04 14:25:38 +08:00
|
|
|
commonpbutil.WithSourceID(paramtable.GetNodeID()),
|
2022-10-21 15:57:28 +08:00
|
|
|
),
|
2021-04-12 16:35:51 +08:00
|
|
|
DbName: "",
|
|
|
|
CollectionName: resp.Schema.Name,
|
|
|
|
CollectionID: resp.CollectionID,
|
|
|
|
})
|
|
|
|
if err = VerifyResponse(presp, err); err != nil {
|
2021-05-29 10:47:29 +08:00
|
|
|
log.Error("show partitions error", zap.String("collectionName", resp.Schema.Name),
|
|
|
|
zap.Int64("collectionID", resp.CollectionID), zap.Error(err))
|
2021-01-26 09:43:41 +08:00
|
|
|
return err
|
|
|
|
}
|
2022-10-10 20:31:22 +08:00
|
|
|
|
|
|
|
properties := make(map[string]string)
|
|
|
|
for _, pair := range resp.Properties {
|
|
|
|
properties[pair.GetKey()] = pair.GetValue()
|
|
|
|
}
|
|
|
|
|
|
|
|
collInfo := &collectionInfo{
|
2021-09-27 20:34:03 +08:00
|
|
|
ID: resp.CollectionID,
|
|
|
|
Schema: resp.Schema,
|
|
|
|
Partitions: presp.PartitionIDs,
|
|
|
|
StartPositions: resp.GetStartPositions(),
|
2022-10-10 20:31:22 +08:00
|
|
|
Properties: properties,
|
2021-04-12 16:35:51 +08:00
|
|
|
}
|
2021-07-07 14:02:01 +08:00
|
|
|
s.meta.AddCollection(collInfo)
|
|
|
|
return nil
|
2021-01-22 11:07:07 +08:00
|
|
|
}
|
2022-05-25 14:34:00 +08:00
|
|
|
|
|
|
|
func (s *Server) reCollectSegmentStats(ctx context.Context) {
|
|
|
|
if s.channelManager == nil {
|
|
|
|
log.Error("null channel manager found, which should NOT happen in non-testing environment")
|
|
|
|
return
|
|
|
|
}
|
2022-08-19 19:50:50 +08:00
|
|
|
nodes := s.sessionManager.getLiveNodeIDs()
|
2022-05-25 14:34:00 +08:00
|
|
|
log.Info("re-collecting segment stats from DataNodes",
|
|
|
|
zap.Int64s("DataNode IDs", nodes))
|
|
|
|
for _, node := range nodes {
|
|
|
|
s.cluster.ReCollectSegmentStats(ctx, node)
|
|
|
|
}
|
|
|
|
}
|