planner_graph.go 14 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468
  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.Type == "source" {
  191. dataFlow[n] = &graph.IOType{
  192. Type: graph.IOINPUT_TYPE_ROW,
  193. RowType: graph.IOROW_TYPE_SINGLE,
  194. CollectionType: graph.IOCOLLECTION_TYPE_ANY,
  195. AllowMulti: false,
  196. }
  197. } else if gn.Type == "sink" {
  198. continue
  199. } else {
  200. nodeIO, ok := graph.OpIO[strings.ToLower(gn.NodeType)]
  201. if !ok {
  202. return nil, fmt.Errorf("can't find the io definiton for node type %s", gn.NodeType)
  203. }
  204. dataInCondition := nodeIO[0]
  205. innodes := reversedEdges[n]
  206. if len(innodes) > 1 {
  207. if dataInCondition.AllowMulti {
  208. for _, innode := range innodes {
  209. _, err = graph.Fit(dataFlow[innode], dataInCondition)
  210. if err != nil {
  211. return nil, fmt.Errorf("node %s output does not match node %s input: %v", innode, n, err)
  212. }
  213. }
  214. } else {
  215. return nil, fmt.Errorf("operator %s of type %s does not allow multiple inputs", n, gn.NodeType)
  216. }
  217. } else if len(innodes) == 1 {
  218. _, err := graph.Fit(dataFlow[innodes[0]], dataInCondition)
  219. if err != nil {
  220. return nil, fmt.Errorf("node %s output does not match node %s input: %v", innodes[0], n, err)
  221. }
  222. } else {
  223. return nil, fmt.Errorf("operator %s of type %s has no input", n, gn.NodeType)
  224. }
  225. out := nodeIO[1]
  226. dataFlow[n] = graph.MapOut(dataFlow[innodes[0]], out)
  227. }
  228. }
  229. // add the linkages
  230. for nodeName, fromNodes := range reversedEdges {
  231. inputs := make([]api.Emitter, len(fromNodes))
  232. for i, fromNode := range fromNodes {
  233. inputs[i] = nodeMap[fromNode].(api.Emitter)
  234. }
  235. n := nodeMap[nodeName]
  236. if n == nil {
  237. return nil, fmt.Errorf("node %s is not defined", nodeName)
  238. }
  239. if _, ok := sinks[nodeName]; ok {
  240. tp.AddSink(inputs, n.(*node.SinkNode))
  241. } else {
  242. tp.AddOperator(inputs, n.(node.OperatorNode))
  243. }
  244. }
  245. return tp, nil
  246. }
  247. func genNodesInOrder(toNodes []string, edges map[string][]string, reversedEdges map[string][]string, nodesInOrder []string, i int) int {
  248. for _, src := range toNodes {
  249. if len(reversedEdges[src]) > 1 {
  250. reversedEdges[src] = reversedEdges[src][1:]
  251. continue
  252. }
  253. nodesInOrder[i] = src
  254. i++
  255. i = genNodesInOrder(edges[src], edges, reversedEdges, nodesInOrder, i)
  256. }
  257. return i
  258. }
  259. func parseOrderBy(props map[string]interface{}) (*operator.OrderOp, error) {
  260. n := &graph.Orderby{}
  261. err := cast.MapToStruct(props, n)
  262. if err != nil {
  263. return nil, err
  264. }
  265. stmt := "ORDER BY"
  266. for _, s := range n.Sorts {
  267. stmt += " " + s.Field + " " + s.Order
  268. }
  269. p, err := xsql.NewParser(strings.NewReader(stmt)).Parse()
  270. if err != nil {
  271. return nil, fmt.Errorf("invalid order by statement error: %v", err)
  272. }
  273. if len(p.SortFields) == 0 {
  274. return nil, fmt.Errorf("order by statement is empty")
  275. }
  276. return &operator.OrderOp{
  277. SortFields: p.SortFields,
  278. }, nil
  279. }
  280. func parseGroupBy(props map[string]interface{}) (*operator.AggregateOp, error) {
  281. n := &graph.Groupby{}
  282. err := cast.MapToStruct(props, n)
  283. if err != nil {
  284. return nil, err
  285. }
  286. if len(n.Dimensions) == 0 {
  287. return nil, fmt.Errorf("groupby must have at least one dimension")
  288. }
  289. stmt := "SELECT * FROM unknown Group By " + strings.Join(n.Dimensions, ",")
  290. p, err := xsql.NewParser(strings.NewReader(stmt)).Parse()
  291. if err != nil {
  292. return nil, fmt.Errorf("invalid join statement error: %v", err)
  293. }
  294. return &operator.AggregateOp{Dimensions: p.Dimensions}, nil
  295. }
  296. func parseJoin(props map[string]interface{}) (*operator.JoinOp, error) {
  297. n := &graph.Join{}
  298. err := cast.MapToStruct(props, n)
  299. if err != nil {
  300. return nil, err
  301. }
  302. stmt := "SELECT * FROM " + n.From
  303. for _, join := range n.Joins {
  304. stmt += " " + join.Type + " JOIN ON " + join.On
  305. }
  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.JoinOp{Joins: p.Joins, From: p.Sources[0].(*ast.Table)}, nil
  311. }
  312. func parseWindow(props map[string]interface{}) (*node.WindowConfig, error) {
  313. n := &graph.Window{}
  314. err := cast.MapToStruct(props, n)
  315. if err != nil {
  316. return nil, err
  317. }
  318. if n.Size <= 0 {
  319. return nil, fmt.Errorf("window size %d is invalid", n.Size)
  320. }
  321. var (
  322. wt ast.WindowType
  323. length int
  324. interval int
  325. )
  326. switch strings.ToLower(n.Type) {
  327. case "tumblingwindow":
  328. wt = ast.TUMBLING_WINDOW
  329. if n.Interval != 0 && n.Interval != n.Size {
  330. return nil, fmt.Errorf("tumbling window interval must equal to size")
  331. }
  332. case "hoppingwindow":
  333. wt = ast.HOPPING_WINDOW
  334. if n.Interval <= 0 {
  335. return nil, fmt.Errorf("hopping window interval must be greater than 0")
  336. }
  337. if n.Interval > n.Size {
  338. return nil, fmt.Errorf("hopping window interval must be less than size")
  339. }
  340. case "sessionwindow":
  341. wt = ast.SESSION_WINDOW
  342. if n.Interval <= 0 {
  343. return nil, fmt.Errorf("hopping window interval must be greater than 0")
  344. }
  345. case "slidingwindow":
  346. wt = ast.SLIDING_WINDOW
  347. if n.Interval != 0 && n.Interval != n.Size {
  348. return nil, fmt.Errorf("tumbling window interval must equal to size")
  349. }
  350. case "countwindow":
  351. wt = ast.COUNT_WINDOW
  352. if n.Interval < 0 {
  353. return nil, fmt.Errorf("count window interval must be greater or equal to 0")
  354. }
  355. if n.Interval > n.Size {
  356. return nil, fmt.Errorf("count window interval must be less than size")
  357. }
  358. if n.Interval == 0 {
  359. n.Interval = n.Size
  360. }
  361. default:
  362. return nil, fmt.Errorf("unknown window type %s", n.Type)
  363. }
  364. if wt == ast.COUNT_WINDOW {
  365. length = n.Size
  366. interval = n.Interval
  367. } else {
  368. var unit = 1
  369. switch strings.ToLower(n.Unit) {
  370. case "dd":
  371. unit = 24 * 3600 * 1000
  372. case "hh":
  373. unit = 3600 * 1000
  374. case "mi":
  375. unit = 60 * 1000
  376. case "ss":
  377. unit = 1000
  378. case "ms":
  379. unit = 1
  380. default:
  381. return nil, fmt.Errorf("Invalid unit %s", n.Unit)
  382. }
  383. length = n.Size * unit
  384. interval = n.Interval * unit
  385. }
  386. return &node.WindowConfig{
  387. Type: wt,
  388. Length: length,
  389. Interval: interval,
  390. }, nil
  391. }
  392. func parsePick(props map[string]interface{}) (*operator.ProjectOp, error) {
  393. n := &graph.Select{}
  394. err := cast.MapToStruct(props, n)
  395. if err != nil {
  396. return nil, err
  397. }
  398. stmt, err := xsql.NewParser(strings.NewReader("select " + strings.Join(n.Fields, ",") + " from nonexist")).Parse()
  399. if err != nil {
  400. return nil, err
  401. }
  402. t := ProjectPlan{
  403. fields: stmt.Fields,
  404. isAggregate: xsql.IsAggStatement(stmt),
  405. }.Init()
  406. 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
  407. }
  408. func parseFunc(props map[string]interface{}) (*operator.FuncOp, error) {
  409. m, ok := props["expr"]
  410. if !ok {
  411. return nil, errors.New("no expr")
  412. }
  413. funcExpr, ok := m.(string)
  414. if !ok {
  415. return nil, fmt.Errorf("expr %v is not string", m)
  416. }
  417. stmt, err := xsql.NewParser(strings.NewReader("select " + funcExpr + " from nonexist")).Parse()
  418. if err != nil {
  419. return nil, err
  420. }
  421. f := stmt.Fields[0]
  422. c, ok := f.Expr.(*ast.Call)
  423. if !ok {
  424. // never happen
  425. return nil, fmt.Errorf("expr %v is not ast.Call", stmt.Fields[0].Expr)
  426. }
  427. var name string
  428. if f.AName != "" {
  429. name = f.AName
  430. } else {
  431. name = f.Name
  432. }
  433. return &operator.FuncOp{CallExpr: c, Name: name}, nil
  434. }
  435. func parseFilter(props map[string]interface{}) (*operator.FilterOp, error) {
  436. m, ok := props["expr"]
  437. if !ok {
  438. return nil, errors.New("no expr")
  439. }
  440. conditionExpr, ok := m.(string)
  441. if !ok {
  442. return nil, fmt.Errorf("expr %v is not string", m)
  443. }
  444. p := xsql.NewParser(strings.NewReader("where " + conditionExpr))
  445. if exp, err := p.ParseCondition(); err != nil {
  446. return nil, err
  447. } else {
  448. if exp != nil {
  449. return &operator.FilterOp{Condition: exp}, nil
  450. }
  451. }
  452. return nil, fmt.Errorf("expr %v is not a condition", m)
  453. }