checkpoint_test.go 6.1 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220
  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 topotest
  15. import (
  16. "testing"
  17. "github.com/lf-edge/ekuiper/internal/conf"
  18. "github.com/lf-edge/ekuiper/pkg/api"
  19. )
  20. // Full lifecycle test: Run window rule; trigger checkpoints by mock timer; restart rule; make sure the result is right;
  21. func TestCheckpoint(t *testing.T) {
  22. conf.IsTesting = true
  23. streamList := []string{"demo"}
  24. HandleStream(false, streamList, t)
  25. tests := []RuleCheckpointTest{
  26. {
  27. RuleTest: RuleTest{
  28. Name: `TestCheckpointRule1`,
  29. Sql: `SELECT * FROM demo GROUP BY HOPPINGWINDOW(ss, 2, 1)`,
  30. R: [][]map[string]interface{}{
  31. {{
  32. "color": "red",
  33. "size": float64(3),
  34. "ts": float64(1541152486013),
  35. }, {
  36. "color": "blue",
  37. "size": float64(6),
  38. "ts": float64(1541152486822),
  39. }},
  40. {{
  41. "color": "red",
  42. "size": float64(3),
  43. "ts": float64(1541152486013),
  44. }, {
  45. "color": "blue",
  46. "size": float64(6),
  47. "ts": float64(1541152486822),
  48. }},
  49. {{
  50. "color": "blue",
  51. "size": float64(2),
  52. "ts": float64(1541152487632),
  53. }},
  54. {{
  55. "color": "blue",
  56. "size": float64(2),
  57. "ts": float64(1541152487632),
  58. }, {
  59. "color": "yellow",
  60. "size": float64(4),
  61. "ts": float64(1541152488442),
  62. }},
  63. {{
  64. "color": "yellow",
  65. "size": float64(4),
  66. "ts": float64(1541152488442),
  67. }, {
  68. "color": "red",
  69. "size": float64(1),
  70. "ts": float64(1541152489252),
  71. }},
  72. },
  73. M: map[string]interface{}{
  74. "op_3_project_0_records_in_total": int64(4),
  75. "op_3_project_0_records_out_total": int64(4),
  76. "sink_mockSink_0_records_in_total": int64(4),
  77. "sink_mockSink_0_records_out_total": int64(4),
  78. "source_demo_0_records_in_total": int64(3),
  79. "source_demo_0_records_out_total": int64(3),
  80. "op_2_window_0_records_in_total": int64(3),
  81. "op_2_window_0_records_out_total": int64(4),
  82. },
  83. },
  84. PauseSize: 3,
  85. Cc: 2,
  86. PauseMetric: map[string]interface{}{
  87. "op_3_project_0_records_in_total": int64(1),
  88. "op_3_project_0_records_out_total": int64(1),
  89. "sink_mockSink_0_records_in_total": int64(1),
  90. "sink_mockSink_0_records_out_total": int64(1),
  91. "source_demo_0_records_in_total": int64(3),
  92. "source_demo_0_records_out_total": int64(3),
  93. "op_2_window_0_records_in_total": int64(3),
  94. "op_2_window_0_records_out_total": int64(1),
  95. },
  96. },
  97. }
  98. HandleStream(true, streamList, t)
  99. options := []*api.RuleOption{
  100. {
  101. BufferLength: 100,
  102. Qos: api.AtLeastOnce,
  103. CheckpointInterval: 600,
  104. SendError: true,
  105. }, {
  106. BufferLength: 100,
  107. Qos: api.ExactlyOnce,
  108. CheckpointInterval: 600,
  109. SendError: true,
  110. },
  111. }
  112. for j, opt := range options {
  113. DoCheckpointRuleTest(t, tests, j, opt)
  114. }
  115. }
  116. func TestTableJoinCheckpoint(t *testing.T) {
  117. conf.IsTesting = true
  118. streamList := []string{"demo", "table1"}
  119. HandleStream(false, streamList, t)
  120. tests := []RuleCheckpointTest{
  121. {
  122. RuleTest: RuleTest{
  123. Name: `TestCheckpointRule2`,
  124. Sql: `SELECT * FROM demo INNER JOIN table1 on demo.ts = table1.id`,
  125. R: [][]map[string]interface{}{
  126. {{
  127. "id": float64(1541152486013),
  128. "name": "name1",
  129. "color": "red",
  130. "size": float64(3),
  131. "ts": float64(1541152486013),
  132. }},
  133. {{
  134. "id": float64(1541152487632),
  135. "name": "name2",
  136. "color": "blue",
  137. "size": float64(2),
  138. "ts": float64(1541152487632),
  139. }},
  140. {{
  141. "id": float64(1541152487632),
  142. "name": "name2",
  143. "color": "blue",
  144. "size": float64(2),
  145. "ts": float64(1541152487632),
  146. }},
  147. {{
  148. "id": float64(1541152489252),
  149. "name": "name3",
  150. "color": "red",
  151. "size": float64(1),
  152. "ts": float64(1541152489252),
  153. }},
  154. },
  155. M: map[string]interface{}{
  156. "op_3_join_aligner_0_records_in_total": int64(4),
  157. "op_3_join_aligner_0_records_out_total": int64(3),
  158. "op_4_join_0_exceptions_total": int64(0),
  159. "op_4_join_0_records_in_total": int64(3),
  160. "op_4_join_0_records_out_total": int64(2),
  161. "op_5_project_0_exceptions_total": int64(0),
  162. "op_5_project_0_records_in_total": int64(2),
  163. "op_5_project_0_records_out_total": int64(2),
  164. "sink_mockSink_0_exceptions_total": int64(0),
  165. "sink_mockSink_0_records_in_total": int64(2),
  166. "sink_mockSink_0_records_out_total": int64(2),
  167. "source_demo_0_exceptions_total": int64(0),
  168. "source_demo_0_records_in_total": int64(3),
  169. "source_demo_0_records_out_total": int64(3),
  170. "source_table1_0_exceptions_total": int64(0),
  171. "source_table1_0_records_in_total": int64(4),
  172. "source_table1_0_records_out_total": int64(1),
  173. },
  174. },
  175. PauseSize: 3,
  176. Cc: 2,
  177. PauseMetric: map[string]interface{}{
  178. "sink_mockSink_0_exceptions_total": int64(0),
  179. "sink_mockSink_0_records_in_total": int64(2),
  180. "sink_mockSink_0_records_out_total": int64(2),
  181. "source_demo_0_exceptions_total": int64(0),
  182. "source_demo_0_records_in_total": int64(3),
  183. "source_demo_0_records_out_total": int64(3),
  184. "source_table1_0_exceptions_total": int64(0),
  185. "source_table1_0_records_in_total": int64(4),
  186. "source_table1_0_records_out_total": int64(1),
  187. },
  188. },
  189. }
  190. HandleStream(true, streamList, t)
  191. options := []*api.RuleOption{
  192. {
  193. BufferLength: 100,
  194. Qos: api.AtLeastOnce,
  195. CheckpointInterval: 600,
  196. SendError: true,
  197. },
  198. }
  199. for j, opt := range options {
  200. DoCheckpointRuleTest(t, tests, j, opt)
  201. }
  202. }