planner_graph.go 14 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504
  1. // Copyright 2022 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/topo"
  19. "github.com/lf-edge/ekuiper/internal/topo/graph"
  20. "github.com/lf-edge/ekuiper/internal/topo/node"
  21. "github.com/lf-edge/ekuiper/internal/topo/operator"
  22. "github.com/lf-edge/ekuiper/internal/xsql"
  23. "github.com/lf-edge/ekuiper/pkg/api"
  24. "github.com/lf-edge/ekuiper/pkg/ast"
  25. "github.com/lf-edge/ekuiper/pkg/cast"
  26. "github.com/lf-edge/ekuiper/pkg/message"
  27. "strings"
  28. )
  29. // PlanByGraph returns a topo.Topo object by a graph
  30. func PlanByGraph(rule *api.Rule) (*topo.Topo, error) {
  31. ruleGraph := rule.Graph
  32. if ruleGraph == nil {
  33. return nil, errors.New("no graph")
  34. }
  35. tp, err := topo.NewWithNameAndQos(rule.Id, rule.Options.Qos, rule.Options.CheckpointInterval)
  36. if err != nil {
  37. return nil, err
  38. }
  39. var (
  40. nodeMap = make(map[string]api.TopNode)
  41. sinks = make(map[string]bool)
  42. sources = make(map[string]bool)
  43. )
  44. for nodeName, gn := range ruleGraph.Nodes {
  45. switch gn.Type {
  46. case "source":
  47. if _, ok := ruleGraph.Topo.Edges[nodeName]; !ok {
  48. return nil, fmt.Errorf("no edge defined for source node %s", nodeName)
  49. }
  50. sourceType, ok := gn.Props["source_type"]
  51. if !ok {
  52. sourceType = "stream"
  53. }
  54. st, ok := sourceType.(string)
  55. if !ok {
  56. return nil, fmt.Errorf("source_type %v is not string", sourceType)
  57. }
  58. st = strings.ToLower(st)
  59. sourceOption := &ast.Options{}
  60. err := cast.MapToStruct(gn.Props, sourceOption)
  61. if err != nil {
  62. return nil, err
  63. }
  64. sourceOption.TYPE = gn.NodeType
  65. switch st {
  66. case "stream":
  67. // TODO deal with conf key
  68. pp, err := operator.NewPreprocessor(true, nil, true, nil, rule.Options.IsEventTime, sourceOption.TIMESTAMP, sourceOption.TIMESTAMP_FORMAT, strings.EqualFold(sourceOption.FORMAT, message.FormatBinary), sourceOption.STRICT_VALIDATION)
  69. if err != nil {
  70. return nil, err
  71. }
  72. srcNode := node.NewSourceNode(nodeName, ast.TypeStream, pp, sourceOption, rule.Options.SendError)
  73. nodeMap[nodeName] = srcNode
  74. tp.AddSrc(srcNode)
  75. case "table":
  76. // TODO add table
  77. return nil, fmt.Errorf("table source is not supported yet")
  78. default:
  79. return nil, fmt.Errorf("unknown source type %s", st)
  80. }
  81. sources[nodeName] = true
  82. case "sink":
  83. if _, ok := ruleGraph.Topo.Edges[nodeName]; ok {
  84. return nil, fmt.Errorf("sink %s has edge", nodeName)
  85. }
  86. nodeMap[nodeName] = node.NewSinkNode(nodeName, gn.NodeType, gn.Props)
  87. sinks[nodeName] = true
  88. case "operator":
  89. if _, ok := ruleGraph.Topo.Edges[nodeName]; !ok {
  90. return nil, fmt.Errorf("no edge defined for operator node %s", nodeName)
  91. }
  92. switch strings.ToLower(gn.NodeType) {
  93. case "function":
  94. fop, err := parseFunc(gn.Props)
  95. if err != nil {
  96. return nil, err
  97. }
  98. op := Transform(fop, nodeName, rule.Options)
  99. nodeMap[nodeName] = op
  100. case "aggfunc":
  101. fop, err := parseFunc(gn.Props)
  102. if err != nil {
  103. return nil, err
  104. }
  105. fop.IsAgg = true
  106. op := Transform(fop, nodeName, rule.Options)
  107. nodeMap[nodeName] = op
  108. case "filter":
  109. fop, err := parseFilter(gn.Props)
  110. if err != nil {
  111. return nil, err
  112. }
  113. op := Transform(fop, nodeName, rule.Options)
  114. nodeMap[nodeName] = op
  115. case "pick":
  116. pop, err := parsePick(gn.Props)
  117. if err != nil {
  118. return nil, err
  119. }
  120. op := Transform(pop, nodeName, rule.Options)
  121. nodeMap[nodeName] = op
  122. case "window":
  123. wconf, err := parseWindow(gn.Props)
  124. if err != nil {
  125. return nil, err
  126. }
  127. op, err := node.NewWindowOp(nodeName, *wconf, ruleGraph.Topo.Sources, rule.Options)
  128. if err != nil {
  129. return nil, err
  130. }
  131. nodeMap[nodeName] = op
  132. case "join":
  133. jop, err := parseJoin(gn.Props)
  134. if err != nil {
  135. return nil, err
  136. }
  137. op := Transform(jop, nodeName, rule.Options)
  138. nodeMap[nodeName] = op
  139. case "groupby":
  140. gop, err := parseGroupBy(gn.Props)
  141. if err != nil {
  142. return nil, err
  143. }
  144. op := Transform(gop, nodeName, rule.Options)
  145. nodeMap[nodeName] = op
  146. case "orderby":
  147. oop, err := parseOrderBy(gn.Props)
  148. if err != nil {
  149. return nil, err
  150. }
  151. op := Transform(oop, nodeName, rule.Options)
  152. nodeMap[nodeName] = op
  153. default: // TODO other node type
  154. return nil, fmt.Errorf("unknown operator type %s", gn.NodeType)
  155. }
  156. default:
  157. return nil, fmt.Errorf("unknown node type %s", gn.Type)
  158. }
  159. }
  160. // validate source node
  161. for _, nodeName := range ruleGraph.Topo.Sources {
  162. if _, ok := sources[nodeName]; !ok {
  163. return nil, fmt.Errorf("source %s is not a source type node", nodeName)
  164. }
  165. }
  166. // reverse edges
  167. reversedEdges := make(map[string][]string)
  168. rclone := make(map[string][]string)
  169. for fromNode, toNodes := range ruleGraph.Topo.Edges {
  170. if _, ok := ruleGraph.Nodes[fromNode]; !ok {
  171. return nil, fmt.Errorf("node %s is not defined", fromNode)
  172. }
  173. for _, toNode := range toNodes {
  174. if _, ok := ruleGraph.Nodes[toNode]; !ok {
  175. return nil, fmt.Errorf("node %s is not defined", toNode)
  176. }
  177. reversedEdges[toNode] = append(reversedEdges[toNode], fromNode)
  178. rclone[toNode] = append(rclone[toNode], fromNode)
  179. }
  180. }
  181. // sort the nodes by topological order
  182. nodesInOrder := make([]string, len(ruleGraph.Nodes))
  183. i := 0
  184. genNodesInOrder(ruleGraph.Topo.Sources, ruleGraph.Topo.Edges, rclone, nodesInOrder, i)
  185. // validate the typo
  186. // the map is to record the output for each node
  187. dataFlow := make(map[string]*graph.IOType)
  188. for _, n := range nodesInOrder {
  189. gn := ruleGraph.Nodes[n]
  190. if gn == nil {
  191. return nil, fmt.Errorf("can't find node %s", n)
  192. }
  193. if gn.Type == "source" {
  194. dataFlow[n] = &graph.IOType{
  195. Type: graph.IOINPUT_TYPE_ROW,
  196. RowType: graph.IOROW_TYPE_SINGLE,
  197. CollectionType: graph.IOCOLLECTION_TYPE_ANY,
  198. AllowMulti: false,
  199. }
  200. } else if gn.Type == "sink" {
  201. continue
  202. } else {
  203. nodeIO, ok := graph.OpIO[strings.ToLower(gn.NodeType)]
  204. if !ok {
  205. return nil, fmt.Errorf("can't find the io definiton for node type %s", gn.NodeType)
  206. }
  207. dataInCondition := nodeIO[0]
  208. innodes := reversedEdges[n]
  209. if len(innodes) > 1 {
  210. if dataInCondition.AllowMulti {
  211. for _, innode := range innodes {
  212. _, err = graph.Fit(dataFlow[innode], dataInCondition)
  213. if err != nil {
  214. return nil, fmt.Errorf("node %s output does not match node %s input: %v", innode, n, err)
  215. }
  216. }
  217. } else {
  218. return nil, fmt.Errorf("operator %s of type %s does not allow multiple inputs", n, gn.NodeType)
  219. }
  220. } else if len(innodes) == 1 {
  221. _, err := graph.Fit(dataFlow[innodes[0]], dataInCondition)
  222. if err != nil {
  223. return nil, fmt.Errorf("node %s output does not match node %s input: %v", innodes[0], n, err)
  224. }
  225. } else {
  226. return nil, fmt.Errorf("operator %s of type %s has no input", n, gn.NodeType)
  227. }
  228. out := nodeIO[1]
  229. in := dataFlow[innodes[0]]
  230. dataFlow[n] = graph.MapOut(in, out)
  231. // convert filter to having if the input is aggregated
  232. if gn.NodeType == "filter" && in.Type == graph.IOINPUT_TYPE_COLLECTION && in.CollectionType == graph.IOCOLLECTION_TYPE_GROUPED {
  233. fop, err := parseHaving(gn.Props)
  234. if err != nil {
  235. return nil, err
  236. }
  237. op := Transform(fop, n, rule.Options)
  238. nodeMap[n] = op
  239. }
  240. }
  241. }
  242. // add the linkages
  243. for nodeName, fromNodes := range reversedEdges {
  244. inputs := make([]api.Emitter, len(fromNodes))
  245. for i, fromNode := range fromNodes {
  246. inputs[i] = nodeMap[fromNode].(api.Emitter)
  247. }
  248. n := nodeMap[nodeName]
  249. if n == nil {
  250. return nil, fmt.Errorf("node %s is not defined", nodeName)
  251. }
  252. if _, ok := sinks[nodeName]; ok {
  253. tp.AddSink(inputs, n.(*node.SinkNode))
  254. } else {
  255. tp.AddOperator(inputs, n.(node.OperatorNode))
  256. }
  257. }
  258. return tp, nil
  259. }
  260. func genNodesInOrder(toNodes []string, edges map[string][]string, reversedEdges map[string][]string, nodesInOrder []string, i int) int {
  261. for _, src := range toNodes {
  262. if len(reversedEdges[src]) > 1 {
  263. reversedEdges[src] = reversedEdges[src][1:]
  264. continue
  265. }
  266. nodesInOrder[i] = src
  267. i++
  268. i = genNodesInOrder(edges[src], edges, reversedEdges, nodesInOrder, i)
  269. }
  270. return i
  271. }
  272. func parseOrderBy(props map[string]interface{}) (*operator.OrderOp, error) {
  273. n := &graph.Orderby{}
  274. err := cast.MapToStruct(props, n)
  275. if err != nil {
  276. return nil, err
  277. }
  278. stmt := "SELECT * FROM unknown ORDER BY"
  279. for _, s := range n.Sorts {
  280. stmt += " " + s.Field + " "
  281. if s.Desc {
  282. stmt += "DESC"
  283. }
  284. }
  285. p, err := xsql.NewParser(strings.NewReader(stmt)).Parse()
  286. if err != nil {
  287. return nil, fmt.Errorf("invalid order by statement error: %v", err)
  288. }
  289. if len(p.SortFields) == 0 {
  290. return nil, fmt.Errorf("order by statement is empty")
  291. }
  292. return &operator.OrderOp{
  293. SortFields: p.SortFields,
  294. }, nil
  295. }
  296. func parseGroupBy(props map[string]interface{}) (*operator.AggregateOp, error) {
  297. n := &graph.Groupby{}
  298. err := cast.MapToStruct(props, n)
  299. if err != nil {
  300. return nil, err
  301. }
  302. if len(n.Dimensions) == 0 {
  303. return nil, fmt.Errorf("groupby must have at least one dimension")
  304. }
  305. stmt := "SELECT * FROM unknown Group By " + strings.Join(n.Dimensions, ",")
  306. p, err := xsql.NewParser(strings.NewReader(stmt)).Parse()
  307. if err != nil {
  308. return nil, fmt.Errorf("invalid join statement error: %v", err)
  309. }
  310. return &operator.AggregateOp{Dimensions: p.Dimensions}, nil
  311. }
  312. func parseJoin(props map[string]interface{}) (*operator.JoinOp, error) {
  313. n := &graph.Join{}
  314. err := cast.MapToStruct(props, n)
  315. if err != nil {
  316. return nil, err
  317. }
  318. stmt := "SELECT * FROM " + n.From
  319. for _, join := range n.Joins {
  320. stmt += " " + join.Type + " JOIN ON " + join.On
  321. }
  322. p, err := xsql.NewParser(strings.NewReader(stmt)).Parse()
  323. if err != nil {
  324. return nil, fmt.Errorf("invalid join statement error: %v", err)
  325. }
  326. return &operator.JoinOp{Joins: p.Joins, From: p.Sources[0].(*ast.Table)}, nil
  327. }
  328. func parseWindow(props map[string]interface{}) (*node.WindowConfig, error) {
  329. n := &graph.Window{}
  330. err := cast.MapToStruct(props, n)
  331. if err != nil {
  332. return nil, err
  333. }
  334. if n.Size <= 0 {
  335. return nil, fmt.Errorf("window size %d is invalid", n.Size)
  336. }
  337. var (
  338. wt ast.WindowType
  339. length int
  340. interval int
  341. )
  342. switch strings.ToLower(n.Type) {
  343. case "tumblingwindow":
  344. wt = ast.TUMBLING_WINDOW
  345. if n.Interval != 0 && n.Interval != n.Size {
  346. return nil, fmt.Errorf("tumbling window interval must equal to size")
  347. }
  348. case "hoppingwindow":
  349. wt = ast.HOPPING_WINDOW
  350. if n.Interval <= 0 {
  351. return nil, fmt.Errorf("hopping window interval must be greater than 0")
  352. }
  353. if n.Interval > n.Size {
  354. return nil, fmt.Errorf("hopping window interval must be less than size")
  355. }
  356. case "sessionwindow":
  357. wt = ast.SESSION_WINDOW
  358. if n.Interval <= 0 {
  359. return nil, fmt.Errorf("hopping window interval must be greater than 0")
  360. }
  361. case "slidingwindow":
  362. wt = ast.SLIDING_WINDOW
  363. if n.Interval != 0 && n.Interval != n.Size {
  364. return nil, fmt.Errorf("tumbling window interval must equal to size")
  365. }
  366. case "countwindow":
  367. wt = ast.COUNT_WINDOW
  368. if n.Interval < 0 {
  369. return nil, fmt.Errorf("count window interval must be greater or equal to 0")
  370. }
  371. if n.Interval > n.Size {
  372. return nil, fmt.Errorf("count window interval must be less than size")
  373. }
  374. if n.Interval == 0 {
  375. n.Interval = n.Size
  376. }
  377. default:
  378. return nil, fmt.Errorf("unknown window type %s", n.Type)
  379. }
  380. if wt == ast.COUNT_WINDOW {
  381. length = n.Size
  382. interval = n.Interval
  383. } else {
  384. var unit = 1
  385. switch strings.ToLower(n.Unit) {
  386. case "dd":
  387. unit = 24 * 3600 * 1000
  388. case "hh":
  389. unit = 3600 * 1000
  390. case "mi":
  391. unit = 60 * 1000
  392. case "ss":
  393. unit = 1000
  394. case "ms":
  395. unit = 1
  396. default:
  397. return nil, fmt.Errorf("Invalid unit %s", n.Unit)
  398. }
  399. length = n.Size * unit
  400. interval = n.Interval * unit
  401. }
  402. return &node.WindowConfig{
  403. Type: wt,
  404. Length: length,
  405. Interval: interval,
  406. }, nil
  407. }
  408. func parsePick(props map[string]interface{}) (*operator.ProjectOp, error) {
  409. n := &graph.Select{}
  410. err := cast.MapToStruct(props, n)
  411. if err != nil {
  412. return nil, err
  413. }
  414. stmt, err := xsql.NewParser(strings.NewReader("select " + strings.Join(n.Fields, ",") + " from nonexist")).Parse()
  415. if err != nil {
  416. return nil, err
  417. }
  418. t := ProjectPlan{
  419. fields: stmt.Fields,
  420. isAggregate: xsql.IsAggStatement(stmt),
  421. }.Init()
  422. return &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}, nil
  423. }
  424. func parseFunc(props map[string]interface{}) (*operator.FuncOp, error) {
  425. m, ok := props["expr"]
  426. if !ok {
  427. return nil, errors.New("no expr")
  428. }
  429. funcExpr, ok := m.(string)
  430. if !ok {
  431. return nil, fmt.Errorf("expr %v is not string", m)
  432. }
  433. stmt, err := xsql.NewParser(strings.NewReader("select " + funcExpr + " from nonexist")).Parse()
  434. if err != nil {
  435. return nil, err
  436. }
  437. f := stmt.Fields[0]
  438. c, ok := f.Expr.(*ast.Call)
  439. if !ok {
  440. // never happen
  441. return nil, fmt.Errorf("expr %v is not ast.Call", stmt.Fields[0].Expr)
  442. }
  443. var name string
  444. if f.AName != "" {
  445. name = f.AName
  446. } else {
  447. name = f.Name
  448. }
  449. return &operator.FuncOp{CallExpr: c, Name: name}, nil
  450. }
  451. func parseFilter(props map[string]interface{}) (*operator.FilterOp, error) {
  452. m, ok := props["expr"]
  453. if !ok {
  454. return nil, errors.New("no expr")
  455. }
  456. conditionExpr, ok := m.(string)
  457. if !ok {
  458. return nil, fmt.Errorf("expr %v is not string", m)
  459. }
  460. p := xsql.NewParser(strings.NewReader("where " + conditionExpr))
  461. if exp, err := p.ParseCondition(); err != nil {
  462. return nil, err
  463. } else {
  464. if exp != nil {
  465. return &operator.FilterOp{Condition: exp}, nil
  466. }
  467. }
  468. return nil, fmt.Errorf("expr %v is not a condition", m)
  469. }
  470. func parseHaving(props map[string]interface{}) (*operator.HavingOp, error) {
  471. m, ok := props["expr"]
  472. if !ok {
  473. return nil, errors.New("no expr")
  474. }
  475. conditionExpr, ok := m.(string)
  476. if !ok {
  477. return nil, fmt.Errorf("expr %v is not string", m)
  478. }
  479. p := xsql.NewParser(strings.NewReader("where " + conditionExpr))
  480. if exp, err := p.ParseCondition(); err != nil {
  481. return nil, err
  482. } else {
  483. if exp != nil {
  484. return &operator.HavingOp{Condition: exp}, nil
  485. }
  486. }
  487. return nil, fmt.Errorf("expr %v is not a condition", m)
  488. }