-
Notifications
You must be signed in to change notification settings - Fork 958
/
JdbcSourceTask.java
556 lines (504 loc) · 21.3 KB
/
JdbcSourceTask.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
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
/*
* Copyright 2018 Confluent Inc.
*
* Licensed under the Confluent Community License (the "License"); you may not use
* this file except in compliance with the License. You may obtain a copy of the
* License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/
package io.confluent.connect.jdbc.source;
import java.sql.SQLNonTransientException;
import java.util.TimeZone;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.utils.SystemTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.source.SourceTask;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.sql.Connection;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Calendar;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors;
import io.confluent.connect.jdbc.dialect.DatabaseDialect;
import io.confluent.connect.jdbc.dialect.DatabaseDialects;
import io.confluent.connect.jdbc.util.CachedConnectionProvider;
import io.confluent.connect.jdbc.util.ColumnDefinition;
import io.confluent.connect.jdbc.util.ColumnId;
import io.confluent.connect.jdbc.util.TableId;
import io.confluent.connect.jdbc.util.Version;
import io.confluent.connect.jdbc.source.JdbcSourceConnectorConfig.TransactionIsolationMode;
/**
* JdbcSourceTask is a Kafka Connect SourceTask implementation that reads from JDBC databases and
* generates Kafka Connect records.
*/
public class JdbcSourceTask extends SourceTask {
// When no results, periodically return control flow to caller to give it a chance to pause us.
private static final int CONSECUTIVE_EMPTY_RESULTS_BEFORE_RETURN = 3;
private static final Logger log = LoggerFactory.getLogger(JdbcSourceTask.class);
private Time time;
private JdbcSourceTaskConfig config;
private DatabaseDialect dialect;
//Visible for Testing
CachedConnectionProvider cachedConnectionProvider;
PriorityQueue<TableQuerier> tableQueue = new PriorityQueue<>();
private final AtomicBoolean running = new AtomicBoolean(false);
private final AtomicLong taskThreadId = new AtomicLong(0);
int maxRetriesPerQuerier;
public JdbcSourceTask() {
this.time = new SystemTime();
}
public JdbcSourceTask(Time time) {
this.time = time;
}
@Override
public String version() {
return Version.getVersion();
}
@Override
public void start(Map<String, String> properties) {
log.info("Starting JDBC source task");
try {
config = new JdbcSourceTaskConfig(properties);
} catch (ConfigException e) {
throw new ConfigException("Couldn't start JdbcSourceTask due to configuration error", e);
}
List<String> tables = config.getList(JdbcSourceTaskConfig.TABLES_CONFIG);
String query = config.getString(JdbcSourceTaskConfig.QUERY_CONFIG);
if ((tables.isEmpty() && query.isEmpty())) {
throw new ConfigException("Task is being killed because"
+ " it was not assigned a table nor a query to execute."
+ " If run in table mode please make sure that the tables"
+ " exist on the database. If the table does exist on"
+ " the database, we recommend using the fully qualified"
+ " table name.");
}
if ((!tables.isEmpty() && !query.isEmpty())) {
throw new ConfigException("Invalid configuration: a JdbcSourceTask"
+ " cannot have both a table and a query assigned to it");
}
final String url = config.getString(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG);
final int maxConnAttempts = config.getInt(JdbcSourceConnectorConfig.CONNECTION_ATTEMPTS_CONFIG);
final long retryBackoff = config.getLong(JdbcSourceConnectorConfig.CONNECTION_BACKOFF_CONFIG);
final String dialectName = config.getString(JdbcSourceConnectorConfig.DIALECT_NAME_CONFIG);
if (dialectName != null && !dialectName.trim().isEmpty()) {
dialect = DatabaseDialects.create(dialectName, config);
} else {
log.info("Finding the database dialect that is best fit for the provided JDBC URL.");
dialect = DatabaseDialects.findBestFor(url, config);
}
log.info("Using JDBC dialect {}", dialect.name());
cachedConnectionProvider = connectionProvider(maxConnAttempts, retryBackoff);
dialect.setConnectionIsolationMode(
cachedConnectionProvider.getConnection(),
TransactionIsolationMode
.valueOf(
config.getString(
JdbcSourceConnectorConfig
.TRANSACTION_ISOLATION_MODE_CONFIG
)
)
);
TableQuerier.QueryMode queryMode = !query.isEmpty() ? TableQuerier.QueryMode.QUERY :
TableQuerier.QueryMode.TABLE;
List<String> tablesOrQuery = queryMode == TableQuerier.QueryMode.QUERY
? Collections.singletonList(query) : tables;
String mode = config.getString(JdbcSourceTaskConfig.MODE_CONFIG);
//used only in table mode
Map<String, List<Map<String, String>>> partitionsByTableFqn = new HashMap<>();
Map<Map<String, String>, Map<String, Object>> offsets = null;
if (mode.equals(JdbcSourceTaskConfig.MODE_INCREMENTING)
|| mode.equals(JdbcSourceTaskConfig.MODE_TIMESTAMP)
|| mode.equals(JdbcSourceTaskConfig.MODE_TIMESTAMP_INCREMENTING)) {
List<Map<String, String>> partitions = new ArrayList<>(tables.size());
switch (queryMode) {
case TABLE:
log.trace("Starting in TABLE mode");
for (String table : tables) {
// Find possible partition maps for different offset protocols
// We need to search by all offset protocol partition keys to support compatibility
List<Map<String, String>> tablePartitions = possibleTablePartitions(table);
partitions.addAll(tablePartitions);
partitionsByTableFqn.put(table, tablePartitions);
}
break;
case QUERY:
log.trace("Starting in QUERY mode");
partitions.add(Collections.singletonMap(JdbcSourceConnectorConstants.QUERY_NAME_KEY,
JdbcSourceConnectorConstants.QUERY_NAME_VALUE));
break;
default:
throw new ConfigException("Unknown query mode: " + queryMode);
}
offsets = context.offsetStorageReader().offsets(partitions);
log.trace("The partition offsets are {}", offsets);
}
String incrementingColumn
= config.getString(JdbcSourceTaskConfig.INCREMENTING_COLUMN_NAME_CONFIG);
List<String> timestampColumns
= config.getList(JdbcSourceTaskConfig.TIMESTAMP_COLUMN_NAME_CONFIG);
Long timestampDelayInterval
= config.getLong(JdbcSourceTaskConfig.TIMESTAMP_DELAY_INTERVAL_MS_CONFIG);
boolean validateNonNulls
= config.getBoolean(JdbcSourceTaskConfig.VALIDATE_NON_NULL_CONFIG);
TimeZone timeZone = config.timeZone();
String suffix = config.getString(JdbcSourceTaskConfig.QUERY_SUFFIX_CONFIG).trim();
for (String tableOrQuery : tablesOrQuery) {
final List<Map<String, String>> tablePartitionsToCheck;
final Map<String, String> partition;
switch (queryMode) {
case TABLE:
if (validateNonNulls) {
validateNonNullable(
mode,
tableOrQuery,
incrementingColumn,
timestampColumns
);
}
tablePartitionsToCheck = partitionsByTableFqn.get(tableOrQuery);
break;
case QUERY:
partition = Collections.singletonMap(
JdbcSourceConnectorConstants.QUERY_NAME_KEY,
JdbcSourceConnectorConstants.QUERY_NAME_VALUE
);
tablePartitionsToCheck = Collections.singletonList(partition);
break;
default:
throw new ConfigException("Unexpected query mode: " + queryMode);
}
// The partition map varies by offset protocol. Since we don't know which protocol each
// table's offsets are keyed by, we need to use the different possible partitions
// (newest protocol version first) to find the actual offsets for each table.
Map<String, Object> offset = null;
if (offsets != null) {
for (Map<String, String> toCheckPartition : tablePartitionsToCheck) {
offset = offsets.get(toCheckPartition);
if (offset != null) {
log.info("Found offset {} for partition {}", offsets, toCheckPartition);
break;
}
}
}
offset = computeInitialOffset(tableOrQuery, offset, timeZone);
String topicPrefix = config.topicPrefix();
JdbcSourceConnectorConfig.TimestampGranularity timestampGranularity
= JdbcSourceConnectorConfig.TimestampGranularity.get(config);
if (mode.equals(JdbcSourceTaskConfig.MODE_BULK)) {
tableQueue.add(
new BulkTableQuerier(
dialect,
queryMode,
tableOrQuery,
topicPrefix,
suffix
)
);
} else if (mode.equals(JdbcSourceTaskConfig.MODE_INCREMENTING)) {
tableQueue.add(
new TimestampIncrementingTableQuerier(
dialect,
queryMode,
tableOrQuery,
topicPrefix,
null,
incrementingColumn,
offset,
timestampDelayInterval,
timeZone,
suffix,
timestampGranularity
)
);
} else if (mode.equals(JdbcSourceTaskConfig.MODE_TIMESTAMP)) {
tableQueue.add(
new TimestampTableQuerier(
dialect,
queryMode,
tableOrQuery,
topicPrefix,
timestampColumns,
offset,
timestampDelayInterval,
timeZone,
suffix,
timestampGranularity
)
);
} else if (mode.endsWith(JdbcSourceTaskConfig.MODE_TIMESTAMP_INCREMENTING)) {
tableQueue.add(
new TimestampIncrementingTableQuerier(
dialect,
queryMode,
tableOrQuery,
topicPrefix,
timestampColumns,
incrementingColumn,
offset,
timestampDelayInterval,
timeZone,
suffix,
timestampGranularity
)
);
}
}
running.set(true);
taskThreadId.set(Thread.currentThread().getId());
log.info("Started JDBC source task");
maxRetriesPerQuerier = config.getInt(JdbcSourceConnectorConfig.QUERY_RETRIES_CONFIG);
}
protected CachedConnectionProvider connectionProvider(int maxConnAttempts, long retryBackoff) {
return new CachedConnectionProvider(dialect, maxConnAttempts, retryBackoff) {
@Override
protected void onConnect(final Connection connection) throws SQLException {
super.onConnect(connection);
connection.setAutoCommit(false);
}
};
}
//This method returns a list of possible partition maps for different offset protocols
//This helps with the upgrades
private List<Map<String, String>> possibleTablePartitions(String table) {
TableId tableId = dialect.parseTableIdentifier(table);
return Arrays.asList(
OffsetProtocols.sourcePartitionForProtocolV1(tableId),
OffsetProtocols.sourcePartitionForProtocolV0(tableId)
);
}
protected Map<String, Object> computeInitialOffset(
String tableOrQuery,
Map<String, Object> partitionOffset,
TimeZone timezone) {
if (!(partitionOffset == null)) {
return partitionOffset;
} else {
Map<String, Object> initialPartitionOffset = null;
// no offsets found
Long timestampInitial = config.getLong(JdbcSourceConnectorConfig.TIMESTAMP_INITIAL_CONFIG);
if (timestampInitial != null) {
// start at the specified timestamp
if (timestampInitial == JdbcSourceConnectorConfig.TIMESTAMP_INITIAL_CURRENT) {
// use the current time
try {
final Connection con = cachedConnectionProvider.getConnection();
Calendar cal = Calendar.getInstance(timezone);
timestampInitial = dialect.currentTimeOnDB(con, cal).getTime();
} catch (SQLException e) {
throw new ConnectException("Error while getting initial timestamp from database", e);
}
}
initialPartitionOffset = new HashMap<String, Object>();
initialPartitionOffset.put(TimestampIncrementingOffset.TIMESTAMP_FIELD, timestampInitial);
log.info("No offsets found for '{}', so using configured timestamp {}", tableOrQuery,
timestampInitial);
}
return initialPartitionOffset;
}
}
@Override
public void stop() throws ConnectException {
log.info("Stopping JDBC source task");
// In earlier versions of Kafka, stop() was not called from the task thread. In this case, all
// resources are closed at the end of 'poll()' when no longer running or if there is an error.
running.set(false);
if (taskThreadId.longValue() == Thread.currentThread().getId()) {
shutdown();
}
}
protected void closeResources() {
log.info("Closing resources for JDBC source task");
try {
if (cachedConnectionProvider != null) {
cachedConnectionProvider.close(true);
}
} catch (Throwable t) {
log.warn("Error while closing the connections", t);
} finally {
cachedConnectionProvider = null;
try {
if (dialect != null) {
dialect.close();
}
} catch (Throwable t) {
log.warn("Error while closing the {} dialect: ", dialect.name(), t);
} finally {
dialect = null;
}
}
}
@Override
public List<SourceRecord> poll() throws InterruptedException {
log.trace("Polling for new data");
Map<TableQuerier, Integer> consecutiveEmptyResults = tableQueue.stream().collect(
Collectors.toMap(Function.identity(), (q) -> 0));
while (running.get()) {
final TableQuerier querier = tableQueue.peek();
if (!querier.querying()) {
// If not in the middle of an update, wait for next update time
final long nextUpdate = querier.getLastUpdate()
+ config.getInt(JdbcSourceTaskConfig.POLL_INTERVAL_MS_CONFIG);
final long now = time.milliseconds();
final long sleepMs = Math.min(nextUpdate - now, 100);
if (sleepMs > 0) {
log.trace("Waiting {} ms to poll {} next", nextUpdate - now, querier.toString());
time.sleep(sleepMs);
continue; // Re-check stop flag before continuing
}
}
final List<SourceRecord> results = new ArrayList<>();
try {
log.debug("Checking for next block of results from {}", querier.toString());
querier.maybeStartQuery(cachedConnectionProvider.getConnection());
int batchMaxRows = config.getInt(JdbcSourceTaskConfig.BATCH_MAX_ROWS_CONFIG);
boolean hadNext = true;
while (results.size() < batchMaxRows && (hadNext = querier.next())) {
results.add(querier.extractRecord());
}
querier.resetRetryCount();
if (!hadNext) {
// If we finished processing the results from the current query, we can reset and send
// the querier to the tail of the queue
resetAndRequeueHead(querier, false);
}
if (results.isEmpty()) {
consecutiveEmptyResults.compute(querier, (k, v) -> v + 1);
log.trace("No updates for {}", querier.toString());
if (Collections.min(consecutiveEmptyResults.values())
>= CONSECUTIVE_EMPTY_RESULTS_BEFORE_RETURN) {
log.trace("More than " + CONSECUTIVE_EMPTY_RESULTS_BEFORE_RETURN
+ " consecutive empty results for all queriers, returning");
return null;
} else {
continue;
}
} else {
consecutiveEmptyResults.put(querier, 0);
}
log.debug("Returning {} records for {}", results.size(), querier);
return results;
} catch (SQLNonTransientException sqle) {
log.error("Non-transient SQL exception while running query for table: {}",
querier, sqle);
resetAndRequeueHead(querier, true);
// This task has failed, so close any resources (may be reopened if needed) before throwing
closeResources();
throw new ConnectException(sqle);
} catch (SQLException sqle) {
log.error(
"SQL exception while running query for table: {}, {}."
+ " Attempting retry {} of {} attempts.",
querier,
sqle,
querier.getAttemptedRetryCount() + 1,
maxRetriesPerQuerier
);
resetAndRequeueHead(querier, true);
if (maxRetriesPerQuerier > 0
&& querier.getAttemptedRetryCount() >= maxRetriesPerQuerier) {
closeResources();
throw new ConnectException("Failed to Query table after retries", sqle);
}
querier.incrementRetryCount();
return null;
} catch (Throwable t) {
log.error("Failed to run query for table: {}", querier, t);
resetAndRequeueHead(querier, true);
// This task has failed, so close any resources (may be reopened if needed) before throwing
closeResources();
throw t;
}
}
shutdown();
return null;
}
private void shutdown() {
final TableQuerier querier = tableQueue.peek();
if (querier != null) {
resetAndRequeueHead(querier, true);
}
closeResources();
}
private void resetAndRequeueHead(TableQuerier expectedHead, boolean resetOffset) {
log.debug("Resetting querier {}", expectedHead.toString());
TableQuerier removedQuerier = tableQueue.poll();
assert removedQuerier == expectedHead;
expectedHead.reset(time.milliseconds(), resetOffset);
tableQueue.add(expectedHead);
}
private void validateNonNullable(
String incrementalMode,
String table,
String incrementingColumn,
List<String> timestampColumns
) {
try {
Set<String> lowercaseTsColumns = new HashSet<>();
for (String timestampColumn: timestampColumns) {
lowercaseTsColumns.add(timestampColumn.toLowerCase(Locale.getDefault()));
}
boolean incrementingOptional = false;
boolean atLeastOneTimestampNotOptional = false;
final Connection conn = cachedConnectionProvider.getConnection();
boolean autoCommit = conn.getAutoCommit();
try {
conn.setAutoCommit(true);
Map<ColumnId, ColumnDefinition> defnsById = dialect.describeColumns(conn, table, null);
for (ColumnDefinition defn : defnsById.values()) {
String columnName = defn.id().name();
if (columnName.equalsIgnoreCase(incrementingColumn)) {
incrementingOptional = defn.isOptional();
} else if (lowercaseTsColumns.contains(columnName.toLowerCase(Locale.getDefault()))) {
if (!defn.isOptional()) {
atLeastOneTimestampNotOptional = true;
}
}
}
} finally {
conn.setAutoCommit(autoCommit);
}
// Validate that requested columns for offsets are NOT NULL. Currently this is only performed
// for table-based copying because custom query mode doesn't allow this to be looked up
// without a query or parsing the query since we don't have a table name.
if ((incrementalMode.equals(JdbcSourceConnectorConfig.MODE_INCREMENTING)
|| incrementalMode.equals(JdbcSourceConnectorConfig.MODE_TIMESTAMP_INCREMENTING))
&& incrementingOptional) {
throw new ConnectException("Cannot make incremental queries using incrementing column "
+ incrementingColumn + " on " + table + " because this column "
+ "is nullable.");
}
if ((incrementalMode.equals(JdbcSourceConnectorConfig.MODE_TIMESTAMP)
|| incrementalMode.equals(JdbcSourceConnectorConfig.MODE_TIMESTAMP_INCREMENTING))
&& !atLeastOneTimestampNotOptional) {
throw new ConnectException("Cannot make incremental queries using timestamp columns "
+ timestampColumns + " on " + table + " because all of these "
+ "columns "
+ "nullable.");
}
} catch (SQLException e) {
throw new ConnectException("Failed trying to validate that columns used for offsets are NOT"
+ " NULL", e);
}
}
}