window_op.go 19 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661
  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.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. case ast.SESSION_WINDOW:
  341. if timeoutTicker != nil {
  342. timeoutTicker.Stop()
  343. timeoutTicker.Reset(time.Duration(o.window.Interval) * time.Millisecond)
  344. } else {
  345. timeoutTicker = conf.GetTimer(o.window.Interval)
  346. timeout = timeoutTicker.C
  347. o.triggerTime = d.Timestamp
  348. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  349. log.Debugf("Session window set start time %d", o.triggerTime)
  350. }
  351. case ast.COUNT_WINDOW:
  352. o.msgCount++
  353. log.Debugf(fmt.Sprintf("msgCount: %d", o.msgCount))
  354. if int64(o.msgCount)%o.window.Interval != 0 {
  355. continue
  356. }
  357. o.msgCount = 0
  358. if tl, er := NewTupleList(inputs, int(o.window.Length)); er != nil {
  359. log.Error(fmt.Sprintf("Found error when trying to "))
  360. infra.DrainError(ctx, er, errCh)
  361. return
  362. } else {
  363. log.Debugf(fmt.Sprintf("It has %d of count window.", tl.count()))
  364. triggerTime := conf.GetNowInMilli()
  365. for tl.hasMoreCountWindow() {
  366. tsets := tl.nextCountWindow()
  367. windowStart := triggerTime
  368. triggerTime = conf.GetNowInMilli()
  369. windowEnd := triggerTime
  370. tsets.WindowRange = xsql.NewWindowRange(windowStart, windowEnd)
  371. log.Debugf("Sent: %v", tsets)
  372. _ = o.Broadcast(tsets)
  373. o.statManager.IncTotalRecordsOut()
  374. }
  375. inputs = tl.getRestTuples()
  376. }
  377. }
  378. o.statManager.ProcessTimeEnd()
  379. o.statManager.SetBufferLength(int64(len(o.input)))
  380. _ = ctx.PutState(WindowInputsKey, inputs)
  381. _ = ctx.PutState(MsgCountKey, o.msgCount)
  382. default:
  383. e := fmt.Errorf("run Window error: expect xsql.Tuple type but got %[1]T(%[1]v)", d)
  384. _ = o.Broadcast(e)
  385. o.statManager.IncTotalExceptions(e.Error())
  386. }
  387. case now := <-firstC:
  388. log.Debugf("First tick at %v(%d), defined at %d", now, now.UnixMilli(), firstTime)
  389. switch o.window.Type {
  390. case ast.TUMBLING_WINDOW:
  391. o.ticker = conf.GetTicker(o.window.Length)
  392. case ast.HOPPING_WINDOW:
  393. o.ticker = conf.GetTicker(o.window.Interval)
  394. case ast.SESSION_WINDOW:
  395. o.ticker = conf.GetTicker(o.window.Length)
  396. }
  397. firstTicker = nil
  398. c = o.ticker.C
  399. inputs = o.tick(ctx, inputs, firstTime, log)
  400. if o.window.Type == ast.SESSION_WINDOW {
  401. nextTime = firstTime + o.window.Length
  402. } else {
  403. nextTime = firstTime + o.interval
  404. }
  405. case now := <-c:
  406. log.Debugf("Successive tick at %v(%d)", now, now.UnixMilli())
  407. inputs = o.tick(ctx, inputs, nextTime, log)
  408. if o.window.Type == ast.SESSION_WINDOW {
  409. nextTime += o.window.Length
  410. } else {
  411. nextTime += o.interval
  412. }
  413. case now := <-timeout:
  414. if len(inputs) > 0 {
  415. o.statManager.ProcessTimeStart()
  416. log.Debugf("triggered by timeout")
  417. inputs = o.scan(inputs, cast.TimeToUnixMilli(now), ctx)
  418. _ = inputs
  419. // expire all inputs, so that when timer scans there is no item
  420. inputs = make([]*xsql.Tuple, 0)
  421. o.statManager.ProcessTimeEnd()
  422. _ = ctx.PutState(WindowInputsKey, inputs)
  423. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  424. timeoutTicker = nil
  425. }
  426. // is cancelling
  427. case <-ctx.Done():
  428. log.Infoln("Cancelling window....")
  429. if o.ticker != nil {
  430. o.ticker.Stop()
  431. }
  432. return
  433. }
  434. }
  435. }
  436. func (o *WindowOperator) tick(ctx api.StreamContext, inputs []*xsql.Tuple, n int64, log api.Logger) []*xsql.Tuple {
  437. if o.window.Type == ast.SESSION_WINDOW {
  438. log.Debugf("session window update trigger time %d with %d inputs", n, len(inputs))
  439. if len(inputs) == 0 || n-o.window.Length < inputs[0].Timestamp {
  440. if len(inputs) > 0 {
  441. log.Debugf("session window last trigger time %d < first tuple %d", n-o.window.Length, inputs[0].Timestamp)
  442. }
  443. return inputs
  444. }
  445. }
  446. o.statManager.ProcessTimeStart()
  447. log.Debugf("triggered by ticker at %d", n)
  448. inputs = o.scan(inputs, n, ctx)
  449. o.statManager.ProcessTimeEnd()
  450. _ = ctx.PutState(WindowInputsKey, inputs)
  451. _ = ctx.PutState(TriggerTimeKey, o.triggerTime)
  452. return inputs
  453. }
  454. type TupleList struct {
  455. tuples []*xsql.Tuple
  456. index int // Current index
  457. size int // The size for count window
  458. }
  459. func NewTupleList(tuples []*xsql.Tuple, windowSize int) (TupleList, error) {
  460. if windowSize <= 0 {
  461. return TupleList{}, fmt.Errorf("Window size should not be less than zero.")
  462. } else if tuples == nil || len(tuples) == 0 {
  463. return TupleList{}, fmt.Errorf("The tuples should not be nil or empty.")
  464. }
  465. tl := TupleList{tuples: tuples, size: windowSize}
  466. return tl, nil
  467. }
  468. func (tl *TupleList) hasMoreCountWindow() bool {
  469. if len(tl.tuples) < tl.size {
  470. return false
  471. }
  472. return tl.index == 0
  473. }
  474. func (tl *TupleList) count() int {
  475. if len(tl.tuples) < tl.size {
  476. return 0
  477. } else {
  478. return 1
  479. }
  480. }
  481. func (tl *TupleList) nextCountWindow() *xsql.WindowTuples {
  482. results := &xsql.WindowTuples{
  483. Content: make([]xsql.TupleRow, 0),
  484. }
  485. var subT []*xsql.Tuple
  486. subT = tl.tuples[len(tl.tuples)-tl.size : len(tl.tuples)]
  487. for _, tuple := range subT {
  488. results = results.AddTuple(tuple)
  489. }
  490. tl.index = tl.index + 1
  491. return results
  492. }
  493. func (tl *TupleList) getRestTuples() []*xsql.Tuple {
  494. if len(tl.tuples) < tl.size {
  495. return tl.tuples
  496. }
  497. return tl.tuples[len(tl.tuples)-tl.size+1:]
  498. }
  499. func (o *WindowOperator) isTimeRelatedWindow() bool {
  500. switch o.window.Type {
  501. case ast.SLIDING_WINDOW:
  502. return o.window.Delay > 0
  503. case ast.TUMBLING_WINDOW:
  504. return true
  505. case ast.HOPPING_WINDOW:
  506. return true
  507. case ast.SESSION_WINDOW:
  508. return true
  509. }
  510. return false
  511. }
  512. func (o *WindowOperator) scan(inputs []*xsql.Tuple, triggerTime int64, ctx api.StreamContext) []*xsql.Tuple {
  513. log := ctx.GetLogger()
  514. log.Debugf("window %s triggered at %s(%d)", o.name, time.Unix(triggerTime/1000, triggerTime%1000), triggerTime)
  515. var (
  516. delta int64
  517. windowStart int64
  518. windowEnd = triggerTime
  519. )
  520. if o.window.Type == ast.HOPPING_WINDOW || o.window.Type == ast.SLIDING_WINDOW {
  521. delta = o.calDelta(triggerTime, log)
  522. }
  523. results := &xsql.WindowTuples{
  524. Content: make([]xsql.TupleRow, 0),
  525. }
  526. i := 0
  527. length := o.window.Length + o.window.Delay
  528. // Sync table
  529. for _, tuple := range inputs {
  530. if o.window.Type == ast.HOPPING_WINDOW || o.window.Type == ast.SLIDING_WINDOW {
  531. diff := triggerTime - tuple.Timestamp
  532. if diff > length+delta {
  533. log.Debugf("diff: %d, length: %d, delta: %d", diff, length, delta)
  534. log.Debugf("tuple %s emitted at %d expired", tuple, tuple.Timestamp)
  535. // Expired tuple, remove it by not adding back to inputs
  536. continue
  537. }
  538. // Added back all inputs for non expired events
  539. inputs[i] = tuple
  540. i++
  541. } else if tuple.Timestamp > triggerTime {
  542. // Only added back early arrived events
  543. inputs[i] = tuple
  544. i++
  545. }
  546. if o.isTimeRelatedWindow() {
  547. if tuple.Timestamp < triggerTime {
  548. results = results.AddTuple(tuple)
  549. }
  550. } else {
  551. if tuple.Timestamp <= triggerTime {
  552. results = results.AddTuple(tuple)
  553. }
  554. }
  555. }
  556. switch o.window.Type {
  557. case ast.TUMBLING_WINDOW, ast.SESSION_WINDOW:
  558. windowStart = o.triggerTime
  559. case ast.HOPPING_WINDOW:
  560. windowStart = o.triggerTime - o.window.Interval
  561. case ast.SLIDING_WINDOW:
  562. windowStart = triggerTime - length
  563. }
  564. if windowStart <= 0 {
  565. windowStart = windowEnd - length
  566. }
  567. results.WindowRange = xsql.NewWindowRange(windowStart, windowEnd)
  568. log.Debugf("window %s triggered for %d tuples", o.name, len(inputs))
  569. log.Debugf("Sent: %v", results)
  570. _ = o.Broadcast(results)
  571. o.statManager.IncTotalRecordsOut()
  572. o.triggerTime = triggerTime
  573. log.Debugf("new trigger time %d", o.triggerTime)
  574. return inputs[:i]
  575. }
  576. func (o *WindowOperator) calDelta(triggerTime int64, log api.Logger) int64 {
  577. var delta int64
  578. lastTriggerTime := o.triggerTime
  579. if lastTriggerTime <= 0 {
  580. delta = math.MaxInt16 // max int, all events for the initial window
  581. } else {
  582. if !o.isEventTime && o.window.Interval > 0 {
  583. delta = triggerTime - lastTriggerTime - o.window.Interval
  584. if delta > 100 {
  585. log.Warnf("Possible long computation in window; Previous eviction time: %d, current eviction time: %d", lastTriggerTime, triggerTime)
  586. }
  587. } else {
  588. delta = 0
  589. }
  590. }
  591. return delta
  592. }
  593. func (o *WindowOperator) GetMetrics() [][]interface{} {
  594. if o.statManager != nil {
  595. return [][]interface{}{
  596. o.statManager.GetMetrics(),
  597. }
  598. } else {
  599. return nil
  600. }
  601. }
  602. func (o *WindowOperator) isMatchCondition(ctx api.StreamContext, d *xsql.Tuple) bool {
  603. if o.triggerCondition == nil || o.window.Type != ast.SLIDING_WINDOW {
  604. return true
  605. }
  606. log := ctx.GetLogger()
  607. fv, _ := xsql.NewFunctionValuersForOp(ctx)
  608. ve := &xsql.ValuerEval{Valuer: xsql.MultiValuer(d, fv)}
  609. result := ve.Eval(o.triggerCondition)
  610. // not match trigger condition
  611. if result == nil {
  612. return false
  613. }
  614. switch v := result.(type) {
  615. case error:
  616. log.Errorf("window %s trigger condition meet error: %v", o.name, v)
  617. return false
  618. case bool:
  619. // match trigger condition
  620. if v {
  621. for _, f := range o.stateFuncs {
  622. _ = ve.Eval(f)
  623. }
  624. }
  625. return v
  626. default:
  627. return false
  628. }
  629. }