topo.go 6.2 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234
  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 PrintableTopo struct {
  30. Sources []string `json:"sources"`
  31. Edges map[string][]string `json:"edges"`
  32. }
  33. type Topo struct {
  34. sources []node.DataSourceNode
  35. sinks []*node.SinkNode
  36. ctx api.StreamContext
  37. cancel context.CancelFunc
  38. drain chan error
  39. ops []node.OperatorNode
  40. name string
  41. qos api.Qos
  42. checkpointInterval int
  43. store api.Store
  44. coordinator *checkpoint.Coordinator
  45. topo *PrintableTopo
  46. mu sync.Mutex
  47. }
  48. func NewWithNameAndQos(name string, qos api.Qos, checkpointInterval int) (*Topo, error) {
  49. tp := &Topo{
  50. name: name,
  51. qos: qos,
  52. checkpointInterval: checkpointInterval,
  53. topo: &PrintableTopo{
  54. Sources: make([]string, 0),
  55. Edges: make(map[string][]string),
  56. },
  57. }
  58. return tp, nil
  59. }
  60. func (s *Topo) GetContext() api.StreamContext {
  61. return s.ctx
  62. }
  63. // Cancel may be called multiple times so must be idempotent
  64. func (s *Topo) Cancel() {
  65. s.mu.Lock()
  66. defer s.mu.Unlock()
  67. // completion signal
  68. infra.DrainError(s.ctx, nil, s.drain)
  69. s.cancel()
  70. s.store = nil
  71. s.coordinator = nil
  72. }
  73. func (s *Topo) AddSrc(src node.DataSourceNode) *Topo {
  74. s.sources = append(s.sources, src)
  75. s.topo.Sources = append(s.topo.Sources, fmt.Sprintf("source_%s", src.GetName()))
  76. return s
  77. }
  78. func (s *Topo) AddSink(inputs []api.Emitter, snk *node.SinkNode) *Topo {
  79. for _, input := range inputs {
  80. input.AddOutput(snk.GetInput())
  81. snk.AddInputCount()
  82. s.addEdge(input.(api.TopNode), snk, "sink")
  83. }
  84. s.sinks = append(s.sinks, snk)
  85. return s
  86. }
  87. func (s *Topo) AddOperator(inputs []api.Emitter, operator node.OperatorNode) *Topo {
  88. for _, input := range inputs {
  89. input.AddOutput(operator.GetInput())
  90. operator.AddInputCount()
  91. s.addEdge(input.(api.TopNode), operator, "op")
  92. }
  93. s.ops = append(s.ops, operator)
  94. return s
  95. }
  96. func (s *Topo) addEdge(from api.TopNode, to api.TopNode, toType string) {
  97. fromType := "op"
  98. if _, ok := from.(node.DataSourceNode); ok {
  99. fromType = "source"
  100. }
  101. f := fmt.Sprintf("%s_%s", fromType, from.GetName())
  102. t := fmt.Sprintf("%s_%s", toType, to.GetName())
  103. e, ok := s.topo.Edges[f]
  104. if !ok {
  105. e = make([]string, 0)
  106. }
  107. s.topo.Edges[f] = append(e, t)
  108. }
  109. // prepareContext setups internal context before
  110. // stream starts execution.
  111. func (s *Topo) prepareContext() {
  112. if s.ctx == nil || s.ctx.Err() != nil {
  113. contextLogger := conf.Log.WithField("rule", s.name)
  114. ctx := kctx.WithValue(kctx.Background(), kctx.LoggerKey, contextLogger)
  115. s.ctx, s.cancel = ctx.WithCancel()
  116. }
  117. }
  118. func (s *Topo) Open() <-chan error {
  119. //if stream has opened, do nothing
  120. if s.ctx != nil && s.ctx.Err() == nil {
  121. s.ctx.GetLogger().Infoln("rule is already running, do nothing")
  122. return s.drain
  123. }
  124. s.prepareContext() // ensure context is set
  125. s.drain = make(chan error)
  126. log := s.ctx.GetLogger()
  127. log.Infoln("Opening stream")
  128. go func() {
  129. err := infra.SafeRun(func() error {
  130. s.mu.Lock()
  131. defer s.mu.Unlock()
  132. var err error
  133. if s.store, err = state.CreateStore(s.name, s.qos); err != nil {
  134. return fmt.Errorf("topo %s create store error %v", s.name, err)
  135. }
  136. s.enableCheckpoint()
  137. // open stream sink, after log sink is ready.
  138. for _, snk := range s.sinks {
  139. snk.Open(s.ctx.WithMeta(s.name, snk.GetName(), s.store), s.drain)
  140. }
  141. //apply operators, if err bail
  142. for _, op := range s.ops {
  143. op.Exec(s.ctx.WithMeta(s.name, op.GetName(), s.store), s.drain)
  144. }
  145. // open source, if err bail
  146. for _, source := range s.sources {
  147. source.Open(s.ctx.WithMeta(s.name, source.GetName(), s.store), s.drain)
  148. }
  149. // activate checkpoint
  150. if s.coordinator != nil {
  151. s.coordinator.Activate()
  152. }
  153. return nil
  154. })
  155. if err != nil {
  156. infra.DrainError(s.ctx, err, s.drain)
  157. }
  158. }()
  159. return s.drain
  160. }
  161. func (s *Topo) enableCheckpoint() error {
  162. if s.qos >= api.AtLeastOnce {
  163. var sources []checkpoint.StreamTask
  164. for _, r := range s.sources {
  165. sources = append(sources, r)
  166. }
  167. var ops []checkpoint.NonSourceTask
  168. for _, r := range s.ops {
  169. ops = append(ops, r)
  170. }
  171. var sinks []checkpoint.SinkTask
  172. for _, r := range s.sinks {
  173. sinks = append(sinks, r)
  174. }
  175. c := checkpoint.NewCoordinator(s.name, sources, ops, sinks, s.qos, s.store, s.checkpointInterval, s.ctx)
  176. s.coordinator = c
  177. }
  178. return nil
  179. }
  180. func (s *Topo) GetCoordinator() *checkpoint.Coordinator {
  181. return s.coordinator
  182. }
  183. func (s *Topo) GetMetrics() (keys []string, values []interface{}) {
  184. for _, sn := range s.sources {
  185. for ins, metrics := range sn.GetMetrics() {
  186. for i, v := range metrics {
  187. keys = append(keys, "source_"+sn.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  188. values = append(values, v)
  189. }
  190. }
  191. }
  192. for _, so := range s.ops {
  193. for ins, metrics := range so.GetMetrics() {
  194. for i, v := range metrics {
  195. keys = append(keys, "op_"+so.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  196. values = append(values, v)
  197. }
  198. }
  199. }
  200. for _, sn := range s.sinks {
  201. for ins, metrics := range sn.GetMetrics() {
  202. for i, v := range metrics {
  203. keys = append(keys, "sink_"+sn.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  204. values = append(values, v)
  205. }
  206. }
  207. }
  208. return
  209. }
  210. func (s *Topo) GetTopo() *PrintableTopo {
  211. return s.topo
  212. }