window_op.go 16 KB

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