analyzer_test.go 9.7 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421
  1. // Copyright 2021-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. "encoding/json"
  17. "fmt"
  18. "reflect"
  19. "strings"
  20. "testing"
  21. "github.com/lf-edge/ekuiper/internal/pkg/store"
  22. "github.com/lf-edge/ekuiper/internal/testx"
  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. )
  27. func init() {
  28. }
  29. type errorStruct struct {
  30. err string
  31. serr *string
  32. }
  33. func newErrorStruct(err string) *errorStruct {
  34. return &errorStruct{
  35. err: err,
  36. }
  37. }
  38. func newErrorStructWithS(err string, serr string) *errorStruct {
  39. return &errorStruct{
  40. err: err,
  41. serr: &serr,
  42. }
  43. }
  44. func (e *errorStruct) Serr() string {
  45. if e.serr != nil {
  46. return *e.serr
  47. }
  48. return e.err
  49. }
  50. var tests = []struct {
  51. sql string
  52. r *errorStruct
  53. }{
  54. { // 0
  55. sql: `SELECT count(*) FROM src1 HAVING sin(temp) > 0.3`,
  56. r: newErrorStruct("Not allowed to call non-aggregate functions in HAVING clause."),
  57. },
  58. { // 1
  59. sql: `SELECT count(*) FROM src1 WHERE name = "dname" HAVING sin(count(*)) > 0.3`,
  60. r: newErrorStruct(""),
  61. },
  62. { // 2
  63. sql: `SELECT count(*) as c FROM src1 WHERE name = "dname" HAVING sin(c) > 0.3`,
  64. r: newErrorStruct(""),
  65. },
  66. { // 3
  67. sql: `SELECT count(*) as c FROM src1 WHERE name = "dname" HAVING sum(c) > 0.3`,
  68. r: newErrorStruct("invalid argument for func sum: aggregate argument is not allowed"),
  69. },
  70. { // 4
  71. sql: `SELECT count(*) as c FROM src1 WHERE name = "dname" GROUP BY sin(c)`,
  72. r: newErrorStruct("Not allowed to call aggregate functions in GROUP BY clause."),
  73. },
  74. { // 5
  75. sql: `SELECT count(*) as c FROM src1 WHERE name = "dname" HAVING sum(c) > 0.3 OR sin(temp) > 3`,
  76. r: newErrorStruct("Not allowed to call non-aggregate functions in HAVING clause."),
  77. },
  78. { // 6
  79. sql: `SELECT collect(*) as c FROM src1 WHERE name = "dname" HAVING c[2]->temp > 20 AND sin(c[0]->temp) > 0`,
  80. r: newErrorStruct(""),
  81. },
  82. { // 7
  83. sql: `SELECT collect(*) as c FROM src1 WHERE name = "dname" HAVING c[2]->temp + temp > 0`,
  84. r: newErrorStruct("Not allowed to call non-aggregate functions in HAVING clause."),
  85. },
  86. { // 8
  87. sql: `SELECT deduplicate(temp, true) as de FROM src1 HAVING cardinality(de) > 20`,
  88. r: newErrorStruct(""),
  89. },
  90. { // 9
  91. sql: `SELECT sin(temp) as temp FROM src1`,
  92. r: newErrorStruct(""),
  93. },
  94. { // 10
  95. sql: `SELECT sum(temp) as temp, count(temp) as temp FROM src1`,
  96. r: newErrorStruct("duplicate alias temp"),
  97. },
  98. { // 11
  99. sql: `SELECT sum(temp) as temp, count(temp) as ct FROM src1`,
  100. r: newErrorStruct(""),
  101. },
  102. { // 12
  103. sql: `SELECT collect(*)->abc FROM src1`,
  104. r: newErrorStruct(""),
  105. },
  106. { // 13
  107. sql: `SELECT sin(temp) as temp1, cos(temp1) FROM src1`,
  108. r: newErrorStructWithS("unknown field temp1", ""),
  109. },
  110. { // 14
  111. sql: `SELECT collect(*)[-1] as current FROM src1 GROUP BY COUNTWINDOW(2, 1) HAVING isNull(current->name) = false`,
  112. r: newErrorStruct(""),
  113. },
  114. { // 15
  115. sql: `SELECT sum(next->nid) as nid FROM src1 WHERE next->nid > 20 `,
  116. r: newErrorStruct(""),
  117. },
  118. { // 16
  119. sql: `SELECT collect(*)[0] as last FROM src1 GROUP BY SlidingWindow(ss,5) HAVING last.temp > 30`,
  120. r: newErrorStruct(""),
  121. },
  122. }
  123. func Test_validation(t *testing.T) {
  124. store, err := store.GetKV("stream")
  125. if err != nil {
  126. t.Error(err)
  127. return
  128. }
  129. streamSqls := map[string]string{
  130. "src1": `CREATE STREAM src1 (
  131. id1 BIGINT,
  132. temp BIGINT,
  133. name string,
  134. next STRUCT(NAME STRING, NID BIGINT)
  135. ) WITH (DATASOURCE="src1", FORMAT="json", KEY="ts");`,
  136. }
  137. types := map[string]ast.StreamType{
  138. "src1": ast.TypeStream,
  139. }
  140. for name, sql := range streamSqls {
  141. s, err := json.Marshal(&xsql.StreamInfo{
  142. StreamType: types[name],
  143. Statement: sql,
  144. })
  145. if err != nil {
  146. t.Error(err)
  147. t.Fail()
  148. }
  149. store.Set(name, string(s))
  150. }
  151. streams := make(map[string]*ast.StreamStmt)
  152. for n := range streamSqls {
  153. streamStmt, err := xsql.GetDataSource(store, n)
  154. if err != nil {
  155. t.Errorf("fail to get stream %s, please check if stream is created", n)
  156. return
  157. }
  158. streams[n] = streamStmt
  159. }
  160. fmt.Printf("The test bucket size is %d.\n\n", len(tests))
  161. for i, tt := range tests {
  162. stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
  163. if err != nil {
  164. t.Errorf("%d. %q: error compile sql: %s\n", i, tt.sql, err)
  165. continue
  166. }
  167. _, err = createLogicalPlan(stmt, &api.RuleOption{
  168. IsEventTime: false,
  169. LateTol: 0,
  170. Concurrency: 0,
  171. BufferLength: 0,
  172. SendMetaToSink: false,
  173. Qos: 0,
  174. CheckpointInterval: 0,
  175. SendError: true,
  176. }, store)
  177. if !reflect.DeepEqual(tt.r.err, testx.Errstring(err)) {
  178. t.Errorf("%d. %q: error mismatch:\n exp=%s\n got=%s\n\n", i, tt.sql, tt.r.err, err)
  179. }
  180. }
  181. }
  182. func Test_validationSchemaless(t *testing.T) {
  183. store, err := store.GetKV("stream")
  184. if err != nil {
  185. t.Error(err)
  186. return
  187. }
  188. streamSqls := map[string]string{
  189. "src1": `CREATE STREAM src1 (
  190. ) WITH (DATASOURCE="src1", FORMAT="json", KEY="ts");`,
  191. }
  192. types := map[string]ast.StreamType{
  193. "src1": ast.TypeStream,
  194. }
  195. for name, sql := range streamSqls {
  196. s, err := json.Marshal(&xsql.StreamInfo{
  197. StreamType: types[name],
  198. Statement: sql,
  199. })
  200. if err != nil {
  201. t.Error(err)
  202. t.Fail()
  203. }
  204. store.Set(name, string(s))
  205. }
  206. streams := make(map[string]*ast.StreamStmt)
  207. for n := range streamSqls {
  208. streamStmt, err := xsql.GetDataSource(store, n)
  209. if err != nil {
  210. t.Errorf("fail to get stream %s, please check if stream is created", n)
  211. return
  212. }
  213. streams[n] = streamStmt
  214. }
  215. fmt.Printf("The test bucket size is %d.\n\n", len(tests))
  216. for i, tt := range tests {
  217. stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
  218. if err != nil {
  219. t.Errorf("%d. %q: error compile sql: %s\n", i, tt.sql, err)
  220. continue
  221. }
  222. _, err = createLogicalPlan(stmt, &api.RuleOption{
  223. IsEventTime: false,
  224. LateTol: 0,
  225. Concurrency: 0,
  226. BufferLength: 0,
  227. SendMetaToSink: false,
  228. Qos: 0,
  229. CheckpointInterval: 0,
  230. SendError: true,
  231. }, store)
  232. serr := tt.r.Serr()
  233. if !reflect.DeepEqual(serr, testx.Errstring(err)) {
  234. t.Errorf("%d. %q: error mismatch:\n exp=%s\n got=%s\n\n", i, tt.sql, serr, err)
  235. }
  236. }
  237. }
  238. func TestConvertStreamInfo(t *testing.T) {
  239. testCases := []struct {
  240. name string
  241. streamStmt *ast.StreamStmt
  242. expected ast.StreamFields
  243. }{
  244. {
  245. name: "with match fields & schema",
  246. streamStmt: &ast.StreamStmt{
  247. StreamFields: []ast.StreamField{
  248. {
  249. Name: "field1",
  250. FieldType: &ast.BasicType{
  251. Type: ast.BIGINT,
  252. },
  253. },
  254. {
  255. Name: "field2",
  256. FieldType: &ast.BasicType{
  257. Type: ast.STRINGS,
  258. },
  259. },
  260. },
  261. Options: &ast.Options{
  262. FORMAT: "protobuf",
  263. SCHEMAID: "myschema.schema1",
  264. TIMESTAMP: "ts",
  265. },
  266. },
  267. expected: []ast.StreamField{
  268. {
  269. Name: "field1",
  270. FieldType: &ast.BasicType{
  271. Type: ast.BIGINT,
  272. },
  273. },
  274. {
  275. Name: "field2",
  276. FieldType: &ast.BasicType{
  277. Type: ast.STRINGS,
  278. },
  279. },
  280. },
  281. },
  282. {
  283. name: "with unmatch fields & schema",
  284. streamStmt: &ast.StreamStmt{
  285. StreamFields: []ast.StreamField{
  286. {
  287. Name: "field1",
  288. FieldType: &ast.BasicType{
  289. Type: ast.STRINGS,
  290. },
  291. },
  292. {
  293. Name: "field2",
  294. FieldType: &ast.BasicType{
  295. Type: ast.STRINGS,
  296. },
  297. },
  298. },
  299. Options: &ast.Options{
  300. FORMAT: "protobuf",
  301. SCHEMAID: "myschema.schema1",
  302. TIMESTAMP: "ts",
  303. },
  304. },
  305. expected: []ast.StreamField{
  306. {
  307. Name: "field1",
  308. FieldType: &ast.BasicType{
  309. Type: ast.BIGINT,
  310. },
  311. },
  312. {
  313. Name: "field2",
  314. FieldType: &ast.BasicType{
  315. Type: ast.STRINGS,
  316. },
  317. },
  318. },
  319. },
  320. {
  321. name: "without schema",
  322. streamStmt: &ast.StreamStmt{
  323. StreamFields: []ast.StreamField{
  324. {
  325. Name: "field1",
  326. FieldType: &ast.BasicType{
  327. Type: ast.FLOAT,
  328. },
  329. },
  330. {
  331. Name: "field2",
  332. FieldType: &ast.BasicType{
  333. Type: ast.STRINGS,
  334. },
  335. },
  336. },
  337. Options: &ast.Options{
  338. FORMAT: "json",
  339. TIMESTAMP: "ts",
  340. },
  341. },
  342. expected: []ast.StreamField{
  343. {
  344. Name: "field1",
  345. FieldType: &ast.BasicType{
  346. Type: ast.FLOAT,
  347. },
  348. },
  349. {
  350. Name: "field2",
  351. FieldType: &ast.BasicType{
  352. Type: ast.STRINGS,
  353. },
  354. },
  355. },
  356. },
  357. {
  358. name: "without fields",
  359. streamStmt: &ast.StreamStmt{
  360. Options: &ast.Options{
  361. FORMAT: "protobuf",
  362. SCHEMAID: "myschema.schema1",
  363. TIMESTAMP: "ts",
  364. },
  365. },
  366. expected: []ast.StreamField{
  367. {
  368. Name: "field1",
  369. FieldType: &ast.BasicType{
  370. Type: ast.BIGINT,
  371. },
  372. },
  373. {
  374. Name: "field2",
  375. FieldType: &ast.BasicType{
  376. Type: ast.STRINGS,
  377. },
  378. },
  379. },
  380. },
  381. {
  382. name: "schemaless",
  383. streamStmt: &ast.StreamStmt{
  384. Options: &ast.Options{
  385. FORMAT: "json",
  386. TIMESTAMP: "ts",
  387. },
  388. },
  389. expected: nil,
  390. },
  391. }
  392. for _, tc := range testCases {
  393. t.Run(tc.name, func(t *testing.T) {
  394. actual, err := convertStreamInfo(tc.streamStmt)
  395. if err != nil {
  396. t.Errorf("unexpected error: %v", err)
  397. return
  398. }
  399. if !reflect.DeepEqual(actual.schema, tc.expected) {
  400. t.Errorf("unexpected result: got %v, want %v", actual.schema, tc.expected)
  401. }
  402. })
  403. }
  404. }