-
Notifications
You must be signed in to change notification settings - Fork 961
/
Copy pathJdbcSourceTaskLifecycleTest.java
379 lines (306 loc) · 12.5 KB
/
JdbcSourceTaskLifecycleTest.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
/*
* 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 org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.connect.source.SourceRecord;
import org.easymock.EasyMock;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.easymock.PowerMock;
import org.powermock.api.easymock.annotation.Mock;
import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.modules.junit4.PowerMockRunner;
import java.sql.Connection;
import java.sql.SQLException;
import java.sql.SQLNonTransientException;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
import io.confluent.connect.jdbc.util.CachedConnectionProvider;
import static org.easymock.EasyMock.anyBoolean;
import static org.easymock.EasyMock.anyLong;
import static org.easymock.EasyMock.anyObject;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.expectLastCall;
import static org.easymock.EasyMock.replay;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.IsInstanceOf.instanceOf;
import static org.hamcrest.core.StringContains.containsString;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
@RunWith(PowerMockRunner.class)
@PowerMockIgnore("javax.management.*")
public class JdbcSourceTaskLifecycleTest extends JdbcSourceTaskTestBase {
@Mock
private CachedConnectionProvider mockCachedConnectionProvider;
@Mock
private Connection conn;
@Test(expected = ConnectException.class)
public void testMissingParentConfig() {
Map<String, String> props = singleTableConfig();
props.remove(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG);
task.start(props);
}
@Test(expected = ConfigException.class)
public void testMissingTables() {
Map<String, String> props = singleTableConfig();
props.remove(JdbcSourceTaskConfig.TABLES_CONFIG);
task.start(props);
}
@Test
public void testStartStopDifferentThreads() throws Exception {
db.createTable(SINGLE_TABLE_NAME, "id", "INT");
// Minimal start/stop functionality
task = new JdbcSourceTask(time) {
@Override
protected CachedConnectionProvider connectionProvider(
int maxConnAttempts,
long retryBackoff
) {
return mockCachedConnectionProvider;
}
};
// Should request a connection, then should close it on stop()
EasyMock.expect(mockCachedConnectionProvider.getConnection()).andReturn(db.getConnection()).anyTimes();
mockCachedConnectionProvider.close(true);
PowerMock.expectLastCall();
PowerMock.replayAll();
ExecutorService executor = Executors.newSingleThreadExecutor();
Object lock = new Object();
AtomicBoolean running = new AtomicBoolean(true);
executor.submit(() -> {
task.start(singleTableConfig());
while (running.get()) {
task.poll();
synchronized (lock) {
lock.notifyAll();
}
}
return null;
});
synchronized (lock) {
lock.wait();
}
try {
task.stop();
synchronized (lock) {
lock.wait();
}
running.set(false);
} finally {
executor.shutdown();
}
PowerMock.verifyAll();
}
@Test
public void testStartStopSameThread() {
// Minimal start/stop functionality
task = new JdbcSourceTask(time) {
@Override
protected CachedConnectionProvider connectionProvider(
int maxConnAttempts,
long retryBackoff
) {
return mockCachedConnectionProvider;
}
};
// Should request a connection, then should close it on stop()
EasyMock.expect(mockCachedConnectionProvider.getConnection()).andReturn(db.getConnection());
EasyMock.expect(mockCachedConnectionProvider.getConnection()).andReturn(db.getConnection());
EasyMock.expect(mockCachedConnectionProvider.getConnection()).andReturn(db.getConnection());
mockCachedConnectionProvider.close(true);
PowerMock.expectLastCall();
PowerMock.replayAll();
task.start(singleTableConfig());
task.stop();
PowerMock.verifyAll();
}
@Test
public void testPollInterval() throws Exception {
// Here we just want to verify behavior of the poll method, not any loading of data, so we
// specifically want an empty
db.createTable(SINGLE_TABLE_NAME, "id", "INT");
// Need data or poll() never returns
db.insert(SINGLE_TABLE_NAME, "id", 1);
long startTime = time.milliseconds();
task.start(singleTableConfig());
// First poll should happen immediately
task.poll();
assertEquals(startTime, time.milliseconds());
// Subsequent polls have to wait for timeout
task.poll();
assertEquals(startTime + JdbcSourceConnectorConfig.POLL_INTERVAL_MS_DEFAULT,
time.milliseconds());
task.poll();
assertEquals(startTime + 2 * JdbcSourceConnectorConfig.POLL_INTERVAL_MS_DEFAULT,
time.milliseconds());
task.stop();
}
@Test
public void testSingleUpdateMultiplePoll() throws Exception {
// Test that splitting up a table update query across multiple poll() calls works
db.createTable(SINGLE_TABLE_NAME, "id", "INT");
Map<String, String> taskConfig = singleTableConfig();
taskConfig.put(JdbcSourceConnectorConfig.BATCH_MAX_ROWS_CONFIG, "1");
long startTime = time.milliseconds();
task.start(taskConfig);
// Two entries should get split across three poll() calls with no delay
db.insert(SINGLE_TABLE_NAME, "id", 1);
db.insert(SINGLE_TABLE_NAME, "id", 2);
List<SourceRecord> records = task.poll();
assertEquals(startTime, time.milliseconds());
assertEquals(1, records.size());
records = task.poll();
assertEquals(startTime, time.milliseconds());
assertEquals(1, records.size());
// Subsequent poll should wait for next timeout
task.poll();
assertEquals(startTime + JdbcSourceConnectorConfig.POLL_INTERVAL_MS_DEFAULT,
time.milliseconds());
}
@Test
public void testMultipleTables() throws Exception {
db.createTable(SINGLE_TABLE_NAME, "id", "INT");
db.createTable(SECOND_TABLE_NAME, "id", "INT");
long startTime = time.milliseconds();
task.start(twoTableConfig());
db.insert(SINGLE_TABLE_NAME, "id", 1);
db.insert(SECOND_TABLE_NAME, "id", 2);
// Both tables should be polled immediately, in order
List<SourceRecord> records = task.poll();
assertEquals(startTime, time.milliseconds());
assertEquals(1, records.size());
assertEquals(SINGLE_TABLE_PARTITION, records.get(0).sourcePartition());
records = task.poll();
assertEquals(startTime, time.milliseconds());
assertEquals(1, records.size());
assertEquals(SECOND_TABLE_PARTITION, records.get(0).sourcePartition());
// Subsequent poll should wait for next timeout
records = task.poll();
assertEquals(startTime + JdbcSourceConnectorConfig.POLL_INTERVAL_MS_DEFAULT,
time.milliseconds());
validatePollResultTable(records, 1, SINGLE_TABLE_NAME);
records = task.poll();
assertEquals(startTime + JdbcSourceConnectorConfig.POLL_INTERVAL_MS_DEFAULT,
time.milliseconds());
validatePollResultTable(records, 1, SECOND_TABLE_NAME);
}
@Test
public void testMultipleTablesMultiplePolls() throws Exception {
// Check correct handling of multiple tables when the tables require multiple poll() calls to
// return one query's data
db.createTable(SINGLE_TABLE_NAME, "id", "INT");
db.createTable(SECOND_TABLE_NAME, "id", "INT");
Map<String, String> taskConfig = twoTableConfig();
taskConfig.put(JdbcSourceConnectorConfig.BATCH_MAX_ROWS_CONFIG, "1");
long startTime = time.milliseconds();
task.start(taskConfig);
db.insert(SINGLE_TABLE_NAME, "id", 1);
db.insert(SINGLE_TABLE_NAME, "id", 2);
db.insert(SECOND_TABLE_NAME, "id", 3);
db.insert(SECOND_TABLE_NAME, "id", 4);
// Both tables should be polled immediately, in order
for(int i = 0; i < 2; i++) {
List<SourceRecord> records = task.poll();
assertEquals(startTime, time.milliseconds());
validatePollResultTable(records, 1, SINGLE_TABLE_NAME);
}
for(int i = 0; i < 2; i++) {
List<SourceRecord> records = task.poll();
assertEquals(startTime, time.milliseconds());
validatePollResultTable(records, 1, SECOND_TABLE_NAME);
}
// Subsequent poll should wait for next timeout
for(int i = 0; i < 2; i++) {
List<SourceRecord> records = task.poll();
assertEquals(startTime + JdbcSourceConnectorConfig.POLL_INTERVAL_MS_DEFAULT,
time.milliseconds());
validatePollResultTable(records, 1, SINGLE_TABLE_NAME);
}
for(int i = 0; i < 2; i++) {
List<SourceRecord> records = task.poll();
assertEquals(startTime + JdbcSourceConnectorConfig.POLL_INTERVAL_MS_DEFAULT,
time.milliseconds());
validatePollResultTable(records, 1, SECOND_TABLE_NAME);
}
}
@Test
public void testMultipleTablesNothingToDoReturns() throws Exception {
db.createTable(SINGLE_TABLE_NAME, "id", "INT");
db.createTable(SECOND_TABLE_NAME, "id", "INT");
task.start(twoTableConfig());
assertNull(task.poll());
}
@Test
public void testNonTransientSQLExceptionThrows() throws Exception {
db.createTable(SINGLE_TABLE_NAME, "id", "INT");
Map<String, String> config = singleTableConfig();
config.put(JdbcSourceTaskConfig.TABLES_CONFIG, "not_existing_table");
task.start(config);
ConnectException e = assertThrows(ConnectException.class, () -> {
task.poll();
});
assertThat(e.getCause(), instanceOf(SQLNonTransientException.class));
assertThat(e.getMessage(), containsString("not_existing_table"));
}
@Test(expected = ConnectException.class)
public void testTransientSQLExceptionRetries() throws Exception {
int retryMax = 2; //max times to retry
TableQuerier bulkTableQuerier = EasyMock.createMock(BulkTableQuerier.class);
for (int i = 0; i < retryMax+1; i++) {
expect(bulkTableQuerier.querying()).andReturn(true);
bulkTableQuerier.maybeStartQuery(anyObject());
expectLastCall().andThrow(new SQLException("This is a transient exception"));
expect(bulkTableQuerier.getAttemptedRetryCount()).andReturn(i);
// Called another time in error logging
expect(bulkTableQuerier.getAttemptedRetryCount()).andReturn(i);
bulkTableQuerier.incrementRetryCount();
expectLastCall().once();
bulkTableQuerier.reset(anyLong(), anyBoolean());
}
replay(bulkTableQuerier);
JdbcSourceTask mockedTask = setUpMockedTask(bulkTableQuerier, retryMax);
for (int i = 0; i < retryMax+1; i++) {
mockedTask.poll();
}
}
private JdbcSourceTask setUpMockedTask(TableQuerier bulkTableQuerier, int retryMax) throws Exception {
CachedConnectionProvider mockCachedConnectionProvider = EasyMock.createMock(CachedConnectionProvider.class);
for (int i = 0; i < retryMax+1; i++) {
expect(mockCachedConnectionProvider.getConnection()).andReturn(null);
}
replay(mockCachedConnectionProvider);
PriorityQueue<TableQuerier> priorityQueue = new PriorityQueue<>();
priorityQueue.add(bulkTableQuerier);
JdbcSourceTask mockedTask = new JdbcSourceTask(time);
mockedTask.start(singleTableConfig());
mockedTask.tableQueue = priorityQueue;
mockedTask.cachedConnectionProvider = mockCachedConnectionProvider;
mockedTask.maxRetriesPerQuerier = retryMax;
return mockedTask;
}
private static void validatePollResultTable(List<SourceRecord> records,
int expected, String table) {
assertEquals(expected, records.size());
for (SourceRecord record : records) {
assertEquals(table, record.sourcePartition().get(JdbcSourceConnectorConstants.TABLE_NAME_KEY));
}
}
}