sink_node.go 11 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381
  1. // Copyright 2021 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/transform"
  21. "github.com/lf-edge/ekuiper/pkg/api"
  22. "github.com/lf-edge/ekuiper/pkg/cast"
  23. "github.com/lf-edge/ekuiper/pkg/errorx"
  24. "strings"
  25. "sync"
  26. "time"
  27. )
  28. type SinkConf struct {
  29. Concurrency int `json:"concurrency"`
  30. RunAsync bool `json:"runAsync"`
  31. RetryInterval int `json:"retryInterval"`
  32. RetryCount int `json:"retryCount"`
  33. CacheLength int `json:"cacheLength"`
  34. CacheSaveInterval int `json:"cacheSaveInterval"`
  35. Omitempty bool `json:"omitIfEmpty"`
  36. SendSingle bool `json:"sendSingle"`
  37. DataTemplate string `json:"dataTemplate"`
  38. }
  39. type SinkNode struct {
  40. *defaultSinkNode
  41. //static
  42. sinkType string
  43. mutex sync.RWMutex
  44. //configs (also static for sinks)
  45. options map[string]interface{}
  46. isMock bool
  47. //states varies after restart
  48. sinks []api.Sink
  49. tch chan struct{} //channel to trigger cache saved, will be trigger by checkpoint only
  50. }
  51. func NewSinkNode(name string, sinkType string, props map[string]interface{}) *SinkNode {
  52. bufferLength := 1024
  53. if c, ok := props["bufferLength"]; ok {
  54. if t, err := cast.ToInt(c, cast.STRICT); err != nil || t <= 0 {
  55. //invalid property bufferLength
  56. } else {
  57. bufferLength = t
  58. }
  59. }
  60. return &SinkNode{
  61. defaultSinkNode: &defaultSinkNode{
  62. input: make(chan interface{}, bufferLength),
  63. defaultNode: &defaultNode{
  64. name: name,
  65. concurrency: 1,
  66. ctx: nil,
  67. },
  68. },
  69. sinkType: sinkType,
  70. options: props,
  71. }
  72. }
  73. // NewSinkNodeWithSink Only for mock source, do not use it in production
  74. func NewSinkNodeWithSink(name string, sink api.Sink, props map[string]interface{}) *SinkNode {
  75. return &SinkNode{
  76. defaultSinkNode: &defaultSinkNode{
  77. input: make(chan interface{}, 1024),
  78. defaultNode: &defaultNode{
  79. name: name,
  80. concurrency: 1,
  81. ctx: nil,
  82. },
  83. },
  84. sinks: []api.Sink{sink},
  85. options: props,
  86. isMock: true,
  87. }
  88. }
  89. func (m *SinkNode) Open(ctx api.StreamContext, result chan<- error) {
  90. m.ctx = ctx
  91. logger := ctx.GetLogger()
  92. logger.Debugf("open sink node %s", m.name)
  93. if m.qos >= api.AtLeastOnce {
  94. m.tch = make(chan struct{})
  95. }
  96. go func() {
  97. sconf := &SinkConf{
  98. Concurrency: 1,
  99. RunAsync: false,
  100. RetryInterval: 1000,
  101. RetryCount: 0,
  102. CacheLength: 1024,
  103. CacheSaveInterval: 1000,
  104. Omitempty: false,
  105. SendSingle: false,
  106. DataTemplate: "",
  107. }
  108. err := cast.MapToStruct(m.options, sconf)
  109. if err != nil {
  110. result <- fmt.Errorf("read properties %v fail with error: %v", m.options, err)
  111. return
  112. }
  113. if sconf.Concurrency <= 0 {
  114. logger.Warnf("invalid type for concurrency property, should be positive integer but found %t", sconf.Concurrency)
  115. sconf.Concurrency = 1
  116. }
  117. m.concurrency = sconf.Concurrency
  118. if sconf.RetryInterval <= 0 {
  119. logger.Warnf("invalid type for retryInterval property, should be positive integer but found %t", sconf.RetryInterval)
  120. sconf.RetryInterval = 1000
  121. }
  122. if sconf.RetryCount < 0 {
  123. logger.Warnf("invalid type for retryCount property, should be positive integer but found %t", sconf.RetryCount)
  124. sconf.RetryCount = 3
  125. }
  126. if sconf.CacheLength < 0 {
  127. logger.Warnf("invalid type for cacheLength property, should be positive integer but found %t", sconf.CacheLength)
  128. sconf.CacheLength = 1024
  129. }
  130. if sconf.CacheSaveInterval < 0 {
  131. logger.Warnf("invalid type for cacheSaveInterval property, should be positive integer but found %t", sconf.CacheSaveInterval)
  132. sconf.CacheSaveInterval = 1000
  133. }
  134. tf, err := transform.GenTransform(sconf.DataTemplate)
  135. if err != nil {
  136. msg := fmt.Sprintf("property dataTemplate %v is invalid: %v", sconf.DataTemplate, err)
  137. logger.Warnf(msg)
  138. result <- fmt.Errorf(msg)
  139. return
  140. }
  141. ctx = context.WithValue(ctx.(*context.DefaultContext), context.TransKey, tf)
  142. m.reset()
  143. logger.Infof("open sink node %d instances", m.concurrency)
  144. for i := 0; i < m.concurrency; i++ { // workers
  145. go func(instance int) {
  146. var sink api.Sink
  147. var err error
  148. if !m.isMock {
  149. logger.Debugf("Trying to get sink for rule %s with options %v\n", ctx.GetRuleId(), m.options)
  150. sink, err = getSink(m.sinkType, m.options)
  151. if err != nil {
  152. m.drainError(result, err, ctx, logger)
  153. return
  154. }
  155. logger.Debugf("Successfully get the sink %s", m.sinkType)
  156. m.mutex.Lock()
  157. m.sinks = append(m.sinks, sink)
  158. m.mutex.Unlock()
  159. logger.Debugf("Now is to open sink for rule %s.\n", ctx.GetRuleId())
  160. if err := sink.Open(ctx); err != nil {
  161. m.drainError(result, err, ctx, logger)
  162. return
  163. }
  164. logger.Debugf("Successfully open sink for rule %s.\n", ctx.GetRuleId())
  165. } else {
  166. sink = m.sinks[instance]
  167. }
  168. stats, err := NewStatManager("sink", ctx)
  169. if err != nil {
  170. m.drainError(result, err, ctx, logger)
  171. return
  172. }
  173. m.mutex.Lock()
  174. m.statManagers = append(m.statManagers, stats)
  175. m.mutex.Unlock()
  176. if conf.Config.Sink.DisableCache {
  177. for {
  178. select {
  179. case data := <-m.input:
  180. if temp, processed := m.preprocess(data); processed {
  181. break
  182. } else {
  183. data = temp
  184. }
  185. stats.SetBufferLength(int64(len(m.input)))
  186. if sconf.RunAsync {
  187. go doCollect(ctx, sink, data, stats, sconf, nil)
  188. } else {
  189. doCollect(ctx, sink, data, stats, sconf, nil)
  190. }
  191. case <-ctx.Done():
  192. logger.Infof("sink node %s instance %d done", m.name, instance)
  193. if err := sink.Close(ctx); err != nil {
  194. logger.Warnf("close sink node %s instance %d fails: %v", m.name, instance, err)
  195. }
  196. return
  197. case <-m.tch:
  198. logger.Debugf("rule %s sink receive checkpoint, do nothing", ctx.GetRuleId())
  199. }
  200. }
  201. } else {
  202. logger.Infof("Creating sink cache")
  203. var cache *Cache
  204. if m.qos >= api.AtLeastOnce {
  205. cache = NewCheckpointbasedCache(m.input, sconf.CacheLength, m.tch, result, ctx)
  206. } else {
  207. cache = NewTimebasedCache(m.input, sconf.CacheLength, sconf.CacheSaveInterval, result, ctx)
  208. }
  209. for {
  210. select {
  211. case data := <-cache.Out:
  212. if temp, processed := m.preprocess(data.data); processed {
  213. break
  214. } else {
  215. data.data = temp
  216. }
  217. stats.SetBufferLength(int64(len(m.input)))
  218. if sconf.RunAsync {
  219. go doCollect(ctx, sink, data, stats, sconf, cache.Complete)
  220. } else {
  221. doCollect(ctx, sink, data, stats, sconf, cache.Complete)
  222. }
  223. case <-ctx.Done():
  224. logger.Infof("sink node %s instance %d done", m.name, instance)
  225. if err := sink.Close(ctx); err != nil {
  226. logger.Warnf("close sink node %s instance %d fails: %v", m.name, instance, err)
  227. }
  228. return
  229. }
  230. }
  231. }
  232. }(i)
  233. }
  234. }()
  235. }
  236. func (m *SinkNode) reset() {
  237. if !m.isMock {
  238. m.sinks = nil
  239. }
  240. m.statManagers = nil
  241. }
  242. func doCollect(ctx api.StreamContext, sink api.Sink, item interface{}, stats StatManager, sconf *SinkConf, signalCh chan<- int) {
  243. stats.IncTotalRecordsIn()
  244. stats.ProcessTimeStart()
  245. defer stats.ProcessTimeEnd()
  246. var outs []map[string]interface{}
  247. switch val := item.(type) {
  248. case error:
  249. outs = []map[string]interface{}{
  250. {"error": val.Error()},
  251. }
  252. case []map[string]interface{}:
  253. outs = val
  254. default:
  255. outs = []map[string]interface{}{
  256. {"error": fmt.Sprintf("result is not a string but found %#v", val)},
  257. }
  258. }
  259. if sconf.Omitempty && (item == nil || len(outs) == 0) {
  260. ctx.GetLogger().Debugf("receive empty in sink")
  261. return
  262. }
  263. if !sconf.SendSingle {
  264. doCollectData(ctx, sink, outs, stats, sconf, signalCh)
  265. } else {
  266. for _, d := range outs {
  267. doCollectData(ctx, sink, d, stats, sconf, signalCh)
  268. }
  269. }
  270. }
  271. // doCollectData outData must be map or []map
  272. func doCollectData(ctx api.StreamContext, sink api.Sink, outData interface{}, stats StatManager, sconf *SinkConf, signalCh chan<- int) {
  273. retries := sconf.RetryCount
  274. for {
  275. select {
  276. case <-ctx.Done():
  277. ctx.GetLogger().Infof("sink node %s instance %d stops data resending", ctx.GetOpId(), ctx.GetInstanceId())
  278. return
  279. default:
  280. if err := sink.Collect(ctx, outData); err != nil {
  281. stats.IncTotalExceptions()
  282. ctx.GetLogger().Warnf("sink node %s instance %d publish %s error: %v", ctx.GetOpId(), ctx.GetInstanceId(), outData, err)
  283. if sconf.RetryInterval > 0 && retries > 0 && strings.HasPrefix(err.Error(), errorx.IOErr) {
  284. retries--
  285. time.Sleep(time.Duration(sconf.RetryInterval) * time.Millisecond)
  286. ctx.GetLogger().Debugf("try again")
  287. } else {
  288. return
  289. }
  290. } else {
  291. ctx.GetLogger().Debugf("success")
  292. stats.IncTotalRecordsOut()
  293. if signalCh != nil {
  294. cacheTuple, ok := outData.(*CacheTuple)
  295. if !ok {
  296. ctx.GetLogger().Warnf("got none cache tuple %v, should not happen", outData)
  297. }
  298. select {
  299. case signalCh <- cacheTuple.index:
  300. default:
  301. ctx.GetLogger().Warnf("sink cache missing response for %d", cacheTuple.index)
  302. }
  303. }
  304. return
  305. }
  306. }
  307. }
  308. }
  309. func getSink(name string, action map[string]interface{}) (api.Sink, error) {
  310. var (
  311. s api.Sink
  312. err error
  313. )
  314. s, err = io.Sink(name)
  315. if s != nil {
  316. err = s.Configure(action)
  317. if err != nil {
  318. return nil, err
  319. }
  320. return s, nil
  321. } else {
  322. if err != nil {
  323. return nil, err
  324. } else {
  325. return nil, fmt.Errorf("sink %s not found", name)
  326. }
  327. }
  328. }
  329. // AddOutput Override defaultNode
  330. func (m *SinkNode) AddOutput(_ chan<- interface{}, name string) error {
  331. return fmt.Errorf("fail to add output %s, sink %s cannot add output", name, m.name)
  332. }
  333. // Broadcast Override defaultNode
  334. func (m *SinkNode) Broadcast(_ interface{}) error {
  335. return fmt.Errorf("sink %s cannot add broadcast", m.name)
  336. }
  337. func (m *SinkNode) drainError(errCh chan<- error, err error, ctx api.StreamContext, logger api.Logger) {
  338. go func() {
  339. select {
  340. case errCh <- err:
  341. ctx.GetLogger().Errorf("error in sink %s", err)
  342. case <-ctx.Done():
  343. m.close(ctx, logger)
  344. }
  345. }()
  346. }
  347. func (m *SinkNode) close(ctx api.StreamContext, logger api.Logger) {
  348. for _, s := range m.sinks {
  349. if err := s.Close(ctx); err != nil {
  350. logger.Warnf("close sink fails: %v", err)
  351. }
  352. }
  353. if m.tch != nil {
  354. close(m.tch)
  355. m.tch = nil
  356. }
  357. }
  358. // SaveCache Only called when checkpoint enabled
  359. func (m *SinkNode) SaveCache() {
  360. m.tch <- struct{}{}
  361. }