planner.go 10.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330
  1. // Copyright 2021 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 planner
  15. import (
  16. "errors"
  17. "fmt"
  18. "github.com/lf-edge/ekuiper/internal/conf"
  19. "github.com/lf-edge/ekuiper/internal/topo"
  20. "github.com/lf-edge/ekuiper/internal/topo/node"
  21. "github.com/lf-edge/ekuiper/internal/topo/operator"
  22. "github.com/lf-edge/ekuiper/internal/xsql"
  23. "github.com/lf-edge/ekuiper/pkg/api"
  24. "github.com/lf-edge/ekuiper/pkg/ast"
  25. "github.com/lf-edge/ekuiper/pkg/kv"
  26. store2 "github.com/lf-edge/ekuiper/pkg/kv/stores"
  27. )
  28. func Plan(rule *api.Rule) (*topo.Topo, error) {
  29. return PlanWithSourcesAndSinks(rule, nil, nil)
  30. }
  31. // For test only
  32. func PlanWithSourcesAndSinks(rule *api.Rule, sources []*node.SourceNode, sinks []*node.SinkNode) (*topo.Topo, error) {
  33. sql := rule.Sql
  34. conf.Log.Infof("Init rule with options %+v", rule.Options)
  35. stmt, err := xsql.GetStatementFromSql(sql)
  36. if err != nil {
  37. return nil, err
  38. }
  39. // validation
  40. streamsFromStmt := xsql.GetStreams(stmt)
  41. //if len(sources) > 0 && len(sources) != len(streamsFromStmt) {
  42. // return nil, fmt.Errorf("Invalid parameter sources or streams, the length cannot match the statement, expect %d sources.", len(streamsFromStmt))
  43. //}
  44. if rule.Options.SendMetaToSink && (len(streamsFromStmt) > 1 || stmt.Dimensions != nil) {
  45. return nil, fmt.Errorf("Invalid option sendMetaToSink, it can not be applied to window")
  46. }
  47. err, store := kv.GetKV("stream")
  48. if err != nil {
  49. return nil, err
  50. }
  51. // Create logical plan and optimize. Logical plans are a linked list
  52. lp, err := createLogicalPlan(stmt, rule.Options, store)
  53. if err != nil {
  54. return nil, err
  55. }
  56. tp, err := createTopo(rule, lp, sources, sinks, streamsFromStmt)
  57. if err != nil {
  58. return nil, err
  59. }
  60. return tp, nil
  61. }
  62. func createTopo(rule *api.Rule, lp LogicalPlan, sources []*node.SourceNode, sinks []*node.SinkNode, streamsFromStmt []string) (*topo.Topo, error) {
  63. // Create topology
  64. tp, err := topo.NewWithNameAndQos(rule.Id, rule.Options.Qos, rule.Options.CheckpointInterval)
  65. if err != nil {
  66. return nil, err
  67. }
  68. input, _, err := buildOps(lp, tp, rule.Options, sources, streamsFromStmt, 0)
  69. if err != nil {
  70. return nil, err
  71. }
  72. inputs := []api.Emitter{input}
  73. // Add actions
  74. if len(sinks) > 0 { // For use of mock sink in testing
  75. for _, sink := range sinks {
  76. tp.AddSink(inputs, sink)
  77. }
  78. } else {
  79. for i, m := range rule.Actions {
  80. for name, action := range m {
  81. props, ok := action.(map[string]interface{})
  82. if !ok {
  83. return nil, fmt.Errorf("expect map[string]interface{} type for the action properties, but found %v", action)
  84. }
  85. tp.AddSink(inputs, node.NewSinkNode(fmt.Sprintf("%s_%d", name, i), name, props))
  86. }
  87. }
  88. }
  89. return tp, nil
  90. }
  91. func buildOps(lp LogicalPlan, tp *topo.Topo, options *api.RuleOption, sources []*node.SourceNode, streamsFromStmt []string, index int) (api.Emitter, int, error) {
  92. var inputs []api.Emitter
  93. newIndex := index
  94. for _, c := range lp.Children() {
  95. input, ni, err := buildOps(c, tp, options, sources, streamsFromStmt, newIndex)
  96. if err != nil {
  97. return nil, 0, err
  98. }
  99. newIndex = ni
  100. inputs = append(inputs, input)
  101. }
  102. newIndex++
  103. var (
  104. op node.OperatorNode
  105. err error
  106. )
  107. switch t := lp.(type) {
  108. case *DataSourcePlan:
  109. switch t.streamStmt.StreamType {
  110. case ast.TypeStream:
  111. pp, err := operator.NewPreprocessor(t.streamFields, t.allMeta, t.metaFields, t.iet, t.timestampField, t.timestampFormat, t.isBinary, t.streamStmt.Options.STRICT_VALIDATION)
  112. if err != nil {
  113. return nil, 0, err
  114. }
  115. var srcNode *node.SourceNode
  116. if len(sources) == 0 {
  117. sourceNode := node.NewSourceNode(string(t.name), t.streamStmt.StreamType, t.streamStmt.Options)
  118. srcNode = sourceNode
  119. } else {
  120. srcNode = getMockSource(sources, string(t.name))
  121. if srcNode == nil {
  122. return nil, 0, fmt.Errorf("can't find predefined source %s", t.name)
  123. }
  124. }
  125. tp.AddSrc(srcNode)
  126. op = Transform(pp, fmt.Sprintf("%d_preprocessor_%s", newIndex, t.name), options)
  127. inputs = []api.Emitter{srcNode}
  128. case ast.TypeTable:
  129. pp, err := operator.NewTableProcessor(string(t.name), t.streamFields, t.streamStmt.Options)
  130. if err != nil {
  131. return nil, 0, err
  132. }
  133. var srcNode *node.SourceNode
  134. if len(sources) > 0 {
  135. srcNode = getMockSource(sources, string(t.name))
  136. }
  137. if srcNode == nil {
  138. srcNode = node.NewSourceNode(string(t.name), t.streamStmt.StreamType, t.streamStmt.Options)
  139. }
  140. tp.AddSrc(srcNode)
  141. op = Transform(pp, fmt.Sprintf("%d_tableprocessor_%s", newIndex, t.name), options)
  142. inputs = []api.Emitter{srcNode}
  143. }
  144. case *WindowPlan:
  145. if t.condition != nil {
  146. wfilterOp := Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_windowFilter", newIndex), options)
  147. wfilterOp.SetConcurrency(options.Concurrency)
  148. tp.AddOperator(inputs, wfilterOp)
  149. inputs = []api.Emitter{wfilterOp}
  150. }
  151. op, err = node.NewWindowOp(fmt.Sprintf("%d_window", newIndex), node.WindowConfig{
  152. Type: t.wtype,
  153. Length: t.length,
  154. Interval: t.interval,
  155. }, streamsFromStmt, options)
  156. if err != nil {
  157. return nil, 0, err
  158. }
  159. case *JoinAlignPlan:
  160. op, err = node.NewJoinAlignNode(fmt.Sprintf("%d_join_aligner", newIndex), t.Emitters, options)
  161. case *JoinPlan:
  162. op = Transform(&operator.JoinOp{Joins: t.joins, From: t.from}, fmt.Sprintf("%d_join", newIndex), options)
  163. case *FilterPlan:
  164. op = Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_filter", newIndex), options)
  165. case *AggregatePlan:
  166. op = Transform(&operator.AggregateOp{Dimensions: t.dimensions}, fmt.Sprintf("%d_aggregate", newIndex), options)
  167. case *HavingPlan:
  168. op = Transform(&operator.HavingOp{Condition: t.condition}, fmt.Sprintf("%d_having", newIndex), options)
  169. case *OrderPlan:
  170. op = Transform(&operator.OrderOp{SortFields: t.SortFields}, fmt.Sprintf("%d_order", newIndex), options)
  171. case *ProjectPlan:
  172. op = Transform(&operator.ProjectOp{Fields: t.fields, IsAggregate: t.isAggregate, SendMeta: t.sendMeta}, fmt.Sprintf("%d_project", newIndex), options)
  173. default:
  174. return nil, 0, fmt.Errorf("unknown logical plan %v", t)
  175. }
  176. if uop, ok := op.(*node.UnaryOperator); ok {
  177. uop.SetConcurrency(options.Concurrency)
  178. }
  179. tp.AddOperator(inputs, op)
  180. return op, newIndex, nil
  181. }
  182. func getMockSource(sources []*node.SourceNode, name string) *node.SourceNode {
  183. for _, source := range sources {
  184. if name == source.GetName() {
  185. return source
  186. }
  187. }
  188. return nil
  189. }
  190. func createLogicalPlan(stmt *ast.SelectStatement, opt *api.RuleOption, store store2.KeyValue) (LogicalPlan, error) {
  191. dimensions := stmt.Dimensions
  192. var (
  193. p LogicalPlan
  194. children []LogicalPlan
  195. // If there are tables, the plan graph will be different for join/window
  196. tableChildren []LogicalPlan
  197. tableEmitters []string
  198. w *ast.Window
  199. ds ast.Dimensions
  200. )
  201. streamStmts, err := decorateStmt(stmt, store)
  202. if err != nil {
  203. return nil, err
  204. }
  205. for _, streamStmt := range streamStmts {
  206. p = DataSourcePlan{
  207. name: streamStmt.Name,
  208. streamStmt: streamStmt,
  209. iet: opt.IsEventTime,
  210. allMeta: opt.SendMetaToSink,
  211. }.Init()
  212. if streamStmt.StreamType == ast.TypeStream {
  213. children = append(children, p)
  214. } else {
  215. tableChildren = append(tableChildren, p)
  216. tableEmitters = append(tableEmitters, string(streamStmt.Name))
  217. }
  218. }
  219. if dimensions != nil {
  220. w = dimensions.GetWindow()
  221. if w != nil {
  222. if len(children) == 0 {
  223. return nil, errors.New("cannot run window for TABLE sources")
  224. }
  225. wp := WindowPlan{
  226. wtype: w.WindowType,
  227. length: w.Length.Val,
  228. isEventTime: opt.IsEventTime,
  229. }.Init()
  230. if w.Interval != nil {
  231. wp.interval = w.Interval.Val
  232. } else if w.WindowType == ast.COUNT_WINDOW {
  233. //if no interval value is set and it's count window, then set interval to length value.
  234. wp.interval = w.Length.Val
  235. }
  236. if w.Filter != nil {
  237. wp.condition = w.Filter
  238. }
  239. // TODO calculate limit
  240. // TODO incremental aggregate
  241. wp.SetChildren(children)
  242. children = []LogicalPlan{wp}
  243. p = wp
  244. }
  245. }
  246. if stmt.Joins != nil {
  247. if len(tableChildren) > 0 {
  248. p = JoinAlignPlan{
  249. Emitters: tableEmitters,
  250. }.Init()
  251. p.SetChildren(append(children, tableChildren...))
  252. children = []LogicalPlan{p}
  253. } else if w == nil {
  254. return nil, errors.New("a time window or count window is required to join multiple streams")
  255. }
  256. // TODO extract on filter
  257. p = JoinPlan{
  258. from: stmt.Sources[0].(*ast.Table),
  259. joins: stmt.Joins,
  260. }.Init()
  261. p.SetChildren(children)
  262. children = []LogicalPlan{p}
  263. }
  264. if stmt.Condition != nil {
  265. p = FilterPlan{
  266. condition: stmt.Condition,
  267. }.Init()
  268. p.SetChildren(children)
  269. children = []LogicalPlan{p}
  270. }
  271. // TODO handle aggregateAlias in optimization as it does not only happen in select fields
  272. if dimensions != nil {
  273. ds = dimensions.GetGroups()
  274. if ds != nil && len(ds) > 0 {
  275. p = AggregatePlan{
  276. dimensions: ds,
  277. }.Init()
  278. p.SetChildren(children)
  279. children = []LogicalPlan{p}
  280. }
  281. }
  282. if stmt.Having != nil {
  283. p = HavingPlan{
  284. condition: stmt.Having,
  285. }.Init()
  286. p.SetChildren(children)
  287. children = []LogicalPlan{p}
  288. }
  289. if stmt.SortFields != nil {
  290. p = OrderPlan{
  291. SortFields: stmt.SortFields,
  292. }.Init()
  293. p.SetChildren(children)
  294. children = []LogicalPlan{p}
  295. }
  296. if stmt.Fields != nil {
  297. p = ProjectPlan{
  298. fields: stmt.Fields,
  299. isAggregate: ast.IsAggStatement(stmt),
  300. sendMeta: opt.SendMetaToSink,
  301. }.Init()
  302. p.SetChildren(children)
  303. }
  304. return optimize(p)
  305. }
  306. func Transform(op node.UnOperation, name string, options *api.RuleOption) *node.UnaryOperator {
  307. unaryOperator := node.New(name, xsql.FuncRegisters, options)
  308. unaryOperator.SetOperation(op)
  309. return unaryOperator
  310. }