ruleState.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550
  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. "strings"
  21. "sync"
  22. "time"
  23. "github.com/robfig/cron/v3"
  24. "github.com/lf-edge/ekuiper/internal/conf"
  25. "github.com/lf-edge/ekuiper/internal/topo"
  26. "github.com/lf-edge/ekuiper/internal/topo/planner"
  27. "github.com/lf-edge/ekuiper/pkg/api"
  28. "github.com/lf-edge/ekuiper/pkg/infra"
  29. "github.com/lf-edge/ekuiper/pkg/schedule"
  30. )
  31. const (
  32. RuleStarted = "Running"
  33. RuleStopped = "Stopped: canceled manually."
  34. RuleTerminated = "Stopped: schedule terminated."
  35. RuleWait = "Stopped: waiting for next schedule."
  36. )
  37. type ActionSignal int
  38. const (
  39. ActionSignalStart ActionSignal = iota
  40. ActionSignalStop
  41. )
  42. type cronInterface interface {
  43. Start()
  44. AddFunc(spec string, cmd func()) (cron.EntryID, error)
  45. Remove(id cron.EntryID)
  46. }
  47. var backgroundCron cronInterface
  48. func init() {
  49. if !conf.IsTesting {
  50. backgroundCron = cron.New()
  51. } else {
  52. backgroundCron = &MockCron{}
  53. }
  54. backgroundCron.Start()
  55. }
  56. type cronStateCtx struct {
  57. cancel context.CancelFunc
  58. entryID cron.EntryID
  59. // isInSchedule indicates the current rule is in scheduled in backgroundCron
  60. isInSchedule bool
  61. startFailedCnt int
  62. // only used for test
  63. cron string
  64. duration string
  65. }
  66. /*********
  67. * RuleState is created for each rule. Each ruleState runs two loops:
  68. * 1. action event loop to accept commands, such as start, stop, getStatus, delete
  69. * 2. topo running loop
  70. * Both loops need to access the status, so lock is needed
  71. */
  72. type RuleState struct {
  73. // Constant, never change. Channel to send signals to manage connection retry. When deleting the rule, close it.
  74. RuleId string
  75. ActionCh chan ActionSignal
  76. // Nearly constant, only change when update the rule
  77. Rule *api.Rule
  78. // States, create through rule in each rule start
  79. Topology *topo.Topo
  80. // 0 stop, 1 running, -1 delete, 2 internal stop, changed in actions
  81. triggered int
  82. // temporary storage for topo graph to make sure even rule close, the graph is still available
  83. topoGraph *api.PrintableTopo
  84. sync.RWMutex
  85. cronState cronStateCtx
  86. }
  87. // NewRuleState Create and initialize a rule state.
  88. // Errors are possible during plan the topo.
  89. // If error happens return immediately without add it to the registry
  90. func NewRuleState(rule *api.Rule) (*RuleState, error) {
  91. rs := &RuleState{
  92. RuleId: rule.Id,
  93. Rule: rule,
  94. ActionCh: make(chan ActionSignal),
  95. }
  96. rs.run()
  97. if tp, err := planner.Plan(rule); err != nil {
  98. return rs, err
  99. } else {
  100. rs.Topology = tp
  101. return rs, nil
  102. }
  103. }
  104. // UpdateTopo update the rule and the topology AND restart the topology
  105. // Do not need to call restart after update
  106. func (rs *RuleState) UpdateTopo(rule *api.Rule) error {
  107. if _, err := planner.Plan(rule); err != nil {
  108. return err
  109. }
  110. if err := rs.Stop(); err != nil {
  111. return err
  112. }
  113. time.Sleep(1 * time.Millisecond)
  114. rs.Rule = rule
  115. // If not triggered, just ignore start the rule
  116. if rule.Triggered {
  117. if err := rs.Start(); err != nil {
  118. return err
  119. }
  120. }
  121. return nil
  122. }
  123. // Run start to run the two loops, do not access any changeable states
  124. func (rs *RuleState) run() {
  125. var (
  126. ctx context.Context
  127. cancel context.CancelFunc
  128. )
  129. // action loop, once start never end until the rule is deleted
  130. go func() {
  131. conf.Log.Infof("Start rulestate %s", rs.RuleId)
  132. for {
  133. s, opened := <-rs.ActionCh
  134. if !opened {
  135. conf.Log.Infof("Stop rulestate %s", rs.RuleId)
  136. if cancel != nil {
  137. cancel()
  138. }
  139. return
  140. }
  141. switch s {
  142. case ActionSignalStart:
  143. if ctx != nil {
  144. conf.Log.Warnf("rule %s is already started", rs.RuleId)
  145. } else {
  146. ctx, cancel = context.WithCancel(context.Background())
  147. go rs.runTopo(ctx)
  148. }
  149. case ActionSignalStop:
  150. // Stop the running loop
  151. if cancel != nil {
  152. cancel()
  153. ctx = nil
  154. cancel = nil
  155. } else {
  156. conf.Log.Warnf("rule %s is already stopped", rs.RuleId)
  157. }
  158. }
  159. }
  160. }()
  161. }
  162. func (rs *RuleState) runTopo(ctx context.Context) {
  163. // Load the changeable states once
  164. rs.Lock()
  165. tp := rs.Topology
  166. option := rs.Rule.Options.Restart
  167. rs.Unlock()
  168. if tp == nil {
  169. conf.Log.Warnf("rule %s is not initialized or just stopped", rs.RuleId)
  170. return
  171. }
  172. err := infra.SafeRun(func() error {
  173. count := 0
  174. d := option.Delay
  175. var er error
  176. ticker := time.NewTicker(time.Duration(d) * time.Millisecond)
  177. defer ticker.Stop()
  178. for {
  179. select {
  180. case e := <-tp.Open():
  181. er = e
  182. if er != nil { // Only restart rule for errors
  183. tp.GetContext().SetError(er)
  184. conf.Log.Errorf("closing rule %s for error: %v", rs.RuleId, er)
  185. tp.Cancel()
  186. } else { // exit normally
  187. return nil
  188. }
  189. }
  190. if count < option.Attempts {
  191. if d > option.MaxDelay {
  192. d = option.MaxDelay
  193. }
  194. if option.JitterFactor > 0 {
  195. d = int(math.Round(float64(d) * ((rand.Float64()*2-1)*0.1 + 1)))
  196. conf.Log.Infof("Rule %s will restart with jitterred delay %d", rs.RuleId, d)
  197. } else {
  198. conf.Log.Infof("Rule %s will restart with delay %d", rs.RuleId, d)
  199. }
  200. // retry after delay
  201. select {
  202. case <-ticker.C:
  203. break
  204. case <-ctx.Done():
  205. conf.Log.Errorf("stop rule %s retry as cancelled", rs.RuleId)
  206. return nil
  207. }
  208. count++
  209. if option.Multiplier > 0 {
  210. d = option.Delay * int(math.Pow(option.Multiplier, float64(count)))
  211. }
  212. } else {
  213. return er
  214. }
  215. }
  216. })
  217. if err != nil { // Exit after retries
  218. rs.Lock()
  219. // The only change the state by error
  220. if rs.triggered != -1 {
  221. rs.triggered = 0
  222. if rs.Topology != nil {
  223. rs.topoGraph = rs.Topology.GetTopo()
  224. }
  225. rs.ActionCh <- ActionSignalStop
  226. }
  227. rs.Unlock()
  228. }
  229. }
  230. // The action functions are state machine.
  231. func (rs *RuleState) Start() error {
  232. rs.Lock()
  233. defer rs.Unlock()
  234. if rs.triggered == -1 {
  235. return fmt.Errorf("rule %s is already deleted", rs.RuleId)
  236. }
  237. if rs.Rule.IsLongRunningScheduleRule() {
  238. isIn, err := schedule.IsInScheduleRanges(conf.GetNow(), rs.Rule.Options.CronDatetimeRange)
  239. if err != nil {
  240. return err
  241. }
  242. // When rule is created, we need to check its schedule range before start it.
  243. if !isIn {
  244. rs.triggered = 2
  245. return nil
  246. }
  247. }
  248. if rs.Rule.IsScheduleRule() {
  249. return rs.startScheduleRule()
  250. }
  251. return rs.start()
  252. }
  253. // startScheduleRule will register the job in the backgroundCron to run.
  254. // Job will do following 2 things:
  255. // 1. start the rule in cron if else the job is already stopped
  256. // 2. after the rule started, start an extract goroutine to stop the rule after specific duration
  257. func (rs *RuleState) startScheduleRule() error {
  258. if rs.cronState.isInSchedule {
  259. return fmt.Errorf("rule %s is already in schedule", rs.RuleId)
  260. }
  261. d, err := time.ParseDuration(rs.Rule.Options.Duration)
  262. if err != nil {
  263. return err
  264. }
  265. var cronCtx context.Context
  266. cronCtx, rs.cronState.cancel = context.WithCancel(context.Background())
  267. now := conf.GetNow()
  268. isInRunningSchedule, remainedDuration, err := rs.isInRunningSchedule(now, d)
  269. if err != nil {
  270. return err
  271. }
  272. if isInRunningSchedule {
  273. if err := rs.runScheduleRule(); err != nil {
  274. return err
  275. }
  276. rs.stopAfterDuration(remainedDuration, cronCtx)
  277. }
  278. entryID, err := backgroundCron.AddFunc(rs.Rule.Options.Cron, func() {
  279. var started bool
  280. var err error
  281. if started, err = func() (bool, error) {
  282. switch backgroundCron.(type) {
  283. case *MockCron:
  284. // skip mutex if this is a unit test
  285. default:
  286. rs.Lock()
  287. defer rs.Unlock()
  288. }
  289. now := conf.GetNow()
  290. allowed, err := rs.isInAllowedTimeRange(now)
  291. if err != nil {
  292. return false, err
  293. }
  294. if !allowed {
  295. return false, nil
  296. }
  297. rs.cronState.cron = rs.Rule.Options.Cron
  298. rs.cronState.duration = rs.Rule.Options.Duration
  299. return true, rs.start()
  300. }(); err != nil {
  301. rs.Lock()
  302. rs.cronState.startFailedCnt++
  303. rs.Unlock()
  304. conf.Log.Errorf(err.Error())
  305. return
  306. }
  307. if started {
  308. rs.stopAfterDuration(d, cronCtx)
  309. }
  310. })
  311. if err != nil {
  312. return err
  313. }
  314. rs.cronState.isInSchedule = true
  315. rs.cronState.entryID = entryID
  316. return nil
  317. }
  318. func (rs *RuleState) runScheduleRule() error {
  319. rs.Lock()
  320. defer rs.Unlock()
  321. rs.cronState.cron = rs.Rule.Options.Cron
  322. rs.cronState.duration = rs.Rule.Options.Duration
  323. err := rs.start()
  324. if err != nil {
  325. return err
  326. }
  327. return nil
  328. }
  329. // stopAfterDuration only for schedule rule
  330. func (rs *RuleState) stopAfterDuration(d time.Duration, cronCtx context.Context) {
  331. after := time.After(d)
  332. go func(ctx context.Context) {
  333. select {
  334. case <-after:
  335. rs.Lock()
  336. defer rs.Unlock()
  337. if err := rs.internalStop(); err != nil {
  338. conf.Log.Errorf("close rule %s failed, err: %v", rs.RuleId, err)
  339. }
  340. return
  341. case <-cronCtx.Done():
  342. return
  343. }
  344. }(cronCtx)
  345. }
  346. func (rs *RuleState) start() error {
  347. if rs.triggered != 1 {
  348. // If the rule has been stopped due to error, the topology is not nil
  349. if rs.Topology != nil {
  350. rs.Topology.Cancel()
  351. }
  352. if tp, err := planner.Plan(rs.Rule); err != nil {
  353. return err
  354. } else {
  355. rs.Topology = tp
  356. }
  357. rs.triggered = 1
  358. }
  359. if rs.Rule.IsScheduleRule() || rs.Rule.IsLongRunningScheduleRule() {
  360. conf.Log.Debugf("rule %v started", rs.RuleId)
  361. }
  362. rs.ActionCh <- ActionSignalStart
  363. return nil
  364. }
  365. // Stop remove the Topology
  366. func (rs *RuleState) Stop() error {
  367. rs.Lock()
  368. defer rs.Unlock()
  369. if rs.Rule.IsScheduleRule() || rs.Rule.IsLongRunningScheduleRule() {
  370. conf.Log.Debugf("rule %v manual stopped", rs.RuleId)
  371. }
  372. rs.stopScheduleRule()
  373. return rs.stop()
  374. }
  375. func (rs *RuleState) stopScheduleRule() {
  376. if rs.Rule.IsScheduleRule() && rs.cronState.isInSchedule {
  377. rs.cronState.isInSchedule = false
  378. if rs.cronState.cancel != nil {
  379. rs.cronState.cancel()
  380. }
  381. rs.cronState.startFailedCnt = 0
  382. backgroundCron.Remove(rs.cronState.entryID)
  383. }
  384. }
  385. func (rs *RuleState) stop() error {
  386. if rs.triggered == -1 {
  387. return fmt.Errorf("rule %s is already deleted", rs.RuleId)
  388. }
  389. rs.triggered = 0
  390. if rs.Topology != nil {
  391. rs.Topology.Cancel()
  392. }
  393. rs.ActionCh <- ActionSignalStop
  394. return nil
  395. }
  396. func (rs *RuleState) InternalStop() error {
  397. rs.Lock()
  398. defer rs.Unlock()
  399. if !rs.Rule.IsLongRunningScheduleRule() {
  400. err := fmt.Errorf("rule %v isn't allowed to execute Internal stop as it's not long running schedule rule", rs.RuleId)
  401. conf.Log.Errorf(err.Error())
  402. return err
  403. } else {
  404. conf.Log.Debugf("rule %v internal stopped", rs.RuleId)
  405. }
  406. return rs.internalStop()
  407. }
  408. func (rs *RuleState) internalStop() error {
  409. if rs.triggered == -1 {
  410. return fmt.Errorf("rule %s is already deleted", rs.RuleId)
  411. }
  412. rs.triggered = 2
  413. if rs.Topology != nil {
  414. rs.Topology.Cancel()
  415. }
  416. rs.ActionCh <- ActionSignalStop
  417. return nil
  418. }
  419. func (rs *RuleState) Close() error {
  420. rs.Lock()
  421. defer rs.Unlock()
  422. if rs.Topology != nil {
  423. rs.Topology.RemoveMetrics()
  424. }
  425. if rs.triggered == 1 && rs.Topology != nil {
  426. rs.Topology.Cancel()
  427. }
  428. rs.triggered = -1
  429. rs.stopScheduleRule()
  430. close(rs.ActionCh)
  431. return nil
  432. }
  433. func (rs *RuleState) GetState() (string, error) {
  434. rs.RLock()
  435. defer rs.RUnlock()
  436. result := ""
  437. if rs.Topology == nil {
  438. result = "Stopped: fail to create the topo."
  439. } else {
  440. c := (*rs.Topology).GetContext()
  441. if c != nil {
  442. err := c.Err()
  443. switch err {
  444. case nil:
  445. result = RuleStarted
  446. case context.Canceled:
  447. result = rs.getStoppedRuleState()
  448. case context.DeadlineExceeded:
  449. result = "Stopped: deadline exceed."
  450. default:
  451. result = fmt.Sprintf("Stopped: %v.", err)
  452. }
  453. } else {
  454. result = rs.getStoppedRuleState()
  455. }
  456. }
  457. if rs.Rule.IsScheduleRule() && rs.cronState.startFailedCnt > 0 {
  458. result = result + fmt.Sprintf(" Start failed count: %v.", rs.cronState.startFailedCnt)
  459. }
  460. return result, nil
  461. }
  462. func (rs *RuleState) getStoppedRuleState() (result string) {
  463. if schedule.IsAfterTimeRanges(conf.GetNow(), rs.Rule.Options.CronDatetimeRange) {
  464. result = RuleTerminated
  465. } else if rs.cronState.isInSchedule {
  466. result = RuleWait
  467. } else if rs.triggered == 0 || rs.triggered == -1 {
  468. result = RuleStopped
  469. } else if rs.triggered == 2 {
  470. result = RuleWait
  471. }
  472. return result
  473. }
  474. func (rs *RuleState) GetTopoGraph() *api.PrintableTopo {
  475. rs.RLock()
  476. defer rs.RUnlock()
  477. if rs.topoGraph != nil {
  478. return rs.topoGraph
  479. } else if rs.Topology != nil {
  480. return rs.Topology.GetTopo()
  481. } else {
  482. return nil
  483. }
  484. }
  485. func (rs *RuleState) isInRunningSchedule(now time.Time, d time.Duration) (bool, time.Duration, error) {
  486. allowed, err := rs.isInAllowedTimeRange(now)
  487. if err != nil {
  488. return false, 0, err
  489. }
  490. if !allowed {
  491. return false, 0, nil
  492. }
  493. cronExpr := rs.Rule.Options.Cron
  494. if strings.HasPrefix(cronExpr, "mock") {
  495. return false, 0, nil
  496. }
  497. return schedule.IsInRunningSchedule(cronExpr, now, d)
  498. }
  499. func (rs *RuleState) isInAllowedTimeRange(now time.Time) (bool, error) {
  500. allowed := true
  501. var err error
  502. for _, timeRange := range rs.Rule.Options.CronDatetimeRange {
  503. allowed, err = schedule.IsInScheduleRange(now, timeRange.Begin, timeRange.End)
  504. if err != nil {
  505. return false, err
  506. }
  507. if allowed {
  508. break
  509. }
  510. }
  511. return allowed, nil
  512. }