coordinator.go 7.8 KB

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