sink_cache.go 6.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265
  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. logger.Debugf("cache saved to key %s", c.key)
  80. //load cache
  81. if err := c.loadCache(); err != nil {
  82. go c.drainError(err)
  83. return
  84. }
  85. }
  86. func (c *Cache) timebasedRun(ctx api.StreamContext, saveInterval int) {
  87. logger := ctx.GetLogger()
  88. c.initStore(ctx)
  89. ticker := common.GetTicker(saveInterval)
  90. var tcount = 0
  91. for {
  92. select {
  93. case item := <-c.in:
  94. index := c.pending.Tail
  95. c.pending.append(item)
  96. //non blocking until limit exceeded
  97. c.Out <- &CacheTuple{
  98. index: index,
  99. data: item,
  100. }
  101. c.changed = true
  102. case index := <-c.Complete:
  103. c.pending.delete(index)
  104. c.changed = true
  105. case <-ticker.C:
  106. tcount++
  107. l := c.pending.length()
  108. if l == 0 {
  109. c.pending.reset()
  110. }
  111. //If the data is still changing, only do a save when the cache has more than threshold to prevent too much file IO
  112. //If the data is not changing in the time slot and have not saved before, save it. This is to prevent the
  113. //data won't be saved as the cache never pass the threshold
  114. //logger.Infof("ticker %t, l=%d\n", c.changed, l)
  115. if (c.changed && l > common.Config.Sink.CacheThreshold) || (tcount == common.Config.Sink.CacheTriggerCount && c.changed) {
  116. logger.Infof("save cache for rule %s, %s", ctx.GetRuleId(), c.pending.String())
  117. clone := c.pending.clone()
  118. c.changed = false
  119. go func() {
  120. if err := c.saveCache(logger, clone); err != nil {
  121. logger.Debugf("%v", err)
  122. c.drainError(err)
  123. }
  124. }()
  125. }
  126. if tcount >= common.Config.Sink.CacheThreshold {
  127. tcount = 0
  128. }
  129. case <-ctx.Done():
  130. err := c.saveCache(logger, c.pending)
  131. if err != nil {
  132. logger.Warnf("Error found during saving cache: %s \n ", err)
  133. }
  134. logger.Infof("sink node %s instance cache %d done", ctx.GetOpId(), ctx.GetInstanceId())
  135. return
  136. }
  137. }
  138. }
  139. func (c *Cache) loadCache() error {
  140. c.pending = &LinkedQueue{
  141. Data: make(map[int]interface{}),
  142. Tail: 0,
  143. }
  144. gob.Register(c.pending)
  145. err := c.store.Open()
  146. if err != nil && err != io.EOF {
  147. return err
  148. }
  149. defer c.store.Close()
  150. if err == nil {
  151. if t, f := c.store.Get(c.key); f {
  152. if mt, ok := t.(*LinkedQueue); ok {
  153. c.pending = mt
  154. c.changed = true
  155. // To store the keys in slice in sorted order
  156. var keys []int
  157. for k := range mt.Data {
  158. keys = append(keys, k)
  159. }
  160. sort.Ints(keys)
  161. for _, k := range keys {
  162. t := &CacheTuple{
  163. index: k,
  164. data: mt.Data[k],
  165. }
  166. c.Out <- t
  167. }
  168. return nil
  169. } else {
  170. return fmt.Errorf("load malform cache, found %t(%v)", t, t)
  171. }
  172. }
  173. }
  174. return nil
  175. }
  176. func (c *Cache) saveCache(_ api.Logger, p *LinkedQueue) error {
  177. err := c.store.Open()
  178. if err != nil {
  179. return err
  180. }
  181. defer c.store.Close()
  182. return c.store.Replace(c.key, p)
  183. }
  184. func (c *Cache) drainError(err error) {
  185. c.errorCh <- err
  186. }
  187. func (c *Cache) Length() int {
  188. return c.pending.length()
  189. }
  190. func NewCheckpointbasedCache(in <-chan interface{}, limit int, tch <-chan struct{}, errCh chan<- error, ctx api.StreamContext) *Cache {
  191. c := &Cache{
  192. in: in,
  193. Out: make(chan *CacheTuple, limit),
  194. Complete: make(chan int),
  195. errorCh: errCh,
  196. }
  197. go c.checkpointbasedRun(ctx, tch)
  198. return c
  199. }
  200. func (c *Cache) checkpointbasedRun(ctx api.StreamContext, tch <-chan struct{}) {
  201. logger := ctx.GetLogger()
  202. c.initStore(ctx)
  203. for {
  204. select {
  205. case item := <-c.in:
  206. // possibility of barrier, ignore if found
  207. if boe, ok := item.(*checkpoints.BufferOrEvent); ok {
  208. if _, ok := boe.Data.(*checkpoints.Barrier); ok {
  209. c.Out <- &CacheTuple{
  210. data: item,
  211. }
  212. logger.Debugf("sink cache send out barrier %v", boe.Data)
  213. break
  214. }
  215. }
  216. index := c.pending.Tail
  217. c.pending.append(item)
  218. //non blocking until limit exceeded
  219. c.Out <- &CacheTuple{
  220. index: index,
  221. data: item,
  222. }
  223. logger.Debugf("sink cache send out tuple %v", item)
  224. c.changed = true
  225. case index := <-c.Complete:
  226. c.pending.delete(index)
  227. c.changed = true
  228. case <-tch:
  229. logger.Infof("save cache for rule %s, %s", ctx.GetRuleId(), c.pending.String())
  230. clone := c.pending.clone()
  231. if c.changed {
  232. go func() {
  233. if err := c.saveCache(logger, clone); err != nil {
  234. logger.Debugf("%v", err)
  235. c.drainError(err)
  236. }
  237. }()
  238. }
  239. c.changed = false
  240. case <-ctx.Done():
  241. logger.Infof("sink node %s instance cache %d done", ctx.GetOpId(), ctx.GetInstanceId())
  242. return
  243. }
  244. }
  245. }