The MongoDB CDC connector allows for reading snapshot data and incremental data from MongoDB. This document describes how to setup the MongoDB CDC connector to run SQL queries against MongoDB.
Dependencies
------------
In order to setup the MongoDB CDC connector, the following table provides dependency information for both projects using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR bundles.
Download [flink-sql-connector-mongodb-cdc-2.3.0.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-mongodb-cdc/2.3.0/flink-sql-connector-mongodb-cdc-2.3.0.jar) and put it under `<FLINK_HOME>/lib/`.
**Note:** flink-sql-connector-mongodb-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-mongodb-cdc-2.3.0.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-mongodb-cdc), the released version will be available in the Maven central warehouse.
We use [change streams](https://docs.mongodb.com/manual/changeStreams/) feature (new in version 3.6) to capture change data.
- Cluster Deployment
[replica sets](https://docs.mongodb.com/manual/replication/) or [sharded clusters](https://docs.mongodb.com/manual/sharding/) is required.
- Storage Engine
[WiredTiger](https://docs.mongodb.com/manual/core/wiredtiger/#std-label-storage-wiredtiger) storage engine is required.
- [Replica set protocol version](https://docs.mongodb.com/manual/reference/replica-configuration/#mongodb-rsconf-rsconf.protocolVersion)
Replica set protocol version 1 [(pv1)](https://docs.mongodb.com/manual/reference/replica-configuration/#mongodb-rsconf-rsconf.protocolVersion) is required. <br>
Starting in version 4.0, MongoDB only supports pv1. pv1 is the default for all new replica sets created with MongoDB 3.2 or later.
- Privileges
`changeStream` and `read` privileges are required by MongoDB Kafka Connector.
You can use the following example for simple authorization.<br>
For more detailed authorization, please refer to [MongoDB Database User Roles](https://docs.mongodb.com/manual/reference/built-in-roles/#database-user-roles).
Note: `heartbeat.interval.ms` is highly recommended setting a proper value larger than 0 **if the collection changes slowly**.
The heartbeat event can push the `resumeToken` forward to avoid `resumeToken` being expired when we recover the Flink job from a checkpoint or savepoint.
<td>Name of the database that contain the row.</td>
</tr>
<tr>
<td>collection_name</td>
<td>STRING NOT NULL</td>
<td>Name of the collection that contain the row.</td>
</tr>
<tr>
<td>op_ts</td>
<td>TIMESTAMP_LTZ(3) NOT NULL</td>
<td>It indicates the time that the change was made in the database. <br>If the record is read from snapshot of the table instead of the change stream, the value is always 0.</td>
The MongoDB CDC connector is a Flink Source connector which will read database snapshot first and then continues to read change stream events with **exactly-once processing** even failures happen.
### Snapshot When Startup Or Not
The config option `copy.existing` specifies whether do snapshot when MongoDB CDC consumer startup. <br>Defaults to `true`.
### Snapshot Data Filters
The config option `copy.existing.pipeline` describing the filters when copying existing data.<br>
This can filter only required data and improve the use of indexes by the copying manager.
In the following example, the `$match` aggregation operator ensures that only documents in which the closed field is set to false are copied.
We integrate the [MongoDB's official Kafka Connector](https://docs.mongodb.com/kafka-connector/current/kafka-source/) to read snapshot or change events from MongoDB and drive it by Debezium's `EmbeddedEngine`.
Debezium's `EmbeddedEngine` provides a mechanism for running a single Kafka Connect `SourceConnector` within an application's process, and it can drive any standard Kafka Connect `SourceConnector` properly even which is not provided by Debezium.
We choose **MongoDB's official Kafka Connector** instead of the **Debezium's MongoDB Connector** because they use a different change data capture mechanism.
MongoDB's `oplog.rs` collection doesn't keep the changed record's update before state, so it's hard to extract the full document state by a single `oplog.rs` record and convert it to change log stream accepted by Flink (Insert Only, Upsert, All).
Additionally, MongoDB 5 (released in July 2021) has changed the oplog format, so the current Debezium connector cannot be used with it.
**Change Stream** is a new feature provided by MongoDB 3.6 for replica sets and sharded clusters that allows applications to access real-time data changes without the complexity and risk of tailing the oplog.<br>
Applications can use change streams to subscribe to all data changes on a single collection, a database, or an entire deployment, and immediately react to them.
**Lookup Full Document for Update Operations** is a feature provided by **Change Stream** which can configure the change stream to return the most current majority-committed version of the updated document. Because of this feature, we can easily collect the latest full document and convert the change log to Flink's **Upsert Changelog Stream**.
By the way, Debezium's MongoDB change streams exploration mentioned by [DBZ-435](https://issues.redhat.com/browse/DBZ-435) is on roadmap.<br>
If it's done, we can consider integrating two kinds of source connector for users to choose.
### DataStream Source
The MongoDB CDC connector can also be a DataStream source. You can create a SourceFunction as the following shows:
[BSON](https://docs.mongodb.com/manual/reference/bson-types/) short for **Binary JSON** is a binary-encoded serialization of JSON-like format used to store documents and make remote procedure calls in MongoDB.
[Flink SQL Data Type](https://nightlies.apache.org/flink/flink-docs-release-1.16/docs/dev/table/types/) is similar to the SQL standard’s data type terminology which describes the logical type of a value in the table ecosystem. It can be used to declare input and/or output types of operations.
In order to enable Flink SQL to process data from heterogeneous data sources, the data types of heterogeneous data sources need to be uniformly converted to Flink SQL data types.
The following is the mapping of BSON type and Flink SQL type.