Debezium connector for MongoDB
Overview
MongoDB’s replication mechanism provides redundancy and high availability, and is the preferred way to run MongoDB in production. MongoDB connector captures the changes in a replica set or sharded cluster.
A MongoDB replica set consists of a set of servers that all have copies of the same data, and replication ensures that all changes made by clients to documents on the replica set’s primary are correctly applied to the other replica set’s servers, called secondaries. MongoDB replication works by having the primary record the changes in its oplog (or operation log), and then each of the secondaries reads the primary’s oplog and applies in order all of the operations to their own documents. When a new server is added to a replica set, that server first performs an snapshot of all of the databases and collections on the primary, and then reads the primary’s oplog to apply all changes that might have been made since it began the snapshot. This new server becomes a secondary (and able to handle queries) when it catches up to the tail of the primary’s oplog.
The MongoDB connector uses this same replication mechanism, though it does not actually become a member of the replica set. Just like MongoDB secondaries, however, the connector always reads the oplog of the replica set’s primary. And, when the connector sees a replica set for the first time, it looks at the oplog to get the last recorded transaction and then performs a snapshot of the primary’s databases and collections. When all the data is copied, the connector then starts streaming changes from the position it read earlier from the oplog. Operations in the MongoDB oplog are idempotent, so no matter how many times the operations are applied, they result in the same end state.
As the MongoDB connector processes changes, it periodically records the position in the oplog where the event originated. When the MongoDB connector stops, it records the last oplog position that it processed, so that upon restart it simply begins streaming from that position. In other words, the connector can be stopped, upgraded or maintained, and restarted some time later, and it will pick up exactly where it left off without losing a single event. Of course, MongoDB’s oplogs are usually capped at a maximum size, which means that the connector should not be stopped for too long, or else some of the operations in the oplog might be purged before the connector has a chance to read them. In this case, upon restart the connector will detect the missing oplog operations, perform a snapshot, and then proceed with streaming the changes.
The MongoDB connector is also quite tolerant of changes in membership and leadership of the replica sets, of additions or removals of shards within a sharded cluster, and network problems that might cause communication failures. The connector always uses the replica set’s primary node to stream changes, so when the replica set undergoes an election and a different node becomes primary, the connector will immediately stop streaming changes, connect to the new primary, and start streaming changes using the new primary node. Likewise, if connector experiences any problems communicating with the replica set primary, it will try to reconnect (using exponential backoff so as to not overwhelm the network or replica set) and continue streaming changes from where it last left off. In this way the connector is able to dynamically adjust to changes in replica set membership and to automatically handle communication failures.
Additional resources
Setting up MongoDB
The MongoDB connector uses MongoDB’s oplog to capture the changes, so the connector works only with MongoDB replica sets or with sharded clusters where each shard is a separate replica set. See the MongoDB documentation for setting up a replica set or sharded cluster. Also, be sure to understand how to enable access control and authentication with replica sets.
You must also have a MongoDB user that has the appropriate roles to read the admin
database where the oplog can be read. Additionally, the user must also be able to read the config
database in the configuration server of a sharded cluster and must have listDatabases
privilege action.
Supported MongoDB topologies
The MongoDB connector can be used with a variety of MongoDB topologies.
MongoDB replica set
The MongoDB connector can capture changes from a single MongoDB replica set. Production replica sets require a minimum of at least three members.
To use the MongoDB connector with a replica set, provide the addresses of one or more replica set servers as seed addresses through the connector’s mongodb.hosts
property. The connector will use these seeds to connect to the replica set, and then once connected will get from the replica set the complete set of members and which member is primary. The connector will start a task to connect to the primary and capture the changes from the primary’s oplog. When the replica set elects a new primary, the task will automatically switch over to the new primary.
When MongoDB is fronted by a proxy (such as with Docker on OS X or Windows), then when a client connects to the replica set and discovers the members, the MongoDB client will exclude the proxy as a valid member and will attempt and fail to connect directly to the members rather than go through the proxy. In such a case, set the connector’s optional |
MongoDB sharded cluster
A MongoDB sharded cluster consists of:
One or more shards, each deployed as a replica set;
A separate replica set that acts as the cluster’s configuration server
One or more routers (also called
mongos
) to which clients connect and that routes requests to the appropriate shards
To use the MongoDB connector with a sharded cluster, configure the connector with the host addresses of the configuration server replica set. When the connector connects to this replica set, it discovers that it is acting as the configuration server for a sharded cluster, discovers the information about each replica set used as a shard in the cluster, and will then start up a separate task to capture the changes from each replica set. If new shards are added to the cluster or existing shards removed, the connector will automatically adjust its tasks accordingly.
MongoDB standalone server
The MongoDB connector is not capable of monitoring the changes of a standalone MongoDB server, since standalone servers do not have an oplog. The connector will work if the standalone server is converted to a replica set with one member.
MongoDB does not recommend running a standalone server in production. |
How the MongoDB connector works
When a MongoDB connector is configured and deployed, it starts by connecting to the MongoDB servers at the seed addresses, and determines the details about each of the available replica sets. Since each replica set has its own independent oplog, the connector will try to use a separate task for each replica set. The connector can limit the maximum number of tasks it will use, and if not enough tasks are available the connector will assign multiple replica sets to each task, although the task will still use a separate thread for each replica set.
When running the connector against a sharded cluster, use a value of |
Logical connector name
The connector configuration property mongodb.name
serves as a logical name for the MongoDB replica set or sharded cluster. The connector uses the logical name in a number of ways: as the prefix for all topic names, and as a unique identifier when recording the oplog position of each replica set.
You should give each MongoDB connector a unique logical name that meaningfully describes the source MongoDB system. We recommend logical names begin with an alphabetic or underscore character, and remaining characters that are alphanumeric or underscore.
Performing a snapshot
When a task starts up using a replica set, it uses the connector’s logical name and the replica set name to find an offset that describes the position where the connector previously stopped reading changes. If an offset can be found and it still exists in the oplog, then the task immediately proceeds with streaming changes, starting at the recorded offset position.
However, if no offset is found or if the oplog no longer contains that position, the task must first obtain the current state of the replica set contents by performing a snapshot. This process starts by recording the current position of the oplog and recording that as the offset (along with a flag that denotes a snapshot has been started). The task will then proceed to copy each collection, spawning as many threads as possible (up to the value of the initial.sync.max.threads
configuration property) to perform this work in parallel. The connector will record a separate read event for each document it sees, and that read event will contain the object’s identifier, the complete state of the object, and source information about the MongoDB replica set where the object was found. The source information will also include a flag that denotes the event was produced during a snapshot.
This snapshot will continue until it has copied all collections that match the connector’s filters. If the connector is stopped before the tasks’ snapshots are completed, upon restart the connector begins the snapshot again.
Try to avoid task reassignment and reconfiguration while the connector is performing a snapshot of any replica sets. The connector does log messages with the progress of the snapshot. For utmost control, run a separate cluster of Kafka Connect for each connector. |
Streaming changes
Once the connector task for a replica set has an offset, it uses the offset to determine the position in the oplog where it should start streaming changes. The task will then connect to the replica set’s primary node and start streaming changes from that position, processing all of the create, insert, and delete operations and converting them into Debezium change events. Each change event includes the position in the oplog where the operation was found, and the connector periodically records this as its most recent offset. The interval at which the offset is recorded is governed by offset.flush.interval.ms, which is a Kafka Connect worker configuration property.
When the connector is stopped gracefully, the last offset processed is recorded so that, upon restart, the connector will continue exactly where it left off. If the connector’s tasks terminate unexpectedly, however, then the tasks may have processed and generated events after it last records the offset but before the last offset is recorded; upon restart, the connector begins at the last recorded offset, possibly generating some the same events that were previously generated just prior to the crash.
When everything is operating nominally, Kafka consumers will actually see every message exactly once. However, when things go wrong Kafka can only guarantee consumers will see every message at least once. Therefore, your consumers need to anticipate seeing messages more than once. |
As mentioned above, the connector tasks always use the replica set’s primary node to stream changes from the oplog, ensuring that the connector sees the most up-to-date operations as possible and can capture the changes with lower latency than if secondaries were to be used instead. When the replica set elects a new primary, the connector immediately stops streaming changes, connects to the new primary, and starts streaming changes from the new primary node at the same position. Likewise, if the connector experiences any problems communicating with the replica set members, it trys to reconnect, by using exponential backoff so as to not overwhelm the replica set, and once connected it continues streaming changes from where it last left off. In this way, the connector is able to dynamically adjust to changes in replica set membership and automatically handle communication failures.
To summarize, the MongoDB connector continues running in most situations. Communication problems might cause the connector to wait until the problems are resolved.
Topics names
The MongoDB connector writes events for all insert, update, and delete operations to documents in each collection to a single Kafka topic. The name of the Kafka topics always takes the form logicalName.databaseName.collectionName, where logicalName is the logical name of the connector as specified with the mongodb.name
configuration property, databaseName is the name of the database where the operation occurred, and collectionName is the name of the MongoDB collection in which the affected document existed.
For example, consider a MongoDB replica set with an inventory
database that contains four collections: products
, products_on_hand
, customers
, and orders
. If the connector monitoring this database were given a logical name of fulfillment
, then the connector would produce events on these four Kafka topics:
fulfillment.inventory.products
fulfillment.inventory.products_on_hand
fulfillment.inventory.customers
fulfillment.inventory.orders
Notice that the topic names do not incorporate the replica set name or shard name. As a result, all changes to a sharded collection (where each shard contains a subset of the collection’s documents) all go to the same Kafka topic.
You can set up Kafka to auto-create the topics as they are needed. If not, then you must use Kafka administration tools to create the topics before starting the connector.
Partitions
The MongoDB connector does not make any explicit determination of the topic partitions for events. Instead, it allows Kafka to determine the partition based on the key. You can change Kafka’s partitioning logic by defining in the Kafka Connect worker configuration the name of the Partitioner
implementation.
Kafka maintains total order only for events written to a single topic partition. Partitioning the events by key does mean that all events with the same key always go to the same partition. This ensures that all events for a specific document are always totally ordered.
Data change events
The Debezium MongoDB connector generates a data change event for each document-level operation that inserts, updates, or deletes data. Each event contains a key and a value. The structure of the key and the value depends on the collection that was changed.
Debezium and Kafka Connect are designed around continuous streams of event messages. However, the structure of these events may change over time, which can be difficult for consumers to handle. To address this, each event contains the schema for its content or, if you are using a schema registry, a schema ID that a consumer can use to obtain the schema from the registry. This makes each event self-contained.
The following skeleton JSON shows the basic four parts of a change event. However, how you configure the Kafka Connect converter that you choose to use in your application determines the representation of these four parts in change events. A schema
field is in a change event only when you configure the converter to produce it. Likewise, the event key and event payload are in a change event only if you configure a converter to produce it. If you use the JSON converver and you configure it to produce all four basic change event parts, change events have this structure:
{
"schema": { (1)
...
},
"payload": { (2)
...
},
"schema": { (3)
...
},
"payload": { (4)
...
},
}
Item | Field name | Description |
---|---|---|
1 |
| The first |
2 |
| The first |
3 |
| The second |
4 |
| The second |
By default, the connector streams change event records to topics with names that are the same as the event’s originating collection. See topic names.
The MongoDB connector ensures that all Kafka Connect schema names adhere to the Avro schema name format. This means that the logical server name must start with a Latin letter or an underscore, that is, a-z, A-Z, or . Each remaining character in the logical server name and each character in the database and collection names must be a Latin letter, a digit, or an underscore, that is, a-z, A-Z, 0-9, or \. If there is an invalid character it is replaced with an underscore character. This can lead to unexpected conflicts if the logical server name, a database name, or a collection name contains invalid characters, and the only characters that distinguish names from one another are invalid and thus replaced with underscores. |
Change event keys
A change event’s key contains the schema for the changed document’s key and the changed document’s actual key. For a given collection, both the schema and its corresponding payload contain a single id
field. The value of this field is the document’s identifier represented as a string that is derived from MongoDB extended JSON serialization strict mode.
Consider a connector with a logical name of fulfillment
, a replica set containing an inventory
database, and a customers
collection that contains documents such as the following.
Example document
{
"_id": 1004,
"first_name": "Anne",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
}
Example change event key
Every change event that captures a change to the customers
collection has the same event key schema. For as long as the customers
collection has the previous definition, every change event that captures a change to the customers
collection has the following key structure. In JSON, it looks like this:
{
"schema": { (1)
"type": "struct",
"name": "fulfillment.inventory.customers.Key", (2)
"optional": false, (3)
"fields": [ (4)
{
"field": "id",
"type": "string",
"optional": false
}
]
},
"payload": { (5)
"id": "1004"
}
}
Item | Field name | Description |
---|---|---|
1 |
| The schema portion of the key specifies a Kafka Connect schema that describes what is in the key’s |
2 |
| Name of the schema that defines the structure of the key’s payload. This schema describes the structure of the key for the document that was changed. Key schema names have the format connector-name.database-name.collection-name.
|
3 |
| Indicates whether the event key must contain a value in its |
4 |
| Specifies each field that is expected in the |
5 |
| Contains the key for the document for which this change event was generated. In this example, the key contains a single |
This example uses a document with an integer identifier, but any valid MongoDB document identifier works the same way, including a document identifier. For a document identifier, an event key’s payload.id
value is a string that represents the updated document’s original _id
field as a MongoDB extended JSON serialization that uses strict mode. The following table provides examples of how different types of _id
fields are represented.
Type | MongoDB _id Value | Key’s payload |
---|---|---|
Integer | 1234 |
|
Float | 12.34 |
|
String | “1234” |
|
Document |
|
|
ObjectId |
|
|
Binary |
|
|
Change event values
The value in a change event is a bit more complicated than the key. Like the key, the value has a schema
section and a payload
section. The schema
section contains the schema that describes the Envelope
structure of the payload
section, including its nested fields. Change events for operations that create, update or delete data all have a value payload with an envelope structure.
Consider the same sample document that was used to show an example of a change event key:
Example document
{
"_id": 1004,
"first_name": "Anne",
"last_name": "Kretchmar",
"email": "annek@noanswer.org"
}
The value portion of a change event for a change to this document is described for each event type:
create events
The following example shows the value portion of a change event that the connector generates for an operation that creates data in the customers
collection:
{
"schema": { (1)
"type": "struct",
"fields": [
{
"type": "string",
"optional": true,
"name": "io.debezium.data.Json", (2)
"version": 1,
"field": "after"
},
{
"type": "string",
"optional": true,
"name": "io.debezium.data.Json",
"version": 1,
"field": "patch"
},
{
"type": "string",
"optional": true,
"name": "io.debezium.data.Json",
"version": 1,
"field": "filter"
},
{
"type": "struct",
"fields": [
{
"type": "string",
"optional": false,
"field": "version"
},
{
"type": "string",
"optional": false,
"field": "connector"
},
{
"type": "string",
"optional": false,
"field": "name"
},
{
"type": "int64",
"optional": false,
"field": "ts_sec"
},
{
"type": "boolean",
"optional": true,
"default": false,
"field": "snapshot"
},
{
"type": "string",
"optional": false,
"field": "db"
},
{
"type": "string",
"optional": false,
"field": "rs"
},
{
"type": "string",
"optional": false,
"field": "collection"
},
{
"type": "int32",
"optional": false,
"field": "ord"
},
{
"type": "int64",
"optional": true,
"field": "h"
}
],
"optional": false,
"name": "io.debezium.connector.mongo.Source", (3)
"field": "source"
},
{
"type": "string",
"optional": true,
"field": "op"
},
{
"type": "int64",
"optional": true,
"field": "ts_ms"
}
],
"optional": false,
"name": "dbserver1.inventory.customers.Envelope" (4)
},
"payload": { (5)
"after": "{\"_id\" : {\"$numberLong\" : \"1004\"},\"first_name\" : \"Anne\",\"last_name\" : \"Kretchmar\",\"email\" : \"annek@noanswer.org\"}", (6)
"patch": null,
"source": { (7)
"version": "1.3.1.Final",
"connector": "mongodb",
"name": "fulfillment",
"ts_sec": 1558965508000,
"snapshot": false,
"db": "inventory",
"rs": "rs0",
"collection": "customers",
"ord": 31,
"h": 1546547425148721999
},
"op": "c", (8)
"ts_ms": 1558965515240 (9)
}
}
Item | Field name | Description |
---|---|---|
1 |
| The value’s schema, which describes the structure of the value’s payload. A change event’s value schema is the same in every change event that the connector generates for a particular collection. |
2 |
| In the |
3 |
|
|
4 |
|
|
5 |
| The value’s actual data. This is the information that the change event is providing. |
6 |
| An optional field that specifies the state of the document after the event occurred. In this example, the |
7 |
| Mandatory field that describes the source metadata for the event. This field contains information that you can use to compare this event with other events, with regard to the origin of the events, the order in which the events occurred, and whether events were part of the same transaction. The source metadata includes:
|
8 |
| Mandatory string that describes the type of operation that caused the connector to generate the event. In this example,
|
9 |
| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
update events
The value of a change event for an update in the sample customers
collection has the same schema as a create event for that collection. Likewise, the event value’s payload has the same structure. However, the event value payload contains different values in an update event. An update event does not have an after
value. Instead, it has these two fields:
patch
is a string field that contains the JSON representation of the idempotent update operationfilter
is a string field that contains the JSON representation of the selection criteria for the update. Thefilter
string can include multiple shard key fields for sharded collections.
Here is an example of a change event value in an event that the connector generates for an update in the customers
collection:
{
"schema": { ... },
"payload": {
"op": "u", (1)
"ts_ms": 1465491461815, (2)
"patch": "{\"$set\":{\"first_name\":\"Anne Marie\"}}", (3)
"filter": "{\"_id\" : {\"$numberLong\" : \"1004\"}}", (4)
"source": { (5)
"version": "1.3.1.Final",
"connector": "mongodb",
"name": "fulfillment",
"ts_sec": 1558965508000,
"snapshot": true,
"db": "inventory",
"rs": "rs0",
"collection": "customers",
"ord": 6,
"h": 1546547425148721999
}
}
}
Item | Field name | Description |
---|---|---|
1 |
| Mandatory string that describes the type of operation that caused the connector to generate the event. In this example, |
2 |
| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
3 |
| Contains the JSON string representation of the actual MongoDB idempotent change to the document. In this example, the update changed the |
4 |
| Contains the JSON string representation of the MongoDB selection criteria that was used to identify the document to be updated. |
5 |
| Mandatory field that describes the source metadata for the event. This field contains the same information as a create event for the same collection, but the values are different since this event is from a different position in the oplog. The source metadata includes:
|
In a Debezium change event, MongoDB provides the content of the |
In MongoDB’s oplog, update events do not contain the before or after states of the changed document. Consequently, it is not possible for a Debezium connector to provide this information. However, a Debezium connector provides a document’s starting state in create and read events. Downstream consumers of the stream can reconstruct document state by keeping the latest state for each document and comparing the state in a new event with the saved state. Debezium connector’s are not able to keep this state. |
delete events
The value in a delete change event has the same schema
portion as create and update events for the same collection. The payload
portion in a delete event contains values that are different from create and update events for the same collection. In particular, a delete event contains neither an after
value nor a patch
value. Here is an example of a delete event for a document in the customers
collection:
{
"schema": { ... },
"payload": {
"op": "d", (1)
"ts_ms": 1465495462115, (2)
"filter": "{\"_id\" : {\"$numberLong\" : \"1004\"}}", (3)
"source": { (4)
"version": "1.3.1.Final",
"connector": "mongodb",
"name": "fulfillment",
"ts_sec": 1558965508000,
"snapshot": true,
"db": "inventory",
"rs": "rs0",
"collection": "customers",
"ord": 6,
"h": 1546547425148721999
}
}
}
Item | Field name | Description |
---|---|---|
1 |
| Mandatory string that describes the type of operation. The |
2 |
| Optional field that displays the time at which the connector processed the event. The time is based on the system clock in the JVM running the Kafka Connect task. |
3 |
| Contains the JSON string representation of the MongoDB selection criteria that was used to identify the document to be deleted. |
4 |
| Mandatory field that describes the source metadata for the event. This field contains the same information as a create or update event for the same collection, but the values are different since this event is from a different position in the oplog. The source metadata includes:
|
MongoDB connector events are designed to work with Kafka log compaction. Log compaction enables removal of some older messages as long as at least the most recent message for every key is kept. This lets Kafka reclaim storage space while ensuring that the topic contains a complete data set and can be used for reloading key-based state.
Tombstone events
All MongoDB connector events for a uniquely identified document have exactly the same key. When a document is deleted, the delete event value still works with log compaction because Kafka can remove all earlier messages that have that same key. However, for Kafka to remove all messages that have that key, the message value must be null
. To make this possible, after Debezium’s MongoDB connector emits a delete event, the connector emits a special tombstone event that has the same key but a null
value. A tombstone event informs Kafka that all messages with that same key can be removed.
Transaction Metadata
Debezium can generate events that represents tranaction metadata boundaries and enrich data messages.
Transaction boundaries
Debezium generates events for every transaction BEGIN
and END
. Every event contains
status
-BEGIN
orEND
id
- string representation of unique transaction identifierevent_count
(forEND
events) - total number of events emmitted by the transactiondata_collections
(forEND
events) - an array of pairs ofdata_collection
andevent_count
that provides number of events emitted by changes originating from given data collection
Following is an example of what a message looks like:
{
"status": "BEGIN",
"id": "1462833718356672513",
"event_count": null,
"data_collections": null
}
{
"status": "END",
"id": "1462833718356672513",
"event_count": 2,
"data_collections": [
{
"data_collection": "rs0.testDB.tablea",
"event_count": 1
},
{
"data_collection": "rs0.testDB.tableb",
"event_count": 1
}
]
}
The transaction events are written to the topic named <database.server.name>.transaction
.
Data events enrichment
When transaction metadata is enabled the data message Envelope
is enriched with a new transaction
field. This field provides information about every event in the form of a composite of fields:
id
- string representation of unique transaction identifiertotal_order
- the absolute position of the event among all events generated by the transactiondata_collection_order
- the per-data collection position of the event among all events that were emitted by the transaction
Following is an example of what a message looks like:
{
"before": null,
"after": {
"pk": "2",
"aa": "1"
},
"source": {
...
},
"op": "c",
"ts_ms": "1580390884335",
"transaction": {
"id": "1462833718356672513",
"total_order": "1",
"data_collection_order": "1"
}
}
Deploying the MongoDB connector
With Zookeeper, Kafka, and Kafka Connect installed, the remaining tasks to deploy a Debezium MongoDB connector are to download the connector’s plug-in archive, extract the JAR files into your Kafka Connect environment, and add the directory with the JAR files to Kafka Connect’s plugin.path. You then need to restart your Kafka Connect process to pick up the new JAR files.
If you are working with immutable containers, see Debezium’s Container images for Zookeeper, Kafka, and Kafka Connect with the MongoDB connector already installed and ready to run. You can also run Debezium on Kubernetes and OpenShift.
The Debezium tutorial walks you through using these images, and this is a great way to learn about Debezium.
Example configuration
To use the connector to produce change events for a particular MongoDB replica set or sharded cluster, create a configuration file in JSON. When the connector starts, it will perform a snapshot of the collections in your MongoDB replica sets and start reading the replica sets’ oplogs, producing events for every inserted, updated, and deleted document. Optionally filter out collections that are not needed.
Following is an example of the configuration for a MongoDB connector that monitors a MongoDB replica set rs0
at port 27017 on 192.168.99.100, which we logically name fullfillment
. Typically, you configure the Debezium MongoDB connector in a .json
file using the configuration properties available for the connector.
{
"name": "inventory-connector", (1)
"config": {
"connector.class": "io.debezium.connector.mongodb.MongoDbConnector", (2)
"mongodb.hosts": "rs0/192.168.99.100:27017", (3)
"mongodb.name": "fullfillment", (4)
"collection.include.list": "inventory[.]*", (5)
}
}
1 | The name of our connector when we register it with a Kafka Connect service. |
2 | The name of the MongoDB connector class. |
3 | The host addresses to use to connect to the MongoDB replica set. |
4 | The logical name of the MongoDB replica set, which forms a namespace for generated events and is used in all the names of the Kafka topics to which the connector writes, the Kafka Connect schema names, and the namespaces of the corresponding Avro schema when the Avro converter is used. |
5 | A list of regular expressions that match the collection namespaces (for example, <dbName>.<collectionName>) of all collections to be monitored. This is optional. |
See the complete list of connector properties that can be specified in these configurations.
This configuration can be sent via POST to a running Kafka Connect service, which will then record the configuration and start up the one connector task that will connect to the MongoDB replica set or sharded cluster, assign tasks for each replica set, perform a snapshot if necessary, read the oplog, and record events to Kafka topics.
Adding connector configuration
To run a Debezium MongoDB connector, create a connector configuration and add the configuration to your Kafka Connect cluster.
Prerequisites
MongoDB is set up to run a Debezium connector.
A Debezium MongoDB connector is installed.
Procedure
Create a configuration for the MongoDB connector.
Use the Kafka Connect REST API to add that connector configuration to your Kafka Connect cluster.
Results
When the connector starts, it performs a consistent snapshot of the MongoDB databases that the connector is configured for. The connector then starts generating data change events for document-level operations and streaming change event records to Kafka topics.
Monitoring
The Debezium MongoDB connector has two metric types in addition to the built-in support for JMX metrics that Zookeeper, Kafka, and Kafka Connect have.
snapshot metrics; for monitoring the connector when performing snapshots
streaming metrics; for monitoring the connector when processing oplog events
Please refer to the monitoring documentation for details of how to expose these metrics via JMX.
Snapshot Metrics
The MBean is debezium.mongodb:type=connector-metrics,context=snapshot,server=*<mongodb.name>*
.
Attributes | Type | Description |
---|---|---|
| The last snapshot event that the connector has read. | |
| The number of milliseconds since the connector has read and processed the most recent event. | |
| The total number of events that this connector has seen since last started or reset. | |
| The number of events that have been filtered by include/exclude list filtering rules configured on the connector. | |
| The list of tables that are monitored by the connector. | |
| The length the queue used to pass events between the snapshotter and the main Kafka Connect loop. | |
| The free capacity of the queue used to pass events between the snapshotter and the main Kafka Connect loop. | |
| The total number of tables that are being included in the snapshot. | |
| The number of tables that the snapshot has yet to copy. | |
| Whether the snapshot was started. | |
| Whether the snapshot was aborted. | |
| Whether the snapshot completed. | |
| The total number of seconds that the snapshot has taken so far, even if not complete. | |
| Map containing the number of rows scanned for each table in the snapshot. Tables are incrementally added to the Map during processing. Updates every 10,000 rows scanned and upon completing a table. |
The Debezium MongoDB connector also provides the following custom snapshot metrics:
Attribute | Type | Description |
---|---|---|
|
| Number of database disconnects. |
Streaming Metrics
The MBean is debezium.sql_server:type=connector-metrics,context=streaming,server=*<mongodb.name>*
.
Attributes | Type | Description |
---|---|---|
| The last streaming event that the connector has read. | |
| The number of milliseconds since the connector has read and processed the most recent event. | |
| The total number of events that this connector has seen since last started or reset. | |
| The number of events that have been filtered by include/exclude list filtering rules configured on the connector. | |
| The list of tables that are monitored by the connector. | |
| The length the queue used to pass events between the streamer and the main Kafka Connect loop. | |
| The free capacity of the queue used to pass events between the streamer and the main Kafka Connect loop. | |
| Flag that denotes whether the connector is currently connected to the database server. | |
| The number of milliseconds between the last change event’s timestamp and the connector processing it. The values will incoporate any differences between the clocks on the machines where the database server and the connector are running. | |
| The number of processed transactions that were committed. | |
| The coordinates of the last received event. | |
| Transaction identifier of the last processed transaction. |
The Debezium MongoDB connector also provides the following custom streaming metrics:
Attribute | Type | Description |
---|---|---|
|
| Number of database disconnects. |
|
| Number of primary node elections. |
Connector properties
The following configuration properties are required unless a default value is available.
Property | Default | Description |
---|---|---|
Unique name for the connector. Attempting to register again with the same name will fail. (This property is required by all Kafka Connect connectors.) | ||
The name of the Java class for the connector. Always use a value of | ||
The comma-separated list of hostname and port pairs (in the form ‘host’ or ‘host:port’) of the MongoDB servers in the replica set. The list can contain a single hostname and port pair. If | ||
A unique name that identifies the connector and/or MongoDB replica set or sharded cluster that this connector monitors. Each server should be monitored by at most one Debezium connector, since this server name prefixes all persisted Kafka topics emanating from the MongoDB replica set or cluster. Only alphanumeric characters and underscores should be used. | ||
Name of the database user to be used when connecting to MongoDB. This is required only when MongoDB is configured to use authentication. | ||
Password to be used when connecting to MongoDB. This is required only when MongoDB is configured to use authentication. | ||
| Database (authentication source) containing MongoDB credentials. This is required only when MongoDB is configured to use authentication with another authentication database than | |
| Connector will use SSL to connect to MongoDB instances. | |
| When SSL is enabled this setting controls whether strict hostname checking is disabled during connection phase. If | |
empty string | An optional comma-separated list of regular expressions that match database names to be monitored; any database name not included in | |
empty string | An optional comma-separated list of regular expressions that match database names to be excluded from monitoring; any database name not included in | |
empty string | An optional comma-separated list of regular expressions that match fully-qualified namespaces for MongoDB collections to be monitored; any collection not included in | |
empty string | An optional comma-separated list of regular expressions that match fully-qualified namespaces for MongoDB collections to be excluded from monitoring; any collection not included in | |
| Specifies the criteria for running a snapshot upon startup of the connector. The default is initial, and specifies the connector reads a snapshot when either no offset is found or if the oplog no longer contains the previous offset. The never option specifies that the connector should never use snapshots, instead the connector should proceed to tail the log. | |
empty string | An optional comma-separated list of the fully-qualified names of fields that should be excluded from change event message values. Fully-qualified names for fields are of the form databaseName.collectionName.fieldName.nestedFieldName, where databaseName and collectionName may contain the wildcard () which matches any characters. | |
empty string | An optional comma-separated list of the fully-qualified replacements of fields that should be used to rename fields in change event message values. Fully-qualified replacements for fields are of the form databaseName.collectionName.fieldName.nestedFieldName:newNestedFieldName, where databaseName and collectionName may contain the wildcard () which matches any characters, the colon character (:) is used to determine rename mapping of field. The next field replacement is applied to the result of the previous field replacement in the list, so keep this in mind when renaming multiple fields that are in the same path. | |
| The maximum number of tasks that should be created for this connector. The MongoDB connector will attempt to use a separate task for each replica set, so the default is acceptable when using the connector with a single MongoDB replica set. When using the connector with a MongoDB sharded cluster, we recommend specifying a value that is equal to or more than the number of shards in the cluster, so that the work for each replica set can be distributed by Kafka Connect. | |
| Positive integer value that specifies the maximum number of threads used to perform an intial sync of the collections in a replica set. Defaults to 1. | |
| Controls whether a tombstone event should be generated after a delete event. | |
An interval in milliseconds that the connector should wait before taking a snapshot after starting up; | ||
| Specifies the maximum number of documents that should be read in one go from each collection while taking a snapshot. The connector will read the collection contents in multiple batches of this size. |
The following advanced configuration properties have good defaults that will work in most situations and therefore rarely need to be specified in the connector’s configuration.
Property | Default | Description |
---|---|---|
| Positive integer value that specifies the maximum size of the blocking queue into which change events read from the database log are placed before they are written to Kafka. This queue can provide backpressure to the oplog reader when, for example, writes to Kafka are slower or if Kafka is not available. Events that appear in the queue are not included in the offsets periodically recorded by this connector. Defaults to 8192, and should always be larger than the maximum batch size specified in the | |
| Positive integer value that specifies the maximum size of each batch of events that should be processed during each iteration of this connector. Defaults to 2048. | |
| Positive integer value that specifies the number of milliseconds the connector should wait during each iteration for new change events to appear. Defaults to 1000 milliseconds, or 1 second. | |
| Positive integer value that specifies the initial delay when trying to reconnect to a primary after the first failed connection attempt or when no primary is available. Defaults to 1 second (1000 ms). | |
| Positive integer value that specifies the maximum delay when trying to reconnect to a primary after repeated failed connection attempts or when no primary is available. Defaults to 120 seconds (120,000 ms). | |
| Positive integer value that specifies the maximum number of failed connection attempts to a replica set primary before an exception occurs and task is aborted. Defaults to 16, which with the defaults for | |
| Boolean value that specifies whether the addresses in ‘mongodb.hosts’ are seeds that should be used to discover all members of the cluster or replica set ( | |
v2 | Schema version for the | |
| Controls how frequently heartbeat messages are sent. Set this parameter to | |
| Controls the naming of the topic to which heartbeat messages are sent. | |
| Whether field names are sanitized to adhere to Avro naming requirements. See Avro naming for more details. | |
comma-separated list of oplog operations that will be skipped during streaming. The operations include: | ||
| When set to See Transaction Metadata for additional details. | |
10000 (10 seconds) | The number of milliseconds to wait before restarting a connector after a retriable error occurs. | |
| The interval in which the connector polls for new, removed, or changed replica sets. | |
10000 (10 seconds) | The number of milliseconds the driver will wait before a new connection attempt is aborted. | |
0 | The number of milliseconds before a send/receive on the socket can take before a timeout occurs. A value of | |
30000 (30 seconds) | The number of milliseconds the driver will wait to select a server before it times out and throws an error. |
MongoDB connector common issues
Debezium is a distributed system that captures all changes in multiple upstream databases, and will never miss or lose an event. Of course, when the system is operating nominally or being administered carefully, then Debezium provides exactly once delivery of every change event. However, if a fault does happen then the system will still not lose any events, although while it is recovering from the fault it may repeat some change events. Thus, in these abnormal situations Debezium (like Kafka) provides at least once delivery of change events.
The rest of this section describes how Debezium handles various kinds of faults and problems.
Configuration and startup errors
The connector will fail upon startup, report an error/exception in the log, and stop running when the connector’s configuration is invalid, or when the connector repeatedly fails to connect to MongoDB using the specified connectivity parameters. Reconnection is done using exponential backoff, and the maximum number of attempts is configurable.
In these cases, the error will have more details about the problem and possibly a suggested work around. The connector can be restarted when the configuration has been corrected or the MongoDB problem has been addressed.
MongoDB becomes unavailable
Once the connector is running, if the primary node of any of the MongoDB replica sets become unavailable or unreachable, the connector will repeatedly attempt to reconnect to the primary node, using exponential backoff to prevent saturating the network or servers. If the primary remains unavailable after the configurable number of connection attempts, the connector will fail.
The attempts to reconnect are controlled by three properties:
connect.backoff.initial.delay.ms
- The delay before attempting to reconnect for the first time, with a default of 1 second (1000 milliseconds).connect.backoff.max.delay.ms
- The maximum delay before attempting to reconnect, with a default of 120 seconds (120,000 milliseconds).connect.max.attempts
- The maximum number of attempts before an error is produced, with a default of 16.
Each delay is double that of the prior delay, up to the maximum delay. Given the default values, the following table shows the delay for each failed connection attempt and the total accumulated time before failure.
Reconnection attempt number | Delay before attempt, in seconds | Total delay before attempt, in minutes and seconds |
---|---|---|
1 | 1 | 00:01 |
2 | 2 | 00:03 |
3 | 4 | 00:07 |
4 | 8 | 00:15 |
5 | 16 | 00:31 |
6 | 32 | 01:03 |
7 | 64 | 02:07 |
8 | 120 | 04:07 |
9 | 120 | 06:07 |
10 | 120 | 08:07 |
11 | 120 | 10:07 |
12 | 120 | 12:07 |
13 | 120 | 14:07 |
14 | 120 | 16:07 |
15 | 120 | 18:07 |
16 | 120 | 20:07 |
Kafka Connect process stops gracefully
If Kafka Connect is being run in distributed mode, and a Kafka Connect process is stopped gracefully, then prior to shutdown of that processes Kafka Connect will migrate all of the process’ connector tasks to another Kafka Connect process in that group, and the new connector tasks will pick up exactly where the prior tasks left off. There is a short delay in processing while the connector tasks are stopped gracefully and restarted on the new processes.
If the group contains only one process and that process is stopped gracefully, then Kafka Connect will stop the connector and record the last offset for each replica set. Upon restart, the replica set tasks will continue exactly where they left off.
Kafka Connect process crashes
If the Kafka Connector process stops unexpectedly, then any connector tasks it was running will terminate without recording their most recently-processed offsets. When Kafka Connect is being run in distributed mode, it will restart those connector tasks on other processes. However, the MongoDB connectors will resume from the last offset recorded by the earlier processes, which means that the new replacement tasks may generate some of the same change events that were processed just prior to the crash. The number of duplicate events depends on the offset flush period and the volume of data changes just before the crash.
Because there is a chance that some events may be duplicated during a recovery from failure, consumers should always anticipate some events may be duplicated. Debezium changes are idempotent, so a sequence of events always results in the same state. Debezium also includes with each change event message the source-specific information about the origin of the event, including the MongoDB event’s unique transaction identifier ( |
Kafka becomes unavailable
As the connector generates change events, the Kafka Connect framework records those events in Kafka using the Kafka producer API. Kafka Connect will also periodically record the latest offset that appears in those change events, at a frequency that you have specified in the Kafka Connect worker configuration. If the Kafka brokers become unavailable, the Kafka Connect worker process running the connectors will simply repeatedly attempt to reconnect to the Kafka brokers. In other words, the connector tasks will simply pause until a connection can be reestablished, at which point the connectors will resume exactly where they left off.
Connector is stopped for a duration
If the connector is gracefully stopped, the replica sets can continue to be used and any new changes are recorded in MongoDB’s oplog. When the connector is restarted, it will resume streaming changes for each replica set where it last left off, recording change events for all of the changes that were made while the connector was stopped. If the connector is stopped long enough such that MongoDB purges from its oplog some operations that the connector has not read, then upon startup the connector will perform a snapshot.
A properly configured Kafka cluster is capable of massive throughput. Kafka Connect is written with Kafka best practices, and given enough resources will also be able to handle very large numbers of database change events. Because of this, when a connector has been restarted after a while, it is very likely to catch up with the database, though how quickly will depend upon the capabilities and performance of Kafka and the volume of changes being made to the data in MongoDB.
If the connector remains stopped for long enough, MongoDB might purge older oplog files and the connector’s last position may be lost. In this case, when the connector configured with initial snapshot mode (the default) is finally restarted, the MongoDB server will no longer have the starting point and the connector will fail with an error. |
MongoDB loses writes
It is possible for MongoDB to lose commits in specific failure situations. For example, if the primary applies a change and records it in its oplog before it then crashes unexpectedly, the secondary nodes may not have had a chance to read those changes from the primary’s oplog before the primary crashed. If one such secondary is then elected as primary, its oplog is missing the last changes that the old primary had recorded and no longer has those changes.
In these cases where MongoDB loses changes recorded in a primary’s oplog, it is possible that the MongoDB connector may or may not capture these lost changes. At this time, there is no way to prevent this side effect of MongoDB.