DBZ-2 Created initial Maven module with a MongoDB connector

Added a new `debezium-connector-mongodb` module that defines a MongoDB connector. The MongoDB connector can capture and record the changes within a MongoDB replica set, or when seeded with addresses of the configuration server of a MongoDB sharded cluster, the connector captures the changes from the each replica set used as a shard. In the latter case, the connector even discovers the addition of or removal of shards.

The connector monitors each replica set using multiple tasks and, if needed, separate threads within each task. When a replica set is being monitored for the first time, the connector will perform an "initial sync" of that replica set's databases and collections. Once the initial sync has completed, the connector will then begin tailing the oplog of the replica set, starting at the exact point in time at which it started the initial sync. This equivalent to how MongoDB replication works.

The connector always uses the replica set's primary node to tail the oplog. If the replica set undergoes an election and different node becomes primary, the connector will immediately stop tailing the oplog, connect to the new primary, and start tailing the oplog using the new primary node. Likewise, if connector experiences any problems communicating with the replica set members, it will try to reconnect (using exponential backoff so as to not overwhelm the replica set) and continue tailing the oplog 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.

The MongoDB oplog contains limited information, and in particular the events describing updates and deletes do not actually have the before or after state of the documents. Instead, the oplog events are all idempotent, so updates contain the effective changes that were made during an update, and deletes merely contain the deleted document identifier. Consequently, the connector is limited in the information it includes in its output events. Create and read events do contain the initial state, but the update contain only the changes (rather than the before and/or after states of the document) and delete events do not have the before state of the deleted document. All connector events, however, do contain the local system timestamp at which the event was processed and _source_ information detailing the origins of the event, including the replica set name, the MongoDB transaction timestamp of the event, and the transactions identifier among other things.

It is possible for MongoDB to lose commits in specific failure situations. For exmaple, 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, it's 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.
This commit is contained in:
Randall Hauch 2016-04-19 15:49:58 -05:00
parent e71a74f4fa
commit 12e7cfb8d3
54 changed files with 6346 additions and 130 deletions

View File

