ruleState.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558
  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. // only used for unit test
  124. var ignoreSignal = false
  125. // Run start to run the two loops, do not access any changeable states
  126. func (rs *RuleState) run() {
  127. var (
  128. ctx context.Context
  129. cancel context.CancelFunc
  130. )
  131. // action loop, once start never end until the rule is deleted
  132. go func() {
  133. conf.Log.Infof("Start rulestate %s", rs.RuleId)
  134. for {
  135. s, opened := <-rs.ActionCh
  136. if !opened {
  137. conf.Log.Infof("Stop rulestate %s", rs.RuleId)
  138. if cancel != nil {
  139. cancel()
  140. }
  141. return
  142. }
  143. if ignoreSignal {
  144. continue
  145. }
  146. switch s {
  147. case ActionSignalStart:
  148. if ctx != nil {
  149. conf.Log.Warnf("rule %s is already started", rs.RuleId)
  150. } else {
  151. ctx, cancel = context.WithCancel(context.Background())
  152. go rs.runTopo(ctx)
  153. }
  154. case ActionSignalStop:
  155. // Stop the running loop
  156. if cancel != nil {
  157. cancel()
  158. ctx = nil
  159. cancel = nil
  160. } else {
  161. conf.Log.Warnf("rule %s is already stopped", rs.RuleId)
  162. }
  163. }
  164. }
  165. }()
  166. }
  167. func (rs *RuleState) runTopo(ctx context.Context) {
  168. // Load the changeable states once
  169. rs.Lock()
  170. tp := rs.Topology
  171. option := rs.Rule.Options.Restart
  172. rs.Unlock()
  173. if tp == nil {
  174. conf.Log.Warnf("rule %s is not initialized or just stopped", rs.RuleId)
  175. return
  176. }
  177. err := infra.SafeRun(func() error {
  178. count := 0
  179. d := option.Delay
  180. var er error
  181. ticker := time.NewTicker(time.Duration(d) * time.Millisecond)
  182. defer ticker.Stop()
  183. for {
  184. select {
  185. case e := <-tp.Open():
  186. er = e
  187. if er != nil { // Only restart rule for errors
  188. tp.GetContext().SetError(er)
  189. conf.Log.Errorf("closing rule %s for error: %v", rs.RuleId, er)
  190. tp.Cancel()
  191. } else { // exit normally
  192. return nil
  193. }
  194. }
  195. if count < option.Attempts {
  196. if d > option.MaxDelay {
  197. d = option.MaxDelay
  198. }
  199. if option.JitterFactor > 0 {
  200. d = int(math.Round(float64(d) * ((rand.Float64()*2-1)*0.1 + 1)))
  201. conf.Log.Infof("Rule %s will restart with jitterred delay %d", rs.RuleId, d)
  202. } else {
  203. conf.Log.Infof("Rule %s will restart with delay %d", rs.RuleId, d)
  204. }
  205. // retry after delay
  206. select {
  207. case <-ticker.C:
  208. break
  209. case <-ctx.Done():
  210. conf.Log.Errorf("stop rule %s retry as cancelled", rs.RuleId)
  211. return nil
  212. }
  213. count++
  214. if option.Multiplier > 0 {
  215. d = option.Delay * int(math.Pow(option.Multiplier, float64(count)))
  216. }
  217. } else {
  218. return er
  219. }
  220. }
  221. })
  222. if err != nil { // Exit after retries
  223. rs.Lock()
  224. // The only change the state by error
  225. if rs.triggered != -1 {
  226. rs.triggered = 0
  227. if rs.Topology != nil {
  228. rs.topoGraph = rs.Topology.GetTopo()
  229. }
  230. rs.ActionCh <- ActionSignalStop
  231. }
  232. rs.Unlock()
  233. }
  234. }
  235. // The action functions are state machine.
  236. func (rs *RuleState) Start() error {
  237. rs.Lock()
  238. defer rs.Unlock()
  239. if rs.triggered == -1 {
  240. return fmt.Errorf("rule %s is already deleted", rs.RuleId)
  241. }
  242. if rs.Rule.IsLongRunningScheduleRule() {
  243. isIn, err := schedule.IsInScheduleRanges(conf.GetNow(), rs.Rule.Options.CronDatetimeRange)
  244. if err != nil {
  245. return err
  246. }
  247. // When rule is created, we need to check its schedule range before start it.
  248. if !isIn {
  249. rs.triggered = 2
  250. return nil
  251. }
  252. }
  253. if rs.Rule.IsScheduleRule() {
  254. return rs.startScheduleRule()
  255. }
  256. return rs.start()
  257. }
  258. // startScheduleRule will register the job in the backgroundCron to run.
  259. // Job will do following 2 things:
  260. // 1. start the rule in cron if else the job is already stopped
  261. // 2. after the rule started, start an extract goroutine to stop the rule after specific duration
  262. func (rs *RuleState) startScheduleRule() error {
  263. if rs.cronState.isInSchedule {
  264. return fmt.Errorf("rule %s is already in schedule", rs.RuleId)
  265. }
  266. d, err := time.ParseDuration(rs.Rule.Options.Duration)
  267. if err != nil {
  268. return err
  269. }
  270. var cronCtx context.Context
  271. cronCtx, rs.cronState.cancel = context.WithCancel(context.Background())
  272. now := conf.GetNow()
  273. isInRunningSchedule, remainedDuration, err := rs.isInRunningSchedule(now, d)
  274. if err != nil {
  275. return err
  276. }
  277. if isInRunningSchedule {
  278. if err := rs.runScheduleRule(); err != nil {
  279. return err
  280. }
  281. rs.stopAfterDuration(remainedDuration, cronCtx)
  282. }
  283. entryID, err := backgroundCron.AddFunc(rs.Rule.Options.Cron, func() {
  284. var started bool
  285. var err error
  286. if started, err = func() (bool, error) {
  287. switch backgroundCron.(type) {
  288. case *MockCron:
  289. // skip mutex if this is a unit test
  290. default:
  291. rs.Lock()
  292. defer rs.Unlock()
  293. }
  294. now := conf.GetNow()
  295. allowed, err := rs.isInAllowedTimeRange(now)
  296. if err != nil {
  297. return false, err
  298. }
  299. if !allowed {
  300. return false, nil
  301. }
  302. rs.cronState.cron = rs.Rule.Options.Cron
  303. rs.cronState.duration = rs.Rule.Options.Duration
  304. return true, rs.start()
  305. }(); err != nil {
  306. rs.Lock()
  307. rs.cronState.startFailedCnt++
  308. rs.Unlock()
  309. conf.Log.Errorf(err.Error())
  310. return
  311. }
  312. if started {
  313. rs.stopAfterDuration(d, cronCtx)
  314. }
  315. })
  316. if err != nil {
  317. return err
  318. }
  319. rs.cronState.isInSchedule = true
  320. rs.cronState.entryID = entryID
  321. return nil
  322. }
  323. func (rs *RuleState) runScheduleRule() error {
  324. rs.Lock()
  325. defer rs.Unlock()
  326. rs.cronState.cron = rs.Rule.Options.Cron
  327. rs.cronState.duration = rs.Rule.Options.Duration
  328. err := rs.start()
  329. if err != nil {
  330. return err
  331. }
  332. return nil
  333. }
  334. // stopAfterDuration only for schedule rule
  335. func (rs *RuleState) stopAfterDuration(d time.Duration, cronCtx context.Context) {
  336. after := time.After(d)
  337. go func(ctx context.Context) {
  338. select {
  339. case <-after:
  340. rs.Lock()
  341. defer rs.Unlock()
  342. if err := rs.internalStop(); err != nil {
  343. conf.Log.Errorf("close rule %s failed, err: %v", rs.RuleId, err)
  344. }
  345. return
  346. case <-cronCtx.Done():
  347. return
  348. }
  349. }(cronCtx)
  350. }
  351. func (rs *RuleState) start() error {
  352. if rs.triggered != 1 {
  353. // If the rule has been stopped due to error, the topology is not nil
  354. if rs.Topology != nil {
  355. rs.Topology.Cancel()
  356. }
  357. if tp, err := planner.Plan(rs.Rule); err != nil {
  358. return err
  359. } else {
  360. rs.Topology = tp
  361. }
  362. rs.triggered = 1
  363. }
  364. if rs.Rule.IsScheduleRule() || rs.Rule.IsLongRunningScheduleRule() {
  365. conf.Log.Debugf("rule %v started", rs.RuleId)
  366. }
  367. rs.ActionCh <- ActionSignalStart
  368. return nil
  369. }
  370. // Stop remove the Topology
  371. func (rs *RuleState) Stop() error {
  372. rs.Lock()
  373. defer rs.Unlock()
  374. if rs.Rule.IsScheduleRule() || rs.Rule.IsLongRunningScheduleRule() {
  375. conf.Log.Debugf("rule %v manual stopped", rs.RuleId)
  376. }
  377. rs.stopScheduleRule()
  378. return rs.stop()
  379. }
  380. func (rs *RuleState) stopScheduleRule() {
  381. if rs.Rule.IsScheduleRule() && rs.cronState.isInSchedule {
  382. rs.cronState.isInSchedule = false
  383. if rs.cronState.cancel != nil {
  384. rs.cronState.cancel()
  385. }
  386. rs.cronState.startFailedCnt = 0
  387. backgroundCron.Remove(rs.cronState.entryID)
  388. }
  389. }
  390. func (rs *RuleState) stop() error {
  391. if rs.triggered == -1 {
  392. return fmt.Errorf("rule %s is already deleted", rs.RuleId)
  393. }
  394. rs.triggered = 0
  395. if rs.Topology != nil {
  396. rs.Topology.Cancel()
  397. }
  398. rs.ActionCh <- ActionSignalStop
  399. return nil
  400. }
  401. func (rs *RuleState) InternalStop() error {
  402. rs.Lock()
  403. defer rs.Unlock()
  404. if !rs.Rule.IsLongRunningScheduleRule() {
  405. err := fmt.Errorf("rule %v isn't allowed to execute Internal stop as it's not long running schedule rule", rs.RuleId)
  406. conf.Log.Errorf(err.Error())
  407. return err
  408. } else {
  409. conf.Log.Debugf("rule %v internal stopped", rs.RuleId)
  410. }
  411. return rs.internalStop()
  412. }
  413. func (rs *RuleState) internalStop() error {
  414. if rs.triggered == -1 {
  415. return fmt.Errorf("rule %s is already deleted", rs.RuleId)
  416. }
  417. rs.triggered = 2
  418. if rs.Topology != nil {
  419. rs.Topology.Cancel()
  420. }
  421. rs.ActionCh <- ActionSignalStop
  422. return nil
  423. }
  424. func (rs *RuleState) Close() error {
  425. rs.Lock()
  426. defer rs.Unlock()
  427. if rs.Topology != nil {
  428. rs.Topology.RemoveMetrics()
  429. }
  430. if rs.triggered == 1 && rs.Topology != nil {
  431. rs.Topology.Cancel()
  432. }
  433. rs.triggered = -1
  434. rs.stopScheduleRule()
  435. close(rs.ActionCh)
  436. return nil
  437. }
  438. func (rs *RuleState) GetState() (string, error) {
  439. rs.RLock()
  440. defer rs.RUnlock()
  441. result := ""
  442. if rs.Topology == nil {
  443. result = "Stopped: fail to create the topo."
  444. } else {
  445. c := (*rs.Topology).GetContext()
  446. if c != nil {
  447. err := c.Err()
  448. switch err {
  449. case nil:
  450. result = RuleStarted
  451. case context.Canceled:
  452. result = rs.getStoppedRuleState()
  453. case context.DeadlineExceeded:
  454. result = "Stopped: deadline exceed."
  455. default:
  456. result = fmt.Sprintf("Stopped: %v.", err)
  457. }
  458. } else {
  459. result = rs.getStoppedRuleState()
  460. }
  461. }
  462. if rs.Rule.IsScheduleRule() && rs.cronState.startFailedCnt > 0 {
  463. result = result + fmt.Sprintf(" Start failed count: %v.", rs.cronState.startFailedCnt)
  464. }
  465. return result, nil
  466. }
  467. func (rs *RuleState) getStoppedRuleState() (result string) {
  468. if schedule.IsAfterTimeRanges(conf.GetNow(), rs.Rule.Options.CronDatetimeRange) {
  469. result = RuleTerminated
  470. } else if rs.cronState.isInSchedule {
  471. result = RuleWait
  472. } else if rs.triggered == 0 || rs.triggered == -1 {
  473. result = RuleStopped
  474. } else if rs.triggered == 2 {
  475. result = RuleWait
  476. }
  477. return result
  478. }
  479. func (rs *RuleState) GetTopoGraph() *api.PrintableTopo {
  480. rs.RLock()
  481. defer rs.RUnlock()
  482. if rs.topoGraph != nil {
  483. return rs.topoGraph
  484. } else if rs.Topology != nil {
  485. return rs.Topology.GetTopo()
  486. } else {
  487. return nil
  488. }
  489. }
  490. func (rs *RuleState) isInRunningSchedule(now time.Time, d time.Duration) (bool, time.Duration, error) {
  491. allowed, err := rs.isInAllowedTimeRange(now)
  492. if err != nil {
  493. return false, 0, err
  494. }
  495. if !allowed {
  496. return false, 0, nil
  497. }
  498. cronExpr := rs.Rule.Options.Cron
  499. if strings.HasPrefix(cronExpr, "mock") {
  500. return false, 0, nil
  501. }
  502. return schedule.IsInRunningSchedule(cronExpr, now, d)
  503. }
  504. func (rs *RuleState) isInAllowedTimeRange(now time.Time) (bool, error) {
  505. allowed := true
  506. var err error
  507. for _, timeRange := range rs.Rule.Options.CronDatetimeRange {
  508. allowed, err = schedule.IsInScheduleRange(now, timeRange.Begin, timeRange.End)
  509. if err != nil {
  510. return false, err
  511. }
  512. if allowed {
  513. break
  514. }
  515. }
  516. return allowed, nil
  517. }