window_op.go 18 KB

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