/
MongoDbStreamingChangeEventSource.java
490 lines (426 loc) · 20.7 KB
/
MongoDbStreamingChangeEventSource.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
/*
* 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.mongodb;
import java.time.Duration;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.OptionalLong;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.connect.errors.ConnectException;
import org.bson.BsonTimestamp;
import org.bson.Document;
import org.bson.conversions.Bson;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.mongodb.CursorType;
import com.mongodb.ServerAddress;
import com.mongodb.client.FindIterable;
import com.mongodb.client.MongoClient;
import com.mongodb.client.MongoCollection;
import com.mongodb.client.MongoCursor;
import com.mongodb.client.model.Filters;
import io.debezium.DebeziumException;
import io.debezium.connector.mongodb.ConnectionContext.MongoPrimary;
import io.debezium.data.Envelope.Operation;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.txmetadata.TransactionContext;
import io.debezium.util.Clock;
import io.debezium.util.Metronome;
import io.debezium.util.Threads;
/**
*
* @author Chris Cranford
*/
public class MongoDbStreamingChangeEventSource implements StreamingChangeEventSource<MongoDbOffsetContext> {
private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbStreamingChangeEventSource.class);
private static final String AUTHORIZATION_FAILURE_MESSAGE = "Command failed with error 13";
private static final String OPERATION_FIELD = "op";
private static final String OBJECT_FIELD = "o";
private static final String OPERATION_CONTROL = "c";
private static final String TX_OPS = "applyOps";
private final MongoDbConnectorConfig connectorConfig;
private final EventDispatcher<CollectionId> dispatcher;
private final ErrorHandler errorHandler;
private final Clock clock;
private final ConnectionContext connectionContext;
private final ReplicaSets replicaSets;
private final MongoDbTaskContext taskContext;
public MongoDbStreamingChangeEventSource(MongoDbConnectorConfig connectorConfig, MongoDbTaskContext taskContext,
ReplicaSets replicaSets,
EventDispatcher<CollectionId> dispatcher, ErrorHandler errorHandler, Clock clock) {
this.connectorConfig = connectorConfig;
this.connectionContext = taskContext.getConnectionContext();
this.dispatcher = dispatcher;
this.errorHandler = errorHandler;
this.clock = clock;
this.replicaSets = replicaSets;
this.taskContext = taskContext;
}
@Override
public void execute(ChangeEventSourceContext context, MongoDbOffsetContext offsetContext) throws InterruptedException {
final List<ReplicaSet> validReplicaSets = replicaSets.validReplicaSets();
if (offsetContext == null) {
offsetContext = initializeOffsets(connectorConfig, replicaSets);
}
try {
if (validReplicaSets.size() == 1) {
// Streams the replica-set changes in the current thread
streamChangesForReplicaSet(context, validReplicaSets.get(0), offsetContext);
}
else if (validReplicaSets.size() > 1) {
// Starts a thread for each replica-set and executes the streaming process
streamChangesForReplicaSets(context, validReplicaSets, offsetContext);
}
}
finally {
taskContext.getConnectionContext().shutdown();
}
}
private void streamChangesForReplicaSet(ChangeEventSourceContext context, ReplicaSet replicaSet,
MongoDbOffsetContext offsetContext) {
MongoPrimary primaryClient = null;
try {
primaryClient = establishConnectionToPrimary(replicaSet);
if (primaryClient != null) {
final AtomicReference<MongoPrimary> primaryReference = new AtomicReference<>(primaryClient);
primaryClient.execute("read from oplog on '" + replicaSet + "'", primary -> {
readOplog(primary, primaryReference.get(), replicaSet, context, offsetContext);
});
}
}
catch (Throwable t) {
LOGGER.error("Streaming for replica set {} failed", replicaSet.replicaSetName(), t);
errorHandler.setProducerThrowable(t);
}
finally {
if (primaryClient != null) {
primaryClient.stop();
}
}
}
private void streamChangesForReplicaSets(ChangeEventSourceContext context, List<ReplicaSet> replicaSets,
MongoDbOffsetContext offsetContext) {
final int threads = replicaSets.size();
final ExecutorService executor = Threads.newFixedThreadPool(MongoDbConnector.class, taskContext.serverName(), "replicator-streaming", threads);
final CountDownLatch latch = new CountDownLatch(threads);
LOGGER.info("Starting {} thread(s) to stream changes for replica sets: {}", threads, replicaSets);
replicaSets.forEach(replicaSet -> {
executor.submit(() -> {
try {
streamChangesForReplicaSet(context, replicaSet, offsetContext);
}
finally {
latch.countDown();
}
});
});
// Wait for the executor service to terminate.
try {
latch.await();
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
executor.shutdown();
}
private MongoPrimary establishConnectionToPrimary(ReplicaSet replicaSet) {
return connectionContext.primaryFor(replicaSet, taskContext.filters(), (desc, error) -> {
// propagate authorization failures
if (error.getMessage() != null && error.getMessage().startsWith(AUTHORIZATION_FAILURE_MESSAGE)) {
throw new ConnectException("Error while attempting to " + desc, error);
}
else {
dispatcher.dispatchConnectorEvent(new DisconnectEvent());
LOGGER.error("Error while attempting to {}: {}", desc, error.getMessage(), error);
throw new ConnectException("Error while attempting to " + desc, error);
}
});
}
private void readOplog(MongoClient primary, MongoPrimary primaryClient, ReplicaSet replicaSet, ChangeEventSourceContext context,
MongoDbOffsetContext offsetContext) {
final ReplicaSetOffsetContext rsOffsetContext = offsetContext.getReplicaSetOffsetContext(replicaSet);
final BsonTimestamp oplogStart = rsOffsetContext.lastOffsetTimestamp();
final OptionalLong txOrder = rsOffsetContext.lastOffsetTxOrder();
final ServerAddress primaryAddress = MongoUtil.getPrimaryAddress(primary);
LOGGER.info("Reading oplog for '{}' primary {} starting at {}", replicaSet, primaryAddress, oplogStart);
// Include none of the cluster-internal operations and only those events since the previous timestamp
MongoCollection<Document> oplog = primary.getDatabase("local").getCollection("oplog.rs");
// DBZ-3331 Verify that the start position is in the oplog; throw exception if not.
if (oplog.countDocuments(Filters.eq("ts", oplogStart)) == 0L) {
throw new DebeziumException("Failed to find starting position '" + oplogStart + "' in oplog");
}
ReplicaSetOplogContext oplogContext = new ReplicaSetOplogContext(rsOffsetContext, primaryClient, replicaSet);
Bson filter = null;
if (!txOrder.isPresent()) {
LOGGER.info("The last event processed was not transactional, resuming at the oplog event after '{}'", oplogStart);
filter = Filters.and(Filters.gt("ts", oplogStart), // start just after our last position
Filters.exists("fromMigrate", false)); // skip internal movements across shards
}
else {
LOGGER.info("The last event processed was transactional, resuming at the oplog event '{}', expecting to skip '{}' events",
oplogStart, txOrder.getAsLong());
filter = Filters.and(Filters.gte("ts", oplogStart), Filters.exists("fromMigrate", false));
oplogContext.setIncompleteEventTimestamp(oplogStart);
oplogContext.setIncompleteTxOrder(txOrder.getAsLong());
}
Bson operationFilter = getSkippedOperationsFilter();
if (operationFilter != null) {
filter = Filters.and(filter, operationFilter);
}
final FindIterable<Document> results = oplog.find(filter)
.sort(new Document("$natural", 1))
.oplogReplay(true)
.cursorType(CursorType.TailableAwait);
try (MongoCursor<Document> cursor = results.iterator()) {
// In Replicator, this used cursor.hasNext() but this is a blocking call and I observed that this can
// delay the shutdown of the connector by up to 15 seconds or longer. By introducing a Metronome, we
// can respond to the stop request much faster and without much overhead.
Metronome pause = Metronome.sleeper(Duration.ofMillis(500), clock);
while (context.isRunning()) {
// Use tryNext which will return null if no document is yet available from the cursor.
// In this situation if not document is available, we'll pause.
final Document event = cursor.tryNext();
if (event != null) {
if (!handleOplogEvent(primaryAddress, event, event, 0, oplogContext, context)) {
// Something happened and we are supposed to stop reading
return;
}
try {
dispatcher.dispatchHeartbeatEvent(oplogContext.getOffset());
}
catch (InterruptedException e) {
LOGGER.info("Replicator thread is interrupted");
Thread.currentThread().interrupt();
return;
}
}
else {
try {
pause.pause();
}
catch (InterruptedException e) {
break;
}
}
}
}
}
private Bson getSkippedOperationsFilter() {
Set<Operation> skippedOperations = taskContext.getConnectorConfig().getSkippedOperations();
if (skippedOperations.isEmpty()) {
return null;
}
Bson skippedOperationsFilter = null;
for (Operation operation : skippedOperations) {
Bson skippedOperationFilter = Filters.ne("op", operation.code());
if (skippedOperationsFilter == null) {
skippedOperationsFilter = skippedOperationFilter;
}
else {
skippedOperationsFilter = Filters.or(skippedOperationsFilter, skippedOperationFilter);
}
}
return skippedOperationsFilter;
}
private boolean handleOplogEvent(ServerAddress primaryAddress, Document event, Document masterEvent, long txOrder, ReplicaSetOplogContext oplogContext,
ChangeEventSourceContext context) {
String ns = event.getString("ns");
Document object = event.get(OBJECT_FIELD, Document.class);
if (Objects.isNull(object)) {
if (LOGGER.isWarnEnabled()) {
LOGGER.warn("Missing 'o' field in event, so skipping {}", event.toJson());
}
return true;
}
if (Objects.isNull(ns) || ns.isEmpty()) {
// These are considered replica set events
String msg = object.getString("msg");
if ("new primary".equals(msg)) {
AtomicReference<ServerAddress> address = new AtomicReference<>();
try {
oplogContext.getPrimary().executeBlocking("conn", mongoClient -> {
ServerAddress currentPrimary = MongoUtil.getPrimaryAddress(mongoClient);
address.set(currentPrimary);
});
}
catch (InterruptedException e) {
LOGGER.error("Get current primary executeBlocking", e);
}
ServerAddress serverAddress = address.get();
if (Objects.nonNull(serverAddress) && !serverAddress.equals(primaryAddress)) {
LOGGER.info("Found new primary event in oplog, so stopping use of {} to continue with new primary {}",
primaryAddress, serverAddress);
}
else {
LOGGER.info("Found new primary event in oplog, current {} is new primary. " +
"Continue to process oplog event.", primaryAddress);
}
dispatcher.dispatchConnectorEvent(new PrimaryElectionEvent(serverAddress));
}
// Otherwise ignore
if (LOGGER.isDebugEnabled()) {
LOGGER.debug("Skipping event with no namespace: {}", event.toJson());
}
return true;
}
final List<Document> txChanges = transactionChanges(event);
if (!txChanges.isEmpty()) {
if (Objects.nonNull(oplogContext.getIncompleteEventTimestamp())) {
if (oplogContext.getIncompleteEventTimestamp().equals(SourceInfo.extractEventTimestamp(event))) {
for (Document change : txChanges) {
txOrder++;
if (txOrder <= oplogContext.getIncompleteTxOrder()) {
LOGGER.debug("Skipping record as it is expected to be already processed: {}", change);
continue;
}
final boolean r = handleOplogEvent(primaryAddress, change, event, txOrder, oplogContext, context);
if (!r) {
return false;
}
}
}
oplogContext.setIncompleteEventTimestamp(null);
return true;
}
try {
dispatcher.dispatchTransactionStartedEvent(getTransactionId(event), oplogContext.getOffset());
for (Document change : txChanges) {
final boolean r = handleOplogEvent(primaryAddress, change, event, ++txOrder, oplogContext, context);
if (!r) {
return false;
}
}
dispatcher.dispatchTransactionCommittedEvent(oplogContext.getOffset());
}
catch (InterruptedException e) {
LOGGER.error("Streaming transaction changes for replica set '{}' was interrupted", oplogContext.getReplicaSetName());
throw new ConnectException("Streaming of transaction changes was interrupted for replica set " + oplogContext.getReplicaSetName(), e);
}
return true;
}
final String operation = event.getString(OPERATION_FIELD);
if (!MongoDbChangeRecordEmitter.isValidOperation(operation)) {
LOGGER.debug("Skipping event with \"op={}\"", operation);
return true;
}
int delimIndex = ns.indexOf('.');
if (delimIndex > 0) {
assert (delimIndex + 1) < ns.length();
final String dbName = ns.substring(0, delimIndex);
final String collectionName = ns.substring(delimIndex + 1);
if ("$cmd".equals(collectionName)) {
// This is a command on the database
// TODO: Probably want to handle some of these when we track creation/removal of collections
LOGGER.debug("Skipping database command event: {}", event.toJson());
return true;
}
// Otherwise it is an event on a document in a collection
if (!taskContext.filters().databaseFilter().test(dbName)) {
LOGGER.debug("Skipping the event for database {} based on database.whitelist", dbName);
return true;
}
oplogContext.getOffset().oplogEvent(event, masterEvent, txOrder);
oplogContext.getOffset().getOffset();
CollectionId collectionId = new CollectionId(oplogContext.getReplicaSetName(), dbName, collectionName);
if (taskContext.filters().collectionFilter().test(collectionId)) {
try {
return dispatcher.dispatchDataChangeEvent(
collectionId,
new MongoDbChangeRecordEmitter(
oplogContext.getOffset(),
clock,
event,
false));
}
catch (Exception e) {
errorHandler.setProducerThrowable(e);
return false;
}
}
}
return true;
}
private List<Document> transactionChanges(Document event) {
final String op = event.getString(OPERATION_FIELD);
final Document o = event.get(OBJECT_FIELD, Document.class);
if (!(OPERATION_CONTROL.equals(op) && Objects.nonNull(o) && o.containsKey(TX_OPS))) {
return Collections.emptyList();
}
return o.get(TX_OPS, List.class);
}
protected MongoDbOffsetContext initializeOffsets(MongoDbConnectorConfig connectorConfig, ReplicaSets replicaSets) {
final Map<ReplicaSet, Document> positions = new LinkedHashMap<>();
replicaSets.onEachReplicaSet(replicaSet -> {
LOGGER.info("Determine Snapshot Offset for replica-set {}", replicaSet.replicaSetName());
MongoPrimary primaryClient = establishConnectionToPrimary(replicaSet);
if (primaryClient != null) {
try {
primaryClient.execute("get oplog position", primary -> {
MongoCollection<Document> oplog = primary.getDatabase("local").getCollection("oplog.rs");
Document last = oplog.find().sort(new Document("$natural", -1)).limit(1).first(); // may be null
positions.put(replicaSet, last);
});
}
finally {
LOGGER.info("Stopping primary client");
primaryClient.stop();
}
}
});
return new MongoDbOffsetContext(new SourceInfo(connectorConfig), new TransactionContext(), positions);
}
private static String getTransactionId(Document event) {
final Long operationId = event.getLong(SourceInfo.OPERATION_ID);
if (operationId != null && operationId != 0L) {
return Long.toString(operationId);
}
return MongoUtil.getOplogSessionTransactionId(event);
}
/**
* A context associated with a given replica set oplog read operation.
*/
private class ReplicaSetOplogContext {
private final ReplicaSetOffsetContext offset;
private final MongoPrimary primary;
private final ReplicaSet replicaSet;
private BsonTimestamp incompleteEventTimestamp;
private long incompleteTxOrder = 0;
ReplicaSetOplogContext(ReplicaSetOffsetContext offsetContext, MongoPrimary primary, ReplicaSet replicaSet) {
this.offset = offsetContext;
this.primary = primary;
this.replicaSet = replicaSet;
}
ReplicaSetOffsetContext getOffset() {
return offset;
}
MongoPrimary getPrimary() {
return primary;
}
String getReplicaSetName() {
return replicaSet.replicaSetName();
}
BsonTimestamp getIncompleteEventTimestamp() {
return incompleteEventTimestamp;
}
public void setIncompleteEventTimestamp(BsonTimestamp incompleteEventTimestamp) {
this.incompleteEventTimestamp = incompleteEventTimestamp;
}
public long getIncompleteTxOrder() {
return incompleteTxOrder;
}
public void setIncompleteTxOrder(long incompleteTxOrder) {
this.incompleteTxOrder = incompleteTxOrder;
}
}
}