sink_node.go 16 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578
  1. // Copyright 2022-2023 EMQ Technologies Co., Ltd.
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package node
  15. import (
  16. "fmt"
  17. "strings"
  18. "sync"
  19. "github.com/lf-edge/ekuiper/internal/binder/io"
  20. "github.com/lf-edge/ekuiper/internal/conf"
  21. sinkUtil "github.com/lf-edge/ekuiper/internal/io/sink"
  22. "github.com/lf-edge/ekuiper/internal/topo/context"
  23. "github.com/lf-edge/ekuiper/internal/topo/node/cache"
  24. nodeConf "github.com/lf-edge/ekuiper/internal/topo/node/conf"
  25. "github.com/lf-edge/ekuiper/internal/topo/node/metric"
  26. "github.com/lf-edge/ekuiper/internal/topo/transform"
  27. "github.com/lf-edge/ekuiper/internal/xsql"
  28. "github.com/lf-edge/ekuiper/pkg/api"
  29. "github.com/lf-edge/ekuiper/pkg/cast"
  30. "github.com/lf-edge/ekuiper/pkg/errorx"
  31. "github.com/lf-edge/ekuiper/pkg/infra"
  32. "github.com/lf-edge/ekuiper/pkg/message"
  33. )
  34. type SinkConf struct {
  35. Concurrency int `json:"concurrency"`
  36. Omitempty bool `json:"omitIfEmpty"`
  37. SendSingle bool `json:"sendSingle"`
  38. DataTemplate string `json:"dataTemplate"`
  39. Format string `json:"format"`
  40. SchemaId string `json:"schemaId"`
  41. Delimiter string `json:"delimiter"`
  42. BufferLength int `json:"bufferLength"`
  43. Fields []string `json:"fields"`
  44. DataField string `json:"dataField"`
  45. BatchSize int `json:"batchSize"`
  46. LingerInterval int `json:"lingerInterval"`
  47. conf.SinkConf
  48. }
  49. func (sc *SinkConf) isBatchSinkEnabled() bool {
  50. if sc.BatchSize > 0 || sc.LingerInterval > 0 {
  51. return true
  52. }
  53. return false
  54. }
  55. type SinkNode struct {
  56. *defaultSinkNode
  57. // static
  58. sinkType string
  59. mutex sync.RWMutex
  60. // configs (also static for sinks)
  61. options map[string]interface{}
  62. isMock bool
  63. // states varies after restart
  64. sinks []api.Sink
  65. }
  66. func NewSinkNode(name string, sinkType string, props map[string]interface{}) *SinkNode {
  67. bufferLength := 1024
  68. if c, ok := props["bufferLength"]; ok {
  69. if t, err := cast.ToInt(c, cast.STRICT); err != nil || t <= 0 {
  70. // invalid property bufferLength
  71. } else {
  72. bufferLength = t
  73. }
  74. }
  75. return &SinkNode{
  76. defaultSinkNode: &defaultSinkNode{
  77. input: make(chan interface{}, bufferLength),
  78. defaultNode: &defaultNode{
  79. name: name,
  80. concurrency: 1,
  81. ctx: nil,
  82. },
  83. },
  84. sinkType: sinkType,
  85. options: props,
  86. }
  87. }
  88. // NewSinkNodeWithSink Only for mock source, do not use it in production
  89. func NewSinkNodeWithSink(name string, sink api.Sink, props map[string]interface{}) *SinkNode {
  90. return &SinkNode{
  91. defaultSinkNode: &defaultSinkNode{
  92. input: make(chan interface{}, 1024),
  93. defaultNode: &defaultNode{
  94. name: name,
  95. concurrency: 1,
  96. ctx: nil,
  97. },
  98. },
  99. sinks: []api.Sink{sink},
  100. options: props,
  101. isMock: true,
  102. }
  103. }
  104. func (m *SinkNode) Open(ctx api.StreamContext, result chan<- error) {
  105. m.ctx = ctx
  106. logger := ctx.GetLogger()
  107. logger.Debugf("open sink node %s", m.name)
  108. go func() {
  109. err := infra.SafeRun(func() error {
  110. sconf, err := m.parseConf(logger)
  111. if err != nil {
  112. return err
  113. }
  114. tf, err := transform.GenTransform(sconf.DataTemplate, sconf.Format, sconf.SchemaId, sconf.Delimiter, sconf.DataField, sconf.Fields)
  115. if err != nil {
  116. msg := fmt.Sprintf("property dataTemplate %v is invalid: %v", sconf.DataTemplate, err)
  117. logger.Warnf(msg)
  118. return fmt.Errorf(msg)
  119. }
  120. ctx = context.WithValue(ctx.(*context.DefaultContext), context.TransKey, tf)
  121. m.reset()
  122. logger.Infof("open sink node %d instances", m.concurrency)
  123. for i := 0; i < m.concurrency; i++ { // workers
  124. go func(instance int) {
  125. panicOrError := infra.SafeRun(func() error {
  126. var (
  127. sink api.Sink
  128. err error
  129. )
  130. if !m.isMock {
  131. logger.Debugf("Trying to get sink for rule %s with options %v\n", ctx.GetRuleId(), m.options)
  132. sink, err = getSink(m.sinkType, m.options)
  133. if err != nil {
  134. return err
  135. }
  136. logger.Debugf("Successfully get the sink %s", m.sinkType)
  137. m.mutex.Lock()
  138. m.sinks = append(m.sinks, sink)
  139. m.mutex.Unlock()
  140. logger.Debugf("Now is to open sink for rule %s.\n", ctx.GetRuleId())
  141. if err := sink.Open(ctx); err != nil {
  142. return err
  143. }
  144. logger.Debugf("Successfully open sink for rule %s.\n", ctx.GetRuleId())
  145. } else {
  146. sink = m.sinks[instance]
  147. }
  148. stats, err := metric.NewStatManager(ctx, "sink")
  149. if err != nil {
  150. return err
  151. }
  152. m.mutex.Lock()
  153. m.statManagers = append(m.statManagers, stats)
  154. m.mutex.Unlock()
  155. // The sink flow is: receive -> batch -> cache -> send.
  156. // In the outside loop, send received data to batch/cache by dataCh and receive data be dataOutCh
  157. // Only need to deal with dataOutCh in the outer loop
  158. dataCh := make(chan []map[string]interface{}, sconf.BufferLength)
  159. var (
  160. dataOutCh <-chan []map[string]interface{}
  161. resendCh chan []map[string]interface{}
  162. sendManager *sinkUtil.SendManager
  163. c *cache.SyncCache
  164. rq *cache.SyncCache
  165. )
  166. if sconf.isBatchSinkEnabled() {
  167. sendManager, err = sinkUtil.NewSendManager(sconf.BatchSize, sconf.LingerInterval)
  168. if err != nil {
  169. return err
  170. }
  171. go sendManager.Run(ctx)
  172. }
  173. if !sconf.EnableCache {
  174. if sendManager != nil {
  175. dataOutCh = sendManager.GetOutputChan()
  176. } else {
  177. dataOutCh = dataCh
  178. }
  179. } else {
  180. if sendManager != nil {
  181. c = cache.NewSyncCache(ctx, sendManager.GetOutputChan(), result, &sconf.SinkConf, sconf.BufferLength)
  182. } else {
  183. c = cache.NewSyncCache(ctx, dataCh, result, &sconf.SinkConf, sconf.BufferLength)
  184. }
  185. if sconf.ResendAlterQueue {
  186. resendCh = make(chan []map[string]interface{}, sconf.BufferLength)
  187. rq = cache.NewSyncCache(ctx, resendCh, result, &sconf.SinkConf, sconf.BufferLength)
  188. }
  189. dataOutCh = c.Out
  190. }
  191. receiveQ := func(data interface{}) {
  192. processed := false
  193. if data, processed = m.preprocess(data); processed {
  194. return
  195. }
  196. stats.IncTotalRecordsIn()
  197. stats.SetBufferLength(bufferLen(dataCh, c, rq))
  198. outs := itemToMap(data)
  199. if sconf.Omitempty && (data == nil || len(outs) == 0) {
  200. ctx.GetLogger().Debugf("receive empty in sink")
  201. return
  202. }
  203. if sconf.isBatchSinkEnabled() {
  204. for _, out := range outs {
  205. sendManager.RecvData(out)
  206. }
  207. } else {
  208. select {
  209. case dataCh <- outs:
  210. default:
  211. ctx.GetLogger().Warnf("sink node %s instance %d buffer is full, drop data %v", m.name, instance, outs)
  212. }
  213. }
  214. if resendCh != nil {
  215. select {
  216. case resendCh <- nil:
  217. ctx.GetLogger().Debugf("resend signal sent")
  218. case <-ctx.Done():
  219. }
  220. }
  221. }
  222. normalQ := func(data []map[string]interface{}) {
  223. stats.ProcessTimeStart()
  224. stats.SetBufferLength(bufferLen(dataCh, c, rq))
  225. ctx.GetLogger().Debugf("sending data: %v", data)
  226. err := doCollectMaps(ctx, sink, sconf, data, stats, false)
  227. if sconf.EnableCache {
  228. ack := checkAck(ctx, data, err)
  229. if sconf.ResendAlterQueue {
  230. // If ack is false, add it to the resend queue
  231. if !ack {
  232. select {
  233. case resendCh <- data:
  234. case <-ctx.Done():
  235. }
  236. }
  237. // Always ack for the normal queue as fail items are handled by the resend queue
  238. select {
  239. case c.Ack <- true:
  240. stats.SetBufferLength(bufferLen(dataCh, c, rq) - 1)
  241. case <-ctx.Done():
  242. }
  243. } else {
  244. select {
  245. case c.Ack <- ack:
  246. if ack { // -1 because the signal length is changed async, just calculate it here
  247. stats.SetBufferLength(bufferLen(dataCh, c, rq) - 1)
  248. }
  249. case <-ctx.Done():
  250. }
  251. }
  252. }
  253. stats.ProcessTimeEnd()
  254. }
  255. resendQ := func(data []map[string]interface{}) {
  256. ctx.GetLogger().Debugf("resend data: %v", data)
  257. stats.SetBufferLength(bufferLen(dataCh, c, rq))
  258. if sconf.ResendIndicatorField != "" {
  259. for _, item := range data {
  260. item[sconf.ResendIndicatorField] = true
  261. }
  262. }
  263. err := doCollectMaps(ctx, sink, sconf, data, stats, true)
  264. ack := checkAck(ctx, data, err)
  265. select {
  266. case rq.Ack <- ack:
  267. if ack {
  268. stats.SetBufferLength(bufferLen(dataCh, c, rq) - 1)
  269. }
  270. case <-ctx.Done():
  271. }
  272. }
  273. doneQ := func() {
  274. logger.Infof("sink node %s instance %d done", m.name, instance)
  275. if err := sink.Close(ctx); err != nil {
  276. logger.Warnf("close sink node %s instance %d fails: %v", m.name, instance, err)
  277. }
  278. }
  279. if resendCh == nil { // no resend strategy
  280. for {
  281. select {
  282. case data := <-m.input:
  283. receiveQ(data)
  284. case data := <-dataOutCh:
  285. normalQ(data)
  286. case <-ctx.Done():
  287. doneQ()
  288. return nil
  289. }
  290. }
  291. } else {
  292. if sconf.ResendPriority == 0 {
  293. for {
  294. select {
  295. case data := <-m.input:
  296. receiveQ(data)
  297. case data := <-dataOutCh:
  298. normalQ(data)
  299. case data := <-rq.Out:
  300. resendQ(data)
  301. case <-ctx.Done():
  302. doneQ()
  303. return nil
  304. }
  305. }
  306. } else if sconf.ResendPriority < 0 { // normal queue has higher priority
  307. for {
  308. select {
  309. case data := <-m.input:
  310. receiveQ(data)
  311. case data := <-dataOutCh:
  312. normalQ(data)
  313. default:
  314. select {
  315. case data := <-m.input:
  316. receiveQ(data)
  317. case data := <-dataOutCh:
  318. normalQ(data)
  319. case data := <-rq.Out:
  320. resendQ(data)
  321. case <-ctx.Done():
  322. doneQ()
  323. return nil
  324. }
  325. }
  326. }
  327. } else {
  328. for {
  329. select {
  330. case data := <-m.input:
  331. receiveQ(data)
  332. case data := <-rq.Out:
  333. resendQ(data)
  334. default:
  335. select {
  336. case data := <-m.input:
  337. receiveQ(data)
  338. case data := <-dataOutCh:
  339. normalQ(data)
  340. case data := <-rq.Out:
  341. resendQ(data)
  342. case <-ctx.Done():
  343. doneQ()
  344. return nil
  345. }
  346. }
  347. }
  348. }
  349. }
  350. })
  351. if panicOrError != nil {
  352. infra.DrainError(ctx, panicOrError, result)
  353. }
  354. }(i)
  355. }
  356. return nil
  357. })
  358. if err != nil {
  359. infra.DrainError(ctx, err, result)
  360. }
  361. }()
  362. }
  363. func bufferLen(dataCh chan []map[string]interface{}, c *cache.SyncCache, rq *cache.SyncCache) int64 {
  364. l := len(dataCh)
  365. if c != nil {
  366. l += c.CacheLength
  367. }
  368. if rq != nil {
  369. l += rq.CacheLength
  370. }
  371. return int64(l)
  372. }
  373. func checkAck(ctx api.StreamContext, data interface{}, err error) bool {
  374. if err != nil {
  375. if strings.HasPrefix(err.Error(), errorx.IOErr) { // do not log to prevent a lot of logs!
  376. return false
  377. } else {
  378. ctx.GetLogger().Warnf("sink node %s instance %d publish %s error: %v", ctx.GetOpId(), ctx.GetInstanceId(), data, err)
  379. }
  380. } else {
  381. ctx.GetLogger().Debugf("sent data: %v", data)
  382. }
  383. return true
  384. }
  385. func (m *SinkNode) parseConf(logger api.Logger) (*SinkConf, error) {
  386. sconf := &SinkConf{
  387. Concurrency: 1,
  388. Omitempty: false,
  389. SendSingle: false,
  390. DataTemplate: "",
  391. SinkConf: *conf.Config.Sink,
  392. BufferLength: 1024,
  393. }
  394. err := cast.MapToStruct(m.options, sconf)
  395. if err != nil {
  396. return nil, fmt.Errorf("read properties %v fail with error: %v", m.options, err)
  397. }
  398. if sconf.Concurrency <= 0 {
  399. logger.Warnf("invalid type for concurrency property, should be positive integer but found %t", sconf.Concurrency)
  400. sconf.Concurrency = 1
  401. }
  402. m.concurrency = sconf.Concurrency
  403. if sconf.Format == "" {
  404. sconf.Format = "json"
  405. } else if sconf.Format != message.FormatJson && sconf.Format != message.FormatProtobuf && sconf.Format != message.FormatBinary && sconf.Format != message.FormatCustom && sconf.Format != message.FormatDelimited {
  406. logger.Warnf("invalid type for format property, should be json protobuf or binary but found %s", sconf.Format)
  407. sconf.Format = "json"
  408. }
  409. err = cast.MapToStruct(m.options, &sconf.SinkConf)
  410. if err != nil {
  411. return nil, fmt.Errorf("read properties %v to cache conf fail with error: %v", m.options, err)
  412. }
  413. if sconf.DataField == "" {
  414. if v, ok := m.options["tableDataField"]; ok {
  415. sconf.DataField = v.(string)
  416. }
  417. }
  418. err = sconf.SinkConf.Validate()
  419. if err != nil {
  420. return nil, fmt.Errorf("invalid cache properties: %v", err)
  421. }
  422. return sconf, err
  423. }
  424. func (m *SinkNode) reset() {
  425. if !m.isMock {
  426. m.sinks = nil
  427. }
  428. m.statManagers = nil
  429. }
  430. func doCollectMaps(ctx api.StreamContext, sink api.Sink, sconf *SinkConf, outs []map[string]interface{}, stats metric.StatManager, isResend bool) error {
  431. if !sconf.SendSingle {
  432. return doCollectData(ctx, sink, outs, stats, isResend)
  433. } else {
  434. var err error
  435. for _, d := range outs {
  436. if sconf.Omitempty && (d == nil || len(d) == 0) {
  437. ctx.GetLogger().Debugf("receive empty in sink")
  438. continue
  439. }
  440. newErr := doCollectData(ctx, sink, d, stats, isResend)
  441. if newErr != nil {
  442. err = newErr
  443. }
  444. }
  445. return err
  446. }
  447. }
  448. func itemToMap(item interface{}) []map[string]interface{} {
  449. var outs []map[string]interface{}
  450. switch val := item.(type) {
  451. case error:
  452. outs = []map[string]interface{}{
  453. {"error": val.Error()},
  454. }
  455. break
  456. case xsql.Collection: // The order is important here, because some element is both a collection and a row, such as WindowTuples, JoinTuples, etc.
  457. outs = val.ToMaps()
  458. break
  459. case xsql.Row:
  460. outs = []map[string]interface{}{
  461. val.ToMap(),
  462. }
  463. break
  464. case []map[string]interface{}: // for test only
  465. outs = val
  466. break
  467. case *xsql.WatermarkTuple:
  468. // just ignore
  469. default:
  470. outs = []map[string]interface{}{
  471. {"error": fmt.Sprintf("result is not a map slice but found %#v", val)},
  472. }
  473. }
  474. return outs
  475. }
  476. // doCollectData outData must be map or []map
  477. func doCollectData(ctx api.StreamContext, sink api.Sink, outData interface{}, stats metric.StatManager, isResend bool) error {
  478. select {
  479. case <-ctx.Done():
  480. ctx.GetLogger().Infof("sink node %s instance %d stops data resending", ctx.GetOpId(), ctx.GetInstanceId())
  481. return nil
  482. default:
  483. if isResend {
  484. return resendDataToSink(ctx, sink, outData, stats)
  485. } else {
  486. return sendDataToSink(ctx, sink, outData, stats)
  487. }
  488. }
  489. }
  490. func sendDataToSink(ctx api.StreamContext, sink api.Sink, outData interface{}, stats metric.StatManager) error {
  491. if err := sink.Collect(ctx, outData); err != nil {
  492. stats.IncTotalExceptions(err.Error())
  493. return err
  494. } else {
  495. ctx.GetLogger().Debugf("success")
  496. stats.IncTotalRecordsOut()
  497. return nil
  498. }
  499. }
  500. func resendDataToSink(ctx api.StreamContext, sink api.Sink, outData interface{}, stats metric.StatManager) error {
  501. ctx.GetLogger().Debugf("start resend")
  502. var err error
  503. switch st := sink.(type) {
  504. case api.ResendSink:
  505. err = st.CollectResend(ctx, outData)
  506. default:
  507. err = st.Collect(ctx, outData)
  508. }
  509. if err != nil {
  510. stats.IncTotalExceptions(err.Error())
  511. return err
  512. } else {
  513. ctx.GetLogger().Debugf("success resend")
  514. stats.IncTotalRecordsOut()
  515. return nil
  516. }
  517. }
  518. func getSink(name string, action map[string]interface{}) (api.Sink, error) {
  519. var (
  520. s api.Sink
  521. err error
  522. )
  523. s, err = io.Sink(name)
  524. if s != nil {
  525. newAction := nodeConf.GetSinkConf(name, action)
  526. err = s.Configure(newAction)
  527. if err != nil {
  528. return nil, err
  529. }
  530. return s, nil
  531. } else {
  532. if err != nil {
  533. return nil, err
  534. } else {
  535. return nil, fmt.Errorf("sink %s not found", name)
  536. }
  537. }
  538. }
  539. // AddOutput Override defaultNode
  540. func (m *SinkNode) AddOutput(_ chan<- interface{}, name string) error {
  541. return fmt.Errorf("fail to add output %s, sink %s cannot add output", name, m.name)
  542. }
  543. // Broadcast Override defaultNode
  544. func (m *SinkNode) Broadcast(_ interface{}) error {
  545. return fmt.Errorf("sink %s cannot add broadcast", m.name)
  546. }