topo.go 6.1 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229
  1. // Copyright 2021-2022 EMQ Technologies Co., Ltd.
  2. //
  3. // Licensed under the Apache License, Version 2.0 (the "License");
  4. // you may not use this file except in compliance with the License.
  5. // You may obtain a copy of the License at
  6. //
  7. // http://www.apache.org/licenses/LICENSE-2.0
  8. //
  9. // Unless required by applicable law or agreed to in writing, software
  10. // distributed under the License is distributed on an "AS IS" BASIS,
  11. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  12. // See the License for the specific language governing permissions and
  13. // limitations under the License.
  14. package topo
  15. import (
  16. "context"
  17. "fmt"
  18. "github.com/lf-edge/ekuiper/internal/conf"
  19. "github.com/lf-edge/ekuiper/internal/topo/checkpoint"
  20. kctx "github.com/lf-edge/ekuiper/internal/topo/context"
  21. "github.com/lf-edge/ekuiper/internal/topo/node"
  22. "github.com/lf-edge/ekuiper/internal/topo/node/metric"
  23. "github.com/lf-edge/ekuiper/internal/topo/state"
  24. "github.com/lf-edge/ekuiper/pkg/api"
  25. "github.com/lf-edge/ekuiper/pkg/infra"
  26. "strconv"
  27. "sync"
  28. )
  29. type Topo struct {
  30. sources []node.DataSourceNode
  31. sinks []*node.SinkNode
  32. ctx api.StreamContext
  33. cancel context.CancelFunc
  34. drain chan error
  35. ops []node.OperatorNode
  36. name string
  37. qos api.Qos
  38. checkpointInterval int
  39. store api.Store
  40. coordinator *checkpoint.Coordinator
  41. topo *api.PrintableTopo
  42. mu sync.Mutex
  43. }
  44. func NewWithNameAndQos(name string, qos api.Qos, checkpointInterval int) (*Topo, error) {
  45. tp := &Topo{
  46. name: name,
  47. qos: qos,
  48. checkpointInterval: checkpointInterval,
  49. topo: &api.PrintableTopo{
  50. Sources: make([]string, 0),
  51. Edges: make(map[string][]string),
  52. },
  53. }
  54. return tp, nil
  55. }
  56. func (s *Topo) GetContext() api.StreamContext {
  57. return s.ctx
  58. }
  59. // Cancel may be called multiple times so must be idempotent
  60. func (s *Topo) Cancel() {
  61. s.mu.Lock()
  62. defer s.mu.Unlock()
  63. // completion signal
  64. infra.DrainError(s.ctx, nil, s.drain)
  65. s.cancel()
  66. s.store = nil
  67. s.coordinator = nil
  68. }
  69. func (s *Topo) AddSrc(src node.DataSourceNode) *Topo {
  70. s.sources = append(s.sources, src)
  71. s.topo.Sources = append(s.topo.Sources, fmt.Sprintf("source_%s", src.GetName()))
  72. return s
  73. }
  74. func (s *Topo) AddSink(inputs []api.Emitter, snk *node.SinkNode) *Topo {
  75. for _, input := range inputs {
  76. input.AddOutput(snk.GetInput())
  77. snk.AddInputCount()
  78. s.addEdge(input.(api.TopNode), snk, "sink")
  79. }
  80. s.sinks = append(s.sinks, snk)
  81. return s
  82. }
  83. func (s *Topo) AddOperator(inputs []api.Emitter, operator node.OperatorNode) *Topo {
  84. for _, input := range inputs {
  85. input.AddOutput(operator.GetInput())
  86. operator.AddInputCount()
  87. s.addEdge(input.(api.TopNode), operator, "op")
  88. }
  89. s.ops = append(s.ops, operator)
  90. return s
  91. }
  92. func (s *Topo) addEdge(from api.TopNode, to api.TopNode, toType string) {
  93. fromType := "op"
  94. if _, ok := from.(node.DataSourceNode); ok {
  95. fromType = "source"
  96. }
  97. f := fmt.Sprintf("%s_%s", fromType, from.GetName())
  98. t := fmt.Sprintf("%s_%s", toType, to.GetName())
  99. e, ok := s.topo.Edges[f]
  100. if !ok {
  101. e = make([]string, 0)
  102. }
  103. s.topo.Edges[f] = append(e, t)
  104. }
  105. // prepareContext setups internal context before
  106. // stream starts execution.
  107. func (s *Topo) prepareContext() {
  108. if s.ctx == nil || s.ctx.Err() != nil {
  109. contextLogger := conf.Log.WithField("rule", s.name)
  110. ctx := kctx.WithValue(kctx.Background(), kctx.LoggerKey, contextLogger)
  111. s.ctx, s.cancel = ctx.WithCancel()
  112. }
  113. }
  114. func (s *Topo) Open() <-chan error {
  115. //if stream has opened, do nothing
  116. if s.ctx != nil && s.ctx.Err() == nil {
  117. s.ctx.GetLogger().Infoln("rule is already running, do nothing")
  118. return s.drain
  119. }
  120. s.prepareContext() // ensure context is set
  121. s.drain = make(chan error)
  122. log := s.ctx.GetLogger()
  123. log.Infoln("Opening stream")
  124. go func() {
  125. err := infra.SafeRun(func() error {
  126. s.mu.Lock()
  127. defer s.mu.Unlock()
  128. var err error
  129. if s.store, err = state.CreateStore(s.name, s.qos); err != nil {
  130. return fmt.Errorf("topo %s create store error %v", s.name, err)
  131. }
  132. s.enableCheckpoint()
  133. // open stream sink, after log sink is ready.
  134. for _, snk := range s.sinks {
  135. snk.Open(s.ctx.WithMeta(s.name, snk.GetName(), s.store), s.drain)
  136. }
  137. //apply operators, if err bail
  138. for _, op := range s.ops {
  139. op.Exec(s.ctx.WithMeta(s.name, op.GetName(), s.store), s.drain)
  140. }
  141. // open source, if err bail
  142. for _, source := range s.sources {
  143. source.Open(s.ctx.WithMeta(s.name, source.GetName(), s.store), s.drain)
  144. }
  145. // activate checkpoint
  146. if s.coordinator != nil {
  147. s.coordinator.Activate()
  148. }
  149. return nil
  150. })
  151. if err != nil {
  152. infra.DrainError(s.ctx, err, s.drain)
  153. }
  154. }()
  155. return s.drain
  156. }
  157. func (s *Topo) enableCheckpoint() error {
  158. if s.qos >= api.AtLeastOnce {
  159. var sources []checkpoint.StreamTask
  160. for _, r := range s.sources {
  161. sources = append(sources, r)
  162. }
  163. var ops []checkpoint.NonSourceTask
  164. for _, r := range s.ops {
  165. ops = append(ops, r)
  166. }
  167. var sinks []checkpoint.SinkTask
  168. for _, r := range s.sinks {
  169. sinks = append(sinks, r)
  170. }
  171. c := checkpoint.NewCoordinator(s.name, sources, ops, sinks, s.qos, s.store, s.checkpointInterval, s.ctx)
  172. s.coordinator = c
  173. }
  174. return nil
  175. }
  176. func (s *Topo) GetCoordinator() *checkpoint.Coordinator {
  177. return s.coordinator
  178. }
  179. func (s *Topo) GetMetrics() (keys []string, values []interface{}) {
  180. for _, sn := range s.sources {
  181. for ins, metrics := range sn.GetMetrics() {
  182. for i, v := range metrics {
  183. keys = append(keys, "source_"+sn.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  184. values = append(values, v)
  185. }
  186. }
  187. }
  188. for _, so := range s.ops {
  189. for ins, metrics := range so.GetMetrics() {
  190. for i, v := range metrics {
  191. keys = append(keys, "op_"+so.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  192. values = append(values, v)
  193. }
  194. }
  195. }
  196. for _, sn := range s.sinks {
  197. for ins, metrics := range sn.GetMetrics() {
  198. for i, v := range metrics {
  199. keys = append(keys, "sink_"+sn.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  200. values = append(values, v)
  201. }
  202. }
  203. }
  204. return
  205. }
  206. func (s *Topo) GetTopo() *api.PrintableTopo {
  207. return s.topo
  208. }