window_op.go 13 KB

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