xsql_processor.go 15 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548
  1. package processors
  2. import (
  3. "bytes"
  4. "encoding/json"
  5. "fmt"
  6. "github.com/emqx/kuiper/common"
  7. "github.com/emqx/kuiper/xsql"
  8. "github.com/emqx/kuiper/xsql/plans"
  9. "github.com/emqx/kuiper/xstream"
  10. "github.com/emqx/kuiper/xstream/api"
  11. "github.com/emqx/kuiper/xstream/nodes"
  12. "github.com/emqx/kuiper/xstream/operators"
  13. "path"
  14. "strings"
  15. )
  16. var log = common.Log
  17. type StreamProcessor struct {
  18. db common.KeyValue
  19. }
  20. //@params d : the directory of the DB to save the stream info
  21. func NewStreamProcessor(d string) *StreamProcessor {
  22. processor := &StreamProcessor{
  23. db: common.GetSimpleKVStore(d),
  24. }
  25. return processor
  26. }
  27. func (p *StreamProcessor) ExecStmt(statement string) (result []string, err error) {
  28. parser := xsql.NewParser(strings.NewReader(statement))
  29. stmt, err := xsql.Language.Parse(parser)
  30. if err != nil {
  31. return nil, err
  32. }
  33. switch s := stmt.(type) {
  34. case *xsql.StreamStmt:
  35. var r string
  36. r, err = p.execCreateStream(s, statement)
  37. result = append(result, r)
  38. case *xsql.ShowStreamsStatement:
  39. result, err = p.execShowStream(s)
  40. case *xsql.DescribeStreamStatement:
  41. var r string
  42. r, err = p.execDescribeStream(s)
  43. result = append(result, r)
  44. case *xsql.ExplainStreamStatement:
  45. var r string
  46. r, err = p.execExplainStream(s)
  47. result = append(result, r)
  48. case *xsql.DropStreamStatement:
  49. var r string
  50. r, err = p.execDropStream(s)
  51. result = append(result, r)
  52. default:
  53. return nil, fmt.Errorf("Invalid stream statement: %s", statement)
  54. }
  55. return
  56. }
  57. func (p *StreamProcessor) execCreateStream(stmt *xsql.StreamStmt, statement string) (string, error) {
  58. err := p.db.Open()
  59. if err != nil {
  60. return "", fmt.Errorf("Create stream fails, error when opening db: %v.", err)
  61. }
  62. defer p.db.Close()
  63. err = p.db.Set(string(stmt.Name), statement)
  64. if err != nil {
  65. return "", fmt.Errorf("Create stream fails: %v.", err)
  66. } else {
  67. info := fmt.Sprintf("Stream %s is created.", stmt.Name)
  68. log.Printf("%s", info)
  69. return info, nil
  70. }
  71. }
  72. func (p *StreamProcessor) ExecStreamSql(statement string) (string, error) {
  73. r, err := p.ExecStmt(statement)
  74. if err != nil {
  75. return "", err
  76. } else {
  77. return strings.Join(r, "\n"), err
  78. }
  79. }
  80. func (p *StreamProcessor) execShowStream(stmt *xsql.ShowStreamsStatement) ([]string, error) {
  81. keys, err := p.ShowStream()
  82. if len(keys) == 0 {
  83. keys = append(keys, "No stream definitions are found.")
  84. }
  85. return keys, err
  86. }
  87. func (p *StreamProcessor) ShowStream() ([]string, error) {
  88. err := p.db.Open()
  89. if err != nil {
  90. return nil, fmt.Errorf("Show stream fails, error when opening db: %v.", err)
  91. }
  92. defer p.db.Close()
  93. return p.db.Keys()
  94. }
  95. func (p *StreamProcessor) execDescribeStream(stmt *xsql.DescribeStreamStatement) (string, error) {
  96. streamStmt, err := p.DescStream(stmt.Name)
  97. if err != nil {
  98. return "", err
  99. }
  100. var buff bytes.Buffer
  101. buff.WriteString("Fields\n--------------------------------------------------------------------------------\n")
  102. for _, f := range streamStmt.StreamFields {
  103. buff.WriteString(f.Name + "\t")
  104. buff.WriteString(xsql.PrintFieldType(f.FieldType))
  105. buff.WriteString("\n")
  106. }
  107. buff.WriteString("\n")
  108. common.PrintMap(streamStmt.Options, &buff)
  109. return buff.String(), err
  110. }
  111. func (p *StreamProcessor) DescStream(name string) (*xsql.StreamStmt, error) {
  112. err := p.db.Open()
  113. if err != nil {
  114. return nil, fmt.Errorf("Describe stream fails, error when opening db: %v.", err)
  115. }
  116. defer p.db.Close()
  117. s, f := p.db.Get(name)
  118. if !f {
  119. return nil, common.NewErrorWithCode(common.NOT_FOUND, fmt.Sprintf("Stream %s is not found.", name))
  120. }
  121. s1 := s.(string)
  122. parser := xsql.NewParser(strings.NewReader(s1))
  123. stream, err := xsql.Language.Parse(parser)
  124. if err != nil {
  125. return nil, err
  126. }
  127. streamStmt, ok := stream.(*xsql.StreamStmt)
  128. if !ok {
  129. return nil, fmt.Errorf("Error resolving the stream %s, the data in db may be corrupted.", name)
  130. }
  131. return streamStmt, nil
  132. }
  133. func (p *StreamProcessor) execExplainStream(stmt *xsql.ExplainStreamStatement) (string, error) {
  134. err := p.db.Open()
  135. if err != nil {
  136. return "", fmt.Errorf("Explain stream fails, error when opening db: %v.", err)
  137. }
  138. defer p.db.Close()
  139. _, f := p.db.Get(stmt.Name)
  140. if !f {
  141. return "", fmt.Errorf("Stream %s is not found.", stmt.Name)
  142. }
  143. return "TO BE SUPPORTED", nil
  144. }
  145. func (p *StreamProcessor) execDropStream(stmt *xsql.DropStreamStatement) (string, error) {
  146. s, err := p.DropStream(stmt.Name)
  147. if err != nil {
  148. return s, fmt.Errorf("Drop stream fails: %s.", err)
  149. }
  150. return s, nil
  151. }
  152. func (p *StreamProcessor) DropStream(name string) (string, error) {
  153. err := p.db.Open()
  154. if err != nil {
  155. return "", fmt.Errorf("error when opening db: %v", err)
  156. }
  157. defer p.db.Close()
  158. err = p.db.Delete(name)
  159. if err != nil {
  160. return "", err
  161. } else {
  162. return fmt.Sprintf("Stream %s is dropped.", name), nil
  163. }
  164. }
  165. func GetStream(m *common.SimpleKVStore, name string) (stmt *xsql.StreamStmt, err error) {
  166. s, f := m.Get(name)
  167. if !f {
  168. return nil, fmt.Errorf("Cannot find key %s. ", name)
  169. }
  170. s1, _ := s.(string)
  171. parser := xsql.NewParser(strings.NewReader(s1))
  172. stream, err := xsql.Language.Parse(parser)
  173. stmt, ok := stream.(*xsql.StreamStmt)
  174. if !ok {
  175. err = fmt.Errorf("Error resolving the stream %s, the data in db may be corrupted.", name)
  176. }
  177. return
  178. }
  179. type RuleProcessor struct {
  180. db common.KeyValue
  181. rootDbDir string
  182. }
  183. func NewRuleProcessor(d string) *RuleProcessor {
  184. processor := &RuleProcessor{
  185. db: common.GetSimpleKVStore(path.Join(d, "rule")),
  186. rootDbDir: d,
  187. }
  188. return processor
  189. }
  190. func (p *RuleProcessor) ExecCreate(name, ruleJson string) (*api.Rule, error) {
  191. rule, err := p.getRuleByJson(name, ruleJson)
  192. if err != nil {
  193. return nil, err
  194. }
  195. err = p.db.Open()
  196. if err != nil {
  197. return nil, err
  198. }
  199. defer p.db.Close()
  200. err = p.db.Set(rule.Id, ruleJson)
  201. if err != nil {
  202. return nil, err
  203. } else {
  204. log.Infof("Rule %s is created.", rule.Id)
  205. }
  206. return rule, nil
  207. }
  208. func (p *RuleProcessor) ExecReplaceRuleState(name string, triggered bool) (err error) {
  209. rule, err := p.GetRuleByName(name)
  210. if err != nil {
  211. return err
  212. }
  213. rule.Triggered = triggered
  214. ruleJson, err := json.Marshal(rule)
  215. if err != nil {
  216. return fmt.Errorf("Marshal rule %s error : %s.", name, err)
  217. }
  218. err = p.db.Open()
  219. if err != nil {
  220. return err
  221. }
  222. defer p.db.Close()
  223. err = p.db.Replace(name, string(ruleJson))
  224. if err != nil {
  225. return err
  226. } else {
  227. log.Infof("Rule %s is replaced.", name)
  228. }
  229. return err
  230. }
  231. func (p *RuleProcessor) GetRuleByName(name string) (*api.Rule, error) {
  232. err := p.db.Open()
  233. if err != nil {
  234. return nil, err
  235. }
  236. defer p.db.Close()
  237. s, f := p.db.Get(name)
  238. if !f {
  239. return nil, common.NewErrorWithCode(common.NOT_FOUND, fmt.Sprintf("Rule %s is not found.", name))
  240. }
  241. s1, _ := s.(string)
  242. return p.getRuleByJson(name, s1)
  243. }
  244. func (p *RuleProcessor) getRuleByJson(name, ruleJson string) (*api.Rule, error) {
  245. var rule api.Rule
  246. if err := json.Unmarshal([]byte(ruleJson), &rule); err != nil {
  247. return nil, fmt.Errorf("Parse rule %s error : %s.", ruleJson, err)
  248. }
  249. //validation
  250. if rule.Id == "" && name == "" {
  251. return nil, fmt.Errorf("Missing rule id.")
  252. }
  253. if name != "" && rule.Id != "" && name != rule.Id {
  254. return nil, fmt.Errorf("Name is not consistent with rule id.")
  255. }
  256. if rule.Id == "" {
  257. rule.Id = name
  258. }
  259. if rule.Sql == "" {
  260. return nil, fmt.Errorf("Missing rule SQL.")
  261. }
  262. if rule.Actions == nil || len(rule.Actions) == 0 {
  263. return nil, fmt.Errorf("Missing rule actions.")
  264. }
  265. return &rule, nil
  266. }
  267. func (p *RuleProcessor) ExecInitRule(rule *api.Rule) (*xstream.TopologyNew, error) {
  268. if tp, inputs, err := p.createTopo(rule); err != nil {
  269. return nil, err
  270. } else {
  271. for i, m := range rule.Actions {
  272. for name, action := range m {
  273. props, ok := action.(map[string]interface{})
  274. if !ok {
  275. return nil, fmt.Errorf("expect map[string]interface{} type for the action properties, but found %v", action)
  276. }
  277. tp.AddSink(inputs, nodes.NewSinkNode(fmt.Sprintf("%s_%d", name, i), name, props))
  278. }
  279. }
  280. return tp, nil
  281. }
  282. }
  283. func (p *RuleProcessor) ExecQuery(ruleid, sql string) (*xstream.TopologyNew, error) {
  284. if tp, inputs, err := p.createTopo(&api.Rule{Id: ruleid, Sql: sql}); err != nil {
  285. return nil, err
  286. } else {
  287. tp.AddSink(inputs, nodes.NewSinkNode("sink_memory_log", "logToMemory", nil))
  288. go func() {
  289. select {
  290. case err := <-tp.Open():
  291. log.Infof("closing query for error: %v", err)
  292. tp.GetContext().SetError(err)
  293. tp.Cancel()
  294. }
  295. }()
  296. return tp, nil
  297. }
  298. }
  299. func (p *RuleProcessor) ExecDesc(name string) (string, error) {
  300. err := p.db.Open()
  301. if err != nil {
  302. return "", err
  303. }
  304. defer p.db.Close()
  305. s, f := p.db.Get(name)
  306. if !f {
  307. return "", fmt.Errorf("Rule %s is not found.", name)
  308. }
  309. s1, _ := s.(string)
  310. dst := &bytes.Buffer{}
  311. if err := json.Indent(dst, []byte(s1), "", " "); err != nil {
  312. return "", err
  313. }
  314. return fmt.Sprintln(dst.String()), nil
  315. }
  316. func (p *RuleProcessor) GetAllRules() ([]string, error) {
  317. err := p.db.Open()
  318. if err != nil {
  319. return nil, err
  320. }
  321. defer p.db.Close()
  322. return p.db.Keys()
  323. }
  324. func (p *RuleProcessor) ExecDrop(name string) (string, error) {
  325. err := p.db.Open()
  326. if err != nil {
  327. return "", err
  328. }
  329. defer p.db.Close()
  330. err = p.db.Delete(name)
  331. if err != nil {
  332. return "", err
  333. } else {
  334. return fmt.Sprintf("Rule %s is dropped.", name), nil
  335. }
  336. }
  337. func (p *RuleProcessor) createTopo(rule *api.Rule) (*xstream.TopologyNew, []api.Emitter, error) {
  338. return p.createTopoWithSources(rule, nil)
  339. }
  340. //For test to mock source
  341. func (p *RuleProcessor) createTopoWithSources(rule *api.Rule, sources []*nodes.SourceNode) (*xstream.TopologyNew, []api.Emitter, error) {
  342. name := rule.Id
  343. sql := rule.Sql
  344. var (
  345. isEventTime bool
  346. lateTol int64
  347. concurrency = 1
  348. bufferLength = 1024
  349. sendMataToSink = false
  350. )
  351. if iet, ok := rule.Options["isEventTime"]; ok {
  352. isEventTime, ok = iet.(bool)
  353. if !ok {
  354. return nil, nil, fmt.Errorf("Invalid rule option isEventTime %v, bool type is required.", iet)
  355. }
  356. }
  357. if isEventTime {
  358. if l, ok := rule.Options["lateTolerance"]; ok {
  359. if fl, ok := l.(float64); ok {
  360. lateTol = int64(fl)
  361. } else {
  362. return nil, nil, fmt.Errorf("Invalid rule option lateTolerance %v, int type is required.", l)
  363. }
  364. }
  365. }
  366. if l, ok := rule.Options["concurrency"]; ok {
  367. if fl, ok := l.(float64); ok {
  368. concurrency = int(fl)
  369. } else {
  370. return nil, nil, fmt.Errorf("Invalid rule option concurrency %v, int type is required.", l)
  371. }
  372. }
  373. if l, ok := rule.Options["bufferLength"]; ok {
  374. if fl, ok := l.(float64); ok {
  375. bufferLength = int(fl)
  376. } else {
  377. return nil, nil, fmt.Errorf("Invalid rule option bufferLength %v, int type is required.", l)
  378. }
  379. }
  380. if l, ok := rule.Options["sendMetaToSink"]; ok {
  381. if fl, ok := l.(bool); ok {
  382. sendMataToSink = fl
  383. } else {
  384. return nil, nil, fmt.Errorf("Invalid rule option sendMetaToSink %v, bool type is required.", l)
  385. }
  386. }
  387. log.Infof("Init rule with options {isEventTime: %v, lateTolerance: %d, concurrency: %d, bufferLength: %d", isEventTime, lateTol, concurrency, bufferLength)
  388. shouldCreateSource := sources == nil
  389. parser := xsql.NewParser(strings.NewReader(sql))
  390. if stmt, err := xsql.Language.Parse(parser); err != nil {
  391. return nil, nil, fmt.Errorf("Parse SQL %s error: %s.", sql, err)
  392. } else {
  393. if selectStmt, ok := stmt.(*xsql.SelectStatement); !ok {
  394. return nil, nil, fmt.Errorf("SQL %s is not a select statement.", sql)
  395. } else {
  396. tp := xstream.NewWithName(name)
  397. var inputs []api.Emitter
  398. streamsFromStmt := xsql.GetStreams(selectStmt)
  399. dimensions := selectStmt.Dimensions
  400. if !shouldCreateSource && len(streamsFromStmt) != len(sources) {
  401. return nil, nil, fmt.Errorf("Invalid parameter sources or streams, the length cannot match the statement, expect %d sources.", len(streamsFromStmt))
  402. }
  403. if sendMataToSink && (len(streamsFromStmt) > 1 || dimensions != nil) {
  404. return nil, nil, fmt.Errorf("Invalid option sendMetaToSink, it can not be applied to window")
  405. }
  406. store := common.GetSimpleKVStore(path.Join(p.rootDbDir, "stream"))
  407. err := store.Open()
  408. if err != nil {
  409. return nil, nil, err
  410. }
  411. defer store.Close()
  412. var alias, aggregateAlias xsql.Fields
  413. for _, f := range selectStmt.Fields {
  414. if f.AName != "" {
  415. if !xsql.HasAggFuncs(f.Expr) {
  416. alias = append(alias, f)
  417. } else {
  418. aggregateAlias = append(aggregateAlias, f)
  419. }
  420. }
  421. }
  422. for i, s := range streamsFromStmt {
  423. streamStmt, err := GetStream(store, s)
  424. if err != nil {
  425. return nil, nil, fmt.Errorf("fail to get stream %s, please check if stream is created", s)
  426. }
  427. pp, err := plans.NewPreprocessor(streamStmt, alias, isEventTime)
  428. if err != nil {
  429. return nil, nil, err
  430. }
  431. if shouldCreateSource {
  432. node := nodes.NewSourceNode(s, streamStmt.Options)
  433. tp.AddSrc(node)
  434. preprocessorOp := xstream.Transform(pp, "preprocessor_"+s, bufferLength)
  435. preprocessorOp.SetConcurrency(concurrency)
  436. tp.AddOperator([]api.Emitter{node}, preprocessorOp)
  437. inputs = append(inputs, preprocessorOp)
  438. } else {
  439. tp.AddSrc(sources[i])
  440. preprocessorOp := xstream.Transform(pp, "preprocessor_"+s, bufferLength)
  441. preprocessorOp.SetConcurrency(concurrency)
  442. tp.AddOperator([]api.Emitter{sources[i]}, preprocessorOp)
  443. inputs = append(inputs, preprocessorOp)
  444. }
  445. }
  446. var w *xsql.Window
  447. if dimensions != nil {
  448. w = dimensions.GetWindow()
  449. if w != nil {
  450. wop, err := operators.NewWindowOp("window", w, isEventTime, lateTol, streamsFromStmt, bufferLength)
  451. if err != nil {
  452. return nil, nil, err
  453. }
  454. tp.AddOperator(inputs, wop)
  455. inputs = []api.Emitter{wop}
  456. }
  457. }
  458. if w != nil && selectStmt.Joins != nil {
  459. joinOp := xstream.Transform(&plans.JoinPlan{Joins: selectStmt.Joins, From: selectStmt.Sources[0].(*xsql.Table)}, "join", bufferLength)
  460. joinOp.SetConcurrency(concurrency)
  461. tp.AddOperator(inputs, joinOp)
  462. inputs = []api.Emitter{joinOp}
  463. }
  464. if selectStmt.Condition != nil {
  465. filterOp := xstream.Transform(&plans.FilterPlan{Condition: selectStmt.Condition}, "filter", bufferLength)
  466. filterOp.SetConcurrency(concurrency)
  467. tp.AddOperator(inputs, filterOp)
  468. inputs = []api.Emitter{filterOp}
  469. }
  470. var ds xsql.Dimensions
  471. if dimensions != nil || len(aggregateAlias) > 0 {
  472. ds = dimensions.GetGroups()
  473. if (ds != nil && len(ds) > 0) || len(aggregateAlias) > 0 {
  474. aggregateOp := xstream.Transform(&plans.AggregatePlan{Dimensions: ds, Alias: aggregateAlias}, "aggregate", bufferLength)
  475. aggregateOp.SetConcurrency(concurrency)
  476. tp.AddOperator(inputs, aggregateOp)
  477. inputs = []api.Emitter{aggregateOp}
  478. }
  479. }
  480. if selectStmt.Having != nil {
  481. havingOp := xstream.Transform(&plans.HavingPlan{selectStmt.Having}, "having", bufferLength)
  482. havingOp.SetConcurrency(concurrency)
  483. tp.AddOperator(inputs, havingOp)
  484. inputs = []api.Emitter{havingOp}
  485. }
  486. if selectStmt.SortFields != nil {
  487. orderOp := xstream.Transform(&plans.OrderPlan{SortFields: selectStmt.SortFields}, "order", bufferLength)
  488. orderOp.SetConcurrency(concurrency)
  489. tp.AddOperator(inputs, orderOp)
  490. inputs = []api.Emitter{orderOp}
  491. }
  492. if selectStmt.Fields != nil {
  493. projectOp := xstream.Transform(&plans.ProjectPlan{Fields: selectStmt.Fields, IsAggregate: xsql.IsAggStatement(selectStmt), SendMeta: sendMataToSink}, "project", bufferLength)
  494. projectOp.SetConcurrency(concurrency)
  495. tp.AddOperator(inputs, projectOp)
  496. inputs = []api.Emitter{projectOp}
  497. }
  498. return tp, inputs, nil
  499. }
  500. }
  501. }