coordinator.go 8.1 KB

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