sink_node.go 11 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397
  1. package nodes
  2. import (
  3. "bytes"
  4. "encoding/json"
  5. "fmt"
  6. "github.com/emqx/kuiper/common"
  7. "github.com/emqx/kuiper/common/templates"
  8. "github.com/emqx/kuiper/plugins"
  9. "github.com/emqx/kuiper/xstream/api"
  10. "github.com/emqx/kuiper/xstream/sinks"
  11. "sync"
  12. "text/template"
  13. "time"
  14. )
  15. type SinkNode struct {
  16. *defaultSinkNode
  17. //static
  18. sinkType string
  19. mutex sync.RWMutex
  20. //configs (also static for sinks)
  21. options map[string]interface{}
  22. isMock bool
  23. //states varies after restart
  24. sinks []api.Sink
  25. tch chan struct{} //channel to trigger cache saved, will be trigger by checkpoint only
  26. }
  27. func NewSinkNode(name string, sinkType string, props map[string]interface{}) *SinkNode {
  28. bufferLength := 1024
  29. if c, ok := props["bufferLength"]; ok {
  30. if t, err := common.ToInt(c); err != nil || t <= 0 {
  31. //invalid property bufferLength
  32. } else {
  33. bufferLength = t
  34. }
  35. }
  36. return &SinkNode{
  37. defaultSinkNode: &defaultSinkNode{
  38. input: make(chan interface{}, bufferLength),
  39. defaultNode: &defaultNode{
  40. name: name,
  41. concurrency: 1,
  42. ctx: nil,
  43. },
  44. },
  45. sinkType: sinkType,
  46. options: props,
  47. }
  48. }
  49. //Only for mock source, do not use it in production
  50. func NewSinkNodeWithSink(name string, sink api.Sink, props map[string]interface{}) *SinkNode {
  51. return &SinkNode{
  52. defaultSinkNode: &defaultSinkNode{
  53. input: make(chan interface{}, 1024),
  54. defaultNode: &defaultNode{
  55. name: name,
  56. concurrency: 1,
  57. ctx: nil,
  58. },
  59. },
  60. sinks: []api.Sink{sink},
  61. options: props,
  62. isMock: true,
  63. }
  64. }
  65. func (m *SinkNode) Open(ctx api.StreamContext, result chan<- error) {
  66. m.ctx = ctx
  67. logger := ctx.GetLogger()
  68. logger.Debugf("open sink node %s", m.name)
  69. if m.qos >= api.AtLeastOnce {
  70. m.tch = make(chan struct{})
  71. }
  72. go func() {
  73. if c, ok := m.options["concurrency"]; ok {
  74. if t, err := common.ToInt(c); err != nil || t <= 0 {
  75. logger.Warnf("invalid type for concurrency property, should be positive integer but found %t", c)
  76. } else {
  77. m.concurrency = t
  78. }
  79. }
  80. runAsync := false
  81. if c, ok := m.options["runAsync"]; ok {
  82. if t, ok := c.(bool); !ok {
  83. logger.Warnf("invalid type for runAsync property, should be bool but found %t", c)
  84. } else {
  85. runAsync = t
  86. }
  87. }
  88. retryInterval := 1000
  89. if c, ok := m.options["retryInterval"]; ok {
  90. if t, err := common.ToInt(c); err != nil || t < 0 {
  91. logger.Warnf("invalid type for retryInterval property, should be positive integer but found %t", c)
  92. } else {
  93. retryInterval = t
  94. }
  95. }
  96. cacheLength := 1024
  97. if c, ok := m.options["cacheLength"]; ok {
  98. if t, err := common.ToInt(c); err != nil || t < 0 {
  99. logger.Warnf("invalid type for cacheLength property, should be positive integer but found %t", c)
  100. } else {
  101. cacheLength = t
  102. }
  103. }
  104. cacheSaveInterval := 1000
  105. if c, ok := m.options["cacheSaveInterval"]; ok {
  106. if t, err := common.ToInt(c); err != nil || t < 0 {
  107. logger.Warnf("invalid type for cacheSaveInterval property, should be positive integer but found %t", c)
  108. } else {
  109. cacheSaveInterval = t
  110. }
  111. }
  112. omitIfEmpty := false
  113. if c, ok := m.options["omitIfEmpty"]; ok {
  114. if t, ok := c.(bool); !ok {
  115. logger.Warnf("invalid type for omitIfEmpty property, should be a bool value 'true/false'.", c)
  116. } else {
  117. omitIfEmpty = t
  118. }
  119. }
  120. sendSingle := false
  121. if c, ok := m.options["sendSingle"]; ok {
  122. if t, ok := c.(bool); !ok {
  123. logger.Warnf("invalid type for sendSingle property, should be a bool value 'true/false'.", c)
  124. } else {
  125. sendSingle = t
  126. }
  127. }
  128. var tp *template.Template = nil
  129. if c, ok := m.options["dataTemplate"]; ok {
  130. if t, ok := c.(string); !ok {
  131. logger.Warnf("invalid type for dateTemplate property, should be a string value.", c)
  132. } else {
  133. funcMap := template.FuncMap{
  134. "json": templates.JsonMarshal,
  135. "base64": templates.Base64Encode,
  136. "add": templates.Add,
  137. }
  138. temp, err := template.New("sink").Funcs(funcMap).Parse(t)
  139. if err != nil {
  140. msg := fmt.Sprintf("property dataTemplate %v is invalid: %v", t, err)
  141. result <- fmt.Errorf(msg)
  142. logger.Warnf(msg)
  143. return
  144. } else {
  145. tp = temp
  146. }
  147. }
  148. }
  149. m.reset()
  150. logger.Infof("open sink node %d instances", m.concurrency)
  151. for i := 0; i < m.concurrency; i++ { // workers
  152. go func(instance int) {
  153. var sink api.Sink
  154. var err error
  155. if !m.isMock {
  156. logger.Debugf(fmt.Sprintf("Trying to get sink for rule %s with options %v\n", ctx.GetRuleId(), m.options))
  157. sink, err = getSink(m.sinkType, m.options)
  158. if err != nil {
  159. m.drainError(result, err, ctx, logger)
  160. return
  161. }
  162. logger.Debugf(fmt.Sprintf("Successfully get the sink %s", m.sinkType))
  163. m.mutex.Lock()
  164. m.sinks = append(m.sinks, sink)
  165. m.mutex.Unlock()
  166. logger.Debugf(fmt.Sprintf("Now is to open sink for rule %s.\n", ctx.GetRuleId()))
  167. if err := sink.Open(ctx); err != nil {
  168. m.drainError(result, err, ctx, logger)
  169. return
  170. }
  171. logger.Debugf(fmt.Sprintf("Successfully open sink for rule %s.\n", ctx.GetRuleId()))
  172. } else {
  173. sink = m.sinks[instance]
  174. }
  175. stats, err := NewStatManager("sink", ctx)
  176. if err != nil {
  177. m.drainError(result, err, ctx, logger)
  178. return
  179. }
  180. m.mutex.Lock()
  181. m.statManagers = append(m.statManagers, stats)
  182. m.mutex.Unlock()
  183. var cache *Cache
  184. if m.qos >= api.AtLeastOnce {
  185. cache = NewCheckpointbasedCache(m.input, cacheLength, m.tch, result, ctx)
  186. } else {
  187. cache = NewTimebasedCache(m.input, cacheLength, cacheSaveInterval, result, ctx)
  188. }
  189. for {
  190. select {
  191. case data := <-cache.Out:
  192. if newdata, processed := m.preprocess(data.data); processed {
  193. break
  194. } else {
  195. data.data = newdata
  196. }
  197. stats.SetBufferLength(int64(cache.Length()))
  198. if runAsync {
  199. go doCollect(sink, data, stats, retryInterval, omitIfEmpty, sendSingle, tp, cache.Complete, ctx)
  200. } else {
  201. doCollect(sink, data, stats, retryInterval, omitIfEmpty, sendSingle, tp, cache.Complete, ctx)
  202. }
  203. case <-ctx.Done():
  204. logger.Infof("sink node %s instance %d done", m.name, instance)
  205. if err := sink.Close(ctx); err != nil {
  206. logger.Warnf("close sink node %s instance %d fails: %v", m.name, instance, err)
  207. }
  208. return
  209. }
  210. }
  211. }(i)
  212. }
  213. }()
  214. }
  215. func (m *SinkNode) reset() {
  216. if !m.isMock {
  217. m.sinks = nil
  218. }
  219. m.statManagers = nil
  220. }
  221. func extractInput(v []byte) ([]map[string]interface{}, error) {
  222. var j []map[string]interface{}
  223. if err := json.Unmarshal(v, &j); err != nil {
  224. return nil, fmt.Errorf("fail to decode the input %s as json: %v", v, err)
  225. }
  226. return j, nil
  227. }
  228. func doCollect(sink api.Sink, item *CacheTuple, stats StatManager, retryInterval int, omitIfEmpty bool, sendSingle bool, tp *template.Template, signalCh chan<- int, ctx api.StreamContext) {
  229. stats.IncTotalRecordsIn()
  230. stats.ProcessTimeStart()
  231. defer stats.ProcessTimeEnd()
  232. logger := ctx.GetLogger()
  233. var outdatas [][]byte
  234. switch val := item.data.(type) {
  235. case []byte:
  236. if omitIfEmpty && string(val) == "[{}]" {
  237. return
  238. }
  239. var (
  240. err error
  241. j []map[string]interface{}
  242. )
  243. if sendSingle || tp != nil {
  244. j, err = extractInput(val)
  245. if err != nil {
  246. stats.IncTotalExceptions()
  247. logger.Warnf("sink node %s instance %d publish %s error: %v", ctx.GetOpId(), ctx.GetInstanceId(), val, err)
  248. return
  249. }
  250. logger.Debugf("receive %d records", len(j))
  251. }
  252. if !sendSingle {
  253. if tp != nil {
  254. var output bytes.Buffer
  255. err := tp.Execute(&output, j)
  256. if err != nil {
  257. logger.Warnf("sink node %s instance %d publish %s decode template error: %v", ctx.GetOpId(), ctx.GetInstanceId(), val, err)
  258. stats.IncTotalExceptions()
  259. return
  260. }
  261. outdatas = append(outdatas, output.Bytes())
  262. } else {
  263. outdatas = [][]byte{val}
  264. }
  265. } else {
  266. for _, r := range j {
  267. if tp != nil {
  268. var output bytes.Buffer
  269. err := tp.Execute(&output, r)
  270. if err != nil {
  271. logger.Warnf("sink node %s instance %d publish %s decode template error: %v", ctx.GetOpId(), ctx.GetInstanceId(), val, err)
  272. stats.IncTotalExceptions()
  273. return
  274. }
  275. outdatas = append(outdatas, output.Bytes())
  276. } else {
  277. if ot, e := json.Marshal(r); e != nil {
  278. logger.Warnf("sink node %s instance %d publish %s marshal error: %v", ctx.GetOpId(), ctx.GetInstanceId(), r, e)
  279. stats.IncTotalExceptions()
  280. return
  281. } else {
  282. outdatas = append(outdatas, ot)
  283. }
  284. }
  285. }
  286. }
  287. case error:
  288. outdatas = [][]byte{[]byte(fmt.Sprintf(`[{"error":"%s"}]`, val.Error()))}
  289. default:
  290. outdatas = [][]byte{[]byte(fmt.Sprintf(`[{"error":"result is not a string but found %#v"}]`, val))}
  291. }
  292. for _, outdata := range outdatas {
  293. outerloop:
  294. for {
  295. select {
  296. case <-ctx.Done():
  297. logger.Infof("sink node %s instance %d stops data resending", ctx.GetOpId(), ctx.GetInstanceId())
  298. return
  299. default:
  300. if err := sink.Collect(ctx, outdata); err != nil {
  301. stats.IncTotalExceptions()
  302. logger.Warnf("sink node %s instance %d publish %s error: %v", ctx.GetOpId(), ctx.GetInstanceId(), outdata, err)
  303. if retryInterval > 0 {
  304. time.Sleep(time.Duration(retryInterval) * time.Millisecond)
  305. logger.Debugf("try again")
  306. } else {
  307. break outerloop
  308. }
  309. } else {
  310. logger.Debugf("success")
  311. stats.IncTotalRecordsOut()
  312. signalCh <- item.index
  313. break outerloop
  314. }
  315. }
  316. }
  317. }
  318. }
  319. func doGetSink(name string, action map[string]interface{}) (api.Sink, error) {
  320. var (
  321. s api.Sink
  322. err error
  323. )
  324. switch name {
  325. case "log":
  326. s = sinks.NewLogSink()
  327. case "logToMemory":
  328. s = sinks.NewLogSinkToMemory()
  329. case "mqtt":
  330. s = &sinks.MQTTSink{}
  331. case "rest":
  332. s = &sinks.RestSink{}
  333. case "nop":
  334. s = &sinks.NopSink{}
  335. default:
  336. s, err = plugins.GetSink(name)
  337. if err != nil {
  338. return nil, err
  339. }
  340. }
  341. err = s.Configure(action)
  342. if err != nil {
  343. return nil, err
  344. }
  345. return s, nil
  346. }
  347. //Override defaultNode
  348. func (m *SinkNode) AddOutput(_ chan<- interface{}, name string) error {
  349. return fmt.Errorf("fail to add output %s, sink %s cannot add output", name, m.name)
  350. }
  351. //Override defaultNode
  352. func (m *SinkNode) Broadcast(_ interface{}) error {
  353. return fmt.Errorf("sink %s cannot add broadcast", m.name)
  354. }
  355. func (m *SinkNode) drainError(errCh chan<- error, err error, ctx api.StreamContext, logger api.Logger) {
  356. go func() {
  357. select {
  358. case errCh <- err:
  359. ctx.GetLogger().Errorf("error in sink %s", err)
  360. case <-ctx.Done():
  361. m.close(ctx, logger)
  362. }
  363. }()
  364. }
  365. func (m *SinkNode) close(ctx api.StreamContext, logger api.Logger) {
  366. for _, s := range m.sinks {
  367. if err := s.Close(ctx); err != nil {
  368. logger.Warnf("close sink fails: %v", err)
  369. }
  370. }
  371. if m.tch != nil {
  372. close(m.tch)
  373. m.tch = nil
  374. }
  375. }
  376. // Only called when checkpoint enabled
  377. func (m *SinkNode) SaveCache() {
  378. m.tch <- struct{}{}
  379. }