window_op.go 14 KB

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