coordinator.go 7.1 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263
  1. package checkpoints
  2. import (
  3. "github.com/benbjohnson/clock"
  4. "github.com/emqx/kuiper/common"
  5. "github.com/emqx/kuiper/xstream/api"
  6. "sync"
  7. "time"
  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(releaseState 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. pendingCheckpoints *sync.Map
  64. completedCheckpoints *checkpointStore
  65. ruleId string
  66. baseInterval int
  67. timeout int
  68. advanceToEndOfEventTime bool
  69. ticker *clock.Ticker //For processing time only
  70. signal chan *Signal
  71. store api.Store
  72. ctx api.StreamContext
  73. }
  74. func NewCoordinator(ruleId string, sources []StreamTask, operators []NonSourceTask, sinks []NonSourceTask, qos api.Qos, store api.Store, interval int, ctx api.StreamContext) *Coordinator {
  75. signal := make(chan *Signal, 1024)
  76. var allResponders, sourceResponders []Responder
  77. for _, r := range sources {
  78. r.SetQos(qos)
  79. re := NewResponderExecutor(signal, r)
  80. allResponders = append(allResponders, re)
  81. sourceResponders = append(sourceResponders, re)
  82. }
  83. for _, r := range operators {
  84. r.SetQos(qos)
  85. re := NewResponderExecutor(signal, r)
  86. handler := createBarrierHandler(re, r.GetInputCount(), qos)
  87. r.SetBarrierHandler(handler)
  88. allResponders = append(allResponders, re)
  89. }
  90. for _, r := range sinks {
  91. r.SetQos(qos)
  92. re := NewResponderExecutor(signal, r)
  93. handler := NewBarrierTracker(re, r.GetInputCount())
  94. r.SetBarrierHandler(handler)
  95. allResponders = append(allResponders, re)
  96. }
  97. //5 minutes by default
  98. if interval <= 0 {
  99. interval = 5000
  100. }
  101. return &Coordinator{
  102. tasksToTrigger: sourceResponders,
  103. tasksToWaitFor: allResponders,
  104. pendingCheckpoints: new(sync.Map),
  105. completedCheckpoints: &checkpointStore{
  106. maxNum: 3,
  107. },
  108. ruleId: ruleId,
  109. signal: signal,
  110. baseInterval: interval,
  111. timeout: 200000,
  112. store: store,
  113. ctx: ctx,
  114. }
  115. }
  116. func createBarrierHandler(re Responder, inputCount int, qos api.Qos) BarrierHandler {
  117. if qos == api.AtLeastOnce {
  118. return NewBarrierTracker(re, inputCount)
  119. } else if qos == api.ExactlyOnce {
  120. return NewBarrierAligner(re, inputCount)
  121. } else {
  122. return nil
  123. }
  124. }
  125. func (c *Coordinator) Activate() error {
  126. logger := c.ctx.GetLogger()
  127. logger.Infoln("Start checkpoint coordinator for rule %s", c.ruleId)
  128. if c.ticker != nil {
  129. c.ticker.Stop()
  130. }
  131. c.ticker = common.GetTicker(c.baseInterval)
  132. tc := c.ticker.C
  133. go func() {
  134. for {
  135. select {
  136. case <-tc:
  137. //trigger checkpoint
  138. //TODO pose max attempt and min pause check for consequent pendingCheckpoints
  139. // TODO Check if all tasks are running
  140. //Create a pending checkpoint
  141. checkpointId := common.GetNowInMilli()
  142. checkpoint := newPendingCheckpoint(checkpointId, c.tasksToWaitFor)
  143. logger.Debugf("Create checkpoint %d", checkpointId)
  144. c.pendingCheckpoints.Store(checkpointId, checkpoint)
  145. //Let the sources send out a barrier
  146. for _, r := range c.tasksToTrigger {
  147. go func(t Responder) {
  148. if err := t.TriggerCheckpoint(checkpointId); err != nil {
  149. logger.Infof("Fail to trigger checkpoint for source %s with error %v", t.GetName(), err)
  150. c.cancel(checkpointId)
  151. } else {
  152. time.Sleep(time.Duration(c.timeout) * time.Microsecond)
  153. c.cancel(checkpointId)
  154. }
  155. }(r)
  156. }
  157. case s := <-c.signal:
  158. switch s.Message {
  159. case STOP:
  160. logger.Debug("Stop checkpoint scheduler")
  161. if c.ticker != nil {
  162. c.ticker.Stop()
  163. }
  164. return
  165. case ACK:
  166. logger.Debugf("Receive ack from %s for checkpoint %d", s.OpId, s.CheckpointId)
  167. if cp, ok := c.pendingCheckpoints.Load(s.CheckpointId); ok {
  168. checkpoint := cp.(*pendingCheckpoint)
  169. checkpoint.ack(s.OpId)
  170. if checkpoint.isFullyAck() {
  171. c.complete(s.CheckpointId)
  172. }
  173. } else {
  174. logger.Debugf("Receive ack from %s for non existing checkpoint %d", s.OpId, s.CheckpointId)
  175. }
  176. case DEC:
  177. logger.Debugf("Receive dec from %s for checkpoint %d", s.OpId, s.CheckpointId)
  178. c.cancel(s.CheckpointId)
  179. }
  180. case <-c.ctx.Done():
  181. logger.Infoln("Cancelling coordinator....")
  182. if c.ticker != nil {
  183. c.ticker.Stop()
  184. }
  185. return
  186. }
  187. }
  188. }()
  189. return nil
  190. }
  191. func (c *Coordinator) Deactivate() error {
  192. if c.ticker != nil {
  193. c.ticker.Stop()
  194. }
  195. c.signal <- &Signal{Message: STOP}
  196. return nil
  197. }
  198. func (c *Coordinator) cancel(checkpointId int64) {
  199. logger := c.ctx.GetLogger()
  200. if checkpoint, ok := c.pendingCheckpoints.Load(checkpointId); ok {
  201. c.pendingCheckpoints.Delete(checkpointId)
  202. checkpoint.(*pendingCheckpoint).dispose(true)
  203. } else {
  204. logger.Debugf("Cancel for non existing checkpoint %d. Just ignored", checkpointId)
  205. }
  206. }
  207. func (c *Coordinator) complete(checkpointId int64) {
  208. logger := c.ctx.GetLogger()
  209. if ccp, ok := c.pendingCheckpoints.Load(checkpointId); ok {
  210. err := c.store.SaveCheckpoint(checkpointId)
  211. if err != nil {
  212. logger.Infof("Cannot save checkpoint %d due to storage error: %v", checkpointId, err)
  213. //TODO handle checkpoint error
  214. return
  215. }
  216. c.completedCheckpoints.add(ccp.(*pendingCheckpoint).finalize())
  217. c.pendingCheckpoints.Delete(checkpointId)
  218. //Drop the previous pendingCheckpoints
  219. c.pendingCheckpoints.Range(func(a1 interface{}, a2 interface{}) bool {
  220. cid := a1.(int64)
  221. cp := a2.(*pendingCheckpoint)
  222. if cid < checkpointId {
  223. //TODO revisit how to abort a checkpoint, discard callback
  224. cp.isDiscarded = true
  225. c.pendingCheckpoints.Delete(cid)
  226. }
  227. return true
  228. })
  229. logger.Debugf("Totally complete checkpoint %d", checkpointId)
  230. } else {
  231. logger.Infof("Cannot find checkpoint %d to complete", checkpointId)
  232. }
  233. }
  234. //For testing
  235. func (c *Coordinator) GetCompleteCount() int {
  236. return len(c.completedCheckpoints.checkpoints)
  237. }
  238. func (c *Coordinator) GetLatest() int64 {
  239. return c.completedCheckpoints.getLatest().checkpointId
  240. }