@ -0,0 +1,49 @@
This module builds and runs two containers based upon the [mongo:3.2](https://hub.docker.com/_/mongo/) Docker image. The first _primary_ container starts MongoDB, while the second _initiator_ container initializes the replica set and then terminates.
## Using MongoDB
As mentioned in the [README.md]() file, our Maven build can be used to start a container using either one of these images. The `mongo:3.2` image is used:
$ mvn docker:start
The command leaves the primary container running so that you can use the running MySQL server. For example, you can establish a `bash` shell inside the container (named `mongo1`) by using Docker in another terminal:
$ docker exec -it mongo1 bash
Or you can run integration tests from your IDE, as described in detail in the [README.md]() file.
To stop and remove the `mongo1` container, simply use the following Maven command:
$ mvn docker:stop
or use the following Docker commands:
$ docker stop mongo1
$ docker rm mongo1
## Using Docker directly
Although using the Maven command is far simpler, the Maven commands really just run for the `alt-server` profile really just runs (via the Jolokia Maven plugin) a Docker command to start the container, so it's equivalent to:
$ docker run -it --rm --name mongo mongo:latest --replSet rs0 --oplogSize=2 --enableMajorityReadConcern
This will use the `mongo:3.2` image to start a new container named `mongo`. This can be repeated multiple times to start multiple MongoDB secondary nodes:
$ docker run -it --rm --name mongo1 mongo:latest --replSet rs0 --oplogSize=2 --enableMajorityReadConcern
$ docker run -it --rm --name mongo2 mongo:latest --replSet rs0 --oplogSize=2 --enableMajorityReadConcern
Then, run the initiator container to initialize the replica set by assigning the `mongo` container as primary and the other containers as secondary nodes:
$ docker run -it --rm --name mongoinit --link mongo:mongo --link mongo1:mongo1 --link mongo2:mongo2 -e REPLICASET=rs0 -e debezium/mongo-replicaset-initiator:3.2
Once the replica set is initialized, the `mongoinit` container will complete and be removed.
### Use MongoDB client
The following command can be used to manually start up a Docker container to run the MongoDB command line client:
$ docker run -it --link mongo:mongo --rm mongo:3.2 sh -c 'exec mongo "$MONGO_PORT_27017_TCP_ADDR:$MONGO_PORT_27017_TCP_PORT"'
Note that it must be linked to the Mongo container to which it will connect.

View File

@ -0,0 +1,82 @@
# Ingesting MongoDB change events
This module defines the connector that ingests change events from MongoDB databases.
## Using the MongoDB connector with Kafka Connect
The MongoDB connector is designed to work with [Kafka Connect](http://kafka.apache.org/documentation.html#connect) and to be deployed to a Kafka Connect runtime service. The deployed connector will monitor one or more databases and write all change events to Kafka topics, which can be independently consumed by one or more clients. Kafka Connect can be distributed to provide fault tolerance to ensure the connectors are running and continually keeping up with changes in the database.
Kafka Connect can also be run standalone as a single process, although doing so is not tolerant of failures.
## Embedding the MongoDB connector
The MongoDB connector can also be used as a library without Kafka or Kafka Connect, enabling applications and services to directly connect to a MongoDB database and obtain the ordered change events. This approach requires the application to record the progress of the connector so that upon restart the connect can continue where it left off. Therefore, this may be a useful approach for less critical use cases. For production use cases, we highly recommend using this connector with Kafka and Kafka Connect.
## Testing
This module contains both unit tests and integration tests.
A *unit test* is a JUnit test class named `*Test.java` or `Test*.java` that never requires or uses external services, though it can use the file system and can run any components within the same JVM process. They should run very quickly, be independent of each other, and clean up after itself.
An *integration test* is a JUnit test class named `*IT.java` or `IT*.java` that uses a MongoDB database server running in a Docker container based upon the [mongo:3.2](https://hub.docker.com/_/mongo/) Docker image maintained by the Docker team. The build will automatically start the MongoDB container before the integration tests are run and automatically stop and remove it after all of the integration tests complete (regardless of whether they suceed or fail). Multiple test methods within a single integration test class can reuse the same database, but generally each integration test class should use its own dedicated database(s).
Running `mvn install` will compile all code and run the unit tests. If there are any compile problems or any of the unit tests fail, the build will stop immediately. Otherwise, the command will continue to create the module's artifacts, start the Docker container, run the integration tests, stop the container (even if there are integration test failures), and run checkstyle on the code. If there are still no problems, the build will then install the module's artifacts into the local Maven repository.
You should always default to using `mvn install`, especially prior to committing changes to Git. However, there are a few situations where you may want to run a different Maven command.
### Running some tests
If you are trying to get the test methods in a single integration test class to pass and would rather not run *all* of the integration tests, you can instruct Maven to just run that one integration test class and to skip all of the others. For example, use the following command to run the tests in the `ConnectionIT.java` class:
$ mvn -Dit.test=ConnectionIT install
Of course, wildcards also work:
$ mvn -Dit.test=Connect*IT install
These commands will automatically manage the MongoDB Docker container.
### Debugging tests
If you want to debug integration tests by stepping through them in your IDE, using the `mvn install` command will be problematic since it will not wait for your IDE's breakpoints. There are ways of doing this, but it is typically far easier to simply start the Docker container and leave it running so that it is available when you run the integration test(s). The following command:
$ mvn docker:start
will start the default MongoDB container and run the database server. Now you can use your IDE to run/debug one or more integration tests. Just be sure that the integration tests clean up their database before (and after) each test, and that you run the tests with VM arguments that define the required system properties including:
* `database.dbname` - the name of the database that your integration test will use; there is no default
* `database.hostname` - the IP address or name of the host where the Docker container is running; defaults to `localhost` which is likely for Linux, but on OS X and Windows Docker it will have to be set to the IP address of the VM that runs Docker (which you can find by looking at the `DOCKER_HOST` environment variable).
* `database.port` - the port on which MongoDB is listening; defaults to `27017` and is what this module's Docker container uses
For example, you can define these properties by passing these arguments to the JVM:
-Ddatabase.dbname=<DATABASE_NAME> -Ddatabase.hostname=<DOCKER_HOST> -Ddatabase.port=27017
When you are finished running the integration tests from your IDE, you have to stop and remove the Docker container (conveniently named "database") before you can run the next build:
$ mvn docker:stop
or using Docker directly:
$ docker stop database
$ docker rm database
### Analyzing the database
Sometimes you may want to inspect the state of the database(s) after one or more integration tests are run. The `mvn install` command runs the tests but shuts down and removes the container after the integration tests complete. To keep the container running after the integration tests complete, use this Maven command:
$ mvn integration-test
### Stopping the Docker container
This instructs Maven to run the normal Maven lifecycle through `integration-test`, and to stop before the `post-integration-test` phase when the Docker container is normally shut down and removed. Be aware that you will need to manually stop and remove the container (conveniently named "database") before running the build again:
$ mvn docker:stop
or using Docker directly:
$ docker stop database
$ docker rm database

View File

@ -0,0 +1,287 @@
<?xml version="1.0"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<groupId>io.debezium</groupId>
<artifactId>debezium-parent</artifactId>
<version>0.3-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>debezium-connector-mongodb</artifactId>
<name>Debezium Connector for MongoDB</name>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>connect-api</artifactId>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</dependency>
<dependency>
<groupId>org.mongodb</groupId>
<artifactId>mongodb-driver</artifactId>
</dependency>
<!-- Testing -->
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-core</artifactId>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-embedded</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-embedded</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</dependency>
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
</dependency>
<dependency>
<groupId>org.easytesting</groupId>
<artifactId>fest-assert</artifactId>
</dependency>
<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-connect-avro-converter</artifactId>
</dependency>
</dependencies>
<properties>
<!-- By default we need to run these 5 containers, although the two init containers exit quickly -->
<docker.image>mongo1,mongo-init</docker.image>
<!-- The command line args passed to each mongod service -->
<mongo.data.replicaset.name>rs0</mongo.data.replicaset.name>
<mongo.cfg.replicaset.name>cfg</mongo.cfg.replicaset.name>
<mongo.data.runOptions>--replSet ${mongo.data.replicaset.name} --oplogSize=2 --enableMajorityReadConcern</mongo.data.runOptions>
<mongo.cfg.runOptions>--configsvr --replSet ${mongo.cfg.replicaset.name} --oplogSize=2 --enableMajorityReadConcern</mongo.cfg.runOptions>
<mongo.router.runOptions>mongos --configdb ${mongo.cfg.replicaset.name}/config:27019</mongo.router.runOptions>
<!--
Set this property to 'true' in a profile to skip the integration tests and not even run the Docker containers.
-->
<docker.skip>false</docker.skip>
<docker.exposeContainerInfo>docker.container</docker.exposeContainerInfo>
</properties>
<build>
<plugins>
<plugin>
<groupId>io.fabric8</groupId>
<artifactId>docker-maven-plugin</artifactId>
<configuration>
<watchInterval>500</watchInterval>
<logDate>default</logDate>
<verbose>true</verbose>
<!--autoPull>always</autoPull-->
<images>
<!-- A container for the data server replica set -->
<image>
<name>mongo:${version.mongo.server}</name>
<alias>mongo1</alias>
<run>
<namingStrategy>alias</namingStrategy>
<cmd>${mongo.data.runOptions}</cmd>
<ports>
<!-- We won't use it thru this port, but we don't want to take 27017 on docker host -->
<port>27017:27017</port>
</ports>
<log>
<prefix>mongo1</prefix>
<enabled>true</enabled>
<color>yellow</color>
</log>
<wait>
<log>waiting for connections on port 27017</log> <!-- internal port -->
<time>30000</time> <!-- 30 seconds max -->
</wait>
</run>
</image>
<!-- A container that initiates the data replica set and adds it as shard to router -->
<image>
<name>debezium/mongo-initiator:3.2</name>
<alias>mongo-init</alias>
<run>
<namingStrategy>alias</namingStrategy>
<env>
<VERBOSE>true</VERBOSE>
<REPLICASET>${mongo.data.replicaset.name}</REPLICASET>
</env>
<links>
<link>mongo1:mongo1</link>
</links>
<log>
<prefix>mongo-init</prefix>
<enabled>true</enabled>
<color>green</color>
</log>
<wait>
<log>Replica set is ready</log>
<time>30000</time> <!-- 30 seconds max -->
</wait>
</run>
</image>
</images>
</configuration>
<!--
Connect this plugin to the maven lifecycle around the integration-test phase:
start the container in pre-integration-test and stop it in post-integration-test.
-->
<executions>
<execution>
<id>start</id>
<phase>pre-integration-test</phase>
<goals>
<goal>build</goal>
<goal>start</goal>
</goals>
</execution>
<execution>
<id>stop</id>
<phase>post-integration-test</phase>
<goals>
<goal>stop</goal>
</goals>
</execution>
</executions>
</plugin>
<!--
Unlike surefire, the failsafe plugin ensures 'post-integration-test' phase always runs, even
when there are failed integration tests. We rely upon this to always shut down the Docker container
after the integration tests (defined as '*IT.java') are run.
-->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-failsafe-plugin</artifactId>
<executions>
<execution>
<id>integration-test</id>
<goals>
<goal>integration-test</goal>
</goals>
</execution>
<execution>
<id>verify</id>
<goals>
<goal>verify</goal>
</goals>
</execution>
</executions>
<configuration>
<skipTests>${skipITs}</skipTests>
<enableAssertions>true</enableAssertions>
<systemPropertyVariables>
<!-- Make these available to the tests via system properties -->
<connector.mongodb.hosts>${mongo.data.replicaset.name}/${docker.host.address}:27017</connector.mongodb.hosts>
<connector.mongodb.members.auto.discover>false</connector.mongodb.members.auto.discover>
<connector.mongodb.name>mongo1</connector.mongodb.name>
<skipLongRunningTests>${skipLongRunningTests}</skipLongRunningTests>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
<resources>
<!-- Apply the properties set in the POM to the resource files -->
<resource>
<filtering>true</filtering>
<directory>src/main/resources</directory>
<includes>
<include>*</include>
<include>**/*</include>
</includes>
</resource>
</resources>
<testResources>
<testResource>
<directory>src/test/resources</directory>
<filtering>true</filtering>
<includes>
<include>*</include>
<include>**/*</include>
</includes>
</testResource>
</testResources>
</build>
<!--
Define several useful profiles
-->
<profiles>
<profile>
<id>assembly</id>
<activation>
<activeByDefault>false</activeByDefault>
</activation>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-assembly-plugin</artifactId>
<dependencies>
<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-assembly-descriptors</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
<executions>
<execution>
<id>default</id>
<phase>package</phase>
<goals>
<goal>single</goal>
</goals>
<configuration>
<finalName>${project.artifactId}-${project.version}</finalName>
<attach>true</attach> <!-- we want attach & deploy these to Maven -->
<descriptorRefs>
<descriptorRef>connector-distribution</descriptorRef>
</descriptorRefs>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
<!-- ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
Do not perform any Docker-related functionality
To use, specify "-DskipITs" on the Maven command line.
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -->
<profile>
<id>skip-integration-tests</id>
<activation>
<activeByDefault>false</activeByDefault>
<property>
<name>skipITs</name>
</property>
</activation>
<properties>
<docker.skip>true</docker.skip>
</properties>
</profile>
</profiles>
</project>

View File

@ -0,0 +1,107 @@
/*
* 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 io.debezium.annotation.Immutable;
/**
* A simple identifier for collections in a replica set.
*
* @author Randall Hauch
*/
@Immutable
public final class CollectionId {
/**
* Parse the supplied string, extracting the first 3 parts into a Collection.
*
* @param str the string representation of the collection identifier; may not be null
* @return the collection ID, or null if it could not be parsed
*/
public static CollectionId parse(String str) {
String[] parts = str.split("[\\" + '.' + "]");
if (parts.length < 3) return null;
return new CollectionId(parts[0], parts[1], parts[2]);
}
private final String replicaSetName;
private final String dbName;
private final String name;
/**
* Create a new collection identifier.
*
* @param replicaSetName the name of the replica set; may not be null
* @param dbName the name of the database; may not be null
* @param collectionName the name of the collection; may not be null
*/
public CollectionId(String replicaSetName, String dbName, String collectionName) {
this.replicaSetName = replicaSetName;
this.dbName = dbName;
this.name = collectionName;
assert this.replicaSetName != null;
assert this.dbName != null;
assert this.name != null;
}
/**
* Get the name of the collection.
*
* @return the collection's name; never null
*/
public String name() {
return name;
}
/**
* Get the name of the database in which the collection exists.
*
* @return the database name; never null
*/
public String dbName() {
return dbName;
}
/**
* Get the name of the replica set in which the collection (and database) exist.
*
* @return the replica set name; never null
*/
public String replicaSetName() {
return replicaSetName;
}
@Override
public int hashCode() {
return name.hashCode();
}
@Override
public boolean equals(Object obj) {
if (obj == this) return true;
if (obj instanceof CollectionId) {
CollectionId that = (CollectionId) obj;
return this.replicaSetName.equals(that.replicaSetName) &&
this.dbName.equals(that.dbName) && this.name.equals(that.name);
}
return false;
}
/**
* Get the namespace of this collection, which is comprised of the {@link #dbName database name} and {@link #name collection
* name}.
*
* @return the namespace for this collection; never null
*/
public String namespace() {
return dbName + "." + name;
}
@Override
public String toString() {
return replicaSetName + "." + dbName + "." + name;
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.util.Set;
import java.util.function.Predicate;
import io.debezium.config.Configuration;
import io.debezium.function.Predicates;
import io.debezium.util.Collect;
/**
* A utility that is contains filters for acceptable collections.
*
* @author Randall Hauch
*/
public final class Filters {
protected static final Set<String> BUILT_IN_DB_NAMES = Collect.unmodifiableSet("local", "admin");
private final Predicate<CollectionId> collectionFilter;
/**
* Create an instance of the filters.
*
* @param config the configuration; may not be null
*/
public Filters(Configuration config) {
String whitelist = config.getString(MongoDbConnectorConfig.COLLECTION_WHITELIST);
String blacklist = config.getString(MongoDbConnectorConfig.COLLECTION_BLACKLIST);
Predicate<CollectionId> collectionFilter = null;
if (whitelist != null && !whitelist.trim().isEmpty()) {
collectionFilter = Predicates.includes(whitelist, CollectionId::namespace);
} else if (blacklist != null && !blacklist.trim().isEmpty()) {
collectionFilter = Predicates.excludes(blacklist, CollectionId::namespace);
} else {
collectionFilter = (id) -> true;
}
Predicate<CollectionId> isNotBuiltIn = this::isNotBuiltIn;
this.collectionFilter = isNotBuiltIn.and(collectionFilter);
}
/**
* Get the predicate function that determines whether the given collection is to be included.
*
* @return the collection filter; never null
*/
public Predicate<CollectionId> collectionFilter() {
return collectionFilter;
}
protected boolean isNotBuiltIn(CollectionId id) {
return !BUILT_IN_DB_NAMES.contains(id.dbName());
}
}

View File

@ -0,0 +1,24 @@
/*
* 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.util.Properties;
import io.debezium.util.IoUtil;
/**
* Information about this module.
*
* @author Randall Hauch
*/
public final class Module {
private static final Properties INFO = IoUtil.loadProperties(Module.class, "io/debezium/connector/mongodb/build.version");
public static String version() {
return INFO.getProperty("version");
}
}

View File

@ -0,0 +1,198 @@
/*
* 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.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArrayList;
import com.mongodb.MongoClient;
import com.mongodb.MongoClientOptions;
import com.mongodb.MongoCredential;
import com.mongodb.ServerAddress;
import io.debezium.annotation.ThreadSafe;
/**
* A connection pool of MongoClient instances. This pool supports creating clients that communicate explicitly with a single
* server, or clients that communicate with any members of a replica set or sharded cluster given a set of seed addresses.
*
* @author Randall Hauch
*/
@ThreadSafe
public class MongoClients {
/**
* Obtain a builder that can be used to configure and {@link Builder#build() create} a connection pool.
*
* @return the new builder; never null
*/
public static Builder create() {
return new Builder();
}
/**
* Configures and builds a ConnectionPool.
*/
public static class Builder {
private final List<MongoCredential> credentials = new CopyOnWriteArrayList<>();
private final MongoClientOptions.Builder optionBuilder = MongoClientOptions.builder();
/**
* Add the given {@link MongoCredential} for use when creating clients.
*
* @param credential the credential; may be {@code null}, though this method does nothing if {@code null}
* @return this builder object so methods can be chained; never null
*/
public Builder withCredential(MongoCredential credential) {
if (credential != null) credentials.add(credential);
return this;
}
/**
* Obtain the options builder for client connections.
*
* @return the option builder; never null
*/
public MongoClientOptions.Builder options() {
return optionBuilder;
}
/**
* Build the client pool that will use the credentials and options already configured on this builder.
*
* @return the new client pool; never null
*/
public MongoClients build() {
return new MongoClients(optionBuilder.build(), credentials);
}
}
private final Map<ServerAddress, MongoClient> directConnections = new ConcurrentHashMap<>();
private final Map<List<ServerAddress>, MongoClient> connections = new ConcurrentHashMap<>();
private final List<MongoCredential> credentials = new CopyOnWriteArrayList<>();
private final MongoClientOptions options;
private MongoClients(MongoClientOptions options, List<MongoCredential> credentials) {
this.options = options;
if (credentials != null) {
credentials.forEach(this.credentials::add);
}
}
/**
* Clear out and close any open connections.
*/
public void clear() {
directConnections.values().forEach(MongoClient::close);
connections.values().forEach(MongoClient::close);
directConnections.clear();
connections.clear();
}
/**
* Obtain a direct client connection to the specified server. This is typically used to connect to a standalone server,
* but it also can be used to obtain a client that will only use this server, even if the server is a member of a replica
* set or sharded cluster.
* <p>
* The format of the supplied string is one of the following:
*
* <pre>
* host:port
* host
* </pre>
*
* where {@code host} contains the resolvable hostname or IP address of the server, and {@code port} is the integral port
* number. If the port is not provided, the {@link ServerAddress#defaultPort() default port} is used. If neither the host
* or port are provided (or {@code addressString} is {@code null}), then an address will use the
* {@link ServerAddress#defaultHost() default host} and {@link ServerAddress#defaultPort() default port}.
*
* @param addressString the string that contains the host and port of the server
* @return the MongoClient instance; never null
*/
public MongoClient clientFor(String addressString) {
return clientFor(MongoUtil.parseAddress(addressString));
}
/**
* Obtain a direct client connection to the specified server. This is typically used to connect to a standalone server,
* but it also can be used to obtain a client that will only use this server, even if the server is a member of a replica
* set or sharded cluster.
*
* @param address the address of the server to use
* @return the MongoClient instance; never null
*/
public MongoClient clientFor(ServerAddress address) {
return directConnections.computeIfAbsent(address, this::directConnection);
}
/**
* Obtain a client connection to the replica set or cluster. The supplied addresses are used as seeds, and once a connection
* is established it will discover all of the members.
* <p>
* The format of the supplied string is one of the following:
*
* <pre>
* replicaSetName/host:port
* replicaSetName/host:port,host2:port2
* replicaSetName/host:port,host2:port2,host3:port3
* host:port
* host:port,host2:port2
* host:port,host2:port2,host3:port3
* </pre>
*
* where {@code replicaSetName} is the name of the replica set, {@code host} contains the resolvable hostname or IP address of
* the server, and {@code port} is the integral port number. If the port is not provided, the
* {@link ServerAddress#defaultPort() default port} is used. If neither the host or port are provided (or
* {@code addressString} is {@code null}), then an address will use the {@link ServerAddress#defaultHost() default host} and
* {@link ServerAddress#defaultPort() default port}.
* <p>
* This method does not use the replica set name.
*
* @param addressList the string containing a comma-separated list of host and port pairs, optionally preceded by a
* replica set name
* @return the MongoClient instance; never null
*/
public MongoClient clientForMembers(String addressList) {
return clientForMembers(MongoUtil.parseAddresses(addressList));
}
/**
* Obtain a client connection to the replica set or cluster. The supplied addresses are used as seeds, and once a connection
* is established it will discover all of the members.
*
* @param seeds the seed addresses
* @return the MongoClient instance; never null
*/
public MongoClient clientForMembers(ServerAddress... seeds) {
List<ServerAddress> addresses = new ArrayList<>();
for (ServerAddress seedAddress : seeds) {
if (seedAddress != null) addresses.add(seedAddress);
}
return clientForMembers(addresses);
}
/**
* Obtain a client connection to the replica set or cluster. The supplied addresses are used as seeds, and once a connection
* is established it will discover all of the members.
*
* @param seedAddresses the seed addresses
* @return the MongoClient instance; never null
*/
public MongoClient clientForMembers(List<ServerAddress> seedAddresses) {
return connections.computeIfAbsent(seedAddresses, this::connection);
}
protected MongoClient directConnection(ServerAddress address) {
return new MongoClient(address, credentials, options);
}
protected MongoClient connection(List<ServerAddress> addresses) {
return new MongoClient(addresses, credentials, options);
}
}

View File

@ -0,0 +1,178 @@
/*
* 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.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.source.SourceConnector;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.config.Configuration;
import io.debezium.util.Clock;
import io.debezium.util.LoggingContext.PreviousContext;
/**
* A Kafka Connect source connector that creates {@link MongoDbConnectorTask tasks} that replicate the context of one or more
* MongoDB replica sets.
*
* <h2>Sharded Clusters</h2>
* This connector is able to fully replicate the content of one <a href="https://docs.mongodb.com/manual/sharding/">sharded
* MongoDB 3.2 cluster</a>. In this case, simply configure the connector with the host addresses of the configuration replica set.
* When the connector starts, it will discover and replicate the replica set for each shard.
*
* <h2>Replica Set</h2>
* The connector is able to fully replicate the content of one <a href="https://docs.mongodb.com/manual/replication/">MongoDB
* 3.2 replica set</a>. (Older MongoDB servers may be work but have not been tested.) In this case, simply configure the connector
* with the host addresses of the replica set. When the connector starts, it will discover the primary node and use it to
* replicate the contents of the replica set.
* <p>
* If necessary, a {@link MongoDbConnectorConfig#AUTO_DISCOVER_MEMBERS configuration property} can be used to disable the
* logic used to discover the primary node, an in this case the connector will use the first host address specified in the
* configuration as the primary node. Obviously this may cause problems when the replica set elects a different node as the
* primary, since the connector will continue to read the oplog using the same node that may no longer be the primary.
*
* <h2>Parallel Replication</h2>
* The connector will concurrently and independently replicate each of the replica sets. When the connector is asked to
* {@link #taskConfigs(int) allocate tasks}, it will attempt to allocate a separate task for each replica set. However, if the
* maximum number of tasks exceeds the number of replica sets, then some tasks may replicate multiple replica sets. Note that
* each task will use a separate thread to replicate each of its assigned replica sets.
*
* <h2>Initial Sync and Reading the Oplog</h2>
* When a connector begins to replicate a sharded cluster or replica set for the first time, it will perform an <em>initial
* sync</em> of the collections in the replica set by generating source records for each document in each collection. Only when
* this initial sync completes successfully will the replication then use the replica set's primary node to read the oplog and
* produce source records for each oplog event. The replication process records the position of each oplog event as an
* <em>offset</em>, so that upon restart the replication process can use the last recorded offset to determine where in the
* oplog it is to begin reading and processing events.
*
* <h2>Use of Topics</h2>
* The connector will write to a separate topic all of the source records that correspond to a single collection. The topic will
* be named "{@code <logicalName>.<databaseName>.<collectionName>}", where {@code <logicalName>} is set via the
* "{@link MongoDbConnectorConfig#LOGICAL_NAME mongodb.name}" configuration property.
*
* <h2>Configuration</h2>
* <p>
* This connector is configured with the set of properties described in {@link MongoDbConnectorConfig}.
*
* @author Randall Hauch
*/
public class MongoDbConnector extends SourceConnector {
private final Logger logger = LoggerFactory.getLogger(getClass());
private Configuration config;
private ReplicaSetMonitorThread monitorThread;
private ReplicationContext replContext;
public MongoDbConnector() {
}
@Override
public String version() {
return Module.version();
}
@Override
public Class<? extends Task> taskClass() {
return MongoDbConnectorTask.class;
}
@Override
public void start(Map<String, String> props) {
// Validate the configuration ...
final Configuration config = Configuration.from(props);
if (!config.validate(MongoDbConnectorConfig.ALL_FIELDS, logger::error)) {
throw new ConnectException("Error configuring an instance of " + getClass().getSimpleName() + "; check the logs for details");
}
this.config = config;
// Set up the replication context ...
replContext = new ReplicationContext(config);
PreviousContext previousLogContext = replContext.configureLoggingContext("conn");
try {
logger.info("Starting MongoDB connector and discovering replica set(s) at {}", replContext.hosts());
// Set up and start the thread that monitors the members of all of the replica sets ...
ReplicaSetDiscovery monitor = new ReplicaSetDiscovery(replContext);
monitorThread = new ReplicaSetMonitorThread(monitor::getReplicaSets, replContext.pollPeriodInSeconds(), TimeUnit.SECONDS,
Clock.SYSTEM, () -> replContext.configureLoggingContext("disc"), this::replicaSetsChanged);
monitorThread.start();
logger.info("Successfully started MongoDB connector, and continuing to discover changes in replica sets", replContext.hosts());
} finally {
previousLogContext.restore();
}
}
protected void replicaSetsChanged(ReplicaSets replicaSets) {
logger.info("Requesting task reconfiguration due to new/removed replica set(s) for MongoDB with seeds {}", replContext.hosts());
logger.info("New replica sets include:");
replicaSets.onEachReplicaSet(replicaSet -> {
logger.info(" {}", replicaSet);
});
context.requestTaskReconfiguration();
}
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
PreviousContext previousLogContext = replContext.configureLoggingContext("conn");
try {
if (config == null) {
logger.error("Configuring a maximum of {} tasks with no connector configuration available", maxTasks);
return Collections.emptyList();
}
// Partitioning the replica sets amongst the number of tasks ...
List<Map<String, String>> taskConfigs = new ArrayList<>(maxTasks);
ReplicaSets replicaSets = monitorThread.getReplicaSets(10, TimeUnit.SECONDS);
if (replicaSets != null) {
logger.info("Subdividing {} MongoDB replica set(s) into at most {} task(s)",
replicaSets.replicaSetCount(), maxTasks);
replicaSets.subdivide(maxTasks, replicaSetsForTask -> {
// Create the configuration for each task ...
int taskId = taskConfigs.size();
logger.info("Configuring MongoDB connector task {} to capture events for replica set(s) at {}", taskId, replicaSetsForTask.hosts());
taskConfigs.add(config.edit()
.with(MongoDbConnectorConfig.HOSTS, replicaSetsForTask.hosts())
.with(MongoDbConnectorConfig.TASK_ID, taskId)
.build()
.asMap());
});
}
logger.debug("Configuring {} MongoDB connector task(s)", taskConfigs.size());
return taskConfigs;
} finally {
previousLogContext.restore();
}
}
@Override
public void stop() {
PreviousContext previousLogContext = replContext != null ? replContext.configureLoggingContext("conn") : null;
try {
logger.info("Stopping MongoDB connector");
this.config = null;
try {
if ( this.monitorThread != null ) this.monitorThread.shutdown();
} finally {
try {
if ( this.replContext != null ) this.replContext.shutdown();
} finally {
logger.info("Stopped MongoDB connector");
}
}
} finally {
if ( previousLogContext != null ) previousLogContext.restore();
}
}
}

View File

@ -0,0 +1,154 @@
/*
* 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.util.Collection;
import java.util.Collections;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import io.debezium.config.Configuration;
import io.debezium.config.Field;
import io.debezium.util.Collect;
/**
* The configuration properties.
*/
public class MongoDbConnectorConfig {
public static final Field HOSTS = Field.create("mongodb.hosts")
.withDescription("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.")
.withValidation(MongoDbConnectorConfig::validateHosts);
public static final Field LOGICAL_NAME = Field.create("mongodb.name")
.withDescription("A unique name that identifies the connector and/or MongoDB replica set or cluster that this connector monitors. Each server should be monitored by at most one Debezium connector, since this server name prefixes all persisted Kakfa topics eminating from this server.")
.withValidation(Field::isRequired);
public static final Field USER = Field.create("mongodb.user")
.withDescription("Name of the database user to be used when connecting to MongoDB. "
+ "This is required only when MongoDB is configured to use authentication.")
.withValidation(Field::isOptional);
public static final Field PASSWORD = Field.create("mongodb.password")
.withDescription("Password to be used when connecting to MongoDB. "
+ "This is required only when MongoDB is configured to use authentication.")
.withValidation(Field::isOptional);
public static final Field POLL_INTERVAL_SEC = Field.create("mongodb.poll.interval.sec")
.withDescription("Frequency in seconds to look for new, removed, or changed replica sets. Defaults to 30 seconds.")
.withDefault(30)
.withValidation(Field::isPositiveInteger);
public static final Field MAX_COPY_THREADS = Field.create("initial.sync.max.threads")
.withDescription("Maximum number of threads used to perform an intial sync of the collections in a replica set. "
+ "Defaults to 1.")
.withDefault(1)
.withValidation(Field::isPositiveInteger);
public static final Field MAX_QUEUE_SIZE = Field.create("max.queue.size")
.withDescription("Maximum size of the queue for change events read from the database log but not yet recorded or forwarded. Defaults to 2048, and should always be larger than the maximum batch size.")
.withDefault(2048)
.withValidation(MongoDbConnectorConfig::validateMaxQueueSize);
public static final Field MAX_BATCH_SIZE = Field.create("max.batch.size")
.withDescription("Maximum size of each batch of source records. Defaults to 1024.")
.withDefault(1024)
.withValidation(Field::isPositiveInteger);
public static final Field POLL_INTERVAL_MS = Field.create("poll.interval.ms")
.withDescription("Frequency in milliseconds to wait after processing no events for new change events to appear. Defaults to 1 second (1000 ms).")
.withDefault(TimeUnit.SECONDS.toMillis(1))
.withValidation(Field::isPositiveInteger);
public static final Field CONNECT_BACKOFF_INITIAL_DELAY_MS = Field.create("connect.backoff.initial.delay.ms")
.withDescription("The initial delay when trying to reconnect to a primary after a connection cannot be made or when no primary is available. Defaults to 1 second (1000 ms).")
.withDefault(TimeUnit.SECONDS.toMillis(1))
.withValidation(Field::isPositiveInteger);
public static final Field CONNECT_BACKOFF_MAX_DELAY_MS = Field.create("connect.backoff.max.delay.ms")
.withDescription("The maximum delay when trying to reconnect to a primary after a connection cannot be made or when no primary is available. Defaults to 120 second (120,000 ms).")
.withDefault(TimeUnit.SECONDS.toMillis(120))
.withValidation(Field::isPositiveInteger);
public static final Field MAX_FAILED_CONNECTIONS = Field.create("connect.max.attempts")
.withDescription("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 '" + CONNECT_BACKOFF_INITIAL_DELAY_MS + "' and '" + CONNECT_BACKOFF_MAX_DELAY_MS + "' results in "
+ "just over 20 minutes of attempts before failing.")
.withDefault(16)
.withValidation(Field::isPositiveInteger);
public static final Field COLLECTION_BLACKLIST = Field.create("collection.blacklist")
.withValidation(MongoDbConnectorConfig::validateCollectionBlacklist)
.withDescription("A comma-separated list of regular expressions that match the fully-qualified namespaces of collections to be excluded from monitoring. "
+ "Fully-qualified namespaces for collections are of the form '<databaseName>.<collectionName>'. "
+ "May not be used with 'collection.whitelist'.");
public static final Field COLLECTION_WHITELIST = Field.create("collection.whitelist")
.withDescription("A comma-separated list of regular expressions that match the fully-qualified namespaces of collections to be monitored. "
+ "Fully-qualified namespaces for collections are of the form '<databaseName>.<collectionName>'. "
+ "May not be used with '" + COLLECTION_BLACKLIST + "'.");
public static final Field AUTO_DISCOVER_MEMBERS = Field.create("mongodb.members.auto.discover",
"Specifies whether the addresses in 'hosts' are seeds that should be used to discover all members of the cluster or replica set ('true'), "
+ "or whether the address(es) in 'hosts' should be used as is ('false'). The default is 'true'.")
.withDefault(true).withValidation(Field::isBoolean);
public static final Field TASK_ID = Field.create("mongodb.task.id")
.withDescription("Internal use only")
.withValidation(Field::isInteger);
public static Collection<Field> ALL_FIELDS = Collections.unmodifiableList(Collect.arrayListOf(USER, PASSWORD, HOSTS, LOGICAL_NAME,
MAX_QUEUE_SIZE, MAX_BATCH_SIZE,
POLL_INTERVAL_MS,
MAX_FAILED_CONNECTIONS,
CONNECT_BACKOFF_INITIAL_DELAY_MS,
CONNECT_BACKOFF_MAX_DELAY_MS,
COLLECTION_WHITELIST,
COLLECTION_BLACKLIST,
AUTO_DISCOVER_MEMBERS));
private static int validateHosts(Configuration config, Field field, Consumer<String> problems) {
String hosts = config.getString(field);
if (hosts == null) {
problems.accept("'" + field + "' is required.");
return 1;
}
int count = 0;
if (ReplicaSets.parse(hosts) == null) {
problems.accept("'" + hosts + "' is not valid host specification");
++count;
}
return count;
}
private static int validateMaxQueueSize(Configuration config, Field field, Consumer<String> problems) {
int maxQueueSize = config.getInteger(field);
int maxBatchSize = config.getInteger(MAX_BATCH_SIZE);
int count = 0;
if (maxQueueSize <= 0) {
maxBatchSize = maxQueueSize / 2;
problems.accept("The " + MAX_QUEUE_SIZE + " value '" + maxQueueSize + "' must be positive");
++count;
}
if (maxQueueSize <= maxBatchSize) {
maxBatchSize = maxQueueSize / 2;
problems.accept("The " + MAX_QUEUE_SIZE + " value '" + maxQueueSize + "' must be larger than " +
MAX_BATCH_SIZE + " of '" + maxBatchSize + ".");
++count;
}
return count;
}
private static int validateCollectionBlacklist(Configuration config, Field field, Consumer<String> problems) {
String whitelist = config.getString(COLLECTION_WHITELIST);
String blacklist = config.getString(COLLECTION_BLACKLIST);
if (whitelist != null && blacklist != null) {
problems.accept("May use either '" + COLLECTION_WHITELIST + "' or '" + COLLECTION_BLACKLIST + "', but not both.");
return 1;
}
return 0;
}
}

View File

@ -0,0 +1,225 @@
/*
* 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.util.ArrayList;
import java.util.Collection;
import java.util.Deque;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BooleanSupplier;
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 io.debezium.annotation.Immutable;
import io.debezium.annotation.ThreadSafe;
import io.debezium.config.Configuration;
import io.debezium.util.Clock;
import io.debezium.util.LoggingContext.PreviousContext;
import io.debezium.util.Metronome;
/**
* A Kafka Connect source task that replicates the changes from one or more MongoDB replica sets, using one {@link Replicator}
* for each replica set.
* <p>
* Generally, the {@link MongoDbConnector} assigns each replica set to a separate task, although multiple
* replica sets will be assigned to each task when the maximum number of tasks is limited. Regardless, every task will use a
* separate thread to replicate the contents of each replica set, and each replication thread may use multiple threads
* to perform an initial sync of the replica set.
*
* @see MongoDbConnector
* @see MongoDbConnectorConfig
* @author Randall Hauch
*/
@ThreadSafe
public final class MongoDbConnectorTask extends SourceTask {
private final Logger logger = LoggerFactory.getLogger(getClass());
private final AtomicBoolean running = new AtomicBoolean(false);
private final Deque<Replicator> replicators = new ConcurrentLinkedDeque<>();
// These are all effectively constants between start(...) and stop(...)
private volatile TaskRecordQueue queue;
private volatile String taskName;
private volatile ReplicationContext replContext;
/**
* Create an instance of the MongoDB task.
*/
public MongoDbConnectorTask() {
}
@Override
public String version() {
return Module.version();
}
@Override
public void start(Map<String, String> props) {
if (!this.running.compareAndSet(false, true)) {
// Already running ...
return;
}
if (context == null) {
throw new ConnectException("Unexpected null context");
}
// Read the configuration and set up the replication context ...
final Configuration config = Configuration.from(props);
this.taskName = "task" + config.getInteger(MongoDbConnectorConfig.TASK_ID);
final ReplicationContext replicationContext = new ReplicationContext(config);
this.replContext = replicationContext;
PreviousContext previousLogContext = replicationContext.configureLoggingContext(taskName);
try {
// Output the configuration ...
logger.info("Starting MongoDB connector task with configuration:");
config.forEach((propName, propValue) -> {
logger.info(" {} = {}", propName, propValue);
});
// The MongoDbConnector.taskConfigs created our configuration, but we still validate the configuration in case of bugs
// ...
if (!config.validate(MongoDbConnectorConfig.ALL_FIELDS, logger::error)) {
throw new ConnectException(
"Error configuring an instance of " + getClass().getSimpleName() + "; check the logs for details");
}
// Read from the configuration the information about the replica sets we are to watch ...
final String hosts = config.getString(MongoDbConnectorConfig.HOSTS);
final ReplicaSets replicaSets = ReplicaSets.parse(hosts);
// Set up the task record queue ...
this.queue = new TaskRecordQueue(config, replicaSets.replicaSetCount(), running::get);
// Get the offsets for each of replica set partition ...
SourceInfo source = replicationContext.source();
Collection<Map<String, String>> partitions = new ArrayList<>();
replicaSets.onEachReplicaSet(replicaSet -> {
String replicaSetName = replicaSet.replicaSetName();
partitions.add(source.partition(replicaSetName));
});
context.offsetStorageReader().offsets(partitions).forEach(source::setOffsetFor);
// Set up a replicator for each replica set ...
final int numThreads = replicaSets.replicaSetCount();
final ExecutorService executor = Executors.newFixedThreadPool(numThreads);
AtomicInteger stillRunning = new AtomicInteger(numThreads);
logger.info("Starting {} thread(s) to replicate replica sets: {}", numThreads, replicaSets);
replicaSets.all().forEach(replicaSet -> {
// Create a replicator for this replica set ...
Replicator replicator = new Replicator(replicationContext, replicaSet, queue::enqueue);
replicators.add(replicator);
// and submit it for execution ...
executor.submit(() -> {
try {
// Configure the logging to use the replica set name ...
replicationContext.configureLoggingContext(replicaSet.replicaSetName());
// Run the replicator, which should run forever until it is stopped ...
replicator.run();
} finally {
try {
replicators.remove(replicator);
} finally {
if (stillRunning.decrementAndGet() == 0) {
// we are the last one, so clean up ...
try {
executor.shutdown();
} finally {
replicationContext.shutdown();
}
}
}
}
});
});
logger.info("Successfully started MongoDB connector task with {} thread(s) for replica sets {}", numThreads, replicaSets);
} finally {
previousLogContext.restore();
}
}
@Override
public List<SourceRecord> poll() throws InterruptedException {
return this.queue.poll();
}
@Override
public void stop() {
PreviousContext previousLogContext = this.replContext.configureLoggingContext(taskName);
try {
// Signal to the 'poll()' method that it should stop what its doing ...
if (this.running.compareAndSet(true, false)) {
logger.info("Stopping MongoDB task");
// Stop all running replicators ...
Replicator replicator = null;
int counter = 0;
while ((replicator = this.replicators.poll()) != null) {
replicator.stop();
++counter;
}
logger.info("Stopped MongoDB replication task by stopping {} replicator threads", counter);
}
} catch (Throwable e) {
logger.error("Unexpected error shutting down the MongoDB replication task", e);
} finally {
previousLogContext.restore();
}
}
@Immutable
protected static class TaskRecordQueue {
// These are all effectively constants between start(...) and stop(...)
private final int maxBatchSize;
private final Metronome metronome;
private final BlockingQueue<SourceRecord> records;
private final BooleanSupplier isRunning;
protected TaskRecordQueue(Configuration config, int numThreads, BooleanSupplier isRunning) {
final int maxQueueSize = config.getInteger(MongoDbConnectorConfig.MAX_QUEUE_SIZE);
final long pollIntervalMs = config.getLong(MongoDbConnectorConfig.POLL_INTERVAL_MS);
maxBatchSize = config.getInteger(MongoDbConnectorConfig.MAX_BATCH_SIZE);
metronome = Metronome.parker(pollIntervalMs, TimeUnit.MILLISECONDS, Clock.SYSTEM);
records = new LinkedBlockingDeque<>(maxQueueSize);
this.isRunning = isRunning;
}
public List<SourceRecord> poll() throws InterruptedException {
List<SourceRecord> batch = new ArrayList<>(maxBatchSize);
while (isRunning.getAsBoolean() && records.drainTo(batch, maxBatchSize) == 0) {
// No events to process, so sleep for a bit ...
metronome.pause();
}
return batch;
}
/**
* Adds the event into the queue for subsequent batch processing.
*
* @param record a record from the MongoDB oplog
* @throws InterruptedException if the thread is interrupted while waiting to enqueue the record
*/
public void enqueue(SourceRecord record) throws InterruptedException {
if (record != null) {
records.put(record);
}
}
}
}

View File

@ -0,0 +1,299 @@
/*
* 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.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.function.Consumer;
import java.util.function.Predicate;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.bson.Document;
import com.mongodb.MongoClient;
import com.mongodb.ServerAddress;
import com.mongodb.client.MongoCollection;
import com.mongodb.client.MongoCursor;
import com.mongodb.client.MongoDatabase;
import com.mongodb.client.MongoIterable;
import io.debezium.function.BlockingConsumer;
import io.debezium.util.Strings;
/**
* Utilities for working with MongoDB.
*
* @author Randall Hauch
*/
public class MongoUtil {
/**
* The delimiter used between addresses.
*/
private static final String ADDRESS_DELIMITER = ",";
/**
* Regular expression that gets the host and (optional) port. The raw expression is {@code ([^:]+)(:(\d+))?}.
*/
private static final Pattern ADDRESS_PATTERN = Pattern.compile("([^:]+)(:(\\d+))?");
/**
* Regular expression that gets the IPv6 host and (optional) port, where the IPv6 address must be surrounded
* by square brackets. The raw expression is {@code (\[[^]]+\])(:(\d+))?}.
*/
private static final Pattern IPV6_ADDRESS_PATTERN = Pattern.compile("(\\[[^]]+\\])(:(\\d+))?");
/**
* Find the name of the replica set precedes the host addresses.
*
* @param addresses the string containing the host addresses, of the form {@code replicaSetName/...}; may not be null
* @return the replica set name, or {@code null} if no replica set name is in the string
*/
public static String replicaSetUsedIn(String addresses) {
if ( addresses.startsWith("[")) {
// Just an IPv6 address, so no replica set name ...
return null;
}
// Either a replica set name + an address, or just an IPv4 address ...
int index = addresses.indexOf('/');
if (index < 0) return null;
return addresses.substring(0, index);
}
/**
* Perform the given operation on each of the database names.
*
* @param client the MongoDB client; may not be null
* @param operation the operation to perform; may not be null
*/
public static void forEachDatabaseName(MongoClient client, Consumer<String> operation) {
forEach(client.listDatabaseNames(), operation);
}
/**
* Perform the given operation on each of the collection names in the named database.
*
* @param client the MongoDB client; may not be null
* @param databaseName the name of the database; may not be null
* @param operation the operation to perform; may not be null
*/
public static void forEachCollectionNameInDatabase(MongoClient client, String databaseName, Consumer<String> operation) {
MongoDatabase db = client.getDatabase(databaseName);
forEach(db.listCollectionNames(), operation);
}
/**
* Perform the given operation on each of the values in the iterable container.
*
* @param iterable the iterable collection obtained from a MongoDB client; may not be null
* @param operation the operation to perform; may not be null
*/
public static <T> void forEach(MongoIterable<T> iterable, Consumer<T> operation) {
try (MongoCursor<T> cursor = iterable.iterator()) {
while (cursor.hasNext()) {
operation.accept(cursor.next());
}
}
}
/**
* Perform the given operation on the database with the given name, only if that database exists.
*
* @param client the MongoDB client; may not be null
* @param dbName the name of the database; may not be null
* @param dbOperation the operation to perform; may not be null
*/
public static void onDatabase(MongoClient client, String dbName, Consumer<MongoDatabase> dbOperation) {
if (contains(client.listDatabaseNames(), dbName)) {
dbOperation.accept(client.getDatabase(dbName));
}
}
/**
* Perform the given operation on the named collection in the named database, if the database and collection both exist.
*
* @param client the MongoDB client; may not be null
* @param dbName the name of the database; may not be null
* @param collectionName the name of the collection; may not be null
* @param collectionOperation the operation to perform; may not be null
*/
public static void onCollection(MongoClient client, String dbName, String collectionName,
Consumer<MongoCollection<Document>> collectionOperation) {
onDatabase(client, dbName, db -> {
if (contains(db.listCollectionNames(), collectionName)) {
collectionOperation.accept(db.getCollection(dbName));
}
});
}
/**
* Perform the given operation on all of the documents inside the named collection in the named database, if the database and
* collection both exist. The operation is called once for each document, so if the collection exists but is empty then the
* function will not be called.
*
* @param client the MongoDB client; may not be null
* @param dbName the name of the database; may not be null
* @param collectionName the name of the collection; may not be null
* @param documentOperation the operation to perform; may not be null
*/
public static void onCollectionDocuments(MongoClient client, String dbName, String collectionName,
BlockingConsumer<Document> documentOperation) {
onCollection(client, dbName, collectionName, collection -> {
try (MongoCursor<Document> cursor = collection.find().iterator()) {
while (cursor.hasNext()) {
try {
documentOperation.accept(cursor.next());
} catch (InterruptedException e) {
Thread.interrupted();
break;
}
}
}
});
}
/**
* Determine if the supplied {@link MongoIterable} contains an element that is equal to the supplied value.
*
* @param iterable the iterable; may not be null
* @param match the value to find in the iterable; may be null
* @return {@code true} if a matching value was found, or {@code false} otherwise
*/
public static <T> boolean contains(MongoIterable<String> iterable, String match) {
return contains(iterable, v -> Objects.equals(v, match));
}
/**
* Determine if the supplied {@link MongoIterable} contains at least one element that satisfies the given predicate.
*
* @param iterable the iterable; may not be null
* @param matcher the predicate function called on each value in the iterable until a match is found; may not be null
* @return {@code true} if a matching value was found, or {@code false} otherwise
*/
public static <T> boolean contains(MongoIterable<T> iterable, Predicate<T> matcher) {
try (MongoCursor<T> cursor = iterable.iterator()) {
while (cursor.hasNext()) {
if (matcher.test(cursor.next())) return true;
}
}
return false;
}
/**
* Parse the server address string, of the form {@code host:port} or {@code host}.
* <p>
* The IP address can be either an IPv4 address, or an IPv6 address surrounded by square brackets.
*
* @param addressStr the string containing the host and port; may be null
* @return the server address, or {@code null} if the string did not contain a host or host:port pair
*/
public static ServerAddress parseAddress(String addressStr) {
if (addressStr != null) {
addressStr = addressStr.trim();
Matcher matcher = ADDRESS_PATTERN.matcher(addressStr);
if (!matcher.matches()) {
matcher = IPV6_ADDRESS_PATTERN.matcher(addressStr);
}
if (matcher.matches()) {
// Both regex have the same groups
String host = matcher.group(1);
String port = matcher.group(3);
if (port == null) {
return new ServerAddress(host.trim());
}
return new ServerAddress(host.trim(), Integer.parseInt(port));
}
}
return null;
}
/**
* Parse the comma-separated list of server addresses. The format of the supplied string is one of the following:
*
* <pre>
* replicaSetName/host:port
* replicaSetName/host:port,host2:port2
* replicaSetName/host:port,host2:port2,host3:port3
* host:port
* host:port,host2:port2
* host:port,host2:port2,host3:port3
* </pre>
*
* where {@code replicaSetName} is the name of the replica set, {@code host} contains the resolvable hostname or IP address of
* the server, and {@code port} is the integral port number. If the port is not provided, the
* {@link ServerAddress#defaultPort() default port} is used. If neither the host or port are provided (or
* {@code addressString} is {@code null}), then an address will use the {@link ServerAddress#defaultHost() default host} and
* {@link ServerAddress#defaultPort() default port}.
* <p>
* The IP address can be either an IPv4 address, or an IPv6 address surrounded by square brackets.
* <p>
* This method does not use the replica set name.
*
* @param addressStr the string containing a comma-separated list of host and port pairs, optionally preceded by a
* replica set name
* @return the list of server addresses; never null, but possibly empty
*/
protected static List<ServerAddress> parseAddresses(String addressStr) {
List<ServerAddress> addresses = new ArrayList<>();
if (addressStr != null) {
addressStr = addressStr.trim();
for (String address : addressStr.split(ADDRESS_DELIMITER)) {
String hostAndPort = null;
if (address.startsWith("[")) {
// Definitely an IPv6 address without a replica set name ...
hostAndPort = address;
} else {
// May start with replica set name ...
int index = address.indexOf("/[");
if (index >= 0) {
if ((index + 2) < address.length()) {
// replica set name with IPv6, so use just the IPv6 address ...
hostAndPort = address.substring(index + 1);
} else {
// replica set name with just opening bracket; this is invalid, so we'll ignore ...
continue;
}
} else {
// possible replica set name with IPv4 only
index = address.indexOf("/");
if (index >= 0) {
if ((index + 1) < address.length()) {
// replica set name with IPv4, so use just the IPv4 address ...
hostAndPort = address.substring(index + 1);
} else {
// replica set name with no address ...
hostAndPort = ServerAddress.defaultHost();
}
} else {
// No replica set name with IPv4, so use the whole address ...
hostAndPort = address;
}
}
}
ServerAddress newAddress = parseAddress(hostAndPort);
if (newAddress != null) addresses.add(newAddress);
}
}
return addresses;
}
protected static String toString(ServerAddress address) {
String host = address.getHost();
if (host.contains(":")) {
// IPv6 address, so wrap with square brackets ...
return "[" + host + "]:" + address.getPort();
}
return host + ":" + address.getPort();
}
protected static String toString(List<ServerAddress> addresses) {
return Strings.join(ADDRESS_DELIMITER, addresses);
}
private MongoUtil() {
}
}

View File

@ -0,0 +1,238 @@
/*
* 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.util.HashMap;
import java.util.Map;
import java.util.function.Function;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.bson.Document;
import org.bson.json.JsonMode;
import org.bson.json.JsonWriterSettings;
import org.bson.types.ObjectId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.annotation.ThreadSafe;
import io.debezium.data.Envelope.FieldName;
import io.debezium.data.Envelope.Operation;
import io.debezium.function.BlockingConsumer;
/**
* A component that makes {@link SourceRecord}s for {@link CollectionId collections} and submits them to a consumer.
*
* @author Randall Hauch
*/
@ThreadSafe
public class RecordMakers {
private static final Map<String, Operation> operationLiterals = new HashMap<>();
static {
operationLiterals.put("i", Operation.CREATE);
operationLiterals.put("u", Operation.UPDATE);
operationLiterals.put("d", Operation.DELETE);
}
private final Logger logger = LoggerFactory.getLogger(getClass());
private final SourceInfo source;
private final TopicSelector topicSelector;
private final Map<CollectionId, RecordsForCollection> recordMakerByCollectionId = new HashMap<>();
private final Function<Document, String> valueTransformer;
private final BlockingConsumer<SourceRecord> recorder;
/**
* Create the record makers using the supplied components.
*
* @param source the connector's source information; may not be null
* @param topicSelector the selector for topic names; may not be null
* @param recorder the potentially blocking consumer function to be called for each generated record; may not be null
*/
public RecordMakers(SourceInfo source, TopicSelector topicSelector, BlockingConsumer<SourceRecord> recorder) {
this.source = source;
this.topicSelector = topicSelector;
JsonWriterSettings writerSettings = new JsonWriterSettings(JsonMode.STRICT, "", ""); // most compact JSON
this.valueTransformer = (doc) -> doc.toJson(writerSettings);
this.recorder = recorder;
}
/**
* Obtain the record maker for the given table, using the specified columns and sending records to the given consumer.
*
* @param collectionId the identifier of the collection for which records are to be produced; may not be null
* @return the table-specific record maker; may be null if the table is not included in the connector
*/
public RecordsForCollection forCollection(CollectionId collectionId) {
return recordMakerByCollectionId.computeIfAbsent(collectionId, id -> {
String topicName = topicSelector.getTopic(collectionId);
return new RecordsForCollection(collectionId, source, topicName, valueTransformer, recorder);
});
}
/**
* A record producer for a given collection.
*/
public static final class RecordsForCollection {
private final CollectionId collectionId;
private final String replicaSetName;
private final SourceInfo source;
private final Map<String, ?> sourcePartition;
private final String topicName;
private final Schema keySchema;
private final Schema valueSchema;
private final Function<Document, String> valueTransformer;
private final BlockingConsumer<SourceRecord> recorder;
protected RecordsForCollection(CollectionId collectionId, SourceInfo source, String topicName,
Function<Document, String> valueTransformer, BlockingConsumer<SourceRecord> recorder) {
this.sourcePartition = source.partition(collectionId.replicaSetName());
this.collectionId = collectionId;
this.replicaSetName = this.collectionId.replicaSetName();
this.source = source;
this.topicName = topicName;
this.keySchema = SchemaBuilder.struct()
.name(topicName + ".Key")
.field("_id", Schema.STRING_SCHEMA)
.build();
this.valueSchema = SchemaBuilder.struct()
.name(topicName + ".Envelope")
.field(FieldName.AFTER, Schema.OPTIONAL_STRING_SCHEMA)
.field("patch", Schema.OPTIONAL_STRING_SCHEMA)
.field(FieldName.SOURCE, SourceInfo.SOURCE_SCHEMA)
.field(FieldName.OPERATION, Schema.OPTIONAL_STRING_SCHEMA)
.field(FieldName.TIMESTAMP, Schema.OPTIONAL_INT64_SCHEMA)
.build();
JsonWriterSettings writerSettings = new JsonWriterSettings(JsonMode.STRICT, "", ""); // most compact JSON
this.valueTransformer = (doc) -> doc.toJson(writerSettings);
this.recorder = recorder;
}
/**
* Get the identifier of the collection to which this producer applies.
*
* @return the collection ID; never null
*/
public CollectionId collectionId() {
return collectionId;
}
/**
* Generate and record one or more source records to describe the given object.
*
* @param id the identifier of the collection in which the document exists; may not be null
* @param object the document; may not be null
* @param timestamp the timestamp at which this operation is occurring
* @return the number of source records that were generated; will be 0 or more
* @throws InterruptedException if the calling thread was interrupted while waiting to submit a record to
* the blocking consumer
*/
public int recordObject(CollectionId id, Document object, long timestamp) throws InterruptedException {
final Struct sourceValue = source.lastOffsetStruct(replicaSetName, id);
final Map<String, ?> offset = source.lastOffset(replicaSetName);
String objId = objectIdLiteralFrom(object);
return createRecords(sourceValue, offset, Operation.READ, objId, object, timestamp);
}
/**
* Generate and record one or more source records to describe the given event.
*
* @param oplogEvent the event; may not be null
* @param timestamp the timestamp at which this operation is occurring
* @return the number of source records that were generated; will be 0 or more
* @throws InterruptedException if the calling thread was interrupted while waiting to submit a record to
* the blocking consumer
*/
public int recordEvent(Document oplogEvent, long timestamp) throws InterruptedException {
final Struct sourceValue = source.offsetStructForEvent(replicaSetName, oplogEvent);
final Map<String, ?> offset = source.lastOffset(replicaSetName);
Document patchObj = oplogEvent.get("o", Document.class);
// Updates have an 'o2' field, since the updated object in 'o' might not have the ObjectID ...
Object o2 = oplogEvent.get("o2");
String objId = o2 != null ? objectIdLiteral(o2) : objectIdLiteralFrom(patchObj);
assert objId != null;
Operation operation = operationLiterals.get(oplogEvent.getString("op"));
return createRecords(sourceValue, offset, operation, objId, patchObj, timestamp);
}
protected int createRecords(Struct source, Map<String, ?> offset, Operation operation, String objId, Document objectValue,
long timestamp)
throws InterruptedException {
Integer partition = null;
Struct key = keyFor(objId);
Struct value = new Struct(valueSchema);
switch (operation) {
case READ:
case CREATE:
// The object is the new document ...
String jsonStr = valueTransformer.apply(objectValue);
value.put(FieldName.AFTER, jsonStr);
break;
case UPDATE:
// The object is the idempotent patch document ...
String patchStr = valueTransformer.apply(objectValue);
value.put("patch", patchStr);
break;
case DELETE:
// The delete event has nothing of any use, other than the _id which we already have in our key.
// So put nothing in the 'after' or 'patch' fields ...
break;
}
value.put(FieldName.SOURCE, source);
value.put(FieldName.OPERATION, operation.code());
value.put(FieldName.TIMESTAMP, timestamp);
SourceRecord record = new SourceRecord(sourcePartition, offset, topicName, partition, keySchema, key, valueSchema, value);
recorder.accept(record);
if (operation == Operation.DELETE) {
// Also generate a tombstone event ...
record = new SourceRecord(sourcePartition, offset, topicName, partition, keySchema, key, null, null);
recorder.accept(record);
return 2;
}
return 1;
}
protected String objectIdLiteralFrom(Document obj) {
if (obj == null) {
return null;
}
Object id = obj.get("_id");
return objectIdLiteral(id);
}
protected String objectIdLiteral(Object id) {
if (id == null) {
return null;
}
if (id instanceof ObjectId) {
return ((ObjectId) id).toHexString();
}
if (id instanceof String) {
return (String) id;
}
if (id instanceof Document) {
return valueTransformer.apply((Document) id);
}
return id.toString();
}
protected Struct keyFor(String objId) {
return new Struct(keySchema).put("_id", objId);
}
}
/**
* Clear all of the cached record makers. This should be done when the logs are rotated, since in that a different table
* numbering scheme will be used by all subsequent TABLE_MAP binlog events.
*/
public void clear() {
logger.debug("Clearing table converters");
recordMakerByCollectionId.clear();
}
}

View File

@ -0,0 +1,186 @@
/*
* 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.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import com.mongodb.ServerAddress;
import io.debezium.annotation.Immutable;
@Immutable
public final class ReplicaSet implements Comparable<ReplicaSet> {
/**
* Regular expression that extracts the hosts for the replica sets. The raw expression is
* {@code ((([^=]+)[=])?(([^/]+)\/))?(.+)}.
*/
private static final Pattern HOST_PATTERN = Pattern.compile("((([^=]+)[=])?(([^/]+)\\/))?(.+)");
/**
* Parse the supplied string for the information about the hosts for a replica set. The string is a shard host
* specification (e.g., "{@code shard01=replicaSet1/host1:27017,host2:27017}"), replica set hosts (e.g.,
* "{@code replicaSet1/host1:27017,host2:27017}"), or standalone host (e.g., "{@code host1:27017}" or
* "{@code 1.2.3.4:27017}").
*
* @param hosts the hosts string; may be null
* @return the replica set; or {@code null} if the host string could not be parsed
*/
public static ReplicaSet parse(String hosts) {
if (hosts != null) {
Matcher matcher = HOST_PATTERN.matcher(hosts);
if (matcher.matches()) {
String shard = matcher.group(3);
String replicaSetName = matcher.group(5);
String host = matcher.group(6);
if (host != null && host.trim().length() != 0) {
return new ReplicaSet(host, replicaSetName, shard);
}
}
}
return null;
}
private final List<ServerAddress> addresses;
private final String replicaSetName;
private final String shardName;
private final int hc;
public ReplicaSet(String addresses, String replicaSetName, String shardName) {
this.addresses = MongoUtil.parseAddresses(addresses);
this.addresses.sort(ReplicaSet::compareServerAddresses);
this.replicaSetName = replicaSetName != null ? replicaSetName.trim() : null;
this.shardName = shardName != null ? shardName.trim() : null;
this.hc = addresses.hashCode();
}
/**
* Get the immutable list of server addresses.
*
* @return the server addresses; never null
*/
public List<ServerAddress> addresses() {
return addresses;
}
/**
* Get the name of this replica set.
*
* @return the replica set name, or {@code null} if the addresses are for standalone servers.
*/
public String replicaSetName() {
return replicaSetName;
}
/**
* Get the shard name for this replica set.
*
* @return the shard name, or {@code null} if this replica set is not used as a shard
*/
public String shardName() {
return shardName;
}
/**
* Return whether the address(es) represent a standalone server, where the {@link #replicaSetName() replica set name} is
* {@code null}. This method returns the opposite of {@link #hasReplicaSetName()}.
*
* @return {@code true} if this represents the address of a standalone server, or {@code false} if it represents the
* address of a replica set
* @see #hasReplicaSetName()
*/
public boolean isStandaloneServer() {
return replicaSetName == null;
}
/**
* Return whether the address(es) represents a replica set, where the {@link #replicaSetName() replica set name} is
* not {@code null}. This method returns the opposite of {@link #isStandaloneServer()}.
*
* @return {@code true} if this represents the address of a replica set, or {@code false} if it represents the
* address of a standalone server
* @see #isStandaloneServer()
*/
public boolean hasReplicaSetName() {
return replicaSetName != null;
}
@Override
public int hashCode() {
return hc;
}
@Override
public boolean equals(Object obj) {
if (obj == this) return true;
if (obj instanceof ReplicaSet) {
ReplicaSet that = (ReplicaSet) obj;
return Objects.equals(this.shardName, that.shardName) && Objects.equals(this.replicaSetName, that.replicaSetName) &&
this.addresses.equals(that.addresses);
}
return false;
}
@Override
public int compareTo(ReplicaSet that) {
if (that == this) return 0;
int diff = compareNullable(this.shardName, that.shardName);
if (diff != 0) return diff;
diff = compareNullable(this.replicaSetName, that.replicaSetName);
if (diff != 0) return diff;
Iterator<ServerAddress> thisIter = this.addresses.iterator();
Iterator<ServerAddress> thatIter = that.addresses.iterator();
while (thisIter.hasNext() && thatIter.hasNext()) {
diff = compare(thisIter.next(), thatIter.next());
if (diff != 0) return diff;
}
if (thisIter.hasNext()) return 1;
if (thatIter.hasNext()) return -1;
return 0;
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
if (this.shardName != null && !this.shardName.isEmpty()) {
sb.append(shardName).append('=');
}
if (this.replicaSetName != null && !this.replicaSetName.isEmpty()) {
sb.append(replicaSetName).append('/');
}
Iterator<ServerAddress> iter = addresses.iterator();
if (iter.hasNext()) sb.append(MongoUtil.toString(iter.next()));
while (iter.hasNext()) {
sb.append(',').append(MongoUtil.toString(iter.next()));
}
return sb.toString();
}
protected static int compareServerAddresses(ServerAddress one, ServerAddress two) {
if (one == two) return 0;
if (one == null) return two == null ? 0 : -1;
if (two == null) return 1;
return compare(one, two);
}
protected static int compareNullable(String str1, String str2) {
if (str1 == str2) return 0;
if (str1 == null) return str2 == null ? 0 : -1;
if (str2 == null) return 1;
return str1.compareTo(str2);
}
protected static int compare(ServerAddress address1, ServerAddress address2) {
int diff = address1.getHost().compareTo(address2.getHost());
if (diff != 0) return diff;
return address1.getPort() - address2.getPort();
}
}

View File

@ -0,0 +1,93 @@
/*
* 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.util.HashSet;
import java.util.Set;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.mongodb.MongoClient;
import com.mongodb.ReplicaSetStatus;
import io.debezium.annotation.ThreadSafe;
import io.debezium.util.Strings;
/**
* A component that monitors a single replica set or the set of replica sets that make up the shards in a sharded cluster.
*
* @author Randall Hauch
*/
@ThreadSafe
public class ReplicaSetDiscovery {
/**
* The database that might be used to check for replica set information in a sharded cluster.
*/
public static final String CONFIG_DATABASE_NAME = "config";
/**
* The database that might be used to check for member information in a replica set.
*/
public static final String ADMIN_DATABASE_NAME = "admin";
private final Logger logger = LoggerFactory.getLogger(getClass());
private final ReplicationContext context;
private final String seedAddresses;
/**
* Create a cluster component.
*
* @param context the replication context; may not be null
*/
public ReplicaSetDiscovery(ReplicationContext context) {
this.context = context;
this.seedAddresses = context.hosts();
}
/**
* Connect to the shard cluster or replica set defined by the seed addresses, and obtain the specifications for each of the
* replica sets.
*
* @return the information about the replica sets; never null but possibly empty
*/
public ReplicaSets getReplicaSets() {
MongoClient client = context.clientFor(seedAddresses);
Set<ReplicaSet> replicaSetSpecs = new HashSet<>();
// First see if the addresses are for a config server replica set ...
MongoUtil.onCollectionDocuments(client, CONFIG_DATABASE_NAME, "shards", doc -> {
logger.info("Checking shard details from configuration replica set {}", seedAddresses);
String shardName = doc.getString("_id");
String hostStr = doc.getString("host");
String replicaSetName = MongoUtil.replicaSetUsedIn(hostStr);
replicaSetSpecs.add(new ReplicaSet(hostStr, replicaSetName, shardName));
});
if (replicaSetSpecs.isEmpty()) {
// The addresses may be a replica set ...
ReplicaSetStatus rsStatus = client.getReplicaSetStatus();
logger.info("Checking current members of replica set at {}", seedAddresses);
if (rsStatus != null) {
// This is a replica set ...
String addressStr = Strings.join(",", client.getServerAddressList());
String replicaSetName = rsStatus.getName();
replicaSetSpecs.add(new ReplicaSet(addressStr, replicaSetName, null));
} else {
logger.debug("Found standalone MongoDB replica set at {}", seedAddresses);
// We aren't connecting to it as a replica set (likely not using auto-discovery of members),
// but we can't monitor standalone servers unless they really are replica sets. We already know
// that we're not connected to a config server replica set, so any replica set name from the seed addresses
// is almost certainly our replica set name ...
String replicaSetName = MongoUtil.replicaSetUsedIn(seedAddresses);
for (String address : seedAddresses.split(",")) {
replicaSetSpecs.add(new ReplicaSet(address, replicaSetName, null));
}
}
}
return new ReplicaSets(replicaSetSpecs);
}
}

View File

@ -0,0 +1,120 @@
/*
* 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.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
import java.util.function.Supplier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.util.Clock;
import io.debezium.util.Metronome;
/**
* A thread that can be used to when new replica sets are added or existing replica sets are removed. The logic does not evaluate
* membership changes of individual replica sets, since that is handled independently by each task.
*
* @author Randall Hauch
*/
public final class ReplicaSetMonitorThread extends Thread {
private final Logger logger = LoggerFactory.getLogger(getClass());
private final Metronome metronome;
private final AtomicBoolean running = new AtomicBoolean(false);
private final CountDownLatch initialized = new CountDownLatch(1);
private final Supplier<ReplicaSets> monitor;
private final Consumer<ReplicaSets> onChange;
private final Runnable onStartup;
private volatile ReplicaSets replicaSets = ReplicaSets.empty();
/**
* @param monitor the component used to periodically obtain the replica set specifications; may not be null
* @param period the time period between polling checks; must be non-negative
* @param unit the time unit for the {@code period}; may not be null
* @param clock the clock to use; may be null if the system clock should be used
* @param onStartup the function to call when the thread is started; may be null if not needed
* @param onChange the function to call when the set of replica set specifications has changed; may be null if not needed
*/
public ReplicaSetMonitorThread(Supplier<ReplicaSets> monitor, long period, TimeUnit unit, Clock clock, Runnable onStartup,
Consumer<ReplicaSets> onChange) {
if (clock == null) clock = Clock.system();
this.monitor = monitor;
this.metronome = Metronome.sleeper(period, unit, clock);
this.onChange = onChange != null ? onChange : (rsSpecs) -> {};
this.onStartup = onStartup != null ? onStartup : () -> {};
}
@Override
public void run() {
if (running.compareAndSet(false, true)) {
// We were not running, but we are now ...
onStartup.run();
while (running.get()) {
try {
ReplicaSets previousReplicaSets = replicaSets;
replicaSets = monitor.get();
initialized.countDown();
// Determine if any replica set specifications have changed ...
if (replicaSets.haveChangedSince(previousReplicaSets)) {
// At least one of the replica sets been added or been removed ...
try {
onChange.accept(replicaSets);
} catch (Throwable t) {
logger.error("Error while calling the function with the new replica set specifications", t);
}
}
} catch (Throwable t) {
logger.error("Error while trying to get information about the replica sets", t);
}
// Check again whether we are running before we pause ...
if (running.get()) {
try {
metronome.pause();
} catch (InterruptedException e) {
Thread.interrupted();
}
}
}
}
}
/**
* Requests that this thread stop running.
*/
public void shutdown() {
if (running.compareAndSet(true, false)) {
logger.debug("Stopping the thread monitoring replica sets");
// We were running, so interrupt the thread if it is paused ...
try {
this.interrupt();
} catch (Throwable t) {
logger.warn("Unable to interrupt the thread monitoring replica sets", t);
}
}
}
/**
* Get the information about each of the replica sets.
*
* @param timeout the time to block until the replica sets are first obtained from MongoDB; may not be negative
* @param unit the time unit for the {@code timeout}; may not be null
* @return the replica sets, or {@code null} if the timeout occurred before the replica set information was obtained
*/
public ReplicaSets getReplicaSets(long timeout, TimeUnit unit) {
try {
if (initialized.await(timeout, unit)) {
return replicaSets;
}
} catch (InterruptedException e) {
Thread.interrupted(); // but do nothing else
}
return null;
}
}

View File

@ -0,0 +1,194 @@
/*
* 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.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Consumer;
import org.apache.kafka.connect.util.ConnectorUtils;
import io.debezium.annotation.Immutable;
import io.debezium.util.Strings;
/**
* A set of replica set specifications.
*
* @author Randall Hauch
*/
@Immutable
public class ReplicaSets {
/**
* Parse the supplied string for the information about the replica set hosts. The string is a semicolon-delimited list of
* shard hosts (e.g., "{@code shard01=replicaSet1/host1:27017,host2:27017}"), replica set hosts (e.g.,
* "{@code replicaSet1/host1:27017,host2:27017}"), and standalone hosts (e.g., "{@code host1:27017}" or
* "{@code 1.2.3.4:27017}").
*
* @param hosts the hosts string; may be null
* @return the replica sets; never null but possibly empty
* @see ReplicaSets#hosts()
*/
public static ReplicaSets parse(String hosts) {
Set<ReplicaSet> replicaSets = new HashSet<>();
if (hosts != null) {
for (String replicaSetStr : hosts.trim().split("[;]")) {
if (!replicaSetStr.isEmpty()) {
ReplicaSet replicaSet = ReplicaSet.parse(replicaSetStr);
if (replicaSetStr != null) {
replicaSets.add(replicaSet);
}
}
}
}
return new ReplicaSets(replicaSets);
}
/**
* Get an instance that contains no replica sets.
*
* @return the empty instance; never null
*/
public static ReplicaSets empty() {
return new ReplicaSets(null);
}
private final Map<String, ReplicaSet> replicaSetsByName = new HashMap<>();
private final List<ReplicaSet> nonReplicaSets = new ArrayList<>();
/**
* Create a set of replica set specifications.
*
* @param rsSpecs the replica set specifications; may be null or empty
*/
public ReplicaSets(Collection<ReplicaSet> rsSpecs) {
if (rsSpecs != null) {
rsSpecs.forEach(replicaSet -> {
if (replicaSet.hasReplicaSetName()) {
replicaSetsByName.put(replicaSet.replicaSetName(), replicaSet);
} else {
nonReplicaSets.add(replicaSet);
}
});
}
Collections.sort(nonReplicaSets);
}
/**
* Get the number of replica sets.
*
* @return the replica set count
*/
public int replicaSetCount() {
return replicaSetsByName.size() + nonReplicaSets.size();
}
/**
* Perform the supplied function on each of the replica sets
*
* @param function the consumer function; may not be null
*/
public void onEachReplicaSet(Consumer<ReplicaSet> function) {
this.replicaSetsByName.values().forEach(function);
this.nonReplicaSets.forEach(function);
}
/**
* Subdivide this collection of replica sets into the maximum number of groups.
*
* @param maxSubdivisionCount the maximum number of subdivisions
* @param subdivisionConsumer the function to be called with each subdivision; may not be null
*/
public void subdivide(int maxSubdivisionCount, Consumer<ReplicaSets> subdivisionConsumer) {
int numGroups = Math.min(replicaSetCount(), maxSubdivisionCount);
if (numGroups <= 1) {
// Just one replica set or subdivision ...
subdivisionConsumer.accept(this);
return;
}
ConnectorUtils.groupPartitions(all(), numGroups).forEach(rsList -> {
subdivisionConsumer.accept(new ReplicaSets(rsList));
});
}
/**
* Get a copy of all of the {@link ReplicaSet} objects.
*
* @return the replica set objects; never null but possibly empty
*/
public List<ReplicaSet> all() {
List<ReplicaSet> replicaSets = new ArrayList<>();
replicaSets.addAll(replicaSetsByName.values());
replicaSets.addAll(nonReplicaSets);
return replicaSets;
}
/**
* Determine if one or more replica sets has been added or removed since the prior state.
*
* @param priorState the prior state of the replica sets; may be null
* @return {@code true} if the replica sets have changed since the prior state, or {@code false} otherwise
*/
public boolean haveChangedSince(ReplicaSets priorState) {
if (priorState.replicaSetCount() != this.replicaSetCount()) {
// At least one replica set has been added or removed ...
return true;
}
if (this.replicaSetsByName.size() != priorState.replicaSetsByName.size()) {
// The total number of replica sets hasn't changed, but the number of named replica sets has changed ...
return true;
}
// We have the same number of named replica sets ...
if (!this.replicaSetsByName.isEmpty()) {
if (!this.replicaSetsByName.keySet().equals(priorState.replicaSetsByName.keySet())) {
// The replica sets have different names ...
return true;
}
// Otherwise, they have the same names and we don't care about the members ...
}
// None of the named replica sets has changed, so we have no choice to be compare the non-replica set members ...
return this.nonReplicaSets.equals(priorState.nonReplicaSets) ? false : true;
}
/**
* Get the string containing the host names for the replica sets. The result is a string with each replica set hosts
* separated by a semicolon.
*
* @return the host names; never null
* @see #parse(String)
*/
public String hosts() {
return Strings.join(";", all());
}
@Override
public int hashCode() {
return Objects.hash(replicaSetsByName, nonReplicaSets);
}
@Override
public boolean equals(Object obj) {
if (obj == this) return true;
if (obj instanceof ReplicaSets) {
ReplicaSets that = (ReplicaSets) obj;
return this.replicaSetsByName.equals(that.replicaSetsByName) && this.nonReplicaSets.equals(that.nonReplicaSets);
}
return false;
}
@Override
public String toString() {
return hosts();
}
}

View File

@ -0,0 +1,380 @@
/*
* 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.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Predicate;
import java.util.function.Supplier;
import org.apache.kafka.connect.errors.ConnectException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.mongodb.MongoClient;
import com.mongodb.MongoCredential;
import com.mongodb.ReplicaSetStatus;
import com.mongodb.ServerAddress;
import io.debezium.config.Configuration;
import io.debezium.function.BlockingConsumer;
import io.debezium.util.Clock;
import io.debezium.util.DelayStrategy;
import io.debezium.util.LoggingContext;
import io.debezium.util.LoggingContext.PreviousContext;
/**
* @author Randall Hauch
*
*/
public class ReplicationContext {
private final Logger logger = LoggerFactory.getLogger(getClass());
private final Configuration config;
private final MongoClients pool;
private final Filters filters;
private final SourceInfo source;
private final ReplicaSets replicaSets;
private final DelayStrategy primaryBackoffStrategy;
private final Clock clock = Clock.system();
private final TopicSelector topicSelector;
private final boolean useHostsAsSeeds;
/**
* @param config the configuration
*/
public ReplicationContext(Configuration config) {
this.config = config;
this.useHostsAsSeeds = config.getBoolean(MongoDbConnectorConfig.AUTO_DISCOVER_MEMBERS);
final String username = config.getString(MongoDbConnectorConfig.USER);
final String password = config.getString(MongoDbConnectorConfig.PASSWORD);
final String configDbName = ReplicaSetDiscovery.CONFIG_DATABASE_NAME;
final String adminDbName = ReplicaSetDiscovery.ADMIN_DATABASE_NAME;
final String serverName = config.getString(MongoDbConnectorConfig.LOGICAL_NAME);
// Set up the client pool so that it ...
MongoClients.Builder clientBuilder = MongoClients.create();
if (username != null || password != null) {
clientBuilder.withCredential(MongoCredential.createCredential(username, configDbName, password.toCharArray()));
clientBuilder.withCredential(MongoCredential.createCredential(username, adminDbName, password.toCharArray()));
}
pool = clientBuilder.build();
this.replicaSets = ReplicaSets.parse(hosts());
this.filters = new Filters(config);
this.source = new SourceInfo(serverName);
final int initialDelayInMs = config.getInteger(MongoDbConnectorConfig.CONNECT_BACKOFF_INITIAL_DELAY_MS);
final long maxDelayInMs = config.getLong(MongoDbConnectorConfig.CONNECT_BACKOFF_MAX_DELAY_MS);
this.primaryBackoffStrategy = DelayStrategy.exponential(initialDelayInMs, maxDelayInMs);
this.topicSelector = TopicSelector.defaultSelector(serverName);
}
public void shutdown() {
try {
// Closing all connections ...
logger.info("Closing all connections to {}", replicaSets);
pool.clear();
} catch (Throwable e) {
logger.error("Unexpected error shutting down the MongoDB clients", e);
}
}
public TopicSelector topicSelector() {
return topicSelector;
}
public MongoClients clients() {
return pool;
}
public Predicate<CollectionId> collectionFilter() {
return filters.collectionFilter();
}
public SourceInfo source() {
return source;
}
public ReplicaSets replicaSets() {
return replicaSets;
}
public boolean performSnapshotEvenIfNotNeeded() {
return false;
}
public MongoClient clientForReplicaSet(ReplicaSet replicaSet) {
return clientFor(replicaSet.addresses());
}
public MongoClient clientFor(String seedAddresses) {
List<ServerAddress> addresses = MongoUtil.parseAddresses(seedAddresses);
return clientFor(addresses);
}
public MongoClient clientFor(List<ServerAddress> addresses) {
if ( this.useHostsAsSeeds || addresses.isEmpty() ) {
return pool.clientForMembers(addresses);
}
return pool.clientFor(addresses.get(0));
}
public String hosts() {
return config.getString(MongoDbConnectorConfig.HOSTS);
}
public int pollPeriodInSeconds() {
return config.getInteger(MongoDbConnectorConfig.POLL_INTERVAL_SEC);
}
public int maxConnectionAttemptsForPrimary() {
return config.getInteger(MongoDbConnectorConfig.MAX_FAILED_CONNECTIONS);
}
public int maxNumberOfCopyThreads() {
return config.getInteger(MongoDbConnectorConfig.MAX_COPY_THREADS);
}
public String serverName() {
return config.getString(MongoDbConnectorConfig.LOGICAL_NAME);
}
public Clock clock() {
return clock;
}
/**
* Obtain a client that will repeated try to obtain a client to the primary node of the replica set, waiting (and using
* this context's back-off strategy) if required until the primary becomes available.
*
* @param replicaSet the replica set information; may not be null
* @param errorHandler the function to be called whenever the primary is unable to
* {@link MongoPrimary#execute(String, Consumer) execute} an operation to completion; may be null
* @return the client, or {@code null} if no primary could be found for the replica set
*/
public ReplicationContext.MongoPrimary primaryFor(ReplicaSet replicaSet, BiConsumer<String, Throwable> errorHandler) {
return new ReplicationContext.MongoPrimary(this, replicaSet, errorHandler);
}
/**
* Obtain a client that will repeated try to obtain a client to the primary node of the replica set, waiting (and using
* this context's back-off strategy) if required until the primary becomes available.
*
* @param replicaSet the replica set information; may not be null
* @return the client, or {@code null} if no primary could be found for the replica set
*/
protected Supplier<MongoClient> primaryClientFor(ReplicaSet replicaSet) {
return primaryClientFor(replicaSet, (attempts, remaining, error) -> {
if (error == null) {
logger.info("Unable to connect to primary node of '{}' after attempt #{} ({} remaining)", replicaSet, attempts, remaining);
} else {
logger.error("Error while attempting to connect to primary node of '{}' after attempt #{} ({} remaining): {}", replicaSet,
attempts, remaining, error.getMessage(), error);
}
});
}
/**
* Obtain a client that will repeated try to obtain a client to the primary node of the replica set, waiting (and using
* this context's back-off strategy) if required until the primary becomes available.
*
* @param replicaSet the replica set information; may not be null
* @param handler the function that will be called when the primary could not be obtained; may not be null
* @return the client, or {@code null} if no primary could be found for the replica set
*/
protected Supplier<MongoClient> primaryClientFor(ReplicaSet replicaSet, PrimaryConnectFailed handler) {
Supplier<MongoClient> factory = () -> clientForPrimary(replicaSet);
int maxAttempts = maxConnectionAttemptsForPrimary();
return () -> {
int attempts = 0;
MongoClient primary = null;
while (primary == null) {
++attempts;
try {
// Try to get the primary
primary = factory.get();
if (primary != null) break;
} catch (Throwable t) {
handler.failed(attempts, maxAttempts - attempts, t);
}
if (attempts > maxAttempts) {
throw new ConnectException("Unable to connect to primary node of '" + replicaSet + "' after " +
attempts + " failed attempts");
}
handler.failed(attempts, maxAttempts - attempts, null);
primaryBackoffStrategy.sleepWhen(true);
continue;
}
return primary;
};
}
@FunctionalInterface
public static interface PrimaryConnectFailed {
void failed(int attemptNumber, int attemptsRemaining, Throwable error);
}
/**
* A supplier of a client that connects only to the primary of a replica set. Operations on the primary will continue
*/
public static class MongoPrimary {
private final ReplicaSet replicaSet;
private final Supplier<MongoClient> primaryConnectionSupplier;
private final BiConsumer<String, Throwable> errorHandler;
protected MongoPrimary(ReplicationContext context, ReplicaSet replicaSet, BiConsumer<String, Throwable> errorHandler) {
this.replicaSet = replicaSet;
this.primaryConnectionSupplier = context.primaryClientFor(replicaSet);
this.errorHandler = errorHandler;
}
/**
* Get the replica set.
*
* @return the replica set; never null
*/
public ReplicaSet replicaSet() {
return replicaSet;
}
/**
* Get the address of the primary node, if there is one.
*
* @return the address of the replica set's primary node, or {@code null} if there is currently no primary
*/
public ServerAddress address() {
AtomicReference<ServerAddress> address = new AtomicReference<>();
execute("get replica set primary", primary -> {
ReplicaSetStatus rsStatus = primary.getReplicaSetStatus();
if (rsStatus != null) {
address.set(rsStatus.getMaster());
}
});
return address.get();
}
/**
* Execute the supplied operation using the primary, blocking until a primary is available. Whenever the operation stops
* (e.g., if the primary is no longer primary), then restart the operation using the current primary.
*
* @param desc the description of the operation, for logging purposes
* @param operation the operation to be performed on the primary.
*/
public void execute(String desc, Consumer<MongoClient> operation) {
while (true) {
MongoClient primary = primaryConnectionSupplier.get();
try {
operation.accept(primary);
return;
} catch (Throwable t) {
errorHandler.accept(desc, t);
}
}
}
/**
* Execute the supplied operation using the primary, blocking until a primary is available. Whenever the operation stops
* (e.g., if the primary is no longer primary), then restart the operation using the current primary.
*
* @param desc the description of the operation, for logging purposes
* @param operation the operation to be performed on the primary.
* @throws InterruptedException if the operation was interrupted
*/
public void executeBlocking(String desc, BlockingConsumer<MongoClient> operation) throws InterruptedException {
while (true) {
MongoClient primary = primaryConnectionSupplier.get();
try {
operation.accept(primary);
return;
} catch (Throwable t) {
errorHandler.accept(desc, t);
}
}
}
/**
* Use the primary to get the names of all the databases in the replica set. This method will block until
* a primary can be obtained to get the names of all databases in the replica set.
*
* @return the database names; never null but possibly empty
*/
public Set<String> databaseNames() {
Set<String> databaseNames = new HashSet<>();
execute("get database names", primary -> {
databaseNames.clear(); // in case we restarted
MongoUtil.forEachDatabaseName(primary, databaseNames::add);
});
return databaseNames;
}
/**
* Use the primary to get the identifiers of all the collections in the replica set. This method will block until
* a primary can be obtained to get the identifiers of all collections in the replica set.
*
* @return the collection identifiers; never null
*/
public List<CollectionId> collections() {
String replicaSetName = replicaSet.replicaSetName();
// For each database, get the list of collections ...
List<CollectionId> collections = new ArrayList<>();
execute("get collections in databases", primary -> {
collections.clear(); // in case we restarted
Set<String> databaseNames = databaseNames();
MongoUtil.forEachDatabaseName(primary, databaseNames::add);
databaseNames.forEach(dbName -> {
MongoUtil.forEachCollectionNameInDatabase(primary, dbName, collectionName -> {
collections.add(new CollectionId(replicaSetName, dbName, collectionName));
});
});
});
return collections;
}
}
/**
* Obtain a client that talks only to the primary node of the replica set.
*
* @param replicaSet the replica set information; may not be null
* @return the client, or {@code null} if no primary could be found for the replica set
*/
protected MongoClient clientForPrimary(ReplicaSet replicaSet) {
MongoClient replicaSetClient = clientForReplicaSet(replicaSet);
ReplicaSetStatus rsStatus = replicaSetClient.getReplicaSetStatus();
if (rsStatus == null) {
if ( !this.useHostsAsSeeds ) {
// No replica set status is available, but it may still be a replica set ...
return replicaSetClient;
}
// This is not a replica set, so there will be no oplog to read ...
throw new ConnectException("The MongoDB server(s) at '" + replicaSet +
"' is not a valid replica set and cannot be used");
}
// It is a replica set ...
ServerAddress primaryAddress = rsStatus.getMaster();
if (primaryAddress != null) {
return pool.clientFor(primaryAddress);
}
return null;
}
/**
* Configure the logger's Mapped Diagnostic Context (MDC) properties for the thread making this call.
*
* @param contextName the name of the context; may not be null
* @return the previous MDC context; never null
* @throws IllegalArgumentException if {@code contextName} is null
*/
public PreviousContext configureLoggingContext(String contextName) {
return LoggingContext.forConnector("MongoDB", serverName(), contextName);
}
}

View File

@ -0,0 +1,426 @@
/*
* 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.util.ArrayList;
import java.util.List;
import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Predicate;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.source.SourceRecord;
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.MongoClient;
import com.mongodb.ServerAddress;
import com.mongodb.client.FindIterable;
import com.mongodb.client.MongoCollection;
import com.mongodb.client.MongoCursor;
import com.mongodb.client.MongoDatabase;
import com.mongodb.client.model.Filters;
import io.debezium.annotation.ThreadSafe;
import io.debezium.connector.mongodb.RecordMakers.RecordsForCollection;
import io.debezium.function.BlockingConsumer;
import io.debezium.util.Clock;
import io.debezium.util.Strings;
/**
* A component that replicates the content of a replica set, starting with an initial sync or continuing to read the oplog where
* it last left off.
*
* <h2>Initial Sync</h2>
* If no offsets have been recorded for this replica set, replication begins with an
* <a href="https://docs.mongodb.com/manual/core/replica-set-sync/#initial-sync">MongoDB initial sync</a> of the replica set.
* <p>
* The logic used in this component to perform an initial sync is similar to that of the
* <a href="https://github.com/mongodb/mongo/blob/master/src/mongo/db/repl/rs_initialsync.cpp">official initial sync</a>
* functionality used by secondary MongoDB nodes, although our logic can be simpler since we are not storing state nor building
* indexes:
* <ol>
* <li>Read the primary node's current oplog time. This is our <em>start time</em>.</li>
* <li>Clone all of the databases and collections from the primary node, using multiple threads. This steps is completed only
* after <em>all</em> collections are successfully copied.</li>
* <li>Start reading the primary node's oplog from <em>start time</em> and applying the changes.</li>
* </ol>
* <p>
* It is important to understand that step 2 is not performing a consistent snapshot. That means that once we start copying a
* collection, clients can make changes and we may or may not see those changes in our copy. However, this is not a problem
* because the MongoDB replication process -- and our logic -- relies upon the fact that every change recorded in the MongoDB
* oplog is <a href="https://docs.mongodb.com/manual/core/replica-set-oplog/">idempotent</a>. So, as long as we read the oplog
* from the same point in time (or earlier) than we <em>started</em> our copy operation, and apply <em>all</em> of the changes
* <em>in the same order</em>, then the state of all documents described by this connector will be the same.
*
* <h2>Restart</h2>
* If prior runs of the replicator have recorded offsets in the {@link ReplicationContext#source() source information}, then
* when the replicator starts it will simply start reading the primary's oplog starting at the same point it last left off.
*
* <h2>Handling problems</h2>
* <p>
* This replicator does each of its tasks using a connection to the primary. If the replicator is not able to establish a
* connection to the primary (e.g., there is no primary, or the replicator cannot communicate with the primary), the replicator
* will continue to try to establish a connection, using an exponential back-off strategy to prevent saturating the system.
* After a {@link ReplicationContext#maxConnectionAttemptsForPrimary() configurable} number of failed attempts, the replicator
* will fail by throwing a {@link ConnectException}.
*
* @author Randall Hauch
*/
@ThreadSafe
public class Replicator {
private final Logger logger = LoggerFactory.getLogger(getClass());
private final ReplicationContext context;
private final ExecutorService copyThreads;
private final ReplicaSet replicaSet;
private final String rsName;
private final AtomicBoolean running = new AtomicBoolean();
private final SourceInfo source;
private final RecordMakers recordMakers;
private final Predicate<CollectionId> collectionFilter;
private final Clock clock;
private ReplicationContext.MongoPrimary primaryClient;
/**
* @param context the replication context; may not be null
* @param replicaSet the replica set to be replicated; may not be null
* @param recorder the recorder for source record produced by this replicator; may not be null
*/
public Replicator(ReplicationContext context, ReplicaSet replicaSet, BlockingConsumer<SourceRecord> recorder) {
assert context != null;
assert replicaSet != null;
assert recorder != null;
this.context = context;
this.source = context.source();
this.replicaSet = replicaSet;
this.rsName = replicaSet.replicaSetName();
this.copyThreads = Executors.newFixedThreadPool(context.maxNumberOfCopyThreads());
this.recordMakers = new RecordMakers(this.source, context.topicSelector(), recorder);
this.collectionFilter = this.context.collectionFilter();
this.clock = this.context.clock();
}
/**
* Stop the replication from running.
* <p>
* This method does nothing if the snapshot is not running
*/
public void stop() {
this.copyThreads.shutdownNow();
}
/**
* Perform the replication logic. This can be run once.
*/
public void run() {
if (this.running.compareAndSet(false, true)) {
try {
if (establishConnectionToPrimary()) {
if (isInitialSyncExpected()) {
recordCurrentOplogPosition();
if (!performInitialSync()) {
return;
}
}
readOplog();
}
} finally {
this.running.set(false);
}
}
}
/**
* Establish a connection to the primary.
*
* @return {@code true} if a connection was established, or {@code false} otherwise
*/
protected boolean establishConnectionToPrimary() {
logger.info("Connecting to '{}'", replicaSet);
primaryClient = context.primaryFor(replicaSet, (desc, error) -> {
logger.error("Error while attempting to {}: {}", desc, error.getMessage(), error);
});
return primaryClient != null;
}
/**
* Obtain the current position of the oplog, and record it in the source.
*/
protected void recordCurrentOplogPosition() {
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
source.offsetStructForEvent(replicaSet.replicaSetName(), last);
});
}
/**
* Determine if an initial sync should be performed. An initial sync is expected if the {@link #source} has no previously
* recorded offsets for this replica set, or if {@link ReplicationContext#performSnapshotEvenIfNotNeeded() a snapshot should
* always be performed}.
*
* @return {@code true} if the initial sync should be performed, or {@code false} otherwise
*/
protected boolean isInitialSyncExpected() {
boolean performSnapshot = true;
if (source.hasOffset(rsName)) {
logger.info("Found existing offset for replica set '{}' at {}", rsName, source.lastOffset(rsName));
performSnapshot = false;
if (context.performSnapshotEvenIfNotNeeded()) {
logger.info("Configured to performing initial sync of replica set '{}'", rsName);
performSnapshot = true;
} else {
// Look to see if our last recorded offset still exists in the oplog.
BsonTimestamp lastRecordedTs = source.lastOffsetTimestamp(rsName);
AtomicReference<BsonTimestamp> firstExistingTs = new AtomicReference<>();
primaryClient.execute("get oplog position", primary -> {
MongoCollection<Document> oplog = primary.getDatabase("local").getCollection("oplog.rs");
Document firstEvent = oplog.find().sort(new Document("$natural", 1)).limit(1).first(); // may be null
firstExistingTs.set(SourceInfo.extractEventTimestamp(firstEvent));
});
BsonTimestamp firstAvailableTs = firstExistingTs.get();
if ( firstAvailableTs == null ) {
logger.info("The oplog contains no entries, so performing initial sync of replica set '{}'", rsName);
performSnapshot = true;
} else if ( lastRecordedTs.compareTo(firstAvailableTs) < 0 ) {
// The last recorded timestamp is *before* the first existing oplog event, which means there is
// almost certainly some history lost since we last processed the oplog ...
logger.info("Initial sync is required since the oplog for replica set '{}' starts at {}, which is later than the timestamp of the last offset {}",
rsName, firstAvailableTs, lastRecordedTs);
performSnapshot = true;
}
}
} else {
logger.info("No existing offset found for replica set '{}', starting initial sync", rsName);
performSnapshot = true;
}
return performSnapshot;
}
/**
* Perform the initial sync of the collections in the replica set.
*
* @return {@code true} if the initial sync was completed, or {@code false} if it was stopped for any reason
*/
protected boolean performInitialSync() {
logger.info("Beginning initial sync of '{}' at {}", rsName, source.lastOffset(rsName));
source.startInitialSync(replicaSet.replicaSetName());
// Get the current timestamp of this processor ...
final long syncStart = clock.currentTimeInMillis();
// We need to copy each collection, so put the collection IDs into a queue ...
final List<CollectionId> collections = new ArrayList<>();
primaryClient.collections().forEach(id -> {
if (collectionFilter.test(id)) collections.add(id);
});
final Queue<CollectionId> collectionsToCopy = new ConcurrentLinkedQueue<>(collections);
final int numThreads = Math.min(collections.size(), context.maxNumberOfCopyThreads());
final CountDownLatch latch = new CountDownLatch(numThreads);
final AtomicBoolean aborted = new AtomicBoolean(false);
final AtomicInteger replicatorThreadCounter = new AtomicInteger(0);
final AtomicInteger numCollectionsCopied = new AtomicInteger();
final AtomicLong numDocumentsCopied = new AtomicLong();
// And start threads to pull collection IDs from the queue and perform the copies ...
logger.info("Preparing to use {} thread(s) to sync {} collection(s): {}",
numThreads, collections.size(), Strings.join(", ", collections));
for (int i = 0; i != numThreads; ++i) {
copyThreads.submit(() -> {
context.configureLoggingContext(replicaSet.replicaSetName() + "-sync" + replicatorThreadCounter.incrementAndGet());
// Continue to pull a collection ID and copy the collection ...
try {
CollectionId id = null;
while (!aborted.get() && (id = collectionsToCopy.poll()) != null) {
long start = clock.currentTimeInMillis();
logger.info("Starting initial sync of '{}'", id);
long numDocs = copyCollection(id, syncStart);
numCollectionsCopied.incrementAndGet();
numDocumentsCopied.addAndGet(numDocs);
long duration = clock.currentTimeInMillis() - start;
logger.info("Completing initial sync of {} documents from '{}' in {}", numDocs, id, Strings.duration(duration));
}
} catch (InterruptedException e) {
// Do nothing so that this thread is terminated ...
aborted.set(true);
} finally {
latch.countDown();
}
});
}
// Wait for all of the threads to complete ...
try {
latch.await();
} catch (InterruptedException e) {
Thread.interrupted();
aborted.set(true);
}
// Stopping the replicator does not interrupt *our* thread but does interrupt the copy threads.
// Therefore, check the aborted state here ...
long syncDuration = clock.currentTimeInMillis() - syncStart;
if (aborted.get()) {
int remaining = collections.size() - numCollectionsCopied.get();
logger.info("Initial sync aborted after {} with {} of {} collections incomplete",
Strings.duration(syncDuration), remaining, collections.size());
return false;
}
// We completed the initial sync, so record this in the source ...
source.stopInitialSync(replicaSet.replicaSetName());
logger.info("Initial sync of {} collections with a total of {} documents completed in {}",
collections.size(), numDocumentsCopied.get(), Strings.duration(syncDuration));
return true;
}
/**
* Copy the collection, sending to the recorder a record for each document.
*
* @param collectionId the identifier of the collection to be copied; may not be null
* @param timestamp the timestamp in milliseconds at which the copy operation was started
* @return number of documents that were copied
* @throws InterruptedException if the thread was interrupted while the copy operation was running
*/
protected long copyCollection(CollectionId collectionId, long timestamp) throws InterruptedException {
AtomicLong docCount = new AtomicLong();
primaryClient.executeBlocking("sync '" + collectionId + "'", primary -> {
docCount.set(copyCollection(primary, collectionId, timestamp));
});
return docCount.get();
}
/**
* Copy the collection, sending to the recorder a record for each document.
*
* @param primary the connection to the replica set's primary node; may not be null
* @param collectionId the identifier of the collection to be copied; may not be null
* @param timestamp the timestamp in milliseconds at which the copy operation was started
* @return number of documents that were copied
* @throws InterruptedException if the thread was interrupted while the copy operation was running
*/
protected long copyCollection(MongoClient primary, CollectionId collectionId, long timestamp) throws InterruptedException {
RecordsForCollection factory = recordMakers.forCollection(collectionId);
MongoDatabase db = primary.getDatabase(collectionId.dbName());
MongoCollection<Document> docCollection = db.getCollection(collectionId.name());
long counter = 0;
try (MongoCursor<Document> cursor = docCollection.find().iterator()) {
while (cursor.hasNext()) {
Document doc = cursor.next();
logger.trace("Found existing doc in {}: {}", collectionId, doc);
counter += factory.recordObject(collectionId, doc, timestamp);
}
}
return counter;
}
/**
* Repeatedly obtain a connection to the replica set's current primary and use that primary to read the oplog.
* This method will continue to run even if there are errors or problems. The method will return when a sufficient
* number of errors occur or if the replicator should stop reading the oplog. The latter occurs when a new primary
* is elected (as identified by an oplog event), of if the current thread doing the reading is interrupted.
*/
protected void readOplog() {
primaryClient.execute("read from oplog on '" + replicaSet + "'", this::readOplog);
}
/**
* Use the given primary to read the oplog.
*
* @param primary the connection to the replica set's primary node; may not be null
*/
protected void readOplog(MongoClient primary) {
BsonTimestamp oplogStart = source.lastOffsetTimestamp(replicaSet.replicaSetName());
logger.info("Reading oplog for '{}' primary {} starting at {}", replicaSet, primary.getAddress(), 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");
Bson filter = Filters.and(Filters.gt("ts", oplogStart), // start just after our last position
Filters.exists("fromMigrate", false)); // skip internal movements across shards
FindIterable<Document> results = oplog.find(filter)
.sort(new Document("$natural", 1)) // force forwards collection scan
.oplogReplay(true) // tells Mongo to not rely on indexes
.noCursorTimeout(true) // don't timeout waiting for events
.cursorType(CursorType.TailableAwait); // tail and await new data
// Read as much of the oplog as we can ...
ServerAddress primaryAddress = primary.getAddress();
try (MongoCursor<Document> cursor = results.iterator()) {
while (running.get() && cursor.hasNext()) {
if (!handleOplogEvent(primaryAddress, cursor.next())) {
// Something happened, and we're supposed to stop reading
return;
}
}
}
}
/**
* Handle a single oplog event.
*
* @param primaryAddress the address of the primary server from which the event was obtained; may not be null
* @param event the oplog event; may not be null
* @return {@code true} if additional events should be processed, or {@code false} if the caller should stop
* processing events
*/
protected boolean handleOplogEvent(ServerAddress primaryAddress, Document event) {
logger.debug("Found event: {}", event);
String ns = event.getString("ns");
Document object = event.get("o", Document.class);
if (object == null) {
logger.warn("Missing 'o' field in event, so skipping {}", event.toJson());
return true;
}
if (ns == null || ns.isEmpty()) {
// These are replica set events ...
String msg = object.getString("msg");
if ("new primary".equals(msg)) {
logger.info("Found new primary event in oplog, so stopping use of {} to continue with new primary",
primaryAddress);
// There is a new primary, so stop using this server and instead use the new primary ...
return false;
}
// Otherwise, ignore this event ...
logger.debug("Skipping event with no namespace: {}", event.toJson());
return true;
}
int delimIndex = ns.indexOf('.');
if (delimIndex > 0) {
assert (delimIndex + 1) < ns.length();
String dbName = ns.substring(0, delimIndex);
String collectionName = ns.substring(delimIndex + 1);
if ("$cmd".equals(collectionName)) {
// This is a command on a 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 ...
CollectionId collectionId = new CollectionId(rsName, dbName, collectionName);
if (collectionFilter.test(collectionId)) {
RecordsForCollection factory = recordMakers.forCollection(collectionId);
try {
factory.recordEvent(event, clock.currentTimeInMillis());
} catch (InterruptedException e) {
Thread.interrupted();
return false;
}
}
}
return true;
}
}

View File

@ -0,0 +1,342 @@
/*
* 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.util.Collections;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.ConnectException;
import org.bson.BsonTimestamp;
import org.bson.Document;
import org.bson.types.BSONTimestamp;
import io.debezium.annotation.Immutable;
import io.debezium.annotation.NotThreadSafe;
import io.debezium.util.Collect;
/**
* Information about the source of information, which includes the partitions and offsets within those partitions. The MongoDB
* connector considers each MongoDB database a separate "partitions" (in Kafka Connect terminology), and each partition has
* an "offset" (if Kafka Connect terminology) that defines the position within that partition/database where the connector
* finds a particular record. As the connector uses the Kafka Connect framework to process each record in a partition, Kafka
* Connect keeps track of the most recent offset for that partition.
* <p>
* The {@link #partition(String) source partition} information identifies the particular MongoDB replica set and the connector's
* logical name of the MongoDB server. A JSON-like representation of the source partition for a database named "customers" hosted
* in a MongoDB replica set named "myMongoServer" is as follows:
*
* <pre>
* {
* "server_id" : "myMongoServer",
* "replicaSetName" : "rs0"
* }
* </pre>
*
* <p>
* The {@link #lastOffset(String) source offset} information describes the position within a particular partition of each record.
* Since each event in MongoDB's oplog is identified by a {@link BSONTimestamp} that tracks the time and the order of the
* event for that particular time (e.g., multiple events that occur at the same time will have unique orders), the offset
* includes the BSONTimetamp representation. (The event's {@code h} field is the unique ID for the operation, so this is also
* included in the offset.) And, if an initial sync is in progress, the offset will include the {@code initsync} field.
* <p>
* Here's a JSON-like representation of an example timestamp:
*
* <pre>
* {
* "sec" = 1422998530,
* "ord" = 0,
* "h" = 398278732173914406,
* "initsync" = true
* }
* </pre>
*
* @author Randall Hauch
*/
@NotThreadSafe
public final class SourceInfo {
public static final int SCHEMA_VERSION = 1;
public static final String SERVER_ID_KEY = "server_id";
public static final String SERVER_NAME = "name";
public static final String REPLICA_SET_NAME = "rs";
public static final String NAMESPACE = "ns";
public static final String TIMESTAMP = "sec";
public static final String ORDER = "ord";
public static final String OPERATION_ID = "h";
public static final String INITIAL_SYNC = "initsync";
private static final BsonTimestamp INITIAL_TIMESTAMP = new BsonTimestamp();
private static final Position INITIAL_POSITION = new Position(INITIAL_TIMESTAMP, null);
/**
* A {@link Schema} definition for a {@link Struct} used to store the {@link #partition(String)} and {@link #lastOffset}
* information.
*/
public static final Schema SOURCE_SCHEMA = SchemaBuilder.struct()
.name("io.debezium.connector.mongo.Source")
.version(SCHEMA_VERSION)
.field(SERVER_NAME, Schema.STRING_SCHEMA)
.field(REPLICA_SET_NAME, Schema.STRING_SCHEMA)
.field(NAMESPACE, Schema.STRING_SCHEMA)
.field(TIMESTAMP, Schema.INT32_SCHEMA)
.field(ORDER, Schema.INT32_SCHEMA)
.field(OPERATION_ID, Schema.OPTIONAL_INT64_SCHEMA)
.field(INITIAL_SYNC, Schema.OPTIONAL_BOOLEAN_SCHEMA)
.build();
private final ConcurrentMap<String, Map<String, String>> sourcePartitionsByReplicaSetName = new ConcurrentHashMap<>();
private final ConcurrentMap<String, Position> positionsByReplicaSetName = new ConcurrentHashMap<>();
private final Set<String> initialSyncReplicaSets = Collections.newSetFromMap(new ConcurrentHashMap<>());
@Immutable
protected static final class Position {
private final Long opId;
private final BsonTimestamp ts;
public Position(int ts, int order, Long opId) {
this(new BsonTimestamp(ts, order), opId);
}
public Position(BsonTimestamp ts, Long opId) {
this.ts = ts;
this.opId = opId;
assert this.ts != null;
}
public BsonTimestamp getTimestamp() {
return this.ts;
}
public int getTime() {
return this.ts.getTime();
}
public int getInc() {
return this.ts.getInc();
}
public Long getOperationId() {
return this.opId;
}
}
private final String serverName;
public SourceInfo(String serverName) {
this.serverName = serverName;
assert this.serverName != null;
}
/**
* Get a {@link Schema} representation of the source {@link #partition(String) partition} and {@link #lastOffset(String)
* offset} information.
*
* @return the source partition and offset {@link Schema}; never null
* @see #offsetStructForEvent(String, Document)
*/
public Schema schema() {
return SOURCE_SCHEMA;
}
/**
* Get the Kafka Connect detail about the source "partition" for the given database in the replica set. If the database is
* not known, this method records the new partition.
*
* @param replicaSetName the name of the replica set name for which the partition is to be obtained; may not be null
* @return the source partition information; never null
*/
public Map<String, String> partition(String replicaSetName) {
return sourcePartitionsByReplicaSetName.computeIfAbsent(replicaSetName, rsName -> {
return Collect.hashMapOf(SERVER_ID_KEY, serverName, REPLICA_SET_NAME, rsName);
});
}
/**
* Get the MongoDB timestamp of the last offset position for the replica set.
*
* @param replicaSetName the name of the replica set name for which the new offset is to be obtained; may not be null
* @return the timestamp of the last offset, or the beginning of time if there is none
*/
public BsonTimestamp lastOffsetTimestamp(String replicaSetName) {
Position existing = positionsByReplicaSetName.get(replicaSetName);
return existing != null ? existing.ts : INITIAL_TIMESTAMP;
}
/**
* Get the Kafka Connect detail about the source "offset" for the named database, which describes the given position in the
* database where we have last read. If the database has not yet been seen, this records the starting position
* for that database. However, if there is a position for the database, the offset representation is returned.
*
* @param replicaSetName the name of the replica set name for which the new offset is to be obtained; may not be null
* @return a copy of the current offset for the database; never null
*/
public Map<String, ?> lastOffset(String replicaSetName) {
Position existing = positionsByReplicaSetName.get(replicaSetName);
if (existing == null) existing = INITIAL_POSITION;
if (initialSyncReplicaSets.contains(replicaSetName)) {
return Collect.hashMapOf(TIMESTAMP, new Integer(existing.getTime()),
ORDER, new Integer(existing.getInc()),
OPERATION_ID, existing.getOperationId(),
INITIAL_SYNC, true);
}
return Collect.hashMapOf(TIMESTAMP, new Integer(existing.getTime()),
ORDER, new Integer(existing.getInc()),
OPERATION_ID, existing.getOperationId());
}
/**
* Get a {@link Struct} representation of the source {@link #partition(String) partition} and {@link #lastOffset(String)
* offset} information where we have last read. The Struct complies with the {@link #schema} for the MongoDB connector.
*
* @param replicaSetName the name of the replica set name for which the new offset is to be obtained; may not be null
* @param collectionId the event's collection identifier; may not be null
* @return the source partition and offset {@link Struct}; never null
* @see #schema()
*/
public Struct lastOffsetStruct(String replicaSetName, CollectionId collectionId) {
return offsetStructFor(replicaSetName, collectionId.namespace(), positionsByReplicaSetName.get(replicaSetName),
initialSyncReplicaSets.contains(replicaSetName));
}
/**
* Get a {@link Struct} representation of the source {@link #partition(String) partition} and {@link #lastOffset(String)
* offset} information. The Struct complies with the {@link #schema} for the MongoDB connector.
*
* @param replicaSetName the name of the replica set name for which the new offset is to be obtained; may not be null
* @param oplogEvent the replica set oplog event that was last read; may be null if the position is the start of
* the oplog
* @return the source partition and offset {@link Struct}; never null
* @see #schema()
*/
public Struct offsetStructForEvent(String replicaSetName, Document oplogEvent) {
Position position = INITIAL_POSITION;
String namespace = "";
if (oplogEvent != null) {
BsonTimestamp ts = extractEventTimestamp(oplogEvent);
Long opId = oplogEvent.getLong("h");
position = new Position(ts, opId);
namespace = oplogEvent.getString("ns");
}
positionsByReplicaSetName.put(replicaSetName, position);
return offsetStructFor(replicaSetName, namespace, position, initialSyncReplicaSets.contains(replicaSetName));
}
/**
* Utility to extract the {@link BsonTimestamp timestamp} value from the event.
* @param oplogEvent the event
* @return the timestamp, or null if the event is null or there is no {@code ts} field
*/
protected static BsonTimestamp extractEventTimestamp(Document oplogEvent) {
return oplogEvent != null ? oplogEvent.get("ts", BsonTimestamp.class) : null;
}
private Struct offsetStructFor(String replicaSetName, String namespace, Position position, boolean isInitialSync) {
if (position == null) position = INITIAL_POSITION;
Struct result = new Struct(SOURCE_SCHEMA);
result.put(SERVER_NAME, serverName);
result.put(REPLICA_SET_NAME, replicaSetName);
result.put(NAMESPACE, namespace);
result.put(TIMESTAMP, position.getTime());
result.put(ORDER, position.getInc());
result.put(OPERATION_ID, position.getOperationId());
if (isInitialSync) {
result.put(INITIAL_SYNC, true);
}
return result;
}
/**
* Determine whether we have previously recorded a MongoDB timestamp for the replica set.
*
* @param replicaSetName the name of the replica set name; may not be null
* @return {@code true} if an offset has been recorded for the replica set, or {@code false} if the replica set has not
* yet been seen
*/
public boolean hasOffset(String replicaSetName) {
return positionsByReplicaSetName.containsKey(replicaSetName);
}
/**
* Set the source offset, as read from Kafka Connect, for the given replica set. This method does nothing if the supplied map
* is null.
*
* @param replicaSetName the name of the replica set name for which the new offset is to be obtained; may not be null
* @param sourceOffset the previously-recorded Kafka Connect source offset; may be null
* @return {@code true} if the offset was recorded, or {@code false} if the source offset is null
* @throws ConnectException if any offset parameter values are missing, invalid, or of the wrong type
*/
public boolean setOffsetFor(String replicaSetName, Map<String, ?> sourceOffset) {
if (replicaSetName == null) throw new IllegalArgumentException("The replica set name may not be null");
if (sourceOffset == null) return false;
// We have previously recorded at least one offset for this database ...
int time = intOffsetValue(sourceOffset, TIMESTAMP);
int order = intOffsetValue(sourceOffset, ORDER);
Long operationId = longOffsetValue(sourceOffset, OPERATION_ID);
positionsByReplicaSetName.put(replicaSetName, new Position(time, order, operationId));
return true;
}
/**
* Set the source offset, as read from Kafka Connect, for the given replica set. This method does nothing if the supplied map
* is null.
*
* @param partition the partition information; may not be null
* @param sourceOffset the previously-recorded Kafka Connect source offset; may be null
* @return {@code true} if the offset was recorded, or {@code false} if the source offset is null
* @throws ConnectException if any offset parameter values are missing, invalid, or of the wrong type
*/
public boolean setOffsetFor(Map<String, String> partition, Map<String, ?> sourceOffset) {
String replicaSetName = partition.get(REPLICA_SET_NAME);
return setOffsetFor(replicaSetName, sourceOffset);
}
/**
* Record that an initial sync has started for the given replica set.
*
* @param replicaSetName the name of the replica set; never null
*/
public void startInitialSync(String replicaSetName) {
initialSyncReplicaSets.add(replicaSetName);
}
/**
* Record that an initial sync has stopped for the given replica set.
*
* @param replicaSetName the name of the replica set; never null
*/
public void stopInitialSync(String replicaSetName) {
initialSyncReplicaSets.remove(replicaSetName);
}
private static int intOffsetValue(Map<String, ?> values, String key) {
Object obj = values.get(key);
if (obj == null) return 0;
if (obj instanceof Number) return ((Number) obj).intValue();
try {
return Integer.parseInt(obj.toString());
} catch (NumberFormatException e) {
throw new ConnectException("Source offset '" + key + "' parameter value " + obj + " could not be converted to an integer");
}
}
private static long longOffsetValue(Map<String, ?> values, String key) {
Object obj = values.get(key);
if (obj == null) return 0;
if (obj instanceof Number) return ((Number) obj).longValue();
try {
return Long.parseLong(obj.toString());
} catch (NumberFormatException e) {
throw new ConnectException("Source offset '" + key + "' parameter value " + obj + " could not be converted to a long");
}
}
}

View File

@ -0,0 +1,75 @@
/*
* 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.util.StringJoiner;
import io.debezium.annotation.ThreadSafe;
/**
* A function that determines the name of topics for data and metadata.
*
* @author Randall Hauch
*/
@ThreadSafe
@FunctionalInterface
public interface TopicSelector {
/**
* Get the name of the topic for a given database name and collection name. This method returns
* "{@code <dbName>.<collectionName>}".
*
* @return the topic selector; never null
*/
static TopicSelector defaultSelector() {
return defaultSelector(null,".");
}
/**
* Get the name of the topic for a given prefix, database name, and collection name. This method returns
* "{@code <prefix>.<dbName>.<collectionName>}", and does not use the replica set name.
*
* @param prefix the prefix; be null or empty if no prefix is needed
* @return the topic selector; never null
*/
static TopicSelector defaultSelector(String prefix) {
return defaultSelector(prefix,".");
}
/**
* Get the name of the topic for a given prefix, database name, and collection name. This method returns
* "{@code <prefix><delimiter><dbName><delimiter><collectionName>}", and does not use the replica set name.
*
* @param delimiter the string delineating the prefix, database, and collection names; may not be null
* @param prefix the prefix; be null or empty if no prefix is needed
* @return the topic selector; never null
*/
static TopicSelector defaultSelector(String prefix, String delimiter) {
if (prefix != null && prefix.trim().length() > 0) {
String trimmedPrefix = prefix.trim();
return (collectionId) -> {
StringJoiner sb = new StringJoiner(delimiter);
sb.add(trimmedPrefix);
sb.add(collectionId.dbName());
sb.add(collectionId.name());
return sb.toString();
};
}
return (collectionId) -> {
StringJoiner sb = new StringJoiner(delimiter);
sb.add(collectionId.dbName());
sb.add(collectionId.name());
return sb.toString();
};
}
/**
* Get the name of the topic for the given server name and database name.
*
* @param collectionId the identifier of the collection for which records are to be produced; may not be null
* @return the topic name; never null
*/
String getTopic(CollectionId collectionId);
}

View File

@ -0,0 +1 @@
version=${project.version}

View File

@ -0,0 +1,110 @@
/*
* 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 static org.junit.Assert.fail;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiConsumer;
import org.junit.After;
import org.junit.Before;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.fest.assertions.Assertions.assertThat;
import io.debezium.config.Configuration;
import io.debezium.connector.mongodb.ReplicationContext.MongoPrimary;
import io.debezium.util.Testing;
public abstract class AbstractMongoIT implements Testing {
protected final static Logger logger = LoggerFactory.getLogger(AbstractMongoIT.class);
protected Configuration config;
protected ReplicationContext context;
protected ReplicaSet replicaSet;
protected MongoPrimary primary;
@Before
public void beforeEach() {
Testing.Print.disable();
Testing.Debug.disable();
useConfiguration(Configuration.fromSystemProperties("connector."));
}
/**
* A method that will initialize the state after the configuration is changed.
*
* @param config the configuration; may not be null
*/
protected void useConfiguration(Configuration config) {
this.config = config;
initialize(true);
}
/**
* A method that will initialize the state after the configuration is changed, reusing the same partition offsets that
* were previously used.
*
* @param config the configuration; may not be null
*/
protected void reuseConfiguration(Configuration config) {
this.config = config;
initialize(false);
}
/**
* A method that will initialize the state after the configuration is changed.
*
* @param restartFromBeginning {@code true} if the context should have no prior partition offsets, or {@code false} if the
* partition offsets that exist at this time should be reused
*/
private void initialize(boolean restartFromBeginning) {
// Record the partition offsets (if there are some) ...
Map<String, String> partition = null;
Map<String, ?> offsetForPartition = null;
if (!restartFromBeginning && context != null && replicaSet != null && context.source().hasOffset(replicaSet.replicaSetName())) {
partition = context.source().partition(replicaSet.replicaSetName());
offsetForPartition = context.source().lastOffset(replicaSet.replicaSetName());
}
context = new ReplicationContext(config);
assertThat(context.hosts()).isNotEmpty();
replicaSet = ReplicaSet.parse(context.hosts());
context.configureLoggingContext(replicaSet.replicaSetName());
// Restore Source position (if there are some) ...
if (partition != null) {
context.source().setOffsetFor(partition, offsetForPartition);
}
// Get a connection to the primary ...
primary = context.primaryFor(replicaSet, connectionErrorHandler(3));
}
@After
public void afterEach() {
if (context != null) {
// close all connections
context.shutdown();
}
}
protected BiConsumer<String, Throwable> connectionErrorHandler(int numErrorsBeforeFailing) {
AtomicInteger attempts = new AtomicInteger();
return (desc, error) -> {
if (attempts.incrementAndGet() > numErrorsBeforeFailing) {
fail("Unable to connect to primary after " + numErrorsBeforeFailing + " errors trying to " + desc + ": " + error);
}
logger.error("Error while attempting to {}: {}", desc, error.getMessage(), error);
};
}
}

View File

@ -0,0 +1,43 @@
/*
* 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 org.junit.Test;
import static org.fest.assertions.Assertions.assertThat;
/**
* @author Randall Hauch
*
*/
public class CollectionIdTest {
private CollectionId id;
@Test
public void shouldParseStringWithThreeSegments() {
assertParseable("a","b","c");
}
@Test
public void shouldNotParseStringWithTwoSegments() {
assertThat(CollectionId.parse("a.b")).isNull();
}
@Test
public void shouldNotParseStringWithOneSegments() {
assertThat(CollectionId.parse("a")).isNull();
}
protected void assertParseable( String replicaSetName, String dbName, String collectionName ) {
String str = replicaSetName + "." + dbName + "." + collectionName;
id = CollectionId.parse(str);
assertThat(id.replicaSetName()).isEqualTo(replicaSetName);
assertThat(id.dbName()).isEqualTo(dbName);
assertThat(id.name()).isEqualTo(collectionName);
}
}

View File

@ -0,0 +1,59 @@
/*
* 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 io.debezium.config.Configuration;
import io.debezium.config.Field;
/**
* A helper for easily building connector configurations for testing.
*
* @author Randall Hauch
*/
public class Configurator {
private Configuration.Builder configBuilder = Configuration.create();
public Configurator with(Field field, String value) {
configBuilder.with(field, value);
return this;
}
public Configurator with(Field field, boolean value) {
configBuilder.with(field, value);
return this;
}
public Configurator with(Field field, int value) {
configBuilder.with(field, value);
return this;
}
public Configurator serverName(String serverName) {
return with(MongoDbConnectorConfig.LOGICAL_NAME, serverName);
}
public Configurator hosts(String hosts) {
return with(MongoDbConnectorConfig.HOSTS, hosts);
}
public Configurator maxBatchSize(int maxBatchSize) {
return with(MongoDbConnectorConfig.MAX_BATCH_SIZE, maxBatchSize);
}
public Configurator includeCollections(String regexList) {
return with(MongoDbConnectorConfig.COLLECTION_WHITELIST, regexList);
}
public Configurator excludeCollections(String regexList) {
return with(MongoDbConnectorConfig.COLLECTION_BLACKLIST, regexList);
}
public Filters createFilters() {
return new Filters(configBuilder.build());
}
}

View File

@ -0,0 +1,117 @@
/*
* 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.util.LinkedList;
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.bson.BsonTimestamp;
import org.bson.Document;
import org.bson.conversions.Bson;
import org.junit.Test;
import com.mongodb.CursorType;
import com.mongodb.client.FindIterable;
import com.mongodb.client.MongoCollection;
import com.mongodb.client.MongoCursor;
import com.mongodb.client.MongoDatabase;
import com.mongodb.client.model.Filters;
import com.mongodb.client.model.InsertOneOptions;
import static org.fest.assertions.Assertions.assertThat;
import io.debezium.util.Testing;
public class ConnectionIT extends AbstractMongoIT {
@Test
public void shouldCreateMovieDatabase() {
//Testing.Print.enable();
//Testing.Debug.enable();
Testing.print("Configuration: " + config);
String dbName = "dbA";
primary.execute("shouldCreateMovieDatabase", mongo -> {
Testing.debug("Getting or creating 'movies' collection");
// Create a database and a collection in that database ...
MongoDatabase db = mongo.getDatabase(dbName);
// Get or create a collection in that database ...
db.getCollection("movies");
Testing.debug("Completed getting 'movies' collection");
});
primary.execute("Add document to movies collection", mongo -> {
Testing.debug("Adding document to 'movies' collection");
// Add a document to that collection ...
MongoDatabase db = mongo.getDatabase(dbName);
MongoCollection<Document> movies = db.getCollection("movies");
InsertOneOptions insertOptions = new InsertOneOptions().bypassDocumentValidation(true);
movies.insertOne(Document.parse("{ \"name\":\"Starter Wars\"}"), insertOptions);
assertThat(db.getCollection("movies").count()).isEqualTo(1);
// Read the collection to make sure we can find our document ...
Bson filter = Filters.eq("name", "Starter Wars");
FindIterable<Document> movieResults = db.getCollection("movies").find(filter);
try (MongoCursor<Document> cursor = movieResults.iterator();) {
assertThat(cursor.tryNext().getString("name")).isEqualTo("Starter Wars");
assertThat(cursor.tryNext()).isNull();
}
Testing.debug("Completed document to 'movies' collection");
});
// Now that we've put at least one document into our collection, verify we can see the database and collection ...
assertThat(primary.databaseNames()).contains("dbA");
assertThat(primary.collections()).contains(new CollectionId(replicaSet.replicaSetName(), dbName, "movies"));
// Read oplog from beginning ...
List<Document> eventQueue = new LinkedList<>();
int minimumEventsExpected = 1;
long maxSeconds = 5;
primary.execute("read oplog from beginning", mongo -> {
Testing.debug("Getting local.oplog.rs");
BsonTimestamp oplogStart = new BsonTimestamp(1,1);
Bson filter = Filters.and(Filters.gt("ts", oplogStart), // start just after our last position
Filters.exists("fromMigrate", false)); // skip internal movements across shards
FindIterable<Document> results = mongo.getDatabase("local")
.getCollection("oplog.rs")
.find(filter)
.sort(new Document("$natural", 1))
.oplogReplay(true) // tells Mongo to not rely on indexes
.noCursorTimeout(true) // don't timeout waiting for events
.cursorType(CursorType.TailableAwait);
Testing.debug("Reading local.oplog.rs");
try (MongoCursor<Document> cursor = results.iterator();) {
Document event = null;
long stopTime = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(maxSeconds);
while (System.currentTimeMillis() < stopTime && eventQueue.size() < minimumEventsExpected) {
while ((event = cursor.tryNext()) != null) {
eventQueue.add(event);
}
}
assertThat(eventQueue.size()).isGreaterThanOrEqualTo(1);
}
Testing.debug("Completed local.oplog.rs");
});
eventQueue.forEach(event -> {
Testing.print("Found: " + event);
BsonTimestamp position = event.get("ts", BsonTimestamp.class);
assert position != null;
});
}
@Test
public void shouldListDatabases() {
Testing.Print.enable();
Testing.print("Databases: " + primary.databaseNames());
}
}

View File

@ -0,0 +1,118 @@
/*
* 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 org.junit.Before;
import org.junit.Test;
import static org.fest.assertions.Assertions.assertThat;
/**
* @author Randall Hauch
*/
public class FiltersTest {
private Configurator build;
private Filters filters;
@Before
public void beforeEach() {
build = new Configurator();
filters = null;
}
@Test
public void shouldIncludeCollectionCoveredByLiteralWithPeriodAsWildcardInWhitelistAndNoBlacklist() {
filters = build.includeCollections("db1.coll[.]?ection[x]?A,db1[.](.*)B").createFilters();
assertCollectionIncluded("db1xcoll.ectionA"); // first '.' is an unescaped wildcard in regex
assertCollectionIncluded("db1.collectionA");
}
@Test
public void shouldIncludeCollectionCoveredByLiteralInWhitelistAndNoBlacklist() {
filters = build.includeCollections("db1.collectionA").createFilters();
assertCollectionIncluded("db1.collectionA");
assertCollectionExcluded("db1.collectionB");
assertCollectionExcluded("db2.collectionA");
}
@Test
public void shouldIncludeCollectionCoveredByLiteralWithEscapedPeriodInWhitelistAndNoBlacklist() {
filters = build.includeCollections("db1[.]collectionA").createFilters();
assertCollectionIncluded("db1.collectionA");
assertCollectionExcluded("db1.collectionB");
assertCollectionExcluded("db2.collectionA");
filters = build.includeCollections("db1\\.collectionA").createFilters();
assertCollectionIncluded("db1.collectionA");
assertCollectionExcluded("db1.collectionB");
assertCollectionExcluded("db2.collectionA");
}
@Test
public void shouldIncludeCollectionCoveredByMultipleLiteralsInWhitelistAndNoBlacklist() {
filters = build.includeCollections("db1.collectionA,db1.collectionB").createFilters();
assertCollectionIncluded("db1.collectionA");
assertCollectionIncluded("db1.collectionB");
assertCollectionExcluded("db2.collectionA");
assertCollectionExcluded("db2.collectionB");
}
@Test
public void shouldIncludeCollectionCoveredByMultipleRegexInWhitelistAndNoBlacklist() {
filters = build.includeCollections("db1.collection[x]?A,db1[.](.*)B").createFilters();
assertCollectionIncluded("db1.collectionA");
assertCollectionIncluded("db1.collectionxA");
assertCollectionExcluded("db1.collectionx");
assertCollectionExcluded("db1.collectioxA");
assertCollectionIncluded("db1.B");
assertCollectionIncluded("db1.collB");
assertCollectionIncluded("db1.collectionB");
assertCollectionExcluded("db2.collectionA");
assertCollectionExcluded("db2.collectionxA");
assertCollectionExcluded("db2.B");
assertCollectionExcluded("db2.collB");
assertCollectionExcluded("db2.collectionB");
}
@Test
public void shouldIncludeCollectionCoveredByRegexWithWildcardInWhitelistAndNoBlacklist() {
filters = build.includeCollections("db1[.](.*)").createFilters();
assertCollectionIncluded("db1.collectionA");
assertCollectionIncluded("db1.collectionxA");
assertCollectionIncluded("db1.collectionx");
assertCollectionIncluded("db1.collectioxA");
assertCollectionIncluded("db1.B");
assertCollectionIncluded("db1.collB");
assertCollectionIncluded("db1.collectionB");
assertCollectionExcluded("db2.collectionA");
assertCollectionExcluded("db2.collectionxA");
assertCollectionExcluded("db12.B");
assertCollectionExcluded("db12.collB");
assertCollectionExcluded("db12.collectionB");
}
@Test
public void shouldExcludeCollectionCoveredByLiteralInBlacklist() {
filters = build.excludeCollections("db1.collectionA").createFilters();
assertCollectionExcluded("db1.collectionA");
assertCollectionIncluded("db1.collectionB");
assertCollectionIncluded("db2.collectionA");
}
protected void assertCollectionIncluded(String fullyQualifiedTableName) {
CollectionId id = CollectionId.parse("rs1." + fullyQualifiedTableName);
assertThat(id).isNotNull();
assertThat(filters.collectionFilter().test(id)).isTrue();
}
protected void assertCollectionExcluded(String fullyQualifiedTableName) {
CollectionId id = CollectionId.parse("rs1." + fullyQualifiedTableName);
assertThat(id).isNotNull();
assertThat(filters.collectionFilter().test(id)).isFalse();
}
}

View File

@ -0,0 +1,24 @@
/*
* 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 org.junit.Test;
import static org.fest.assertions.Assertions.assertThat;
/**
* @author Randall Hauch
*
*/
public class ModuleTest {
@Test
public void shouldReturnVersion() {
assertThat(Module.version()).isNotNull();
assertThat(Module.version()).isNotEmpty();
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.util.List;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import com.mongodb.MongoClient;
import com.mongodb.ServerAddress;
import static org.fest.assertions.Assertions.assertThat;
import io.debezium.config.Configuration;
/**
* @author Randall Hauch
*
*/
public class MongoClientsIT {
private static List<ServerAddress> addresses;
@BeforeClass
public static void beforeAll() {
Configuration config = Configuration.fromSystemProperties("connector.");
String host = config.getString(MongoDbConnectorConfig.HOSTS);
addresses = MongoUtil.parseAddresses(host);
}
private MongoClients clients;
@Before
public void beforeEach() {
clients = MongoClients.create().build();
}
@After
public void afterEach() {
if (clients != null) {
try {
clients.clear();
} finally {
clients = null;
}
}
}
@Test
public void shouldReturnSameInstanceForSameAddress() {
addresses.forEach(address -> {
MongoClient client1 = clients.clientFor(address);
MongoClient client2 = clients.clientFor(address);
assertThat(client1).isSameAs(client2);
MongoClient client3 = clients.clientFor(address.toString());
MongoClient client4 = clients.clientFor(address);
assertThat(client3).isSameAs(client4);
assertThat(client3).isSameAs(client1);
MongoClient client5 = clients.clientFor(address.toString());
MongoClient client6 = clients.clientFor(address.toString());
assertThat(client5).isSameAs(client6);
assertThat(client5).isSameAs(client1);
});
}
@Test
public void shouldReturnSameInstanceForSameAddresses() {
MongoClient client1 = clients.clientForMembers(addresses);
MongoClient client2 = clients.clientForMembers(addresses);
assertThat(client1).isSameAs(client2);
ServerAddress[] array = addresses.toArray(new ServerAddress[addresses.size()]);
MongoClient client3 = clients.clientForMembers(array);
MongoClient client4 = clients.clientForMembers(array);
assertThat(client3).isSameAs(client4);
assertThat(client3).isSameAs(client1);
String addressesStr = MongoUtil.toString(addresses);
MongoClient client5 = clients.clientForMembers(addressesStr);
MongoClient client6 = clients.clientForMembers(addressesStr);
assertThat(client5).isSameAs(client6);
assertThat(client5).isSameAs(client1);
}
}

View File

@ -0,0 +1,262 @@
/*
* 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 static org.junit.Assert.fail;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiConsumer;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.bson.Document;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import com.mongodb.client.MongoCollection;
import com.mongodb.client.MongoDatabase;
import com.mongodb.client.model.InsertOneOptions;
import static org.fest.assertions.Assertions.assertThat;
import io.debezium.config.Configuration;
import io.debezium.connector.mongodb.ReplicationContext.MongoPrimary;
import io.debezium.data.Envelope;
import io.debezium.data.Envelope.Operation;
import io.debezium.embedded.AbstractConnectorTest;
import io.debezium.util.IoUtil;
import io.debezium.util.Testing;
/**
* @author Randall Hauch
*
*/
public class MongoDbConnectorIT extends AbstractConnectorTest {
private Configuration config;
private ReplicationContext context;
@Before
public void beforeEach() {
stopConnector();
initializeConnectorTestFramework();
}
@After
public void afterEach() {
try {
stopConnector();
} finally {
if (context != null) context.shutdown();
}
}
/**
* Verifies that the connector doesn't run with an invalid configuration. This does not actually connect to the MySQL server.
*/
@Test
public void shouldNotStartWithInvalidConfiguration() {
config = Configuration.create()
.with(MongoDbConnectorConfig.AUTO_DISCOVER_MEMBERS, "true")
.build();
// we expect the engine will log at least one error, so preface it ...
logger.info("Attempting to start the connector with an INVALID configuration, so MULTIPLE error messages & one exceptions will appear in the log");
start(MongoDbConnector.class, config, (success, msg, error) -> {
assertThat(success).isFalse();
assertThat(error).isNotNull();
});
assertConnectorNotRunning();
}
@Test
public void shouldConsumeAllEventsFromDatabase() throws InterruptedException, IOException {
Testing.Debug.enable();
Testing.Print.enable();
// Use the DB configuration to define the connector's configuration ...
config = Configuration.create()
.with(MongoDbConnectorConfig.HOSTS, System.getProperty("connector.mongodb.hosts"))
.with(MongoDbConnectorConfig.AUTO_DISCOVER_MEMBERS,
System.getProperty("connector.mongodb.members.auto.discover"))
.with(MongoDbConnectorConfig.LOGICAL_NAME, System.getProperty("connector.mongodb.name"))
.with(MongoDbConnectorConfig.POLL_INTERVAL_MS, 10)
.with(MongoDbConnectorConfig.COLLECTION_WHITELIST, "dbit.*")
.with(MongoDbConnectorConfig.LOGICAL_NAME, "mongo")
.build();
// Set up the replication context for connections ...
context = new ReplicationContext(config);
// Before starting the connector, add data to the databases ...
storeDocuments("dbit", "simpletons", "simple_objects.json");
storeDocuments("dbit", "restaurants", "restaurants1.json");
// Start the connector ...
start(MongoDbConnector.class, config);
// ---------------------------------------------------------------------------------------------------------------
// Consume all of the events due to startup and initialization of the database
// ---------------------------------------------------------------------------------------------------------------
SourceRecords records = consumeRecordsByTopic(12);
assertThat(records.recordsForTopic("mongo.dbit.simpletons").size()).isEqualTo(6);
assertThat(records.recordsForTopic("mongo.dbit.restaurants").size()).isEqualTo(6);
assertThat(records.topics().size()).isEqualTo(2);
records.forEach(record -> {
// Check that all records are valid, and can be serialized and deserialized ...
validate(record);
verifyFromInitialSync(record);
verifyReadOperation(record);
});
// At this point, the connector has performed the initial sync and awaits changes ...
// ---------------------------------------------------------------------------------------------------------------
// Store more documents while the connector is still running
// ---------------------------------------------------------------------------------------------------------------
storeDocuments("dbit", "restaurants", "restaurants2.json");
// Wait until we can consume the 4 documents we just added ...
SourceRecords records2 = consumeRecordsByTopic(4);
assertThat(records2.recordsForTopic("mongo.dbit.restaurants").size()).isEqualTo(4);
assertThat(records2.topics().size()).isEqualTo(1);
records2.forEach(record -> {
// Check that all records are valid, and can be serialized and deserialized ...
validate(record);
verifyNotFromInitialSync(record);
verifyCreateOperation(record);
});
// ---------------------------------------------------------------------------------------------------------------
// Stop the connector
// ---------------------------------------------------------------------------------------------------------------
stopConnector();
// ---------------------------------------------------------------------------------------------------------------
// Store more documents while the connector is NOT running
// ---------------------------------------------------------------------------------------------------------------
storeDocuments("dbit", "restaurants", "restaurants3.json");
// ---------------------------------------------------------------------------------------------------------------
// Start the connector and we should only see the documents added since it was stopped
// ---------------------------------------------------------------------------------------------------------------
start(MongoDbConnector.class, config);
// Wait until we can consume the 4 documents we just added ...
SourceRecords records3 = consumeRecordsByTopic(5);
assertThat(records3.recordsForTopic("mongo.dbit.restaurants").size()).isEqualTo(5);
assertThat(records3.topics().size()).isEqualTo(1);
records3.forEach(record -> {
// Check that all records are valid, and can be serialized and deserialized ...
validate(record);
verifyNotFromInitialSync(record);
verifyCreateOperation(record);
});
// ---------------------------------------------------------------------------------------------------------------
// Store more documents while the connector is still running
// ---------------------------------------------------------------------------------------------------------------
storeDocuments("dbit", "restaurants", "restaurants4.json");
// Wait until we can consume the 4 documents we just added ...
SourceRecords records4 = consumeRecordsByTopic(8);
assertThat(records4.recordsForTopic("mongo.dbit.restaurants").size()).isEqualTo(8);
assertThat(records4.topics().size()).isEqualTo(1);
records4.forEach(record -> {
// Check that all records are valid, and can be serialized and deserialized ...
validate(record);
verifyNotFromInitialSync(record);
verifyCreateOperation(record);
});
}
protected void verifyFromInitialSync( SourceRecord record ) {
assertThat(record.sourceOffset().containsKey(SourceInfo.INITIAL_SYNC)).isTrue();
Struct value = (Struct)record.value();
assertThat(value.getStruct(Envelope.FieldName.SOURCE).getBoolean(SourceInfo.INITIAL_SYNC)).isTrue();
}
protected void verifyNotFromInitialSync( SourceRecord record ) {
assertThat(record.sourceOffset().containsKey(SourceInfo.INITIAL_SYNC)).isFalse();
Struct value = (Struct)record.value();
assertThat(value.getStruct(Envelope.FieldName.SOURCE).getBoolean(SourceInfo.INITIAL_SYNC)).isNull();
}
protected void verifyCreateOperation( SourceRecord record ) {
verifyOperation(record,Operation.CREATE);
}
protected void verifyReadOperation( SourceRecord record ) {
verifyOperation(record,Operation.READ);
}
protected void verifyUpdateOperation( SourceRecord record ) {
verifyOperation(record,Operation.UPDATE);
}
protected void verifyDeleteOperation( SourceRecord record ) {
verifyOperation(record,Operation.DELETE);
}
protected void verifyOperation( SourceRecord record, Operation expected ) {
Struct value = (Struct)record.value();
assertThat(value.getString(Envelope.FieldName.OPERATION)).isEqualTo(expected.code());
}
protected MongoPrimary primary() {
ReplicaSet replicaSet = ReplicaSet.parse(context.hosts());
return context.primaryFor(replicaSet, connectionErrorHandler(3));
}
protected void storeDocuments(String dbName, String collectionName, String pathOnClasspath) {
primary().execute("storing documents", mongo -> {
Testing.debug("Storing in '" + dbName + "." + collectionName + "' documents loaded from from '" + pathOnClasspath + "'");
MongoDatabase db1 = mongo.getDatabase(dbName);
MongoCollection<Document> coll = db1.getCollection(collectionName);
storeDocuments(coll, pathOnClasspath);
});
}
protected void storeDocuments(MongoCollection<Document> collection, String pathOnClasspath) {
InsertOneOptions insertOptions = new InsertOneOptions().bypassDocumentValidation(true);
loadTestDocuments(pathOnClasspath).forEach(doc -> {
assertThat(doc).isNotNull();
assertThat(doc.size()).isGreaterThan(0);
collection.insertOne(doc, insertOptions);
});
}
protected List<Document> loadTestDocuments(String pathOnClasspath) {
List<Document> results = new ArrayList<>();
try (InputStream stream = Testing.Files.readResourceAsStream(pathOnClasspath);) {
assertThat(stream).isNotNull();
IoUtil.readLines(stream, line -> {
Document doc = Document.parse(line);
assertThat(doc.size()).isGreaterThan(0);
results.add(doc);
});
} catch ( IOException e ) {
fail("Unable to find or read file '" + pathOnClasspath + "': " + e.getMessage());
}
return results;
}
protected BiConsumer<String, Throwable> connectionErrorHandler(int numErrorsBeforeFailing) {
AtomicInteger attempts = new AtomicInteger();
return (desc, error) -> {
if (attempts.incrementAndGet() > numErrorsBeforeFailing) {
fail("Unable to connect to primary after " + numErrorsBeforeFailing + " errors trying to " + desc + ": " + error);
}
logger.error("Error while attempting to {}: {}", desc, error.getMessage(), error);
};
}
}

View File

@ -0,0 +1,167 @@
/*
* 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.util.ArrayList;
import java.util.List;
import org.junit.Test;
import com.mongodb.ServerAddress;
import static org.fest.assertions.Assertions.assertThat;
/**
* @author Randall Hauch
*
*/
public class MongoUtilTest {
private ServerAddress address;
private List<ServerAddress> addresses = new ArrayList<>();
@Test
public void shouldParseIPv4ServerAddressWithoutPort() {
address = MongoUtil.parseAddress("localhost");
assertThat(address.getHost()).isEqualTo("localhost");
assertThat(address.getPort()).isEqualTo(ServerAddress.defaultPort());
}
@Test
public void shouldParseIPv4ServerAddressWithoPort() {
address = MongoUtil.parseAddress("localhost:28017");
assertThat(address.getHost()).isEqualTo("localhost");
assertThat(address.getPort()).isEqualTo(28017);
}
@Test
public void shouldParseIPv6ServerAddressWithoutPort() {
address = MongoUtil.parseAddress("[::1/128]");
assertThat(address.getHost()).isEqualTo("::1/128"); // removes brackets
assertThat(address.getPort()).isEqualTo(ServerAddress.defaultPort());
}
@Test
public void shouldParseIPv6ServerAddressWithPort() {
address = MongoUtil.parseAddress("[::1/128]:28017");
assertThat(address.getHost()).isEqualTo("::1/128"); // removes brackets
assertThat(address.getPort()).isEqualTo(28017);
}
@Test
public void shouldParseServerAddressesWithoutPort() {
addresses = MongoUtil.parseAddresses("host1,host2,[::1/128],host4");
assertThat(addresses.size()).isEqualTo(4);
assertThat(addresses.get(0).getHost()).isEqualTo("host1");
assertThat(addresses.get(0).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(1).getHost()).isEqualTo("host2");
assertThat(addresses.get(1).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(2).getHost()).isEqualTo("::1/128");
assertThat(addresses.get(2).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(3).getHost()).isEqualTo("host4");
assertThat(addresses.get(3).getPort()).isEqualTo(ServerAddress.defaultPort());
}
@Test
public void shouldParseServerAddressesWithPort() {
addresses = MongoUtil.parseAddresses("host1:2111,host2:3111,[ff02::2:ff00:0/104]:4111,host4:5111");
assertThat(addresses.size()).isEqualTo(4);
assertThat(addresses.get(0).getHost()).isEqualTo("host1");
assertThat(addresses.get(0).getPort()).isEqualTo(2111);
assertThat(addresses.get(1).getHost()).isEqualTo("host2");
assertThat(addresses.get(1).getPort()).isEqualTo(3111);
assertThat(addresses.get(2).getHost()).isEqualTo("ff02::2:ff00:0/104");
assertThat(addresses.get(2).getPort()).isEqualTo(4111);
assertThat(addresses.get(3).getHost()).isEqualTo("host4");
assertThat(addresses.get(3).getPort()).isEqualTo(5111);
}
@Test
public void shouldParseServerAddressesWithReplicaSetNameAndWithoutPort() {
addresses = MongoUtil.parseAddresses("replicaSetName/host1,host2,[::1/128],host4");
assertThat(addresses.size()).isEqualTo(4);
assertThat(addresses.get(0).getHost()).isEqualTo("host1");
assertThat(addresses.get(0).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(1).getHost()).isEqualTo("host2");
assertThat(addresses.get(1).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(2).getHost()).isEqualTo("::1/128");
assertThat(addresses.get(2).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(3).getHost()).isEqualTo("host4");
assertThat(addresses.get(3).getPort()).isEqualTo(ServerAddress.defaultPort());
}
@Test
public void shouldParseServerAddressesWithReplicaSetNameAndWithPort() {
addresses = MongoUtil.parseAddresses("replicaSetName/host1:2111,host2:3111,[ff02::2:ff00:0/104]:4111,host4:5111");
assertThat(addresses.size()).isEqualTo(4);
assertThat(addresses.get(0).getHost()).isEqualTo("host1");
assertThat(addresses.get(0).getPort()).isEqualTo(2111);
assertThat(addresses.get(1).getHost()).isEqualTo("host2");
assertThat(addresses.get(1).getPort()).isEqualTo(3111);
assertThat(addresses.get(2).getHost()).isEqualTo("ff02::2:ff00:0/104");
assertThat(addresses.get(2).getPort()).isEqualTo(4111);
assertThat(addresses.get(3).getHost()).isEqualTo("host4");
assertThat(addresses.get(3).getPort()).isEqualTo(5111);
}
@Test
public void shouldParseServerIPv6AddressesWithReplicaSetNameAndWithoutPort() {
addresses = MongoUtil.parseAddresses("replicaSetName/[::1/128],host2,[ff02::2:ff00:0/104],host4");
assertThat(addresses.size()).isEqualTo(4);
assertThat(addresses.get(0).getHost()).isEqualTo("::1/128");
assertThat(addresses.get(0).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(1).getHost()).isEqualTo("host2");
assertThat(addresses.get(1).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(2).getHost()).isEqualTo("ff02::2:ff00:0/104");
assertThat(addresses.get(2).getPort()).isEqualTo(ServerAddress.defaultPort());
assertThat(addresses.get(3).getHost()).isEqualTo("host4");
assertThat(addresses.get(3).getPort()).isEqualTo(ServerAddress.defaultPort());
}
@Test
public void shouldParseServerIPv6AddressesWithReplicaSetNameAndWithPort() {
addresses = MongoUtil.parseAddresses("replicaSetName/[::1/128]:2111,host2:3111,[ff02::2:ff00:0/104]:4111,host4:5111");
assertThat(addresses.size()).isEqualTo(4);
assertThat(addresses.get(0).getHost()).isEqualTo("::1/128");
assertThat(addresses.get(0).getPort()).isEqualTo(2111);
assertThat(addresses.get(1).getHost()).isEqualTo("host2");
assertThat(addresses.get(1).getPort()).isEqualTo(3111);
assertThat(addresses.get(2).getHost()).isEqualTo("ff02::2:ff00:0/104");
assertThat(addresses.get(2).getPort()).isEqualTo(4111);
assertThat(addresses.get(3).getHost()).isEqualTo("host4");
assertThat(addresses.get(3).getPort()).isEqualTo(5111);
}
@Test
public void shouldNotParseServerAddressesWithReplicaSetNameAndOpenBracket() {
addresses = MongoUtil.parseAddresses("replicaSetName/[");
assertThat(addresses.size()).isEqualTo(0);
}
@Test
public void shouldNotParseServerAddressesWithReplicaSetNameAndNoAddress() {
addresses = MongoUtil.parseAddresses("replicaSetName/");
assertThat(addresses.size()).isEqualTo(1);
assertThat(addresses.get(0).getHost()).isEqualTo(ServerAddress.defaultHost());
assertThat(addresses.get(0).getPort()).isEqualTo(ServerAddress.defaultPort());
}
@Test
public void shouldParseReplicaSetName() {
assertThat(MongoUtil.replicaSetUsedIn("rs0/")).isEqualTo("rs0");
assertThat(MongoUtil.replicaSetUsedIn("rs0/localhost")).isEqualTo("rs0");
assertThat(MongoUtil.replicaSetUsedIn("rs0/[::1/128]")).isEqualTo("rs0");
}
@Test
public void shouldNotParseReplicaSetName() {
assertThat(MongoUtil.replicaSetUsedIn("")).isNull();
assertThat(MongoUtil.replicaSetUsedIn("localhost")).isNull();
assertThat(MongoUtil.replicaSetUsedIn("[::1/128]")).isNull();
}
}

View File

@ -0,0 +1,158 @@
/*
* 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.util.ArrayList;
import java.util.List;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.bson.BsonTimestamp;
import org.bson.Document;
import org.bson.json.JsonMode;
import org.bson.json.JsonWriterSettings;
import org.bson.types.ObjectId;
import org.junit.Before;
import org.junit.Test;
import static org.fest.assertions.Assertions.assertThat;
import io.debezium.connector.mongodb.RecordMakers.RecordsForCollection;
import io.debezium.data.Envelope.FieldName;
import io.debezium.data.Envelope.Operation;
/**
* @author Randall Hauch
*
*/
public class RecordMakersTest {
private static final String SERVER_NAME = "serverX.";
private static final String PREFIX = SERVER_NAME + ".";
private static final JsonWriterSettings WRITER_SETTINGS = new JsonWriterSettings(JsonMode.STRICT, "", ""); // most compact
// JSON
private SourceInfo source;
private RecordMakers recordMakers;
private TopicSelector topicSelector;
private List<SourceRecord> produced;
@Before
public void beforeEach() {
source = new SourceInfo(SERVER_NAME);
topicSelector = TopicSelector.defaultSelector(PREFIX);
produced = new ArrayList<>();
recordMakers = new RecordMakers(source, topicSelector, produced::add);
}
@Test
public void shouldAlwaysFindRecordMakerForCollection() {
for (int i = 0; i != 100; ++i) {
CollectionId id = new CollectionId("rs0", "dbA", "c" + i);
RecordsForCollection records = recordMakers.forCollection(id);
assertThat(records).isNotNull();
assertThat(records.collectionId()).isSameAs(id);
}
}
@Test
public void shouldGenerateRecordForInsertEvent() throws InterruptedException {
CollectionId collectionId = new CollectionId("rs0", "dbA", "c1");
BsonTimestamp ts = new BsonTimestamp(1000, 1);
ObjectId objId = new ObjectId();
Document obj = new Document().append("_id", objId).append("name", "Sally");
Document event = new Document().append("o", obj)
.append("ns", "dbA.c1")
.append("ts", ts)
.append("h", new Long(12345678))
.append("op", "i");
RecordsForCollection records = recordMakers.forCollection(collectionId);
records.recordEvent(event, 1002);
assertThat(produced.size()).isEqualTo(1);
SourceRecord record = produced.get(0);
Struct key = (Struct) record.key();
Struct value = (Struct) record.value();
assertThat(key.schema()).isSameAs(record.keySchema());
assertThat(key.get("_id")).isEqualTo(objId.toString());
assertThat(value.schema()).isSameAs(record.valueSchema());
//assertThat(value.getString(FieldName.BEFORE)).isNull();
assertThat(value.getString(FieldName.AFTER)).isEqualTo(obj.toJson(WRITER_SETTINGS));
assertThat(value.getString(FieldName.OPERATION)).isEqualTo(Operation.CREATE.code());
assertThat(value.getInt64(FieldName.TIMESTAMP)).isEqualTo(1002L);
Struct actualSource = value.getStruct(FieldName.SOURCE);
Struct expectedSource = source.lastOffsetStruct("rs0", collectionId);
assertThat(actualSource).isEqualTo(expectedSource);
}
@Test
public void shouldGenerateRecordForUpdateEvent() throws InterruptedException {
BsonTimestamp ts = new BsonTimestamp(1000, 1);
CollectionId collectionId = new CollectionId("rs0", "dbA", "c1");
ObjectId objId = new ObjectId();
Document obj = new Document().append("$set", new Document("name", "Sally"));
Document event = new Document().append("o", obj)
.append("o2", objId)
.append("ns", "dbA.c1")
.append("ts", ts)
.append("h", new Long(12345678))
.append("op", "u");
RecordsForCollection records = recordMakers.forCollection(collectionId);
records.recordEvent(event, 1002);
assertThat(produced.size()).isEqualTo(1);
SourceRecord record = produced.get(0);
Struct key = (Struct) record.key();
Struct value = (Struct) record.value();
assertThat(key.schema()).isSameAs(record.keySchema());
assertThat(key.get("_id")).isEqualTo(objId.toString());
assertThat(value.schema()).isSameAs(record.valueSchema());
//assertThat(value.getString(FieldName.BEFORE)).isNull();
assertThat(value.getString(FieldName.AFTER)).isNull();
assertThat(value.getString("patch")).isEqualTo(obj.toJson(WRITER_SETTINGS));
assertThat(value.getString(FieldName.OPERATION)).isEqualTo(Operation.UPDATE.code());
assertThat(value.getInt64(FieldName.TIMESTAMP)).isEqualTo(1002L);
Struct actualSource = value.getStruct(FieldName.SOURCE);
Struct expectedSource = source.lastOffsetStruct("rs0", collectionId);
assertThat(actualSource).isEqualTo(expectedSource);
}
@Test
public void shouldGenerateRecordForDeleteEvent() throws InterruptedException {
BsonTimestamp ts = new BsonTimestamp(1000, 1);
CollectionId collectionId = new CollectionId("rs0", "dbA", "c1");
ObjectId objId = new ObjectId();
Document obj = new Document("_id", objId);
Document event = new Document().append("o", obj)
.append("ns", "dbA.c1")
.append("ts", ts)
.append("h", new Long(12345678))
.append("op", "d");
RecordsForCollection records = recordMakers.forCollection(collectionId);
records.recordEvent(event, 1002);
assertThat(produced.size()).isEqualTo(2);
SourceRecord record = produced.get(0);
Struct key = (Struct) record.key();
Struct value = (Struct) record.value();
assertThat(key.schema()).isSameAs(record.keySchema());
assertThat(key.get("_id")).isEqualTo(objId.toString());
assertThat(value.schema()).isSameAs(record.valueSchema());
assertThat(value.getString(FieldName.AFTER)).isNull();
assertThat(value.getString("patch")).isNull();
assertThat(value.getString(FieldName.OPERATION)).isEqualTo(Operation.DELETE.code());
assertThat(value.getInt64(FieldName.TIMESTAMP)).isEqualTo(1002L);
Struct actualSource = value.getStruct(FieldName.SOURCE);
Struct expectedSource = source.lastOffsetStruct("rs0", collectionId);
assertThat(actualSource).isEqualTo(expectedSource);
SourceRecord tombstone = produced.get(1);
Struct key2 = (Struct) tombstone.key();
assertThat(key2.schema()).isSameAs(tombstone.keySchema());
assertThat(key2.get("_id")).isEqualTo(objId.toString());
assertThat(tombstone.value()).isNull();
assertThat(tombstone.valueSchema()).isNull();
}
}

View File

@ -0,0 +1,278 @@
/*
* 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.util.ArrayList;
import java.util.List;
import org.junit.Test;
import com.mongodb.ServerAddress;
import static org.fest.assertions.Assertions.assertThat;
/**
* @author Randall Hauch
*
*/
public class ReplicaSetsTest {
private ReplicaSets sets;
private ReplicaSet rs;
@Test
public void shouldHaveNoReplicaSetsInEmptyInstance() {
assertThat(ReplicaSets.empty().replicaSetCount()).isEqualTo(0);
}
@Test
public void shouldParseNullHostString() {
assertThat(ReplicaSets.parse(null)).isEqualTo(ReplicaSets.empty());
}
@Test
public void shouldParseEmptyHostString() {
assertThat(ReplicaSets.parse("")).isEqualTo(ReplicaSets.empty());
}
@Test
public void shouldParseBlankHostString() {
assertThat(ReplicaSets.parse(" ")).isEqualTo(ReplicaSets.empty());
}
@Test
public void shouldParseHostStringWithStandaloneAddress() {
sets = ReplicaSets.parse("localhost:27017");
assertThat(sets.replicaSetCount()).isEqualTo(1);
assertThat(sets.hosts()).isEqualTo("localhost:27017");
rs = sets.all().get(0);
assertThat(rs.hasReplicaSetName()).isFalse();
assertThat(rs.isStandaloneServer()).isTrue();
assertThat(rs.replicaSetName()).isNull();
assertThat(rs.shardName()).isNull();
ServerAddress expected = new ServerAddress("localhost", 27017);
assertThat(rs.addresses().size()).isEqualTo(1);
assertThat(rs.addresses()).containsOnly(expected);
}
@Test
public void shouldParseHostStringWithStandaloneAddresses() {
sets = ReplicaSets.parse("localhost:27017,1.2.3.4:27017,localhost:28017,[fe80::601:9bff:feab:ec01]:27017");
assertThat(sets.replicaSetCount()).isEqualTo(1);
assertThat(sets.hosts()).isEqualTo("1.2.3.4:27017,[fe80::601:9bff:feab:ec01]:27017,localhost:27017,localhost:28017");
rs = sets.all().get(0);
assertThat(rs.hasReplicaSetName()).isFalse();
assertThat(rs.isStandaloneServer()).isTrue();
assertThat(rs.replicaSetName()).isNull();
assertThat(rs.shardName()).isNull();
ServerAddress expected1 = new ServerAddress("1.2.3.4", 27017);
ServerAddress expected2 = new ServerAddress("[fe80::601:9bff:feab:ec01]", 27017);
ServerAddress expected3 = new ServerAddress("localhost", 27017);
ServerAddress expected4 = new ServerAddress("localhost", 28017);
assertThat(rs.addresses().size()).isEqualTo(4);
assertThat(rs.addresses()).containsOnly(expected1, expected2, expected3, expected4);
}
@Test
public void shouldParseHostStringWithAddressForOneReplicaSet() {
sets = ReplicaSets.parse("myReplicaSet/localhost:27017");
assertThat(sets.replicaSetCount()).isEqualTo(1);
assertThat(sets.hosts()).isEqualTo("myReplicaSet/localhost:27017");
rs = sets.all().get(0);
assertThat(rs.hasReplicaSetName()).isTrue();
assertThat(rs.isStandaloneServer()).isFalse();
assertThat(rs.replicaSetName()).isEqualTo("myReplicaSet");
assertThat(rs.shardName()).isNull();
ServerAddress expected = new ServerAddress("localhost", 27017);
assertThat(rs.addresses().size()).isEqualTo(1);
assertThat(rs.addresses()).containsOnly(expected);
}
@Test
public void shouldParseHostStringWithIpv6AddressForOneReplicaSet() {
sets = ReplicaSets.parse("myReplicaSet/[fe80::601:9bff:feab:ec01]:27017");
assertThat(sets.replicaSetCount()).isEqualTo(1);
assertThat(sets.hosts()).isEqualTo("myReplicaSet/[fe80::601:9bff:feab:ec01]:27017");
rs = sets.all().get(0);
assertThat(rs.hasReplicaSetName()).isTrue();
assertThat(rs.isStandaloneServer()).isFalse();
assertThat(rs.replicaSetName()).isEqualTo("myReplicaSet");
assertThat(rs.shardName()).isNull();
ServerAddress expected = new ServerAddress("[fe80::601:9bff:feab:ec01]", 27017);
assertThat(rs.addresses().size()).isEqualTo(1);
assertThat(rs.addresses()).containsOnly(expected);
}
@Test
public void shouldParseHostStringWithAddressesForOneReplicaSet() {
sets = ReplicaSets.parse("myReplicaSet/localhost:27017,1.2.3.4:27017,localhost:28017,[fe80::601:9bff:feab:ec01]:27017");
assertThat(sets.replicaSetCount()).isEqualTo(1);
assertThat(sets.hosts()).isEqualTo("myReplicaSet/1.2.3.4:27017,[fe80::601:9bff:feab:ec01]:27017,localhost:27017,localhost:28017");
rs = sets.all().get(0);
assertThat(rs.hasReplicaSetName()).isTrue();
assertThat(rs.isStandaloneServer()).isFalse();
assertThat(rs.replicaSetName()).isEqualTo("myReplicaSet");
assertThat(rs.shardName()).isNull();
ServerAddress expected1 = new ServerAddress("1.2.3.4", 27017);
ServerAddress expected2 = new ServerAddress("[fe80::601:9bff:feab:ec01]", 27017);
ServerAddress expected3 = new ServerAddress("localhost", 27017);
ServerAddress expected4 = new ServerAddress("localhost", 28017);
assertThat(rs.addresses().size()).isEqualTo(4);
assertThat(rs.addresses()).containsOnly(expected1, expected2, expected3, expected4);
}
@Test
public void shouldParseHostStringWithAddressesForMultipleReplicaSet() {
sets = ReplicaSets.parse("myReplicaSet/host1:27017,[fe80::601:9bff:feab:ec01]:27017;otherReplicaset/1.2.3.4:27017,localhost:28017");
assertThat(sets.replicaSetCount()).isEqualTo(2);
assertThat(sets.hosts()).isEqualTo("myReplicaSet/[fe80::601:9bff:feab:ec01]:27017,host1:27017;otherReplicaset/1.2.3.4:27017,localhost:28017");
rs = sets.all().get(0);
assertThat(rs.hasReplicaSetName()).isTrue();
assertThat(rs.isStandaloneServer()).isFalse();
assertThat(rs.replicaSetName()).isEqualTo("myReplicaSet");
assertThat(rs.shardName()).isNull();
ServerAddress expected1 = new ServerAddress("[fe80::601:9bff:feab:ec01]", 27017);
ServerAddress expected2 = new ServerAddress("host1", 27017);
assertThat(rs.addresses().size()).isEqualTo(2);
assertThat(rs.addresses()).containsOnly(expected1, expected2);
rs = sets.all().get(1);
assertThat(rs.hasReplicaSetName()).isTrue();
assertThat(rs.isStandaloneServer()).isFalse();
assertThat(rs.replicaSetName()).isEqualTo("otherReplicaset");
assertThat(rs.shardName()).isNull();
expected1 = new ServerAddress("1.2.3.4", 27017);
expected2 = new ServerAddress("localhost", 28017);
assertThat(rs.addresses().size()).isEqualTo(2);
assertThat(rs.addresses()).containsOnly(expected1, expected2);
}
@Test
public void shouldParseHostStringWithAddressesForOneShard() {
sets = ReplicaSets.parse("shard1=myReplicaSet/localhost:27017,1.2.3.4:27017,localhost:28017,[fe80::601:9bff:feab:ec01]:27017");
assertThat(sets.replicaSetCount()).isEqualTo(1);
assertThat(sets.hosts()).isEqualTo("shard1=myReplicaSet/1.2.3.4:27017,[fe80::601:9bff:feab:ec01]:27017,localhost:27017,localhost:28017");
rs = sets.all().get(0);
assertThat(rs.hasReplicaSetName()).isTrue();
assertThat(rs.isStandaloneServer()).isFalse();
assertThat(rs.replicaSetName()).isEqualTo("myReplicaSet");
assertThat(rs.shardName()).isEqualTo("shard1");
ServerAddress expected1 = new ServerAddress("1.2.3.4", 27017);
ServerAddress expected2 = new ServerAddress("[fe80::601:9bff:feab:ec01]", 27017);
ServerAddress expected3 = new ServerAddress("localhost", 27017);
ServerAddress expected4 = new ServerAddress("localhost", 28017);
assertThat(rs.addresses().size()).isEqualTo(4);
assertThat(rs.addresses()).containsOnly(expected1, expected2, expected3, expected4);
}
@Test
public void shouldParseHostStringWithAddressesForMultipleShard() {
sets = ReplicaSets.parse("shard1=myReplicaSet/host1:27017,[fe80::601:9bff:feab:ec01]:27017;shard2=otherReplicaset/1.2.3.4:27017,localhost:28017");
assertThat(sets.replicaSetCount()).isEqualTo(2);
assertThat(sets.hosts()).isEqualTo("shard1=myReplicaSet/[fe80::601:9bff:feab:ec01]:27017,host1:27017;shard2=otherReplicaset/1.2.3.4:27017,localhost:28017");
rs = sets.all().get(0);
assertThat(rs.hasReplicaSetName()).isTrue();
assertThat(rs.isStandaloneServer()).isFalse();
assertThat(rs.replicaSetName()).isEqualTo("myReplicaSet");
assertThat(rs.shardName()).isEqualTo("shard1");
ServerAddress expected1 = new ServerAddress("[fe80::601:9bff:feab:ec01]", 27017);
ServerAddress expected2 = new ServerAddress("host1", 27017);
assertThat(rs.addresses().size()).isEqualTo(2);
assertThat(rs.addresses()).containsOnly(expected1, expected2);
rs = sets.all().get(1);
assertThat(rs.hasReplicaSetName()).isTrue();
assertThat(rs.isStandaloneServer()).isFalse();
assertThat(rs.replicaSetName()).isEqualTo("otherReplicaset");
assertThat(rs.shardName()).isEqualTo("shard2");
expected1 = new ServerAddress("1.2.3.4", 27017);
expected2 = new ServerAddress("localhost", 28017);
assertThat(rs.addresses().size()).isEqualTo(2);
assertThat(rs.addresses()).containsOnly(expected1, expected2);
}
@Test
public void shouldConsiderUnchangedSameInstance() {
sets = ReplicaSets.parse("localhost:27017");
assertThat(sets.haveChangedSince(sets)).isFalse();
}
@Test
public void shouldConsiderUnchangedSimilarReplicaSets() {
ReplicaSets sets1 = ReplicaSets.parse("localhost:27017");
ReplicaSets sets2 = ReplicaSets.parse("localhost:27017");
assertThat(sets1.haveChangedSince(sets2)).isFalse();
sets1 = ReplicaSets.parse("shard1=myReplicaSet/host1:27017,[fe80::601:9bff:feab:ec01]:27017;shard2=otherReplicaset/1.2.3.4:27017,localhost:28017");
sets2 = ReplicaSets.parse("shard1=myReplicaSet/host1:27017,[fe80::601:9bff:feab:ec01]:27017;shard2=otherReplicaset/1.2.3.4:27017,localhost:28017");
assertThat(sets1.haveChangedSince(sets2)).isFalse();
}
@Test
public void shouldConsiderChangedReplicaSetsWithOneReplicaSetContainingDifferentLocalServers() {
ReplicaSets sets1 = ReplicaSets.parse("localhost:27017");
ReplicaSets sets2 = ReplicaSets.parse("localhost:27017,host2:27017");
assertThat(sets1.haveChangedSince(sets2)).isTrue();
}
@Test
public void shouldConsiderUnchangedReplicaSetsWithAdditionalServerAddressInExistingReplicaSet() {
ReplicaSets sets1 = ReplicaSets.parse("rs1/localhost:27017");
ReplicaSets sets2 = ReplicaSets.parse("rs1/localhost:27017,host2:27017");
assertThat(sets1.haveChangedSince(sets2)).isFalse();
}
@Test
public void shouldConsiderChangedReplicaSetsWithAdditionalReplicaSet() {
ReplicaSets sets1 = ReplicaSets.parse("rs1/localhost:27017;rs2/host2:17017");
ReplicaSets sets2 = ReplicaSets.parse("rs1/localhost:27017");
assertThat(sets1.haveChangedSince(sets2)).isTrue();
}
@Test
public void shouldConsiderChangedReplicaSetsWithRemovedReplicaSet() {
ReplicaSets sets1 = ReplicaSets.parse("rs1/localhost:27017");
ReplicaSets sets2 = ReplicaSets.parse("rs1/localhost:27017;rs2/host2:17017");
assertThat(sets1.haveChangedSince(sets2)).isTrue();
}
@Test
public void shouldNotSubdivideOneReplicaSet() {
sets = ReplicaSets.parse("rs1/host1:27017,host2:27017");
List<ReplicaSets> divided = new ArrayList<>();
sets.subdivide(1, divided::add);
assertThat(divided.size()).isEqualTo(1);
assertThat(divided.get(0)).isSameAs(sets);
}
@Test
public void shouldNotSubdivideMultipleReplicaSetsIntoOneGroup() {
sets = ReplicaSets.parse("rs1/host1:27017,host2:27017;rs2/host3:27017");
List<ReplicaSets> divided = new ArrayList<>();
sets.subdivide(1, divided::add);
assertThat(divided.size()).isEqualTo(1);
assertThat(divided.get(0)).isSameAs(sets);
}
@Test
public void shouldSubdivideMultipleReplicaSetsWithIntoMultipleGroups() {
sets = ReplicaSets.parse("rs1/host1:27017,host2:27017;rs2/host3:27017");
List<ReplicaSets> divided = new ArrayList<>();
sets.subdivide(2, divided::add);
assertThat(divided.size()).isEqualTo(2);
ReplicaSets dividedSet1 = divided.get(0);
assertThat(dividedSet1.replicaSetCount()).isEqualTo(1);
assertThat(dividedSet1.all().get(0)).isSameAs(sets.all().get(0));
ReplicaSets dividedSet2 = divided.get(1);
assertThat(dividedSet2.replicaSetCount()).isEqualTo(1);
assertThat(dividedSet2.all().get(0)).isSameAs(sets.all().get(1));
}
}

View File

@ -0,0 +1,257 @@
/*
* 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.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.bson.Document;
import org.bson.conversions.Bson;
import org.bson.types.ObjectId;
import org.junit.Test;
import com.mongodb.client.FindIterable;
import com.mongodb.client.MongoCollection;
import com.mongodb.client.MongoCursor;
import com.mongodb.client.MongoDatabase;
import com.mongodb.client.model.Filters;
import com.mongodb.client.model.InsertOneOptions;
import static org.fest.assertions.Assertions.assertThat;
import io.debezium.data.Envelope.Operation;
import io.debezium.data.VerifyRecord;
import io.debezium.util.Testing;
public class ReplicatorIT extends AbstractMongoIT {
@Test
public void shouldReplicateContent() throws InterruptedException {
Testing.Print.disable();
// Update the configuration to add a collection filter ...
useConfiguration(config.edit()
.with(MongoDbConnectorConfig.MAX_FAILED_CONNECTIONS, 1)
.with(MongoDbConnectorConfig.COLLECTION_WHITELIST, "dbA.contacts")
.build());
// ------------------------------------------------------------------------------
// ADD A DOCUMENT
// ------------------------------------------------------------------------------
// Add a document to the 'contacts' database ...
primary.execute("shouldCreateContactsDatabase", mongo -> {
Testing.debug("Populating the 'dbA.contacts' collection");
// Create a database and a collection in that database ...
MongoDatabase db = mongo.getDatabase("dbA");
MongoCollection<Document> contacts = db.getCollection("contacts");
InsertOneOptions insertOptions = new InsertOneOptions().bypassDocumentValidation(true);
contacts.insertOne(Document.parse("{ \"name\":\"Jon Snow\"}"), insertOptions);
assertThat(db.getCollection("contacts").count()).isEqualTo(1);
// Read the collection to make sure we can find our document ...
Bson filter = Filters.eq("name", "Jon Snow");
FindIterable<Document> movieResults = db.getCollection("contacts").find(filter);
try (MongoCursor<Document> cursor = movieResults.iterator();) {
assertThat(cursor.tryNext().getString("name")).isEqualTo("Jon Snow");
assertThat(cursor.tryNext()).isNull();
}
Testing.debug("Completed document to 'dbA.contacts' collection");
});
// Start the replicator ...
List<SourceRecord> records = new LinkedList<>();
Replicator replicator = new Replicator(context, replicaSet, records::add);
Thread thread = new Thread(replicator::run);
thread.start();
// Sleep for 2 seconds ...
Thread.sleep(2000);
// ------------------------------------------------------------------------------
// ADD A SECOND DOCUMENT
// ------------------------------------------------------------------------------
// Add more documents to the 'contacts' database ...
final Object[] expectedNames = { "Jon Snow", "Sally Hamm" };
primary.execute("shouldCreateContactsDatabase", mongo -> {
Testing.debug("Populating the 'dbA.contacts' collection");
// Create a database and a collection in that database ...
MongoDatabase db = mongo.getDatabase("dbA");
MongoCollection<Document> contacts = db.getCollection("contacts");
InsertOneOptions insertOptions = new InsertOneOptions().bypassDocumentValidation(true);
contacts.insertOne(Document.parse("{ \"name\":\"Sally Hamm\"}"), insertOptions);
assertThat(db.getCollection("contacts").count()).isEqualTo(2);
// Read the collection to make sure we can find our documents ...
FindIterable<Document> movieResults = db.getCollection("contacts").find();
Set<String> foundNames = new HashSet<>();
try (MongoCursor<Document> cursor = movieResults.iterator();) {
while (cursor.hasNext()) {
String name = cursor.next().getString("name");
foundNames.add(name);
}
}
assertThat(foundNames).containsOnly(expectedNames);
Testing.debug("Completed document to 'dbA.contacts' collection");
});
// For for a minimum number of events or max time ...
int numEventsExpected = 2; // both documents
long stop = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(3);
while (records.size() < numEventsExpected && System.currentTimeMillis() < stop) {
Thread.sleep(100);
}
// ------------------------------------------------------------------------------
// STOP REPLICATOR AND VERIFY WE FOUND A TOTAL OF 2 EVENTS
// ------------------------------------------------------------------------------
replicator.stop();
// Verify each record is valid and that we found the two records we expect ...
final Set<String> foundNames = new HashSet<>();
records.forEach(record -> {
VerifyRecord.isValid(record);
Struct value = (Struct) record.value();
String after = value.getString("after");
Document afterDoc = Document.parse(after);
foundNames.add(afterDoc.getString("name"));
Operation op = Operation.forCode(value.getString("op"));
assertThat(op == Operation.READ || op == Operation.CREATE).isTrue();
});
assertThat(records.size()).isEqualTo(2);
assertThat(foundNames).containsOnly(expectedNames);
// ------------------------------------------------------------------------------
// RESTART REPLICATOR FROM SAME POSITON
// ------------------------------------------------------------------------------
reuseConfiguration(config);
// Start the replicator again ...
records = new LinkedList<>();
replicator = new Replicator(context, replicaSet, records::add);
thread = new Thread(replicator::run);
thread.start();
// Sleep for 2 seconds ...
Thread.sleep(2000);
// Stop the replicator ...
replicator.stop();
// We should not have found any new records ...
records.forEach(record -> {
VerifyRecord.isValid(record);
});
assertThat(records.isEmpty()).isTrue();
// ------------------------------------------------------------------------------
// START REPLICATOR AND ALSO REMOVE A DOCUMENT
// ------------------------------------------------------------------------------
// Update the configuration and don't use a collection filter ...
reuseConfiguration(config.edit()
.with(MongoDbConnectorConfig.MAX_FAILED_CONNECTIONS, 1)
.build());
// Start the replicator again ...
records = new LinkedList<>();
replicator = new Replicator(context, replicaSet, records::add);
thread = new Thread(replicator::run);
thread.start();
// Sleep for 2 seconds ...
Thread.sleep(2000);
// Remove Jon Snow ...
AtomicReference<ObjectId> jonSnowId = new AtomicReference<>();
primary.execute("removeJonSnow", mongo -> {
MongoDatabase db = mongo.getDatabase("dbA");
MongoCollection<Document> contacts = db.getCollection("contacts");
// Read the collection to make sure we can find our document ...
Bson filter = Filters.eq("name", "Jon Snow");
FindIterable<Document> movieResults = db.getCollection("contacts").find(filter);
try (MongoCursor<Document> cursor = movieResults.iterator();) {
Document doc = cursor.tryNext();
assertThat(doc.getString("name")).isEqualTo("Jon Snow");
assertThat(cursor.tryNext()).isNull();
jonSnowId.set(doc.getObjectId("_id"));
assertThat(jonSnowId.get()).isNotNull();
}
// Remove the document by filter ...
contacts.deleteOne(Filters.eq("name", "Jon Snow"));
Testing.debug("Removed the Jon Snow document from 'dbA.contacts' collection");
});
// For for a minimum number of events or max time ...
numEventsExpected = 1; // just one delete event
stop = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(3);
while (records.size() < numEventsExpected && System.currentTimeMillis() < stop) {
Thread.sleep(100);
}
// Stop the replicator ...
replicator.stop();
// Verify each record is valid and that we found the one new DELETE record we expect ...
Set<ObjectId> foundIds = new HashSet<>();
records.forEach(record -> {
VerifyRecord.isValid(record);
Struct key = (Struct) record.key();
ObjectId id = new ObjectId(key.getString("_id"));
foundIds.add(id);
if (record.value() != null) {
Struct value = (Struct) record.value();
Operation op = Operation.forCode(value.getString("op"));
assertThat(op).isEqualTo(Operation.DELETE);
}
});
assertThat(records.size()).isEqualTo(2); // 1 delete and 1 tombstone
// ------------------------------------------------------------------------------
// START REPLICATOR TO PERFORM SNAPSHOT
// ------------------------------------------------------------------------------
// Update the configuration and don't use a collection filter ...
useConfiguration(config);
// Start the replicator again ...
records = new LinkedList<>();
replicator = new Replicator(context, replicaSet, records::add);
thread = new Thread(replicator::run);
thread.start();
// Sleep for 2 seconds ...
Thread.sleep(2000);
// Stop the replicator ...
replicator.stop();
// Verify each record is valid and that we found the two records we expect ...
foundNames.clear();
records.forEach(record -> {
VerifyRecord.isValid(record);
Struct value = (Struct) record.value();
String after = value.getString("after");
Document afterDoc = Document.parse(after);
foundNames.add(afterDoc.getString("name"));
Operation op = Operation.forCode(value.getString("op"));
assertThat(op).isEqualTo(Operation.READ);
});
// We should not have found any new records ...
assertThat(records.size()).isEqualTo(1);
Object[] allExpectedNames = { "Sally Hamm" };
assertThat(foundNames).containsOnly(allExpectedNames);
}
}

View File

@ -0,0 +1,212 @@
/*
* 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.util.Map;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.Struct;
import org.bson.BsonTimestamp;
import org.bson.Document;
import org.junit.Before;
import org.junit.Test;
import static org.fest.assertions.Assertions.assertThat;
/**
* @author Randall Hauch
*
*/
public class SourceInfoTest {
private static String REPLICA_SET_NAME = "myReplicaSet";
private SourceInfo source;
private Map<String, String> partition;
@Before
public void beforeEach() {
source = new SourceInfo("serverX");
}
@Test
public void shouldHaveSchemaForSource() {
Schema schema = source.schema();
assertThat(schema.name()).isNotEmpty();
assertThat(schema.version()).isNotNull();
assertThat(schema.field(SourceInfo.SERVER_NAME).schema()).isEqualTo(Schema.STRING_SCHEMA);
assertThat(schema.field(SourceInfo.REPLICA_SET_NAME).schema()).isEqualTo(Schema.STRING_SCHEMA);
assertThat(schema.field(SourceInfo.NAMESPACE).schema()).isEqualTo(Schema.STRING_SCHEMA);
assertThat(schema.field(SourceInfo.TIMESTAMP).schema()).isEqualTo(Schema.INT32_SCHEMA);
assertThat(schema.field(SourceInfo.ORDER).schema()).isEqualTo(Schema.INT32_SCHEMA);
assertThat(schema.field(SourceInfo.OPERATION_ID).schema()).isEqualTo(Schema.OPTIONAL_INT64_SCHEMA);
assertThat(schema.field(SourceInfo.INITIAL_SYNC).schema()).isEqualTo(Schema.OPTIONAL_BOOLEAN_SCHEMA);
}
@Test
public void shouldProducePartitionMap() {
partition = source.partition(REPLICA_SET_NAME);
assertThat(partition.get(SourceInfo.REPLICA_SET_NAME)).isEqualTo(REPLICA_SET_NAME);
assertThat(partition.get(SourceInfo.SERVER_ID_KEY)).isEqualTo("serverX");
assertThat(partition.size()).isEqualTo(2);
}
@Test
public void shouldReturnSamePartitionMapForSameReplicaName() {
partition = source.partition(REPLICA_SET_NAME);
assertThat(partition).isSameAs(source.partition(REPLICA_SET_NAME));
}
@Test
public void shouldSetAndReturnRecordedOffset() {
Document event = new Document().append("ts", new BsonTimestamp(100, 2))
.append("h", new Long(1987654321))
.append("ns", "dbA.collectA");
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(false);
source.offsetStructForEvent(REPLICA_SET_NAME, event);
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(true);
Map<String, ?> offset = source.lastOffset(REPLICA_SET_NAME);
assertThat(offset.get(SourceInfo.TIMESTAMP)).isEqualTo(100);
assertThat(offset.get(SourceInfo.ORDER)).isEqualTo(2);
assertThat(offset.get(SourceInfo.OPERATION_ID)).isEqualTo(1987654321L);
// Create a new source info and set the offset ...
Map<String,String> partition = source.partition(REPLICA_SET_NAME);
source = new SourceInfo("serverX");
source.setOffsetFor(partition, offset);
offset = source.lastOffset(REPLICA_SET_NAME);
assertThat(offset.get(SourceInfo.TIMESTAMP)).isEqualTo(100);
assertThat(offset.get(SourceInfo.ORDER)).isEqualTo(2);
assertThat(offset.get(SourceInfo.OPERATION_ID)).isEqualTo(1987654321L);
BsonTimestamp ts = source.lastOffsetTimestamp(REPLICA_SET_NAME);
assertThat(ts.getTime()).isEqualTo(100);
assertThat(ts.getInc()).isEqualTo(2);
Struct struct = source.lastOffsetStruct(REPLICA_SET_NAME,new CollectionId(REPLICA_SET_NAME,"dbA","collectA"));
assertThat(struct.getInt32(SourceInfo.TIMESTAMP)).isEqualTo(100);
assertThat(struct.getInt32(SourceInfo.ORDER)).isEqualTo(2);
assertThat(struct.getInt64(SourceInfo.OPERATION_ID)).isEqualTo(1987654321L);
assertThat(struct.getString(SourceInfo.NAMESPACE)).isEqualTo("dbA.collectA");
assertThat(struct.getString(SourceInfo.REPLICA_SET_NAME)).isEqualTo(REPLICA_SET_NAME);
assertThat(struct.getString(SourceInfo.SERVER_NAME)).isEqualTo("serverX");
assertThat(struct.getBoolean(SourceInfo.INITIAL_SYNC)).isNull();
}
@Test
public void shouldReturnOffsetForUnusedReplicaName() {
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(false);
Map<String, ?> offset = source.lastOffset(REPLICA_SET_NAME);
assertThat(offset.get(SourceInfo.TIMESTAMP)).isEqualTo(0);
assertThat(offset.get(SourceInfo.ORDER)).isEqualTo(0);
assertThat(offset.get(SourceInfo.OPERATION_ID)).isNull();
BsonTimestamp ts = source.lastOffsetTimestamp(REPLICA_SET_NAME);
assertThat(ts.getTime()).isEqualTo(0);
assertThat(ts.getInc()).isEqualTo(0);
Struct struct = source.lastOffsetStruct(REPLICA_SET_NAME,new CollectionId(REPLICA_SET_NAME,"dbA","collectA"));
assertThat(struct.getInt32(SourceInfo.TIMESTAMP)).isEqualTo(0);
assertThat(struct.getInt32(SourceInfo.ORDER)).isEqualTo(0);
assertThat(struct.getInt64(SourceInfo.OPERATION_ID)).isNull();
assertThat(struct.getString(SourceInfo.NAMESPACE)).isEqualTo("dbA.collectA");
assertThat(struct.getString(SourceInfo.REPLICA_SET_NAME)).isEqualTo(REPLICA_SET_NAME);
assertThat(struct.getString(SourceInfo.SERVER_NAME)).isEqualTo("serverX");
assertThat(struct.getBoolean(SourceInfo.INITIAL_SYNC)).isNull();
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(false);
}
@Test
public void shouldReturnRecordedOffsetForUsedReplicaName() {
Document event = new Document().append("ts", new BsonTimestamp(100, 2))
.append("h", new Long(1987654321))
.append("ns", "dbA.collectA");
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(false);
source.offsetStructForEvent(REPLICA_SET_NAME, event);
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(true);
Map<String, ?> offset = source.lastOffset(REPLICA_SET_NAME);
assertThat(offset.get(SourceInfo.TIMESTAMP)).isEqualTo(100);
assertThat(offset.get(SourceInfo.ORDER)).isEqualTo(2);
assertThat(offset.get(SourceInfo.OPERATION_ID)).isEqualTo(1987654321L);
BsonTimestamp ts = source.lastOffsetTimestamp(REPLICA_SET_NAME);
assertThat(ts.getTime()).isEqualTo(100);
assertThat(ts.getInc()).isEqualTo(2);
Struct struct = source.lastOffsetStruct(REPLICA_SET_NAME,new CollectionId(REPLICA_SET_NAME,"dbA","collectA"));
assertThat(struct.getInt32(SourceInfo.TIMESTAMP)).isEqualTo(100);
assertThat(struct.getInt32(SourceInfo.ORDER)).isEqualTo(2);
assertThat(struct.getInt64(SourceInfo.OPERATION_ID)).isEqualTo(1987654321L);
assertThat(struct.getString(SourceInfo.NAMESPACE)).isEqualTo("dbA.collectA");
assertThat(struct.getString(SourceInfo.REPLICA_SET_NAME)).isEqualTo(REPLICA_SET_NAME);
assertThat(struct.getString(SourceInfo.SERVER_NAME)).isEqualTo("serverX");
assertThat(struct.getBoolean(SourceInfo.INITIAL_SYNC)).isNull();
}
@Test
public void shouldReturnOffsetForUnusedReplicaNameDuringInitialSync() {
source.startInitialSync(REPLICA_SET_NAME);
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(false);
Map<String, ?> offset = source.lastOffset(REPLICA_SET_NAME);
assertThat(offset.get(SourceInfo.TIMESTAMP)).isEqualTo(0);
assertThat(offset.get(SourceInfo.ORDER)).isEqualTo(0);
assertThat(offset.get(SourceInfo.OPERATION_ID)).isNull();
BsonTimestamp ts = source.lastOffsetTimestamp(REPLICA_SET_NAME);
assertThat(ts.getTime()).isEqualTo(0);
assertThat(ts.getInc()).isEqualTo(0);
Struct struct = source.lastOffsetStruct(REPLICA_SET_NAME,new CollectionId(REPLICA_SET_NAME,"dbA","collectA"));
assertThat(struct.getInt32(SourceInfo.TIMESTAMP)).isEqualTo(0);
assertThat(struct.getInt32(SourceInfo.ORDER)).isEqualTo(0);
assertThat(struct.getInt64(SourceInfo.OPERATION_ID)).isNull();
assertThat(struct.getString(SourceInfo.NAMESPACE)).isEqualTo("dbA.collectA");
assertThat(struct.getString(SourceInfo.REPLICA_SET_NAME)).isEqualTo(REPLICA_SET_NAME);
assertThat(struct.getString(SourceInfo.SERVER_NAME)).isEqualTo("serverX");
assertThat(struct.getBoolean(SourceInfo.INITIAL_SYNC)).isEqualTo(true);
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(false);
}
@Test
public void shouldReturnRecordedOffsetForUsedReplicaNameDuringInitialSync() {
source.startInitialSync(REPLICA_SET_NAME);
Document event = new Document().append("ts", new BsonTimestamp(100, 2))
.append("h", new Long(1987654321))
.append("ns", "dbA.collectA");
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(false);
source.offsetStructForEvent(REPLICA_SET_NAME, event);
assertThat(source.hasOffset(REPLICA_SET_NAME)).isEqualTo(true);
Map<String, ?> offset = source.lastOffset(REPLICA_SET_NAME);
assertThat(offset.get(SourceInfo.TIMESTAMP)).isEqualTo(100);
assertThat(offset.get(SourceInfo.ORDER)).isEqualTo(2);
assertThat(offset.get(SourceInfo.OPERATION_ID)).isEqualTo(1987654321L);
BsonTimestamp ts = source.lastOffsetTimestamp(REPLICA_SET_NAME);
assertThat(ts.getTime()).isEqualTo(100);
assertThat(ts.getInc()).isEqualTo(2);
Struct struct = source.lastOffsetStruct(REPLICA_SET_NAME,new CollectionId(REPLICA_SET_NAME,"dbA","collectA"));
assertThat(struct.getInt32(SourceInfo.TIMESTAMP)).isEqualTo(100);
assertThat(struct.getInt32(SourceInfo.ORDER)).isEqualTo(2);
assertThat(struct.getInt64(SourceInfo.OPERATION_ID)).isEqualTo(1987654321L);
assertThat(struct.getString(SourceInfo.NAMESPACE)).isEqualTo("dbA.collectA");
assertThat(struct.getString(SourceInfo.REPLICA_SET_NAME)).isEqualTo(REPLICA_SET_NAME);
assertThat(struct.getString(SourceInfo.SERVER_NAME)).isEqualTo("serverX");
assertThat(struct.getBoolean(SourceInfo.INITIAL_SYNC)).isEqualTo(true);
}
}

View File

@ -0,0 +1,43 @@
/*
* 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 org.fest.assertions.StringAssert;
import org.junit.Before;
import org.junit.Test;
import static org.fest.assertions.Assertions.assertThat;
/**
* @author Randall Hauch
*
*/
public class TopicSelectorTest {
private TopicSelector noPrefix;
private TopicSelector withPrefix;
@Before
public void beforeEach() {
noPrefix = TopicSelector.defaultSelector(null);
withPrefix = TopicSelector.defaultSelector("prefix");
}
@Test
public void shouldHandleCollectionIdWithDatabaseAndCollection() {
assertTopic(noPrefix,dbAndCollection("db","coll")).isEqualTo("db.coll");
assertTopic(withPrefix,dbAndCollection("db","coll")).isEqualTo("prefix.db.coll");
}
protected StringAssert assertTopic(TopicSelector selector, CollectionId collectionId) {
return assertThat(selector.getTopic(collectionId));
}
protected CollectionId dbAndCollection( String dbName, String collectionName ) {
return new CollectionId("rs0",dbName,collectionName);
}
}

View File

@ -0,0 +1,11 @@
# Direct log messages to stdout
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.Target=System.out
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %X{dbz.connectorType}|%X{dbz.connectorName}|%X{dbz.connectorContext} %m [%c]%n
# Root logger option
log4j.rootLogger=INFO, stdout
# Set up the default logging to be INFO level, then override specific units
log4j.logger.io.debezium=INFO

View File

@ -0,0 +1,6 @@
{"address": {"building": "1007", "coord": [-73.856077, 40.848447], "street": "Morris Park Ave", "zipcode": "10462"}, "borough": "Bronx", "cuisine": "Bakery", "grades": [{"date": {"$date": 1393804800000}, "grade": "A", "score": 2}, {"date": {"$date": 1378857600000}, "grade": "A", "score": 6}, {"date": {"$date": 1358985600000}, "grade": "A", "score": 10}, {"date": {"$date": 1322006400000}, "grade": "A", "score": 9}, {"date": {"$date": 1299715200000}, "grade": "B", "score": 14}], "name": "Morris Park Bake Shop", "restaurant_id": "30075445"}
{"address": {"building": "469", "coord": [-73.961704, 40.662942], "street": "Flatbush Avenue", "zipcode": "11225"}, "borough": "Brooklyn", "cuisine": "Hamburgers", "grades": [{"date": {"$date": 1419897600000}, "grade": "A", "score": 8}, {"date": {"$date": 1404172800000}, "grade": "B", "score": 23}, {"date": {"$date": 1367280000000}, "grade": "A", "score": 12}, {"date": {"$date": 1336435200000}, "grade": "A", "score": 12}], "name": "Wendy'S", "restaurant_id": "30112340"}
{"address": {"building": "351", "coord": [-73.98513559999999, 40.7676919], "street": "West 57 Street", "zipcode": "10019"}, "borough": "Manhattan", "cuisine": "Irish", "grades": [{"date": {"$date": 1409961600000}, "grade": "A", "score": 2}, {"date": {"$date": 1374451200000}, "grade": "A", "score": 11}, {"date": {"$date": 1343692800000}, "grade": "A", "score": 12}, {"date": {"$date": 1325116800000}, "grade": "A", "score": 12}], "name": "Dj Reynolds Pub And Restaurant", "restaurant_id": "30191841"}
{"address": {"building": "2780", "coord": [-73.98241999999999, 40.579505], "street": "Stillwell Avenue", "zipcode": "11224"}, "borough": "Brooklyn", "cuisine": "American ", "grades": [{"date": {"$date": 1402358400000}, "grade": "A", "score": 5}, {"date": {"$date": 1370390400000}, "grade": "A", "score": 7}, {"date": {"$date": 1334275200000}, "grade": "A", "score": 12}, {"date": {"$date": 1318377600000}, "grade": "A", "score": 12}], "name": "Riviera Caterer", "restaurant_id": "40356018"}
{"address": {"building": "97-22", "coord": [-73.8601152, 40.7311739], "street": "63 Road", "zipcode": "11374"}, "borough": "Queens", "cuisine": "Jewish/Kosher", "grades": [{"date": {"$date": 1416787200000}, "grade": "Z", "score": 20}, {"date": {"$date": 1358380800000}, "grade": "A", "score": 13}, {"date": {"$date": 1343865600000}, "grade": "A", "score": 13}, {"date": {"$date": 1323907200000}, "grade": "B", "score": 25}], "name": "Tov Kosher Kitchen", "restaurant_id": "40356068"}
{"address": {"building": "8825", "coord": [-73.8803827, 40.7643124], "street": "Astoria Boulevard", "zipcode": "11369"}, "borough": "Queens", "cuisine": "American ", "grades": [{"date": {"$date": 1416009600000}, "grade": "Z", "score": 38}, {"date": {"$date": 1398988800000}, "grade": "A", "score": 10}, {"date": {"$date": 1362182400000}, "grade": "A", "score": 7}, {"date": {"$date": 1328832000000}, "grade": "A", "score": 13}], "name": "Brunos On The Boulevard", "restaurant_id": "40356151"}

View File

@ -0,0 +1,4 @@
{"address": {"building": "7114", "coord": [-73.9068506, 40.6199034], "street": "Avenue U", "zipcode": "11234"}, "borough": "Brooklyn", "cuisine": "Delicatessen", "grades": [{"date": {"$date": 1401321600000}, "grade": "A", "score": 10}, {"date": {"$date": 1389657600000}, "grade": "A", "score": 10}, {"date": {"$date": 1375488000000}, "grade": "A", "score": 8}, {"date": {"$date": 1342569600000}, "grade": "A", "score": 10}, {"date": {"$date": 1331251200000}, "grade": "A", "score": 13}, {"date": {"$date": 1318550400000}, "grade": "A", "score": 9}], "name": "Wilken'S Fine Food", "restaurant_id": "40356483"}
{"address": {"building": "6409", "coord": [-74.00528899999999, 40.628886], "street": "11 Avenue", "zipcode": "11219"}, "borough": "Brooklyn", "cuisine": "American ", "grades": [{"date": {"$date": 1405641600000}, "grade": "A", "score": 12}, {"date": {"$date": 1375142400000}, "grade": "A", "score": 12}, {"date": {"$date": 1360713600000}, "grade": "A", "score": 11}, {"date": {"$date": 1345075200000}, "grade": "A", "score": 2}, {"date": {"$date": 1313539200000}, "grade": "A", "score": 11}], "name": "Regina Caterers", "restaurant_id": "40356649"}
{"address": {"building": "1839", "coord": [-73.9482609, 40.6408271], "street": "Nostrand Avenue", "zipcode": "11226"}, "borough": "Brooklyn", "cuisine": "Ice Cream, Gelato, Yogurt, Ices", "grades": [{"date": {"$date": 1405296000000}, "grade": "A", "score": 12}, {"date": {"$date": 1373414400000}, "grade": "A", "score": 8}, {"date": {"$date": 1341964800000}, "grade": "A", "score": 5}, {"date": {"$date": 1329955200000}, "grade": "A", "score": 8}], "name": "Taste The Tropics Ice Cream", "restaurant_id": "40356731"}
{"address": {"building": "2300", "coord": [-73.8786113, 40.8502883], "street": "Southern Boulevard", "zipcode": "10460"}, "borough": "Bronx", "cuisine": "American ", "grades": [{"date": {"$date": 1401235200000}, "grade": "A", "score": 11}, {"date": {"$date": 1371600000000}, "grade": "A", "score": 4}, {"date": {"$date": 1339718400000}, "grade": "A", "score": 3}], "name": "Wild Asia", "restaurant_id": "40357217"}

View File

@ -0,0 +1,5 @@
{"address": {"building": "976", "coord": [-73.92701509999999, 40.6620192], "street": "Rutland Road", "zipcode": "11212"}, "borough": "Brooklyn", "cuisine": "Chinese", "grades": [{"date": {"$date": 1398211200000}, "grade": "A", "score": 13}, {"date": {"$date": 1364256000000}, "grade": "A", "score": 10}, {"date": {"$date": 1331596800000}, "grade": "A", "score": 4}, {"date": {"$date": 1321401600000}, "grade": "A", "score": 13}], "name": "Golden Pavillion", "restaurant_id": "40363920"}
{"address": {"building": "148", "coord": [-73.9806854, 40.7778589], "street": "West 72 Street", "zipcode": "10023"}, "borough": "Manhattan", "cuisine": "Pizza", "grades": [{"date": {"$date": 1417996800000}, "grade": "A", "score": 13}, {"date": {"$date": 1399248000000}, "grade": "B", "score": 18}, {"date": {"$date": 1365120000000}, "grade": "A", "score": 13}, {"date": {"$date": 1333065600000}, "grade": "A", "score": 9}], "name": "Domino'S Pizza", "restaurant_id": "40363945"}
{"address": {"building": "364", "coord": [-73.96084119999999, 40.8014307], "street": "West 110 Street", "zipcode": "10025"}, "borough": "Manhattan", "cuisine": "American ", "grades": [{"date": {"$date": 1409788800000}, "grade": "B", "score": 20}, {"date": {"$date": 1393372800000}, "grade": "B", "score": 23}, {"date": {"$date": 1364169600000}, "grade": "A", "score": 12}, {"date": {"$date": 1329782400000}, "grade": "A", "score": 8}], "name": "Spoon Bread Catering", "restaurant_id": "40364179"}
{"address": {"building": "1423", "coord": [-73.9615132, 40.6253268], "street": "Avenue J", "zipcode": "11230"}, "borough": "Brooklyn", "cuisine": "Jewish/Kosher", "grades": [{"date": {"$date": 1418947200000}, "grade": "A", "score": 10}, {"date": {"$date": 1386201600000}, "grade": "A", "score": 11}, {"date": {"$date": 1354752000000}, "grade": "A", "score": 9}], "name": "Kosher Bagel Hole", "restaurant_id": "40364220"}
{"address": {"building": "0", "coord": [-84.2040813, 9.9986585], "street": "Guardia Airport Parking", "zipcode": "11371"}, "borough": "Queens", "cuisine": "American ", "grades": [{"date": {"$date": 1400198400000}, "grade": "A", "score": 9}, {"date": {"$date": 1368144000000}, "grade": "A", "score": 10}, {"date": {"$date": 1337040000000}, "grade": "A", "score": 9}, {"date": {"$date": 1320192000000}, "grade": "C", "score": 32}], "name": "Terminal Cafe/Yankee Clipper", "restaurant_id": "40364262"}

View File

@ -0,0 +1,8 @@
{"address": {"building": "1", "coord": [-73.97166039999999, 40.764832], "street": "East 60 Street", "zipcode": "10022"}, "borough": "Manhattan", "cuisine": "American ", "grades": [{"date": {"$date": 1413417600000}, "grade": "B", "score": 24}, {"date": {"$date": 1398988800000}, "grade": "A", "score": 4}, {"date": {"$date": 1364860800000}, "grade": "A", "score": 13}, {"date": {"$date": 1350604800000}, "grade": "A", "score": 12}, {"date": {"$date": 1335484800000}, "grade": "B", "score": 17}, {"date": {"$date": 1322524800000}, "grade": "A", "score": 11}], "name": "Metropolitan Club", "restaurant_id": "40364347"}
{"address": {"building": "837", "coord": [-73.9712, 40.751703], "street": "2 Avenue", "zipcode": "10017"}, "borough": "Manhattan", "cuisine": "American ", "grades": [{"date": {"$date": 1405987200000}, "grade": "B", "score": 19}, {"date": {"$date": 1380153600000}, "grade": "A", "score": 12}, {"date": {"$date": 1361836800000}, "grade": "A", "score": 11}, {"date": {"$date": 1335744000000}, "grade": "A", "score": 8}, {"date": {"$date": 1317772800000}, "grade": "A", "score": 12}], "name": "Palm Restaurant", "restaurant_id": "40364355"}
{"address": {"building": "21", "coord": [-73.9774394, 40.7604522], "street": "West 52 Street", "zipcode": "10019"}, "borough": "Manhattan", "cuisine": "American ", "grades": [{"date": {"$date": 1400025600000}, "grade": "A", "score": 12}, {"date": {"$date": 1376352000000}, "grade": "A", "score": 12}, {"date": {"$date": 1333497600000}, "grade": "A", "score": 12}], "name": "21 Club", "restaurant_id": "40364362"}
{"address": {"building": "658", "coord": [-73.81363999999999, 40.82941100000001], "street": "Clarence Ave", "zipcode": "10465"}, "borough": "Bronx", "cuisine": "American ", "grades": [{"date": {"$date": 1403308800000}, "grade": "A", "score": 5}, {"date": {"$date": 1341964800000}, "grade": "A", "score": 10}], "name": "Manhem Club", "restaurant_id": "40364363"}
{"address": {"building": "1028", "coord": [-73.966032, 40.762832], "street": "3 Avenue", "zipcode": "10065"}, "borough": "Manhattan", "cuisine": "Italian", "grades": [{"date": {"$date": 1410825600000}, "grade": "A", "score": 13}, {"date": {"$date": 1393200000000}, "grade": "A", "score": 10}, {"date": {"$date": 1367539200000}, "grade": "A", "score": 10}, {"date": {"$date": 1345420800000}, "grade": "A", "score": 7}, {"date": {"$date": 1329091200000}, "grade": "A", "score": 9}], "name": "Isle Of Capri Resturant", "restaurant_id": "40364373"}
{"address": {"building": "45", "coord": [-73.9891878, 40.7375638], "street": "East 18 Street", "zipcode": "10003"}, "borough": "Manhattan", "cuisine": "American ", "grades": [{"date": {"$date": 1412726400000}, "grade": "A", "score": 10}, {"date": {"$date": 1381363200000}, "grade": "A", "score": 9}, {"date": {"$date": 1366761600000}, "grade": "C", "score": 36}, {"date": {"$date": 1326067200000}, "grade": "A", "score": 9}], "name": "Old Town Bar & Restaurant", "restaurant_id": "40364389"}
{"address": {"building": "261", "coord": [-73.94839189999999, 40.7224876], "street": "Driggs Avenue", "zipcode": "11222"}, "borough": "Brooklyn", "cuisine": "Polish", "grades": [{"date": {"$date": 1401494400000}, "grade": "A", "score": 2}, {"date": {"$date": 1368144000000}, "grade": "A", "score": 3}, {"date": {"$date": 1329436800000}, "grade": "A", "score": 6}, {"date": {"$date": 1318550400000}, "grade": "C", "score": 54}], "name": "Polish National Home", "restaurant_id": "40364404"}
{"address": {"building": "62", "coord": [-74.00310999999999, 40.7348888], "street": "Charles Street", "zipcode": "10014"}, "borough": "Manhattan", "cuisine": "Latin (Cuban, Dominican, Puerto Rican, South & Central American)", "grades": [{"date": {"$date": 1398988800000}, "grade": "A", "score": 11}, {"date": {"$date": 1369008000000}, "grade": "A", "score": 11}, {"date": {"$date": 1337817600000}, "grade": "A", "score": 7}, {"date": {"$date": 1326844800000}, "grade": "A", "score": 13}, {"date": {"$date": 1317600000000}, "grade": "A", "score": 10}], "name": "Seville Restaurant", "restaurant_id": "40364439"}

View File

@ -0,0 +1,6 @@
{ "_id" : 1, "data" : "hello, 1" }
{ "_id" : 3, "data" : "hello, 3" }
{ "_id" : 5, "data" : "hello, 5" }
{ "_id" : 7, "data" : "hello, 7" }
{ "_id" : 9, "data" : "hello, 9" }
{ "_id" : 11, "data" : "hello, 11" }

View File

@ -147,27 +147,31 @@ public List<SourceRecord> poll() throws InterruptedException {
@Override @Override
public void stop() { public void stop() {
if (context != null) {
logger.info("Stopping MySQL connector task"); logger.info("Stopping MySQL connector task");
// We need to explicitly stop both readers, in this order. If we were to instead call 'currentReader.stop()', there // We need to explicitly stop both readers, in this order. If we were to instead call 'currentReader.stop()', there
// is a chance without synchronization that we'd miss the transition and stop only the snapshot reader. And stopping both // is a chance without synchronization that we'd miss the transition and stop only the snapshot reader. And stopping
// both
// is far simpler and more efficient than synchronizing ... // is far simpler and more efficient than synchronizing ...
try { try {
this.snapshotReader.stop(); if ( this.snapshotReader != null ) this.snapshotReader.stop();
} finally { } finally {
try { try {
this.binlogReader.stop(); if (this.binlogReader != null ) this.binlogReader.stop();
} finally { } finally {
try { try {
// Flush and stop database history, close all JDBC connections ... // Flush and stop database history, close all JDBC connections ...
taskContext.shutdown(); if (this.taskContext != null ) taskContext.shutdown();
} catch (Throwable e) { } catch (Throwable e) {
logger.error("Unexpected error shutting down the database history and/or closing JDBC connections", e); logger.error("Unexpected error shutting down the database history and/or closing JDBC connections", e);
} finally { } finally {
context = null;
logger.info("Connector task successfully stopped"); logger.info("Connector task successfully stopped");
} }
} }
} }
} }
}
protected void transitionToReadBinlog() { protected void transitionToReadBinlog() {
logger.debug("Transitioning from snapshot reader to binlog reader"); logger.debug("Transitioning from snapshot reader to binlog reader");

View File

@ -417,6 +417,178 @@ default B withDefault(Field field, Class<?> value) {
*/ */
B apply(Consumer<B> function); B apply(Consumer<B> function);
/**
* Apply the function to this builder to change a potentially existing boolean field.
*
* @param key the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
* @throws NumberFormatException if the existing value is not a boolean
*/
default B changeBoolean(String key, Function<Boolean, Boolean> function) {
Function<String, String> strFunction = (existingStr) -> {
Boolean result = function.apply(existingStr != null ? Boolean.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(key, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing string field.
*
* @param key the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
*/
B changeString(String key, Function<String, String> function);
/**
* Apply the function to this builder to change a potentially existing double field.
*
* @param key the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
* @throws NumberFormatException if the existing value is not a double
*/
default B changeDouble(String key, Function<Double, Double> function) {
Function<String, String> strFunction = (existingStr) -> {
Double result = function.apply(existingStr != null ? Double.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(key, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing float field.
*
* @param key the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
* @throws NumberFormatException if the existing value is not a float
*/
default B changeFloat(String key, Function<Float, Float> function) {
Function<String, String> strFunction = (existingStr) -> {
Float result = function.apply(existingStr != null ? Float.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(key, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing long field.
*
* @param key the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
* @throws NumberFormatException if the existing value is not a long
*/
default B changeLong(String key, Function<Long, Long> function) {
Function<String, String> strFunction = (existingStr) -> {
Long result = function.apply(existingStr != null ? Long.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(key, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing integer field.
*
* @param key the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
* @throws NumberFormatException if the existing value is not an integer
*/
default B changeInteger(String key, Function<Integer, Integer> function) {
Function<String, String> strFunction = (existingStr) -> {
Integer result = function.apply(existingStr != null ? Integer.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(key, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing boolean field.
*
* @param field the predefined field for the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
*/
default B changeBoolean(Field field, Function<Boolean, Boolean> function) {
Function<String, String> strFunction = (existingStr) -> {
Boolean result = function.apply(existingStr != null ? Boolean.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(field, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing string field.
*
* @param field the predefined field for the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
*/
B changeString(Field field, Function<String, String> function);
/**
* Apply the function to this builder to change a potentially existing double field.
*
* @param field the predefined field for the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
*/
default B changeDouble(Field field, Function<Double, Double> function) {
Function<String, String> strFunction = (existingStr) -> {
Double result = function.apply(existingStr != null ? Double.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(field, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing float field.
*
* @param field the predefined field for the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
*/
default B changeFloat(Field field, Function<Float, Float> function) {
Function<String, String> strFunction = (existingStr) -> {
Float result = function.apply(existingStr != null ? Float.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(field, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing long field.
*
* @param field the predefined field for the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
*/
default B changeLong(Field field, Function<Long, Long> function) {
Function<String, String> strFunction = (existingStr) -> {
Long result = function.apply(existingStr != null ? Long.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(field, strFunction);
}
/**
* Apply the function to this builder to change a potentially existing integer field.
*
* @param field the predefined field for the key
* @param function the function that computes the new value given a possibly-existing value; may not be null
* @return this builder object so methods can be chained together; never null
*/
default B changeInteger(Field field, Function<Integer, Integer> function) {
Function<String, String> strFunction = (existingStr) -> {
Integer result = function.apply(existingStr != null ? Integer.valueOf(existingStr) : null);
return result != null ? result.toString() : null;
};
return changeString(field, strFunction);
}
/** /**
* Build and return the immutable configuration. * Build and return the immutable configuration.
* *
@ -458,6 +630,23 @@ public Builder apply(Consumer<Builder> function) {
return this; return this;
} }
@Override
public Builder changeString(String key, Function<String, String> function) {
return changeString(key,null,function);
}
@Override
public Builder changeString(Field field, Function<String, String> function) {
return changeString(field.name(),field.defaultValue(),function);
}
protected Builder changeString(String key, String defaultValue, Function<String, String> function) {
String existing = props.getProperty(key);
if ( existing == null ) existing = defaultValue;
String newValue = function.apply(existing);
return with(key,newValue);
}
@Override @Override
public Configuration build() { public Configuration build() {
return Configuration.from(props); return Configuration.from(props);
@ -1525,4 +1714,15 @@ default <T> void forEachMatchingFieldName(Pattern regex, int groupNumber, BiFunc
} }
}); });
} }
/**
* Call the supplied function for each of the fields.
*
* @param function the consumer that takes the field name and the string value extracted from the field; may
* not be null
*/
default <T> void forEach(BiConsumer<String, String> function) {
this.asMap().forEach(function);
}
} }

View File

@ -411,6 +411,11 @@ public static int isRequired(Configuration config, Field field, Consumer<String>
return 1; return 1;
} }
public static int isOptional(Configuration config, Field field, Consumer<String> problems) {
// optional fields are valid whether or not there is a value
return 0;
}
public static int isBoolean(Configuration config, Field field, Consumer<String> problems) { public static int isBoolean(Configuration config, Field field, Consumer<String> problems) {
String value = config.getString(field); String value = config.getString(field);
if (value == null || if (value == null ||

View File

@ -245,7 +245,7 @@ public Schema schema() {
* @param timestamp the timestamp for this message; may be null * @param timestamp the timestamp for this message; may be null
* @return the read message; never null * @return the read message; never null
*/ */
public Struct read(Struct record, Struct source, Long timestamp) { public Struct read(Object record, Struct source, Long timestamp) {
Struct struct = new Struct(schema); Struct struct = new Struct(schema);
struct.put(FieldName.OPERATION, Operation.READ.code()); struct.put(FieldName.OPERATION, Operation.READ.code());
struct.put(FieldName.AFTER, record); struct.put(FieldName.AFTER, record);
@ -262,7 +262,7 @@ public Struct read(Struct record, Struct source, Long timestamp) {
* @param timestamp the timestamp for this message; may be null * @param timestamp the timestamp for this message; may be null
* @return the create message; never null * @return the create message; never null
*/ */
public Struct create(Struct record, Struct source, Long timestamp) { public Struct create(Object record, Struct source, Long timestamp) {
Struct struct = new Struct(schema); Struct struct = new Struct(schema);
struct.put(FieldName.OPERATION, Operation.CREATE.code()); struct.put(FieldName.OPERATION, Operation.CREATE.code());
struct.put(FieldName.AFTER, record); struct.put(FieldName.AFTER, record);
@ -280,7 +280,7 @@ public Struct create(Struct record, Struct source, Long timestamp) {
* @param timestamp the timestamp for this message; may be null * @param timestamp the timestamp for this message; may be null
* @return the update message; never null * @return the update message; never null
*/ */
public Struct update(Struct before, Struct after, Struct source, Long timestamp) { public Struct update(Object before, Struct after, Struct source, Long timestamp) {
Struct struct = new Struct(schema); Struct struct = new Struct(schema);
struct.put(FieldName.OPERATION, Operation.UPDATE.code()); struct.put(FieldName.OPERATION, Operation.UPDATE.code());
if (before != null) struct.put(FieldName.BEFORE, before); if (before != null) struct.put(FieldName.BEFORE, before);
@ -298,7 +298,7 @@ public Struct update(Struct before, Struct after, Struct source, Long timestamp)
* @param timestamp the timestamp for this message; may be null * @param timestamp the timestamp for this message; may be null
* @return the delete message; never null * @return the delete message; never null
*/ */
public Struct delete(Struct before, Struct source, Long timestamp) { public Struct delete(Object before, Struct source, Long timestamp) {
Struct struct = new Struct(schema); Struct struct = new Struct(schema);
struct.put(FieldName.OPERATION, Operation.DELETE.code()); struct.put(FieldName.OPERATION, Operation.DELETE.code());
if (before != null) struct.put(FieldName.BEFORE, before); if (before != null) struct.put(FieldName.BEFORE, before);

View File

@ -76,6 +76,17 @@ public static <T> Predicate<T> excludes(String regexPatterns, Function<T, String
return includes(regexPatterns, conversion).negate(); return includes(regexPatterns, conversion).negate();
} }
/**
* Create a predicate function that allows only those values are allowed or not disallowed by the supplied predicates.
*
* @param allowed the predicate that defines the allowed values; may be null
* @param disallowed the predicate that defines the disallowed values; may be null
* @return the predicate function; never null
*/
public static <T> Predicate<T> filter( Predicate<T> allowed, Predicate<T> disallowed ) {
return allowed != null ? allowed : (disallowed != null ? disallowed : (id)->true);
}
public static <R> Predicate<R> not(Predicate<R> predicate) { public static <R> Predicate<R> not(Predicate<R> predicate) {
return predicate.negate(); return predicate.negate();
} }

View File

@ -5,9 +5,9 @@
*/ */
package io.debezium.jdbc; package io.debezium.jdbc;
import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Predicate; import java.util.function.Predicate;
import io.debezium.annotation.Immutable; import io.debezium.annotation.Immutable;
@ -146,19 +146,17 @@ default Builder withPort(int port) {
*/ */
public static Builder copy(Configuration config) { public static Builder copy(Configuration config) {
return new Builder() { return new Builder() {
private Properties props = config.asProperties(); private Configuration.Builder builder = Configuration.copy(config);
@Override @Override
public Builder with(String key, String value) { public Builder with(String key, String value) {
props.setProperty(key, value); builder.with(key, value);
return this; return this;
} }
@Override @Override
public Builder withDefault(String key, String value) { public Builder withDefault(String key, String value) {
if (!props.containsKey(key)) { builder.withDefault(key, value);
props.setProperty(key, value);
}
return this; return this;
} }
@ -168,14 +166,26 @@ public Builder apply(Consumer<Builder> function) {
return this; return this;
} }
@Override
public Builder changeString(Field field, Function<String, String> function) {
changeString(field,function);
return this;
}
@Override
public Builder changeString(String key, Function<String, String> function) {
changeString(key,function);
return this;
}
@Override @Override
public JdbcConfiguration build() { public JdbcConfiguration build() {
return JdbcConfiguration.adapt(Configuration.from(props)); return JdbcConfiguration.adapt(builder.build());
} }
@Override @Override
public String toString() { public String toString() {
return props.toString(); return builder.toString();
} }
}; };
} }
@ -187,19 +197,17 @@ public String toString() {
*/ */
public static Builder create() { public static Builder create() {
return new Builder() { return new Builder() {
private Properties props = new Properties(); private Configuration.Builder builder = Configuration.create();
@Override @Override
public Builder with(String key, String value) { public Builder with(String key, String value) {
props.setProperty(key, value); builder.with(key, value);
return this; return this;
} }
@Override @Override
public Builder withDefault(String key, String value) { public Builder withDefault(String key, String value) {
if (!props.containsKey(key)) { builder.withDefault(key, value);
props.setProperty(key, value);
}
return this; return this;
} }
@ -209,14 +217,26 @@ public Builder apply(Consumer<Builder> function) {
return this; return this;
} }
@Override
public Builder changeString(Field field, Function<String, String> function) {
changeString(field,function);
return this;
}
@Override
public Builder changeString(String key, Function<String, String> function) {
changeString(key,function);
return this;
}
@Override @Override
public JdbcConfiguration build() { public JdbcConfiguration build() {
return JdbcConfiguration.adapt(Configuration.from(props)); return JdbcConfiguration.adapt(builder.build());
} }
@Override @Override
public String toString() { public String toString() {
return props.toString(); return builder.toString();
} }
}; };
} }

View File

@ -0,0 +1,148 @@
/*
* 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.util;
import java.util.function.BooleanSupplier;
/**
* Encapsulates the logic of determining a delay when some criteria is met.
*
* @author Randall Hauch
*/
@FunctionalInterface
public interface DelayStrategy {
/**
* Attempt to sleep when the specified criteria is met.
*
* @param criteria {@code true} if this method should sleep, or {@code false} if there is no need to sleep
* @return {@code true} if this invocation caused the thread to sleep, or {@code false} if this method did not sleep
*/
default boolean sleepWhen(BooleanSupplier criteria) {
return sleepWhen(criteria.getAsBoolean());
}
/**
* Attempt to sleep when the specified criteria is met.
*
* @param criteria {@code true} if this method should sleep, or {@code false} if there is no need to sleep
* @return {@code true} if this invocation caused the thread to sleep, or {@code false} if this method did not sleep
*/
boolean sleepWhen(boolean criteria);
/**
* Create a delay strategy that never delays.
*
* @return the strategy; never null
*/
public static DelayStrategy none() {
return (criteria) -> false;
}
/**
* Create a delay strategy that applies an linearly-increasing delay as long as the criteria is met. As soon as
* the criteria is not met, the delay resets to zero.
*
* @param delayInMilliseconds the initial delay; must be positive
* @return the strategy; never null
*/
public static DelayStrategy constant(long delayInMilliseconds) {
return (criteria) -> {
if (!criteria) return false;
try {
Thread.sleep(delayInMilliseconds);
} catch (InterruptedException e) {
Thread.interrupted();
}
return true;
};
}
/**
* Create a delay strategy that applies an linearly-increasing delay as long as the criteria is met. As soon as
* the criteria is not met, the delay resets to zero.
*
* @param delayInMilliseconds the initial delay; must be positive
* @return the strategy; never null
*/
public static DelayStrategy linear(long delayInMilliseconds) {
if (delayInMilliseconds <= 0) throw new IllegalArgumentException("Initial delay must be positive");
return new DelayStrategy() {
private long misses = 0;
@Override
public boolean sleepWhen(boolean criteria) {
if (!criteria) {
// Don't sleep ...
misses = 0;
return false;
}
// Compute how long to delay ...
++misses;
try {
Thread.sleep(misses * delayInMilliseconds);
} catch (InterruptedException e) {
Thread.interrupted();
}
return true;
}
};
}
/**
* Create a delay strategy that applies an exponentially-increasing delay as long as the criteria is met. As soon as
* the criteria is not met, the delay resets to zero.
*
* @param initialDelayInMilliseconds the initial delay; must be positive
* @param maxDelayInMilliseconds the maximum delay; must be greater than the initial delay
* @return the strategy; never null
*/
public static DelayStrategy exponential(long initialDelayInMilliseconds, long maxDelayInMilliseconds) {
return exponential(initialDelayInMilliseconds, maxDelayInMilliseconds, 2.0);
}
/**
* Create a delay strategy that applies an exponentially-increasing delay as long as the criteria is met. As soon as
* the criteria is not met, the delay resets to zero.
*
* @param initialDelayInMilliseconds the initial delay; must be positive
* @param maxDelayInMilliseconds the maximum delay; must be greater than the initial delay
* @param backOffMultiplier the factor by which the delay increases each pass
* @return the strategy
*/
public static DelayStrategy exponential(long initialDelayInMilliseconds, long maxDelayInMilliseconds, double backOffMultiplier) {
if (backOffMultiplier <= 1.0) throw new IllegalArgumentException("Backup multiplier must be greater than 1");
if (initialDelayInMilliseconds <= 0) throw new IllegalArgumentException("Initial delay must be positive");
if (initialDelayInMilliseconds >= maxDelayInMilliseconds)
throw new IllegalArgumentException("Maximum delay must be greater than initial delay");
return new DelayStrategy() {
private long previousDelay = 0;
@Override
public boolean sleepWhen(boolean criteria) {
if (!criteria) {
// Don't sleep ...
previousDelay = 0;
return false;
}
// Compute how long to delay ...
if (previousDelay == 0) {
previousDelay = initialDelayInMilliseconds;
} else {
long nextDelay = (long) (previousDelay * backOffMultiplier);
previousDelay = Math.min(nextDelay, maxDelayInMilliseconds);
}
// We expect to sleep ...
try {
Thread.sleep(previousDelay);
} catch (InterruptedException e) {
Thread.interrupted();
}
return true;
}
};
}
}

View File

@ -5,6 +5,7 @@
*/ */
package io.debezium.util; package io.debezium.util;
import java.util.Collections;
import java.util.Map; import java.util.Map;
import org.slf4j.MDC; import org.slf4j.MDC;
@ -38,18 +39,20 @@ private LoggingContext() {
* A snapshot of an MDC context that can be {@link #restore()}. * A snapshot of an MDC context that can be {@link #restore()}.
*/ */
public static final class PreviousContext { public static final class PreviousContext {
private static final Map<String, String> EMPTY_CONTEXT = Collections.emptyMap();
private final Map<String, String> context; private final Map<String, String> context;
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
protected PreviousContext() { protected PreviousContext() {
context = MDC.getCopyOfContextMap(); Map<String, String> context = MDC.getCopyOfContextMap();
this.context = context != null ? context : EMPTY_CONTEXT;
} }
/** /**
* Restore this logging context. * Restore this logging context.
*/ */
public void restore() { public void restore() {
for ( Map.Entry<String, String> entry : context.entrySet() ) { MDC.setContextMap(context);
MDC.put(entry.getKey(), entry.getValue());
}
} }
} }

View File

@ -12,6 +12,7 @@
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
@ -159,6 +160,47 @@ public static String join(CharSequence delimiter, int[] values) {
return sb.toString(); return sb.toString();
} }
/**
* Returns a new String composed of the supplied values joined together with a copy of the specified {@code delimiter}.
*
* @param delimiter the delimiter that separates each element
* @param values the values to join together.
* @return a new {@code String} that is composed of the {@code elements} separated by the {@code delimiter}
*
* @throws NullPointerException If {@code delimiter} or {@code elements} is {@code null}
* @see java.lang.String#join
*/
public static <T> String join(CharSequence delimiter, Iterable<T> values) {
return join(delimiter,values,v->{
return v != null ? v.toString() : "null";
});
}
/**
* Returns a new String composed of the supplied values joined together with a copy of the specified {@code delimiter}.
*
* @param delimiter the delimiter that separates each element
* @param values the values to join together.
* @param conversion the function that converts the supplied values into strings
* @return a new {@code String} that is composed of the {@code elements} separated by the {@code delimiter}
*
* @throws NullPointerException If {@code delimiter} or {@code elements} is {@code null}
* @see java.lang.String#join
*/
public static <T> String join(CharSequence delimiter, Iterable<T> values, Function<T,String> conversion ) {
Objects.requireNonNull(delimiter);
Objects.requireNonNull(values);
Iterator<T> iter = values.iterator();
if ( !iter.hasNext() ) return "";
StringBuilder sb = new StringBuilder();
sb.append(iter.next());
while ( iter.hasNext() ) {
sb.append(delimiter);
sb.append(conversion.apply(iter.next()));
}
return sb.toString();
}
/** /**
* Trim away any leading or trailing whitespace characters. * Trim away any leading or trailing whitespace characters.
* <p> * <p>

View File

@ -203,12 +203,18 @@ public static void isValidTombstone(SourceRecord record, String pkField, int pk)
* Assert that the supplied {@link Struct} is {@link Struct#validate() valid} and its {@link Struct#schema() schema} * Assert that the supplied {@link Struct} is {@link Struct#validate() valid} and its {@link Struct#schema() schema}
* matches that of the supplied {@code schema}. * matches that of the supplied {@code schema}.
* *
* @param value the value with a schema; may not be null * @param schemaAndValue the value with a schema; may not be null
*/ */
public static void schemaMatchesStruct(SchemaAndValue value) { public static void schemaMatchesStruct(SchemaAndValue schemaAndValue) {
Object val = value.value(); Object value = schemaAndValue.value();
assertThat(val).isInstanceOf(Struct.class); if (value == null) {
fieldsInSchema((Struct) val, value.schema()); // The schema should also be null ...
assertThat(schemaAndValue.schema()).isNull();
} else {
// Both value and schema should exist and be valid ...
assertThat(value).isInstanceOf(Struct.class);
fieldsInSchema((Struct) value, schemaAndValue.schema());
}
} }
/** /**

View File

@ -14,7 +14,7 @@
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer; import java.util.function.Consumer;
import org.apache.kafka.connect.connector.ConnectorContext; import org.apache.kafka.connect.connector.ConnectorContext;
@ -252,7 +252,7 @@ public EmbeddedEngine build() {
private final ClassLoader classLoader; private final ClassLoader classLoader;
private final Consumer<SourceRecord> consumer; private final Consumer<SourceRecord> consumer;
private final CompletionCallback completionCallback; private final CompletionCallback completionCallback;
private final AtomicBoolean running = new AtomicBoolean(false); private final AtomicReference<Thread> runningThread = new AtomicReference<>();
private final VariableLatch latch = new VariableLatch(0); private final VariableLatch latch = new VariableLatch(0);
private final Converter keyConverter; private final Converter keyConverter;
private final Converter valueConverter; private final Converter valueConverter;
@ -289,7 +289,7 @@ private EmbeddedEngine(Configuration config, ClassLoader classLoader, Clock cloc
* @return {@code true} if running, or {@code false} otherwise * @return {@code true} if running, or {@code false} otherwise
*/ */
protected boolean isRunning() { protected boolean isRunning() {
return this.running.get(); return this.runningThread.get() != null;
} }
private void fail(String msg) { private void fail(String msg) {
@ -305,7 +305,8 @@ private void succeed(String msg) {
} }
/** /**
* Run this embedded connector and deliver database changes to the registered {@link Consumer}. * Run this embedded connector and deliver database changes to the registered {@link Consumer}. This method blocks until
* the connector is stopped.
* <p> * <p>
* First, the method checks to see if this instance is currently {@link #run() running}, and if so immediately returns. * First, the method checks to see if this instance is currently {@link #run() running}, and if so immediately returns.
* <p> * <p>
@ -316,12 +317,12 @@ private void succeed(String msg) {
* {@link #stop() stopped}. * {@link #stop() stopped}.
* <p> * <p>
* Note that there are two ways to stop a connector running on a thread: calling {@link #stop()} from another thread, or * Note that there are two ways to stop a connector running on a thread: calling {@link #stop()} from another thread, or
* interrupting the thread (e.g., via {@link ExecutorService#shutdownNow()}). However, interrupting the thread may result * interrupting the thread (e.g., via {@link ExecutorService#shutdownNow()}).
* in source records being repeated upon next startup, so {@link #stop()} should always be used when possible.
*/ */
@Override @Override
public void run() { public void run() {
if (running.compareAndSet(false, true)) { if (runningThread.compareAndSet(null, Thread.currentThread())) {
final String engineName = config.getString(ENGINE_NAME); final String engineName = config.getString(ENGINE_NAME);
final String connectorClassName = config.getString(CONNECTOR_CLASS); final String connectorClassName = config.getString(CONNECTOR_CLASS);
// Only one thread can be in this part of the method at a time ... // Only one thread can be in this part of the method at a time ...
@ -403,10 +404,12 @@ public void run() {
recordsSinceLastCommit = 0; recordsSinceLastCommit = 0;
timeSinceLastCommitMillis = clock.currentTimeInMillis(); timeSinceLastCommitMillis = clock.currentTimeInMillis();
while (running.get()) { while (runningThread.get() != null) {
try { try {
logger.debug("Embedded engine is polling task for records");
List<SourceRecord> changeRecords = task.poll(); // blocks until there are values ... List<SourceRecord> changeRecords = task.poll(); // blocks until there are values ...
if (changeRecords != null && !changeRecords.isEmpty()) { if (changeRecords != null && !changeRecords.isEmpty()) {
logger.debug("Received {} records from the task", changeRecords.size());
// First forward the records to the connector's consumer ... // First forward the records to the connector's consumer ...
for (SourceRecord record : changeRecords) { for (SourceRecord record : changeRecords) {
@ -425,6 +428,8 @@ public void run() {
// Flush the offsets to storage if necessary ... // Flush the offsets to storage if necessary ...
recordsSinceLastCommit += changeRecords.size(); recordsSinceLastCommit += changeRecords.size();
maybeFlush(offsetWriter, offsetCommitPolicy, commitTimeoutMs); maybeFlush(offsetWriter, offsetCommitPolicy, commitTimeoutMs);
} else {
logger.debug("Received no records from the task");
} }
} catch (InterruptedException e) { } catch (InterruptedException e) {
// This thread was interrupted, which signals that the thread should stop work. // This thread was interrupted, which signals that the thread should stop work.
@ -436,7 +441,15 @@ public void run() {
break; break;
} }
} }
try {
// First stop the task ...
logger.debug("Stopping the task and engine");
task.stop();
} finally {
// Always commit offsets that were captured from the source records we actually processed ...
commitOffsets(offsetWriter, commitTimeoutMs);
succeed("Connector '" + connectorClassName + "' completed normally."); succeed("Connector '" + connectorClassName + "' completed normally.");
}
} catch (Throwable t) { } catch (Throwable t) {
fail("Error while trying to run connector class '" + connectorClassName + "'", t); fail("Error while trying to run connector class '" + connectorClassName + "'", t);
return; return;
@ -450,7 +463,7 @@ public void run() {
} }
} finally { } finally {
latch.countDown(); latch.countDown();
running.set(false); runningThread.set(null);
} }
} }
} }
@ -466,7 +479,17 @@ protected void maybeFlush(OffsetStorageWriter offsetWriter, OffsetCommitPolicy p
// Determine if we need to commit to offset storage ... // Determine if we need to commit to offset storage ...
if (policy.performCommit(recordsSinceLastCommit, timeSinceLastCommitMillis, if (policy.performCommit(recordsSinceLastCommit, timeSinceLastCommitMillis,
TimeUnit.MILLISECONDS)) { TimeUnit.MILLISECONDS)) {
commitOffsets(offsetWriter, commitTimeoutMs);
}
}
/**
* Flush offsets to storage.
*
* @param offsetWriter the offset storage writer; may not be null
* @param commitTimeoutMs the timeout to wait for commit results
*/
protected void commitOffsets(OffsetStorageWriter offsetWriter, long commitTimeoutMs) {
long started = clock.currentTimeInMillis(); long started = clock.currentTimeInMillis();
long timeout = started + commitTimeoutMs; long timeout = started + commitTimeoutMs;
if (!offsetWriter.beginFlush()) return; if (!offsetWriter.beginFlush()) return;
@ -489,7 +512,6 @@ protected void maybeFlush(OffsetStorageWriter offsetWriter, OffsetCommitPolicy p
offsetWriter.cancelFlush(); offsetWriter.cancelFlush();
} }
} }
}
protected void completedFlush(Throwable error, Void result) { protected void completedFlush(Throwable error, Void result) {
if (error != null) { if (error != null) {
@ -508,7 +530,15 @@ protected void completedFlush(Throwable error, Void result) {
* @see #await(long, TimeUnit) * @see #await(long, TimeUnit)
*/ */
public boolean stop() { public boolean stop() {
return running.getAndSet(false); logger.debug("Stopping the embedded engine");
// Signal that the run() method should stop ...
Thread thread = this.runningThread.getAndSet(null);
if (thread != null) {
// Interrupt the thread in case it is blocked while polling the task for records ...
thread.interrupt();
return true;
}
return false;
} }
/** /**

View File

@ -42,6 +42,7 @@
import io.debezium.embedded.EmbeddedEngine.CompletionCallback; import io.debezium.embedded.EmbeddedEngine.CompletionCallback;
import io.debezium.function.BooleanConsumer; import io.debezium.function.BooleanConsumer;
import io.debezium.relational.history.HistoryRecord; import io.debezium.relational.history.HistoryRecord;
import io.debezium.util.LoggingContext;
import io.debezium.util.Testing; import io.debezium.util.Testing;
/** /**
@ -72,6 +73,7 @@ public abstract class AbstractConnectorTest implements Testing {
@Before @Before
public final void initializeConnectorTestFramework() { public final void initializeConnectorTestFramework() {
LoggingContext.forConnector(getClass().getSimpleName(),"","test");
keyJsonConverter = new JsonConverter(); keyJsonConverter = new JsonConverter();
valueJsonConverter = new JsonConverter(); valueJsonConverter = new JsonConverter();
keyJsonDeserializer = new JsonDeserializer(); keyJsonDeserializer = new JsonDeserializer();
@ -86,6 +88,7 @@ public final void initializeConnectorTestFramework() {
resetBeforeEachTest(); resetBeforeEachTest();
consumedLines = new ArrayBlockingQueue<>(getMaximumEnqueuedRecordCount()); consumedLines = new ArrayBlockingQueue<>(getMaximumEnqueuedRecordCount());
Testing.Files.delete(OFFSET_STORE_PATH); Testing.Files.delete(OFFSET_STORE_PATH);
OFFSET_STORE_PATH.getParent().toFile().mkdirs();
} }
/** /**
@ -103,11 +106,12 @@ public final void stopConnector() {
*/ */
public void stopConnector(BooleanConsumer callback) { public void stopConnector(BooleanConsumer callback) {
try { try {
logger.info("Stopping the connector");
// Try to stop the connector ... // Try to stop the connector ...
if (engine != null && engine.isRunning()) { if (engine != null && engine.isRunning()) {
engine.stop(); engine.stop();
try { try {
engine.await(5, TimeUnit.SECONDS); engine.await(8, TimeUnit.SECONDS);
} catch (InterruptedException e) { } catch (InterruptedException e) {
Thread.interrupted(); Thread.interrupted();
} }
@ -192,6 +196,7 @@ protected void start(Class<? extends SourceConnector> connectorClass, Configurat
} finally { } finally {
latch.countDown(); latch.countDown();
} }
Testing.debug("Stopped connector");
}; };
// Create the connector ... // Create the connector ...
@ -211,7 +216,10 @@ protected void start(Class<? extends SourceConnector> connectorClass, Configurat
// Submit the connector for asynchronous execution ... // Submit the connector for asynchronous execution ...
assertThat(executor).isNull(); assertThat(executor).isNull();
executor = Executors.newFixedThreadPool(1); executor = Executors.newFixedThreadPool(1);
executor.execute(engine); executor.execute(()->{
LoggingContext.forConnector(getClass().getSimpleName(),"","engine");
engine.run();
});
} }
/** /**
@ -280,10 +288,25 @@ protected int consumeRecords(int numberOfRecords, Consumer<SourceRecord> recordC
return recordsConsumed; return recordsConsumed;
} }
/**
* Try to consume and capture exactly the specified number of records from the connector.
*
* @param numRecords the number of records that should be consumed
* @return the collector into which the records were captured; never null
* @throws InterruptedException if the thread was interrupted while waiting for a record to be returned
*/
protected SourceRecords consumeRecordsByTopic(int numRecords) throws InterruptedException { protected SourceRecords consumeRecordsByTopic(int numRecords) throws InterruptedException {
return consumeRecordsByTopic(numRecords, new SourceRecords()); return consumeRecordsByTopic(numRecords, new SourceRecords());
} }
/**
* Try to consume and capture exactly the specified number of records from the connector.
*
* @param numRecords the number of records that should be consumed
* @param records the collector into which all consumed messages should be placed
* @return the actual number of records that were consumed
* @throws InterruptedException if the thread was interrupted while waiting for a record to be returned
*/
protected SourceRecords consumeRecordsByTopic(int numRecords, SourceRecords records) throws InterruptedException { protected SourceRecords consumeRecordsByTopic(int numRecords, SourceRecords records) throws InterruptedException {
consumeRecords(numRecords, records::add); consumeRecords(numRecords, records::add);
return records; return records;
@ -312,18 +335,36 @@ protected String getAffectedDatabase(SourceRecord record) {
return null; return null;
} }
/**
* Get the DDL events for the named database.
* @param dbName the name of the database; may not be null
* @return the DDL-related events; never null but possibly empty
*/
public List<SourceRecord> ddlRecordsForDatabase(String dbName) { public List<SourceRecord> ddlRecordsForDatabase(String dbName) {
return ddlRecordsByDbName.get(dbName); return ddlRecordsByDbName.get(dbName);
} }
/**
* Get the names of the databases that were affected by the DDL statements.
* @return the set of database names; never null but possibly empty
*/
public Set<String> databaseNames() { public Set<String> databaseNames() {
return ddlRecordsByDbName.keySet(); return ddlRecordsByDbName.keySet();
} }
/**
* Get the records on the given topic.
* @param topicName the name of the topic.
* @return the records for the topic; possibly null if there were no records produced on the topic
*/
public List<SourceRecord> recordsForTopic(String topicName) { public List<SourceRecord> recordsForTopic(String topicName) {
return recordsByTopic.get(topicName); return recordsByTopic.get(topicName);
} }
/**
* Get the set of topics for which records were received.
* @return the names of the topics; never null
*/
public Set<String> topics() { public Set<String> topics() {
return recordsByTopic.keySet(); return recordsByTopic.keySet();
} }

12
pom.xml
View File

@ -65,6 +65,8 @@
<version.mysql.server>5.7</version.mysql.server> <version.mysql.server>5.7</version.mysql.server>
<version.mysql.driver>5.1.39</version.mysql.driver> <version.mysql.driver>5.1.39</version.mysql.driver>
<version.mysql.binlog>0.3.1</version.mysql.binlog> <version.mysql.binlog>0.3.1</version.mysql.binlog>
<version.mongo.server>3.2.6</version.mongo.server>
<version.mongo.driver>3.2.2</version.mongo.driver>
<!-- Testing --> <!-- Testing -->
<version.junit>4.12</version.junit> <version.junit>4.12</version.junit>
@ -77,7 +79,7 @@
<version.war.plugin>2.5</version.war.plugin> <version.war.plugin>2.5</version.war.plugin>
<version.codehaus.helper.plugin>1.8</version.codehaus.helper.plugin> <version.codehaus.helper.plugin>1.8</version.codehaus.helper.plugin>
<version.google.formatter.plugin>0.3.1</version.google.formatter.plugin> <version.google.formatter.plugin>0.3.1</version.google.formatter.plugin>
<version.docker.maven.plugin>0.14.1</version.docker.maven.plugin> <version.docker.maven.plugin>0.15.0</version.docker.maven.plugin>
<version.staging.plugin>1.6.3</version.staging.plugin> <version.staging.plugin>1.6.3</version.staging.plugin>
<!-- Dockerfiles --> <!-- Dockerfiles -->
@ -96,6 +98,7 @@
<module>debezium-core</module> <module>debezium-core</module>
<module>debezium-embedded</module> <module>debezium-embedded</module>
<module>debezium-connector-mysql</module> <module>debezium-connector-mysql</module>
<module>debezium-connector-mongodb</module>
</modules> </modules>
<distributionManagement> <distributionManagement>
@ -209,6 +212,13 @@
<version>${version.mysql.binlog}</version> <version>${version.mysql.binlog}</version>
</dependency> </dependency>
<!-- MongoDB Java driver -->
<dependency>
<groupId>org.mongodb</groupId>
<artifactId>mongodb-driver</artifactId>
<version>${version.mongo.driver}</version>
</dependency>
<!-- Logging --> <!-- Logging -->
<dependency> <dependency>
<groupId>org.slf4j</groupId> <groupId>org.slf4j</groupId>