lookup_node.go 7.9 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287
  1. // Copyright 2021-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/conf"
  18. "github.com/lf-edge/ekuiper/internal/topo/lookup"
  19. "github.com/lf-edge/ekuiper/internal/topo/lookup/cache"
  20. nodeConf "github.com/lf-edge/ekuiper/internal/topo/node/conf"
  21. "github.com/lf-edge/ekuiper/internal/topo/node/metric"
  22. "github.com/lf-edge/ekuiper/internal/xsql"
  23. "github.com/lf-edge/ekuiper/pkg/api"
  24. "github.com/lf-edge/ekuiper/pkg/ast"
  25. "github.com/lf-edge/ekuiper/pkg/cast"
  26. "github.com/lf-edge/ekuiper/pkg/infra"
  27. )
  28. type LookupConf struct {
  29. Cache bool `json:"cache"`
  30. CacheTTL int `json:"cacheTtl"`
  31. CacheMissingKey bool `json:"cacheMissingKey"`
  32. }
  33. // LookupNode will look up the data from the external source when receiving an event
  34. type LookupNode struct {
  35. *defaultSinkNode
  36. statManager metric.StatManager
  37. sourceType string
  38. joinType ast.JoinType
  39. vals []ast.Expr
  40. srcOptions *ast.Options
  41. conf *LookupConf
  42. fields []string
  43. keys []string
  44. }
  45. func NewLookupNode(name string, fields []string, keys []string, joinType ast.JoinType, vals []ast.Expr, srcOptions *ast.Options, options *api.RuleOption) (*LookupNode, error) {
  46. t := srcOptions.TYPE
  47. if t == "" {
  48. return nil, fmt.Errorf("source type is not specified")
  49. }
  50. props := nodeConf.GetSourceConf(t, srcOptions)
  51. lookupConf := &LookupConf{}
  52. if lc, ok := props["lookup"].(map[string]interface{}); ok {
  53. err := cast.MapToStruct(lc, lookupConf)
  54. if err != nil {
  55. return nil, err
  56. }
  57. }
  58. n := &LookupNode{
  59. fields: fields,
  60. keys: keys,
  61. srcOptions: srcOptions,
  62. conf: lookupConf,
  63. sourceType: t,
  64. joinType: joinType,
  65. vals: vals,
  66. }
  67. n.defaultSinkNode = &defaultSinkNode{
  68. input: make(chan interface{}, options.BufferLength),
  69. defaultNode: &defaultNode{
  70. outputs: make(map[string]chan<- interface{}),
  71. name: name,
  72. sendError: options.SendError,
  73. },
  74. }
  75. return n, nil
  76. }
  77. func (n *LookupNode) Exec(ctx api.StreamContext, errCh chan<- error) {
  78. n.ctx = ctx
  79. log := ctx.GetLogger()
  80. log.Debugf("LookupNode %s is started", n.name)
  81. if len(n.outputs) <= 0 {
  82. infra.DrainError(ctx, fmt.Errorf("no output channel found"), errCh)
  83. return
  84. }
  85. stats, err := metric.NewStatManager(ctx, "op")
  86. if err != nil {
  87. infra.DrainError(ctx, fmt.Errorf("no output channel found"), errCh)
  88. return
  89. }
  90. n.statManager = stats
  91. go func() {
  92. err := infra.SafeRun(func() error {
  93. ns, err := lookup.Attach(n.name)
  94. if err != nil {
  95. return err
  96. }
  97. defer lookup.Detach(n.name)
  98. fv, _ := xsql.NewFunctionValuersForOp(ctx)
  99. var c *cache.Cache
  100. if n.conf.Cache {
  101. c = cache.NewCache(n.conf.CacheTTL, n.conf.CacheMissingKey)
  102. defer c.Close()
  103. }
  104. // Start the lookup source loop
  105. for {
  106. log.Debugf("LookupNode %s is looping", n.name)
  107. select {
  108. // process incoming item from both streams(transformed) and tables
  109. case item, opened := <-n.input:
  110. processed := false
  111. if item, processed = n.preprocess(item); processed {
  112. break
  113. }
  114. n.statManager.IncTotalRecordsIn()
  115. n.statManager.ProcessTimeStart()
  116. if !opened {
  117. n.statManager.IncTotalExceptions("input channel closed")
  118. break
  119. }
  120. switch d := item.(type) {
  121. case error:
  122. n.Broadcast(d)
  123. n.statManager.IncTotalExceptions(d.Error())
  124. case xsql.TupleRow:
  125. log.Debugf("Lookup Node receive tuple input %s", d)
  126. n.statManager.ProcessTimeStart()
  127. sets := &xsql.JoinTuples{Content: make([]*xsql.JoinTuple, 0)}
  128. err := n.lookup(ctx, d, fv, ns, sets, c)
  129. if err != nil {
  130. n.Broadcast(err)
  131. n.statManager.IncTotalExceptions(err.Error())
  132. } else {
  133. n.Broadcast(sets)
  134. n.statManager.IncTotalRecordsOut()
  135. }
  136. n.statManager.ProcessTimeEnd()
  137. n.statManager.SetBufferLength(int64(len(n.input)))
  138. case *xsql.WindowTuples:
  139. log.Debugf("Lookup Node receive window input %s", d)
  140. n.statManager.ProcessTimeStart()
  141. sets := &xsql.JoinTuples{Content: make([]*xsql.JoinTuple, 0)}
  142. err := d.Range(func(i int, r xsql.ReadonlyRow) (bool, error) {
  143. tr, ok := r.(xsql.TupleRow)
  144. if !ok {
  145. return false, fmt.Errorf("Invalid window element, must be a tuple row but got %v", r)
  146. }
  147. err := n.lookup(ctx, tr, fv, ns, sets, c)
  148. if err != nil {
  149. return false, err
  150. }
  151. return true, nil
  152. })
  153. if err != nil {
  154. n.Broadcast(err)
  155. n.statManager.IncTotalExceptions(err.Error())
  156. } else {
  157. n.Broadcast(sets)
  158. n.statManager.IncTotalRecordsOut()
  159. }
  160. n.statManager.ProcessTimeEnd()
  161. n.statManager.SetBufferLength(int64(len(n.input)))
  162. default:
  163. e := fmt.Errorf("run lookup node error: invalid input type but got %[1]T(%[1]v)", d)
  164. n.Broadcast(e)
  165. n.statManager.IncTotalExceptions(e.Error())
  166. }
  167. case <-ctx.Done():
  168. log.Infoln("Cancelling lookup node....")
  169. return nil
  170. }
  171. }
  172. })
  173. if err != nil {
  174. infra.DrainError(ctx, err, errCh)
  175. }
  176. }()
  177. }
  178. // lookup will lookup the cache firstly, if expires, read the external source
  179. func (n *LookupNode) lookup(ctx api.StreamContext, d xsql.TupleRow, fv *xsql.FunctionValuer, ns api.LookupSource, tuples *xsql.JoinTuples, c *cache.Cache) error {
  180. ve := &xsql.ValuerEval{Valuer: xsql.MultiValuer(d, fv)}
  181. cvs := make([]interface{}, len(n.vals))
  182. hasNil := false
  183. for i, val := range n.vals {
  184. cvs[i] = ve.Eval(val)
  185. if cvs[i] == nil {
  186. hasNil = true
  187. }
  188. }
  189. var (
  190. r []api.SourceTuple
  191. e error
  192. ok bool
  193. )
  194. if !hasNil { // if any of the value is nil, the lookup will always return empty result
  195. if c != nil {
  196. k := fmt.Sprintf("%v", cvs)
  197. r, ok = c.Get(k)
  198. if !ok {
  199. r, e = ns.Lookup(ctx, n.fields, n.keys, cvs)
  200. if e != nil {
  201. return e
  202. }
  203. c.Set(k, r)
  204. }
  205. } else {
  206. r, e = ns.Lookup(ctx, n.fields, n.keys, cvs)
  207. }
  208. }
  209. if e != nil {
  210. return e
  211. } else {
  212. if len(r) == 0 {
  213. if n.joinType == ast.LEFT_JOIN {
  214. merged := &xsql.JoinTuple{}
  215. merged.AddTuple(d)
  216. tuples.Content = append(tuples.Content, merged)
  217. } else {
  218. ctx.GetLogger().Debugf("Lookup Node %s no result found for tuple %s", n.name, d)
  219. return nil
  220. }
  221. }
  222. for _, v := range r {
  223. merged := &xsql.JoinTuple{}
  224. merged.AddTuple(d)
  225. t := &xsql.Tuple{
  226. Emitter: n.name,
  227. Message: v.Message(),
  228. Metadata: v.Meta(),
  229. Timestamp: conf.GetNowInMilli(),
  230. }
  231. merged.AddTuple(t)
  232. tuples.Content = append(tuples.Content, merged)
  233. }
  234. return nil
  235. }
  236. }
  237. func (n *LookupNode) GetMetrics() [][]interface{} {
  238. if n.statManager != nil {
  239. return [][]interface{}{
  240. n.statManager.GetMetrics(),
  241. }
  242. } else {
  243. return nil
  244. }
  245. }
  246. func (n *LookupNode) merge(ctx api.StreamContext, d xsql.TupleRow, r []map[string]interface{}) {
  247. n.statManager.ProcessTimeStart()
  248. sets := &xsql.JoinTuples{Content: make([]*xsql.JoinTuple, 0)}
  249. if len(r) == 0 {
  250. if n.joinType == ast.LEFT_JOIN {
  251. merged := &xsql.JoinTuple{}
  252. merged.AddTuple(d)
  253. sets.Content = append(sets.Content, merged)
  254. } else {
  255. ctx.GetLogger().Debugf("Lookup Node %s no result found for tuple %s", n.name, d)
  256. return
  257. }
  258. }
  259. for _, v := range r {
  260. merged := &xsql.JoinTuple{}
  261. merged.AddTuple(d)
  262. t := &xsql.Tuple{
  263. Emitter: n.name,
  264. Message: v,
  265. Timestamp: conf.GetNowInMilli(),
  266. }
  267. merged.AddTuple(t)
  268. sets.Content = append(sets.Content, merged)
  269. }
  270. n.Broadcast(sets)
  271. n.statManager.ProcessTimeEnd()
  272. n.statManager.IncTotalRecordsOut()
  273. n.statManager.SetBufferLength(int64(len(n.input)))
  274. }