kafka-exactly-once-sink.td 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387
  1. # Copyright Materialize, Inc. and contributors. All rights reserved.
  2. #
  3. # Use of this software is governed by the Business Source License
  4. # included in the LICENSE file at the root of this repository.
  5. #
  6. # As of the Change Date specified in that file, in accordance with
  7. # the Business Source License, use of this software will be governed
  8. # by the Apache License, Version 2.0.
  9. $ set-arg-default default-storage-size=1
  10. $ set-arg-default single-replica-cluster=quickstart
  11. $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr}
  12. ALTER SYSTEM SET enable_envelope_materialize = true
  13. ALTER SYSTEM SET max_clusters = 20
  14. $ set cdcv2-schema=[
  15. {
  16. "type": "array",
  17. "items": {
  18. "type": "record",
  19. "name": "update",
  20. "namespace": "com.materialize.cdc",
  21. "fields": [
  22. {
  23. "name": "data",
  24. "type": {
  25. "type": "record",
  26. "name": "data",
  27. "fields": [
  28. {"name": "a", "type": "long"},
  29. {"name": "b", "type": "long"}
  30. ]
  31. }
  32. },
  33. {
  34. "name": "time",
  35. "type": "long"
  36. },
  37. {
  38. "name": "diff",
  39. "type": "long"
  40. }
  41. ]
  42. }
  43. },
  44. {
  45. "type": "record",
  46. "name": "progress",
  47. "namespace": "com.materialize.cdc",
  48. "fields": [
  49. {
  50. "name": "lower",
  51. "type": {
  52. "type": "array",
  53. "items": "long"
  54. }
  55. },
  56. {
  57. "name": "upper",
  58. "type": {
  59. "type": "array",
  60. "items": "long"
  61. }
  62. },
  63. {
  64. "name": "counts",
  65. "type": {
  66. "type": "array",
  67. "items": {
  68. "type": "record",
  69. "name": "counts",
  70. "fields": [
  71. {
  72. "name": "time",
  73. "type": "long"
  74. },
  75. {
  76. "name": "count",
  77. "type": "long"
  78. }
  79. ]
  80. }
  81. }
  82. }
  83. ]
  84. }
  85. ]
  86. $ set dbz-key-schema={
  87. "type": "record",
  88. "name": "Key",
  89. "fields": [
  90. {
  91. "name": "a", "type": "long"
  92. }
  93. ]
  94. }
  95. $ set dbz-schema={
  96. "type": "record",
  97. "name": "envelope",
  98. "fields": [
  99. {
  100. "name": "before",
  101. "type": [
  102. {
  103. "name": "row",
  104. "type": "record",
  105. "fields": [
  106. {"name": "a", "type": "long"},
  107. {"name": "b", "type": "long"}
  108. ]
  109. },
  110. "null"
  111. ]
  112. },
  113. { "name": "op", "type": "string" },
  114. { "name": "after", "type": ["row", "null"] },
  115. {
  116. "name": "source",
  117. "type": {
  118. "type": "record",
  119. "name": "Source",
  120. "namespace": "whatever",
  121. "fields": [
  122. {
  123. "name": "snapshot",
  124. "type": [
  125. {
  126. "type": "string",
  127. "connect.version": 1,
  128. "connect.parameters": {
  129. "allowed": "true,last,false"
  130. },
  131. "connect.default": "false",
  132. "connect.name": "io.debezium.data.Enum"
  133. },
  134. "null"
  135. ],
  136. "default": "false"
  137. },
  138. {
  139. "name": "lsn",
  140. "type": ["long", "null"]
  141. },
  142. {
  143. "name": "sequence",
  144. "type": ["string", "null"]
  145. }
  146. ]
  147. }
  148. }
  149. ]
  150. }
  151. $ kafka-create-topic topic=input_dbz
  152. $ kafka-create-topic topic=input_cdcv2
  153. $ kafka-ingest format=avro topic=input_dbz key-format=avro key-schema=${dbz-key-schema} schema=${dbz-schema} timestamp=1
  154. {"a": 1} {"before": null, "after": {"row": {"a": 1, "b": 1}}, "source": {"lsn": {"long": 3}, "sequence": {"string": "[\"1\", \"3\"]"}, "snapshot": {"string": "false"}}, "op": "c"}
  155. > CREATE CONNECTION kafka_conn
  156. TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT);
  157. > CREATE CONNECTION IF NOT EXISTS csr_conn TO CONFLUENT SCHEMA REGISTRY (
  158. URL '${testdrive.schema-registry-url}'
  159. );
  160. > CREATE CLUSTER input_kafka_cdcv2_cluster SIZE '${arg.default-storage-size}';
  161. > CREATE SOURCE input_kafka_cdcv2
  162. IN CLUSTER input_kafka_cdcv2_cluster
  163. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-input_cdcv2-${testdrive.seed}')
  164. FORMAT AVRO USING SCHEMA '${cdcv2-schema}' ENVELOPE MATERIALIZE
  165. > CREATE CLUSTER input_kafka_dbz_cluster SIZE '${arg.default-storage-size}';
  166. > CREATE SOURCE input_kafka_dbz
  167. IN CLUSTER input_kafka_dbz_cluster
  168. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-input_dbz-${testdrive.seed}')
  169. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM
  170. > CREATE TABLE input_table (a bigint, b bigint)
  171. > CREATE MATERIALIZED VIEW input_kafka_cdcv2_mview AS SELECT a + 2 AS a , b + 10 AS b from input_kafka_cdcv2;
  172. > CREATE MATERIALIZED VIEW input_kafka_cdcv2_mview_view AS SELECT * FROM input_kafka_cdcv2_mview;
  173. > CREATE VIEW input_kafka_dbz_view AS SELECT a + 2 AS a , b + 10 AS b from input_kafka_dbz;
  174. > CREATE MATERIALIZED VIEW input_kafka_dbz_view_mview AS SELECT * FROM input_kafka_dbz_view;
  175. > CREATE MATERIALIZED VIEW input_table_mview AS SELECT a + 2 AS a , b + 10 AS b from input_table;
  176. > CREATE VIEW input_values_view AS VALUES (1), (2), (3);
  177. > CREATE MATERIALIZED VIEW input_values_mview AS VALUES (1), (2), (3);
  178. > CREATE MATERIALIZED VIEW input_kafka_dbz_derived_table AS SELECT * FROM ( SELECT * FROM input_kafka_dbz ) AS a1;
  179. $ kafka-create-topic topic=static
  180. $ kafka-ingest topic=static format=bytes
  181. city,state,zip
  182. Rochester,NY,14618
  183. New York,NY,10004
  184. "bad,place""",CA,92679
  185. > CREATE CLUSTER input_csv_cluster SIZE '${arg.default-storage-size}';
  186. > CREATE SOURCE input_csv
  187. IN CLUSTER input_csv_cluster
  188. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-static-${testdrive.seed}')
  189. FORMAT CSV WITH 3 COLUMNS
  190. > CREATE CLUSTER output1_cluster SIZE '${arg.default-storage-size}';
  191. > CREATE SINK output1
  192. IN CLUSTER output1_cluster
  193. FROM input_kafka_cdcv2
  194. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output1-view-${testdrive.seed}')
  195. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  196. ENVELOPE DEBEZIUM
  197. > CREATE CLUSTER output2_cluster SIZE '${arg.default-storage-size}';
  198. > CREATE SINK output2
  199. IN CLUSTER output2_cluster
  200. FROM input_kafka_dbz
  201. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output2-view-${testdrive.seed}')
  202. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  203. ENVELOPE DEBEZIUM
  204. > CREATE CLUSTER output3_cluster SIZE '${arg.default-storage-size}';
  205. > CREATE SINK output3
  206. IN CLUSTER output3_cluster
  207. FROM input_table
  208. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output3-view-${testdrive.seed}')
  209. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  210. ENVELOPE DEBEZIUM
  211. > CREATE CLUSTER output4_cluster SIZE '${arg.default-storage-size}';
  212. > CREATE SINK output4
  213. IN CLUSTER output4_cluster
  214. FROM input_kafka_cdcv2_mview
  215. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output4-view-${testdrive.seed}')
  216. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  217. ENVELOPE DEBEZIUM
  218. > CREATE CLUSTER output4_view_cluster SIZE '${arg.default-storage-size}';
  219. > CREATE SINK output4_view
  220. IN CLUSTER output4_view_cluster
  221. FROM input_kafka_cdcv2_mview_view
  222. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output4b-view-${testdrive.seed}')
  223. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  224. ENVELOPE DEBEZIUM
  225. ! CREATE SINK output5
  226. IN CLUSTER ${arg.single-replica-cluster}
  227. FROM input_kafka_dbz_view
  228. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output5-view-${testdrive.seed}')
  229. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  230. ENVELOPE DEBEZIUM
  231. contains:input_kafka_dbz_view is a view, which cannot be exported as a sink
  232. > CREATE CLUSTER output5_view_cluster SIZE '${arg.default-storage-size}';
  233. > CREATE SINK output5_view
  234. IN CLUSTER output5_view_cluster
  235. FROM input_kafka_dbz_view_mview
  236. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output5b-view-${testdrive.seed}')
  237. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  238. ENVELOPE DEBEZIUM
  239. > CREATE CLUSTER output6_cluster SIZE '${arg.default-storage-size}';
  240. > CREATE SINK output6
  241. IN CLUSTER output6_cluster
  242. FROM input_table_mview
  243. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output6-view-${testdrive.seed}')
  244. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  245. ENVELOPE DEBEZIUM
  246. ! CREATE SINK output7
  247. IN CLUSTER ${arg.single-replica-cluster}
  248. FROM input_values_view
  249. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output7-view-${testdrive.seed}')
  250. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  251. ENVELOPE DEBEZIUM
  252. contains:input_values_view is a view, which cannot be exported as a sink
  253. > CREATE CLUSTER output8_cluster SIZE '${arg.default-storage-size}';
  254. > CREATE SINK output8
  255. IN CLUSTER output8_cluster
  256. FROM input_values_mview
  257. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output8-view-${testdrive.seed}')
  258. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  259. ENVELOPE DEBEZIUM
  260. > CREATE CLUSTER output12_cluster SIZE '${arg.default-storage-size}';
  261. > CREATE SINK output12
  262. IN CLUSTER output12_cluster
  263. FROM input_kafka_dbz_derived_table
  264. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output12-view-${testdrive.seed}')
  265. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  266. ENVELOPE DEBEZIUM
  267. > CREATE CLUSTER output13_cluster SIZE '${arg.default-storage-size}';
  268. > CREATE SINK output13
  269. IN CLUSTER output13_cluster
  270. FROM input_csv
  271. INTO KAFKA CONNECTION kafka_conn (TOPIC 'output13-view-${testdrive.seed}')
  272. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  273. ENVELOPE DEBEZIUM
  274. # We need some data -- any data -- to start creating timestamp bindings
  275. $ kafka-ingest format=avro topic=input_cdcv2 schema=${cdcv2-schema}
  276. {"array":[{"data":{"a":1,"b":1},"time":1,"diff":1}]}
  277. {"com.materialize.cdc.progress":{"lower":[0],"upper":[1],"counts":[{"time":1,"count":1}]}}
  278. # verify output of progress topic with fixed timestamps
  279. $ kafka-create-topic topic=progress-test-input
  280. > CREATE CONNECTION kafka_fixed TO KAFKA (
  281. BROKER '${testdrive.kafka-addr}',
  282. PROGRESS TOPIC 'testdrive-progress-fixed-${testdrive.seed}',
  283. SECURITY PROTOCOL PLAINTEXT
  284. );
  285. > CREATE CLUSTER compaction_test_input_cluster SIZE '${arg.default-storage-size}';
  286. > CREATE SOURCE compaction_test_input
  287. IN CLUSTER compaction_test_input_cluster
  288. FROM KAFKA CONNECTION kafka_fixed (TOPIC 'testdrive-progress-test-input-${testdrive.seed}')
  289. FORMAT AVRO USING SCHEMA '${cdcv2-schema}' ENVELOPE MATERIALIZE
  290. > CREATE CLUSTER compaction_test_sink_cluster SIZE '${arg.default-storage-size}';
  291. > CREATE SINK compaction_test_sink
  292. IN CLUSTER compaction_test_sink_cluster
  293. FROM compaction_test_input
  294. INTO KAFKA CONNECTION kafka_fixed (TOPIC 'compaction-test-output-${testdrive.seed}')
  295. FORMAT AVRO
  296. USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM
  297. # NB: the final timestamp is incomplete and should be present in neither output nor progress topic
  298. $ kafka-ingest format=avro topic=progress-test-input schema=${cdcv2-schema}
  299. {"array":[{"data":{"a":1,"b":1},"time":1,"diff":1}]}
  300. {"array":[{"data":{"a":2,"b":2},"time":1,"diff":1}]}
  301. {"com.materialize.cdc.progress":{"lower":[0],"upper":[2],"counts":[{"time":1,"count":2}]}}
  302. {"array":[{"data":{"a":2,"b":2},"time":3,"diff":1}]}
  303. $ kafka-verify-topic sink=materialize.public.compaction_test_sink await-value-schema=true
  304. $ kafka-verify-data headers=materialize-timestamp format=avro sink=materialize.public.compaction_test_sink sort-messages=true
  305. 1 {"before": null, "after": {"row": {"a": 1, "b": 1}}}
  306. 1 {"before": null, "after": {"row": {"a": 2, "b": 2}}}
  307. > CREATE CLUSTER compaction_test_sink_check_cluster SIZE '${arg.default-storage-size}';
  308. > CREATE SOURCE compaction_test_sink_check
  309. IN CLUSTER compaction_test_sink_check_cluster
  310. FROM KAFKA CONNECTION kafka_fixed (TOPIC 'testdrive-progress-fixed-${testdrive.seed}')
  311. FORMAT JSON ENVELOPE NONE
  312. # Retrieve all the progress messages that are beyond [2]. There should be
  313. # exactly one of them because the upper of the CDCv2 stream stops at [2].
  314. > SELECT data->'frontier'
  315. FROM compaction_test_sink_check
  316. WHERE data->'frontier'->0 IS NULL OR (data->'frontier'->0)::int >= 2
  317. [2]
  318. # verify output with real-time timestamps
  319. $ kafka-create-topic topic=rt-binding-progress-test-input
  320. $ kafka-ingest format=avro topic=rt-binding-progress-test-input key-format=avro key-schema=${dbz-key-schema} schema=${dbz-schema} timestamp=1
  321. {"a": 1} {"before": null, "after": {"row": {"a": 1, "b": 1}}, "source": {"lsn": {"long": 3}, "sequence": {"string": "[\"1\", \"3\"]"}, "snapshot": {"string": "false"}}, "op": "c"}
  322. > CREATE CLUSTER rt_binding_progress_test_source_cluster SIZE '${arg.default-storage-size}';
  323. > CREATE SOURCE rt_binding_progress_test_source
  324. IN CLUSTER rt_binding_progress_test_source_cluster
  325. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-rt-binding-progress-test-input-${testdrive.seed}')
  326. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM
  327. > CREATE CLUSTER rt_binding_progress_test_sink_cluster SIZE '${arg.default-storage-size}';
  328. > CREATE SINK rt_binding_progress_test_sink
  329. IN CLUSTER rt_binding_progress_test_sink_cluster
  330. FROM rt_binding_progress_test_source
  331. INTO KAFKA CONNECTION kafka_conn (TOPIC 'rt-binding-progress-test-output-${testdrive.seed}')
  332. FORMAT AVRO
  333. USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM
  334. $ kafka-verify-topic sink=materialize.public.rt_binding_progress_test_sink await-value-schema=true
  335. $ kafka-verify-data format=avro sink=materialize.public.rt_binding_progress_test_sink
  336. {"before": null, "after": {"row": {"a": 1, "b": 1}}}