-
Notifications
You must be signed in to change notification settings - Fork 43
/
JdbcChangeEventSink.java
368 lines (321 loc) · 14.6 KB
/
JdbcChangeEventSink.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
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
/*
* 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.jdbc;
import static io.debezium.connector.jdbc.JdbcSinkConnectorConfig.SchemaEvolutionMode.NONE;
import java.sql.SQLException;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.sink.SinkRecord;
import org.hibernate.SessionFactory;
import org.hibernate.StatelessSession;
import org.hibernate.Transaction;
import org.hibernate.dialect.DatabaseVersion;
import org.hibernate.query.NativeQuery;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.connector.jdbc.SinkRecordDescriptor.FieldDescriptor;
import io.debezium.connector.jdbc.dialect.DatabaseDialect;
import io.debezium.connector.jdbc.dialect.DatabaseDialectResolver;
import io.debezium.connector.jdbc.naming.TableNamingStrategy;
import io.debezium.connector.jdbc.relational.TableDescriptor;
import io.debezium.connector.jdbc.relational.TableId;
import io.debezium.pipeline.sink.spi.ChangeEventSink;
/**
* A {@link ChangeEventSink} for a JDBC relational database.
*
* @author Chris Cranford
*/
public class JdbcChangeEventSink implements ChangeEventSink {
private static final Logger LOGGER = LoggerFactory.getLogger(JdbcChangeEventSink.class);
private final JdbcSinkConnectorConfig config;
private final SessionFactory sessionFactory;
private final DatabaseDialect dialect;
private final StatelessSession session;
private final TableNamingStrategy tableNamingStrategy;
public JdbcChangeEventSink(JdbcSinkConnectorConfig config) {
this.config = config;
this.sessionFactory = config.getHibernateConfiguration().buildSessionFactory();
this.tableNamingStrategy = config.getTableNamingStrategy();
this.dialect = DatabaseDialectResolver.resolve(config, sessionFactory);
this.session = this.sessionFactory.openStatelessSession();
final DatabaseVersion version = this.dialect.getVersion();
LOGGER.info("Database version {}.{}.{}", version.getMajor(), version.getMinor(), version.getMicro());
}
@Override
public void execute(SinkRecord record) {
try {
// Examine the sink record and prepare a descriptor
final SinkRecordDescriptor descriptor = SinkRecordDescriptor.builder()
.withPrimaryKeyMode(config.getPrimaryKeyMode())
.withPrimaryKeyFields(config.getPrimaryKeyFields())
.withSinkRecord(record)
.withDialect(dialect)
.build();
if (descriptor.isTombstone()) {
LOGGER.debug("Skipping tombstone record {}", descriptor);
return;
}
String tableName = tableNamingStrategy.resolveTableName(config, record);
if (tableName == null) {
LOGGER.warn("Ignored to write record from topic '{}' partition '{}' offset '{}'", record.topic(), record.kafkaPartition(), record.kafkaOffset());
return;
}
final TableId tableId = dialect.getTableId(tableName);
final TableDescriptor table = checkAndApplyTableChangesIfNeeded(tableId, descriptor);
write(table, descriptor);
}
catch (Exception e) {
throw new ConnectException("Failed to process a sink record", e);
}
}
@Override
public void close() throws Exception {
if (session != null && session.isOpen()) {
LOGGER.info("Closing session.");
session.close();
}
else {
LOGGER.info("Session already closed.");
}
if (sessionFactory != null && sessionFactory.isOpen()) {
LOGGER.info("Closing the session factory");
sessionFactory.close();
}
else {
LOGGER.info("Session factory already closed");
}
}
private TableDescriptor checkAndApplyTableChangesIfNeeded(TableId tableId, SinkRecordDescriptor descriptor) throws SQLException {
if (!hasTable(tableId)) {
// Table does not exist, lets attempt to create it.
try {
return createTable(tableId, descriptor);
}
catch (SQLException ce) {
// It's possible the table may have been created in the interim, so try to alter.
LOGGER.warn("Table creation failed for '{}', attempting to alter the table", tableId.toFullIdentiferString(), ce);
try {
return alterTableIfNeeded(tableId, descriptor);
}
catch (SQLException ae) {
// The alter failed, hard stop.
LOGGER.error("Failed to alter the table '{}'.", tableId.toFullIdentiferString(), ae);
throw ae;
}
}
}
else {
// Table exists, lets attempt to alter it if necessary.
try {
return alterTableIfNeeded(tableId, descriptor);
}
catch (SQLException ae) {
LOGGER.error("Failed to alter the table '{}'.", tableId.toFullIdentiferString(), ae);
throw ae;
}
}
}
private boolean hasTable(TableId tableId) {
return session.doReturningWork((connection) -> dialect.tableExists(connection, tableId));
}
private TableDescriptor readTable(TableId tableId) {
return session.doReturningWork((connection) -> dialect.readTable(connection, tableId));
}
private TableDescriptor createTable(TableId tableId, SinkRecordDescriptor record) throws SQLException {
LOGGER.debug("Attempting to create table '{}'.", tableId.toFullIdentiferString());
if (NONE.equals(config.getSchemaEvolutionMode())) {
LOGGER.warn("Table '{}' cannot be created because schema evolution is disabled.", tableId.toFullIdentiferString());
throw new SQLException("Cannot create table " + tableId.toFullIdentiferString() + " because schema evolution is disabled");
}
Transaction transaction = session.beginTransaction();
try {
final String createSql = dialect.getCreateTableStatement(record, tableId);
LOGGER.trace("SQL: {}", createSql);
session.createNativeQuery(createSql, Object.class).executeUpdate();
transaction.commit();
}
catch (Exception e) {
transaction.rollback();
throw e;
}
return readTable(tableId);
}
private TableDescriptor alterTableIfNeeded(TableId tableId, SinkRecordDescriptor record) throws SQLException {
LOGGER.debug("Attempting to alter table '{}'.", tableId.toFullIdentiferString());
if (!hasTable(tableId)) {
LOGGER.error("Table '{}' does not exist and cannot be altered.", tableId.toFullIdentiferString());
throw new SQLException("Could not find table: " + tableId.toFullIdentiferString());
}
// Resolve table metadata from the database
final TableDescriptor table = readTable(tableId);
// Delegating to dialect to deal with database case sensitivity.
Set<String> missingFields = dialect.resolveMissingFields(record, table);
if (missingFields.isEmpty()) {
// There are no missing fields, simply return
// todo: should we check column type changes or default value changes?
return table;
}
LOGGER.debug("The follow fields are missing in the table: {}", missingFields);
for (String missingFieldName : missingFields) {
final FieldDescriptor fieldDescriptor = record.getFields().get(missingFieldName);
if (!fieldDescriptor.getSchema().isOptional() && fieldDescriptor.getSchema().defaultValue() == null) {
throw new SQLException(String.format(
"Cannot ALTER table '%s' because field '%s' is not optional but has no default value",
tableId.toFullIdentiferString(), fieldDescriptor.getName()));
}
}
if (NONE.equals(config.getSchemaEvolutionMode())) {
LOGGER.warn("Table '{}' cannot be altered because schema evolution is disabled.", tableId.toFullIdentiferString());
throw new SQLException("Cannot alter table " + tableId.toFullIdentiferString() + " because schema evolution is disabled");
}
Transaction transaction = session.beginTransaction();
try {
final String alterSql = dialect.getAlterTableStatement(table, record, missingFields);
LOGGER.trace("SQL: {}", alterSql);
session.createNativeQuery(alterSql, Object.class).executeUpdate();
transaction.commit();
}
catch (Exception e) {
transaction.rollback();
throw e;
}
return readTable(tableId);
}
private void write(TableDescriptor table, SinkRecordDescriptor record) throws SQLException {
if (record.isDelete()) {
writeDelete(dialect.getDeleteStatement(table, record), record);
}
else if (record.isTruncate()) {
writeTruncate(dialect.getTruncateStatement(table), record);
}
else {
switch (config.getInsertMode()) {
case INSERT:
writeInsert(dialect.getInsertStatement(table, record), record);
break;
case UPSERT:
if (record.getKeyFieldNames().isEmpty()) {
throw new ConnectException("Cannot write to table " + table.getId().getTableName() + " with no key fields defined.");
}
writeUpsert(dialect.getUpsertStatement(table, record), record);
break;
case UPDATE:
writeUpdate(dialect.getUpdateStatement(table, record), record);
break;
}
}
}
private void writeInsert(String sql, SinkRecordDescriptor record) throws SQLException {
final Transaction transaction = session.beginTransaction();
try {
LOGGER.trace("SQL: {}", sql);
final NativeQuery<?> query = session.createNativeQuery(sql, Object.class);
int index = bindKeyValuesToQuery(record, query, 1);
bindNonKeyValuesToQuery(record, query, index);
final int result = query.executeUpdate();
if (result != 1) {
throw new SQLException("Failed to insert row from table");
}
transaction.commit();
}
catch (SQLException e) {
transaction.rollback();
throw e;
}
}
private void writeUpsert(String sql, SinkRecordDescriptor record) throws SQLException {
final Transaction transaction = session.beginTransaction();
try {
LOGGER.trace("SQL: {}", sql);
final NativeQuery<?> query = session.createNativeQuery(sql, Object.class);
int index = bindKeyValuesToQuery(record, query, 1);
bindNonKeyValuesToQuery(record, query, index);
query.executeUpdate();
transaction.commit();
}
catch (Exception e) {
transaction.rollback();
throw e;
}
}
private void writeUpdate(String sql, SinkRecordDescriptor record) throws SQLException {
final Transaction transaction = session.beginTransaction();
try {
LOGGER.trace("SQL: {}", sql);
final NativeQuery<?> query = session.createNativeQuery(sql, Object.class);
int index = bindNonKeyValuesToQuery(record, query, 1);
bindKeyValuesToQuery(record, query, index);
query.executeUpdate();
transaction.commit();
}
catch (Exception e) {
transaction.rollback();
throw e;
}
}
private void writeDelete(String sql, SinkRecordDescriptor record) throws SQLException {
if (!config.isDeleteEnabled()) {
LOGGER.debug("Deletes are not enabled, skipping delete for topic '{}'", record.getTopicName());
return;
}
final Transaction transaction = session.beginTransaction();
try {
LOGGER.trace("SQL: {}", sql);
final NativeQuery<?> query = session.createNativeQuery(sql, Object.class);
bindKeyValuesToQuery(record, query, 1);
query.executeUpdate();
transaction.commit();
}
catch (Exception e) {
transaction.rollback();
throw e;
}
}
private void writeTruncate(String sql, SinkRecordDescriptor record) throws SQLException {
if (!config.isTruncateEnabled()) {
LOGGER.debug("Truncates are not enabled, skipping truncate for topic '{}'", record.getTopicName());
return;
}
final Transaction transaction = session.beginTransaction();
try {
LOGGER.trace("SQL: {}", sql);
final NativeQuery<?> query = session.createNativeQuery(sql, Object.class);
query.executeUpdate();
transaction.commit();
}
catch (Exception e) {
transaction.rollback();
throw e;
}
}
private int bindKeyValuesToQuery(SinkRecordDescriptor record, NativeQuery<?> query, int index) {
if (Objects.requireNonNull(config.getPrimaryKeyMode()) == JdbcSinkConnectorConfig.PrimaryKeyMode.KAFKA) {
query.setParameter(index++, record.getTopicName());
query.setParameter(index++, record.getPartition());
query.setParameter(index++, record.getOffset());
}
else {
final Struct keySource = record.getKeyStruct(config.getPrimaryKeyMode());
if (keySource != null) {
index = bindFieldValuesToQuery(record, query, index, keySource, record.getKeyFieldNames());
}
}
return index;
}
private int bindNonKeyValuesToQuery(SinkRecordDescriptor record, NativeQuery<?> query, int index) {
return bindFieldValuesToQuery(record, query, index, record.getAfterStruct(), record.getNonKeyFieldNames());
}
private int bindFieldValuesToQuery(SinkRecordDescriptor record, NativeQuery<?> query, int index, Struct source, List<String> fields) {
for (String fieldName : fields) {
final FieldDescriptor field = record.getFields().get(fieldName);
index += dialect.bindValue(field, query, index, source.get(fieldName));
}
return index;
}
}