kafka-upsert-debezium-sources.td 9.4 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248
  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 single-replica-cluster=quickstart
  10. # must be a subset of the keys in the rows
  11. $ set keyschema={
  12. "type": "record",
  13. "name": "Key",
  14. "fields": [
  15. {"name": "id", "type": "long"}
  16. ]
  17. }
  18. $ set schema={
  19. "type" : "record",
  20. "name" : "envelope",
  21. "fields" : [
  22. {
  23. "name": "before",
  24. "type": [
  25. {
  26. "name": "row",
  27. "type": "record",
  28. "fields": [
  29. {
  30. "name": "id",
  31. "type": "long"
  32. },
  33. {
  34. "name": "creature",
  35. "type": "string"
  36. }]
  37. },
  38. "null"
  39. ]
  40. },
  41. { "name": "op", "type": "string" },
  42. {
  43. "name": "after",
  44. "type": ["row", "null"]
  45. },
  46. {
  47. "name": "source",
  48. "type": {
  49. "type": "record",
  50. "name": "Source",
  51. "namespace": "io.debezium.connector.mysql",
  52. "fields": [
  53. {
  54. "name": "file",
  55. "type": "string"
  56. },
  57. {
  58. "name": "pos",
  59. "type": "long"
  60. },
  61. {
  62. "name": "row",
  63. "type": "int"
  64. },
  65. {
  66. "name": "snapshot",
  67. "type": [
  68. {
  69. "type": "boolean",
  70. "connect.default": false
  71. },
  72. "null"
  73. ],
  74. "default": false
  75. }
  76. ],
  77. "connect.name": "io.debezium.connector.mysql.Source"
  78. }
  79. }
  80. ]
  81. }
  82. > CREATE CONNECTION kafka_conn
  83. TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT);
  84. > CREATE CONNECTION IF NOT EXISTS csr_conn TO CONFLUENT SCHEMA REGISTRY (
  85. URL '${testdrive.schema-registry-url}'
  86. );
  87. $ kafka-create-topic topic=dbzupsert partitions=1
  88. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=1
  89. {"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}}}
  90. {"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}}}
  91. {"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}}}
  92. > CREATE SOURCE doin_upsert
  93. IN CLUSTER ${arg.single-replica-cluster}
  94. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-dbzupsert-${testdrive.seed}')
  95. ! CREATE TABLE doin_upsert_tbl FROM SOURCE doin_upsert (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  96. FORMAT AVRO USING SCHEMA '${schema}'
  97. ENVELOPE DEBEZIUM
  98. contains:ENVELOPE [DEBEZIUM] UPSERT requires that KEY FORMAT be specified
  99. ! CREATE TABLE doin_upsert_tbl FROM SOURCE doin_upsert (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  100. KEY FORMAT JSON VALUE FORMAT JSON
  101. ENVELOPE DEBEZIUM
  102. contains:ENVELOPE DEBEZIUM requires that VALUE FORMAT is set to AVRO
  103. > CREATE TABLE doin_upsert_tbl FROM SOURCE doin_upsert (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  104. FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  105. ENVELOPE DEBEZIUM
  106. > SELECT * FROM doin_upsert_tbl
  107. id creature
  108. -----------
  109. 1 lizard
  110. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=2
  111. {"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}}}
  112. > SELECT * FROM doin_upsert_tbl
  113. id creature
  114. -----------
  115. 1 dino
  116. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=3
  117. {"id": 2} {"before": null, "after": {"row": {"id": 2, "creature": "archeopteryx"}}, "op": "c", "source": {"file": "binlog5", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  118. {"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}}}
  119. > SELECT * FROM doin_upsert_tbl ORDER BY creature
  120. id creature
  121. ------------
  122. 1 dino
  123. 2 velociraptor
  124. # test duplicates
  125. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=4
  126. {"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}}}
  127. {"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}}}
  128. > SELECT * FROM doin_upsert_tbl WHERE id = 3
  129. id creature
  130. -----------
  131. 3 triceratops
  132. # test removal and reinsertion
  133. $ kafka-ingest format=avro topic=dbzupsert key-format=avro key-schema=${keyschema} schema=${schema} timestamp=5
  134. {"id": 4} {"before": null, "after": {"row": {"id": 4, "creature": "moros"}}, "op": "c", "source": {"file": "binlog9", "pos": 1, "row": 1, "snapshot": {"boolean": false}}}
  135. > SELECT creature FROM doin_upsert_tbl WHERE id = 4
  136. creature
  137. --------
  138. moros
  139. > CREATE SOURCE doin_upsert_metadata
  140. IN CLUSTER ${arg.single-replica-cluster}
  141. FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-dbzupsert-${testdrive.seed}')
  142. # [btv] uncomment if we bring back classic debezium mode
  143. # ! CREATE TABLE doin_upsert_metadata_tbl FROM SOURCE doin_upsert_metadata (REFERENCE "testdrive-dbzupsert-${testdrive.seed}")
  144. # FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn
  145. # INCLUDE OFFSET
  146. # ENVELOPE DEBEZIUM
  147. # contains:INCLUDE OFFSET with Debezium requires UPSERT semantics
  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 ${arg.single-replica-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 ${arg.single-replica-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 ${arg.single-replica-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