streams.go 4.7 KB

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