planner.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432
  1. // Copyright 2022-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 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. store, err := 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. srcNode, err := transformSourceNode(t, sources, options)
  114. if err != nil {
  115. return nil, 0, err
  116. }
  117. tp.AddSrc(srcNode)
  118. inputs = []api.Emitter{srcNode}
  119. op = srcNode
  120. case *AnalyticFuncsPlan:
  121. op = Transform(&operator.AnalyticFuncsOp{Funcs: t.funcs}, fmt.Sprintf("%d_analytic", newIndex), options)
  122. case *WindowPlan:
  123. if t.condition != nil {
  124. wfilterOp := Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_windowFilter", newIndex), options)
  125. wfilterOp.SetConcurrency(options.Concurrency)
  126. tp.AddOperator(inputs, wfilterOp)
  127. inputs = []api.Emitter{wfilterOp}
  128. }
  129. op, err = node.NewWindowOp(fmt.Sprintf("%d_window", newIndex), node.WindowConfig{
  130. Type: t.wtype,
  131. Length: t.length,
  132. Interval: t.interval,
  133. }, streamsFromStmt, options)
  134. if err != nil {
  135. return nil, 0, err
  136. }
  137. case *LookupPlan:
  138. op, err = node.NewLookupNode(t.joinExpr.Name, t.fields, t.keys, t.joinExpr.JoinType, t.valvars, t.options, options)
  139. case *JoinAlignPlan:
  140. op, err = node.NewJoinAlignNode(fmt.Sprintf("%d_join_aligner", newIndex), t.Emitters, options)
  141. case *JoinPlan:
  142. op = Transform(&operator.JoinOp{Joins: t.joins, From: t.from}, fmt.Sprintf("%d_join", newIndex), options)
  143. case *FilterPlan:
  144. op = Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_filter", newIndex), options)
  145. case *AggregatePlan:
  146. op = Transform(&operator.AggregateOp{Dimensions: t.dimensions}, fmt.Sprintf("%d_aggregate", newIndex), options)
  147. case *HavingPlan:
  148. op = Transform(&operator.HavingOp{Condition: t.condition}, fmt.Sprintf("%d_having", newIndex), options)
  149. case *OrderPlan:
  150. op = Transform(&operator.OrderOp{SortFields: t.SortFields}, fmt.Sprintf("%d_order", newIndex), options)
  151. case *ProjectPlan:
  152. 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)
  153. case *ProjectSetPlan:
  154. op = Transform(&operator.ProjectSetOperator{SrfMapping: t.SrfMapping}, fmt.Sprintf("%d_projectset", newIndex), options)
  155. default:
  156. err = fmt.Errorf("unknown logical plan %v", t)
  157. }
  158. if err != nil {
  159. return nil, 0, err
  160. }
  161. if uop, ok := op.(*node.UnaryOperator); ok {
  162. uop.SetConcurrency(options.Concurrency)
  163. }
  164. if onode, ok := op.(node.OperatorNode); ok {
  165. tp.AddOperator(inputs, onode)
  166. }
  167. return op, newIndex, nil
  168. }
  169. func transformSourceNode(t *DataSourcePlan, sources []*node.SourceNode, options *api.RuleOption) (*node.SourceNode, error) {
  170. isSchemaless := t.isSchemaless
  171. switch t.streamStmt.StreamType {
  172. case ast.TypeStream:
  173. var (
  174. pp node.UnOperation
  175. err error
  176. )
  177. if t.iet || (!isSchemaless && (t.streamStmt.Options.STRICT_VALIDATION || t.isBinary)) {
  178. pp, err = operator.NewPreprocessor(isSchemaless, t.streamFields, t.allMeta, t.metaFields, t.iet, t.timestampField, t.timestampFormat, t.isBinary, t.streamStmt.Options.STRICT_VALIDATION)
  179. if err != nil {
  180. return nil, err
  181. }
  182. }
  183. var srcNode *node.SourceNode
  184. if len(sources) == 0 {
  185. sourceNode := node.NewSourceNode(string(t.name), t.streamStmt.StreamType, pp, t.streamStmt.Options, options.SendError)
  186. srcNode = sourceNode
  187. } else {
  188. srcNode = getMockSource(sources, string(t.name))
  189. if srcNode == nil {
  190. return nil, fmt.Errorf("can't find predefined source %s", t.name)
  191. }
  192. }
  193. return srcNode, nil
  194. case ast.TypeTable:
  195. pp, err := operator.NewTableProcessor(isSchemaless, string(t.name), t.streamFields, t.streamStmt.Options)
  196. if err != nil {
  197. return nil, err
  198. }
  199. var srcNode *node.SourceNode
  200. if len(sources) > 0 {
  201. srcNode = getMockSource(sources, string(t.name))
  202. }
  203. if srcNode == nil {
  204. srcNode = node.NewSourceNode(string(t.name), t.streamStmt.StreamType, pp, t.streamStmt.Options, options.SendError)
  205. }
  206. return srcNode, nil
  207. }
  208. return nil, fmt.Errorf("unknown stream type %d", t.streamStmt.StreamType)
  209. }
  210. func getMockSource(sources []*node.SourceNode, name string) *node.SourceNode {
  211. for _, source := range sources {
  212. if name == source.GetName() {
  213. return source
  214. }
  215. }
  216. return nil
  217. }
  218. func createLogicalPlan(stmt *ast.SelectStatement, opt *api.RuleOption, store kv.KeyValue) (LogicalPlan, error) {
  219. dimensions := stmt.Dimensions
  220. var (
  221. p LogicalPlan
  222. children []LogicalPlan
  223. // If there are tables, the plan graph will be different for join/window
  224. lookupTableChildren map[string]*ast.Options
  225. scanTableChildren []LogicalPlan
  226. scanTableEmitters []string
  227. w *ast.Window
  228. ds ast.Dimensions
  229. )
  230. streamStmts, analyticFuncs, err := decorateStmt(stmt, store)
  231. if err != nil {
  232. return nil, err
  233. }
  234. for _, sInfo := range streamStmts {
  235. if sInfo.stmt.StreamType == ast.TypeTable && sInfo.stmt.Options.KIND == ast.StreamKindLookup {
  236. if lookupTableChildren == nil {
  237. lookupTableChildren = make(map[string]*ast.Options)
  238. }
  239. lookupTableChildren[string(sInfo.stmt.Name)] = sInfo.stmt.Options
  240. } else {
  241. p = DataSourcePlan{
  242. name: sInfo.stmt.Name,
  243. streamStmt: sInfo.stmt,
  244. streamFields: sInfo.schema.ToJsonSchema(),
  245. isSchemaless: sInfo.schema == nil,
  246. iet: opt.IsEventTime,
  247. allMeta: opt.SendMetaToSink,
  248. }.Init()
  249. if sInfo.stmt.StreamType == ast.TypeStream {
  250. children = append(children, p)
  251. } else {
  252. scanTableChildren = append(scanTableChildren, p)
  253. scanTableEmitters = append(scanTableEmitters, string(sInfo.stmt.Name))
  254. }
  255. }
  256. }
  257. if len(analyticFuncs) > 0 {
  258. p = AnalyticFuncsPlan{
  259. funcs: analyticFuncs,
  260. }.Init()
  261. p.SetChildren(children)
  262. children = []LogicalPlan{p}
  263. }
  264. if dimensions != nil {
  265. w = dimensions.GetWindow()
  266. if w != nil {
  267. if len(children) == 0 {
  268. return nil, errors.New("cannot run window for TABLE sources")
  269. }
  270. wp := WindowPlan{
  271. wtype: w.WindowType,
  272. length: w.Length.Val,
  273. isEventTime: opt.IsEventTime,
  274. }.Init()
  275. if w.Interval != nil {
  276. wp.interval = w.Interval.Val
  277. } else if w.WindowType == ast.COUNT_WINDOW {
  278. // if no interval value is set, and it's count window, then set interval to length value.
  279. wp.interval = w.Length.Val
  280. }
  281. if w.Filter != nil {
  282. wp.condition = w.Filter
  283. }
  284. // TODO calculate limit
  285. // TODO incremental aggregate
  286. wp.SetChildren(children)
  287. children = []LogicalPlan{wp}
  288. p = wp
  289. }
  290. }
  291. if stmt.Joins != nil {
  292. if len(lookupTableChildren) == 0 && len(scanTableChildren) == 0 && w == nil {
  293. return nil, errors.New("a time window or count window is required to join multiple streams")
  294. }
  295. if len(lookupTableChildren) > 0 {
  296. var joins []ast.Join
  297. for _, join := range stmt.Joins {
  298. if streamOpt, ok := lookupTableChildren[join.Name]; ok {
  299. lookupPlan := LookupPlan{
  300. joinExpr: join,
  301. options: streamOpt,
  302. }
  303. if !lookupPlan.validateAndExtractCondition() {
  304. return nil, fmt.Errorf("join condition %s is invalid, at least one equi-join predicate is required", join.Expr)
  305. }
  306. p = lookupPlan.Init()
  307. p.SetChildren(children)
  308. children = []LogicalPlan{p}
  309. delete(lookupTableChildren, join.Name)
  310. } else {
  311. joins = append(joins, join)
  312. }
  313. }
  314. if len(lookupTableChildren) > 0 {
  315. return nil, fmt.Errorf("cannot find lookup table %v in any join", lookupTableChildren)
  316. }
  317. stmt.Joins = joins
  318. }
  319. // Not all joins are lookup joins, so we need to create a join plan for the remaining joins
  320. if len(stmt.Joins) > 0 {
  321. if len(scanTableChildren) > 0 {
  322. p = JoinAlignPlan{
  323. Emitters: scanTableEmitters,
  324. }.Init()
  325. p.SetChildren(append(children, scanTableChildren...))
  326. children = []LogicalPlan{p}
  327. }
  328. // TODO extract on filter
  329. p = JoinPlan{
  330. from: stmt.Sources[0].(*ast.Table),
  331. joins: stmt.Joins,
  332. }.Init()
  333. p.SetChildren(children)
  334. children = []LogicalPlan{p}
  335. }
  336. }
  337. if stmt.Condition != nil {
  338. p = FilterPlan{
  339. condition: stmt.Condition,
  340. }.Init()
  341. p.SetChildren(children)
  342. children = []LogicalPlan{p}
  343. }
  344. if dimensions != nil {
  345. ds = dimensions.GetGroups()
  346. if ds != nil && len(ds) > 0 {
  347. p = AggregatePlan{
  348. dimensions: ds,
  349. }.Init()
  350. p.SetChildren(children)
  351. children = []LogicalPlan{p}
  352. }
  353. }
  354. if stmt.Having != nil {
  355. p = HavingPlan{
  356. condition: stmt.Having,
  357. }.Init()
  358. p.SetChildren(children)
  359. children = []LogicalPlan{p}
  360. }
  361. if stmt.SortFields != nil {
  362. p = OrderPlan{
  363. SortFields: stmt.SortFields,
  364. }.Init()
  365. p.SetChildren(children)
  366. children = []LogicalPlan{p}
  367. }
  368. if stmt.Fields != nil {
  369. p = ProjectPlan{
  370. fields: stmt.Fields,
  371. isAggregate: xsql.IsAggStatement(stmt),
  372. sendMeta: opt.SendMetaToSink,
  373. }.Init()
  374. p.SetChildren(children)
  375. children = []LogicalPlan{p}
  376. }
  377. srfMapping := extractSRFMapping(stmt)
  378. if len(srfMapping) > 0 {
  379. p = ProjectSetPlan{
  380. SrfMapping: srfMapping,
  381. }.Init()
  382. p.SetChildren(children)
  383. }
  384. return optimize(p)
  385. }
  386. // extractSRFMapping extracts the set-returning-function in the field
  387. func extractSRFMapping(stmt *ast.SelectStatement) map[string]struct{} {
  388. m := make(map[string]struct{})
  389. for _, field := range stmt.Fields {
  390. var curExpr ast.Expr
  391. var name string
  392. if len(field.AName) > 0 {
  393. curExpr = field.Expr.(*ast.FieldRef).AliasRef.Expression
  394. name = field.AName
  395. } else {
  396. curExpr = field.Expr
  397. name = field.Name
  398. }
  399. if f, ok := curExpr.(*ast.Call); ok && f.FuncType == ast.FuncTypeSrf {
  400. m[name] = struct{}{}
  401. }
  402. }
  403. return m
  404. }
  405. func Transform(op node.UnOperation, name string, options *api.RuleOption) *node.UnaryOperator {
  406. unaryOperator := node.New(name, options)
  407. unaryOperator.SetOperation(op)
  408. return unaryOperator
  409. }