planner.go 14 KB

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