planner.go 17 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558
  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 the 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.NewWithNameAndOptions(rule.Id, rule.Options)
  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 *WatermarkPlan:
  121. op = node.NewWatermarkOp(fmt.Sprintf("%d_watermark", newIndex), t.SendWatermark, t.Emitters, options)
  122. case *AnalyticFuncsPlan:
  123. op = Transform(&operator.AnalyticFuncsOp{Funcs: t.funcs, FieldFuncs: t.fieldFuncs}, fmt.Sprintf("%d_analytic", newIndex), options)
  124. case *WindowPlan:
  125. if t.condition != nil {
  126. wfilterOp := Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_windowFilter", newIndex), options)
  127. wfilterOp.SetConcurrency(options.Concurrency)
  128. tp.AddOperator(inputs, wfilterOp)
  129. inputs = []api.Emitter{wfilterOp}
  130. }
  131. l, i, d := convertFromDuration(t)
  132. var rawInterval int
  133. switch t.wtype {
  134. case ast.TUMBLING_WINDOW, ast.SESSION_WINDOW:
  135. rawInterval = t.length
  136. case ast.HOPPING_WINDOW:
  137. rawInterval = t.interval
  138. }
  139. t.ExtractStateFunc()
  140. op, err = node.NewWindowOp(fmt.Sprintf("%d_window", newIndex), node.WindowConfig{
  141. Type: t.wtype,
  142. Delay: d,
  143. Length: l,
  144. Interval: i,
  145. RawInterval: rawInterval,
  146. TimeUnit: t.timeUnit,
  147. TriggerCondition: t.triggerCondition,
  148. StateFuncs: t.stateFuncs,
  149. }, options)
  150. if err != nil {
  151. return nil, 0, err
  152. }
  153. case *LookupPlan:
  154. op, err = node.NewLookupNode(t.joinExpr.Name, t.fields, t.keys, t.joinExpr.JoinType, t.valvars, t.options, options)
  155. case *JoinAlignPlan:
  156. op, err = node.NewJoinAlignNode(fmt.Sprintf("%d_join_aligner", newIndex), t.Emitters, options)
  157. case *JoinPlan:
  158. op = Transform(&operator.JoinOp{Joins: t.joins, From: t.from}, fmt.Sprintf("%d_join", newIndex), options)
  159. case *FilterPlan:
  160. t.ExtractStateFunc()
  161. op = Transform(&operator.FilterOp{Condition: t.condition, StateFuncs: t.stateFuncs}, fmt.Sprintf("%d_filter", newIndex), options)
  162. case *AggregatePlan:
  163. op = Transform(&operator.AggregateOp{Dimensions: t.dimensions}, fmt.Sprintf("%d_aggregate", newIndex), options)
  164. case *HavingPlan:
  165. t.ExtractStateFunc()
  166. op = Transform(&operator.HavingOp{Condition: t.condition, StateFuncs: t.stateFuncs}, fmt.Sprintf("%d_having", newIndex), options)
  167. case *OrderPlan:
  168. op = Transform(&operator.OrderOp{SortFields: t.SortFields}, fmt.Sprintf("%d_order", newIndex), options)
  169. case *ProjectPlan:
  170. op = Transform(&operator.ProjectOp{ColNames: t.colNames, AliasNames: t.aliasNames, AliasFields: t.aliasFields, ExprFields: t.exprFields, ExceptNames: t.exceptNames, IsAggregate: t.isAggregate, AllWildcard: t.allWildcard, WildcardEmitters: t.wildcardEmitters, ExprNames: t.exprNames, SendMeta: t.sendMeta, LimitCount: t.limitCount, EnableLimit: t.enableLimit, WindowFuncNames: t.windowFuncNames}, fmt.Sprintf("%d_project", newIndex), options)
  171. case *ProjectSetPlan:
  172. op = Transform(&operator.ProjectSetOperator{SrfMapping: t.SrfMapping, LimitCount: t.limitCount, EnableLimit: t.enableLimit}, fmt.Sprintf("%d_projectset", newIndex), options)
  173. case *WindowFuncPlan:
  174. op = Transform(&operator.WindowFuncOperator{WindowFuncFields: t.windowFuncFields}, fmt.Sprintf("%d_windowFunc", newIndex), options)
  175. default:
  176. err = fmt.Errorf("unknown logical plan %v", t)
  177. }
  178. if err != nil {
  179. return nil, 0, err
  180. }
  181. if uop, ok := op.(*node.UnaryOperator); ok {
  182. uop.SetConcurrency(options.Concurrency)
  183. }
  184. if onode, ok := op.(node.OperatorNode); ok {
  185. tp.AddOperator(inputs, onode)
  186. }
  187. return op, newIndex, nil
  188. }
  189. func convertFromUnit(t ast.Token, v int64) int64 {
  190. unit := 1
  191. switch t {
  192. case ast.DD:
  193. unit = 24 * 3600 * 1000
  194. case ast.HH:
  195. unit = 3600 * 1000
  196. case ast.MI:
  197. unit = 60 * 1000
  198. case ast.SS:
  199. unit = 1000
  200. case ast.MS:
  201. unit = 1
  202. }
  203. return int64(unit) * v
  204. }
  205. func convertFromDuration(t *WindowPlan) (int64, int64, int64) {
  206. var unit int64 = 1
  207. switch t.timeUnit {
  208. case ast.DD:
  209. unit = 24 * 3600 * 1000
  210. case ast.HH:
  211. unit = 3600 * 1000
  212. case ast.MI:
  213. unit = 60 * 1000
  214. case ast.SS:
  215. unit = 1000
  216. case ast.MS:
  217. unit = 1
  218. }
  219. return int64(t.length) * unit, int64(t.interval) * unit, t.delay * unit
  220. }
  221. func transformSourceNode(t *DataSourcePlan, sources []*node.SourceNode, options *api.RuleOption) (*node.SourceNode, error) {
  222. isSchemaless := t.isSchemaless
  223. switch t.streamStmt.StreamType {
  224. case ast.TypeStream:
  225. var (
  226. pp node.UnOperation
  227. err error
  228. )
  229. if t.iet || (!isSchemaless && (t.streamStmt.Options.STRICT_VALIDATION || t.isBinary)) {
  230. pp, err = operator.NewPreprocessor(isSchemaless, t.streamFields, t.allMeta, t.metaFields, t.iet, t.timestampField, t.timestampFormat, t.isBinary, t.streamStmt.Options.STRICT_VALIDATION)
  231. if err != nil {
  232. return nil, err
  233. }
  234. }
  235. var srcNode *node.SourceNode
  236. if len(sources) == 0 {
  237. var sourceNode *node.SourceNode
  238. schema := t.streamFields
  239. if t.isSchemaless {
  240. schema = nil
  241. }
  242. sourceNode = node.NewSourceNode(string(t.name), t.streamStmt.StreamType, pp, t.streamStmt.Options, options.SendError, schema)
  243. srcNode = sourceNode
  244. } else {
  245. srcNode = getMockSource(sources, string(t.name))
  246. if srcNode == nil {
  247. return nil, fmt.Errorf("can't find predefined source %s", t.name)
  248. }
  249. }
  250. return srcNode, nil
  251. case ast.TypeTable:
  252. pp, err := operator.NewTableProcessor(isSchemaless, string(t.name), t.streamFields, t.streamStmt.Options)
  253. if err != nil {
  254. return nil, err
  255. }
  256. var srcNode *node.SourceNode
  257. if len(sources) > 0 {
  258. srcNode = getMockSource(sources, string(t.name))
  259. }
  260. if srcNode == nil {
  261. schema := t.streamFields
  262. if t.isSchemaless {
  263. schema = nil
  264. }
  265. srcNode = node.NewSourceNode(string(t.name), t.streamStmt.StreamType, pp, t.streamStmt.Options, options.SendError, schema)
  266. }
  267. return srcNode, nil
  268. }
  269. return nil, fmt.Errorf("unknown stream type %d", t.streamStmt.StreamType)
  270. }
  271. func getMockSource(sources []*node.SourceNode, name string) *node.SourceNode {
  272. for _, source := range sources {
  273. if name == source.GetName() {
  274. return source
  275. }
  276. }
  277. return nil
  278. }
  279. func createLogicalPlan(stmt *ast.SelectStatement, opt *api.RuleOption, store kv.KeyValue) (LogicalPlan, error) {
  280. dimensions := stmt.Dimensions
  281. var (
  282. p LogicalPlan
  283. children []LogicalPlan
  284. // If there are tables, the plan graph will be different for join/window
  285. lookupTableChildren map[string]*ast.Options
  286. scanTableChildren []LogicalPlan
  287. scanTableEmitters []string
  288. streamEmitters []string
  289. w *ast.Window
  290. ds ast.Dimensions
  291. )
  292. streamStmts, analyticFuncs, analyticFieldFuncs, err := decorateStmt(stmt, store)
  293. if err != nil {
  294. return nil, err
  295. }
  296. for _, sInfo := range streamStmts {
  297. if sInfo.stmt.StreamType == ast.TypeTable && sInfo.stmt.Options.KIND == ast.StreamKindLookup {
  298. if lookupTableChildren == nil {
  299. lookupTableChildren = make(map[string]*ast.Options)
  300. }
  301. lookupTableChildren[string(sInfo.stmt.Name)] = sInfo.stmt.Options
  302. } else {
  303. p = DataSourcePlan{
  304. name: sInfo.stmt.Name,
  305. streamStmt: sInfo.stmt,
  306. streamFields: sInfo.schema.ToJsonSchema(),
  307. isSchemaless: sInfo.schema == nil,
  308. iet: opt.IsEventTime,
  309. allMeta: opt.SendMetaToSink,
  310. }.Init()
  311. if sInfo.stmt.StreamType == ast.TypeStream {
  312. children = append(children, p)
  313. streamEmitters = append(streamEmitters, string(sInfo.stmt.Name))
  314. } else {
  315. scanTableChildren = append(scanTableChildren, p)
  316. scanTableEmitters = append(scanTableEmitters, string(sInfo.stmt.Name))
  317. }
  318. }
  319. }
  320. hasWindow := dimensions != nil && dimensions.GetWindow() != nil
  321. if opt.IsEventTime {
  322. p = WatermarkPlan{
  323. SendWatermark: hasWindow,
  324. Emitters: streamEmitters,
  325. }.Init()
  326. p.SetChildren(children)
  327. children = []LogicalPlan{p}
  328. }
  329. if len(analyticFuncs) > 0 || len(analyticFieldFuncs) > 0 {
  330. p = AnalyticFuncsPlan{
  331. funcs: analyticFuncs,
  332. fieldFuncs: analyticFieldFuncs,
  333. }.Init()
  334. p.SetChildren(children)
  335. children = []LogicalPlan{p}
  336. }
  337. if dimensions != nil {
  338. w = dimensions.GetWindow()
  339. if w != nil {
  340. if len(children) == 0 {
  341. return nil, errors.New("cannot run window for TABLE sources")
  342. }
  343. wp := WindowPlan{
  344. wtype: w.WindowType,
  345. length: w.Length.Val,
  346. isEventTime: opt.IsEventTime,
  347. }.Init()
  348. if w.Delay != nil {
  349. wp.delay = int64(w.Delay.Val)
  350. }
  351. if w.Interval != nil {
  352. wp.interval = w.Interval.Val
  353. } else if w.WindowType == ast.COUNT_WINDOW {
  354. // if no interval value is set, and it's a count window, then set interval to length value.
  355. wp.interval = w.Length.Val
  356. }
  357. if w.TimeUnit != nil {
  358. wp.timeUnit = w.TimeUnit.Val
  359. }
  360. if w.Filter != nil {
  361. wp.condition = w.Filter
  362. }
  363. if w.TriggerCondition != nil {
  364. wp.triggerCondition = w.TriggerCondition
  365. }
  366. // TODO calculate limit
  367. // TODO incremental aggregate
  368. wp.SetChildren(children)
  369. children = []LogicalPlan{wp}
  370. p = wp
  371. }
  372. }
  373. if stmt.Joins != nil {
  374. if len(lookupTableChildren) == 0 && len(scanTableChildren) == 0 && w == nil {
  375. return nil, errors.New("a time window or count window is required to join multiple streams")
  376. }
  377. if len(lookupTableChildren) > 0 {
  378. var joins []ast.Join
  379. for _, join := range stmt.Joins {
  380. if streamOpt, ok := lookupTableChildren[join.Name]; ok {
  381. lookupPlan := LookupPlan{
  382. joinExpr: join,
  383. options: streamOpt,
  384. }
  385. if !lookupPlan.validateAndExtractCondition() {
  386. return nil, fmt.Errorf("join condition %s is invalid, at least one equi-join predicate is required", join.Expr)
  387. }
  388. p = lookupPlan.Init()
  389. p.SetChildren(children)
  390. children = []LogicalPlan{p}
  391. delete(lookupTableChildren, join.Name)
  392. } else {
  393. joins = append(joins, join)
  394. }
  395. }
  396. if len(lookupTableChildren) > 0 {
  397. return nil, fmt.Errorf("cannot find lookup table %v in any join", lookupTableChildren)
  398. }
  399. stmt.Joins = joins
  400. }
  401. // Not all joins are lookup joins, so we need to create a join plan for the remaining joins
  402. if len(stmt.Joins) > 0 {
  403. if len(scanTableChildren) > 0 {
  404. p = JoinAlignPlan{
  405. Emitters: scanTableEmitters,
  406. }.Init()
  407. p.SetChildren(append(children, scanTableChildren...))
  408. children = []LogicalPlan{p}
  409. }
  410. p = JoinPlan{
  411. from: stmt.Sources[0].(*ast.Table),
  412. joins: stmt.Joins,
  413. }.Init()
  414. p.SetChildren(children)
  415. children = []LogicalPlan{p}
  416. }
  417. }
  418. if stmt.Condition != nil {
  419. p = FilterPlan{
  420. condition: stmt.Condition,
  421. }.Init()
  422. p.SetChildren(children)
  423. children = []LogicalPlan{p}
  424. }
  425. if dimensions != nil {
  426. ds = dimensions.GetGroups()
  427. if ds != nil && len(ds) > 0 {
  428. p = AggregatePlan{
  429. dimensions: ds,
  430. }.Init()
  431. p.SetChildren(children)
  432. children = []LogicalPlan{p}
  433. }
  434. }
  435. if stmt.Having != nil {
  436. p = HavingPlan{
  437. condition: stmt.Having,
  438. }.Init()
  439. p.SetChildren(children)
  440. children = []LogicalPlan{p}
  441. }
  442. if stmt.SortFields != nil {
  443. p = OrderPlan{
  444. SortFields: stmt.SortFields,
  445. }.Init()
  446. p.SetChildren(children)
  447. children = []LogicalPlan{p}
  448. }
  449. windowFuncFields, windowFuncsNames := extractWindowFuncFields(stmt)
  450. if len(windowFuncFields) > 0 {
  451. p = WindowFuncPlan{
  452. windowFuncFields: windowFuncFields,
  453. }.Init()
  454. p.SetChildren(children)
  455. children = []LogicalPlan{p}
  456. }
  457. srfMapping := extractSRFMapping(stmt)
  458. if stmt.Fields != nil {
  459. enableLimit := false
  460. limitCount := 0
  461. if stmt.Limit != nil && len(srfMapping) == 0 {
  462. enableLimit = true
  463. limitCount = stmt.Limit.(*ast.LimitExpr).LimitCount.Val
  464. }
  465. p = ProjectPlan{
  466. windowFuncNames: windowFuncsNames,
  467. fields: stmt.Fields,
  468. isAggregate: xsql.WithAggFields(stmt),
  469. sendMeta: opt.SendMetaToSink,
  470. enableLimit: enableLimit,
  471. limitCount: limitCount,
  472. }.Init()
  473. p.SetChildren(children)
  474. children = []LogicalPlan{p}
  475. }
  476. if len(srfMapping) > 0 {
  477. enableLimit := false
  478. limitCount := 0
  479. if stmt.Limit != nil {
  480. enableLimit = true
  481. limitCount = stmt.Limit.(*ast.LimitExpr).LimitCount.Val
  482. }
  483. p = ProjectSetPlan{
  484. SrfMapping: srfMapping,
  485. enableLimit: enableLimit,
  486. limitCount: limitCount,
  487. }.Init()
  488. p.SetChildren(children)
  489. }
  490. return optimize(p)
  491. }
  492. // extractSRFMapping extracts the set-returning-function in the field
  493. func extractSRFMapping(stmt *ast.SelectStatement) map[string]struct{} {
  494. m := make(map[string]struct{})
  495. for _, field := range stmt.Fields {
  496. var curExpr ast.Expr
  497. var name string
  498. if len(field.AName) > 0 {
  499. curExpr = field.Expr.(*ast.FieldRef).AliasRef.Expression
  500. name = field.AName
  501. } else {
  502. curExpr = field.Expr
  503. name = field.Name
  504. }
  505. if f, ok := curExpr.(*ast.Call); ok && f.FuncType == ast.FuncTypeSrf {
  506. m[name] = struct{}{}
  507. }
  508. }
  509. return m
  510. }
  511. func Transform(op node.UnOperation, name string, options *api.RuleOption) *node.UnaryOperator {
  512. unaryOperator := node.New(name, options)
  513. unaryOperator.SetOperation(op)
  514. return unaryOperator
  515. }
  516. func extractWindowFuncFields(stmt *ast.SelectStatement) (ast.Fields, map[string]struct{}) {
  517. windowFuncsName := make(map[string]struct{})
  518. windowFuncFields := make([]ast.Field, 0)
  519. for _, field := range stmt.Fields {
  520. if wf, ok := field.Expr.(*ast.Call); ok && wf.FuncType == ast.FuncTypeWindow {
  521. windowFuncFields = append(windowFuncFields, field)
  522. windowFuncsName[wf.Name] = struct{}{}
  523. continue
  524. }
  525. if ref, ok := field.Expr.(*ast.FieldRef); ok && ref.AliasRef != nil {
  526. if wf, ok := ref.AliasRef.Expression.(*ast.Call); ok && wf.FuncType == ast.FuncTypeWindow {
  527. windowFuncFields = append(windowFuncFields, field)
  528. windowFuncsName[ref.Name] = struct{}{}
  529. continue
  530. }
  531. }
  532. }
  533. return windowFuncFields, windowFuncsName
  534. }