Commit Graph

7458 Commits

Author SHA1 Message Date
Randall Hauch
447acb797d DBZ-62 Upgraded to Kafka and Kafka Connect 0.10.0.0
Upgraded from Kafka 0.9.0.1 to Kafka 0.10.0. The only required change was to override the `Connector.config()` method, which returns `null` or a `ConfigDef` instance that contains detailed metadata for each of the configuration fields, including supporting recommended values and marking fields as not visible (e.g., if they don't make sense given other configuration field values). This can be used by user interfaces to data-drive the configuration of a connector. Also, the default validation logic of the Connector implementations uses a `Validator` that is pretty restrictive in its functionality.

Debezium already had a fairly decent and simple `Configuration` framework. After several attempts to try and merge these concepts, reconciling the two validation mechanisms was very complicated and involved a lot of changes. It was easier to simply continue Debezium-specific validation and to override the `Connector.validate(...)` method to use Debezium's `Configuration`-based validation. Connector-based validation logic includes determining recommended values, so Debezium's `Field` class (used to define each configuration property) was enhanced with a new `Recommender` class that is similar to Kafka's.

Additional integration tests were added to verify that the `ConfigDef` result is acceptable and that the new connector validation logic works as expected, including getting recommended values for some fields (e.g., database names, table/collection names) from MySQL and MongoDB by connecting and dynamically reading the values. This was done in a way that remains backward compatible with the regular expression formats of these fields, but in a user interface that uses the `ConfigDef` mechanism the user can simply select the databases and table/collection identifiers.
2016-07-25 14:21:31 -05:00
Randall Hauch
4f749e84e2 Merge pull request #74 from rhauch/dbz-85
DBZ-85 Added test case and made small correction to temporal values
2016-07-21 09:03:49 -05:00
Randall Hauch
30777e3345 DBZ-85 Added test case and made correction to temporal values
Added an integration test case to diagnose the loss of the fractional seconds from MySQL temporal values. The problem appears to be a bug in the MySQL Binary Log Connector library that we used, and this bug was reported as https://github.com/shyiko/mysql-binlog-connector-java/issues/103. That was fixed in version 0.3.2 of the library, which Stanley was kind enough to release for us.

During testing, though, several issues were discovered in how temporal values are handled and converted from the MySQL events, through the MySQL Binary Log client library, and through the Debezium MySQL connector to conform with Kafka Connect's various temporal logical schema types. Most of the issues involved converting most of the temporal values from local time zone (which is how they are created by the MySQL Binary Log client) into UTC (which is how Kafka Connect expects them). Really, java.util.Date doesn't have time zone information and instead tracks the number of milliseconds past epoch, but the conversion of normal timestamp information to the milliseconds past epoch in UTC depends on the time zone in which that conversion happens.
2016-07-20 17:07:56 -05:00
Randall Hauch
4a84a1d8d9 Merge pull request #73 from rhauch/dbz-87
DBZ-87 Changed mapping of MySQL TINYINT and SMALLINT columns from INT32 to INT16
2016-07-19 11:25:01 -05:00
Randall Hauch
a5f4d0bf31 DBZ-87 Changed mapping of MySQL TINYINT and SMALLINT columns from INT32 to INT16
The MySQL connector now maps TINYINT and SMALLINT columns to INT16 (rather than INT32) because INT16 is smaller and yet still large enough for all TINYINT and SMALLINT values. Note that the range of TINYINT values is either -128 to 127 for signed or 0 to 255 for unsigned, and thus INT8 is not an acceptable choice since it can only handle values in the range 0 to 255. Additionally, the JDBC Specification also suggests the proper Java type for SQL-99's TINYINT is short, which maps to Kafka Connect's INT16.

This change will be backward compatible, although the generated Kafka Connect schema will be different than in previous versions. This shouldn't cause a problem, since clients should expect to handle schema changes, and this schema change does comply with Avro schema evolution rules.
2016-07-19 11:11:05 -05:00
Randall Hauch
fc36fe1d54 Merge pull request #72 from rhauch/dbz-84
DBZ-84 Tried to replicate error with MySQL TINYINT columns
2016-07-19 11:05:55 -05:00
Randall Hauch
04eef2da5c DBZ-84 Tried to replicate error with MySQL TINYINT columns
Tried unsuccessfully to replicate the problem reported in DBZ-84 with a new regression integration test.
2016-07-19 10:58:28 -05:00
Randall Hauch
92f754a206 Merge pull request #71 from rhauch/dbz-86
DBZ-86 Generated Kafka Schema names will now also be valid Avro fullnames
2016-07-18 13:15:40 -05:00
Randall Hauch
ed1b494fdf DBZ-86 Cleaned up logging and error message in MySQL connector 2016-07-15 16:37:47 -05:00
Randall Hauch
bf6f9a8243 DBZ-86 Cleaned up unnecessary printing in MongoDB connector 2016-07-15 16:37:12 -05:00
Randall Hauch
a88bcb9ae7 DBZ-86 Generated Kafka Schema names will now also be valid Avro fullnames 2016-07-15 16:29:52 -05:00
Randall Hauch
85c9d4e5fe Merge pull request #35 from rhauch/dbz-2
DBZ-2 Created Maven module with a MongoDB connector
2016-07-14 14:11:35 -05:00
Randall Hauch
12e7cfb8d3 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.
2016-07-14 13:02:36 -05:00
Randall Hauch
d59303a6d8 Removed reference to 'real time' 2016-06-30 09:45:12 -05:00
Randall Hauch
78e32e5273 Merge pull request #69 from rhauch/dbz-83-master
DBZ-83 Correctly handle MySQL REFERENCES clause
2016-06-27 16:16:18 -05:00
Randall Hauch
cc68a1beb7 DBZ-83 Correctly handle MySQL REFERENCES clause 2016-06-27 13:02:57 -05:00
Randall Hauch
e71a74f4fa Merge pull request #67 from rhauch/dbz-82
DBZ-82 Changed snapshot query to support pre-5.6.5 versions of MySQL
2016-06-27 09:35:07 -05:00
Randall Hauch
f0d67143bd DBZ-82 Changed snapshot query to support pre-5.6.5 versions of MySQL 2016-06-27 09:23:12 -05:00
Randall Hauch
1c7aabf14f Changed MySQL file comment format to use standard prefix 2016-06-22 18:19:50 -05:00
Randall Hauch
49150689af Updated changelog for the 0.2.2 release 2016-06-22 16:15:06 -05:00
Randall Hauch
83c44ba046 Merge pull request #65 from rhauch/dbz-79
DBZ-79 Changed public methods in GtidSet to reflect the MySQL Binary Log Connector's class
2016-06-16 11:06:16 -05:00
Randall Hauch
a589d9ea84 DBZ-79 Changed public methods in GtidSet to reflect the MySQL Binary Log Connector's class
Removed several of the `GtidSet` convenience methods that are not in the [improved](https://github.com/shyiko/mysql-binlog-connector-java/pull/100) `com.github.shyiko.mysql.binlog.GtidSet` class. Getting these out of our API will make it easier to reuse the improved `com.github.shyiko.mysql.binlog.GtidSet` class.
2016-06-16 10:04:02 -05:00
Randall Hauch
88ceab3c48 Merge pull request #63 from rhauch/dbz-73
DBZ-73, DBZ-77 Added offset tests and fix for incomplete snapshot bug
2016-06-15 12:23:16 -05:00
Randall Hauch
d9cca5d254 DBZ-77 Corrected completion of offset snapshot mode
The snapshot mode within the offsets now are marked as complete with the last source record produced during the snapshot. This is the only sure way to update the offset.

Note that the `source` field shows the snapshot is in effect for _all_ records produced during the snapshot, including the very last one. This distinction w/r/t the offset was made possible due to recent changes for DBZ-73.

Previously, when the snapshot reader completed all generation of records, it then attempted to record an empty DDL statement. However, since this statement had no net effect on the schemas, no source record was produced and thus the offset's snapshot mode was never changed. Consequently, if the connector were stopped immediately after the snapshot completed but before other events could be read or produced, upon restart the connector would perform another snapshot.
2016-06-15 12:01:16 -05:00
Randall Hauch
ed27faa5f6 DBZ-73 Added unit tests to verify behavior of SourceInfo 2016-06-15 11:51:42 -05:00
Randall Hauch
84427e3648 Merge pull request #61 from rhauch/dbz-73
DBZ-73, DBZ-76 Corrected how binlog coordinates are recorded and put into change events
2016-06-14 17:57:12 -05:00
Randall Hauch
49322dc9c1 DBZ-73, DBZ-76 Corrected how binlog coordinates are recorded and put into change events
Fixes two issues with how the binlog coordinates are handled.

The first, DBZ-73, fixes how the offsets are recording the _next_ binlog coordinates within the offsets, which is fine for single-row events but which can result in dropped events should Kafka Connect flush the offset of some but not all of the rows before the Kafka Connect crashes. Upon restart, the offset contains the binlog coordinates for the _next_ event, so any of the last rows from the previous events will be lost.

With this fix, the offset used with all but the last row (in the binlog event) has the binlog coordinates of the current event, with the event row number set to be the next row that needs to be processed. The offset for the last row will have the binlog coordinates of the next event.

The second issue, DBZ-76, is somewhat related: the `source` field of the change events has the binlog coordinates of the _next_ issue. The fix involves putting the binlog coordinates for the _current_ event into the `source` field.

Both of these issues are related and influenced a fix that could address both problems. Essentially, the `SourceInfo` is now recording the previous and next position, and the next and previous row numbers. The offset is created with parameters that specify the row number and the total number of rows, so this method correctly adjusts the binlog coordinates of the offset. The `struct` field produces the value for the `source` field, and it is always using the previous position and previous row number that reflect the change event in which it is used.
2016-06-14 17:43:58 -05:00
Randall Hauch
f02c1458ce Updated RELEASE.md with additional validation steps 2016-06-10 14:03:47 -05:00
Randall Hauch
f565932dd2 Added commit log for 0.2.1 and placeholder for 0.3 changes 2016-06-10 10:06:22 -05:00
Randall Hauch
270150bcad DBZ-72 Corrected the naming of the Schemas for the keys and values 2016-06-09 21:30:29 -05:00
Randall Hauch
0f3ed9f50f DBZ-71 Corrected MySQL connector plugin archives and upgraded MySQL JDBC driver from 5.1.38 to 5.1.39 (the latest) 2016-06-09 21:15:34 -05:00
Randall Hauch
d2e930847c Documented the release process 2016-06-08 14:38:39 -05:00
Randall Hauch
6749518f66 [maven-release-plugin] prepare for next development iteration 2016-06-08 13:00:50 -05:00
Randall Hauch
d5bbb116ed [maven-release-plugin] prepare release v0.2.0 2016-06-08 13:00:50 -05:00
Randall Hauch
3b7db43bf9 Updated change log for 0.2 2016-06-08 12:56:50 -05:00
Randall Hauch
cc11f23cd0 Merge pull request #58 from rhauch/dbz-37
DBZ-37 Changed build to support running integration tests against multiple MySQL configurations
2016-06-08 12:00:42 -05:00
Randall Hauch
ff49ba1742 DBZ-37 Renamed MySQL Docker images used in integration tests 2016-06-08 11:45:35 -05:00
Randall Hauch
d63a2e17a0 DBZ-37 Added documentation of various profiles to the MySQL module's README 2016-06-08 11:19:03 -05:00
Randall Hauch
825dee3eab Changed Travis build to use assembly profile 2016-06-08 11:03:43 -05:00
Randall Hauch
3c7882ee9d DBZ-37 Run integration tests against MySQL and MySQL w/ GTIDs
Changed the build so that the `assembly` profile runs the MySQL integration tests three times, once against each of the three MySQL configurations:

# MySQL server w/o GTIDs
# MySQL server w/ GTIDs
# The Docker team's MySQL server image w/o GTIDs

The normal profiles are still available:

# The default profile runs the integration tests once against MySQL server w/o GTIDs
# `gtid-mysql` runs the integration tests against MySQL server w/ GTIDs
# `alt-mysql` runs the integration tests against the Docker team's MySQL server image w/o GTIDs
# `skip-integration-tests` (or `-DskipITs`) skips the integration tests altogether
2016-06-08 11:03:03 -05:00
Randall Hauch
b80ed3d5ed Merge pull request #57 from rhauch/pom-fix
Removed duplicate versions in POMs
2016-06-08 10:05:32 -05:00
Randall Hauch
cf26a5c4e0 Removed duplicate versions in POMs 2016-06-08 09:46:05 -05:00
Randall Hauch
0a9133d276 Merge pull request #56 from rhauch/dbz-61
DBZ-61 Improved MySQL connector's handling of binary values
2016-06-07 20:32:28 -05:00
Randall Hauch
a143871abd DBZ-61 Improved MySQL connector's handling of binary values
Binary values read from the MySQL binlog may include strings, in which case they need to be converted to binary values.

Interestingly, work on this uncovered [KAFKA-3803](https://issues.apache.org/jira/browse/KAFKA-3803) whereby Kafka Connect's `Struct.equals` method does not properly handle comparing `byte[]` values. Upon researching the problem and potentially supplying a patch, it was discovered that the Kafka Connect codebase and the Avro converter all use `ByteBuffer` objects rather than `byte[]`. Consequently, the Debezium code that converts JDBC values to Kafka Connect values was changed to return `ByteBuffer` objects rather than `byte[]` objects.

Unfortunately, the JSON converter rehydrates objects with just `byte[]`, so that still means that Debezium's `VerifyRecords` logic cannot rely upon `Struct.equals` for comparison, and instead needs custom logic.
2016-06-07 17:53:07 -05:00
Randall Hauch
4f02efc788 Merge pull request #55 from rhauch/dbz-37
DBZ-37 Added support for MySQL GTIDs
2016-06-07 12:47:43 -05:00
Randall Hauch
f48d48e114 DBZ-37 Added integration test with MySQL GTIDs
Added a Maven profile to the MySQL connector component with a Docker image that runs MySQL with GTIDs enabled. The same integration tests can be run with it using `-Pgtid-mysql` or `-Dgtid-mysql` in the Maven build.

When the MySQL connector starts up, it now queries the MySQL server to detect whether GTIDs are enabled, and if they are it will also verify that any GTID sets from the most recently recorded offset are still available in the MySQL server (similarly to how it was already doing this for binlog filenames). If the server does not have the correct coordinates/GTIDs, the connector fails with a useful error message.

This commit also tests and adjusts the `GtidSet` class to better deal with comparisons of GTID sets for proper ordering.

It also changes the connector to output MySQL's timestamp for each event using _second_ precision rather than artificially in _millisecond_ precision. To clarify the different, this change renames the field in the event's `source` structure that records the MySQL timestamp from `ts` to `ts_sec`. Similarly, the envelope's field that records the time that the connector processed each record was renamed from `ts` to `ts_ms`.

All unit and integration tests pass with the default profile and with the new GTID-enabled profile.
2016-06-07 12:01:51 -05:00
Randall Hauch
a276d983f5 DBZ-37 Changed several constants related to MySQL offsets. This does not affect the offsets themselves. 2016-06-04 16:32:26 -05:00
Randall Hauch
e91aac5b18 DBZ-37 DatabaseHistory can now use custom logic to compare offsets
DatabaseHistory stores the DDL changes with the offset describing the position in the source where those DDL statements were found. When a connector restarts at a specific offset (supplied by Kafka Connect), connectors such as the MySQL connector reconstruct the database schemas by having DatabaseHistory load the history starting from the beginning and stopping at (or just before) the connector's starting offset. This change allows connectors to supply a custom comparison function.

To support GTIDs, the MySQL connector needed to store additional information in the offsets. This means the logic needed to compare offsets with and without GTIDs is non-trivial and unique to the MySQL connector. This commit adds a custom comparison function for offsets.

Per [MySQL documentation](https://dev.mysql.com/doc/refman/5.7/en/replication-gtids-failover.html), slaves are always expected to start with the same set of GTIDs as the master, so no matter which the MySQL connector follows it should always have the complete set of GTIDs seen by that server. Therefore:

* Two offsets with GTID sets can be compared using only the GTID sets.
* Any offset with a GTID set is always assumed to be newer than an offset without, since it is assumed once GTIDs are enabled they will remain enabled. (Otherwise, the connector likely needs to be restarted with a snapshot and tied to a specific master or slave with no failover.)
* Two offsets without GTIDs are compared using the binlog coordinates (filename, position, and row number).
* An offsets that is identical to another except for being in snapshot mode is considered earlier than without the snapshot. This is because snapshot mode begins by recording the position of the snapshot, and once complete the offset is recorded without the snapshot flag.
2016-06-04 16:20:26 -05:00
Randall Hauch
8fd89dacbf DBZ-37 Corrected JavaDoc 2016-06-02 19:06:13 -05:00
Randall Hauch
655aac7d4f DBZ-37 Added support for MySQL GTIDs
The BinlogClient library our MySQL connector uses already has support for GTIDs. This change makes use of that and adds the GTIDs from the server to the offsets created by the connector and used upon restarts.
2016-06-02 18:30:26 -05:00