coordinator.go 7.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281
  1. package checkpoint
  2. import (
  3. "github.com/benbjohnson/clock"
  4. "github.com/lf-edge/ekuiper/internal/conf"
  5. "github.com/lf-edge/ekuiper/pkg/api"
  6. "github.com/lf-edge/ekuiper/pkg/cast"
  7. "sync"
  8. )
  9. type pendingCheckpoint struct {
  10. checkpointId int64
  11. isDiscarded bool
  12. notYetAckTasks map[string]bool
  13. }
  14. func newPendingCheckpoint(checkpointId int64, tasksToWaitFor []Responder) *pendingCheckpoint {
  15. pc := &pendingCheckpoint{checkpointId: checkpointId}
  16. nyat := make(map[string]bool)
  17. for _, r := range tasksToWaitFor {
  18. nyat[r.GetName()] = true
  19. }
  20. pc.notYetAckTasks = nyat
  21. return pc
  22. }
  23. func (c *pendingCheckpoint) ack(opId string) bool {
  24. if c.isDiscarded {
  25. return false
  26. }
  27. delete(c.notYetAckTasks, opId)
  28. //TODO serialize state
  29. return true
  30. }
  31. func (c *pendingCheckpoint) isFullyAck() bool {
  32. return len(c.notYetAckTasks) == 0
  33. }
  34. func (c *pendingCheckpoint) finalize() *completedCheckpoint {
  35. ccp := &completedCheckpoint{checkpointId: c.checkpointId}
  36. return ccp
  37. }
  38. func (c *pendingCheckpoint) dispose(_ bool) {
  39. c.isDiscarded = true
  40. }
  41. type completedCheckpoint struct {
  42. checkpointId int64
  43. }
  44. type checkpointStore struct {
  45. maxNum int
  46. checkpoints []*completedCheckpoint
  47. }
  48. func (s *checkpointStore) add(c *completedCheckpoint) {
  49. s.checkpoints = append(s.checkpoints, c)
  50. if len(s.checkpoints) > s.maxNum {
  51. s.checkpoints = s.checkpoints[1:]
  52. }
  53. }
  54. func (s *checkpointStore) getLatest() *completedCheckpoint {
  55. if len(s.checkpoints) > 0 {
  56. return s.checkpoints[len(s.checkpoints)-1]
  57. }
  58. return nil
  59. }
  60. type Coordinator struct {
  61. tasksToTrigger []Responder
  62. tasksToWaitFor []Responder
  63. sinkTasks []SinkTask
  64. pendingCheckpoints *sync.Map
  65. completedCheckpoints *checkpointStore
  66. ruleId string
  67. baseInterval int
  68. cleanThreshold int
  69. advanceToEndOfEventTime bool
  70. ticker *clock.Ticker //For processing time only
  71. signal chan *Signal
  72. store api.Store
  73. ctx api.StreamContext
  74. activated bool
  75. }
  76. func NewCoordinator(ruleId string, sources []StreamTask, operators []NonSourceTask, sinks []SinkTask, qos api.Qos, store api.Store, interval int, ctx api.StreamContext) *Coordinator {
  77. logger := ctx.GetLogger()
  78. logger.Infof("create new coordinator for rule %s", ruleId)
  79. signal := make(chan *Signal, 1024)
  80. var allResponders, sourceResponders []Responder
  81. for _, r := range sources {
  82. r.SetQos(qos)
  83. re := NewResponderExecutor(signal, r)
  84. allResponders = append(allResponders, re)
  85. sourceResponders = append(sourceResponders, re)
  86. }
  87. for _, r := range operators {
  88. r.SetQos(qos)
  89. re := NewResponderExecutor(signal, r)
  90. handler := createBarrierHandler(re, r.GetInputCount(), qos)
  91. r.SetBarrierHandler(handler)
  92. allResponders = append(allResponders, re)
  93. }
  94. for _, r := range sinks {
  95. r.SetQos(qos)
  96. re := NewResponderExecutor(signal, r)
  97. handler := NewBarrierTracker(re, r.GetInputCount())
  98. r.SetBarrierHandler(handler)
  99. allResponders = append(allResponders, re)
  100. }
  101. //5 minutes by default
  102. if interval <= 0 {
  103. interval = 300000
  104. }
  105. return &Coordinator{
  106. tasksToTrigger: sourceResponders,
  107. tasksToWaitFor: allResponders,
  108. sinkTasks: sinks,
  109. pendingCheckpoints: new(sync.Map),
  110. completedCheckpoints: &checkpointStore{
  111. maxNum: 3,
  112. },
  113. ruleId: ruleId,
  114. signal: signal,
  115. baseInterval: interval,
  116. store: store,
  117. ctx: ctx,
  118. cleanThreshold: 100,
  119. }
  120. }
  121. func createBarrierHandler(re Responder, inputCount int, qos api.Qos) BarrierHandler {
  122. if qos == api.AtLeastOnce {
  123. return NewBarrierTracker(re, inputCount)
  124. } else if qos == api.ExactlyOnce {
  125. return NewBarrierAligner(re, inputCount)
  126. } else {
  127. return nil
  128. }
  129. }
  130. func (c *Coordinator) Activate() error {
  131. logger := c.ctx.GetLogger()
  132. logger.Infof("Start checkpoint coordinator for rule %s at %d", c.ruleId, conf.GetNowInMilli())
  133. if c.ticker != nil {
  134. c.ticker.Stop()
  135. }
  136. c.ticker = conf.GetTicker(c.baseInterval)
  137. tc := c.ticker.C
  138. go func() {
  139. c.activated = true
  140. toBeClean := 0
  141. for {
  142. select {
  143. case n := <-tc:
  144. //trigger checkpoint
  145. //TODO pose max attempt and min pause check for consequent pendingCheckpoints
  146. // TODO Check if all tasks are running
  147. //Create a pending checkpoint
  148. checkpointId := cast.TimeToUnixMilli(n)
  149. checkpoint := newPendingCheckpoint(checkpointId, c.tasksToWaitFor)
  150. logger.Debugf("Create checkpoint %d", checkpointId)
  151. c.pendingCheckpoints.Store(checkpointId, checkpoint)
  152. //Let the sources send out a barrier
  153. for _, r := range c.tasksToTrigger {
  154. go func(t Responder) {
  155. if err := t.TriggerCheckpoint(checkpointId); err != nil {
  156. logger.Infof("Fail to trigger checkpoint for source %s with error %v, cancel it", t.GetName(), err)
  157. c.cancel(checkpointId)
  158. }
  159. }(r)
  160. }
  161. toBeClean++
  162. if toBeClean >= c.cleanThreshold {
  163. c.store.Clean()
  164. toBeClean = 0
  165. }
  166. case s := <-c.signal:
  167. switch s.Message {
  168. case STOP:
  169. logger.Debug("Stop checkpoint scheduler")
  170. if c.ticker != nil {
  171. c.ticker.Stop()
  172. }
  173. return
  174. case ACK:
  175. logger.Debugf("Receive ack from %s for checkpoint %d", s.OpId, s.CheckpointId)
  176. if cp, ok := c.pendingCheckpoints.Load(s.CheckpointId); ok {
  177. checkpoint := cp.(*pendingCheckpoint)
  178. checkpoint.ack(s.OpId)
  179. if checkpoint.isFullyAck() {
  180. c.complete(s.CheckpointId)
  181. }
  182. } else {
  183. logger.Debugf("Receive ack from %s for non existing checkpoint %d", s.OpId, s.CheckpointId)
  184. }
  185. case DEC:
  186. logger.Debugf("Receive dec from %s for checkpoint %d, cancel it", s.OpId, s.CheckpointId)
  187. c.cancel(s.CheckpointId)
  188. }
  189. case <-c.ctx.Done():
  190. logger.Infoln("Cancelling coordinator....")
  191. if c.ticker != nil {
  192. c.ticker.Stop()
  193. logger.Infoln("Stop coordinator ticker")
  194. }
  195. return
  196. }
  197. }
  198. }()
  199. return nil
  200. }
  201. func (c *Coordinator) Deactivate() error {
  202. if c.ticker != nil {
  203. c.ticker.Stop()
  204. }
  205. c.signal <- &Signal{Message: STOP}
  206. return nil
  207. }
  208. func (c *Coordinator) cancel(checkpointId int64) {
  209. logger := c.ctx.GetLogger()
  210. if checkpoint, ok := c.pendingCheckpoints.Load(checkpointId); ok {
  211. c.pendingCheckpoints.Delete(checkpointId)
  212. checkpoint.(*pendingCheckpoint).dispose(true)
  213. } else {
  214. logger.Debugf("Cancel for non existing checkpoint %d. Just ignored", checkpointId)
  215. }
  216. }
  217. func (c *Coordinator) complete(checkpointId int64) {
  218. logger := c.ctx.GetLogger()
  219. if ccp, ok := c.pendingCheckpoints.Load(checkpointId); ok {
  220. err := c.store.SaveCheckpoint(checkpointId)
  221. if err != nil {
  222. logger.Infof("Cannot save checkpoint %d due to storage error: %v", checkpointId, err)
  223. //TODO handle checkpoint error
  224. return
  225. }
  226. //sink save cache
  227. for _, sink := range c.sinkTasks {
  228. sink.SaveCache()
  229. }
  230. c.completedCheckpoints.add(ccp.(*pendingCheckpoint).finalize())
  231. c.pendingCheckpoints.Delete(checkpointId)
  232. //Drop the previous pendingCheckpoints
  233. c.pendingCheckpoints.Range(func(a1 interface{}, a2 interface{}) bool {
  234. cid := a1.(int64)
  235. cp := a2.(*pendingCheckpoint)
  236. if cid < checkpointId {
  237. //TODO revisit how to abort a checkpoint, discard callback
  238. cp.isDiscarded = true
  239. c.pendingCheckpoints.Delete(cid)
  240. }
  241. return true
  242. })
  243. logger.Debugf("Totally complete checkpoint %d", checkpointId)
  244. } else {
  245. logger.Infof("Cannot find checkpoint %d to complete", checkpointId)
  246. }
  247. }
  248. //For testing
  249. func (c *Coordinator) GetCompleteCount() int {
  250. return len(c.completedCheckpoints.checkpoints)
  251. }
  252. func (c *Coordinator) GetLatest() int64 {
  253. return c.completedCheckpoints.getLatest().checkpointId
  254. }
  255. func (c *Coordinator) IsActivated() bool {
  256. return c.activated
  257. }