streams.go 5.6 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218
  1. package xstream
  2. import (
  3. "context"
  4. "fmt"
  5. "github.com/emqx/kuiper/common"
  6. "github.com/emqx/kuiper/xstream/api"
  7. "github.com/emqx/kuiper/xstream/checkpoints"
  8. "github.com/emqx/kuiper/xstream/contexts"
  9. "github.com/emqx/kuiper/xstream/nodes"
  10. "github.com/emqx/kuiper/xstream/states"
  11. "strconv"
  12. )
  13. type PrintableTopo struct {
  14. Sources []string `json:"sources"`
  15. Edges map[string][]string `json:"edges"`
  16. }
  17. type TopologyNew struct {
  18. sources []nodes.DataSourceNode
  19. sinks []*nodes.SinkNode
  20. ctx api.StreamContext
  21. cancel context.CancelFunc
  22. drain chan error
  23. ops []nodes.OperatorNode
  24. name string
  25. qos api.Qos
  26. checkpointInterval int
  27. store api.Store
  28. coordinator *checkpoints.Coordinator
  29. topo *PrintableTopo
  30. }
  31. func NewWithNameAndQos(name string, qos api.Qos, checkpointInterval int) (*TopologyNew, error) {
  32. tp := &TopologyNew{
  33. name: name,
  34. qos: qos,
  35. checkpointInterval: checkpointInterval,
  36. topo: &PrintableTopo{
  37. Sources: make([]string, 0),
  38. Edges: make(map[string][]string),
  39. },
  40. }
  41. return tp, nil
  42. }
  43. func (s *TopologyNew) GetContext() api.StreamContext {
  44. return s.ctx
  45. }
  46. // may be called multiple times so must be idempotent
  47. func (s *TopologyNew) Cancel() {
  48. // completion signal
  49. s.drainErr(nil)
  50. s.cancel()
  51. s.store = nil
  52. s.coordinator = nil
  53. }
  54. func (s *TopologyNew) AddSrc(src nodes.DataSourceNode) *TopologyNew {
  55. s.sources = append(s.sources, src)
  56. s.topo.Sources = append(s.topo.Sources, fmt.Sprintf("source_%s", src.GetName()))
  57. return s
  58. }
  59. func (s *TopologyNew) AddSink(inputs []api.Emitter, snk *nodes.SinkNode) *TopologyNew {
  60. for _, input := range inputs {
  61. input.AddOutput(snk.GetInput())
  62. snk.AddInputCount()
  63. s.addEdge(input.(api.TopNode), snk, "sink")
  64. }
  65. s.sinks = append(s.sinks, snk)
  66. return s
  67. }
  68. func (s *TopologyNew) AddOperator(inputs []api.Emitter, operator nodes.OperatorNode) *TopologyNew {
  69. for _, input := range inputs {
  70. input.AddOutput(operator.GetInput())
  71. operator.AddInputCount()
  72. s.addEdge(input.(api.TopNode), operator, "op")
  73. }
  74. s.ops = append(s.ops, operator)
  75. return s
  76. }
  77. func (s *TopologyNew) addEdge(from api.TopNode, to api.TopNode, toType string) {
  78. fromType := "op"
  79. if _, ok := from.(nodes.DataSourceNode); ok {
  80. fromType = "source"
  81. }
  82. f := fmt.Sprintf("%s_%s", fromType, from.GetName())
  83. t := fmt.Sprintf("%s_%s", toType, to.GetName())
  84. e, ok := s.topo.Edges[f]
  85. if !ok {
  86. e = make([]string, 0)
  87. }
  88. s.topo.Edges[f] = append(e, t)
  89. }
  90. // prepareContext setups internal context before
  91. // stream starts execution.
  92. func (s *TopologyNew) prepareContext() {
  93. if s.ctx == nil || s.ctx.Err() != nil {
  94. contextLogger := common.Log.WithField("rule", s.name)
  95. ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
  96. s.ctx, s.cancel = ctx.WithCancel()
  97. }
  98. }
  99. func (s *TopologyNew) drainErr(err error) {
  100. select {
  101. case s.drain <- err:
  102. s.ctx.GetLogger().Errorf("topo %s drain error %v", s.name, err)
  103. default:
  104. s.ctx.GetLogger().Infof("topo %s drain error %v, but receiver closed so ignored", s.name, err)
  105. }
  106. }
  107. func (s *TopologyNew) Open() <-chan error {
  108. //if stream has opened, do nothing
  109. if s.ctx != nil && s.ctx.Err() == nil {
  110. s.ctx.GetLogger().Infoln("rule is already running, do nothing")
  111. return s.drain
  112. }
  113. s.prepareContext() // ensure context is set
  114. s.drain = make(chan error)
  115. log := s.ctx.GetLogger()
  116. log.Infoln("Opening stream")
  117. // open stream
  118. go func() {
  119. var err error
  120. if s.store, err = states.CreateStore(s.name, s.qos); err != nil {
  121. fmt.Println(err)
  122. s.drain <- err
  123. return
  124. }
  125. s.enableCheckpoint()
  126. // open stream sink, after log sink is ready.
  127. for _, snk := range s.sinks {
  128. snk.Open(s.ctx.WithMeta(s.name, snk.GetName(), s.store), s.drain)
  129. }
  130. //apply operators, if err bail
  131. for _, op := range s.ops {
  132. op.Exec(s.ctx.WithMeta(s.name, op.GetName(), s.store), s.drain)
  133. }
  134. // open source, if err bail
  135. for _, node := range s.sources {
  136. node.Open(s.ctx.WithMeta(s.name, node.GetName(), s.store), s.drain)
  137. }
  138. // activate checkpoint
  139. if s.coordinator != nil {
  140. s.coordinator.Activate()
  141. }
  142. }()
  143. return s.drain
  144. }
  145. func (s *TopologyNew) enableCheckpoint() error {
  146. if s.qos >= api.AtLeastOnce {
  147. var sources []checkpoints.StreamTask
  148. for _, r := range s.sources {
  149. sources = append(sources, r)
  150. }
  151. var ops []checkpoints.NonSourceTask
  152. for _, r := range s.ops {
  153. ops = append(ops, r)
  154. }
  155. var sinks []checkpoints.SinkTask
  156. for _, r := range s.sinks {
  157. sinks = append(sinks, r)
  158. }
  159. c := checkpoints.NewCoordinator(s.name, sources, ops, sinks, s.qos, s.store, s.checkpointInterval, s.ctx)
  160. s.coordinator = c
  161. }
  162. return nil
  163. }
  164. func (s *TopologyNew) GetCoordinator() *checkpoints.Coordinator {
  165. return s.coordinator
  166. }
  167. func (s *TopologyNew) GetMetrics() (keys []string, values []interface{}) {
  168. for _, node := range s.sources {
  169. for ins, metrics := range node.GetMetrics() {
  170. for i, v := range metrics {
  171. keys = append(keys, "source_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  172. values = append(values, v)
  173. }
  174. }
  175. }
  176. for _, node := range s.ops {
  177. for ins, metrics := range node.GetMetrics() {
  178. for i, v := range metrics {
  179. keys = append(keys, "op_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  180. values = append(values, v)
  181. }
  182. }
  183. }
  184. for _, node := range s.sinks {
  185. for ins, metrics := range node.GetMetrics() {
  186. for i, v := range metrics {
  187. keys = append(keys, "sink_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  188. values = append(values, v)
  189. }
  190. }
  191. }
  192. return
  193. }
  194. func (s *TopologyNew) GetTopo() *PrintableTopo {
  195. return s.topo
  196. }