sink_node.go 12 KB

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