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"
|
|
|
|
"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/rootcoord"
|
|
|
|
"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-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"
|
2021-08-19 13:00:12 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
2021-01-22 11:07:07 +08:00
|
|
|
)
|
|
|
|
|
2021-09-09 19:08:03 +08:00
|
|
|
const connEtcdMaxRetryTime = 100000
|
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-10-05 20:58:04 +08:00
|
|
|
// makes sure Server implements `positionProvider`
|
|
|
|
var _ positionProvider = (*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-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-07-06 09:24:05 +08:00
|
|
|
kvClient *etcdkv.EtcdKV
|
|
|
|
meta *meta
|
|
|
|
segmentManager Manager
|
|
|
|
allocator allocator
|
2021-07-12 11:03:52 +08:00
|
|
|
cluster *Cluster
|
2021-11-05 22:25:00 +08:00
|
|
|
sessionManager *SessionManager
|
2021-10-14 15:44:34 +08:00
|
|
|
channelManager *ChannelManager
|
2021-07-06 09:24:05 +08:00
|
|
|
rootCoordClient types.RootCoord
|
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-06-29 10:46:13 +08:00
|
|
|
// CreateServer create `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,
|
|
|
|
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-05-25 15:06:05 +08:00
|
|
|
// Register register data service at etcd
|
|
|
|
func (s *Server) Register() 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-10-14 16:40:35 +08:00
|
|
|
s.session.Init(typeutil.DataCoordRole, Params.IP, true)
|
2021-05-25 15:06:05 +08:00
|
|
|
Params.NodeID = s.session.ServerID
|
2021-10-01 08:52:50 +08:00
|
|
|
Params.SetLogger(typeutil.UniqueID(-1))
|
2021-05-25 15:06:05 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-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-01-22 19:43:27 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
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-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-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-07-28 11:43:22 +08:00
|
|
|
log.Debug("dataCoordinator 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-10-14 15:44:34 +08:00
|
|
|
s.channelManager, err = NewChannelManager(s.kvClient, s)
|
|
|
|
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-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-05-26 19:06:56 +08:00
|
|
|
log.Debug("registered sessions", zap.Any("sessions", sessions))
|
|
|
|
|
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-06-29 14:52:14 +08:00
|
|
|
s.eventCh = s.session.WatchServices(typeutil.DataNodeRole, rev+1)
|
2021-01-22 19:43:27 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-06-03 19:06:33 +08:00
|
|
|
func (s *Server) startSegmentManager() {
|
2021-07-06 09:24:05 +08:00
|
|
|
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-09-28 16:10:04 +08:00
|
|
|
s.serverLoopWg.Add(4)
|
2021-10-29 21:30:49 +08:00
|
|
|
s.startStatsChannel(s.serverLoopCtx)
|
|
|
|
s.startDataNodeTtLoop(s.serverLoopCtx)
|
|
|
|
s.startWatchService(s.serverLoopCtx)
|
|
|
|
s.startFlushLoop(s.serverLoopCtx)
|
2021-10-14 16:40:35 +08:00
|
|
|
go s.session.LivenessCheck(s.serverLoopCtx, func() {
|
2021-10-30 10:24:38 +08:00
|
|
|
log.Error("Data Coord disconnected from etcd, process will exit", zap.Int64("Server Id", s.session.ServerID))
|
|
|
|
if err := s.Stop(); err != nil {
|
|
|
|
log.Fatal("failed to stop server", zap.Error(err))
|
|
|
|
}
|
2021-09-28 16:10:04 +08:00
|
|
|
})
|
2021-01-26 09:43:41 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *Server) startStatsChannel(ctx context.Context) {
|
2021-02-08 14:30:54 +08:00
|
|
|
statsStream, _ := s.msFactory.NewMsgStream(ctx)
|
2021-06-21 11:40:15 +08:00
|
|
|
statsStream.AsConsumer([]string{Params.StatisticsChannelName}, Params.DataCoordSubscriptionName)
|
2021-07-28 11:43:22 +08:00
|
|
|
log.Debug("dataCoord create stats channel consumer",
|
2021-06-03 19:06:33 +08:00
|
|
|
zap.String("channelName", Params.StatisticsChannelName),
|
2021-06-21 11:40:15 +08:00
|
|
|
zap.String("descriptionName", Params.DataCoordSubscriptionName))
|
2021-01-26 09:43:41 +08:00
|
|
|
statsStream.Start()
|
2021-10-29 21:30:49 +08:00
|
|
|
go func() {
|
|
|
|
defer logutil.LogPanic()
|
|
|
|
defer s.serverLoopWg.Done()
|
|
|
|
defer statsStream.Close()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
log.Debug("stats channel shutdown")
|
|
|
|
return
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
msgPack := statsStream.Consume()
|
|
|
|
if msgPack == nil {
|
|
|
|
log.Debug("receive nil stats msg, shutdown stats channel")
|
|
|
|
return
|
2021-02-23 09:58:06 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
for _, msg := range msgPack.Msgs {
|
|
|
|
if msg.Type() != commonpb.MsgType_SegmentStatistics {
|
|
|
|
log.Warn("receive unknown msg from segment statistics channel",
|
|
|
|
zap.Stringer("msgType", msg.Type()))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
ssMsg := msg.(*msgstream.SegmentStatisticsMsg)
|
|
|
|
for _, stat := range ssMsg.SegStats {
|
|
|
|
s.meta.SetCurrentRows(stat.GetSegmentID(), stat.GetNumRows())
|
|
|
|
}
|
2021-05-19 14:13:53 +08:00
|
|
|
}
|
2021-01-26 09:43:41 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
}()
|
2021-01-26 09:43:41 +08:00
|
|
|
}
|
|
|
|
|
2021-05-25 15:35:37 +08:00
|
|
|
func (s *Server) startDataNodeTtLoop(ctx context.Context) {
|
|
|
|
ttMsgStream, err := s.msFactory.NewMsgStream(ctx)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("new msg stream failed", zap.Error(err))
|
|
|
|
return
|
|
|
|
}
|
2021-10-15 11:46:33 +08:00
|
|
|
ttMsgStream.AsConsumerWithPosition([]string{Params.TimeTickChannelName},
|
|
|
|
Params.DataCoordSubscriptionName, mqclient.SubscriptionPositionLatest)
|
2021-07-28 11:43:22 +08:00
|
|
|
log.Debug("dataCoord create time tick channel consumer",
|
|
|
|
zap.String("timeTickChannelName", Params.TimeTickChannelName),
|
|
|
|
zap.String("subscriptionName", 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-10-29 21:30:49 +08:00
|
|
|
defer logutil.LogPanic()
|
|
|
|
defer s.serverLoopWg.Done()
|
|
|
|
defer ttMsgStream.Close()
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-ctx.Done():
|
|
|
|
log.Debug("data node tt loop shutdown")
|
|
|
|
return
|
|
|
|
default:
|
2021-01-26 09:43:41 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
msgPack := ttMsgStream.Consume()
|
|
|
|
if msgPack == nil {
|
|
|
|
log.Debug("receive nil tt msg, shutdown tt channel")
|
|
|
|
return
|
2021-08-12 19:58:08 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
for _, msg := range msgPack.Msgs {
|
|
|
|
if msg.Type() != commonpb.MsgType_DataNodeTt {
|
|
|
|
log.Warn("receive unexpected msg type from tt channel",
|
|
|
|
zap.Stringer("msgType", msg.Type()))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
ttMsg := msg.(*msgstream.DataNodeTtMsg)
|
|
|
|
if enableTtChecker {
|
|
|
|
checker.Check()
|
|
|
|
}
|
2021-05-25 15:35:37 +08:00
|
|
|
|
2021-10-29 21:30:49 +08:00
|
|
|
ch := ttMsg.ChannelName
|
|
|
|
ts := ttMsg.Timestamp
|
|
|
|
if err := s.segmentManager.ExpireAllocations(ch, ts); err != nil {
|
|
|
|
log.Warn("failed to expire allocations", zap.Error(err))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
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("tt", physical))
|
|
|
|
}
|
|
|
|
segments, err := s.segmentManager.GetFlushableSegments(ctx, ch, ts)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("get flushable segments failed", zap.Error(err))
|
|
|
|
continue
|
|
|
|
}
|
2021-05-26 19:06:56 +08:00
|
|
|
|
2021-10-29 21:30:49 +08:00
|
|
|
staleSegments := s.meta.SelectSegments(func(info *SegmentInfo) bool {
|
2021-11-01 22:30:04 +08:00
|
|
|
return info.GetInsertChannel() == ch &&
|
|
|
|
!info.lastFlushTime.IsZero() &&
|
|
|
|
time.Since(info.lastFlushTime).Minutes() >= segmentTimedFlushDuration
|
2021-10-29 21:30:49 +08:00
|
|
|
})
|
2021-10-20 15:02:36 +08:00
|
|
|
|
2021-10-29 21:30:49 +08:00
|
|
|
if len(segments)+len(staleSegments) == 0 {
|
2021-05-25 15:35:37 +08:00
|
|
|
continue
|
2021-05-19 14:13:53 +08:00
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
log.Debug("flush segments", zap.Int64s("segmentIDs", segments), zap.Int("markSegments count", len(staleSegments)))
|
|
|
|
segmentInfos := make([]*datapb.SegmentInfo, 0, len(segments))
|
|
|
|
for _, id := range segments {
|
|
|
|
sInfo := s.meta.GetSegment(id)
|
|
|
|
if sInfo == nil {
|
|
|
|
log.Error("get segment from meta error", zap.Int64("id", id),
|
|
|
|
zap.Error(err))
|
2021-10-20 15:02:36 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-10-29 21:30:49 +08:00
|
|
|
segmentInfos = append(segmentInfos, sInfo.SegmentInfo)
|
|
|
|
s.meta.SetLastFlushTime(id, time.Now())
|
|
|
|
}
|
|
|
|
markSegments := make([]*datapb.SegmentInfo, 0, len(staleSegments))
|
|
|
|
for _, segment := range staleSegments {
|
|
|
|
for _, fSeg := range segmentInfos {
|
|
|
|
// check segment needs flush first
|
|
|
|
if segment.GetID() == fSeg.GetID() {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
}
|
|
|
|
markSegments = append(markSegments, segment.SegmentInfo)
|
|
|
|
s.meta.SetLastFlushTime(segment.GetID(), time.Now())
|
|
|
|
}
|
|
|
|
if len(segmentInfos)+len(markSegments) > 0 {
|
|
|
|
s.cluster.Flush(s.ctx, segmentInfos, markSegments)
|
2021-10-20 15:02:36 +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-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-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-05-28 09:55:21 +08:00
|
|
|
s.stopServerLoop()
|
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
|
|
|
}
|
|
|
|
|
2021-08-19 13:00:12 +08:00
|
|
|
// GetVChanPositions get vchannel latest postitions with provided dml channel names
|
2021-10-14 15:44:34 +08:00
|
|
|
func (s *Server) GetVChanPositions(channel string, collectionID UniqueID, seekFromStartPosition bool) *datapb.VchannelInfo {
|
|
|
|
segments := s.meta.GetSegmentsByChannel(channel)
|
|
|
|
flushed := make([]*datapb.SegmentInfo, 0)
|
|
|
|
unflushed := make([]*datapb.SegmentInfo, 0)
|
|
|
|
var seekPosition *internalpb.MsgPosition
|
|
|
|
for _, s := range segments {
|
|
|
|
if s.State == commonpb.SegmentState_Flushing || s.State == commonpb.SegmentState_Flushed {
|
2021-10-22 14:35:13 +08:00
|
|
|
flushed = append(flushed, trimSegmentInfo(s.SegmentInfo))
|
2021-10-28 14:30:26 +08:00
|
|
|
if seekPosition == nil || (s.DmlPosition.Timestamp < seekPosition.Timestamp) {
|
2021-10-14 15:44:34 +08:00
|
|
|
seekPosition = s.DmlPosition
|
2021-08-19 13:00:12 +08:00
|
|
|
}
|
2021-10-14 15:44:34 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-05-20 11:34:45 +08:00
|
|
|
|
2021-10-28 14:30:26 +08:00
|
|
|
if s.DmlPosition == nil { // segment position all nil
|
2021-10-14 15:44:34 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-08-19 13:00:12 +08:00
|
|
|
|
2021-10-22 14:35:13 +08:00
|
|
|
unflushed = append(unflushed, trimSegmentInfo(s.SegmentInfo))
|
2021-08-19 13:00:12 +08:00
|
|
|
|
2021-10-28 14:30:26 +08:00
|
|
|
segmentPosition := s.DmlPosition
|
|
|
|
if seekFromStartPosition {
|
|
|
|
// need to use start position when load collection/partition, querynode does not support seek from checkpoint yet
|
|
|
|
// TODO silverxia remove seek from start logic after checkpoint supported in querynode
|
|
|
|
segmentPosition = s.StartPosition
|
|
|
|
}
|
|
|
|
|
|
|
|
if seekPosition == nil || segmentPosition.Timestamp < seekPosition.Timestamp {
|
|
|
|
seekPosition = segmentPosition
|
2021-05-26 12:21:55 +08:00
|
|
|
}
|
2021-10-14 15:44:34 +08:00
|
|
|
}
|
|
|
|
// use collection start position when segment position is not found
|
|
|
|
if seekPosition == nil {
|
|
|
|
coll := s.meta.GetCollection(collectionID)
|
|
|
|
if coll != nil {
|
|
|
|
for _, sp := range coll.GetStartPositions() {
|
|
|
|
if sp.GetKey() == rootcoord.ToPhysicalChannel(channel) {
|
|
|
|
seekPosition = &internalpb.MsgPosition{
|
|
|
|
ChannelName: channel,
|
|
|
|
MsgID: sp.GetData(),
|
2021-10-08 14:51:13 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-10-14 15:44:34 +08:00
|
|
|
}
|
2021-10-08 14:51:13 +08:00
|
|
|
|
2021-10-14 15:44:34 +08:00
|
|
|
return &datapb.VchannelInfo{
|
|
|
|
CollectionID: collectionID,
|
|
|
|
ChannelName: channel,
|
|
|
|
SeekPosition: seekPosition,
|
|
|
|
FlushedSegments: flushed,
|
|
|
|
UnflushedSegments: unflushed,
|
2021-08-19 13:00:12 +08:00
|
|
|
}
|
2021-05-20 11:34:45 +08:00
|
|
|
}
|
2021-10-22 14:35:13 +08:00
|
|
|
|
|
|
|
// trimSegmentInfo returns a shallow copy of datapb.SegmentInfo and sets ALL binlog info to nil
|
|
|
|
func trimSegmentInfo(info *datapb.SegmentInfo) *datapb.SegmentInfo {
|
|
|
|
return &datapb.SegmentInfo{
|
|
|
|
ID: info.ID,
|
|
|
|
CollectionID: info.CollectionID,
|
|
|
|
PartitionID: info.PartitionID,
|
|
|
|
InsertChannel: info.InsertChannel,
|
|
|
|
NumOfRows: info.NumOfRows,
|
|
|
|
State: info.State,
|
|
|
|
MaxRowNum: info.MaxRowNum,
|
|
|
|
LastExpireTime: info.LastExpireTime,
|
|
|
|
StartPosition: info.StartPosition,
|
|
|
|
DmlPosition: info.DmlPosition,
|
|
|
|
}
|
|
|
|
}
|