topo.go 5.5 KB

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