window_op.go 17 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579
  1. // Copyright 2021-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 node
  15. import (
  16. "encoding/gob"
  17. "fmt"
  18. "math"
  19. "time"
  20. "github.com/benbjohnson/clock"
  21. "github.com/lf-edge/ekuiper/internal/conf"
  22. "github.com/lf-edge/ekuiper/internal/topo/node/metric"
  23. "github.com/lf-edge/ekuiper/internal/xsql"
  24. "github.com/lf-edge/ekuiper/pkg/api"
  25. "github.com/lf-edge/ekuiper/pkg/ast"
  26. "github.com/lf-edge/ekuiper/pkg/cast"
  27. "github.com/lf-edge/ekuiper/pkg/infra"
  28. )
  29. type WindowConfig struct {
  30. Type ast.WindowType
  31. Length int64
  32. Interval int64 // If the interval is not set, it is equals to Length
  33. RawInterval int
  34. TimeUnit ast.Token
  35. }
  36. type WindowOperator struct {
  37. *defaultSinkNode
  38. window *WindowConfig
  39. interval int64
  40. isEventTime bool
  41. trigger *EventTimeTrigger // For event time only
  42. statManager metric.StatManager
  43. ticker *clock.Ticker // For processing time only
  44. // states
  45. triggerTime int64
  46. msgCount int
  47. }
  48. const (
  49. WindowInputsKey = "$$windowInputs"
  50. TriggerTimeKey = "$$triggerTime"
  51. MsgCountKey = "$$msgCount"
  52. )
  53. func init() {
  54. gob.Register([]*xsql.Tuple{})
  55. gob.Register([]map[string]interface{}{})
  56. }
  57. func NewWindowOp(name string, w WindowConfig, options *api.RuleOption) (*WindowOperator, error) {
  58. o := new(WindowOperator)
  59. o.defaultSinkNode = &defaultSinkNode{
  60. input: make(chan interface{}, options.BufferLength),
  61. defaultNode: &defaultNode{
  62. outputs: make(map[string]chan<- interface{}),
  63. name: name,
  64. sendError: options.SendError,
  65. },
  66. }
  67. o.isEventTime = options.IsEventTime
  68. o.window = &w
  69. if o.window.Interval == 0 && o.window.Type == ast.COUNT_WINDOW {
  70. // if no interval value is set, and it's a count window, then set interval to length value.
  71. o.window.Interval = o.window.Length
  72. }
  73. if options.IsEventTime {
  74. // Create watermark generator
  75. if w, err := NewEventTimeTrigger(o.window); err != nil {
  76. return nil, err
  77. } else {
  78. o.trigger = w
  79. }
  80. }
  81. return o, nil
  82. }
  83. // Exec is the entry point for the executor
  84. // input: *xsql.Tuple from preprocessor
  85. // output: xsql.WindowTuplesSet
  86. func (o *WindowOperator) Exec(ctx api.StreamContext, errCh chan<- error) {
  87. o.ctx = ctx
  88. log := ctx.GetLogger()
  89. log.Debugf("Window operator %s is started", o.name)
  90. if len(o.outputs) <= 0 {
  91. infra.DrainError(ctx, fmt.Errorf("no output channel found"), errCh)
  92. return
  93. }
  94. stats, err := metric.NewStatManager(ctx, "op")
  95. if err != nil {
  96. infra.DrainError(ctx, err, errCh)
  97. return
  98. }
  99. o.statManager = stats
  100. var inputs []*xsql.Tuple
  101. if s, err := ctx.GetState(WindowInputsKey); err == nil {
  102. switch st := s.(type) {
  103. case []*xsql.Tuple:
  104. inputs = st
  105. log.Infof("Restore window state %+v", inputs)
  106. case nil:
  107. log.Debugf("Restore window state, nothing")
  108. default:
  109. infra.DrainError(ctx, fmt.Errorf("restore window state `inputs` %v error, invalid type", st), errCh)
  110. return
  111. }
  112. } else {
  113. log.Warnf("Restore window state fails: %s", err)
  114. }
  115. if !o.isEventTime {
  116. o.triggerTime = conf.GetNowInMilli()
  117. }
  118. if s, err := ctx.GetState(TriggerTimeKey); err == nil && s != nil {
  119. if si, ok := s.(int64); ok {
  120. o.triggerTime = si
  121. } else {
  122. errCh <- fmt.Errorf("restore window state `triggerTime` %v error, invalid type", s)
  123. }
  124. }
  125. o.msgCount = 0
  126. if s, err := ctx.GetState(MsgCountKey); err == nil && s != nil {
  127. if si, ok := s.(int); ok {
  128. o.msgCount = si
  129. } else {
  130. infra.DrainError(ctx, fmt.Errorf("restore window state `msgCount` %v error, invalid type", s), errCh)
  131. return
  132. }
  133. }
  134. log.Infof("Start with window state triggerTime: %d, msgCount: %d", o.triggerTime, o.msgCount)
  135. if o.isEventTime {
  136. go func() {
  137. err := infra.SafeRun(func() error {
  138. o.execEventWindow(ctx, inputs, errCh)
  139. return nil
  140. })
  141. if err != nil {
  142. infra.DrainError(ctx, err, errCh)
  143. }
  144. }()
  145. } else {
  146. go func() {
  147. err := infra.SafeRun(func() error {
  148. o.execProcessingWindow(ctx, inputs, errCh)
  149. return nil
  150. })
  151. if err != nil {
  152. infra.DrainError(ctx, err, errCh)
  153. }
  154. }()
  155. }
  156. }
  157. func getAlignedWindowEndTime(n time.Time, interval int, timeUnit ast.Token) time.Time {
  158. switch timeUnit {
  159. case ast.DD: // The interval * days starting today
  160. return time.Date(n.Year(), n.Month(), n.Day()+interval, 0, 0, 0, 0, n.Location())
  161. case ast.HH:
  162. gap := interval
  163. if n.Hour() > interval {
  164. gap = interval * (n.Hour()/interval + 1)
  165. }
  166. return time.Date(n.Year(), n.Month(), n.Day(), 0, 0, 0, 0, n.Location()).Add(time.Duration(gap) * time.Hour)
  167. case ast.MI:
  168. gap := interval
  169. if n.Minute() > interval {
  170. gap = interval * (n.Minute()/interval + 1)
  171. }
  172. return time.Date(n.Year(), n.Month(), n.Day(), n.Hour(), 0, 0, 0, n.Location()).Add(time.Duration(gap) * time.Minute)
  173. case ast.SS:
  174. gap := interval
  175. if n.Second() > interval {
  176. gap = interval * (n.Second()/interval + 1)
  177. }
  178. return time.Date(n.Year(), n.Month(), n.Day(), n.Hour(), n.Minute(), 0, 0, n.Location()).Add(time.Duration(gap) * time.Second)
  179. case ast.MS:
  180. milli := n.Nanosecond() / int(time.Millisecond)
  181. gap := interval
  182. if milli > interval {
  183. gap = interval * (milli/interval + 1)
  184. }
  185. return time.Date(n.Year(), n.Month(), n.Day(), n.Hour(), n.Minute(), n.Second(), 0, n.Location()).Add(time.Duration(gap) * time.Millisecond)
  186. default: // should never happen
  187. conf.Log.Errorf("invalid time unit %s", timeUnit)
  188. return n
  189. }
  190. }
  191. func getFirstTimer(ctx api.StreamContext, rawInerval int, timeUnit ast.Token) (int64, *clock.Timer) {
  192. next := getAlignedWindowEndTime(conf.GetNow(), rawInerval, timeUnit)
  193. ctx.GetLogger().Infof("align window timer to %v(%d)", next, next.UnixMilli())
  194. return next.UnixMilli(), conf.GetTimerByTime(next)
  195. }
  196. func (o *WindowOperator) execProcessingWindow(ctx api.StreamContext, inputs []*xsql.Tuple, errCh chan<- error) {
  197. log := ctx.GetLogger()
  198. var (
  199. timeoutTicker *clock.Timer
  200. // The first ticker to align the first window to the nature time
  201. firstTicker *clock.Timer
  202. firstTime int64
  203. nextTime int64
  204. firstC <-chan time.Time
  205. timeout <-chan time.Time
  206. c <-chan time.Time
  207. )
  208. switch o.window.Type {
  209. case ast.NOT_WINDOW:
  210. case ast.TUMBLING_WINDOW:
  211. firstTime, firstTicker = getFirstTimer(ctx, o.window.RawInterval, o.window.TimeUnit)
  212. o.interval = o.window.Length
  213. case ast.HOPPING_WINDOW:
  214. firstTime, firstTicker = getFirstTimer(ctx, o.window.RawInterval, o.window.TimeUnit)
  215. o.interval = o.window.Interval
  216. case ast.SLIDING_WINDOW:
  217. o.interval = o.window.Length
  218. case ast.SESSION_WINDOW:
  219. firstTime, firstTicker = getFirstTimer(ctx, o.window.RawInterval, o.window.TimeUnit)
  220. o.interval = o.window.Interval
  221. case ast.COUNT_WINDOW:
  222. o.interval = o.window.Interval
  223. }
  224. if firstTicker != nil {
  225. firstC = firstTicker.C
  226. // resume the previous window
  227. if len(inputs) > 0 && o.triggerTime > 0 {
  228. nextTick := conf.GetNowInMilli() + o.interval
  229. next := o.triggerTime
  230. switch o.window.Type {
  231. case ast.TUMBLING_WINDOW, ast.HOPPING_WINDOW:
  232. for {
  233. next = next + o.interval
  234. if next > nextTick {
  235. break
  236. }
  237. log.Debugf("triggered by restore inputs")
  238. inputs = o.scan(inputs, next, ctx)
  239. _ = ctx.PutState(WindowInputsKey, inputs)
  240. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  241. }
  242. case ast.SESSION_WINDOW:
  243. timeout, duration := o.window.Interval, o.window.Length
  244. for {
  245. et := inputs[0].Timestamp
  246. tick := et + (duration - et%duration)
  247. if et%duration == 0 {
  248. tick = et
  249. }
  250. var p int64
  251. for _, tuple := range inputs {
  252. var r int64 = math.MaxInt64
  253. if p > 0 {
  254. if tuple.Timestamp-p > timeout {
  255. r = p + timeout
  256. }
  257. }
  258. if tuple.Timestamp > tick {
  259. if tick-duration > et && tick < r {
  260. r = tick
  261. }
  262. tick += duration
  263. }
  264. if r < math.MaxInt64 {
  265. next = r
  266. break
  267. }
  268. p = tuple.Timestamp
  269. }
  270. if next > nextTick {
  271. break
  272. }
  273. log.Debugf("triggered by restore inputs")
  274. inputs = o.scan(inputs, next, ctx)
  275. _ = ctx.PutState(WindowInputsKey, inputs)
  276. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  277. }
  278. }
  279. }
  280. }
  281. for {
  282. select {
  283. // process incoming item
  284. case item, opened := <-o.input:
  285. processed := false
  286. if item, processed = o.preprocess(item); processed {
  287. break
  288. }
  289. o.statManager.IncTotalRecordsIn()
  290. o.statManager.ProcessTimeStart()
  291. if !opened {
  292. o.statManager.IncTotalExceptions("input channel closed")
  293. break
  294. }
  295. switch d := item.(type) {
  296. case error:
  297. _ = o.Broadcast(d)
  298. o.statManager.IncTotalExceptions(d.Error())
  299. case *xsql.Tuple:
  300. log.Debugf("Event window receive tuple %s", d.Message)
  301. inputs = append(inputs, d)
  302. switch o.window.Type {
  303. case ast.NOT_WINDOW:
  304. inputs = o.scan(inputs, d.Timestamp, ctx)
  305. case ast.SLIDING_WINDOW:
  306. inputs = o.scan(inputs, d.Timestamp, ctx)
  307. case ast.SESSION_WINDOW:
  308. if timeoutTicker != nil {
  309. timeoutTicker.Stop()
  310. timeoutTicker.Reset(time.Duration(o.window.Interval) * time.Millisecond)
  311. } else {
  312. timeoutTicker = conf.GetTimer(o.window.Interval)
  313. timeout = timeoutTicker.C
  314. o.triggerTime = d.Timestamp
  315. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  316. log.Debugf("Session window set start time %d", o.triggerTime)
  317. }
  318. case ast.COUNT_WINDOW:
  319. o.msgCount++
  320. log.Debugf(fmt.Sprintf("msgCount: %d", o.msgCount))
  321. if int64(o.msgCount)%o.window.Interval != 0 {
  322. continue
  323. }
  324. o.msgCount = 0
  325. if tl, er := NewTupleList(inputs, int(o.window.Length)); er != nil {
  326. log.Error(fmt.Sprintf("Found error when trying to "))
  327. infra.DrainError(ctx, er, errCh)
  328. return
  329. } else {
  330. log.Debugf(fmt.Sprintf("It has %d of count window.", tl.count()))
  331. triggerTime := conf.GetNowInMilli()
  332. for tl.hasMoreCountWindow() {
  333. tsets := tl.nextCountWindow()
  334. windowStart := triggerTime
  335. triggerTime = conf.GetNowInMilli()
  336. windowEnd := triggerTime
  337. tsets.WindowRange = xsql.NewWindowRange(windowStart, windowEnd)
  338. log.Debugf("Sent: %v", tsets)
  339. _ = o.Broadcast(tsets)
  340. o.statManager.IncTotalRecordsOut()
  341. }
  342. inputs = tl.getRestTuples()
  343. }
  344. }
  345. o.statManager.ProcessTimeEnd()
  346. o.statManager.SetBufferLength(int64(len(o.input)))
  347. _ = ctx.PutState(WindowInputsKey, inputs)
  348. _ = ctx.PutState(MsgCountKey, o.msgCount)
  349. default:
  350. e := fmt.Errorf("run Window error: expect xsql.Tuple type but got %[1]T(%[1]v)", d)
  351. _ = o.Broadcast(e)
  352. o.statManager.IncTotalExceptions(e.Error())
  353. }
  354. case now := <-firstC:
  355. log.Debugf("First tick at %v(%d), defined at %d", now, now.UnixMilli(), firstTime)
  356. switch o.window.Type {
  357. case ast.TUMBLING_WINDOW:
  358. o.ticker = conf.GetTicker(o.window.Length)
  359. case ast.HOPPING_WINDOW:
  360. o.ticker = conf.GetTicker(o.window.Interval)
  361. case ast.SESSION_WINDOW:
  362. o.ticker = conf.GetTicker(o.window.Length)
  363. }
  364. firstTicker = nil
  365. c = o.ticker.C
  366. inputs = o.tick(ctx, inputs, firstTime, log)
  367. if o.window.Type == ast.SESSION_WINDOW {
  368. nextTime = firstTime + o.window.Length
  369. } else {
  370. nextTime = firstTime + o.interval
  371. }
  372. case now := <-c:
  373. log.Debugf("Successive tick at %v(%d)", now, now.UnixMilli())
  374. inputs = o.tick(ctx, inputs, nextTime, log)
  375. if o.window.Type == ast.SESSION_WINDOW {
  376. nextTime += o.window.Length
  377. } else {
  378. nextTime += o.interval
  379. }
  380. case now := <-timeout:
  381. if len(inputs) > 0 {
  382. o.statManager.ProcessTimeStart()
  383. log.Debugf("triggered by timeout")
  384. inputs = o.scan(inputs, cast.TimeToUnixMilli(now), ctx)
  385. _ = inputs
  386. // expire all inputs, so that when timer scans there is no item
  387. inputs = make([]*xsql.Tuple, 0)
  388. o.statManager.ProcessTimeEnd()
  389. _ = ctx.PutState(WindowInputsKey, inputs)
  390. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  391. timeoutTicker = nil
  392. }
  393. // is cancelling
  394. case <-ctx.Done():
  395. log.Infoln("Cancelling window....")
  396. if o.ticker != nil {
  397. o.ticker.Stop()
  398. }
  399. return
  400. }
  401. }
  402. }
  403. func (o *WindowOperator) tick(ctx api.StreamContext, inputs []*xsql.Tuple, n int64, log api.Logger) []*xsql.Tuple {
  404. if o.window.Type == ast.SESSION_WINDOW {
  405. log.Debugf("session window update trigger time %d with %d inputs", n, len(inputs))
  406. if len(inputs) == 0 || n-o.window.Length < inputs[0].Timestamp {
  407. if len(inputs) > 0 {
  408. log.Debugf("session window last trigger time %d < first tuple %d", n-o.window.Length, inputs[0].Timestamp)
  409. }
  410. return inputs
  411. }
  412. }
  413. o.statManager.ProcessTimeStart()
  414. log.Debugf("triggered by ticker at %d", n)
  415. inputs = o.scan(inputs, n, ctx)
  416. o.statManager.ProcessTimeEnd()
  417. _ = ctx.PutState(WindowInputsKey, inputs)
  418. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  419. return inputs
  420. }
  421. type TupleList struct {
  422. tuples []*xsql.Tuple
  423. index int // Current index
  424. size int // The size for count window
  425. }
  426. func NewTupleList(tuples []*xsql.Tuple, windowSize int) (TupleList, error) {
  427. if windowSize <= 0 {
  428. return TupleList{}, fmt.Errorf("Window size should not be less than zero.")
  429. } else if tuples == nil || len(tuples) == 0 {
  430. return TupleList{}, fmt.Errorf("The tuples should not be nil or empty.")
  431. }
  432. tl := TupleList{tuples: tuples, size: windowSize}
  433. return tl, nil
  434. }
  435. func (tl *TupleList) hasMoreCountWindow() bool {
  436. if len(tl.tuples) < tl.size {
  437. return false
  438. }
  439. return tl.index == 0
  440. }
  441. func (tl *TupleList) count() int {
  442. if len(tl.tuples) < tl.size {
  443. return 0
  444. } else {
  445. return 1
  446. }
  447. }
  448. func (tl *TupleList) nextCountWindow() *xsql.WindowTuples {
  449. results := &xsql.WindowTuples{
  450. Content: make([]xsql.TupleRow, 0),
  451. }
  452. var subT []*xsql.Tuple
  453. subT = tl.tuples[len(tl.tuples)-tl.size : len(tl.tuples)]
  454. for _, tuple := range subT {
  455. results = results.AddTuple(tuple)
  456. }
  457. tl.index = tl.index + 1
  458. return results
  459. }
  460. func (tl *TupleList) getRestTuples() []*xsql.Tuple {
  461. if len(tl.tuples) < tl.size {
  462. return tl.tuples
  463. }
  464. return tl.tuples[len(tl.tuples)-tl.size+1:]
  465. }
  466. func (o *WindowOperator) scan(inputs []*xsql.Tuple, triggerTime int64, ctx api.StreamContext) []*xsql.Tuple {
  467. log := ctx.GetLogger()
  468. log.Debugf("window %s triggered at %s(%d)", o.name, time.Unix(triggerTime/1000, triggerTime%1000), triggerTime)
  469. var (
  470. delta int64
  471. windowStart int64
  472. windowEnd = triggerTime
  473. )
  474. if o.window.Type == ast.HOPPING_WINDOW || o.window.Type == ast.SLIDING_WINDOW {
  475. delta = o.calDelta(triggerTime, log)
  476. }
  477. results := &xsql.WindowTuples{
  478. Content: make([]xsql.TupleRow, 0),
  479. }
  480. i := 0
  481. // Sync table
  482. for _, tuple := range inputs {
  483. if o.window.Type == ast.HOPPING_WINDOW || o.window.Type == ast.SLIDING_WINDOW {
  484. diff := triggerTime - tuple.Timestamp
  485. if diff > o.window.Length+delta {
  486. log.Debugf("diff: %d, length: %d, delta: %d", diff, o.window.Length, delta)
  487. log.Debugf("tuple %s emitted at %d expired", tuple, tuple.Timestamp)
  488. // Expired tuple, remove it by not adding back to inputs
  489. continue
  490. }
  491. // Added back all inputs for non expired events
  492. inputs[i] = tuple
  493. i++
  494. } else if tuple.Timestamp > triggerTime {
  495. // Only added back early arrived events
  496. inputs[i] = tuple
  497. i++
  498. }
  499. if tuple.Timestamp <= triggerTime {
  500. results = results.AddTuple(tuple)
  501. }
  502. }
  503. switch o.window.Type {
  504. case ast.TUMBLING_WINDOW, ast.SESSION_WINDOW:
  505. windowStart = o.triggerTime
  506. case ast.HOPPING_WINDOW:
  507. windowStart = o.triggerTime - o.window.Interval
  508. case ast.SLIDING_WINDOW:
  509. windowStart = triggerTime - o.window.Length
  510. }
  511. if windowStart <= 0 {
  512. windowStart = windowEnd - o.window.Length
  513. }
  514. results.WindowRange = xsql.NewWindowRange(windowStart, windowEnd)
  515. log.Debugf("window %s triggered for %d tuples", o.name, len(inputs))
  516. log.Debugf("Sent: %v", results)
  517. _ = o.Broadcast(results)
  518. o.statManager.IncTotalRecordsOut()
  519. o.triggerTime = triggerTime
  520. log.Debugf("new trigger time %d", o.triggerTime)
  521. return inputs[:i]
  522. }
  523. func (o *WindowOperator) calDelta(triggerTime int64, log api.Logger) int64 {
  524. var delta int64
  525. lastTriggerTime := o.triggerTime
  526. if lastTriggerTime <= 0 {
  527. delta = math.MaxInt16 // max int, all events for the initial window
  528. } else {
  529. if !o.isEventTime && o.window.Interval > 0 {
  530. delta = triggerTime - lastTriggerTime - o.window.Interval
  531. if delta > 100 {
  532. log.Warnf("Possible long computation in window; Previous eviction time: %d, current eviction time: %d", lastTriggerTime, triggerTime)
  533. }
  534. } else {
  535. delta = 0
  536. }
  537. }
  538. return delta
  539. }
  540. func (o *WindowOperator) GetMetrics() [][]interface{} {
  541. if o.statManager != nil {
  542. return [][]interface{}{
  543. o.statManager.GetMetrics(),
  544. }
  545. } else {
  546. return nil
  547. }
  548. }