planner.go 12 KB

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