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.SourceNode
  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. func (s *TopologyNew) Cancel() {
  47. if s.drain != nil {
  48. close(s.drain)
  49. }
  50. s.cancel()
  51. s.store = nil
  52. s.coordinator = nil
  53. }
  54. func (s *TopologyNew) AddSrc(src *nodes.SourceNode) *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.SourceNode); 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. func Transform(op nodes.UnOperation, name string, bufferLength int) *nodes.UnaryOperator {
  91. operator := nodes.New(name, bufferLength)
  92. operator.SetOperation(op)
  93. return operator
  94. }
  95. // prepareContext setups internal context before
  96. // stream starts execution.
  97. func (s *TopologyNew) prepareContext() {
  98. if s.ctx == nil || s.ctx.Err() != nil {
  99. contextLogger := common.Log.WithField("rule", s.name)
  100. ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
  101. s.ctx, s.cancel = ctx.WithCancel()
  102. }
  103. }
  104. func (s *TopologyNew) drainErr(err error) {
  105. go func() { s.drain <- err }()
  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. var err error
  116. if s.store, err = states.CreateStore(s.name, s.qos); err != nil {
  117. s.drainErr(err)
  118. return s.drain
  119. }
  120. s.enableCheckpoint()
  121. log := s.ctx.GetLogger()
  122. log.Infoln("Opening stream")
  123. // open stream
  124. go func() {
  125. // open stream sink, after log sink is ready.
  126. for _, snk := range s.sinks {
  127. snk.Open(s.ctx.WithMeta(s.name, snk.GetName(), s.store), s.drain)
  128. }
  129. //apply operators, if err bail
  130. for _, op := range s.ops {
  131. op.Exec(s.ctx.WithMeta(s.name, op.GetName(), s.store), s.drain)
  132. }
  133. // open source, if err bail
  134. for _, node := range s.sources {
  135. node.Open(s.ctx.WithMeta(s.name, node.GetName(), s.store), s.drain)
  136. }
  137. // activate checkpoint
  138. if s.coordinator != nil {
  139. s.coordinator.Activate()
  140. }
  141. }()
  142. return s.drain
  143. }
  144. func (s *TopologyNew) enableCheckpoint() error {
  145. if s.qos >= api.AtLeastOnce {
  146. var sources []checkpoints.StreamTask
  147. for _, r := range s.sources {
  148. sources = append(sources, r)
  149. }
  150. var ops []checkpoints.NonSourceTask
  151. for _, r := range s.ops {
  152. ops = append(ops, r)
  153. }
  154. var sinks []checkpoints.SinkTask
  155. for _, r := range s.sinks {
  156. sinks = append(sinks, r)
  157. }
  158. c := checkpoints.NewCoordinator(s.name, sources, ops, sinks, s.qos, s.store, s.checkpointInterval, s.ctx)
  159. s.coordinator = c
  160. }
  161. return nil
  162. }
  163. func (s *TopologyNew) GetCoordinator() *checkpoints.Coordinator {
  164. return s.coordinator
  165. }
  166. func (s *TopologyNew) GetMetrics() (keys []string, values []interface{}) {
  167. for _, node := range s.sources {
  168. for ins, metrics := range node.GetMetrics() {
  169. for i, v := range metrics {
  170. keys = append(keys, "source_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  171. values = append(values, v)
  172. }
  173. }
  174. }
  175. for _, node := range s.ops {
  176. for ins, metrics := range node.GetMetrics() {
  177. for i, v := range metrics {
  178. keys = append(keys, "op_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  179. values = append(values, v)
  180. }
  181. }
  182. }
  183. for _, node := range s.sinks {
  184. for ins, metrics := range node.GetMetrics() {
  185. for i, v := range metrics {
  186. keys = append(keys, "sink_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  187. values = append(values, v)
  188. }
  189. }
  190. }
  191. return
  192. }
  193. func (s *TopologyNew) GetTopo() *PrintableTopo {
  194. return s.topo
  195. }