watermark.go 8.1 KB

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