ruleState.go 13 KB

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