streams.go 5.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215
  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. drain: make(chan error),
  35. qos: qos,
  36. checkpointInterval: checkpointInterval,
  37. topo: &PrintableTopo{
  38. Sources: make([]string, 0),
  39. Edges: make(map[string][]string),
  40. },
  41. }
  42. return tp, nil
  43. }
  44. func (s *TopologyNew) GetContext() api.StreamContext {
  45. return s.ctx
  46. }
  47. func (s *TopologyNew) Cancel() {
  48. s.cancel()
  49. s.store = nil
  50. s.coordinator = nil
  51. }
  52. func (s *TopologyNew) AddSrc(src *nodes.SourceNode) *TopologyNew {
  53. s.sources = append(s.sources, src)
  54. s.topo.Sources = append(s.topo.Sources, fmt.Sprintf("source_%s", src.GetName()))
  55. return s
  56. }
  57. func (s *TopologyNew) AddSink(inputs []api.Emitter, snk *nodes.SinkNode) *TopologyNew {
  58. for _, input := range inputs {
  59. input.AddOutput(snk.GetInput())
  60. snk.AddInputCount()
  61. s.addEdge(input.(api.TopNode), snk, "sink")
  62. }
  63. s.sinks = append(s.sinks, snk)
  64. return s
  65. }
  66. func (s *TopologyNew) AddOperator(inputs []api.Emitter, operator nodes.OperatorNode) *TopologyNew {
  67. for _, input := range inputs {
  68. input.AddOutput(operator.GetInput())
  69. operator.AddInputCount()
  70. s.addEdge(input.(api.TopNode), operator, "op")
  71. }
  72. s.ops = append(s.ops, operator)
  73. return s
  74. }
  75. func (s *TopologyNew) addEdge(from api.TopNode, to api.TopNode, toType string) {
  76. fromType := "op"
  77. if _, ok := from.(*nodes.SourceNode); ok {
  78. fromType = "source"
  79. }
  80. f := fmt.Sprintf("%s_%s", fromType, from.GetName())
  81. t := fmt.Sprintf("%s_%s", toType, to.GetName())
  82. e, ok := s.topo.Edges[f]
  83. if !ok {
  84. e = make([]string, 0)
  85. }
  86. s.topo.Edges[f] = append(e, t)
  87. }
  88. func Transform(op nodes.UnOperation, name string, bufferLength int) *nodes.UnaryOperator {
  89. operator := nodes.New(name, bufferLength)
  90. operator.SetOperation(op)
  91. return operator
  92. }
  93. // prepareContext setups internal context before
  94. // stream starts execution.
  95. func (s *TopologyNew) prepareContext() {
  96. if s.ctx == nil || s.ctx.Err() != nil {
  97. contextLogger := common.Log.WithField("rule", s.name)
  98. ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
  99. s.ctx, s.cancel = ctx.WithCancel()
  100. }
  101. }
  102. func (s *TopologyNew) drainErr(err error) {
  103. go func() { s.drain <- err }()
  104. }
  105. func (s *TopologyNew) Open() <-chan error {
  106. //if stream has opened, do nothing
  107. if s.ctx != nil && s.ctx.Err() == nil {
  108. s.ctx.GetLogger().Infoln("rule is already running, do nothing")
  109. return s.drain
  110. }
  111. s.prepareContext() // ensure context is set
  112. var err error
  113. if s.store, err = states.CreateStore(s.name, s.qos); err != nil {
  114. s.drainErr(err)
  115. return s.drain
  116. }
  117. s.enableCheckpoint()
  118. log := s.ctx.GetLogger()
  119. log.Infoln("Opening stream")
  120. // open stream
  121. go func() {
  122. // open stream sink, after log sink is ready.
  123. for _, snk := range s.sinks {
  124. snk.Open(s.ctx.WithMeta(s.name, snk.GetName(), s.store), s.drain)
  125. }
  126. //apply operators, if err bail
  127. for _, op := range s.ops {
  128. op.Exec(s.ctx.WithMeta(s.name, op.GetName(), s.store), s.drain)
  129. }
  130. // open source, if err bail
  131. for _, node := range s.sources {
  132. node.Open(s.ctx.WithMeta(s.name, node.GetName(), s.store), s.drain)
  133. }
  134. // activate checkpoint
  135. if s.coordinator != nil {
  136. s.coordinator.Activate()
  137. }
  138. }()
  139. return s.drain
  140. }
  141. func (s *TopologyNew) enableCheckpoint() error {
  142. if s.qos >= api.AtLeastOnce {
  143. var sources []checkpoints.StreamTask
  144. for _, r := range s.sources {
  145. sources = append(sources, r)
  146. }
  147. var ops []checkpoints.NonSourceTask
  148. for _, r := range s.ops {
  149. ops = append(ops, r)
  150. }
  151. var sinks []checkpoints.SinkTask
  152. for _, r := range s.sinks {
  153. sinks = append(sinks, r)
  154. }
  155. c := checkpoints.NewCoordinator(s.name, sources, ops, sinks, s.qos, s.store, s.checkpointInterval, s.ctx)
  156. s.coordinator = c
  157. }
  158. return nil
  159. }
  160. func (s *TopologyNew) GetCoordinator() *checkpoints.Coordinator {
  161. return s.coordinator
  162. }
  163. func (s *TopologyNew) GetMetrics() (keys []string, values []interface{}) {
  164. for _, node := range s.sources {
  165. for ins, metrics := range node.GetMetrics() {
  166. for i, v := range metrics {
  167. keys = append(keys, "source_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  168. values = append(values, v)
  169. }
  170. }
  171. }
  172. for _, node := range s.ops {
  173. for ins, metrics := range node.GetMetrics() {
  174. for i, v := range metrics {
  175. keys = append(keys, "op_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  176. values = append(values, v)
  177. }
  178. }
  179. }
  180. for _, node := range s.sinks {
  181. for ins, metrics := range node.GetMetrics() {
  182. for i, v := range metrics {
  183. keys = append(keys, "sink_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  184. values = append(values, v)
  185. }
  186. }
  187. }
  188. return
  189. }
  190. func (s *TopologyNew) GetTopo() *PrintableTopo {
  191. return s.topo
  192. }