xsql_processor.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488
  1. package processors
  2. import (
  3. "bytes"
  4. "encoding/json"
  5. "engine/common"
  6. "engine/xsql"
  7. "engine/xsql/plans"
  8. "engine/xstream"
  9. "engine/xstream/api"
  10. "engine/xstream/extensions"
  11. "engine/xstream/nodes"
  12. "engine/xstream/operators"
  13. "engine/xstream/sinks"
  14. "fmt"
  15. "github.com/dgraph-io/badger"
  16. "github.com/go-yaml/yaml"
  17. "path"
  18. "plugin"
  19. "strings"
  20. )
  21. var log = common.Log
  22. type StreamProcessor struct {
  23. statement string
  24. badgerDir string
  25. }
  26. //@params s : the sql string of create stream statement
  27. //@params d : the directory of the badger DB to save the stream info
  28. func NewStreamProcessor(s, d string) *StreamProcessor {
  29. processor := &StreamProcessor{
  30. statement: s,
  31. badgerDir: d,
  32. }
  33. return processor
  34. }
  35. func (p *StreamProcessor) Exec() (result []string, err error) {
  36. parser := xsql.NewParser(strings.NewReader(p.statement))
  37. stmt, err := xsql.Language.Parse(parser)
  38. if err != nil {
  39. return
  40. }
  41. db, err := common.DbOpen(p.badgerDir)
  42. if err != nil {
  43. return
  44. }
  45. defer common.DbClose(db)
  46. switch s := stmt.(type) {
  47. case *xsql.StreamStmt:
  48. var r string
  49. r, err = p.execCreateStream(s, db)
  50. result = append(result, r)
  51. case *xsql.ShowStreamsStatement:
  52. result, err = p.execShowStream(s, db)
  53. case *xsql.DescribeStreamStatement:
  54. var r string
  55. r, err = p.execDescribeStream(s, db)
  56. result = append(result, r)
  57. case *xsql.ExplainStreamStatement:
  58. var r string
  59. r, err = p.execExplainStream(s, db)
  60. result = append(result, r)
  61. case *xsql.DropStreamStatement:
  62. var r string
  63. r, err = p.execDropStream(s, db)
  64. result = append(result, r)
  65. }
  66. return
  67. }
  68. func (p *StreamProcessor) execCreateStream(stmt *xsql.StreamStmt, db *badger.DB) (string, error) {
  69. err := common.DbSet(db, string(stmt.Name), p.statement)
  70. if err != nil {
  71. return "", err
  72. }else{
  73. return fmt.Sprintf("Stream %s is created.", stmt.Name), nil
  74. }
  75. }
  76. func (p *StreamProcessor) execShowStream(stmt *xsql.ShowStreamsStatement, db *badger.DB) ([]string,error) {
  77. keys, err := common.DbKeys(db)
  78. if len(keys) == 0 {
  79. keys = append(keys, "No stream definitions are found.")
  80. }
  81. return keys, err
  82. }
  83. func (p *StreamProcessor) execDescribeStream(stmt *xsql.DescribeStreamStatement, db *badger.DB) (string,error) {
  84. s, err := common.DbGet(db, string(stmt.Name))
  85. if err != nil {
  86. return "", fmt.Errorf("Stream %s is not found.", stmt.Name)
  87. }
  88. parser := xsql.NewParser(strings.NewReader(s))
  89. stream, err := xsql.Language.Parse(parser)
  90. streamStmt, ok := stream.(*xsql.StreamStmt)
  91. if !ok{
  92. return "", fmt.Errorf("Error resolving the stream %s, the data in db may be corrupted.", stmt.Name)
  93. }
  94. var buff bytes.Buffer
  95. buff.WriteString("Fields\n--------------------------------------------------------------------------------\n")
  96. for _, f := range streamStmt.StreamFields {
  97. buff.WriteString(f.Name + "\t")
  98. xsql.PrintFieldType(f.FieldType, &buff)
  99. buff.WriteString("\n")
  100. }
  101. buff.WriteString("\n")
  102. common.PrintMap(streamStmt.Options, &buff)
  103. return buff.String(), err
  104. }
  105. func (p *StreamProcessor) execExplainStream(stmt *xsql.ExplainStreamStatement, db *badger.DB) (string,error) {
  106. _, err := common.DbGet(db, string(stmt.Name))
  107. if err != nil{
  108. return "", fmt.Errorf("Stream %s is not found.", stmt.Name)
  109. }
  110. return "TO BE SUPPORTED", nil
  111. }
  112. func (p *StreamProcessor) execDropStream(stmt *xsql.DropStreamStatement, db *badger.DB) (string, error) {
  113. err := common.DbDelete(db, string(stmt.Name))
  114. if err != nil {
  115. return "", err
  116. }else{
  117. return fmt.Sprintf("Stream %s is dropped.", stmt.Name), nil
  118. }
  119. }
  120. func GetStream(db *badger.DB, name string) (stmt *xsql.StreamStmt, err error){
  121. s, err := common.DbGet(db, name)
  122. if err != nil {
  123. return
  124. }
  125. parser := xsql.NewParser(strings.NewReader(s))
  126. stream, err := xsql.Language.Parse(parser)
  127. stmt, ok := stream.(*xsql.StreamStmt)
  128. if !ok{
  129. err = fmt.Errorf("Error resolving the stream %s, the data in db may be corrupted.", name)
  130. }
  131. return
  132. }
  133. type RuleProcessor struct {
  134. badgerDir string
  135. }
  136. func NewRuleProcessor(d string) *RuleProcessor {
  137. processor := &RuleProcessor{
  138. badgerDir: d,
  139. }
  140. return processor
  141. }
  142. func (p *RuleProcessor) ExecCreate(name, ruleJson string) (*api.Rule, error) {
  143. rule, err := p.getRuleByJson(name, ruleJson)
  144. if err != nil {
  145. return nil, err
  146. }
  147. db, err := common.DbOpen(path.Join(p.badgerDir, "rule"))
  148. if err != nil {
  149. return nil, err
  150. }
  151. err = common.DbSet(db, string(name), ruleJson)
  152. if err != nil {
  153. common.DbClose(db)
  154. return nil, err
  155. }else{
  156. log.Infof("Rule %s is created.", name)
  157. common.DbClose(db)
  158. }
  159. return rule, nil
  160. }
  161. func (p *RuleProcessor) GetRuleByName(name string) (*api.Rule, error) {
  162. db, err := common.DbOpen(path.Join(p.badgerDir, "rule"))
  163. if err != nil {
  164. return nil, err
  165. }
  166. defer common.DbClose(db)
  167. s, err := common.DbGet(db, string(name))
  168. if err != nil {
  169. return nil, fmt.Errorf("Rule %s is not found.", name)
  170. }
  171. return p.getRuleByJson(name, s)
  172. }
  173. func (p *RuleProcessor) getRuleByJson(name, ruleJson string) (*api.Rule, error) {
  174. var rule api.Rule
  175. if err := json.Unmarshal([]byte(ruleJson), &rule); err != nil {
  176. return nil, fmt.Errorf("Parse rule %s error : %s.", ruleJson, err)
  177. }
  178. rule.Id = name
  179. //validation
  180. if name == ""{
  181. return nil, fmt.Errorf("Missing rule id.")
  182. }
  183. if rule.Sql == ""{
  184. return nil, fmt.Errorf("Missing rule SQL.")
  185. }
  186. if rule.Actions == nil || len(rule.Actions) == 0{
  187. return nil, fmt.Errorf("Missing rule actions.")
  188. }
  189. return &rule, nil
  190. }
  191. func (p *RuleProcessor) ExecInitRule(rule *api.Rule) (*xstream.TopologyNew, error) {
  192. if tp, inputs, err := p.createTopo(rule); err != nil {
  193. return nil, err
  194. }else{
  195. for _, m := range rule.Actions {
  196. for name, action := range m {
  197. switch name {
  198. case "log":
  199. log.Printf("Create log sink with %s.", action)
  200. tp.AddSink(inputs, nodes.NewSinkNode("sink_log", sinks.NewLogSink()))
  201. case "mqtt":
  202. log.Printf("Create mqtt sink with %s.", action)
  203. if ms, err := sinks.NewMqttSink(action); err != nil{
  204. return nil, err
  205. }else{
  206. tp.AddSink(inputs, nodes.NewSinkNode("sink_mqtt", ms))
  207. }
  208. default:
  209. return nil, fmt.Errorf("unsupported action: %s.", name)
  210. }
  211. }
  212. }
  213. return tp, nil
  214. }
  215. }
  216. func (p *RuleProcessor) ExecQuery(ruleid, sql string) (*xstream.TopologyNew, error) {
  217. if tp, inputs, err := p.createTopo(&api.Rule{Id: ruleid, Sql: sql}); err != nil {
  218. return nil, err
  219. } else {
  220. tp.AddSink(inputs, nodes.NewSinkNode("sink_memory_log", sinks.NewLogSinkToMemory()))
  221. go func() {
  222. select {
  223. case err := <-tp.Open():
  224. log.Println(err)
  225. tp.Cancel()
  226. }
  227. }()
  228. return tp, nil
  229. }
  230. }
  231. func (p *RuleProcessor) ExecDesc(name string) (string, error) {
  232. db, err := common.DbOpen(path.Join(p.badgerDir, "rule"))
  233. if err != nil {
  234. return "", err
  235. }
  236. defer common.DbClose(db)
  237. s, err := common.DbGet(db, string(name))
  238. if err != nil {
  239. return "", fmt.Errorf("Rule %s is not found.", name)
  240. }
  241. dst := &bytes.Buffer{}
  242. if err := json.Indent(dst, []byte(s), "", " "); err != nil {
  243. return "", err
  244. }
  245. return fmt.Sprintln(dst.String()), nil
  246. }
  247. func (p *RuleProcessor) ExecShow() (string, error) {
  248. keys, err := p.GetAllRules()
  249. if err != nil{
  250. return "", err
  251. }
  252. if len(keys) == 0 {
  253. keys = append(keys, "No rule definitions are found.")
  254. }
  255. var result string
  256. for _, c := range keys{
  257. result = result + fmt.Sprintln(c)
  258. }
  259. return result, nil
  260. }
  261. func (p *RuleProcessor) GetAllRules() ([]string, error) {
  262. db, err := common.DbOpen(path.Join(p.badgerDir, "rule"))
  263. if err != nil {
  264. return nil, err
  265. }
  266. defer common.DbClose(db)
  267. return common.DbKeys(db)
  268. }
  269. func (p *RuleProcessor) ExecDrop(name string) (string, error) {
  270. db, err := common.DbOpen(path.Join(p.badgerDir, "rule"))
  271. if err != nil {
  272. return "", err
  273. }
  274. defer common.DbClose(db)
  275. err = common.DbDelete(db, string(name))
  276. if err != nil {
  277. return "", err
  278. }else{
  279. return fmt.Sprintf("Rule %s is dropped.", name), nil
  280. }
  281. }
  282. func (p *RuleProcessor) createTopo(rule *api.Rule) (*xstream.TopologyNew, []api.Emitter, error) {
  283. return p.createTopoWithSources(rule, nil)
  284. }
  285. //For test to mock source
  286. func (p *RuleProcessor) createTopoWithSources(rule *api.Rule, sources []*nodes.SourceNode) (*xstream.TopologyNew, []api.Emitter, error){
  287. name := rule.Id
  288. sql := rule.Sql
  289. var isEventTime bool
  290. var lateTol int64
  291. if iet, ok := rule.Options["isEventTime"]; ok{
  292. isEventTime, ok = iet.(bool)
  293. if !ok{
  294. return nil, nil, fmt.Errorf("Invalid rule option isEventTime %v, bool type is required.", iet)
  295. }
  296. }
  297. if isEventTime {
  298. if l, ok := rule.Options["lateTolerance"]; ok{
  299. if fl, ok := l.(float64); ok{
  300. lateTol = int64(fl)
  301. }else{
  302. return nil, nil, fmt.Errorf("Invalid rule option lateTolerance %v, int type is required.", l)
  303. }
  304. }
  305. }
  306. shouldCreateSource := sources == nil
  307. parser := xsql.NewParser(strings.NewReader(sql))
  308. if stmt, err := xsql.Language.Parse(parser); err != nil{
  309. return nil, nil, fmt.Errorf("Parse SQL %s error: %s.", sql , err)
  310. }else {
  311. if selectStmt, ok := stmt.(*xsql.SelectStatement); !ok {
  312. return nil, nil, fmt.Errorf("SQL %s is not a select statement.", sql)
  313. } else {
  314. tp := xstream.NewWithName(name)
  315. var inputs []api.Emitter
  316. streamsFromStmt := xsql.GetStreams(selectStmt)
  317. if !shouldCreateSource && len(streamsFromStmt) != len(sources){
  318. return nil, nil, fmt.Errorf("Invalid parameter sources or streams, the length cannot match the statement, expect %d sources.", len(streamsFromStmt))
  319. }
  320. db, err := common.DbOpen(path.Join(p.badgerDir, "stream"))
  321. if err != nil {
  322. return nil, nil, err
  323. }
  324. defer common.DbClose(db)
  325. for i, s := range streamsFromStmt {
  326. streamStmt, err := GetStream(db, s)
  327. if err != nil {
  328. return nil, nil, fmt.Errorf("fail to get stream %s, please check if stream is created", s)
  329. }
  330. pp, err := plans.NewPreprocessor(streamStmt, selectStmt.Fields, isEventTime)
  331. if err != nil{
  332. return nil, nil, err
  333. }
  334. if shouldCreateSource{
  335. src, err := getSource(streamStmt)
  336. if err != nil {
  337. return nil, nil, fmt.Errorf("fail to get source: %v", err)
  338. }
  339. node := nodes.NewSourceNode(s, src)
  340. tp.AddSrc(node)
  341. preprocessorOp := xstream.Transform(pp, "preprocessor_"+s)
  342. tp.AddOperator([]api.Emitter{node}, preprocessorOp)
  343. inputs = append(inputs, preprocessorOp)
  344. } else {
  345. tp.AddSrc(sources[i])
  346. preprocessorOp := xstream.Transform(pp, "preprocessor_"+s)
  347. tp.AddOperator([]api.Emitter{sources[i]}, preprocessorOp)
  348. inputs = append(inputs, preprocessorOp)
  349. }
  350. }
  351. dimensions := selectStmt.Dimensions
  352. var w *xsql.Window
  353. if dimensions != nil {
  354. w = dimensions.GetWindow()
  355. if w != nil {
  356. wop, err := operators.NewWindowOp("window", w, isEventTime, lateTol, streamsFromStmt)
  357. if err != nil {
  358. return nil, nil, err
  359. }
  360. tp.AddOperator(inputs, wop)
  361. inputs = []api.Emitter{wop}
  362. }
  363. }
  364. if w != nil && selectStmt.Joins != nil {
  365. joinOp := xstream.Transform(&plans.JoinPlan{Joins: selectStmt.Joins, From: selectStmt.Sources[0].(*xsql.Table)}, "join")
  366. //TODO concurrency setting by command
  367. //joinOp.SetConcurrency(3)
  368. tp.AddOperator(inputs, joinOp)
  369. inputs = []api.Emitter{joinOp}
  370. }
  371. if selectStmt.Condition != nil {
  372. filterOp := xstream.Transform(&plans.FilterPlan{Condition: selectStmt.Condition}, "filter")
  373. //TODO concurrency setting by command
  374. // filterOp.SetConcurrency(3)
  375. tp.AddOperator(inputs, filterOp)
  376. inputs = []api.Emitter{filterOp}
  377. }
  378. var ds xsql.Dimensions
  379. if dimensions != nil {
  380. ds = dimensions.GetGroups()
  381. if ds != nil && len(ds) > 0 {
  382. aggregateOp := xstream.Transform(&plans.AggregatePlan{Dimensions: ds}, "aggregate")
  383. tp.AddOperator(inputs, aggregateOp)
  384. inputs = []api.Emitter{aggregateOp}
  385. }
  386. }
  387. if selectStmt.Having != nil {
  388. havingOp := xstream.Transform(&plans.HavingPlan{selectStmt.Having}, "having")
  389. tp.AddOperator(inputs, havingOp)
  390. inputs = []api.Emitter{havingOp}
  391. }
  392. if selectStmt.SortFields != nil {
  393. orderOp := xstream.Transform(&plans.OrderPlan{SortFields: selectStmt.SortFields}, "order")
  394. tp.AddOperator(inputs, orderOp)
  395. inputs = []api.Emitter{orderOp}
  396. }
  397. if selectStmt.Fields != nil {
  398. projectOp := xstream.Transform(&plans.ProjectPlan{Fields: selectStmt.Fields, IsAggregate: xsql.IsAggStatement(selectStmt)}, "project")
  399. tp.AddOperator(inputs, projectOp)
  400. inputs = []api.Emitter{projectOp}
  401. }
  402. return tp, inputs, nil
  403. }
  404. }
  405. }
  406. func getSource(streamStmt *xsql.StreamStmt) (api.Source, error) {
  407. if t, ok := streamStmt.Options["TYPE"]; ok{
  408. mod := "plugins/" + t + ".so"
  409. plug, err := plugin.Open(mod)
  410. if err != nil {
  411. return nil, fmt.Errorf("cannot open %s: %v", mod, err)
  412. }
  413. nf, err := plug.Lookup(t)
  414. if err != nil{
  415. return nil, fmt.Errorf("cannot find symbol %s, please check if it is exported", t)
  416. }
  417. s, ok := nf.(api.Source)
  418. if !ok {
  419. return nil, fmt.Errorf("exported symbol %s is not type of api.Source", t)
  420. }
  421. conf, err := common.LoadConf(t + ".yaml")
  422. props := make(map[string]interface{})
  423. if err == nil{
  424. cfg := make(map[string]map[string]interface{})
  425. if err := yaml.Unmarshal(conf, &cfg); err != nil {
  426. log.Warnf("fail to parse yaml for source %s. Return an empty configuration", t)
  427. }else{
  428. props, ok = cfg[streamStmt.Options["CONF_KEY"]]
  429. if !ok {
  430. log.Warnf("conf for conf_key %s not found, use default conf instead", streamStmt.Options["CONF_KEY"])
  431. props = cfg["default"]
  432. }
  433. }
  434. }else{
  435. log.Warnf("config file %s.yaml is not loaded properly. Return an empty configuration", t)
  436. }
  437. s.Configure(streamStmt.Options["DATASOURCE"], props)
  438. log.Tracef("Source %s created", t)
  439. return s, nil
  440. }else{
  441. mqs, err := extensions.NewMQTTSource(streamStmt.Options["DATASOURCE"], streamStmt.Options["CONF_KEY"])
  442. if err != nil {
  443. return nil, err
  444. }
  445. return mqs, nil
  446. }
  447. }