sink_node.go 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416
  1. // Copyright 2022 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. "github.com/lf-edge/ekuiper/internal/binder/io"
  18. "github.com/lf-edge/ekuiper/internal/conf"
  19. "github.com/lf-edge/ekuiper/internal/topo/context"
  20. "github.com/lf-edge/ekuiper/internal/topo/node/cache"
  21. nodeConf "github.com/lf-edge/ekuiper/internal/topo/node/conf"
  22. "github.com/lf-edge/ekuiper/internal/topo/node/metric"
  23. "github.com/lf-edge/ekuiper/internal/topo/transform"
  24. "github.com/lf-edge/ekuiper/internal/xsql"
  25. "github.com/lf-edge/ekuiper/pkg/api"
  26. "github.com/lf-edge/ekuiper/pkg/cast"
  27. "github.com/lf-edge/ekuiper/pkg/errorx"
  28. "github.com/lf-edge/ekuiper/pkg/infra"
  29. "github.com/lf-edge/ekuiper/pkg/message"
  30. "strings"
  31. "sync"
  32. )
  33. type SinkConf struct {
  34. Concurrency int `json:"concurrency"`
  35. RunAsync bool `json:"runAsync"`
  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. conf.SinkConf
  44. }
  45. type SinkNode struct {
  46. *defaultSinkNode
  47. //static
  48. sinkType string
  49. mutex sync.RWMutex
  50. //configs (also static for sinks)
  51. options map[string]interface{}
  52. isMock bool
  53. //states varies after restart
  54. sinks []api.Sink
  55. }
  56. func NewSinkNode(name string, sinkType string, props map[string]interface{}) *SinkNode {
  57. bufferLength := 1024
  58. if c, ok := props["bufferLength"]; ok {
  59. if t, err := cast.ToInt(c, cast.STRICT); err != nil || t <= 0 {
  60. //invalid property bufferLength
  61. } else {
  62. bufferLength = t
  63. }
  64. }
  65. return &SinkNode{
  66. defaultSinkNode: &defaultSinkNode{
  67. input: make(chan interface{}, bufferLength),
  68. defaultNode: &defaultNode{
  69. name: name,
  70. concurrency: 1,
  71. ctx: nil,
  72. },
  73. },
  74. sinkType: sinkType,
  75. options: props,
  76. }
  77. }
  78. // NewSinkNodeWithSink Only for mock source, do not use it in production
  79. func NewSinkNodeWithSink(name string, sink api.Sink, props map[string]interface{}) *SinkNode {
  80. return &SinkNode{
  81. defaultSinkNode: &defaultSinkNode{
  82. input: make(chan interface{}, 1024),
  83. defaultNode: &defaultNode{
  84. name: name,
  85. concurrency: 1,
  86. ctx: nil,
  87. },
  88. },
  89. sinks: []api.Sink{sink},
  90. options: props,
  91. isMock: true,
  92. }
  93. }
  94. func (m *SinkNode) Open(ctx api.StreamContext, result chan<- error) {
  95. m.ctx = ctx
  96. logger := ctx.GetLogger()
  97. logger.Debugf("open sink node %s", m.name)
  98. go func() {
  99. err := infra.SafeRun(func() error {
  100. sconf, err := m.parseConf(logger)
  101. if err != nil {
  102. return err
  103. }
  104. tf, err := transform.GenTransform(sconf.DataTemplate, sconf.Format, sconf.SchemaId, sconf.Delimiter)
  105. if err != nil {
  106. msg := fmt.Sprintf("property dataTemplate %v is invalid: %v", sconf.DataTemplate, err)
  107. logger.Warnf(msg)
  108. return fmt.Errorf(msg)
  109. }
  110. ctx = context.WithValue(ctx.(*context.DefaultContext), context.TransKey, tf)
  111. m.reset()
  112. logger.Infof("open sink node %d instances", m.concurrency)
  113. for i := 0; i < m.concurrency; i++ { // workers
  114. go func(instance int) {
  115. panicOrError := infra.SafeRun(func() error {
  116. var (
  117. sink api.Sink
  118. err error
  119. )
  120. if !m.isMock {
  121. logger.Debugf("Trying to get sink for rule %s with options %v\n", ctx.GetRuleId(), m.options)
  122. sink, err = getSink(m.sinkType, m.options)
  123. if err != nil {
  124. return err
  125. }
  126. logger.Debugf("Successfully get the sink %s", m.sinkType)
  127. m.mutex.Lock()
  128. m.sinks = append(m.sinks, sink)
  129. m.mutex.Unlock()
  130. logger.Debugf("Now is to open sink for rule %s.\n", ctx.GetRuleId())
  131. if err := sink.Open(ctx); err != nil {
  132. return err
  133. }
  134. logger.Debugf("Successfully open sink for rule %s.\n", ctx.GetRuleId())
  135. } else {
  136. sink = m.sinks[instance]
  137. }
  138. stats, err := metric.NewStatManager(ctx, "sink")
  139. if err != nil {
  140. return err
  141. }
  142. m.mutex.Lock()
  143. m.statManagers = append(m.statManagers, stats)
  144. m.mutex.Unlock()
  145. if !sconf.EnableCache {
  146. for {
  147. select {
  148. case data := <-m.input:
  149. if temp, processed := m.preprocess(data); !processed {
  150. data = temp
  151. } else {
  152. break
  153. }
  154. stats.SetBufferLength(int64(len(m.input)))
  155. stats.IncTotalRecordsIn()
  156. if sconf.RunAsync {
  157. go func() {
  158. p := infra.SafeRun(func() error {
  159. err := doCollect(ctx, sink, data, stats, sconf)
  160. if err != nil {
  161. logger.Warnf("sink collect error: %v", err)
  162. }
  163. return nil
  164. })
  165. if p != nil {
  166. infra.DrainError(ctx, p, result)
  167. }
  168. }()
  169. } else {
  170. err := doCollect(ctx, sink, data, stats, sconf)
  171. if err != nil {
  172. logger.Warnf("sink collect error: %v", err)
  173. }
  174. }
  175. case <-ctx.Done():
  176. logger.Infof("sink node %s instance %d done", m.name, instance)
  177. if err := sink.Close(ctx); err != nil {
  178. logger.Warnf("close sink node %s instance %d fails: %v", m.name, instance, err)
  179. }
  180. return nil
  181. }
  182. }
  183. } else {
  184. logger.Infof("Creating sink cache")
  185. if sconf.RunAsync { // async mode, the ack must have an id
  186. // is not supported and validated in the configure, should not go here
  187. return fmt.Errorf("async mode is not supported for cache sink")
  188. } else { // sync mode, the ack is already in order
  189. dataCh := make(chan []map[string]interface{}, sconf.BufferLength)
  190. c := cache.NewSyncCache(ctx, dataCh, result, stats, &sconf.SinkConf, sconf.BufferLength)
  191. for {
  192. select {
  193. case data := <-m.input:
  194. if temp, processed := m.preprocess(data); !processed {
  195. data = temp
  196. } else {
  197. break
  198. }
  199. stats.IncTotalRecordsIn()
  200. outs := itemToMap(data)
  201. if sconf.Omitempty && (data == nil || len(outs) == 0) {
  202. ctx.GetLogger().Debugf("receive empty in sink")
  203. return nil
  204. }
  205. select {
  206. case dataCh <- outs:
  207. case <-ctx.Done():
  208. }
  209. case data := <-c.Out:
  210. stats.ProcessTimeStart()
  211. ack := true
  212. err := doCollectMaps(ctx, sink, sconf, data, stats)
  213. // Only recoverable error should be cached
  214. if err != nil {
  215. if strings.HasPrefix(err.Error(), errorx.IOErr) { // do not log to prevent a lot of logs!
  216. ack = false
  217. } else {
  218. ctx.GetLogger().Warnf("sink node %s instance %d publish %s error: %v", ctx.GetOpId(), ctx.GetInstanceId(), data, err)
  219. }
  220. } else {
  221. ctx.GetLogger().Debugf("sent data to MQTT: %v", data)
  222. }
  223. select {
  224. case c.Ack <- ack:
  225. case <-ctx.Done():
  226. }
  227. stats.ProcessTimeEnd()
  228. case <-ctx.Done():
  229. logger.Infof("sink node %s instance %d done", m.name, instance)
  230. if err := sink.Close(ctx); err != nil {
  231. logger.Warnf("close sink node %s instance %d fails: %v", m.name, instance, err)
  232. }
  233. return nil
  234. }
  235. }
  236. }
  237. }
  238. })
  239. if panicOrError != nil {
  240. infra.DrainError(ctx, panicOrError, result)
  241. }
  242. }(i)
  243. }
  244. return nil
  245. })
  246. if err != nil {
  247. infra.DrainError(ctx, err, result)
  248. }
  249. }()
  250. }
  251. func (m *SinkNode) parseConf(logger api.Logger) (*SinkConf, error) {
  252. sconf := &SinkConf{
  253. Concurrency: 1,
  254. RunAsync: false,
  255. Omitempty: false,
  256. SendSingle: false,
  257. DataTemplate: "",
  258. SinkConf: *conf.Config.Sink,
  259. BufferLength: 1024,
  260. }
  261. err := cast.MapToStruct(m.options, sconf)
  262. if err != nil {
  263. return nil, fmt.Errorf("read properties %v fail with error: %v", m.options, err)
  264. }
  265. if sconf.Concurrency <= 0 {
  266. logger.Warnf("invalid type for concurrency property, should be positive integer but found %t", sconf.Concurrency)
  267. sconf.Concurrency = 1
  268. }
  269. m.concurrency = sconf.Concurrency
  270. if sconf.Format == "" {
  271. sconf.Format = "json"
  272. } else if sconf.Format != message.FormatJson && sconf.Format != message.FormatProtobuf && sconf.Format != message.FormatBinary && sconf.Format != message.FormatCustom && sconf.Format != message.FormatDelimited {
  273. logger.Warnf("invalid type for format property, should be json protobuf or binary but found %s", sconf.Format)
  274. sconf.Format = "json"
  275. }
  276. err = cast.MapToStruct(m.options, &sconf.SinkConf)
  277. if err != nil {
  278. return nil, fmt.Errorf("read properties %v to cache conf fail with error: %v", m.options, err)
  279. }
  280. if sconf.SinkConf.EnableCache && sconf.RunAsync {
  281. return nil, fmt.Errorf("cache is not supported for async sink, do not use enableCache and runAsync properties together")
  282. }
  283. err = sconf.SinkConf.Validate()
  284. if err != nil {
  285. return nil, fmt.Errorf("invalid cache properties: %v", err)
  286. }
  287. return sconf, err
  288. }
  289. func (m *SinkNode) reset() {
  290. if !m.isMock {
  291. m.sinks = nil
  292. }
  293. m.statManagers = nil
  294. }
  295. func doCollect(ctx api.StreamContext, sink api.Sink, item interface{}, stats metric.StatManager, sconf *SinkConf) error {
  296. stats.ProcessTimeStart()
  297. defer stats.ProcessTimeEnd()
  298. outs := itemToMap(item)
  299. if sconf.Omitempty && (item == nil || len(outs) == 0) {
  300. ctx.GetLogger().Debugf("receive empty in sink")
  301. return nil
  302. }
  303. return doCollectMaps(ctx, sink, sconf, outs, stats)
  304. }
  305. func doCollectMaps(ctx api.StreamContext, sink api.Sink, sconf *SinkConf, outs []map[string]interface{}, stats metric.StatManager) error {
  306. if !sconf.SendSingle {
  307. return doCollectData(ctx, sink, outs, stats)
  308. } else {
  309. var err error
  310. for _, d := range outs {
  311. if sconf.Omitempty && (d == nil || len(d) == 0) {
  312. ctx.GetLogger().Debugf("receive empty in sink")
  313. continue
  314. }
  315. newErr := doCollectData(ctx, sink, d, stats)
  316. if newErr != nil {
  317. err = newErr
  318. }
  319. }
  320. return err
  321. }
  322. }
  323. func itemToMap(item interface{}) []map[string]interface{} {
  324. var outs []map[string]interface{}
  325. switch val := item.(type) {
  326. case error:
  327. outs = []map[string]interface{}{
  328. {"error": val.Error()},
  329. }
  330. break
  331. case xsql.Collection: // The order is important here, because some element is both a collection and a row, such as WindowTuples, JoinTuples, etc.
  332. outs = val.ToMaps()
  333. break
  334. case xsql.Row:
  335. outs = []map[string]interface{}{
  336. val.ToMap(),
  337. }
  338. break
  339. case []map[string]interface{}: // for test only
  340. outs = val
  341. break
  342. default:
  343. outs = []map[string]interface{}{
  344. {"error": fmt.Sprintf("result is not a map slice but found %#v", val)},
  345. }
  346. }
  347. return outs
  348. }
  349. // doCollectData outData must be map or []map
  350. func doCollectData(ctx api.StreamContext, sink api.Sink, outData interface{}, stats metric.StatManager) error {
  351. select {
  352. case <-ctx.Done():
  353. ctx.GetLogger().Infof("sink node %s instance %d stops data resending", ctx.GetOpId(), ctx.GetInstanceId())
  354. return nil
  355. default:
  356. if err := sink.Collect(ctx, outData); err != nil {
  357. stats.IncTotalExceptions(err.Error())
  358. return err
  359. } else {
  360. ctx.GetLogger().Debugf("success")
  361. stats.IncTotalRecordsOut()
  362. return nil
  363. }
  364. }
  365. }
  366. func getSink(name string, action map[string]interface{}) (api.Sink, error) {
  367. var (
  368. s api.Sink
  369. err error
  370. )
  371. s, err = io.Sink(name)
  372. if s != nil {
  373. newAction := nodeConf.GetSinkConf(name, action)
  374. err = s.Configure(newAction)
  375. if err != nil {
  376. return nil, err
  377. }
  378. return s, nil
  379. } else {
  380. if err != nil {
  381. return nil, err
  382. } else {
  383. return nil, fmt.Errorf("sink %s not found", name)
  384. }
  385. }
  386. }
  387. // AddOutput Override defaultNode
  388. func (m *SinkNode) AddOutput(_ chan<- interface{}, name string) error {
  389. return fmt.Errorf("fail to add output %s, sink %s cannot add output", name, m.name)
  390. }
  391. // Broadcast Override defaultNode
  392. func (m *SinkNode) Broadcast(_ interface{}) error {
  393. return fmt.Errorf("sink %s cannot add broadcast", m.name)
  394. }