streams.go 4.6 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182
  1. package xstream
  2. import (
  3. "context"
  4. "github.com/emqx/kuiper/common"
  5. "github.com/emqx/kuiper/xsql"
  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. "strconv"
  11. )
  12. type TopologyNew struct {
  13. sources []*nodes.SourceNode
  14. sinks []*nodes.SinkNode
  15. ctx api.StreamContext
  16. cancel context.CancelFunc
  17. drain chan error
  18. ops []nodes.OperatorNode
  19. name string
  20. qos xsql.Qos
  21. checkpointInterval int
  22. store checkpoints.Store
  23. coordinator *checkpoints.Coordinator
  24. }
  25. func NewWithName(name string) *TopologyNew {
  26. return NewWithNameAndQos(name, xsql.AtMostOnce)
  27. }
  28. func NewWithNameAndQos(name string, qos xsql.Qos) *TopologyNew {
  29. tp := &TopologyNew{
  30. name: name,
  31. drain: make(chan error),
  32. qos: qos,
  33. }
  34. if qos >= xsql.AtLeastOnce {
  35. tp.store = checkpoints.GetKVStore(name)
  36. }
  37. return tp
  38. }
  39. func (s *TopologyNew) GetContext() api.StreamContext {
  40. return s.ctx
  41. }
  42. func (s *TopologyNew) Cancel() {
  43. s.cancel()
  44. }
  45. func (s *TopologyNew) AddSrc(src *nodes.SourceNode) *TopologyNew {
  46. s.sources = append(s.sources, src)
  47. return s
  48. }
  49. func (s *TopologyNew) AddSink(inputs []api.Emitter, snk *nodes.SinkNode) *TopologyNew {
  50. for _, input := range inputs {
  51. input.AddOutput(snk.GetInput())
  52. snk.AddInputCount()
  53. }
  54. s.sinks = append(s.sinks, snk)
  55. return s
  56. }
  57. func (s *TopologyNew) AddOperator(inputs []api.Emitter, operator nodes.OperatorNode) *TopologyNew {
  58. for _, input := range inputs {
  59. input.AddOutput(operator.GetInput())
  60. operator.AddInputCount()
  61. }
  62. s.ops = append(s.ops, operator)
  63. return s
  64. }
  65. func Transform(op nodes.UnOperation, name string, bufferLength int) *nodes.UnaryOperator {
  66. operator := nodes.New(name, bufferLength)
  67. operator.SetOperation(op)
  68. return operator
  69. }
  70. // prepareContext setups internal context before
  71. // stream starts execution.
  72. func (s *TopologyNew) prepareContext() {
  73. if s.ctx == nil || s.ctx.Err() != nil {
  74. contextLogger := common.Log.WithField("rule", s.name)
  75. ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
  76. s.ctx, s.cancel = ctx.WithCancel()
  77. }
  78. }
  79. func (s *TopologyNew) drainErr(err error) {
  80. go func() { s.drain <- err }()
  81. }
  82. func (s *TopologyNew) Open() <-chan error {
  83. //if stream has opened, do nothing
  84. if s.ctx != nil && s.ctx.Err() == nil {
  85. s.ctx.GetLogger().Infoln("rule is already running, do nothing")
  86. return s.drain
  87. }
  88. s.prepareContext() // ensure context is set
  89. s.enableCheckpoint()
  90. log := s.ctx.GetLogger()
  91. log.Infoln("Opening stream")
  92. // open stream
  93. go func() {
  94. // open stream sink, after log sink is ready.
  95. for _, snk := range s.sinks {
  96. snk.Open(s.ctx.WithMeta(s.name, snk.GetName()), s.drain)
  97. }
  98. //apply operators, if err bail
  99. for _, op := range s.ops {
  100. op.Exec(s.ctx.WithMeta(s.name, op.GetName()), s.drain)
  101. }
  102. // open source, if err bail
  103. for _, node := range s.sources {
  104. node.Open(s.ctx.WithMeta(s.name, node.GetName()), s.drain)
  105. }
  106. // activate checkpoint
  107. if s.coordinator != nil {
  108. s.coordinator.Activate()
  109. }
  110. }()
  111. return s.drain
  112. }
  113. func (s *TopologyNew) enableCheckpoint() error {
  114. if s.qos >= xsql.AtLeastOnce {
  115. var sources []checkpoints.StreamTask
  116. for _, r := range s.sources {
  117. sources = append(sources, r)
  118. }
  119. var ops []checkpoints.NonSourceTask
  120. for _, r := range s.ops {
  121. ops = append(ops, r)
  122. }
  123. var sinks []checkpoints.NonSourceTask
  124. for _, r := range s.sinks {
  125. sinks = append(sinks, r)
  126. }
  127. c := checkpoints.NewCoordinator(s.name, sources, ops, sinks, s.qos, s.store, s.checkpointInterval, s.ctx)
  128. s.coordinator = c
  129. }
  130. return nil
  131. }
  132. func (s *TopologyNew) GetCoordinator() *checkpoints.Coordinator {
  133. return s.coordinator
  134. }
  135. func (s *TopologyNew) GetMetrics() (keys []string, values []interface{}) {
  136. for _, node := range s.sources {
  137. for ins, metrics := range node.GetMetrics() {
  138. for i, v := range metrics {
  139. keys = append(keys, "source_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  140. values = append(values, v)
  141. }
  142. }
  143. }
  144. for _, node := range s.ops {
  145. for ins, metrics := range node.GetMetrics() {
  146. for i, v := range metrics {
  147. keys = append(keys, "op_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  148. values = append(values, v)
  149. }
  150. }
  151. }
  152. for _, node := range s.sinks {
  153. for ins, metrics := range node.GetMetrics() {
  154. for i, v := range metrics {
  155. keys = append(keys, "sink_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
  156. values = append(values, v)
  157. }
  158. }
  159. }
  160. return
  161. }