ruleState.go 10 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420
  1. // Copyright 2022-2023 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 rule
  15. import (
  16. "context"
  17. "fmt"
  18. "math"
  19. "math/rand"
  20. "sync"
  21. "time"
  22. "github.com/robfig/cron/v3"
  23. "github.com/lf-edge/ekuiper/internal/conf"
  24. "github.com/lf-edge/ekuiper/internal/topo"
  25. "github.com/lf-edge/ekuiper/internal/topo/planner"
  26. "github.com/lf-edge/ekuiper/pkg/api"
  27. "github.com/lf-edge/ekuiper/pkg/infra"
  28. )
  29. type ActionSignal int
  30. const (
  31. ActionSignalStart ActionSignal = iota
  32. ActionSignalStop
  33. )
  34. type cronInterface interface {
  35. Start()
  36. AddFunc(spec string, cmd func()) (cron.EntryID, error)
  37. Remove(id cron.EntryID)
  38. }
  39. var backgroundCron cronInterface
  40. func init() {
  41. if !conf.IsTesting {
  42. backgroundCron = cron.New()
  43. } else {
  44. backgroundCron = &MockCron{}
  45. }
  46. backgroundCron.Start()
  47. }
  48. type cronStateCtx struct {
  49. cancel context.CancelFunc
  50. entryID cron.EntryID
  51. // isInSchedule indicates the current rule is in scheduled in backgroundCron
  52. isInSchedule bool
  53. startFailedCnt int
  54. }
  55. /*********
  56. * RuleState is created for each rule. Each ruleState runs two loops:
  57. * 1. action event loop to accept commands, such as start, stop, getStatus, delete
  58. * 2. topo running loop
  59. * Both loops need to access the status, so lock is needed
  60. */
  61. type RuleState struct {
  62. // Constant, never change. Channel to send signals to manage connection retry. When deleting the rule, close it.
  63. RuleId string
  64. ActionCh chan ActionSignal
  65. // Nearly constant, only change when update the rule
  66. Rule *api.Rule
  67. // States, create through rule in each rule start
  68. Topology *topo.Topo
  69. // 0 stop, 1 start, -1 delete, changed in actions
  70. triggered int
  71. // temporary storage for topo graph to make sure even rule close, the graph is still available
  72. topoGraph *api.PrintableTopo
  73. sync.RWMutex
  74. cronState cronStateCtx
  75. }
  76. // NewRuleState Create and initialize a rule state.
  77. // Errors are possible during plan the topo.
  78. // If error happens return immediately without add it to the registry
  79. func NewRuleState(rule *api.Rule) (*RuleState, error) {
  80. rs := &RuleState{
  81. RuleId: rule.Id,
  82. Rule: rule,
  83. ActionCh: make(chan ActionSignal),
  84. }
  85. rs.run()
  86. if tp, err := planner.Plan(rule); err != nil {
  87. return rs, err
  88. } else {
  89. rs.Topology = tp
  90. return rs, nil
  91. }
  92. }
  93. // UpdateTopo update the rule and the topology AND restart the topology
  94. // Do not need to call restart after update
  95. func (rs *RuleState) UpdateTopo(rule *api.Rule) error {
  96. if tp, err := planner.Plan(rule); err != nil {
  97. return err
  98. } else {
  99. rs.Lock()
  100. defer rs.Unlock()
  101. // Update rule
  102. rs.Rule = rule
  103. rs.topoGraph = nil
  104. // Stop the old topo
  105. if rs.triggered == 1 {
  106. rs.Topology.Cancel()
  107. rs.ActionCh <- ActionSignalStop
  108. // wait a little to make sure the old topo is stopped
  109. time.Sleep(1 * time.Millisecond)
  110. }
  111. // Update the topo and start
  112. rs.Topology = tp
  113. rs.triggered = 1
  114. rs.ActionCh <- ActionSignalStart
  115. return nil
  116. }
  117. }
  118. // Run start to run the two loops, do not access any changeable states
  119. func (rs *RuleState) run() {
  120. var (
  121. ctx context.Context
  122. cancel context.CancelFunc
  123. )
  124. // action loop, once start never end until the rule is deleted
  125. go func() {
  126. conf.Log.Infof("Start rulestate %s", rs.RuleId)
  127. for {
  128. s, opened := <-rs.ActionCh
  129. if !opened {
  130. conf.Log.Infof("Stop rulestate %s", rs.RuleId)
  131. if cancel != nil {
  132. cancel()
  133. }
  134. return
  135. }
  136. switch s {
  137. case ActionSignalStart:
  138. if ctx != nil {
  139. conf.Log.Warnf("rule %s is already started", rs.RuleId)
  140. } else {
  141. ctx, cancel = context.WithCancel(context.Background())
  142. go rs.runTopo(ctx)
  143. }
  144. case ActionSignalStop:
  145. // Stop the running loop
  146. if cancel != nil {
  147. cancel()
  148. ctx = nil
  149. cancel = nil
  150. } else {
  151. conf.Log.Warnf("rule %s is already stopped", rs.RuleId)
  152. }
  153. }
  154. }
  155. }()
  156. }
  157. func (rs *RuleState) runTopo(ctx context.Context) {
  158. // Load the changeable states once
  159. rs.Lock()
  160. tp := rs.Topology
  161. option := rs.Rule.Options.Restart
  162. rs.Unlock()
  163. if tp == nil {
  164. conf.Log.Warnf("rule %s is not initialized or just stopped", rs.RuleId)
  165. return
  166. }
  167. err := infra.SafeRun(func() error {
  168. count := 0
  169. d := option.Delay
  170. var er error
  171. ticker := time.NewTicker(time.Duration(d) * time.Millisecond)
  172. defer ticker.Stop()
  173. for {
  174. select {
  175. case e := <-tp.Open():
  176. er = e
  177. if er != nil { // Only restart rule for errors
  178. tp.GetContext().SetError(er)
  179. conf.Log.Errorf("closing rule %s for error: %v", rs.RuleId, er)
  180. tp.Cancel()
  181. } else { // exit normally
  182. return nil
  183. }
  184. }
  185. if count < option.Attempts {
  186. if d > option.MaxDelay {
  187. d = option.MaxDelay
  188. }
  189. if option.JitterFactor > 0 {
  190. d = int(math.Round(float64(d) * ((rand.Float64()*2-1)*0.1 + 1)))
  191. conf.Log.Infof("Rule %s will restart with jitterred delay %d", rs.RuleId, d)
  192. } else {
  193. conf.Log.Infof("Rule %s will restart with delay %d", rs.RuleId, d)
  194. }
  195. // retry after delay
  196. select {
  197. case <-ticker.C:
  198. break
  199. case <-ctx.Done():
  200. conf.Log.Errorf("stop rule %s retry as cancelled", rs.RuleId)
  201. return nil
  202. }
  203. count++
  204. if option.Multiplier > 0 {
  205. d = option.Delay * int(math.Pow(option.Multiplier, float64(count)))
  206. }
  207. } else {
  208. return er
  209. }
  210. }
  211. })
  212. if err != nil { // Exit after retries
  213. rs.Lock()
  214. // The only change the state by error
  215. if rs.triggered != -1 {
  216. rs.triggered = 0
  217. if rs.Topology != nil {
  218. rs.topoGraph = rs.Topology.GetTopo()
  219. }
  220. rs.ActionCh <- ActionSignalStop
  221. }
  222. rs.Unlock()
  223. }
  224. }
  225. // The action functions are state machine.
  226. func (rs *RuleState) Start() error {
  227. rs.Lock()
  228. defer rs.Unlock()
  229. if rs.triggered == -1 {
  230. return fmt.Errorf("rule %s is already deleted", rs.RuleId)
  231. }
  232. if rs.Rule.IsScheduleRule() {
  233. return rs.startScheduleRule()
  234. }
  235. return rs.start()
  236. }
  237. // startScheduleRule will register the job in the backgroundCron to run.
  238. // Job will do following 2 things:
  239. // 1. start the rule in cron if else the job is already stopped
  240. // 2. after the rule started, start an extract goroutine to stop the rule after specific duration
  241. func (rs *RuleState) startScheduleRule() error {
  242. if rs.cronState.isInSchedule {
  243. return fmt.Errorf("rule %s is already in schedule", rs.RuleId)
  244. }
  245. d, err := time.ParseDuration(rs.Rule.Options.Duration)
  246. if err != nil {
  247. return err
  248. }
  249. var cronCtx context.Context
  250. cronCtx, rs.cronState.cancel = context.WithCancel(context.Background())
  251. entryID, err := backgroundCron.AddFunc(rs.Rule.Options.Cron, func() {
  252. if err := func() error {
  253. rs.Lock()
  254. defer rs.Unlock()
  255. return rs.start()
  256. }(); err != nil {
  257. rs.Lock()
  258. rs.cronState.startFailedCnt++
  259. rs.Unlock()
  260. conf.Log.Errorf(err.Error())
  261. return
  262. }
  263. after := time.After(d)
  264. go func(ctx context.Context) {
  265. select {
  266. case <-after:
  267. rs.Lock()
  268. defer rs.Unlock()
  269. if err := rs.stop(); err != nil {
  270. conf.Log.Errorf("close rule %s failed, err: %v", rs.RuleId, err)
  271. }
  272. return
  273. case <-cronCtx.Done():
  274. return
  275. }
  276. }(cronCtx)
  277. })
  278. if err != nil {
  279. return err
  280. }
  281. rs.cronState.isInSchedule = true
  282. rs.cronState.entryID = entryID
  283. return nil
  284. }
  285. func (rs *RuleState) start() error {
  286. if rs.triggered != 1 {
  287. // If the rule has been stopped due to error, the topology is not nil
  288. if rs.Topology != nil {
  289. rs.Topology.Cancel()
  290. }
  291. if tp, err := planner.Plan(rs.Rule); err != nil {
  292. return err
  293. } else {
  294. rs.Topology = tp
  295. }
  296. rs.triggered = 1
  297. }
  298. rs.ActionCh <- ActionSignalStart
  299. return nil
  300. }
  301. // Stop remove the Topology
  302. func (rs *RuleState) Stop() error {
  303. rs.Lock()
  304. defer rs.Unlock()
  305. if rs.Rule.IsScheduleRule() && rs.cronState.isInSchedule {
  306. rs.cronState.isInSchedule = false
  307. if rs.cronState.cancel != nil {
  308. rs.cronState.cancel()
  309. }
  310. rs.cronState.startFailedCnt = 0
  311. backgroundCron.Remove(rs.cronState.entryID)
  312. }
  313. return rs.stop()
  314. }
  315. func (rs *RuleState) stop() error {
  316. if rs.triggered == -1 {
  317. return fmt.Errorf("rule %s is already deleted", rs.RuleId)
  318. }
  319. rs.triggered = 0
  320. if rs.Topology != nil {
  321. rs.Topology.Cancel()
  322. }
  323. rs.ActionCh <- ActionSignalStop
  324. return nil
  325. }
  326. func (rs *RuleState) Close() error {
  327. rs.Lock()
  328. defer rs.Unlock()
  329. if rs.Topology != nil {
  330. rs.Topology.RemoveMetrics()
  331. }
  332. if rs.triggered == 1 && rs.Topology != nil {
  333. rs.Topology.Cancel()
  334. }
  335. rs.triggered = -1
  336. if rs.Rule.IsScheduleRule() && rs.cronState.isInSchedule {
  337. rs.cronState.isInSchedule = false
  338. if rs.cronState.cancel != nil {
  339. rs.cronState.cancel()
  340. }
  341. rs.cronState.startFailedCnt = 0
  342. backgroundCron.Remove(rs.cronState.entryID)
  343. }
  344. close(rs.ActionCh)
  345. return nil
  346. }
  347. func (rs *RuleState) GetState() (string, error) {
  348. rs.RLock()
  349. defer rs.RUnlock()
  350. result := ""
  351. if rs.Topology == nil {
  352. result = "Stopped: fail to create the topo."
  353. } else {
  354. c := (*rs.Topology).GetContext()
  355. if c != nil {
  356. err := c.Err()
  357. switch err {
  358. case nil:
  359. result = "Running"
  360. case context.Canceled:
  361. if rs.Rule.IsScheduleRule() && rs.cronState.isInSchedule {
  362. result = "Stopped: waiting for next schedule."
  363. } else {
  364. result = "Stopped: canceled manually."
  365. }
  366. case context.DeadlineExceeded:
  367. result = "Stopped: deadline exceed."
  368. default:
  369. result = fmt.Sprintf("Stopped: %v.", err)
  370. }
  371. } else {
  372. if rs.cronState.isInSchedule {
  373. result = "Stopped: waiting for next schedule."
  374. } else {
  375. result = "Stopped: canceled manually."
  376. }
  377. }
  378. }
  379. if rs.Rule.IsScheduleRule() && rs.cronState.startFailedCnt > 0 {
  380. result = result + fmt.Sprintf(" Start failed count: %v.", rs.cronState.startFailedCnt)
  381. }
  382. return result, nil
  383. }
  384. func (rs *RuleState) GetTopoGraph() *api.PrintableTopo {
  385. rs.RLock()
  386. defer rs.RUnlock()
  387. if rs.topoGraph != nil {
  388. return rs.topoGraph
  389. } else if rs.Topology != nil {
  390. return rs.Topology.GetTopo()
  391. } else {
  392. return nil
  393. }
  394. }