planner.go 15 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511
  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.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 *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}, 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}, fmt.Sprintf("%d_project", newIndex), options)
  171. case *ProjectSetPlan:
  172. op = Transform(&operator.ProjectSetOperator{SrfMapping: t.SrfMapping}, fmt.Sprintf("%d_projectset", newIndex), options)
  173. default:
  174. err = fmt.Errorf("unknown logical plan %v", t)
  175. }
  176. if err != nil {
  177. return nil, 0, err
  178. }
  179. if uop, ok := op.(*node.UnaryOperator); ok {
  180. uop.SetConcurrency(options.Concurrency)
  181. }
  182. if onode, ok := op.(node.OperatorNode); ok {
  183. tp.AddOperator(inputs, onode)
  184. }
  185. return op, newIndex, nil
  186. }
  187. func convertFromUnit(t ast.Token, v int64) int64 {
  188. unit := 1
  189. switch t {
  190. case ast.DD:
  191. unit = 24 * 3600 * 1000
  192. case ast.HH:
  193. unit = 3600 * 1000
  194. case ast.MI:
  195. unit = 60 * 1000
  196. case ast.SS:
  197. unit = 1000
  198. case ast.MS:
  199. unit = 1
  200. }
  201. return int64(unit) * v
  202. }
  203. func convertFromDuration(t *WindowPlan) (int64, int64, int64) {
  204. var unit int64 = 1
  205. switch t.timeUnit {
  206. case ast.DD:
  207. unit = 24 * 3600 * 1000
  208. case ast.HH:
  209. unit = 3600 * 1000
  210. case ast.MI:
  211. unit = 60 * 1000
  212. case ast.SS:
  213. unit = 1000
  214. case ast.MS:
  215. unit = 1
  216. }
  217. return int64(t.length) * unit, int64(t.interval) * unit, t.delay * unit
  218. }
  219. func transformSourceNode(t *DataSourcePlan, sources []*node.SourceNode, options *api.RuleOption) (*node.SourceNode, error) {
  220. isSchemaless := t.isSchemaless
  221. switch t.streamStmt.StreamType {
  222. case ast.TypeStream:
  223. var (
  224. pp node.UnOperation
  225. err error
  226. )
  227. if t.iet || (!isSchemaless && (t.streamStmt.Options.STRICT_VALIDATION || t.isBinary)) {
  228. pp, err = operator.NewPreprocessor(isSchemaless, t.streamFields, t.allMeta, t.metaFields, t.iet, t.timestampField, t.timestampFormat, t.isBinary, t.streamStmt.Options.STRICT_VALIDATION)
  229. if err != nil {
  230. return nil, err
  231. }
  232. }
  233. var srcNode *node.SourceNode
  234. if len(sources) == 0 {
  235. var sourceNode *node.SourceNode
  236. schema := t.streamFields
  237. if t.isSchemaless {
  238. schema = nil
  239. }
  240. sourceNode = node.NewSourceNode(string(t.name), t.streamStmt.StreamType, pp, t.streamStmt.Options, options.SendError, schema)
  241. srcNode = sourceNode
  242. } else {
  243. srcNode = getMockSource(sources, string(t.name))
  244. if srcNode == nil {
  245. return nil, fmt.Errorf("can't find predefined source %s", t.name)
  246. }
  247. }
  248. return srcNode, nil
  249. case ast.TypeTable:
  250. pp, err := operator.NewTableProcessor(isSchemaless, string(t.name), t.streamFields, t.streamStmt.Options)
  251. if err != nil {
  252. return nil, err
  253. }
  254. var srcNode *node.SourceNode
  255. if len(sources) > 0 {
  256. srcNode = getMockSource(sources, string(t.name))
  257. }
  258. if srcNode == nil {
  259. schema := t.streamFields
  260. if t.isSchemaless {
  261. schema = nil
  262. }
  263. srcNode = node.NewSourceNode(string(t.name), t.streamStmt.StreamType, pp, t.streamStmt.Options, options.SendError, schema)
  264. }
  265. return srcNode, nil
  266. }
  267. return nil, fmt.Errorf("unknown stream type %d", t.streamStmt.StreamType)
  268. }
  269. func getMockSource(sources []*node.SourceNode, name string) *node.SourceNode {
  270. for _, source := range sources {
  271. if name == source.GetName() {
  272. return source
  273. }
  274. }
  275. return nil
  276. }
  277. func createLogicalPlan(stmt *ast.SelectStatement, opt *api.RuleOption, store kv.KeyValue) (LogicalPlan, error) {
  278. dimensions := stmt.Dimensions
  279. var (
  280. p LogicalPlan
  281. children []LogicalPlan
  282. // If there are tables, the plan graph will be different for join/window
  283. lookupTableChildren map[string]*ast.Options
  284. scanTableChildren []LogicalPlan
  285. scanTableEmitters []string
  286. streamEmitters []string
  287. w *ast.Window
  288. ds ast.Dimensions
  289. )
  290. streamStmts, analyticFuncs, err := decorateStmt(stmt, store)
  291. if err != nil {
  292. return nil, err
  293. }
  294. for _, sInfo := range streamStmts {
  295. if sInfo.stmt.StreamType == ast.TypeTable && sInfo.stmt.Options.KIND == ast.StreamKindLookup {
  296. if lookupTableChildren == nil {
  297. lookupTableChildren = make(map[string]*ast.Options)
  298. }
  299. lookupTableChildren[string(sInfo.stmt.Name)] = sInfo.stmt.Options
  300. } else {
  301. p = DataSourcePlan{
  302. name: sInfo.stmt.Name,
  303. streamStmt: sInfo.stmt,
  304. streamFields: sInfo.schema.ToJsonSchema(),
  305. isSchemaless: sInfo.schema == nil,
  306. iet: opt.IsEventTime,
  307. allMeta: opt.SendMetaToSink,
  308. }.Init()
  309. if sInfo.stmt.StreamType == ast.TypeStream {
  310. children = append(children, p)
  311. streamEmitters = append(streamEmitters, string(sInfo.stmt.Name))
  312. } else {
  313. scanTableChildren = append(scanTableChildren, p)
  314. scanTableEmitters = append(scanTableEmitters, string(sInfo.stmt.Name))
  315. }
  316. }
  317. }
  318. hasWindow := dimensions != nil && dimensions.GetWindow() != nil
  319. if opt.IsEventTime {
  320. p = WatermarkPlan{
  321. SendWatermark: hasWindow,
  322. Emitters: streamEmitters,
  323. }.Init()
  324. p.SetChildren(children)
  325. children = []LogicalPlan{p}
  326. }
  327. if len(analyticFuncs) > 0 {
  328. p = AnalyticFuncsPlan{
  329. funcs: analyticFuncs,
  330. }.Init()
  331. p.SetChildren(children)
  332. children = []LogicalPlan{p}
  333. }
  334. if dimensions != nil {
  335. w = dimensions.GetWindow()
  336. if w != nil {
  337. if len(children) == 0 {
  338. return nil, errors.New("cannot run window for TABLE sources")
  339. }
  340. wp := WindowPlan{
  341. wtype: w.WindowType,
  342. length: w.Length.Val,
  343. isEventTime: opt.IsEventTime,
  344. }.Init()
  345. if w.Delay != nil {
  346. wp.delay = int64(w.Delay.Val)
  347. }
  348. if w.Interval != nil {
  349. wp.interval = w.Interval.Val
  350. } else if w.WindowType == ast.COUNT_WINDOW {
  351. // if no interval value is set, and it's a count window, then set interval to length value.
  352. wp.interval = w.Length.Val
  353. }
  354. if w.TimeUnit != nil {
  355. wp.timeUnit = w.TimeUnit.Val
  356. }
  357. if w.Filter != nil {
  358. wp.condition = w.Filter
  359. }
  360. if w.TriggerCondition != nil {
  361. wp.triggerCondition = w.TriggerCondition
  362. }
  363. // TODO calculate limit
  364. // TODO incremental aggregate
  365. wp.SetChildren(children)
  366. children = []LogicalPlan{wp}
  367. p = wp
  368. }
  369. }
  370. if stmt.Joins != nil {
  371. if len(lookupTableChildren) == 0 && len(scanTableChildren) == 0 && w == nil {
  372. return nil, errors.New("a time window or count window is required to join multiple streams")
  373. }
  374. if len(lookupTableChildren) > 0 {
  375. var joins []ast.Join
  376. for _, join := range stmt.Joins {
  377. if streamOpt, ok := lookupTableChildren[join.Name]; ok {
  378. lookupPlan := LookupPlan{
  379. joinExpr: join,
  380. options: streamOpt,
  381. }
  382. if !lookupPlan.validateAndExtractCondition() {
  383. return nil, fmt.Errorf("join condition %s is invalid, at least one equi-join predicate is required", join.Expr)
  384. }
  385. p = lookupPlan.Init()
  386. p.SetChildren(children)
  387. children = []LogicalPlan{p}
  388. delete(lookupTableChildren, join.Name)
  389. } else {
  390. joins = append(joins, join)
  391. }
  392. }
  393. if len(lookupTableChildren) > 0 {
  394. return nil, fmt.Errorf("cannot find lookup table %v in any join", lookupTableChildren)
  395. }
  396. stmt.Joins = joins
  397. }
  398. // Not all joins are lookup joins, so we need to create a join plan for the remaining joins
  399. if len(stmt.Joins) > 0 {
  400. if len(scanTableChildren) > 0 {
  401. p = JoinAlignPlan{
  402. Emitters: scanTableEmitters,
  403. }.Init()
  404. p.SetChildren(append(children, scanTableChildren...))
  405. children = []LogicalPlan{p}
  406. }
  407. p = JoinPlan{
  408. from: stmt.Sources[0].(*ast.Table),
  409. joins: stmt.Joins,
  410. }.Init()
  411. p.SetChildren(children)
  412. children = []LogicalPlan{p}
  413. }
  414. }
  415. if stmt.Condition != nil {
  416. p = FilterPlan{
  417. condition: stmt.Condition,
  418. }.Init()
  419. p.SetChildren(children)
  420. children = []LogicalPlan{p}
  421. }
  422. if dimensions != nil {
  423. ds = dimensions.GetGroups()
  424. if ds != nil && len(ds) > 0 {
  425. p = AggregatePlan{
  426. dimensions: ds,
  427. }.Init()
  428. p.SetChildren(children)
  429. children = []LogicalPlan{p}
  430. }
  431. }
  432. if stmt.Having != nil {
  433. p = HavingPlan{
  434. condition: stmt.Having,
  435. }.Init()
  436. p.SetChildren(children)
  437. children = []LogicalPlan{p}
  438. }
  439. if stmt.SortFields != nil {
  440. p = OrderPlan{
  441. SortFields: stmt.SortFields,
  442. }.Init()
  443. p.SetChildren(children)
  444. children = []LogicalPlan{p}
  445. }
  446. if stmt.Fields != nil {
  447. p = ProjectPlan{
  448. fields: stmt.Fields,
  449. isAggregate: xsql.WithAggFields(stmt),
  450. sendMeta: opt.SendMetaToSink,
  451. }.Init()
  452. p.SetChildren(children)
  453. children = []LogicalPlan{p}
  454. }
  455. srfMapping := extractSRFMapping(stmt)
  456. if len(srfMapping) > 0 {
  457. p = ProjectSetPlan{
  458. SrfMapping: srfMapping,
  459. }.Init()
  460. p.SetChildren(children)
  461. }
  462. return optimize(p)
  463. }
  464. // extractSRFMapping extracts the set-returning-function in the field
  465. func extractSRFMapping(stmt *ast.SelectStatement) map[string]struct{} {
  466. m := make(map[string]struct{})
  467. for _, field := range stmt.Fields {
  468. var curExpr ast.Expr
  469. var name string
  470. if len(field.AName) > 0 {
  471. curExpr = field.Expr.(*ast.FieldRef).AliasRef.Expression
  472. name = field.AName
  473. } else {
  474. curExpr = field.Expr
  475. name = field.Name
  476. }
  477. if f, ok := curExpr.(*ast.Call); ok && f.FuncType == ast.FuncTypeSrf {
  478. m[name] = struct{}{}
  479. }
  480. }
  481. return m
  482. }
  483. func Transform(op node.UnOperation, name string, options *api.RuleOption) *node.UnaryOperator {
  484. unaryOperator := node.New(name, options)
  485. unaryOperator.SetOperation(op)
  486. return unaryOperator
  487. }