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:
smellthemoon 2023-11-13 10:50:17 +08:00 committed by GitHub
parent 5365748338
commit 0aa90de141
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 202 additions and 93 deletions

View File

@ -81,7 +81,7 @@ type nodeConfig struct {
serverID UniqueID
}
// start the flow graph in datasyncservice
// start the flow graph in dataSyncService
func (dsService *dataSyncService) start() {
if dsService.fg != nil {
log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID),

View File

@ -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
func (node *DataNode) checkWatchedList() error {
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}

View File

@ -276,7 +276,9 @@ func (ddn *ddNode) isDropped(segID UniqueID) bool {
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,
sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, compactor *compactionExecutor,

View File

@ -30,20 +30,22 @@ import (
// TimeTickedFlowGraph flowgraph with input from tt msg stream
type TimeTickedFlowGraph struct {
nodeCtx map[NodeName]*nodeCtx
nodeCtxManager *nodeCtxManager
stopOnce sync.Once
startOnce sync.Once
closeWg *sync.WaitGroup
closeGracefully *atomic.Bool
}
// AddNode add Node into flowgraph
// AddNode add Node into flowgraph and fill nodeCtxManager
func (fg *TimeTickedFlowGraph) AddNode(node Node) {
nodeCtx := nodeCtx{
node: node,
closeCh: make(chan struct{}),
closeWg: fg.closeWg,
}
fg.nodeCtx[node.Name()] = &nodeCtx
if node.IsInputNode() {
fg.nodeCtxManager = NewNodeCtxManager(&nodeCtx, fg.closeWg)
}
}
// 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() {
fg.startOnce.Do(func() {
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 {
flowGraph := TimeTickedFlowGraph{
nodeCtx: make(map[string]*nodeCtx),
nodeCtxManager: &nodeCtxManager{},
closeWg: &sync.WaitGroup{},
closeGracefully: atomic.NewBool(CloseImmediately),
}

View File

@ -35,7 +35,7 @@ import (
// nodeD: count c = b + 2
type nodeA struct {
BaseNode
InputNode
inputChan chan float64
a float64
}
@ -47,7 +47,7 @@ type nodeB struct {
type nodeC struct {
BaseNode
d float64
c float64
outputChan chan float64
}
@ -117,9 +117,11 @@ func createExampleFlowGraph() (*TimeTickedFlowGraph, chan float64, chan float64,
fg := NewTimeTickedFlowGraph(ctx)
var a Node = &nodeA{
InputNode: InputNode{
BaseNode: BaseNode{
maxQueueLength: MaxQueueLength,
},
},
inputChan: inputChan,
}
var b Node = &nodeB{
@ -175,9 +177,11 @@ func TestTimeTickedFlowGraph_AddNode(t *testing.T) {
fg := NewTimeTickedFlowGraph(context.TODO())
var a Node = &nodeA{
InputNode: InputNode{
BaseNode: BaseNode{
maxQueueLength: MaxQueueLength,
},
},
inputChan: inputChan,
}
var b Node = &nodeB{

View File

@ -19,7 +19,6 @@ package flowgraph
import (
"context"
"fmt"
"sync"
"go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/trace"
@ -49,7 +48,6 @@ type InputNode struct {
collectionID int64
dataType string
closeOnce sync.Once
closeGracefully *atomic.Bool
}

View File

@ -53,26 +53,169 @@ type BaseNode struct {
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
type nodeCtx struct {
node Node
inputChannel chan []Msg
downstream *nodeCtx
closeCh chan struct{} // notify work to exit
closeWg *sync.WaitGroup
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() {
// input node operate function will be blocking
if !nodeCtx.node.IsInputNode() {
@ -99,67 +242,14 @@ func isCloseMsg(msgs []Msg) bool {
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
func (nodeCtx *nodeCtx) Close() {
if nodeCtx.node.IsInputNode() {
for nodeCtx != nil {
nodeCtx.node.Close()
log.Debug("flow graph node closed", zap.String("nodeName", nodeCtx.node.Name()))
nodeCtx = nodeCtx.downstream
}
}
}

View File

@ -56,7 +56,7 @@ func generateMsgPack() msgstream.MsgPack {
return msgPack
}
func TestNodeCtx_Start(t *testing.T) {
func TestNodeManager_Start(t *testing.T) {
t.Setenv("ROCKSMQ_PATH", "/tmp/MilvusTest/FlowGraph/TestNodeStart")
factory := dependency.NewDefaultFactory(true)
@ -76,19 +76,30 @@ func TestNodeCtx_Start(t *testing.T) {
nodeName := "input_node"
inputNode := NewInputNode(msgStream.Chan(), nodeName, 100, 100, "", 0, 0, "")
node := &nodeCtx{
ddNode := BaseNode{}
node0 := &nodeCtx{
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{},
}
node.inputChannel = make(chan []Msg)
assert.NotPanics(t, func() {
node.Start()
nodeCtxManager.Start()
})
node.Close()
nodeCtxManager.Close()
}
func TestBaseNode(t *testing.T) {