sink_cache.go 6.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264
  1. package nodes
  2. import (
  3. "encoding/gob"
  4. "fmt"
  5. "github.com/emqx/kuiper/common"
  6. "github.com/emqx/kuiper/xstream/api"
  7. "github.com/emqx/kuiper/xstream/checkpoints"
  8. "io"
  9. "path"
  10. "sort"
  11. "strconv"
  12. )
  13. type CacheTuple struct {
  14. index int
  15. data interface{}
  16. }
  17. type LinkedQueue struct {
  18. Data map[int]interface{}
  19. Tail int
  20. }
  21. func (l *LinkedQueue) append(item interface{}) {
  22. l.Data[l.Tail] = item
  23. l.Tail++
  24. }
  25. func (l *LinkedQueue) delete(index int) {
  26. delete(l.Data, index)
  27. }
  28. func (l *LinkedQueue) reset() {
  29. l.Tail = 0
  30. }
  31. func (l *LinkedQueue) length() int {
  32. return len(l.Data)
  33. }
  34. func (l *LinkedQueue) clone() *LinkedQueue {
  35. result := &LinkedQueue{
  36. Data: make(map[int]interface{}),
  37. Tail: l.Tail,
  38. }
  39. for k, v := range l.Data {
  40. result.Data[k] = v
  41. }
  42. return result
  43. }
  44. func (l *LinkedQueue) String() string {
  45. return fmt.Sprintf("tail: %d, data: %v", l.Tail, l.Data)
  46. }
  47. type Cache struct {
  48. //Data and control channels
  49. in <-chan interface{}
  50. Out chan *CacheTuple
  51. Complete chan int
  52. errorCh chan<- error
  53. //states
  54. pending *LinkedQueue
  55. changed bool
  56. //serialize
  57. key string //the key for current cache
  58. store common.KeyValue
  59. }
  60. func NewTimebasedCache(in <-chan interface{}, limit int, saveInterval int, errCh chan<- error, ctx api.StreamContext) *Cache {
  61. c := &Cache{
  62. in: in,
  63. Out: make(chan *CacheTuple, limit),
  64. Complete: make(chan int),
  65. errorCh: errCh,
  66. }
  67. go c.timebasedRun(ctx, saveInterval)
  68. return c
  69. }
  70. func (c *Cache) initStore(ctx api.StreamContext) {
  71. logger := ctx.GetLogger()
  72. dbDir, err := common.GetDataLoc()
  73. logger.Debugf("cache saved to %s", dbDir)
  74. if err != nil {
  75. c.drainError(err)
  76. }
  77. c.store = common.GetSimpleKVStore(path.Join(dbDir, "sink"))
  78. c.key = ctx.GetRuleId() + ctx.GetOpId() + strconv.Itoa(ctx.GetInstanceId())
  79. //load cache
  80. if err := c.loadCache(); err != nil {
  81. go c.drainError(err)
  82. return
  83. }
  84. }
  85. func (c *Cache) timebasedRun(ctx api.StreamContext, saveInterval int) {
  86. logger := ctx.GetLogger()
  87. c.initStore(ctx)
  88. ticker := common.GetTicker(saveInterval)
  89. var tcount = 0
  90. for {
  91. select {
  92. case item := <-c.in:
  93. index := c.pending.Tail
  94. c.pending.append(item)
  95. //non blocking until limit exceeded
  96. c.Out <- &CacheTuple{
  97. index: index,
  98. data: item,
  99. }
  100. c.changed = true
  101. case index := <-c.Complete:
  102. c.pending.delete(index)
  103. c.changed = true
  104. case <-ticker.C:
  105. tcount++
  106. l := c.pending.length()
  107. if l == 0 {
  108. c.pending.reset()
  109. }
  110. //If the data is still changing, only do a save when the cache has more than threshold to prevent too much file IO
  111. //If the data is not changing in the time slot and have not saved before, save it. This is to prevent the
  112. //data won't be saved as the cache never pass the threshold
  113. //logger.Infof("ticker %t, l=%d\n", c.changed, l)
  114. if (c.changed && l > common.Config.Sink.CacheThreshold) || (tcount == common.Config.Sink.CacheTriggerCount && c.changed) {
  115. logger.Infof("save cache for rule %s, %s", ctx.GetRuleId(), c.pending.String())
  116. clone := c.pending.clone()
  117. c.changed = false
  118. go func() {
  119. if err := c.saveCache(logger, clone); err != nil {
  120. logger.Debugf("%v", err)
  121. c.drainError(err)
  122. }
  123. }()
  124. }
  125. if tcount >= common.Config.Sink.CacheThreshold {
  126. tcount = 0
  127. }
  128. case <-ctx.Done():
  129. err := c.saveCache(logger, c.pending)
  130. if err != nil {
  131. logger.Warnf("Error found during saving cache: %s \n ", err)
  132. }
  133. logger.Infof("sink node %s instance cache %d done", ctx.GetOpId(), ctx.GetInstanceId())
  134. return
  135. }
  136. }
  137. }
  138. func (c *Cache) loadCache() error {
  139. c.pending = &LinkedQueue{
  140. Data: make(map[int]interface{}),
  141. Tail: 0,
  142. }
  143. gob.Register(c.pending)
  144. err := c.store.Open()
  145. if err != nil && err != io.EOF {
  146. return err
  147. }
  148. defer c.store.Close()
  149. if err == nil {
  150. if t, f := c.store.Get(c.key); f {
  151. if mt, ok := t.(*LinkedQueue); ok {
  152. c.pending = mt
  153. c.changed = true
  154. // To store the keys in slice in sorted order
  155. var keys []int
  156. for k := range mt.Data {
  157. keys = append(keys, k)
  158. }
  159. sort.Ints(keys)
  160. for _, k := range keys {
  161. t := &CacheTuple{
  162. index: k,
  163. data: mt.Data[k],
  164. }
  165. c.Out <- t
  166. }
  167. return nil
  168. } else {
  169. return fmt.Errorf("load malform cache, found %t(%v)", t, t)
  170. }
  171. }
  172. }
  173. return nil
  174. }
  175. func (c *Cache) saveCache(_ api.Logger, p *LinkedQueue) error {
  176. err := c.store.Open()
  177. if err != nil {
  178. return err
  179. }
  180. defer c.store.Close()
  181. return c.store.Replace(c.key, p)
  182. }
  183. func (c *Cache) drainError(err error) {
  184. c.errorCh <- err
  185. }
  186. func (c *Cache) Length() int {
  187. return c.pending.length()
  188. }
  189. func NewCheckpointbasedCache(in <-chan interface{}, limit int, tch <-chan struct{}, errCh chan<- error, ctx api.StreamContext) *Cache {
  190. c := &Cache{
  191. in: in,
  192. Out: make(chan *CacheTuple, limit),
  193. Complete: make(chan int),
  194. errorCh: errCh,
  195. }
  196. go c.checkpointbasedRun(ctx, tch)
  197. return c
  198. }
  199. func (c *Cache) checkpointbasedRun(ctx api.StreamContext, tch <-chan struct{}) {
  200. logger := ctx.GetLogger()
  201. c.initStore(ctx)
  202. for {
  203. select {
  204. case item := <-c.in:
  205. // possibility of barrier, ignore if found
  206. if boe, ok := item.(*checkpoints.BufferOrEvent); ok {
  207. if _, ok := boe.Data.(*checkpoints.Barrier); ok {
  208. c.Out <- &CacheTuple{
  209. data: item,
  210. }
  211. logger.Debugf("sink cache send out barrier %v", boe.Data)
  212. break
  213. }
  214. }
  215. index := c.pending.Tail
  216. c.pending.append(item)
  217. //non blocking until limit exceeded
  218. c.Out <- &CacheTuple{
  219. index: index,
  220. data: item,
  221. }
  222. logger.Debugf("sink cache send out tuple %v", item)
  223. c.changed = true
  224. case index := <-c.Complete:
  225. c.pending.delete(index)
  226. c.changed = true
  227. case <-tch:
  228. logger.Infof("save cache for rule %s, %s", ctx.GetRuleId(), c.pending.String())
  229. clone := c.pending.clone()
  230. if c.changed {
  231. go func() {
  232. if err := c.saveCache(logger, clone); err != nil {
  233. logger.Debugf("%v", err)
  234. c.drainError(err)
  235. }
  236. }()
  237. }
  238. c.changed = false
  239. case <-ctx.Done():
  240. logger.Infof("sink node %s instance cache %d done", ctx.GetOpId(), ctx.GetInstanceId())
  241. return
  242. }
  243. }
  244. }