window_op.go 19 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667
  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. o.statManagers = []metric.StatManager{stats}
  114. var inputs []*xsql.Tuple
  115. if s, err := ctx.GetState(WindowInputsKey); err == nil {
  116. switch st := s.(type) {
  117. case []*xsql.Tuple:
  118. inputs = st
  119. log.Infof("Restore window state %+v", inputs)
  120. case nil:
  121. log.Debugf("Restore window state, nothing")
  122. default:
  123. infra.DrainError(ctx, fmt.Errorf("restore window state `inputs` %v error, invalid type", st), errCh)
  124. return
  125. }
  126. } else {
  127. log.Warnf("Restore window state fails: %s", err)
  128. }
  129. if !o.isEventTime {
  130. o.triggerTime = conf.GetNowInMilli()
  131. }
  132. if s, err := ctx.GetState(TriggerTimeKey); err == nil && s != nil {
  133. if si, ok := s.(int64); ok {
  134. o.triggerTime = si
  135. } else {
  136. errCh <- fmt.Errorf("restore window state `triggerTime` %v error, invalid type", s)
  137. }
  138. }
  139. o.msgCount = 0
  140. if s, err := ctx.GetState(MsgCountKey); err == nil && s != nil {
  141. if si, ok := s.(int); ok {
  142. o.msgCount = si
  143. } else {
  144. infra.DrainError(ctx, fmt.Errorf("restore window state `msgCount` %v error, invalid type", s), errCh)
  145. return
  146. }
  147. }
  148. log.Infof("Start with window state triggerTime: %d, msgCount: %d", o.triggerTime, o.msgCount)
  149. if o.isEventTime {
  150. go func() {
  151. err := infra.SafeRun(func() error {
  152. o.execEventWindow(ctx, inputs, errCh)
  153. return nil
  154. })
  155. if err != nil {
  156. infra.DrainError(ctx, err, errCh)
  157. }
  158. }()
  159. } else {
  160. go func() {
  161. err := infra.SafeRun(func() error {
  162. o.execProcessingWindow(ctx, inputs, errCh)
  163. return nil
  164. })
  165. if err != nil {
  166. infra.DrainError(ctx, err, errCh)
  167. }
  168. }()
  169. }
  170. }
  171. func getAlignedWindowEndTime(n time.Time, interval int, timeUnit ast.Token) time.Time {
  172. switch timeUnit {
  173. case ast.DD: // The interval * days starting today
  174. return time.Date(n.Year(), n.Month(), n.Day()+interval, 0, 0, 0, 0, n.Location())
  175. case ast.HH:
  176. gap := interval
  177. if n.Hour() > interval {
  178. gap = interval * (n.Hour()/interval + 1)
  179. }
  180. return time.Date(n.Year(), n.Month(), n.Day(), 0, 0, 0, 0, n.Location()).Add(time.Duration(gap) * time.Hour)
  181. case ast.MI:
  182. gap := interval
  183. if n.Minute() > interval {
  184. gap = interval * (n.Minute()/interval + 1)
  185. }
  186. return time.Date(n.Year(), n.Month(), n.Day(), n.Hour(), 0, 0, 0, n.Location()).Add(time.Duration(gap) * time.Minute)
  187. case ast.SS:
  188. gap := interval
  189. if n.Second() > interval {
  190. gap = interval * (n.Second()/interval + 1)
  191. }
  192. return time.Date(n.Year(), n.Month(), n.Day(), n.Hour(), n.Minute(), 0, 0, n.Location()).Add(time.Duration(gap) * time.Second)
  193. case ast.MS:
  194. milli := n.Nanosecond() / int(time.Millisecond)
  195. gap := interval
  196. if milli > interval {
  197. gap = interval * (milli/interval + 1)
  198. }
  199. return time.Date(n.Year(), n.Month(), n.Day(), n.Hour(), n.Minute(), n.Second(), 0, n.Location()).Add(time.Duration(gap) * time.Millisecond)
  200. default: // should never happen
  201. conf.Log.Errorf("invalid time unit %s", timeUnit)
  202. return n
  203. }
  204. }
  205. func getFirstTimer(ctx api.StreamContext, rawInerval int, timeUnit ast.Token) (int64, *clock.Timer) {
  206. next := getAlignedWindowEndTime(conf.GetNow(), rawInerval, timeUnit)
  207. ctx.GetLogger().Infof("align window timer to %v(%d)", next, next.UnixMilli())
  208. return next.UnixMilli(), conf.GetTimerByTime(next)
  209. }
  210. func (o *WindowOperator) execProcessingWindow(ctx api.StreamContext, inputs []*xsql.Tuple, errCh chan<- error) {
  211. log := ctx.GetLogger()
  212. var (
  213. timeoutTicker *clock.Timer
  214. // The first ticker to align the first window to the nature time
  215. firstTicker *clock.Timer
  216. firstTime int64
  217. nextTime int64
  218. firstC <-chan time.Time
  219. timeout <-chan time.Time
  220. c <-chan time.Time
  221. )
  222. switch o.window.Type {
  223. case ast.NOT_WINDOW:
  224. case ast.TUMBLING_WINDOW:
  225. firstTime, firstTicker = getFirstTimer(ctx, o.window.RawInterval, o.window.TimeUnit)
  226. o.interval = o.window.Length
  227. case ast.HOPPING_WINDOW:
  228. firstTime, firstTicker = getFirstTimer(ctx, o.window.RawInterval, o.window.TimeUnit)
  229. o.interval = o.window.Interval
  230. case ast.SLIDING_WINDOW:
  231. o.interval = o.window.Length
  232. case ast.SESSION_WINDOW:
  233. firstTime, firstTicker = getFirstTimer(ctx, o.window.RawInterval, o.window.TimeUnit)
  234. o.interval = o.window.Interval
  235. case ast.COUNT_WINDOW:
  236. o.interval = o.window.Interval
  237. }
  238. if firstTicker != nil {
  239. firstC = firstTicker.C
  240. // resume the previous window
  241. if len(inputs) > 0 && o.triggerTime > 0 {
  242. nextTick := conf.GetNowInMilli() + o.interval
  243. next := o.triggerTime
  244. switch o.window.Type {
  245. case ast.TUMBLING_WINDOW, ast.HOPPING_WINDOW:
  246. for {
  247. next = next + o.interval
  248. if next > nextTick {
  249. break
  250. }
  251. log.Debugf("triggered by restore inputs")
  252. inputs = o.scan(inputs, next, ctx)
  253. _ = ctx.PutState(WindowInputsKey, inputs)
  254. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  255. }
  256. case ast.SESSION_WINDOW:
  257. timeout, duration := o.window.Interval, o.window.Length
  258. for {
  259. et := inputs[0].Timestamp
  260. tick := et + (duration - et%duration)
  261. if et%duration == 0 {
  262. tick = et
  263. }
  264. var p int64
  265. for _, tuple := range inputs {
  266. var r int64 = math.MaxInt64
  267. if p > 0 {
  268. if tuple.Timestamp-p > timeout {
  269. r = p + timeout
  270. }
  271. }
  272. if tuple.Timestamp > tick {
  273. if tick-duration > et && tick < r {
  274. r = tick
  275. }
  276. tick += duration
  277. }
  278. if r < math.MaxInt64 {
  279. next = r
  280. break
  281. }
  282. p = tuple.Timestamp
  283. }
  284. if next > nextTick {
  285. break
  286. }
  287. log.Debugf("triggered by restore inputs")
  288. inputs = o.scan(inputs, next, ctx)
  289. _ = ctx.PutState(WindowInputsKey, inputs)
  290. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  291. }
  292. }
  293. }
  294. }
  295. delayCh := make(chan int64, 100)
  296. for {
  297. select {
  298. case delayTS := <-delayCh:
  299. o.statManager.ProcessTimeStart()
  300. inputs = o.scan(inputs, delayTS, ctx)
  301. o.statManager.ProcessTimeEnd()
  302. o.statManager.SetBufferLength(int64(len(o.input)))
  303. _ = ctx.PutState(WindowInputsKey, inputs)
  304. _ = ctx.PutState(MsgCountKey, o.msgCount)
  305. // process incoming item
  306. case item, opened := <-o.input:
  307. processed := false
  308. if item, processed = o.preprocess(item); processed {
  309. break
  310. }
  311. o.statManager.IncTotalRecordsIn()
  312. o.statManager.ProcessTimeStart()
  313. if !opened {
  314. o.statManager.IncTotalExceptions("input channel closed")
  315. break
  316. }
  317. switch d := item.(type) {
  318. case error:
  319. _ = o.Broadcast(d)
  320. o.statManager.IncTotalExceptions(d.Error())
  321. case *xsql.Tuple:
  322. log.Debugf("Event window receive tuple %s", d.Message)
  323. inputs = append(inputs, d)
  324. switch o.window.Type {
  325. case ast.NOT_WINDOW:
  326. inputs = o.scan(inputs, d.Timestamp, ctx)
  327. case ast.SLIDING_WINDOW:
  328. if o.isMatchCondition(ctx, d) {
  329. if o.window.Delay > 0 {
  330. go func(ts int64) {
  331. after := time.After(time.Duration(o.window.Delay) * time.Millisecond)
  332. select {
  333. case <-after:
  334. delayCh <- ts
  335. }
  336. }(d.Timestamp + o.window.Delay)
  337. } else {
  338. inputs = o.scan(inputs, d.Timestamp, ctx)
  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) isTimeRelatedWindow() bool {
  501. switch o.window.Type {
  502. case ast.SLIDING_WINDOW:
  503. return o.window.Delay > 0
  504. case ast.TUMBLING_WINDOW:
  505. return true
  506. case ast.HOPPING_WINDOW:
  507. return true
  508. case ast.SESSION_WINDOW:
  509. return true
  510. }
  511. return false
  512. }
  513. func (o *WindowOperator) scan(inputs []*xsql.Tuple, triggerTime int64, ctx api.StreamContext) []*xsql.Tuple {
  514. log := ctx.GetLogger()
  515. log.Debugf("window %s triggered at %s(%d)", o.name, time.Unix(triggerTime/1000, triggerTime%1000), triggerTime)
  516. var (
  517. delta int64
  518. windowStart int64
  519. windowEnd = triggerTime
  520. )
  521. if o.window.Type == ast.HOPPING_WINDOW || o.window.Type == ast.SLIDING_WINDOW {
  522. delta = o.calDelta(triggerTime, log)
  523. }
  524. results := &xsql.WindowTuples{
  525. Content: make([]xsql.TupleRow, 0),
  526. }
  527. i := 0
  528. length := o.window.Length + o.window.Delay
  529. // Sync table
  530. for _, tuple := range inputs {
  531. if o.window.Type == ast.HOPPING_WINDOW || o.window.Type == ast.SLIDING_WINDOW {
  532. diff := triggerTime - tuple.Timestamp
  533. if diff > length+delta {
  534. log.Debugf("diff: %d, length: %d, delta: %d", diff, length, delta)
  535. log.Debugf("tuple %s emitted at %d expired", tuple, tuple.Timestamp)
  536. // Expired tuple, remove it by not adding back to inputs
  537. continue
  538. }
  539. // Added back all inputs for non expired events
  540. inputs[i] = tuple
  541. i++
  542. } else {
  543. // time-related window is left-closed,right-opened, so that we need keep the tuple if its timestamp >= trigger time
  544. if o.isTimeRelatedWindow() {
  545. if tuple.Timestamp >= triggerTime {
  546. // Only added back early arrived events
  547. inputs[i] = tuple
  548. i++
  549. }
  550. } else {
  551. if tuple.Timestamp > triggerTime {
  552. // Only added back early arrived events
  553. inputs[i] = tuple
  554. i++
  555. }
  556. }
  557. }
  558. if o.isTimeRelatedWindow() {
  559. if tuple.Timestamp < triggerTime {
  560. results = results.AddTuple(tuple)
  561. }
  562. } else {
  563. if tuple.Timestamp <= triggerTime {
  564. results = results.AddTuple(tuple)
  565. }
  566. }
  567. }
  568. switch o.window.Type {
  569. case ast.TUMBLING_WINDOW, ast.SESSION_WINDOW:
  570. windowStart = o.triggerTime
  571. case ast.HOPPING_WINDOW:
  572. windowStart = o.triggerTime - o.window.Interval
  573. case ast.SLIDING_WINDOW:
  574. windowStart = triggerTime - length
  575. }
  576. if windowStart <= 0 {
  577. windowStart = windowEnd - length
  578. }
  579. results.WindowRange = xsql.NewWindowRange(windowStart, windowEnd)
  580. log.Debugf("window %s triggered for %d tuples", o.name, len(inputs))
  581. log.Debugf("Sent: %v", results)
  582. _ = o.Broadcast(results)
  583. o.statManager.IncTotalRecordsOut()
  584. o.triggerTime = triggerTime
  585. log.Debugf("new trigger time %d", o.triggerTime)
  586. return inputs[:i]
  587. }
  588. func (o *WindowOperator) calDelta(triggerTime int64, log api.Logger) int64 {
  589. var delta int64
  590. lastTriggerTime := o.triggerTime
  591. if lastTriggerTime <= 0 {
  592. delta = math.MaxInt16 // max int, all events for the initial window
  593. } else {
  594. if !o.isEventTime && o.window.Interval > 0 {
  595. delta = triggerTime - lastTriggerTime - o.window.Interval
  596. if delta > 100 {
  597. log.Warnf("Possible long computation in window; Previous eviction time: %d, current eviction time: %d", lastTriggerTime, triggerTime)
  598. }
  599. } else {
  600. delta = 0
  601. }
  602. }
  603. return delta
  604. }
  605. func (o *WindowOperator) GetMetrics() [][]interface{} {
  606. return o.defaultNode.GetMetrics()
  607. }
  608. func (o *WindowOperator) isMatchCondition(ctx api.StreamContext, d *xsql.Tuple) bool {
  609. if o.triggerCondition == nil || o.window.Type != ast.SLIDING_WINDOW {
  610. return true
  611. }
  612. log := ctx.GetLogger()
  613. fv, _ := xsql.NewFunctionValuersForOp(ctx)
  614. ve := &xsql.ValuerEval{Valuer: xsql.MultiValuer(d, fv)}
  615. result := ve.Eval(o.triggerCondition)
  616. // not match trigger condition
  617. if result == nil {
  618. return false
  619. }
  620. switch v := result.(type) {
  621. case error:
  622. log.Errorf("window %s trigger condition meet error: %v", o.name, v)
  623. return false
  624. case bool:
  625. // match trigger condition
  626. if v {
  627. for _, f := range o.stateFuncs {
  628. _ = ve.Eval(f)
  629. }
  630. }
  631. return v
  632. default:
  633. return false
  634. }
  635. }