coordinator.go 7.7 KB

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