kafka-upsert-debezium-sources.td 10.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265
  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. # must be a subset of the keys in the rows
  10. $ set keyschema={
  11. "type": "record",
  12. "name": "Key",
  13. "fields": [
  14. {"name": "id", "type": "long"}
  15. ]
  16. }
  17. $ set schema={
  18. "type" : "record",
  19. "name" : "envelope",
  20. "fields" : [
  21. {
  22. "name": "before",
  23. "type": [
  24. {
  25. "name": "row",
  26. "type": "record",
  27. "fields": [
  28. {
  29. "name": "id",
  30. "type": "long"
  31. },
  32. {
  33. "name": "creature",
  34. "type": "string"
  35. }]
  36. },
  37. "null"
  38. ]
  39. },
  40. { "name": "op", "type": "string" },
  41. {
  42. "name": "after",
  43. "type": ["row", "null"]
  44. },
  45. {
  46. "name": "source",
  47. "type": {
  48. "type": "record",
  49. "name": "Source",
  50. "namespace": "io.debezium.connector.mysql",
  51. "fields": [
  52. {
  53. "name": "file",
  54. "type": "string"
  55. },
  56. {
  57. "name": "pos",
  58. "type": "long"
  59. },
  60. {
  61. "name": "row",
  62. "type": "int"
  63. },
  64. {
  65. "name": "snapshot",
  66. "type": [
  67. {
  68. "type": "boolean",
  69. "connect.default": false
  70. },
  71. "null"
  72. ],
  73. "default": false
  74. }
  75. ],
  76. "connect.name": "io.debezium.connector.mysql.Source"
  77. }
  78. }
  79. ]
  80. }
  81. # The quickstart cluster doesn't seem to inherit from `disk_cluster_replicas_default`.
  82. > CREATE CLUSTER test_cluster SIZE '4'
  83. > CREATE CONNECTION kafka_conn
  84. TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT);
  85. > CREATE CONNECTION IF NOT EXISTS csr_conn TO CONFLUENT SCHEMA REGISTRY (
  86. URL '${testdrive.schema-registry-url}'
  87. );
  88. $ kafka-create-topic topic=dbzupsert partitions=1
  89. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=1
  90. {"id": 1} {"before": {"row": {"id": 1, "creature": "fish"}}, "after": {"row": {"id": 1, "creature": "mudskipper"}}, "op": "u", "source": {"file": "binlog1", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  91. {"id": 1} {"before": {"row": {"id": 1, "creature": "mudskipper"}}, "after": {"row": {"id": 1, "creature": "salamander"}}, "op": "u", "source": {"file": "binlog2", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  92. {"id": 1} {"before": {"row": {"id": 1, "creature": "salamander"}}, "after": {"row": {"id": 1, "creature": "lizard"}}, "op": "u", "source": {"file": "binlog3", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  93. > CREATE SOURCE doin_upsert
  94. IN CLUSTER test_cluster
  95. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-dbzupsert-${testdrive.seed}')
  96. ! CREATE TABLE doin_upsert_tbl FROM SOURCE doin_upsert (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  97. FORMAT AVRO USING SCHEMA '${schema}'
  98. ENVELOPE DEBEZIUM
  99. contains:ENVELOPE [DEBEZIUM] UPSERT requires that KEY FORMAT be specified
  100. > CREATE TABLE doin_upsert_tbl FROM SOURCE doin_upsert (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  101. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  102. ENVELOPE DEBEZIUM
  103. > SELECT * FROM doin_upsert_tbl
  104. id creature
  105. -----------
  106. 1 lizard
  107. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=2
  108. {"id": 1} {"before": {"row": {"id": 1, "creature": "lizard"}}, "after": {"row": {"id": 1, "creature": "dino"}}, "op": "u", "source": {"file": "binlog4", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  109. > SELECT * FROM doin_upsert_tbl
  110. id creature
  111. -----------
  112. 1 dino
  113. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=3
  114. {"id": 2} {"before": null, "after": {"row": {"id": 2, "creature": "archeopteryx"}}, "op": "c", "source": {"file": "binlog5", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  115. {"id": 2} {"before": {"row": {"id": 2, "creature": "archeopteryx"}}, "after": {"row": {"id": 2, "creature": "velociraptor"}}, "op": "u", "source": {"file": "binlog6", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  116. > SELECT * FROM doin_upsert_tbl ORDER BY creature
  117. id creature
  118. ------------
  119. 1 dino
  120. 2 velociraptor
  121. # test duplicates
  122. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=4
  123. {"id": 3} {"before": {"row": {"id": 3, "creature": "protoceratops"}}, "after": {"row": {"id": 3, "creature": "triceratops"}}, "op": "u", "source": {"file": "binlog7", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  124. {"id": 3} {"before": {"row": {"id": 3, "creature": "protoceratops"}}, "after": {"row": {"id": 3, "creature": "triceratops"}}, "op": "u", "source": {"file": "binlog8", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  125. > SELECT * FROM doin_upsert_tbl WHERE id = 3
  126. id creature
  127. -----------
  128. 3 triceratops
  129. # test removal and reinsertion
  130. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=5
  131. {"id": 4} {"before": null, "after": {"row": {"id": 4, "creature": "moros"}}, "op": "c", "source": {"file": "binlog9", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  132. > SELECT creature FROM doin_upsert_tbl WHERE id = 4
  133. creature
  134. --------
  135. moros
  136. # [btv] uncomment if we bring back classic debezium mode
  137. # > CREATE SOURCE doin_upsert_metadata
  138. # FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-dbzupsert-${testdrive.seed}')
  139. #
  140. # ! CREATE TABLE doin_upsert_metadata_tbl FROM SOURCE doin_upsert_metadata (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  141. # FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  142. # INCLUDE OFFSET
  143. # ENVELOPE DEBEZIUM
  144. # contains:INCLUDE OFFSET with Debezium requires UPSERT semantics
  145. > CREATE SOURCE doin_upsert_metadata
  146. IN CLUSTER test_cluster
  147. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-dbzupsert-${testdrive.seed}')
  148. > CREATE TABLE doin_upsert_metadata_tbl FROM SOURCE doin_upsert_metadata (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  149. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  150. INCLUDE PARTITION, OFFSET AS test_kafka_offset
  151. ENVELOPE DEBEZIUM
  152. > SELECT * FROM doin_upsert_metadata_tbl WHERE id = 4
  153. id creature partition test_kafka_offset
  154. ---------------------------------------
  155. 4 moros 0 8
  156. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=6
  157. {"id": 4} {"before": {"row": {"id": 4, "creature": "trex"}}, "after": null, "op": "d", "source": {"file": "binlog10", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  158. > SELECT creature FROM doin_upsert_tbl WHERE id = 4
  159. creature
  160. --------
  161. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=7
  162. {"id": 4} {"before": {"row": {"id": 4, "creature": "trex"}}, "after": {"row": {"id": 4, "creature": "chicken"}}, "op": "u", "source": {"file": "binlog11", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  163. > SELECT creature FROM doin_upsert_tbl WHERE id = 4
  164. creature
  165. --------
  166. chicken
  167. > SELECT * FROM doin_upsert_tbl WHERE id = 3
  168. id creature
  169. -----------
  170. 3 triceratops
  171. # Test that `WITH (START OFFSET=<whatever>)` works
  172. > CREATE SOURCE upsert_fast_forward
  173. IN CLUSTER test_cluster
  174. FROM KAFKA CONNECTION kafka_conn (START OFFSET = [6], TOPIC 'testdrive-dbzupsert-${testdrive.seed}')
  175. > CREATE TABLE upsert_fast_forward_tbl FROM SOURCE upsert_fast_forward (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  176. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  177. ENVELOPE DEBEZIUM
  178. > SELECT * FROM upsert_fast_forward_tbl WHERE id = 3
  179. id creature
  180. -----------
  181. 3 triceratops
  182. # test include metadata
  183. > CREATE SOURCE upsert_metadata
  184. IN CLUSTER test_cluster
  185. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-dbzupsert-${testdrive.seed}')
  186. > CREATE TABLE upsert_metadata_tbl FROM SOURCE upsert_metadata (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  187. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  188. INCLUDE OFFSET, PARTITION
  189. ENVELOPE DEBEZIUM
  190. > SELECT * FROM upsert_metadata_tbl
  191. id creature offset partition
  192. ------------------------------------
  193. 1 dino 3 0
  194. 2 velociraptor 5 0
  195. 3 triceratops 7 0
  196. 4 chicken 10 0
  197. # test include metadata respects metadata order
  198. > CREATE SOURCE upsert_metadata_reordered
  199. IN CLUSTER test_cluster
  200. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-dbzupsert-${testdrive.seed}')
  201. > CREATE TABLE upsert_metadata_reordered_tbl FROM SOURCE upsert_metadata_reordered (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  202. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  203. INCLUDE PARTITION, OFFSET
  204. ENVELOPE DEBEZIUM
  205. > SELECT * FROM upsert_metadata_reordered_tbl
  206. id creature partition offset
  207. ------------------------------------
  208. 1 dino 0 3
  209. 2 velociraptor 0 5
  210. 3 triceratops 0 7
  211. 4 chicken 0 10
  212. > ALTER CONNECTION kafka_conn SET (broker = 'abcd') WITH (validate = false);
  213. > ALTER CONNECTION kafka_conn SET (broker = '${testdrive.kafka-addr}') WITH (validate = true);
  214. > SELECT * FROM doin_upsert_tbl WHERE id = 3
  215. id creature
  216. -----------
  217. 3 triceratops
  218. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=7
  219. {"id": 3} {"before": {"row": {"id": 3, "creature": "triceratops"}}, "after": {"row": {"id": 3, "creature": "altered"}}, "op": "u", "source": {"file": "binlog11", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  220. > SELECT * FROM doin_upsert_tbl WHERE id = 3
  221. id creature
  222. -----------
  223. 3 altered