forked from debezium/debezium-incubator
-
Notifications
You must be signed in to change notification settings - Fork 0
/
TransactionMetadataIT.java
222 lines (184 loc) · 9.78 KB
/
TransactionMetadataIT.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import static org.fest.assertions.Assertions.assertThat;
import java.math.BigDecimal;
import java.sql.SQLException;
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import io.debezium.config.Configuration;
import io.debezium.connector.oracle.OracleConnectorConfig.SnapshotMode;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.data.Envelope;
import io.debezium.data.VerifyRecord;
import io.debezium.doc.FixFor;
import io.debezium.embedded.AbstractConnectorTest;
import io.debezium.util.Collect;
import io.debezium.util.Testing;
/**
* Integration test to check transaction metadata.
*
* @author Jiri Pechanec
*/
public class TransactionMetadataIT extends AbstractConnectorTest {
private static OracleConnection connection;
@BeforeClass
public static void beforeClass() throws SQLException {
connection = TestHelper.testConnection();
TestHelper.dropTable(connection, "debezium.customer");
TestHelper.dropTable(connection, "debezium.orders");
String ddl = "create table debezium.customer (" +
" id numeric(9,0) not null, " +
" name varchar2(1000), " +
" score decimal(6, 2), " +
" registered timestamp, " +
" primary key (id)" +
")";
connection.execute(ddl);
connection.execute("GRANT SELECT ON debezium.customer to " + TestHelper.getConnectorUserName());
connection.execute("ALTER TABLE debezium.customer ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS");
ddl = "create table debezium.orders (" +
" id number(6) not null primary key, " +
" order_date date not null, " +
" purchaser number(4) not null, " +
" quantity number(4) not null, " +
" product_id number(4) not null" +
")";
connection.execute(ddl);
connection.execute("GRANT SELECT ON debezium.orders to " + TestHelper.getConnectorUserName());
connection.execute("ALTER TABLE debezium.orders ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS");
}
@AfterClass
public static void closeConnection() throws SQLException {
if (connection != null) {
connection.close();
}
}
@Before
public void before() throws SQLException {
connection.execute("delete from debezium.customer");
connection.execute("delete from debezium.orders");
setConsumeTimeout(TestHelper.defaultMessageConsumerPollTimeout(), TimeUnit.SECONDS);
initializeConnectorTestFramework();
Testing.Files.delete(TestHelper.DB_HISTORY_PATH);
}
@Test
public void transactionMetadata() throws Exception {
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CUSTOMER,DEBEZIUM\\.ORDERS")
.with(OracleConnectorConfig.SNAPSHOT_MODE, SnapshotMode.SCHEMA_ONLY)
.with(OracleConnectorConfig.PROVIDE_TRANSACTION_METADATA, true)
.with(OracleConnectorConfig.LOG_MINING_STRATEGY, OracleConnectorConfig.LogMiningStrategy.ONLINE_CATALOG)
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
connection.executeWithoutCommitting("INSERT INTO debezium.customer VALUES (1, 'Billie-Bob', 1234.56, TO_DATE('2018/02/22', 'yyyy-mm-dd'))");
connection.executeWithoutCommitting("INSERT INTO debezium.orders VALUES (1, '01-FEB-2021', 1001, 1, 102)");
connection.execute("COMMIT");
// TX BEGIN, insert x2, TX END
final int expectedRecordCount = 1 + 2 + 1;
List<SourceRecord> records = consumeRecordsByTopic(expectedRecordCount).allRecordsInOrder();
assertThat(records).hasSize(expectedRecordCount);
// TX Begin
SourceRecord record = records.get(0);
String expectedTxId = assertBeginTransaction(record);
// Insert customer
record = records.get(1);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = (Struct) ((Struct) record.value()).get(Envelope.FieldName.AFTER);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("NAME")).isEqualTo("Billie-Bob");
assertThat(after.get("SCORE")).isEqualTo(BigDecimal.valueOf(1234.56));
assertRecordTransactionMetadata(record, expectedTxId, 1, 1);
// Insert orders
record = records.get(2);
VerifyRecord.isValidInsert(record, "ID", 1);
after = (Struct) ((Struct) record.value()).get(Envelope.FieldName.AFTER);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("ORDER_DATE")).isEqualTo(1612137600000L);
assertThat(after.get("PURCHASER")).isEqualTo((short) 1001);
assertThat(after.get("QUANTITY")).isEqualTo((short) 1);
assertThat(after.get("PRODUCT_ID")).isEqualTo((short) 102);
assertRecordTransactionMetadata(record, expectedTxId, 2, 1);
// TX End
record = records.get(3);
final String dbName = TestHelper.getDatabaseName();
assertEndTransaction(record, expectedTxId, 2, Collect.hashMapOf(dbName + ".DEBEZIUM.CUSTOMER", 1, dbName + ".DEBEZIUM.ORDERS", 1));
}
@Test
@FixFor("DBZ-3090")
public void transactionMetadataMultipleTransactions() throws Exception {
try (OracleConnection secondaryConn = TestHelper.testConnection()) {
final String dbName = TestHelper.getDatabaseName();
Configuration config = TestHelper.defaultConfig()
.with(OracleConnectorConfig.TABLE_INCLUDE_LIST, "DEBEZIUM\\.CUSTOMER,DEBEZIUM\\.ORDERS")
.with(OracleConnectorConfig.SNAPSHOT_MODE, SnapshotMode.SCHEMA_ONLY)
.with(OracleConnectorConfig.PROVIDE_TRANSACTION_METADATA, true)
.with(OracleConnectorConfig.LOG_MINING_STRATEGY, OracleConnectorConfig.LogMiningStrategy.ONLINE_CATALOG)
.build();
start(OracleConnector.class, config);
assertConnectorIsRunning();
waitForSnapshotToBeCompleted(TestHelper.CONNECTOR_NAME, TestHelper.SERVER_NAME);
// Create multiple transaction commits, notice commit order
connection.executeWithoutCommitting("INSERT INTO debezium.customer VALUES (1, 'Billie-Bob', 1234.56, TO_DATE('2018/02/22', 'yyyy-mm-dd'))");
connection.executeWithoutCommitting("INSERT INTO debezium.orders VALUES (2, '01-FEB-2021', 1001, 2, 102)");
secondaryConn.executeWithoutCommitting("INSERT INTO debezium.orders VALUES (1, '01-FEB-2021', 1001, 1, 102)");
secondaryConn.execute("COMMIT");
connection.execute("COMMIT");
// 2 TX BEGIN, 3 insert, 2 TX END
final int expectedRecordCount = 2 + 3 + 2;
List<SourceRecord> records = consumeRecordsByTopic(expectedRecordCount).allRecordsInOrder();
assertThat(records).hasSize(expectedRecordCount);
// TX Begin
SourceRecord record = records.get(0);
String expectedTxId = assertBeginTransaction(record);
// Insert orders (secondaryConn commit)
record = records.get(1);
VerifyRecord.isValidInsert(record, "ID", 1);
Struct after = (Struct) ((Struct) record.value()).get(Envelope.FieldName.AFTER);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("ORDER_DATE")).isEqualTo(1612137600000L);
assertThat(after.get("PURCHASER")).isEqualTo((short) 1001);
assertThat(after.get("QUANTITY")).isEqualTo((short) 1);
assertThat(after.get("PRODUCT_ID")).isEqualTo((short) 102);
assertRecordTransactionMetadata(record, expectedTxId, 1, 1);
// TX End
record = records.get(2);
assertEndTransaction(record, expectedTxId, 1, Collect.hashMapOf(dbName + ".DEBEZIUM.ORDERS", 1));
// TX Begin
record = records.get(3);
expectedTxId = assertBeginTransaction(record);
// Insert customer (connection commit)
record = records.get(4);
VerifyRecord.isValidInsert(record, "ID", 1);
after = (Struct) ((Struct) record.value()).get(Envelope.FieldName.AFTER);
assertThat(after.get("ID")).isEqualTo(1);
assertThat(after.get("NAME")).isEqualTo("Billie-Bob");
assertThat(after.get("SCORE")).isEqualTo(BigDecimal.valueOf(1234.56));
assertRecordTransactionMetadata(record, expectedTxId, 1, 1);
// Insert orders (connection commit)
record = records.get(5);
VerifyRecord.isValidInsert(record, "ID", 2);
after = (Struct) ((Struct) record.value()).get(Envelope.FieldName.AFTER);
assertThat(after.get("ID")).isEqualTo(2);
assertThat(after.get("ORDER_DATE")).isEqualTo(1612137600000L);
assertThat(after.get("PURCHASER")).isEqualTo((short) 1001);
assertThat(after.get("QUANTITY")).isEqualTo((short) 2);
assertThat(after.get("PRODUCT_ID")).isEqualTo((short) 102);
assertRecordTransactionMetadata(record, expectedTxId, 2, 1);
// TX End
record = records.get(6);
assertEndTransaction(record, expectedTxId, 2, Collect.hashMapOf(dbName + ".DEBEZIUM.CUSTOMER", 1, dbName + ".DEBEZIUM.ORDERS", 1));
}
}
}