window_op.go 18 KB

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