window_op.go 16 KB

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