mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-02 20:09:57 +08:00
Reduce the goroutine in flowgraph to 2 (#28233)
Signed-off-by: lixinguo <xinguo.li@zilliz.com> Co-authored-by: lixinguo <xinguo.li@zilliz.com>
This commit is contained in:
parent
5365748338
commit
0aa90de141
@ -81,7 +81,7 @@ type nodeConfig struct {
|
|||||||
serverID UniqueID
|
serverID UniqueID
|
||||||
}
|
}
|
||||||
|
|
||||||
// start the flow graph in datasyncservice
|
// start the flow graph in dataSyncService
|
||||||
func (dsService *dataSyncService) start() {
|
func (dsService *dataSyncService) start() {
|
||||||
if dsService.fg != nil {
|
if dsService.fg != nil {
|
||||||
log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID),
|
log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID),
|
||||||
|
@ -84,7 +84,7 @@ func (node *DataNode) StartWatchChannels(ctx context.Context) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// checkWatchedList list all nodes under [prefix]/channel/{node_id} and make sure all nodeds are watched
|
// checkWatchedList list all nodes under [prefix]/channel/{node_id} and make sure all nodes are watched
|
||||||
// serves the corner case for etcd connection lost and missing some events
|
// serves the corner case for etcd connection lost and missing some events
|
||||||
func (node *DataNode) checkWatchedList() error {
|
func (node *DataNode) checkWatchedList() error {
|
||||||
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
|
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
|
||||||
|
@ -276,7 +276,9 @@ func (ddn *ddNode) isDropped(segID UniqueID) bool {
|
|||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ddn *ddNode) Close() {}
|
func (ddn *ddNode) Close() {
|
||||||
|
log.Info("Flowgraph DD Node closing")
|
||||||
|
}
|
||||||
|
|
||||||
func newDDNode(ctx context.Context, collID UniqueID, vChannelName string, droppedSegmentIDs []UniqueID,
|
func newDDNode(ctx context.Context, collID UniqueID, vChannelName string, droppedSegmentIDs []UniqueID,
|
||||||
sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, compactor *compactionExecutor,
|
sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, compactor *compactionExecutor,
|
||||||
|
@ -30,20 +30,22 @@ import (
|
|||||||
// TimeTickedFlowGraph flowgraph with input from tt msg stream
|
// TimeTickedFlowGraph flowgraph with input from tt msg stream
|
||||||
type TimeTickedFlowGraph struct {
|
type TimeTickedFlowGraph struct {
|
||||||
nodeCtx map[NodeName]*nodeCtx
|
nodeCtx map[NodeName]*nodeCtx
|
||||||
|
nodeCtxManager *nodeCtxManager
|
||||||
stopOnce sync.Once
|
stopOnce sync.Once
|
||||||
startOnce sync.Once
|
startOnce sync.Once
|
||||||
closeWg *sync.WaitGroup
|
closeWg *sync.WaitGroup
|
||||||
closeGracefully *atomic.Bool
|
closeGracefully *atomic.Bool
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddNode add Node into flowgraph
|
// AddNode add Node into flowgraph and fill nodeCtxManager
|
||||||
func (fg *TimeTickedFlowGraph) AddNode(node Node) {
|
func (fg *TimeTickedFlowGraph) AddNode(node Node) {
|
||||||
nodeCtx := nodeCtx{
|
nodeCtx := nodeCtx{
|
||||||
node: node,
|
node: node,
|
||||||
closeCh: make(chan struct{}),
|
|
||||||
closeWg: fg.closeWg,
|
|
||||||
}
|
}
|
||||||
fg.nodeCtx[node.Name()] = &nodeCtx
|
fg.nodeCtx[node.Name()] = &nodeCtx
|
||||||
|
if node.IsInputNode() {
|
||||||
|
fg.nodeCtxManager = NewNodeCtxManager(&nodeCtx, fg.closeWg)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// SetEdges set directed edges from in nodes to out nodes
|
// SetEdges set directed edges from in nodes to out nodes
|
||||||
@ -79,8 +81,9 @@ func (fg *TimeTickedFlowGraph) SetEdges(nodeName string, out []string) error {
|
|||||||
func (fg *TimeTickedFlowGraph) Start() {
|
func (fg *TimeTickedFlowGraph) Start() {
|
||||||
fg.startOnce.Do(func() {
|
fg.startOnce.Do(func() {
|
||||||
for _, v := range fg.nodeCtx {
|
for _, v := range fg.nodeCtx {
|
||||||
v.Start()
|
v.node.Start()
|
||||||
}
|
}
|
||||||
|
fg.nodeCtxManager.Start()
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -120,6 +123,7 @@ func (fg *TimeTickedFlowGraph) Close() {
|
|||||||
func NewTimeTickedFlowGraph(ctx context.Context) *TimeTickedFlowGraph {
|
func NewTimeTickedFlowGraph(ctx context.Context) *TimeTickedFlowGraph {
|
||||||
flowGraph := TimeTickedFlowGraph{
|
flowGraph := TimeTickedFlowGraph{
|
||||||
nodeCtx: make(map[string]*nodeCtx),
|
nodeCtx: make(map[string]*nodeCtx),
|
||||||
|
nodeCtxManager: &nodeCtxManager{},
|
||||||
closeWg: &sync.WaitGroup{},
|
closeWg: &sync.WaitGroup{},
|
||||||
closeGracefully: atomic.NewBool(CloseImmediately),
|
closeGracefully: atomic.NewBool(CloseImmediately),
|
||||||
}
|
}
|
||||||
|
@ -35,7 +35,7 @@ import (
|
|||||||
// nodeD: count c = b + 2
|
// nodeD: count c = b + 2
|
||||||
|
|
||||||
type nodeA struct {
|
type nodeA struct {
|
||||||
BaseNode
|
InputNode
|
||||||
inputChan chan float64
|
inputChan chan float64
|
||||||
a float64
|
a float64
|
||||||
}
|
}
|
||||||
@ -47,7 +47,7 @@ type nodeB struct {
|
|||||||
|
|
||||||
type nodeC struct {
|
type nodeC struct {
|
||||||
BaseNode
|
BaseNode
|
||||||
d float64
|
c float64
|
||||||
outputChan chan float64
|
outputChan chan float64
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -117,9 +117,11 @@ func createExampleFlowGraph() (*TimeTickedFlowGraph, chan float64, chan float64,
|
|||||||
fg := NewTimeTickedFlowGraph(ctx)
|
fg := NewTimeTickedFlowGraph(ctx)
|
||||||
|
|
||||||
var a Node = &nodeA{
|
var a Node = &nodeA{
|
||||||
|
InputNode: InputNode{
|
||||||
BaseNode: BaseNode{
|
BaseNode: BaseNode{
|
||||||
maxQueueLength: MaxQueueLength,
|
maxQueueLength: MaxQueueLength,
|
||||||
},
|
},
|
||||||
|
},
|
||||||
inputChan: inputChan,
|
inputChan: inputChan,
|
||||||
}
|
}
|
||||||
var b Node = &nodeB{
|
var b Node = &nodeB{
|
||||||
@ -175,9 +177,11 @@ func TestTimeTickedFlowGraph_AddNode(t *testing.T) {
|
|||||||
fg := NewTimeTickedFlowGraph(context.TODO())
|
fg := NewTimeTickedFlowGraph(context.TODO())
|
||||||
|
|
||||||
var a Node = &nodeA{
|
var a Node = &nodeA{
|
||||||
|
InputNode: InputNode{
|
||||||
BaseNode: BaseNode{
|
BaseNode: BaseNode{
|
||||||
maxQueueLength: MaxQueueLength,
|
maxQueueLength: MaxQueueLength,
|
||||||
},
|
},
|
||||||
|
},
|
||||||
inputChan: inputChan,
|
inputChan: inputChan,
|
||||||
}
|
}
|
||||||
var b Node = &nodeB{
|
var b Node = &nodeB{
|
||||||
|
@ -19,7 +19,6 @@ package flowgraph
|
|||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"sync"
|
|
||||||
|
|
||||||
"go.opentelemetry.io/otel"
|
"go.opentelemetry.io/otel"
|
||||||
"go.opentelemetry.io/otel/trace"
|
"go.opentelemetry.io/otel/trace"
|
||||||
@ -49,7 +48,6 @@ type InputNode struct {
|
|||||||
collectionID int64
|
collectionID int64
|
||||||
dataType string
|
dataType string
|
||||||
|
|
||||||
closeOnce sync.Once
|
|
||||||
closeGracefully *atomic.Bool
|
closeGracefully *atomic.Bool
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -53,26 +53,169 @@ type BaseNode struct {
|
|||||||
maxParallelism int32
|
maxParallelism int32
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// manage nodeCtx
|
||||||
|
type nodeCtxManager struct {
|
||||||
|
inputNodeCtx *nodeCtx
|
||||||
|
closeWg *sync.WaitGroup
|
||||||
|
closeOnce sync.Once
|
||||||
|
|
||||||
|
inputNodeCloseCh chan struct{} // notify input node work to exit
|
||||||
|
workNodeCh chan struct{} // notify ddnode and downstream node work to exit
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewNodeCtxManager init with the inputNode and fg.closeWg
|
||||||
|
func NewNodeCtxManager(nodeCtx *nodeCtx, closeWg *sync.WaitGroup) *nodeCtxManager {
|
||||||
|
return &nodeCtxManager{
|
||||||
|
inputNodeCtx: nodeCtx,
|
||||||
|
closeWg: closeWg,
|
||||||
|
inputNodeCloseCh: make(chan struct{}),
|
||||||
|
workNodeCh: make(chan struct{}),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Start invoke Node `Start` method and start a worker goroutine
|
||||||
|
func (nodeCtxManager *nodeCtxManager) Start() {
|
||||||
|
// in dmInputNode, message from mq to channel, alloc goroutines
|
||||||
|
// limit the goroutines in other node to prevent huge goroutines numbers
|
||||||
|
nodeCtxManager.closeWg.Add(2)
|
||||||
|
go nodeCtxManager.inputNodeStart()
|
||||||
|
go nodeCtxManager.workNodeStart()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (nodeCtxManager *nodeCtxManager) inputNodeStart() {
|
||||||
|
defer nodeCtxManager.closeWg.Done()
|
||||||
|
inputNode := nodeCtxManager.inputNodeCtx
|
||||||
|
name := fmt.Sprintf("nodeCtxTtChecker-%s", inputNode.node.Name())
|
||||||
|
// tt checker start
|
||||||
|
var checker *timerecord.GroupChecker
|
||||||
|
if enableTtChecker {
|
||||||
|
checker = timerecord.GetGroupChecker("fgNode", nodeCtxTtInterval, func(list []string) {
|
||||||
|
log.Warn("some node(s) haven't received input", zap.Strings("list", list), zap.Duration("duration ", nodeCtxTtInterval))
|
||||||
|
})
|
||||||
|
checker.Check(name)
|
||||||
|
defer checker.Remove(name)
|
||||||
|
}
|
||||||
|
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-nodeCtxManager.inputNodeCloseCh:
|
||||||
|
return
|
||||||
|
// handles node work spinning
|
||||||
|
// 1. collectMessage from upstream or just produce Msg from InputNode
|
||||||
|
// 2. invoke node.Operate
|
||||||
|
// 3. deliver the Operate result to downstream nodes
|
||||||
|
default:
|
||||||
|
// inputs from inputsMessages for Operate
|
||||||
|
var input, output []Msg
|
||||||
|
// inputNode.input not from nodeCtx.inputChannel
|
||||||
|
// the input message decides whether the operate method is executed
|
||||||
|
n := inputNode.node
|
||||||
|
inputNode.blockMutex.RLock()
|
||||||
|
if !n.IsValidInMsg(input) {
|
||||||
|
inputNode.blockMutex.RUnlock()
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
output = n.Operate(input)
|
||||||
|
inputNode.blockMutex.RUnlock()
|
||||||
|
// the output decide whether the node should be closed.
|
||||||
|
if isCloseMsg(output) {
|
||||||
|
close(nodeCtxManager.inputNodeCloseCh)
|
||||||
|
// inputNode.Close()
|
||||||
|
if inputNode.inputChannel != nil {
|
||||||
|
close(inputNode.inputChannel)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// deliver to all following flow graph node.
|
||||||
|
inputNode.downstream.inputChannel <- output
|
||||||
|
if enableTtChecker {
|
||||||
|
checker.Check(name)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (nodeCtxManager *nodeCtxManager) workNodeStart() {
|
||||||
|
defer nodeCtxManager.closeWg.Done()
|
||||||
|
ddNode := nodeCtxManager.inputNodeCtx.downstream
|
||||||
|
curNode := ddNode
|
||||||
|
// tt checker start
|
||||||
|
var checker *timerecord.GroupChecker
|
||||||
|
if enableTtChecker {
|
||||||
|
checker = timerecord.GetGroupChecker("fgNode", nodeCtxTtInterval, func(list []string) {
|
||||||
|
log.Warn("some node(s) haven't received input", zap.Strings("list", list), zap.Duration("duration ", nodeCtxTtInterval))
|
||||||
|
})
|
||||||
|
for curNode != nil {
|
||||||
|
name := fmt.Sprintf("nodeCtxTtChecker-%s", curNode.node.Name())
|
||||||
|
checker.Check(name)
|
||||||
|
curNode = curNode.downstream
|
||||||
|
defer checker.Remove(name)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-nodeCtxManager.workNodeCh:
|
||||||
|
return
|
||||||
|
// handles node work spinning
|
||||||
|
// 1. collectMessage from upstream or just produce Msg from InputNode
|
||||||
|
// 2. invoke node.Operate
|
||||||
|
// 3. deliver the Operate result to downstream nodes
|
||||||
|
default:
|
||||||
|
// goroutine will work loop for all node(expect inpuNode) even when closeCh notify to exit
|
||||||
|
// input node will close all node
|
||||||
|
curNode = ddNode
|
||||||
|
for curNode != nil {
|
||||||
|
// inputs from inputsMessages for Operate
|
||||||
|
var input, output []Msg
|
||||||
|
input = <-curNode.inputChannel
|
||||||
|
// the input message decides whether the operate method is executed
|
||||||
|
n := curNode.node
|
||||||
|
curNode.blockMutex.RLock()
|
||||||
|
if !n.IsValidInMsg(input) {
|
||||||
|
curNode.blockMutex.RUnlock()
|
||||||
|
curNode = ddNode
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
output = n.Operate(input)
|
||||||
|
curNode.blockMutex.RUnlock()
|
||||||
|
// the output decide whether the node should be closed.
|
||||||
|
if isCloseMsg(output) {
|
||||||
|
nodeCtxManager.closeOnce.Do(func() {
|
||||||
|
close(nodeCtxManager.workNodeCh)
|
||||||
|
})
|
||||||
|
if curNode.inputChannel != nil {
|
||||||
|
close(curNode.inputChannel)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// deliver to all following flow graph node.
|
||||||
|
if curNode.downstream != nil {
|
||||||
|
curNode.downstream.inputChannel <- output
|
||||||
|
}
|
||||||
|
if enableTtChecker {
|
||||||
|
checker.Check(fmt.Sprintf("nodeCtxTtChecker-%s", curNode.node.Name()))
|
||||||
|
}
|
||||||
|
curNode = curNode.downstream
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Close handles cleanup logic and notify worker to quit
|
||||||
|
func (nodeCtxManager *nodeCtxManager) Close() {
|
||||||
|
nodeCtx := nodeCtxManager.inputNodeCtx
|
||||||
|
nodeCtx.Close()
|
||||||
|
}
|
||||||
|
|
||||||
// nodeCtx maintains the running context for a Node in flowgragh
|
// nodeCtx maintains the running context for a Node in flowgragh
|
||||||
type nodeCtx struct {
|
type nodeCtx struct {
|
||||||
node Node
|
node Node
|
||||||
inputChannel chan []Msg
|
inputChannel chan []Msg
|
||||||
downstream *nodeCtx
|
downstream *nodeCtx
|
||||||
|
|
||||||
closeCh chan struct{} // notify work to exit
|
|
||||||
closeWg *sync.WaitGroup
|
|
||||||
|
|
||||||
blockMutex sync.RWMutex
|
blockMutex sync.RWMutex
|
||||||
}
|
}
|
||||||
|
|
||||||
// Start invoke Node `Start` method and start a worker goroutine
|
|
||||||
func (nodeCtx *nodeCtx) Start() {
|
|
||||||
nodeCtx.node.Start()
|
|
||||||
|
|
||||||
nodeCtx.closeWg.Add(1)
|
|
||||||
go nodeCtx.work()
|
|
||||||
}
|
|
||||||
|
|
||||||
func (nodeCtx *nodeCtx) Block() {
|
func (nodeCtx *nodeCtx) Block() {
|
||||||
// input node operate function will be blocking
|
// input node operate function will be blocking
|
||||||
if !nodeCtx.node.IsInputNode() {
|
if !nodeCtx.node.IsInputNode() {
|
||||||
@ -99,67 +242,14 @@ func isCloseMsg(msgs []Msg) bool {
|
|||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
// work handles node work spinning
|
|
||||||
// 1. collectMessage from upstream or just produce Msg from InputNode
|
|
||||||
// 2. invoke node.Operate
|
|
||||||
// 3. deliver the Operate result to downstream nodes
|
|
||||||
func (nodeCtx *nodeCtx) work() {
|
|
||||||
name := fmt.Sprintf("nodeCtxTtChecker-%s", nodeCtx.node.Name())
|
|
||||||
var checker *timerecord.GroupChecker
|
|
||||||
if enableTtChecker {
|
|
||||||
checker = timerecord.GetGroupChecker("fgNode", nodeCtxTtInterval, func(list []string) {
|
|
||||||
log.Warn("some node(s) haven't received input", zap.Strings("list", list), zap.Duration("duration ", nodeCtxTtInterval))
|
|
||||||
})
|
|
||||||
checker.Check(name)
|
|
||||||
defer checker.Remove(name)
|
|
||||||
}
|
|
||||||
|
|
||||||
for {
|
|
||||||
select {
|
|
||||||
case <-nodeCtx.closeCh:
|
|
||||||
log.Debug("flow graph node closed", zap.String("nodeName", nodeCtx.node.Name()))
|
|
||||||
return
|
|
||||||
default:
|
|
||||||
// inputs from inputsMessages for Operate
|
|
||||||
var input, output []Msg
|
|
||||||
if !nodeCtx.node.IsInputNode() {
|
|
||||||
input = <-nodeCtx.inputChannel
|
|
||||||
}
|
|
||||||
// the input message decides whether the operate method is executed
|
|
||||||
n := nodeCtx.node
|
|
||||||
nodeCtx.blockMutex.RLock()
|
|
||||||
if !n.IsValidInMsg(input) {
|
|
||||||
nodeCtx.blockMutex.RUnlock()
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
output = n.Operate(input)
|
|
||||||
nodeCtx.blockMutex.RUnlock()
|
|
||||||
// the output decide whether the node should be closed.
|
|
||||||
if isCloseMsg(output) {
|
|
||||||
close(nodeCtx.closeCh)
|
|
||||||
nodeCtx.closeWg.Done()
|
|
||||||
nodeCtx.node.Close()
|
|
||||||
if nodeCtx.inputChannel != nil {
|
|
||||||
close(nodeCtx.inputChannel)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if enableTtChecker {
|
|
||||||
checker.Check(name)
|
|
||||||
}
|
|
||||||
|
|
||||||
// deliver to all following flow graph node.
|
|
||||||
if nodeCtx.downstream != nil {
|
|
||||||
nodeCtx.downstream.inputChannel <- output
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Close handles cleanup logic and notify worker to quit
|
// Close handles cleanup logic and notify worker to quit
|
||||||
func (nodeCtx *nodeCtx) Close() {
|
func (nodeCtx *nodeCtx) Close() {
|
||||||
if nodeCtx.node.IsInputNode() {
|
if nodeCtx.node.IsInputNode() {
|
||||||
|
for nodeCtx != nil {
|
||||||
nodeCtx.node.Close()
|
nodeCtx.node.Close()
|
||||||
|
log.Debug("flow graph node closed", zap.String("nodeName", nodeCtx.node.Name()))
|
||||||
|
nodeCtx = nodeCtx.downstream
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -56,7 +56,7 @@ func generateMsgPack() msgstream.MsgPack {
|
|||||||
return msgPack
|
return msgPack
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestNodeCtx_Start(t *testing.T) {
|
func TestNodeManager_Start(t *testing.T) {
|
||||||
t.Setenv("ROCKSMQ_PATH", "/tmp/MilvusTest/FlowGraph/TestNodeStart")
|
t.Setenv("ROCKSMQ_PATH", "/tmp/MilvusTest/FlowGraph/TestNodeStart")
|
||||||
factory := dependency.NewDefaultFactory(true)
|
factory := dependency.NewDefaultFactory(true)
|
||||||
|
|
||||||
@ -76,19 +76,30 @@ func TestNodeCtx_Start(t *testing.T) {
|
|||||||
nodeName := "input_node"
|
nodeName := "input_node"
|
||||||
inputNode := NewInputNode(msgStream.Chan(), nodeName, 100, 100, "", 0, 0, "")
|
inputNode := NewInputNode(msgStream.Chan(), nodeName, 100, 100, "", 0, 0, "")
|
||||||
|
|
||||||
node := &nodeCtx{
|
ddNode := BaseNode{}
|
||||||
|
|
||||||
|
node0 := &nodeCtx{
|
||||||
node: inputNode,
|
node: inputNode,
|
||||||
closeCh: make(chan struct{}),
|
}
|
||||||
|
|
||||||
|
node1 := &nodeCtx{
|
||||||
|
node: &ddNode,
|
||||||
|
}
|
||||||
|
|
||||||
|
node0.downstream = node1
|
||||||
|
|
||||||
|
node0.inputChannel = make(chan []Msg)
|
||||||
|
|
||||||
|
nodeCtxManager := &nodeCtxManager{
|
||||||
|
inputNodeCtx: node0,
|
||||||
closeWg: &sync.WaitGroup{},
|
closeWg: &sync.WaitGroup{},
|
||||||
}
|
}
|
||||||
|
|
||||||
node.inputChannel = make(chan []Msg)
|
|
||||||
|
|
||||||
assert.NotPanics(t, func() {
|
assert.NotPanics(t, func() {
|
||||||
node.Start()
|
nodeCtxManager.Start()
|
||||||
})
|
})
|
||||||
|
|
||||||
node.Close()
|
nodeCtxManager.Close()
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestBaseNode(t *testing.T) {
|
func TestBaseNode(t *testing.T) {
|
||||||
|
Loading…
Reference in New Issue
Block a user