planner.go 13 KB

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