mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-02 11:59:00 +08:00
Hide liveChn into sessionutil and fix liveness init order (#9860)
Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
This commit is contained in:
parent
044b06421a
commit
ec6e8e4127
@ -104,7 +104,6 @@ type Server struct {
|
||||
msFactory msgstream.Factory
|
||||
|
||||
session *sessionutil.Session
|
||||
liveCh <-chan bool
|
||||
eventCh <-chan *sessionutil.SessionEvent
|
||||
|
||||
dataNodeCreator dataNodeCreatorFunc
|
||||
@ -187,7 +186,7 @@ func (s *Server) Register() error {
|
||||
if s.session == nil {
|
||||
return errors.New("failed to initialize session")
|
||||
}
|
||||
s.liveCh = s.session.Init(typeutil.DataCoordRole, Params.IP, true)
|
||||
s.session.Init(typeutil.DataCoordRole, Params.IP, true)
|
||||
Params.NodeID = s.session.ServerID
|
||||
Params.SetLogger(typeutil.UniqueID(-1))
|
||||
return nil
|
||||
@ -310,7 +309,7 @@ func (s *Server) startServerLoop() {
|
||||
go s.startDataNodeTtLoop(s.serverLoopCtx)
|
||||
go s.startWatchService(s.serverLoopCtx)
|
||||
go s.startFlushLoop(s.serverLoopCtx)
|
||||
go s.session.LivenessCheck(s.serverLoopCtx, s.liveCh, func() {
|
||||
go s.session.LivenessCheck(s.serverLoopCtx, func() {
|
||||
if err := s.Stop(); err != nil {
|
||||
log.Error("failed to stop server", zap.Error(err))
|
||||
}
|
||||
|
@ -101,7 +101,6 @@ type DataNode struct {
|
||||
dataCoord types.DataCoord
|
||||
|
||||
session *sessionutil.Session
|
||||
liveCh <-chan bool
|
||||
kvClient *etcdkv.EtcdKV
|
||||
|
||||
closer io.Closer
|
||||
@ -169,12 +168,19 @@ func (node *DataNode) SetNodeID(id UniqueID) {
|
||||
// Register register datanode to etcd
|
||||
func (node *DataNode) Register() error {
|
||||
node.session = sessionutil.NewSession(node.ctx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
node.liveCh = node.session.Init(typeutil.DataNodeRole, Params.IP+":"+strconv.Itoa(Params.Port), false)
|
||||
node.session.Init(typeutil.DataNodeRole, Params.IP+":"+strconv.Itoa(Params.Port), false)
|
||||
Params.NodeID = node.session.ServerID
|
||||
node.NodeID = node.session.ServerID
|
||||
Params.SetLogger(Params.NodeID)
|
||||
// Start node watch node
|
||||
go node.StartWatchChannels(node.ctx)
|
||||
// Start liveness check
|
||||
go node.session.LivenessCheck(node.ctx, func() {
|
||||
err := node.Stop()
|
||||
if err != nil {
|
||||
log.Warn("node stop failed", zap.Error(err))
|
||||
}
|
||||
})
|
||||
|
||||
Params.initMsgChannelSubName()
|
||||
//TODO reset
|
||||
@ -407,13 +413,6 @@ func (node *DataNode) Start() error {
|
||||
|
||||
go node.BackGroundGC(node.clearSignal)
|
||||
|
||||
go node.session.LivenessCheck(node.ctx, node.liveCh, func() {
|
||||
err := node.Stop()
|
||||
if err != nil {
|
||||
log.Warn("node stop failed", zap.Error(err))
|
||||
}
|
||||
})
|
||||
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
|
||||
|
@ -62,7 +62,6 @@ type IndexCoord struct {
|
||||
|
||||
sched *TaskScheduler
|
||||
session *sessionutil.Session
|
||||
liveCh <-chan bool
|
||||
|
||||
eventChan <-chan *sessionutil.SessionEvent
|
||||
|
||||
@ -115,7 +114,7 @@ func (i *IndexCoord) Register() error {
|
||||
if i.session == nil {
|
||||
return errors.New("failed to initialize session")
|
||||
}
|
||||
i.liveCh = i.session.Init(typeutil.IndexCoordRole, Params.Address, true)
|
||||
i.session.Init(typeutil.IndexCoordRole, Params.Address, true)
|
||||
Params.SetLogger(typeutil.UniqueID(-1))
|
||||
return nil
|
||||
}
|
||||
@ -248,7 +247,7 @@ func (i *IndexCoord) Start() error {
|
||||
i.loopWg.Add(1)
|
||||
go i.watchMetaLoop()
|
||||
|
||||
go i.session.LivenessCheck(i.loopCtx, i.liveCh, func() {
|
||||
go i.session.LivenessCheck(i.loopCtx, func() {
|
||||
i.Stop()
|
||||
})
|
||||
|
||||
|
@ -71,7 +71,6 @@ type IndexNode struct {
|
||||
|
||||
kv kv.BaseKV
|
||||
session *sessionutil.Session
|
||||
liveCh <-chan bool
|
||||
|
||||
// Add callback functions at different stages
|
||||
startCallbacks []func()
|
||||
@ -110,7 +109,7 @@ func (i *IndexNode) Register() error {
|
||||
if i.session == nil {
|
||||
return errors.New("failed to initialize session")
|
||||
}
|
||||
i.liveCh = i.session.Init(typeutil.IndexNodeRole, Params.IP+":"+strconv.Itoa(Params.Port), false)
|
||||
i.session.Init(typeutil.IndexNodeRole, Params.IP+":"+strconv.Itoa(Params.Port), false)
|
||||
Params.NodeID = i.session.ServerID
|
||||
Params.SetLogger(Params.NodeID)
|
||||
return nil
|
||||
@ -185,7 +184,7 @@ func (i *IndexNode) Start() error {
|
||||
Params.UpdatedTime = time.Now()
|
||||
|
||||
//start liveness check
|
||||
go i.session.LivenessCheck(i.loopCtx, i.liveCh, func() {
|
||||
go i.session.LivenessCheck(i.loopCtx, func() {
|
||||
i.Stop()
|
||||
})
|
||||
|
||||
|
@ -67,7 +67,6 @@ type QueryCoord struct {
|
||||
rootCoordClient types.RootCoord
|
||||
|
||||
session *sessionutil.Session
|
||||
liveCh <-chan bool
|
||||
eventChan <-chan *sessionutil.SessionEvent
|
||||
|
||||
stateCode atomic.Value
|
||||
@ -80,7 +79,7 @@ type QueryCoord struct {
|
||||
func (qc *QueryCoord) Register() error {
|
||||
log.Debug("query coord session info", zap.String("metaPath", Params.MetaRootPath), zap.Strings("etcdEndPoints", Params.EtcdEndpoints), zap.String("address", Params.Address))
|
||||
qc.session = sessionutil.NewSession(qc.loopCtx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
qc.liveCh = qc.session.Init(typeutil.QueryCoordRole, Params.Address, true)
|
||||
qc.session.Init(typeutil.QueryCoordRole, Params.Address, true)
|
||||
Params.NodeID = uint64(qc.session.ServerID)
|
||||
Params.SetLogger(typeutil.UniqueID(-1))
|
||||
return nil
|
||||
@ -147,7 +146,7 @@ func (qc *QueryCoord) Start() error {
|
||||
qc.loopWg.Add(1)
|
||||
go qc.watchMetaLoop()
|
||||
|
||||
go qc.session.LivenessCheck(qc.loopCtx, qc.liveCh, func() {
|
||||
go qc.session.LivenessCheck(qc.loopCtx, func() {
|
||||
qc.Stop()
|
||||
})
|
||||
|
||||
|
@ -67,8 +67,6 @@ type QueryNode struct {
|
||||
|
||||
//call once
|
||||
initOnce sync.Once
|
||||
// liveness channel with etcd
|
||||
liveCh <-chan bool
|
||||
|
||||
// internal components
|
||||
historical *historical
|
||||
@ -110,7 +108,12 @@ func NewQueryNode(ctx context.Context, factory msgstream.Factory) *QueryNode {
|
||||
func (node *QueryNode) Register() error {
|
||||
log.Debug("query node session info", zap.String("metaPath", Params.MetaRootPath), zap.Strings("etcdEndPoints", Params.EtcdEndpoints))
|
||||
node.session = sessionutil.NewSession(node.queryNodeLoopCtx, Params.MetaRootPath, Params.EtcdEndpoints)
|
||||
node.liveCh = node.session.Init(typeutil.QueryNodeRole, Params.QueryNodeIP+":"+strconv.FormatInt(Params.QueryNodePort, 10), false)
|
||||
node.session.Init(typeutil.QueryNodeRole, Params.QueryNodeIP+":"+strconv.FormatInt(Params.QueryNodePort, 10), false)
|
||||
// start liveness check
|
||||
go node.session.LivenessCheck(node.queryNodeLoopCtx, func() {
|
||||
node.Stop()
|
||||
})
|
||||
|
||||
Params.QueryNodeID = node.session.ServerID
|
||||
Params.SetLogger(Params.QueryNodeID)
|
||||
log.Debug("query nodeID", zap.Int64("nodeID", Params.QueryNodeID))
|
||||
@ -210,11 +213,6 @@ func (node *QueryNode) Start() error {
|
||||
// start services
|
||||
go node.historical.start()
|
||||
|
||||
// start liveness check
|
||||
go node.session.LivenessCheck(node.queryNodeLoopCtx, node.liveCh, func() {
|
||||
node.Stop()
|
||||
})
|
||||
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
|
||||
|
@ -151,8 +151,7 @@ type Core struct {
|
||||
startOnce sync.Once
|
||||
//isInit atomic.Value
|
||||
|
||||
session *sessionutil.Session
|
||||
sessCloseCh <-chan bool
|
||||
session *sessionutil.Session
|
||||
|
||||
msFactory ms.Factory
|
||||
}
|
||||
@ -288,25 +287,6 @@ func (c *Core) tsLoop() {
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Core) sessionLoop() {
|
||||
defer c.wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
case _, ok := <-c.sessCloseCh:
|
||||
if !ok {
|
||||
log.Error("rootcoord disconnected from etcd, process will exit in 1 second")
|
||||
go func() {
|
||||
time.Sleep(time.Second)
|
||||
os.Exit(-1)
|
||||
}()
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Core) checkFlushedSegmentsLoop() {
|
||||
defer c.wg.Done()
|
||||
ticker := time.NewTicker(10 * time.Minute)
|
||||
@ -862,7 +842,7 @@ func (c *Core) Register() error {
|
||||
if c.session == nil {
|
||||
return fmt.Errorf("session is nil, the etcd client connection may have failed")
|
||||
}
|
||||
c.sessCloseCh = c.session.Init(typeutil.RootCoordRole, Params.Address, true)
|
||||
c.session.Init(typeutil.RootCoordRole, Params.Address, true)
|
||||
Params.SetLogger(typeutil.UniqueID(-1))
|
||||
return nil
|
||||
}
|
||||
@ -1118,13 +1098,19 @@ func (c *Core) Start() error {
|
||||
log.Debug("RootCoord Start reSendDdMsg failed", zap.Error(err))
|
||||
return
|
||||
}
|
||||
c.wg.Add(5)
|
||||
c.wg.Add(4)
|
||||
go c.startTimeTickLoop()
|
||||
go c.tsLoop()
|
||||
go c.sessionLoop()
|
||||
go c.chanTimeTick.StartWatch(&c.wg)
|
||||
go c.checkFlushedSegmentsLoop()
|
||||
|
||||
go c.session.LivenessCheck(c.ctx, func() {
|
||||
log.Error("rootcoord disconnected from etcd, process will exit in 1 second")
|
||||
go func() {
|
||||
time.Sleep(time.Second)
|
||||
os.Exit(-1)
|
||||
}()
|
||||
})
|
||||
Params.CreatedTime = time.Now()
|
||||
Params.UpdatedTime = time.Now()
|
||||
|
||||
|
@ -49,6 +49,7 @@ type Session struct {
|
||||
Address string `json:"Address,omitempty"`
|
||||
Exclusive bool `json:"Exclusive,omitempty"`
|
||||
|
||||
liveCh <-chan bool
|
||||
etcdCli *clientv3.Client
|
||||
leaseID clientv3.LeaseID
|
||||
cancel context.CancelFunc
|
||||
@ -94,7 +95,7 @@ func NewSession(ctx context.Context, metaRoot string, etcdEndpoints []string) *S
|
||||
// Init will initialize base struct of the Session, including ServerName, ServerID,
|
||||
// Address, Exclusive. ServerID is obtained in getServerID.
|
||||
// Finally it will process keepAliveResponse to keep alive with etcd.
|
||||
func (s *Session) Init(serverName, address string, exclusive bool) <-chan bool {
|
||||
func (s *Session) Init(serverName, address string, exclusive bool) {
|
||||
s.ServerName = serverName
|
||||
s.Address = address
|
||||
s.Exclusive = exclusive
|
||||
@ -108,7 +109,7 @@ func (s *Session) Init(serverName, address string, exclusive bool) <-chan bool {
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return s.processKeepAliveResponse(ch)
|
||||
s.liveCh = s.processKeepAliveResponse(ch)
|
||||
}
|
||||
|
||||
func (s *Session) getServerID() (int64, error) {
|
||||
@ -348,10 +349,10 @@ func (s *Session) WatchServices(prefix string, revision int64) (eventChannel <-c
|
||||
// ctx controls the liveness check loop
|
||||
// ch is the liveness signal channel, ch is closed only when the session is expired
|
||||
// callback is the function to call when ch is closed, note that callback will not be invoked when loop exits due to context
|
||||
func (s *Session) LivenessCheck(ctx context.Context, ch <-chan bool, callback func()) {
|
||||
func (s *Session) LivenessCheck(ctx context.Context, callback func()) {
|
||||
for {
|
||||
select {
|
||||
case _, ok := <-ch:
|
||||
case _, ok := <-s.liveCh:
|
||||
// ok, still alive
|
||||
if ok {
|
||||
continue
|
||||
|
@ -170,11 +170,12 @@ func TestSessionLivenessCheck(t *testing.T) {
|
||||
s := &Session{}
|
||||
ctx := context.Background()
|
||||
ch := make(chan bool)
|
||||
s.liveCh = ch
|
||||
signal := make(chan struct{}, 1)
|
||||
|
||||
flag := false
|
||||
|
||||
go s.LivenessCheck(ctx, ch, func() {
|
||||
go s.LivenessCheck(ctx, func() {
|
||||
flag = true
|
||||
signal <- struct{}{}
|
||||
})
|
||||
@ -191,10 +192,10 @@ func TestSessionLivenessCheck(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
cancel()
|
||||
ch = make(chan bool)
|
||||
|
||||
s.liveCh = ch
|
||||
flag = false
|
||||
|
||||
go s.LivenessCheck(ctx, ch, func() {
|
||||
go s.LivenessCheck(ctx, func() {
|
||||
flag = true
|
||||
signal <- struct{}{}
|
||||
})
|
||||
|
Loading…
Reference in New Issue
Block a user