planner.go 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384
  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 *AnalyticFuncsPlan:
  150. op = Transform(&operator.AnalyticFuncsOp{Funcs: t.funcs}, fmt.Sprintf("%d_analytic", newIndex), options)
  151. case *WindowPlan:
  152. if t.condition != nil {
  153. wfilterOp := Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_windowFilter", newIndex), options)
  154. wfilterOp.SetConcurrency(options.Concurrency)
  155. tp.AddOperator(inputs, wfilterOp)
  156. inputs = []api.Emitter{wfilterOp}
  157. }
  158. op, err = node.NewWindowOp(fmt.Sprintf("%d_window", newIndex), node.WindowConfig{
  159. Type: t.wtype,
  160. Length: t.length,
  161. Interval: t.interval,
  162. }, streamsFromStmt, options)
  163. if err != nil {
  164. return nil, 0, err
  165. }
  166. case *LookupPlan:
  167. op, err = node.NewLookupNode(t.joinExpr.Name, t.fields, t.keys, t.joinExpr.JoinType, t.valvars, t.options, options)
  168. case *JoinAlignPlan:
  169. op, err = node.NewJoinAlignNode(fmt.Sprintf("%d_join_aligner", newIndex), t.Emitters, options)
  170. case *JoinPlan:
  171. op = Transform(&operator.JoinOp{Joins: t.joins, From: t.from}, fmt.Sprintf("%d_join", newIndex), options)
  172. case *FilterPlan:
  173. op = Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_filter", newIndex), options)
  174. case *AggregatePlan:
  175. op = Transform(&operator.AggregateOp{Dimensions: t.dimensions}, fmt.Sprintf("%d_aggregate", newIndex), options)
  176. case *HavingPlan:
  177. op = Transform(&operator.HavingOp{Condition: t.condition}, fmt.Sprintf("%d_having", newIndex), options)
  178. case *OrderPlan:
  179. op = Transform(&operator.OrderOp{SortFields: t.SortFields}, fmt.Sprintf("%d_order", newIndex), options)
  180. case *ProjectPlan:
  181. 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)
  182. default:
  183. return nil, 0, fmt.Errorf("unknown logical plan %v", t)
  184. }
  185. if uop, ok := op.(*node.UnaryOperator); ok {
  186. uop.SetConcurrency(options.Concurrency)
  187. }
  188. if onode, ok := op.(node.OperatorNode); ok {
  189. tp.AddOperator(inputs, onode)
  190. }
  191. return op, newIndex, nil
  192. }
  193. func getMockSource(sources []*node.SourceNode, name string) *node.SourceNode {
  194. for _, source := range sources {
  195. if name == source.GetName() {
  196. return source
  197. }
  198. }
  199. return nil
  200. }
  201. func createLogicalPlan(stmt *ast.SelectStatement, opt *api.RuleOption, store kv.KeyValue) (LogicalPlan, error) {
  202. dimensions := stmt.Dimensions
  203. var (
  204. p LogicalPlan
  205. children []LogicalPlan
  206. // If there are tables, the plan graph will be different for join/window
  207. lookupTableChildren map[string]*ast.Options
  208. scanTableChildren []LogicalPlan
  209. scanTableEmitters []string
  210. w *ast.Window
  211. ds ast.Dimensions
  212. )
  213. streamStmts, analyticFuncs, err := decorateStmt(stmt, store)
  214. if err != nil {
  215. return nil, err
  216. }
  217. for _, streamStmt := range streamStmts {
  218. if streamStmt.StreamType == ast.TypeTable && streamStmt.Options.KIND == ast.StreamKindLookup {
  219. if lookupTableChildren == nil {
  220. lookupTableChildren = make(map[string]*ast.Options)
  221. }
  222. lookupTableChildren[string(streamStmt.Name)] = streamStmt.Options
  223. } else {
  224. p = DataSourcePlan{
  225. name: streamStmt.Name,
  226. streamStmt: streamStmt,
  227. iet: opt.IsEventTime,
  228. allMeta: opt.SendMetaToSink,
  229. }.Init()
  230. if streamStmt.StreamType == ast.TypeStream {
  231. children = append(children, p)
  232. } else {
  233. scanTableChildren = append(scanTableChildren, p)
  234. scanTableEmitters = append(scanTableEmitters, string(streamStmt.Name))
  235. }
  236. }
  237. }
  238. if len(analyticFuncs) > 0 {
  239. p = AnalyticFuncsPlan{
  240. funcs: analyticFuncs,
  241. }.Init()
  242. p.SetChildren(children)
  243. children = []LogicalPlan{p}
  244. }
  245. if dimensions != nil {
  246. w = dimensions.GetWindow()
  247. if w != nil {
  248. if len(children) == 0 {
  249. return nil, errors.New("cannot run window for TABLE sources")
  250. }
  251. wp := WindowPlan{
  252. wtype: w.WindowType,
  253. length: w.Length.Val,
  254. isEventTime: opt.IsEventTime,
  255. }.Init()
  256. if w.Interval != nil {
  257. wp.interval = w.Interval.Val
  258. } else if w.WindowType == ast.COUNT_WINDOW {
  259. //if no interval value is set, and it's count window, then set interval to length value.
  260. wp.interval = w.Length.Val
  261. }
  262. if w.Filter != nil {
  263. wp.condition = w.Filter
  264. }
  265. // TODO calculate limit
  266. // TODO incremental aggregate
  267. wp.SetChildren(children)
  268. children = []LogicalPlan{wp}
  269. p = wp
  270. }
  271. }
  272. if stmt.Joins != nil {
  273. if len(lookupTableChildren) == 0 && len(scanTableChildren) == 0 && w == nil {
  274. return nil, errors.New("a time window or count window is required to join multiple streams")
  275. }
  276. if len(lookupTableChildren) > 0 {
  277. var joins []ast.Join
  278. for _, join := range stmt.Joins {
  279. if streamOpt, ok := lookupTableChildren[join.Name]; ok {
  280. lookupPlan := LookupPlan{
  281. joinExpr: join,
  282. options: streamOpt,
  283. }
  284. if !lookupPlan.validateAndExtractCondition() {
  285. return nil, fmt.Errorf("join condition %s is invalid, at least one equi-join predicate is required", join.Expr)
  286. }
  287. p = lookupPlan.Init()
  288. p.SetChildren(children)
  289. children = []LogicalPlan{p}
  290. delete(lookupTableChildren, join.Name)
  291. } else {
  292. joins = append(joins, join)
  293. }
  294. }
  295. if len(lookupTableChildren) > 0 {
  296. return nil, fmt.Errorf("cannot find lookup table %v in any join", lookupTableChildren)
  297. }
  298. stmt.Joins = joins
  299. }
  300. // Not all joins are lookup joins, so we need to create a join plan for the remaining joins
  301. if len(stmt.Joins) > 0 {
  302. if len(scanTableChildren) > 0 {
  303. p = JoinAlignPlan{
  304. Emitters: scanTableEmitters,
  305. }.Init()
  306. p.SetChildren(append(children, scanTableChildren...))
  307. children = []LogicalPlan{p}
  308. }
  309. // TODO extract on filter
  310. p = JoinPlan{
  311. from: stmt.Sources[0].(*ast.Table),
  312. joins: stmt.Joins,
  313. }.Init()
  314. p.SetChildren(children)
  315. children = []LogicalPlan{p}
  316. }
  317. }
  318. if stmt.Condition != nil {
  319. p = FilterPlan{
  320. condition: stmt.Condition,
  321. }.Init()
  322. p.SetChildren(children)
  323. children = []LogicalPlan{p}
  324. }
  325. // TODO handle aggregateAlias in optimization as it does not only happen in select fields
  326. if dimensions != nil {
  327. ds = dimensions.GetGroups()
  328. if ds != nil && len(ds) > 0 {
  329. p = AggregatePlan{
  330. dimensions: ds,
  331. }.Init()
  332. p.SetChildren(children)
  333. children = []LogicalPlan{p}
  334. }
  335. }
  336. if stmt.Having != nil {
  337. p = HavingPlan{
  338. condition: stmt.Having,
  339. }.Init()
  340. p.SetChildren(children)
  341. children = []LogicalPlan{p}
  342. }
  343. if stmt.SortFields != nil {
  344. p = OrderPlan{
  345. SortFields: stmt.SortFields,
  346. }.Init()
  347. p.SetChildren(children)
  348. children = []LogicalPlan{p}
  349. }
  350. if stmt.Fields != nil {
  351. p = ProjectPlan{
  352. fields: stmt.Fields,
  353. isAggregate: xsql.IsAggStatement(stmt),
  354. sendMeta: opt.SendMetaToSink,
  355. }.Init()
  356. p.SetChildren(children)
  357. }
  358. return optimize(p)
  359. }
  360. func Transform(op node.UnOperation, name string, options *api.RuleOption) *node.UnaryOperator {
  361. unaryOperator := node.New(name, options)
  362. unaryOperator.SetOperation(op)
  363. return unaryOperator
  364. }