topo.go 7.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280
  1. // Copyright 2021-2023 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. "io"
  19. "os"
  20. "path"
  21. "strconv"
  22. "sync"
  23. "time"
  24. rotatelogs "github.com/lestrrat-go/file-rotatelogs"
  25. "github.com/sirupsen/logrus"
  26. "github.com/lf-edge/ekuiper/internal/conf"
  27. "github.com/lf-edge/ekuiper/internal/topo/checkpoint"
  28. kctx "github.com/lf-edge/ekuiper/internal/topo/context"
  29. "github.com/lf-edge/ekuiper/internal/topo/node"
  30. "github.com/lf-edge/ekuiper/internal/topo/node/metric"
  31. "github.com/lf-edge/ekuiper/internal/topo/state"
  32. "github.com/lf-edge/ekuiper/pkg/api"
  33. "github.com/lf-edge/ekuiper/pkg/infra"
  34. )
  35. type Topo struct {
  36. sources []node.DataSourceNode
  37. sinks []*node.SinkNode
  38. ctx api.StreamContext
  39. cancel context.CancelFunc
  40. drain chan error
  41. ops []node.OperatorNode
  42. name string
  43. options *api.RuleOption
  44. store api.Store
  45. coordinator *checkpoint.Coordinator
  46. topo *api.PrintableTopo
  47. mu sync.Mutex
  48. }
  49. func NewWithNameAndOptions(name string, options *api.RuleOption) (*Topo, error) {
  50. tp := &Topo{
  51. name: name,
  52. options: options,
  53. topo: &api.PrintableTopo{
  54. Sources: make([]string, 0),
  55. Edges: make(map[string][]interface{}),
  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. if s.cancel != nil {
  70. s.cancel()
  71. }
  72. s.store = nil
  73. s.coordinator = nil
  74. }
  75. func (s *Topo) AddSrc(src node.DataSourceNode) *Topo {
  76. s.sources = append(s.sources, src)
  77. s.topo.Sources = append(s.topo.Sources, fmt.Sprintf("source_%s", src.GetName()))
  78. return s
  79. }
  80. func (s *Topo) AddSink(inputs []api.Emitter, snk *node.SinkNode) *Topo {
  81. for _, input := range inputs {
  82. input.AddOutput(snk.GetInput())
  83. snk.AddInputCount()
  84. s.addEdge(input.(api.TopNode), snk, "sink")
  85. }
  86. s.sinks = append(s.sinks, snk)
  87. return s
  88. }
  89. func (s *Topo) AddOperator(inputs []api.Emitter, operator node.OperatorNode) *Topo {
  90. for _, input := range inputs {
  91. input.AddOutput(operator.GetInput())
  92. operator.AddInputCount()
  93. s.addEdge(input.(api.TopNode), operator, "op")
  94. }
  95. s.ops = append(s.ops, operator)
  96. return s
  97. }
  98. func (s *Topo) addEdge(from api.TopNode, to api.TopNode, toType string) {
  99. fromType := "op"
  100. if _, ok := from.(node.DataSourceNode); ok {
  101. fromType = "source"
  102. }
  103. f := fmt.Sprintf("%s_%s", fromType, from.GetName())
  104. t := fmt.Sprintf("%s_%s", toType, to.GetName())
  105. e, ok := s.topo.Edges[f]
  106. if !ok {
  107. e = make([]interface{}, 0)
  108. }
  109. s.topo.Edges[f] = append(e, t)
  110. }
  111. // prepareContext setups internal context before
  112. // stream starts execution.
  113. func (s *Topo) prepareContext() {
  114. if s.ctx == nil || s.ctx.Err() != nil {
  115. contextLogger := conf.Log.WithField("rule", s.name)
  116. if s.options.Debug || s.options.LogFilename != "" {
  117. contextLogger.Logger = &logrus.Logger{
  118. Out: conf.Log.Out,
  119. Hooks: conf.Log.Hooks,
  120. Level: conf.Log.Level,
  121. Formatter: conf.Log.Formatter,
  122. ReportCaller: conf.Log.ReportCaller,
  123. ExitFunc: conf.Log.ExitFunc,
  124. BufferPool: conf.Log.BufferPool,
  125. }
  126. if conf.Config.Basic.Debug || s.options.Debug {
  127. contextLogger.Logger.SetLevel(logrus.DebugLevel)
  128. }
  129. if s.options.LogFilename != "" {
  130. logDir, _ := conf.GetLogLoc()
  131. file := path.Join(logDir, path.Base(s.options.LogFilename))
  132. output, err := rotatelogs.New(
  133. file+".%Y-%m-%d_%H-%M-%S",
  134. rotatelogs.WithLinkName(file),
  135. rotatelogs.WithRotationTime(time.Hour*time.Duration(conf.Config.Basic.RotateTime)),
  136. rotatelogs.WithMaxAge(time.Hour*time.Duration(conf.Config.Basic.MaxAge)),
  137. )
  138. if err != nil {
  139. conf.Log.Warnf("Create rule log file failed: %s", file)
  140. } else if conf.Config.Basic.ConsoleLog {
  141. contextLogger.Logger.SetOutput(io.MultiWriter(output, os.Stdout))
  142. } else if !conf.Config.Basic.ConsoleLog {
  143. contextLogger.Logger.SetOutput(output)
  144. }
  145. }
  146. }
  147. ctx := kctx.WithValue(kctx.Background(), kctx.LoggerKey, contextLogger)
  148. s.ctx, s.cancel = ctx.WithCancel()
  149. }
  150. }
  151. func (s *Topo) Open() <-chan error {
  152. // if stream has opened, do nothing
  153. if s.ctx != nil && s.ctx.Err() == nil {
  154. s.ctx.GetLogger().Infoln("rule is already running, do nothing")
  155. return s.drain
  156. }
  157. s.prepareContext() // ensure context is set
  158. s.drain = make(chan error)
  159. log := s.ctx.GetLogger()
  160. log.Infoln("Opening stream")
  161. go func() {
  162. err := infra.SafeRun(func() error {
  163. s.mu.Lock()
  164. defer s.mu.Unlock()
  165. var err error
  166. if s.store, err = state.CreateStore(s.name, s.options.Qos); err != nil {
  167. return fmt.Errorf("topo %s create store error %v", s.name, err)
  168. }
  169. s.enableCheckpoint()
  170. // open stream sink, after log sink is ready.
  171. for _, snk := range s.sinks {
  172. snk.Open(s.ctx.WithMeta(s.name, snk.GetName(), s.store), s.drain)
  173. }
  174. // apply operators, if err bail
  175. for _, op := range s.ops {
  176. op.Exec(s.ctx.WithMeta(s.name, op.GetName(), s.store), s.drain)
  177. }
  178. // open source, if err bail
  179. for _, source := range s.sources {
  180. source.Open(s.ctx.WithMeta(s.name, source.GetName(), s.store), s.drain)
  181. }
  182. // activate checkpoint
  183. if s.coordinator != nil {
  184. s.coordinator.Activate()
  185. }
  186. return nil
  187. })
  188. if err != nil {
  189. infra.DrainError(s.ctx, err, s.drain)
  190. }
  191. }()
  192. return s.drain
  193. }
  194. func (s *Topo) enableCheckpoint() error {
  195. if s.options.Qos >= api.AtLeastOnce {
  196. var sources []checkpoint.StreamTask
  197. for _, r := range s.sources {
  198. sources = append(sources, r)
  199. }
  200. var ops []checkpoint.NonSourceTask
  201. for _, r := range s.ops {
  202. ops = append(ops, r)
  203. }
  204. var sinks []checkpoint.SinkTask
  205. for _, r := range s.sinks {
  206. sinks = append(sinks, r)
  207. }
  208. c := checkpoint.NewCoordinator(s.name, sources, ops, sinks, s.options.Qos, s.store, s.options.CheckpointInterval, s.ctx)
  209. s.coordinator = c
  210. }
  211. return nil
  212. }
  213. func (s *Topo) GetCoordinator() *checkpoint.Coordinator {
  214. return s.coordinator
  215. }
  216. func (s *Topo) GetMetrics() (keys []string, values []interface{}) {
  217. for _, sn := range s.sources {
  218. for ins, metrics := range sn.GetMetrics() {
  219. for i, v := range metrics {
  220. keys = append(keys, "source_"+sn.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  221. values = append(values, v)
  222. }
  223. }
  224. }
  225. for _, so := range s.ops {
  226. for ins, metrics := range so.GetMetrics() {
  227. for i, v := range metrics {
  228. keys = append(keys, "op_"+so.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  229. values = append(values, v)
  230. }
  231. }
  232. }
  233. for _, sn := range s.sinks {
  234. for ins, metrics := range sn.GetMetrics() {
  235. for i, v := range metrics {
  236. keys = append(keys, "sink_"+sn.GetName()+"_"+strconv.Itoa(ins)+"_"+metric.MetricNames[i])
  237. values = append(values, v)
  238. }
  239. }
  240. }
  241. return
  242. }
  243. func (s *Topo) RemoveMetrics() {
  244. for _, sn := range s.sources {
  245. sn.RemoveMetrics(s.name)
  246. }
  247. for _, so := range s.ops {
  248. so.RemoveMetrics(s.name)
  249. }
  250. for _, sn := range s.sinks {
  251. sn.RemoveMetrics(s.name)
  252. }
  253. }
  254. func (s *Topo) GetTopo() *api.PrintableTopo {
  255. return s.topo
  256. }