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"
|
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"
|
|
|
|
|
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-05-28 09:55:21 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/logutil"
|
2021-10-15 11:46:33 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/mqclient"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
2021-11-10 19:03:38 +08:00
|
|
|
"github.com/minio/minio-go/v7"
|
|
|
|
"github.com/minio/minio-go/v7/pkg/credentials"
|
2021-05-27 18:45:24 +08:00
|
|
|
"go.uber.org/zap"
|
2021-03-04 16:01:30 +08:00
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/msgstream"
|
|
|
|
"github.com/milvus-io/milvus/internal/types"
|
|
|
|
"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-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
2021-03-05 20:41:34 +08:00
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
2021-01-22 11:07:07 +08:00
|
|
|
)
|
|
|
|
|
2021-11-13 08:45:09 +08:00
|
|
|
const (
|
|
|
|
connEtcdMaxRetryTime = 100000
|
|
|
|
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"
|
|
|
|
ttMaxInterval = 3 * time.Minute
|
|
|
|
ttCheckerWarnMsg = fmt.Sprintf("we haven't received tt for %f minutes", ttMaxInterval.Minutes())
|
|
|
|
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-09-08 11:35:59 +08:00
|
|
|
// ServerState type alias, presents datacoord Server State
|
2021-06-29 10:46:13 +08:00
|
|
|
type ServerState = int64
|
|
|
|
|
|
|
|
const (
|
|
|
|
// ServerStateStopped state stands for just created or stopped `Server` instance
|
|
|
|
ServerStateStopped ServerState = 0
|
|
|
|
// ServerStateInitializing state stands initializing `Server` instance
|
|
|
|
ServerStateInitializing ServerState = 1
|
|
|
|
// ServerStateHealthy state stands for healthy `Server` instance
|
|
|
|
ServerStateHealthy ServerState = 2
|
|
|
|
)
|
|
|
|
|
2021-10-14 15:44:34 +08:00
|
|
|
type dataNodeCreatorFunc func(ctx context.Context, addr string) (types.DataNode, error)
|
|
|
|
type rootCoordCreatorFunc func(ctx context.Context, metaRootPath string, etcdEndpoints []string) (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)
|
|
|
|
|
2021-06-29 10:46:13 +08:00
|
|
|
// Server implements `types.Datacoord`
|
|
|
|
// handles Data Cooridinator 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{}
|
2021-06-29 10:46:13 +08:00
|
|
|
isServing ServerState
|
2021-08-16 11:00:09 +08:00
|
|
|
helper ServerHelper
|
2021-05-26 19:06:56 +08:00
|
|
|
|
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
|
|
|
|
|
2021-07-02 11:16:20 +08:00
|
|
|
flushCh chan UniqueID
|
|
|
|
msFactory msgstream.Factory
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-09-28 16:10:04 +08:00
|
|
|
session *sessionutil.Session
|
|
|
|
eventCh <-chan *sessionutil.SessionEvent
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-10-14 15:44:34 +08:00
|
|
|
dataNodeCreator dataNodeCreatorFunc
|
|
|
|
rootCoordClientCreator rootCoordCreatorFunc
|
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
|
2021-07-19 10:41:15 +08:00
|
|
|
func CreateServer(ctx context.Context, factory msgstream.Factory, opts ...Option) (*Server, error) {
|
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{}),
|
2021-06-22 18:24:08 +08:00
|
|
|
msFactory: factory,
|
|
|
|
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(),
|
2021-09-03 17:15:26 +08:00
|
|
|
|
|
|
|
metricsCacheManager: metricsinfo.NewMetricsCacheManager(),
|
2021-01-26 15:14:49 +08:00
|
|
|
}
|
2021-07-19 10:41:15 +08:00
|
|
|
|
|
|
|
for _, opt := range opts {
|
|
|
|
opt(s)
|
|
|
|
}
|
2021-01-26 15:14:49 +08:00
|
|
|
return s, nil
|
|
|
|
}
|
|
|
|
|
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-06-24 19:05:06 +08:00
|
|
|
func defaultRootCoordCreatorFunc(ctx context.Context, metaRootPath string, etcdEndpoints []string) (types.RootCoord, error) {
|
|
|
|
return rootcoordclient.NewClient(ctx, metaRootPath, etcdEndpoints)
|
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()
|
|
|
|
go s.session.LivenessCheck(s.serverLoopCtx, func() {
|
|
|
|
log.Error("DataCoord 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
|
|
|
|
syscall.Kill(syscall.Getpid(), syscall.SIGINT)
|
|
|
|
})
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) initSession() error {
|
2021-06-11 22:04:41 +08:00
|
|
|
s.session = sessionutil.NewSession(s.ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
2021-09-09 10:06:29 +08:00
|
|
|
if s.session == nil {
|
|
|
|
return errors.New("failed to initialize session")
|
|
|
|
}
|
2021-11-12 21:25:08 +08:00
|
|
|
s.session.Init(typeutil.DataCoordRole, Params.Address, true)
|
2021-05-25 15:06:05 +08:00
|
|
|
Params.NodeID = s.session.ServerID
|
2021-12-15 11:47:10 +08:00
|
|
|
Params.SetLogger(Params.NodeID)
|
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 {
|
2021-06-29 10:46:13 +08:00
|
|
|
atomic.StoreInt64(&s.isServing, ServerStateInitializing)
|
2021-12-15 11:47:10 +08:00
|
|
|
return s.initSession()
|
2021-01-22 19:43:27 +08:00
|
|
|
}
|
|
|
|
|
2021-06-29 10:46:13 +08:00
|
|
|
// Start initialize `Server` members and start loops, follow steps are taken:
|
|
|
|
// 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
|
2021-01-22 19:43:27 +08:00
|
|
|
func (s *Server) Start() error {
|
2021-01-26 09:43:41 +08:00
|
|
|
var err error
|
2021-05-28 09:55:21 +08:00
|
|
|
m := map[string]interface{}{
|
|
|
|
"PulsarAddress": Params.PulsarAddress,
|
|
|
|
"ReceiveBufSize": 1024,
|
|
|
|
"PulsarBufSize": 1024}
|
|
|
|
err = s.msFactory.SetParams(m)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
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
|
|
|
|
2021-05-28 09:55:21 +08:00
|
|
|
if err = s.initMeta(); err != nil {
|
|
|
|
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)
|
2021-11-05 22:25:00 +08:00
|
|
|
if Params.EnableCompaction {
|
|
|
|
s.createCompactionHandler()
|
|
|
|
s.createCompactionTrigger()
|
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-06-03 19:06:33 +08:00
|
|
|
s.startSegmentManager()
|
2021-05-28 09:55:21 +08:00
|
|
|
if err = s.initServiceDiscovery(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-11-10 19:03:38 +08:00
|
|
|
if err = s.initGarbageCollection(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2021-05-28 09:55:21 +08:00
|
|
|
s.startServerLoop()
|
2021-09-26 17:50:07 +08:00
|
|
|
Params.CreatedTime = time.Now()
|
|
|
|
Params.UpdatedTime = time.Now()
|
2021-06-29 10:46:13 +08:00
|
|
|
atomic.StoreInt64(&s.isServing, ServerStateHealthy)
|
2021-12-20 11:14:51 +08:00
|
|
|
log.Debug("DataCoord startup success")
|
2021-09-26 17:50:07 +08:00
|
|
|
|
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
|
2021-11-17 23:25:12 +08:00
|
|
|
s.channelManager, err = NewChannelManager(s.kvClient, s.handler)
|
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
|
|
|
|
2021-11-05 22:25:00 +08:00
|
|
|
func (s *Server) createCompactionHandler() {
|
|
|
|
s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator, s.flushCh)
|
|
|
|
s.compactionHandler.start()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) stopCompactionHandler() {
|
|
|
|
s.compactionHandler.stop()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) createCompactionTrigger() {
|
|
|
|
s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator)
|
|
|
|
s.compactionTrigger.start()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) stopCompactionTrigger() {
|
|
|
|
s.compactionTrigger.stop()
|
|
|
|
}
|
|
|
|
|
2021-11-10 19:03:38 +08:00
|
|
|
func (s *Server) initGarbageCollection() error {
|
|
|
|
var cli *minio.Client
|
|
|
|
var err error
|
|
|
|
if Params.EnableGarbageCollection {
|
|
|
|
cli, err = minio.New(Params.MinioAddress, &minio.Options{
|
|
|
|
Creds: credentials.NewStaticV4(Params.MinioAccessKeyID, Params.MinioSecretAccessKey, ""),
|
|
|
|
Secure: Params.MinioUseSSL,
|
|
|
|
})
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
has, err := cli.BucketExists(context.TODO(), Params.MinioBucketName)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
if !has {
|
|
|
|
err = cli.MakeBucket(context.TODO(), Params.MinioBucketName, minio.MakeBucketOptions{})
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
s.garbageCollector = newGarbageCollector(s.meta, GcOption{
|
|
|
|
cli: cli,
|
|
|
|
enabled: Params.EnableGarbageCollection,
|
|
|
|
bucketName: Params.MinioBucketName,
|
|
|
|
rootPath: Params.MinioRootPath,
|
|
|
|
|
2021-11-18 22:29:40 +08:00
|
|
|
checkInterval: Params.GCInterval,
|
|
|
|
missingTolerance: Params.GCMissingTolerance,
|
|
|
|
dropTolerance: Params.GCDropTolerance,
|
2021-11-10 19:03:38 +08:00
|
|
|
})
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-05-26 19:06:56 +08:00
|
|
|
func (s *Server) initServiceDiscovery() error {
|
|
|
|
sessions, rev, err := s.session.GetSessions(typeutil.DataNodeRole)
|
|
|
|
if err != nil {
|
2021-10-20 12:46:42 +08:00
|
|
|
log.Debug("dataCoord initServiceDiscovery failed", zap.Error(err))
|
2021-05-19 18:36:05 +08:00
|
|
|
return err
|
|
|
|
}
|
2021-12-20 11:02:45 +08:00
|
|
|
log.Debug("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
|
|
|
|
2021-07-12 11:03:52 +08:00
|
|
|
s.cluster.Startup(datanodes)
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-12-08 10:11:04 +08:00
|
|
|
s.eventCh = s.session.WatchServices(typeutil.DataNodeRole, rev+1, nil)
|
2021-01-22 19:43:27 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-06-03 19:06:33 +08:00
|
|
|
func (s *Server) startSegmentManager() {
|
2021-11-12 00:22:42 +08:00
|
|
|
if s.segmentManager == nil {
|
|
|
|
s.segmentManager = newSegmentManager(s.meta, s.allocator)
|
|
|
|
}
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
2021-01-22 11:07:07 +08:00
|
|
|
func (s *Server) initMeta() error {
|
2021-02-26 15:17:47 +08:00
|
|
|
connectEtcdFn := func() error {
|
2021-08-13 11:04:09 +08:00
|
|
|
etcdKV, err := etcdkv.NewEtcdKV(Params.EtcdEndpoints, Params.MetaRootPath)
|
2021-02-26 15:17:47 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-08-13 11:04:09 +08:00
|
|
|
|
|
|
|
s.kvClient = etcdKV
|
2021-10-02 23:09:56 +08:00
|
|
|
s.meta, err = newMeta(s.kvClient)
|
2021-02-26 15:17:47 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
2021-01-22 11:07:07 +08:00
|
|
|
}
|
2021-06-23 09:24:10 +08:00
|
|
|
return retry.Do(s.ctx, connectEtcdFn, 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) {
|
|
|
|
ttMsgStream, err := s.msFactory.NewMsgStream(ctx)
|
|
|
|
if err != nil {
|
2021-12-20 11:18:49 +08:00
|
|
|
log.Error("DataCoord failed to create timetick channel", zap.Error(err))
|
2021-05-25 15:35:37 +08:00
|
|
|
return
|
|
|
|
}
|
2021-10-15 11:46:33 +08:00
|
|
|
ttMsgStream.AsConsumerWithPosition([]string{Params.TimeTickChannelName},
|
|
|
|
Params.DataCoordSubscriptionName, mqclient.SubscriptionPositionLatest)
|
2021-12-21 09:17:00 +08:00
|
|
|
log.Debug("DataCoord creates the timetick channel consumer",
|
|
|
|
zap.String("timeTickChannel", Params.TimeTickChannelName),
|
|
|
|
zap.String("subscription", Params.DataCoordSubscriptionName))
|
2021-05-25 15:35:37 +08:00
|
|
|
ttMsgStream.Start()
|
2021-08-12 19:58:08 +08:00
|
|
|
|
2021-10-29 21:30:49 +08:00
|
|
|
go func() {
|
|
|
|
var checker *LongTermChecker
|
|
|
|
if enableTtChecker {
|
|
|
|
checker = NewLongTermChecker(ctx, ttCheckerName, ttMaxInterval, ttCheckerWarnMsg)
|
|
|
|
checker.Start()
|
|
|
|
defer checker.Stop()
|
2021-04-16 16:30:55 +08:00
|
|
|
}
|
2021-12-15 10:53:16 +08:00
|
|
|
|
2021-10-29 21:30:49 +08:00
|
|
|
defer logutil.LogPanic()
|
|
|
|
defer s.serverLoopWg.Done()
|
|
|
|
defer ttMsgStream.Close()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
2021-12-20 11:06:43 +08:00
|
|
|
log.Debug("DataNode timetick loop shutdown")
|
2021-10-29 21:30:49 +08:00
|
|
|
return
|
|
|
|
default:
|
2021-01-26 09:43:41 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
msgPack := ttMsgStream.Consume()
|
|
|
|
if msgPack == nil {
|
2021-12-20 11:16:52 +08:00
|
|
|
log.Debug("receive nil timetick msg and shutdown timetick channel")
|
2021-10-29 21:30:49 +08:00
|
|
|
return
|
2021-08-12 19:58:08 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
for _, msg := range msgPack.Msgs {
|
2021-12-15 10:53:16 +08:00
|
|
|
ttMsg, ok := msg.(*msgstream.DataNodeTtMsg)
|
|
|
|
if !ok {
|
|
|
|
log.Warn("receive unexpected msg type from tt channel")
|
2021-10-29 21:30:49 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
if enableTtChecker {
|
|
|
|
checker.Check()
|
|
|
|
}
|
2021-05-25 15:35:37 +08:00
|
|
|
|
2021-12-15 10:53:16 +08:00
|
|
|
if err := s.handleTimetickMessage(ctx, ttMsg); err != nil {
|
|
|
|
log.Error("failed to handle timetick message", zap.Error(err))
|
2021-05-25 15:35:37 +08:00
|
|
|
continue
|
2021-05-19 14:13:53 +08:00
|
|
|
}
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
s.helper.eventAfterHandleDataNodeTt()
|
2021-05-26 19:06:56 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +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))
|
|
|
|
}
|
|
|
|
|
|
|
|
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)
|
|
|
|
|
|
|
|
staleSegments := s.getStaleSegmentsInfo(ch)
|
|
|
|
staleSegments = s.filterWithFlushableSegments(staleSegments, flushableIDs)
|
|
|
|
|
|
|
|
if len(flushableSegments)+len(staleSegments) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debug("flush segments", zap.Int64s("segmentIDs", flushableIDs), zap.Int("markSegments count", len(staleSegments)))
|
|
|
|
|
|
|
|
s.setLastFlushTime(flushableSegments)
|
|
|
|
s.setLastFlushTime(staleSegments)
|
|
|
|
|
|
|
|
finfo, minfo := make([]*datapb.SegmentInfo, 0, len(flushableSegments)), make([]*datapb.SegmentInfo, 0, len(staleSegments))
|
|
|
|
for _, info := range flushableSegments {
|
|
|
|
finfo = append(finfo, info.SegmentInfo)
|
|
|
|
}
|
|
|
|
for _, info := range staleSegments {
|
|
|
|
minfo = append(minfo, info.SegmentInfo)
|
|
|
|
}
|
|
|
|
s.cluster.Flush(s.ctx, finfo, minfo)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) updateSegmentStatistics(stats []*datapb.SegmentStats) {
|
|
|
|
for _, stat := range stats {
|
|
|
|
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) getStaleSegmentsInfo(ch string) []*SegmentInfo {
|
|
|
|
return s.meta.SelectSegments(func(info *SegmentInfo) bool {
|
|
|
|
return isSegmentHealthy(info) &&
|
|
|
|
info.GetInsertChannel() == ch &&
|
|
|
|
!info.lastFlushTime.IsZero() &&
|
|
|
|
time.Since(info.lastFlushTime).Minutes() >= segmentTimedFlushDuration
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) filterWithFlushableSegments(staleSegments []*SegmentInfo, flushableIDs []int64) []*SegmentInfo {
|
|
|
|
filter := map[int64]struct{}{}
|
|
|
|
for _, sid := range flushableIDs {
|
|
|
|
filter[sid] = struct{}{}
|
|
|
|
}
|
|
|
|
|
|
|
|
res := make([]*SegmentInfo, 0, len(staleSegments))
|
|
|
|
for _, sinfo := range staleSegments {
|
|
|
|
if _, ok := filter[sinfo.GetID()]; ok {
|
|
|
|
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)
|
|
|
|
}
|
|
|
|
|
|
|
|
// watchService watchs services
|
|
|
|
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():
|
|
|
|
log.Debug("watch service shutdown")
|
|
|
|
return
|
2021-10-14 19:20:35 +08:00
|
|
|
case event, ok := <-s.eventCh:
|
|
|
|
if !ok {
|
|
|
|
//TODO add retry logic
|
|
|
|
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 {
|
|
|
|
log.Warn("datacoord server stop error", zap.Error(err))
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
return
|
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
}
|
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
|
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 {
|
|
|
|
log.Warn("failed to regisger node", zap.Int64("id", node.NodeID), zap.String("address", node.Address), zap.Error(err))
|
|
|
|
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 {
|
|
|
|
log.Warn("failed to deregisger node", zap.Int64("id", node.NodeID), zap.String("address", node.Address), zap.Error(err))
|
|
|
|
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():
|
|
|
|
log.Debug("flush loop shutdown")
|
|
|
|
return
|
|
|
|
case segmentID := <-s.flushCh:
|
|
|
|
//Ignore return error
|
|
|
|
_ = s.postFlush(ctx, segmentID)
|
|
|
|
}
|
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 {
|
|
|
|
log.Warn("failed to get flused segment", zap.Int64("id", segmentID))
|
|
|
|
return errors.New("segment not found")
|
|
|
|
}
|
|
|
|
// Notify RootCoord segment is flushed
|
|
|
|
req := &datapb.SegmentFlushCompletedMsg{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_SegmentFlushDone,
|
|
|
|
},
|
|
|
|
Segment: segment.SegmentInfo,
|
|
|
|
}
|
|
|
|
resp, err := s.rootCoordClient.SegmentFlushCompleted(ctx, req)
|
|
|
|
if err = VerifyResponse(resp, err); err != nil {
|
|
|
|
log.Warn("failed to call SegmentFlushComplete", zap.Int64("segmentID", segmentID), zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
// set segment to SegmentState_Flushed
|
|
|
|
if err = s.meta.SetState(segmentID, commonpb.SegmentState_Flushed); err != nil {
|
|
|
|
log.Error("flush segment complete failed", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
log.Debug("flush segment complete", zap.Int64("id", segmentID))
|
|
|
|
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
|
2021-06-24 19:05:06 +08:00
|
|
|
if s.rootCoordClient, err = s.rootCoordClientCreator(s.ctx, Params.MetaRootPath, Params.EtcdEndpoints); 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,
|
|
|
|
// stop message stream client and stop server loops
|
2021-01-22 11:07:07 +08:00
|
|
|
func (s *Server) Stop() error {
|
2021-06-29 10:46:13 +08:00
|
|
|
if !atomic.CompareAndSwapInt64(&s.isServing, ServerStateHealthy, ServerStateStopped) {
|
2021-05-28 09:55:21 +08:00
|
|
|
return nil
|
|
|
|
}
|
2021-07-28 11:43:22 +08:00
|
|
|
log.Debug("dataCoord 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
|
|
|
|
|
|
|
if Params.EnableCompaction {
|
|
|
|
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
|
|
|
|
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{
|
2021-02-26 09:23:39 +08:00
|
|
|
Base: &commonpb.MsgBase{
|
2021-03-10 14:45:35 +08:00
|
|
|
MsgType: commonpb.MsgType_DescribeCollection,
|
2021-02-26 09:23:39 +08:00
|
|
|
SourceID: Params.NodeID,
|
|
|
|
},
|
|
|
|
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{
|
2021-04-12 16:35:51 +08:00
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_ShowPartitions,
|
2021-06-23 12:10:12 +08:00
|
|
|
MsgID: 0,
|
|
|
|
Timestamp: 0,
|
2021-04-12 16:35:51 +08:00
|
|
|
SourceID: Params.NodeID,
|
2021-01-26 09:43:41 +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
|
|
|
|
}
|
2021-04-12 16:35:51 +08:00
|
|
|
collInfo := &datapb.CollectionInfo{
|
2021-09-27 20:34:03 +08:00
|
|
|
ID: resp.CollectionID,
|
|
|
|
Schema: resp.Schema,
|
|
|
|
Partitions: presp.PartitionIDs,
|
|
|
|
StartPositions: resp.GetStartPositions(),
|
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
|
|
|
}
|