planner.go 15 KB

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