sink_node.go 16 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577
  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. case <-ctx.Done():
  211. }
  212. }
  213. if resendCh != nil {
  214. select {
  215. case resendCh <- nil:
  216. ctx.GetLogger().Debugf("resend signal sent")
  217. case <-ctx.Done():
  218. }
  219. }
  220. }
  221. normalQ := func(data []map[string]interface{}) {
  222. stats.ProcessTimeStart()
  223. stats.SetBufferLength(bufferLen(dataCh, c, rq))
  224. ctx.GetLogger().Debugf("sending data: %v", data)
  225. err := doCollectMaps(ctx, sink, sconf, data, stats, false)
  226. if sconf.EnableCache {
  227. ack := checkAck(ctx, data, err)
  228. if sconf.ResendAlterQueue {
  229. // If ack is false, add it to the resend queue
  230. if !ack {
  231. select {
  232. case resendCh <- data:
  233. case <-ctx.Done():
  234. }
  235. }
  236. // Always ack for the normal queue as fail items are handled by the resend queue
  237. select {
  238. case c.Ack <- true:
  239. stats.SetBufferLength(bufferLen(dataCh, c, rq) - 1)
  240. case <-ctx.Done():
  241. }
  242. } else {
  243. select {
  244. case c.Ack <- ack:
  245. if ack { // -1 because the signal length is changed async, just calculate it here
  246. stats.SetBufferLength(bufferLen(dataCh, c, rq) - 1)
  247. }
  248. case <-ctx.Done():
  249. }
  250. }
  251. }
  252. stats.ProcessTimeEnd()
  253. }
  254. resendQ := func(data []map[string]interface{}) {
  255. ctx.GetLogger().Debugf("resend data: %v", data)
  256. stats.SetBufferLength(bufferLen(dataCh, c, rq))
  257. if sconf.ResendIndicatorField != "" {
  258. for _, item := range data {
  259. item[sconf.ResendIndicatorField] = true
  260. }
  261. }
  262. err := doCollectMaps(ctx, sink, sconf, data, stats, true)
  263. ack := checkAck(ctx, data, err)
  264. select {
  265. case rq.Ack <- ack:
  266. if ack {
  267. stats.SetBufferLength(bufferLen(dataCh, c, rq) - 1)
  268. }
  269. case <-ctx.Done():
  270. }
  271. }
  272. doneQ := func() {
  273. logger.Infof("sink node %s instance %d done", m.name, instance)
  274. if err := sink.Close(ctx); err != nil {
  275. logger.Warnf("close sink node %s instance %d fails: %v", m.name, instance, err)
  276. }
  277. }
  278. if resendCh == nil { // no resend strategy
  279. for {
  280. select {
  281. case data := <-m.input:
  282. receiveQ(data)
  283. case data := <-dataOutCh:
  284. normalQ(data)
  285. case <-ctx.Done():
  286. doneQ()
  287. return nil
  288. }
  289. }
  290. } else {
  291. if sconf.ResendPriority == 0 {
  292. for {
  293. select {
  294. case data := <-m.input:
  295. receiveQ(data)
  296. case data := <-dataOutCh:
  297. normalQ(data)
  298. case data := <-rq.Out:
  299. resendQ(data)
  300. case <-ctx.Done():
  301. doneQ()
  302. return nil
  303. }
  304. }
  305. } else if sconf.ResendPriority < 0 { // normal queue has higher priority
  306. for {
  307. select {
  308. case data := <-m.input:
  309. receiveQ(data)
  310. case data := <-dataOutCh:
  311. normalQ(data)
  312. default:
  313. select {
  314. case data := <-m.input:
  315. receiveQ(data)
  316. case data := <-dataOutCh:
  317. normalQ(data)
  318. case data := <-rq.Out:
  319. resendQ(data)
  320. case <-ctx.Done():
  321. doneQ()
  322. return nil
  323. }
  324. }
  325. }
  326. } else {
  327. for {
  328. select {
  329. case data := <-m.input:
  330. receiveQ(data)
  331. case data := <-rq.Out:
  332. resendQ(data)
  333. default:
  334. select {
  335. case data := <-m.input:
  336. receiveQ(data)
  337. case data := <-dataOutCh:
  338. normalQ(data)
  339. case data := <-rq.Out:
  340. resendQ(data)
  341. case <-ctx.Done():
  342. doneQ()
  343. return nil
  344. }
  345. }
  346. }
  347. }
  348. }
  349. })
  350. if panicOrError != nil {
  351. infra.DrainError(ctx, panicOrError, result)
  352. }
  353. }(i)
  354. }
  355. return nil
  356. })
  357. if err != nil {
  358. infra.DrainError(ctx, err, result)
  359. }
  360. }()
  361. }
  362. func bufferLen(dataCh chan []map[string]interface{}, c *cache.SyncCache, rq *cache.SyncCache) int64 {
  363. l := len(dataCh)
  364. if c != nil {
  365. l += c.CacheLength
  366. }
  367. if rq != nil {
  368. l += rq.CacheLength
  369. }
  370. return int64(l)
  371. }
  372. func checkAck(ctx api.StreamContext, data interface{}, err error) bool {
  373. if err != nil {
  374. if strings.HasPrefix(err.Error(), errorx.IOErr) { // do not log to prevent a lot of logs!
  375. return false
  376. } else {
  377. ctx.GetLogger().Warnf("sink node %s instance %d publish %s error: %v", ctx.GetOpId(), ctx.GetInstanceId(), data, err)
  378. }
  379. } else {
  380. ctx.GetLogger().Debugf("sent data: %v", data)
  381. }
  382. return true
  383. }
  384. func (m *SinkNode) parseConf(logger api.Logger) (*SinkConf, error) {
  385. sconf := &SinkConf{
  386. Concurrency: 1,
  387. Omitempty: false,
  388. SendSingle: false,
  389. DataTemplate: "",
  390. SinkConf: *conf.Config.Sink,
  391. BufferLength: 1024,
  392. }
  393. err := cast.MapToStruct(m.options, sconf)
  394. if err != nil {
  395. return nil, fmt.Errorf("read properties %v fail with error: %v", m.options, err)
  396. }
  397. if sconf.Concurrency <= 0 {
  398. logger.Warnf("invalid type for concurrency property, should be positive integer but found %t", sconf.Concurrency)
  399. sconf.Concurrency = 1
  400. }
  401. m.concurrency = sconf.Concurrency
  402. if sconf.Format == "" {
  403. sconf.Format = "json"
  404. } else if sconf.Format != message.FormatJson && sconf.Format != message.FormatProtobuf && sconf.Format != message.FormatBinary && sconf.Format != message.FormatCustom && sconf.Format != message.FormatDelimited {
  405. logger.Warnf("invalid type for format property, should be json protobuf or binary but found %s", sconf.Format)
  406. sconf.Format = "json"
  407. }
  408. err = cast.MapToStruct(m.options, &sconf.SinkConf)
  409. if err != nil {
  410. return nil, fmt.Errorf("read properties %v to cache conf fail with error: %v", m.options, err)
  411. }
  412. if sconf.DataField == "" {
  413. if v, ok := m.options["tableDataField"]; ok {
  414. sconf.DataField = v.(string)
  415. }
  416. }
  417. err = sconf.SinkConf.Validate()
  418. if err != nil {
  419. return nil, fmt.Errorf("invalid cache properties: %v", err)
  420. }
  421. return sconf, err
  422. }
  423. func (m *SinkNode) reset() {
  424. if !m.isMock {
  425. m.sinks = nil
  426. }
  427. m.statManagers = nil
  428. }
  429. func doCollectMaps(ctx api.StreamContext, sink api.Sink, sconf *SinkConf, outs []map[string]interface{}, stats metric.StatManager, isResend bool) error {
  430. if !sconf.SendSingle {
  431. return doCollectData(ctx, sink, outs, stats, isResend)
  432. } else {
  433. var err error
  434. for _, d := range outs {
  435. if sconf.Omitempty && (d == nil || len(d) == 0) {
  436. ctx.GetLogger().Debugf("receive empty in sink")
  437. continue
  438. }
  439. newErr := doCollectData(ctx, sink, d, stats, isResend)
  440. if newErr != nil {
  441. err = newErr
  442. }
  443. }
  444. return err
  445. }
  446. }
  447. func itemToMap(item interface{}) []map[string]interface{} {
  448. var outs []map[string]interface{}
  449. switch val := item.(type) {
  450. case error:
  451. outs = []map[string]interface{}{
  452. {"error": val.Error()},
  453. }
  454. break
  455. case xsql.Collection: // The order is important here, because some element is both a collection and a row, such as WindowTuples, JoinTuples, etc.
  456. outs = val.ToMaps()
  457. break
  458. case xsql.Row:
  459. outs = []map[string]interface{}{
  460. val.ToMap(),
  461. }
  462. break
  463. case []map[string]interface{}: // for test only
  464. outs = val
  465. break
  466. case *xsql.WatermarkTuple:
  467. // just ignore
  468. default:
  469. outs = []map[string]interface{}{
  470. {"error": fmt.Sprintf("result is not a map slice but found %#v", val)},
  471. }
  472. }
  473. return outs
  474. }
  475. // doCollectData outData must be map or []map
  476. func doCollectData(ctx api.StreamContext, sink api.Sink, outData interface{}, stats metric.StatManager, isResend bool) error {
  477. select {
  478. case <-ctx.Done():
  479. ctx.GetLogger().Infof("sink node %s instance %d stops data resending", ctx.GetOpId(), ctx.GetInstanceId())
  480. return nil
  481. default:
  482. if isResend {
  483. return resendDataToSink(ctx, sink, outData, stats)
  484. } else {
  485. return sendDataToSink(ctx, sink, outData, stats)
  486. }
  487. }
  488. }
  489. func sendDataToSink(ctx api.StreamContext, sink api.Sink, outData interface{}, stats metric.StatManager) error {
  490. if err := sink.Collect(ctx, outData); err != nil {
  491. stats.IncTotalExceptions(err.Error())
  492. return err
  493. } else {
  494. ctx.GetLogger().Debugf("success")
  495. stats.IncTotalRecordsOut()
  496. return nil
  497. }
  498. }
  499. func resendDataToSink(ctx api.StreamContext, sink api.Sink, outData interface{}, stats metric.StatManager) error {
  500. ctx.GetLogger().Debugf("start resend")
  501. var err error
  502. switch st := sink.(type) {
  503. case api.ResendSink:
  504. err = st.CollectResend(ctx, outData)
  505. default:
  506. err = st.Collect(ctx, outData)
  507. }
  508. if err != nil {
  509. stats.IncTotalExceptions(err.Error())
  510. return err
  511. } else {
  512. ctx.GetLogger().Debugf("success resend")
  513. stats.IncTotalRecordsOut()
  514. return nil
  515. }
  516. }
  517. func getSink(name string, action map[string]interface{}) (api.Sink, error) {
  518. var (
  519. s api.Sink
  520. err error
  521. )
  522. s, err = io.Sink(name)
  523. if s != nil {
  524. newAction := nodeConf.GetSinkConf(name, action)
  525. err = s.Configure(newAction)
  526. if err != nil {
  527. return nil, err
  528. }
  529. return s, nil
  530. } else {
  531. if err != nil {
  532. return nil, err
  533. } else {
  534. return nil, fmt.Errorf("sink %s not found", name)
  535. }
  536. }
  537. }
  538. // AddOutput Override defaultNode
  539. func (m *SinkNode) AddOutput(_ chan<- interface{}, name string) error {
  540. return fmt.Errorf("fail to add output %s, sink %s cannot add output", name, m.name)
  541. }
  542. // Broadcast Override defaultNode
  543. func (m *SinkNode) Broadcast(_ interface{}) error {
  544. return fmt.Errorf("sink %s cannot add broadcast", m.name)
  545. }