Skip to content

Commit 3d23ce4

Browse files
authored
feat: support wait sink node close (#3373)
Signed-off-by: Song Gao <[email protected]>
1 parent bfbb96d commit 3d23ce4

File tree

4 files changed

+45
-2
lines changed

4 files changed

+45
-2
lines changed

internal/server/rule_manager.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -341,6 +341,7 @@ func stopRule(name string) (result string, err error) {
341341
if err != nil {
342342
conf.Log.Warn(err)
343343
}
344+
rs.Topology.WaitOperatorClose()
344345
_, err = ruleProcessor.ExecReplaceRuleState(name, false)
345346
if err != nil {
346347
conf.Log.Warnf("stop rule found error: %s", err.Error())

internal/topo/context/default.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ import (
3434
const (
3535
LoggerKey = "$$logger"
3636
RuleStartKey = "$$ruleStart"
37+
RuleOpsWg = "$$ops_wg"
3738
)
3839

3940
type DefaultContext struct {

internal/topo/node/sink_node.go

Lines changed: 34 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,10 +16,12 @@ package node
1616

1717
import (
1818
"fmt"
19+
"sync"
1920

2021
"github.com/lf-edge/ekuiper/internal/binder/io"
2122
"github.com/lf-edge/ekuiper/internal/conf"
2223
"github.com/lf-edge/ekuiper/internal/topo/context"
24+
kctx "github.com/lf-edge/ekuiper/internal/topo/context"
2325
"github.com/lf-edge/ekuiper/internal/topo/node/cache"
2426
nodeConf "github.com/lf-edge/ekuiper/internal/topo/node/conf"
2527
"github.com/lf-edge/ekuiper/internal/topo/node/metric"
@@ -55,6 +57,7 @@ type SinkNode struct {
5557
// configs (also static for sinks)
5658
options map[string]interface{}
5759
isMock bool
60+
wg *sync.WaitGroup
5861
// states varies after restart
5962
sink api.Sink
6063
}
@@ -77,8 +80,21 @@ func NewSinkNodeWithSink(name string, sink api.Sink, props map[string]interface{
7780
}
7881
}
7982

83+
func (m *SinkNode) Close(ctx api.StreamContext) {
84+
if m.wg != nil {
85+
m.wg.Done()
86+
}
87+
}
88+
8089
func (m *SinkNode) Open(ctx api.StreamContext, result chan<- error) {
8190
m.ctx = ctx
91+
v := ctx.Value(kctx.RuleOpsWg)
92+
if v != nil {
93+
wg, ok := v.(*sync.WaitGroup)
94+
if ok {
95+
m.wg = wg
96+
}
97+
}
8298
logger := ctx.GetLogger()
8399
logger.Debugf("open sink node %s", m.name)
84100
go func() {
@@ -109,6 +125,20 @@ func (m *SinkNode) Open(ctx api.StreamContext, result chan<- error) {
109125

110126
go func(instance int) {
111127
panicOrError := infra.SafeRun(func() error {
128+
if m.wg != nil {
129+
m.wg.Add(1)
130+
}
131+
var exitCh1 chan struct{}
132+
var exitCh2 chan struct{}
133+
defer func() {
134+
if exitCh1 != nil {
135+
<-exitCh1
136+
}
137+
if exitCh2 != nil {
138+
<-exitCh2
139+
}
140+
m.Close(ctx)
141+
}()
112142
var (
113143
sink api.Sink
114144
err error
@@ -148,10 +178,12 @@ func (m *SinkNode) Open(ctx api.StreamContext, result chan<- error) {
148178
if !sconf.EnableCache {
149179
dataOutCh = dataCh
150180
} else {
151-
c = cache.NewSyncCache(ctx, dataCh, result, &sconf.SinkConf, sconf.BufferLength)
181+
exitCh1 = make(chan struct{}, 2)
182+
c = cache.NewSyncCacheWithExitChanel(ctx, dataCh, result, &sconf.SinkConf, sconf.BufferLength, exitCh1)
152183
if sconf.ResendAlterQueue {
153184
resendCh = make(chan []map[string]interface{}, sconf.BufferLength)
154-
rq = cache.NewSyncCache(ctx, resendCh, result, &sconf.SinkConf, sconf.BufferLength)
185+
exitCh2 = make(chan struct{}, 2)
186+
rq = cache.NewSyncCacheWithExitChanel(ctx, resendCh, result, &sconf.SinkConf, sconf.BufferLength, exitCh2)
155187
}
156188
dataOutCh = c.Out
157189
}

internal/topo/topo.go

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,7 @@ type Topo struct {
5151
topo *api.PrintableTopo
5252
mu sync.Mutex
5353
hasOpened atomic.Bool
54+
wg *sync.WaitGroup
5455
}
5556

5657
func NewWithNameAndOptions(name string, options *api.RuleOption) (*Topo, error) {
@@ -152,6 +153,12 @@ func (s *Topo) addEdge(from api.TopNode, to api.TopNode, toType string) {
152153
s.topo.Edges[f] = append(e, t)
153154
}
154155

156+
func (s *Topo) WaitOperatorClose() {
157+
if s.wg != nil {
158+
s.wg.Wait()
159+
}
160+
}
161+
155162
// prepareContext setups internal context before
156163
// stream starts execution.
157164
func (s *Topo) prepareContext() {
@@ -191,6 +198,8 @@ func (s *Topo) prepareContext() {
191198
}
192199
ctx := kctx.WithValue(kctx.Background(), kctx.LoggerKey, contextLogger)
193200
ctx = kctx.WithValue(ctx, kctx.RuleStartKey, conf.GetNowInMilli())
201+
s.wg = &sync.WaitGroup{}
202+
ctx = kctx.WithValue(ctx, kctx.RuleOpsWg, s.wg)
194203
s.ctx, s.cancel = ctx.WithCancel()
195204
}
196205
}

0 commit comments

Comments
 (0)