window_op.go 13 KB

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