window_op.go 19 KB

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