streams.go 4.8 KB

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