watermark.go 6.8 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250
  1. package nodes
  2. import (
  3. "context"
  4. "fmt"
  5. "github.com/emqx/kuiper/xsql"
  6. "github.com/emqx/kuiper/xstream/api"
  7. "math"
  8. "sort"
  9. )
  10. type WatermarkTuple struct {
  11. Timestamp int64
  12. }
  13. func (t *WatermarkTuple) GetTimestamp() int64 {
  14. return t.Timestamp
  15. }
  16. func (t *WatermarkTuple) IsWatermark() bool {
  17. return true
  18. }
  19. const WATERMARK_KEY = "$$wartermark"
  20. type WatermarkGenerator struct {
  21. inputTopics []string
  22. topicToTs map[string]int64
  23. window *WindowConfig
  24. lateTolerance int64
  25. interval int
  26. //ticker *clock.Ticker
  27. stream chan<- interface{}
  28. //state
  29. lastWatermarkTs int64
  30. }
  31. func NewWatermarkGenerator(window *WindowConfig, l int64, s []string, stream chan<- interface{}) (*WatermarkGenerator, error) {
  32. w := &WatermarkGenerator{
  33. window: window,
  34. topicToTs: make(map[string]int64),
  35. lateTolerance: l,
  36. inputTopics: s,
  37. stream: stream,
  38. }
  39. switch window.Type {
  40. case xsql.NOT_WINDOW:
  41. case xsql.TUMBLING_WINDOW:
  42. w.interval = window.Length
  43. case xsql.HOPPING_WINDOW:
  44. w.interval = window.Interval
  45. case xsql.SLIDING_WINDOW:
  46. w.interval = window.Length
  47. case xsql.SESSION_WINDOW:
  48. //Use timeout to update watermark
  49. w.interval = window.Interval
  50. default:
  51. return nil, fmt.Errorf("unsupported window type %d", window.Type)
  52. }
  53. return w, nil
  54. }
  55. func (w *WatermarkGenerator) track(s string, ts int64, ctx api.StreamContext) bool {
  56. log := ctx.GetLogger()
  57. log.Debugf("watermark generator track event from topic %s at %d", s, ts)
  58. currentVal, ok := w.topicToTs[s]
  59. if !ok || ts > currentVal {
  60. w.topicToTs[s] = ts
  61. }
  62. r := ts >= w.lastWatermarkTs
  63. if r {
  64. w.trigger(ctx)
  65. }
  66. return r
  67. }
  68. func (w *WatermarkGenerator) trigger(ctx api.StreamContext) {
  69. log := ctx.GetLogger()
  70. watermark := w.computeWatermarkTs(ctx)
  71. log.Debugf("compute watermark event at %d with last %d", watermark, w.lastWatermarkTs)
  72. if watermark > w.lastWatermarkTs {
  73. t := &WatermarkTuple{Timestamp: watermark}
  74. select {
  75. case w.stream <- t:
  76. default: //TODO need to set buffer
  77. }
  78. w.lastWatermarkTs = watermark
  79. ctx.PutState(WATERMARK_KEY, w.lastWatermarkTs)
  80. log.Debugf("scan watermark event at %d", watermark)
  81. }
  82. }
  83. func (w *WatermarkGenerator) computeWatermarkTs(_ context.Context) int64 {
  84. var ts int64
  85. if len(w.topicToTs) >= len(w.inputTopics) {
  86. ts = math.MaxInt64
  87. for _, key := range w.inputTopics {
  88. if ts > w.topicToTs[key] {
  89. ts = w.topicToTs[key]
  90. }
  91. }
  92. }
  93. return ts - w.lateTolerance
  94. }
  95. //If window end cannot be determined yet, return max int64 so that it can be recalculated for the next watermark
  96. func (w *WatermarkGenerator) getNextWindow(inputs []*xsql.Tuple, current int64, watermark int64, triggered bool) int64 {
  97. switch w.window.Type {
  98. case xsql.TUMBLING_WINDOW, xsql.HOPPING_WINDOW:
  99. if triggered {
  100. return current + int64(w.interval)
  101. } else {
  102. interval := int64(w.interval)
  103. nextTs := getEarliestEventTs(inputs, current, watermark)
  104. if nextTs == math.MaxInt64 || nextTs%interval == 0 {
  105. return nextTs
  106. }
  107. return nextTs + (interval - nextTs%interval)
  108. }
  109. case xsql.SLIDING_WINDOW:
  110. nextTs := getEarliestEventTs(inputs, current, watermark)
  111. return nextTs
  112. case xsql.SESSION_WINDOW:
  113. if len(inputs) > 0 {
  114. timeout, duration := int64(w.window.Interval), int64(w.window.Length)
  115. sort.SliceStable(inputs, func(i, j int) bool {
  116. return inputs[i].Timestamp < inputs[j].Timestamp
  117. })
  118. et := inputs[0].Timestamp
  119. tick := et + (duration - et%duration)
  120. if et%duration == 0 {
  121. tick = et
  122. }
  123. var p int64
  124. for _, tuple := range inputs {
  125. var r int64 = math.MaxInt64
  126. if p > 0 {
  127. if tuple.Timestamp-p > timeout {
  128. r = p + timeout
  129. }
  130. }
  131. if tuple.Timestamp > tick {
  132. if tick-duration > et && tick < r {
  133. r = tick
  134. }
  135. tick += duration
  136. }
  137. if r < math.MaxInt64 {
  138. return r
  139. }
  140. p = tuple.Timestamp
  141. }
  142. }
  143. return math.MaxInt64
  144. default:
  145. return math.MaxInt64
  146. }
  147. }
  148. func (o *WindowOperator) execEventWindow(ctx api.StreamContext, inputs []*xsql.Tuple, errCh chan<- error) {
  149. log := ctx.GetLogger()
  150. var (
  151. triggered bool
  152. nextWindowEndTs int64
  153. prevWindowEndTs int64
  154. )
  155. o.watermarkGenerator.lastWatermarkTs = 0
  156. if s, err := ctx.GetState(WATERMARK_KEY); err == nil && s != nil {
  157. if si, ok := s.(int64); ok {
  158. o.watermarkGenerator.lastWatermarkTs = si
  159. } else {
  160. errCh <- fmt.Errorf("restore window state `lastWatermarkTs` %v error, invalid type", s)
  161. }
  162. }
  163. log.Infof("Start with window state lastWatermarkTs: %d", o.watermarkGenerator.lastWatermarkTs)
  164. for {
  165. select {
  166. // process incoming item
  167. case item, opened := <-o.input:
  168. processed := false
  169. if item, processed = o.preprocess(item); processed {
  170. break
  171. }
  172. o.statManager.ProcessTimeStart()
  173. if !opened {
  174. o.statManager.IncTotalExceptions()
  175. break
  176. }
  177. switch d := item.(type) {
  178. case error:
  179. o.statManager.IncTotalRecordsIn()
  180. o.Broadcast(d)
  181. o.statManager.IncTotalExceptions()
  182. case xsql.Event:
  183. if d.IsWatermark() {
  184. watermarkTs := d.GetTimestamp()
  185. windowEndTs := nextWindowEndTs
  186. //Session window needs a recalculation of window because its window end depends the inputs
  187. if windowEndTs == math.MaxInt64 || o.window.Type == xsql.SESSION_WINDOW || o.window.Type == xsql.SLIDING_WINDOW {
  188. windowEndTs = o.watermarkGenerator.getNextWindow(inputs, prevWindowEndTs, watermarkTs, triggered)
  189. }
  190. for windowEndTs <= watermarkTs && windowEndTs >= 0 {
  191. log.Debugf("Window end ts %d Watermark ts %d", windowEndTs, watermarkTs)
  192. log.Debugf("Current input count %d", len(inputs))
  193. //scan all events and find out the event in the current window
  194. inputs, triggered = o.scan(inputs, windowEndTs, ctx)
  195. prevWindowEndTs = windowEndTs
  196. windowEndTs = o.watermarkGenerator.getNextWindow(inputs, windowEndTs, watermarkTs, triggered)
  197. }
  198. nextWindowEndTs = windowEndTs
  199. log.Debugf("next window end %d", nextWindowEndTs)
  200. } else {
  201. o.statManager.IncTotalRecordsIn()
  202. tuple, ok := d.(*xsql.Tuple)
  203. if !ok {
  204. log.Debugf("receive non tuple element %v", d)
  205. }
  206. log.Debugf("event window receive tuple %s", tuple.Message)
  207. if o.watermarkGenerator.track(tuple.Emitter, d.GetTimestamp(), ctx) {
  208. inputs = append(inputs, tuple)
  209. }
  210. }
  211. o.statManager.ProcessTimeEnd()
  212. ctx.PutState(WINDOW_INPUTS_KEY, inputs)
  213. default:
  214. o.statManager.IncTotalRecordsIn()
  215. o.Broadcast(fmt.Errorf("run Window error: expect xsql.Event type but got %[1]T(%[1]v)", d))
  216. o.statManager.IncTotalExceptions()
  217. }
  218. // is cancelling
  219. case <-ctx.Done():
  220. log.Infoln("Cancelling window....")
  221. if o.ticker != nil {
  222. o.ticker.Stop()
  223. }
  224. return
  225. }
  226. }
  227. }
  228. func getEarliestEventTs(inputs []*xsql.Tuple, startTs int64, endTs int64) int64 {
  229. var minTs int64 = math.MaxInt64
  230. for _, t := range inputs {
  231. if t.Timestamp > startTs && t.Timestamp <= endTs && t.Timestamp < minTs {
  232. minTs = t.Timestamp
  233. }
  234. }
  235. return minTs
  236. }