window_op.go 16 KB

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