Adds support for MySQL 5.7's `JSON` type, which is capable of holding JSON objects, JSON arrays, and scalar values. The Debezium MySQL connector represents `JSON` values as string with a `io.debezium.data.Json` semantic type (which is basically a string schema that has a special name to denote the semantics), and the _contents_ of that string will be the JSON representation of the object, array, or scalar value.
When a connector is originally connected to a MySQL server, it will record the GTID set that identifies the position in the binlog. When all of the interesting transactions originate on a different server (i.e., the server we're listening to is a replica), the server we're listening to will still include some transactions in the binlog (e.g., for the information schema, performance, or other internal databases), and so the GTID set will include a GTID range for our server. If we stop the connector and want to point it to a different MySQL server, asking MySQL to position the binlog using the complete GTID set (including the GTID range for our old replica) will cause an error, since the new server does not have any GTID ranges from the old replica. Therefore, the connector needs to be able to exclude some GTID ranges that originated on the original replica, using the `server_uuid` property of the replica server.
This change adds two configuration properties: `gtid.source.includes` and `gtid.source.excludes`. Both are optional, but at most only one of these can be used. These properties contain a comma-separated list of GTID sources (i.e., the `server_uuid` value for the server where the transaction originated) or regular expressions matching GTID sources, and upon connector startup the connector uses the list to filter the previously-recorded GTID set against the available GTID set in the current MySQL server. By including specific GTID sources, an administrator can control the subset of GTID ranges that govern the binlog position.
These properties will not be useful in some topologies, especially when the MySQL server from which the binlog is being read is the originating server for some of the transactions. However, these properties may be very useful in any topology where the connector is _only_ reading from replicas, so that the connector can be switched to another replica at any time. In some cases it may be easier to exclude all of the replicas' `server_uuid` values, while in other cases it may be easier to include all of the `server_uuid` values where transactions can originate.
Updated MysqlParser to return list of String for allowed enum and set values
And also added code fix to get a enum value at a particular index and for set option too.
Used debezium string utility to join list of string into deliminator seperated String.
Updating old test cases as per required to handle list of strings.
When the MySQL connector is reading the binlog, it outputs INFO log messages reporting status at an exponentially-increasing rate, starting at every 5 seconds and doubling until a max period of 1 hour. This output is useful when the connector starts to know that it is working, but thereafter the usefulness decreases. Once an hour is probably acceptable output.
This is not intended to replace the capturing of metrics, but is merely an aid to easily tell via the logs whether the connector continues to work.
Also improved the log message when the binlog reader stops to capture the total number of events recorded by Kafka Connect and the last recorded offset.
Added logic to verify that MySQL's row-level binlog is enabled, and whether it is likely that when snapshots are not performed that the binlog is likely to have been purged. Some situations will result in an error, while others are logged as warnings.
Corrected how the MySQL connector is treating columns of type `BIT(n)`, where _n_ is the number of bits in the value. When `n=1`, the resulting values are booleans; when `n>1`, the resulting values are little endian `byte[]` that have the minimum number of bytes to hold the `n` bits.
The `KafkaDatabaseHistory` was always creating a new producer whenever its `start()` method was called, even if it were called more than once. And, the `MySqlSchema` was calling `start()` twice, resulting in multiple producers being created and registered with JMX. Both issues were fixed.
Also, UUIDs were being used as the name of the JMX MBean for the producer, unless the `database.history.consumer.client.id` and `database.history.producer.client.id` properties were being explicitly set. Now, the MySQL connector will by default set the `client.id` property on both the database history's Kafka consumer and producer to `{connectorName}-dbhistory`. Of course, the `database.history.consumer.client.id` and `database.history.producer.client.id` properties can still be set to define the name of the producer and consumer.
MySQL supports "zero-value" dates and timestamps, but these cannot be represented as valid dates or timestamps using the Java types. For example, the zero-value `0000-00-00` for a date has what Java considers to be an invalid month and day-of-the-month.
This commit changes how the MySQL connector handles these values to not throw exceptions. When columns allow nulls, such values will be treated as nulls; when columns do not allow null values, these values will be converted to a "zero-value" for the corresponding Java representation (e.g., the epoch day or timestamp). A new test case verifies the behaviors.
The MySQL connector now outputs an INFO log message whenever its task's `poll()` method returns a non-empty list of `SourceRecord` objects, where the message includes the number of records and the offset of the last record.
The MySQL connector was improperly comparing the GTID set required by the connector to the GTID set of the MySQL instance. In particular, when the GTID set of the MySQL server contained a newline character, the comparison logic failed. (This should have been fixed as part of DBZ-107.)
Added a table with data to one of the MySQL databases used in the integration tests. It verifies that the UTF-8 data stored in the table is able to be handled properly when obtaining a snapshot and reading the binlog.
The MySQL binlog events contain the binary representation of string-like values as encoded per the column's character set. Properly decoding these into Java strings requires capturing the column, table, and database character set when parsing the DDL statements.
Unfortunately, MySQL DDL allows columns (at the time the columns are created or modified) to inherit the default character set for the table, or if that is not defined the default character set for the database, or if that is not defined the character set for the server. So, in addition to modifying the MySQL DDL parser to support capturing the character set name for each column, it also had to be changed to know what these default character set names are.
The default character sets are all available via MySQL server/session/local variables. Although strictly speaking the character set variables cannot be set globally, MySQL DDL does allow session and local variables to be set with `SET` statements. Therefore, this commit enhances the MySQL DDL parser to parse `SET` statements and to track the various global, session, and local variables as seen by the DDL parser. Upon connector startup, a subset of server variables (related to character sets and collations) are read from the database via JDBC and used to initialize the DDL parser via `SET` methods.
In addition to initializing the DDL parser with the system variables related to character sets and collation, it is important to also capture the server and database default character sets in the database history so that the correct character sets are used for columns even when the default character sets have changed on the database and/or the server. Therefore, upon startup or snapshot the MySQL connector records in the database history a `SET` statement for the `character_set_server` and `collation_server` system variables so that, upon a later restart, the history's DDL statements can be re-parsed with the correct default server and database character sets. Also, when the MySQL connector reloads the database history (upon startup), the recorded default server character set is compared with the MySQL instance's current server character set, and if they are different the current character set is recorded with a new `SET` statement.
These extra steps ensure that the connector use the correct character set for each column, even when the connector restarts and reloads the database history captured by a previous version of the connector. IOW, the MySQL connector can be safely upgraded, and the new version will correctly start using the columns' character sets to decode the string-like values.
The DDL parser and in-memory models of the relational schemas were changed to capture the character set for each column whose type is a string (e.g., `CHAR`, `VARCHAR`, etc.). This required handling `SET` statements used to change the system variables that hold the names of the default character set for the server and for each database. So, even if a column does not explicitly define the character set, the column's actual character set is identified from the table's character set, which might default to the current database's character set, which if not set defaults to the system character set.
These changes merely affect how MySQL DDL is parsed and the in-memory relational schema representation to accommodate the character set at various levels. It does not change the behavior of the MySQL connector; that will be done in a subsequent commit.
All tests pass with these changes, including quite a few additional tests for the new functionality.
The binlog reader and JDBC operations might throw exceptions with this information, so in these cases the connector now captures the error code and SQLSTATE code from the exception and includes them in the message.
Changed the MySQL connector to have several new configuration properties for setting up the SSL key store and trust store (which can be used in place of System or JDK properties) used for MySQL secure connections, and another property to specify what kind of SSL connection be used.
Modified several integration tests to ensure all MySQL connections are made with `useSSL=false`.
The ENUM and SET values read from the binlog contain the indexes of the options that are included in the value, but this doesn't compared with the string values returned by MySQL and JDBC that contain the comma-separated options. With this change, the values read from the binlog will also be comma-separated strings.
Rewrote how the MySQL connector converts temporal values to use schemas with names that identify the semantic
type of temporal value, and customized how the MySQL binlog client library creates Java object values from the
raw binlog events.
Several new "semantic" schema types were defined:
* `io.debezium.time.Year` represents a year number as an INT32 value (e.g., 2016, -345, etc.).
* `io.debezium.time.Date` represents a date by storing the epoch seconds (that is, the number of seconds past the epoch) as an INT64 value.
* `io.debezium.time.Time` represents a time by storing the milliseconds past midnight as an INT32 value.
* `io.debezium.time.MicroTime` represents a time by storing the microsconds past midnight as an INT32 value.
* `io.debezium.time.NanoTime` represents a time by storing the nanoseconds past midnight as an INT32 value.
* `io.debezium.time.Timestamp` represents a date and time (without timezone information) by storing the milliseconds past epoch as an INT64 value.
* `io.debezium.time.MicroTimestamp` represents a date and time (without timezone information) by storing the microseconds past epoch as an INT64 value.
* `io.debezium.time.NanoTimestamp` represents a date and time (without timezone information) by storing the nanoseconds past epoch as an INT64 value.
* `io.debezium.time.ZonedTime` represents a time with timezone and optional fractions of a second (but no date) by storing the ISO8601 form as a STRING value (e.g., `10:15:30+01:00`)
* `io.debezium.time.ZonedTimestamp` represents a date and time with timezone and optional fractions of a second by storing the ISO8601 form as a STRING value (e.g., `2011-12-03T10:15:30.030431+01:00`)
This range of semantic types allows for a far more accurate representation in the events of the temporal values stored within the database. The MySQL connector chooses the semantic type based upon the precision of the MySQL type (e.g., `TIMESTAMP(6)` will be represented with `io.debezium.time.MicroTimestamp`, whereas `TIMESTAMP(3)` will be represented with `io.debezium.time.Timestamp`). This ensures that the events do not lose precision and that the semantics of the database column values are retained in the events even though the values are represented with primitive values.
Obviously these Kafka Connect schema representations are different and more precise than the built-in `org.apache.kafka.connect.data.Date`, `org.apache.kafka.connect.data.Time`, and `org.apache.kafka.connect.data.Timestamp` logical types provided by Kafka Connect and used by the MySQL connector in all 0.2.x and 0.1.x versions. Migration to the new MySQL connector should be possible, although consumers may still need to know about these types to properly handle temporal values and the correct precision (i.e., consumers can just assume all date INT64 values represent milliseconds).
The MySQL binlog client library converted the raw binary event information to JDBC types using a local Calendar instance, which obviously incorporates the local timezone and cannot retain more than millisecond precision. This change extends the library's deserializers to instead use the Java 8 `javax.time` classes and to retain the exact semantics of the database values and to not lose any precisions (since the `javax.time` classes have nanosecond precision).
The same logic is also used to convert the JDBC values obtained during a snapshot from the MySQL Connect/J JDBC driver. The latter has a few quirks, such as not returning any fractional seconds for `TIME` columns, even though `java.sql.Time` can store up to milliseconds.
Most of the logic of the conversions of values and mapping to Kafka Connect schemas is handled in the new `JdbcValueConverters`, which was extracted from the existing `TableSchemaBuilder`. The MySQL connector reuses and actually extends the `JdbcValueConverters` class with its own `MySqlValueConverters` class that also adds support for MySQL-specific types such as `YEAR`. Other connectors whose values are based on JDBC types should be able to reuse and/or extend the `JdbcValueConverters` class.
Integration tests that deal with temporal types were modified to use proper expected values and comparisons.
By default the MySQL JDBC driver will put the entire result set into memory, which obviously doesn't work for tables of even moderate sizes. This change adds support for streaming rows in result sets when the tables have more than a configurable number of rows (defaults to 1,000).
This posed a problem for how we were previously finding the last row in the last table; the MySQL driver does not support `ResultSet.isLast()` on result sets that are streamed. Instead, this commit wraps the consumer to which the snapshot reader writes all source records, with a consumer that buffers the last record. When the snapshot completes, the offset is updated (denoting the end of the snapshot) and set on the last buffered record before that record is flushed to the normal consumer. This should add minimal overhead while simplifying the logic to ensure the last source record has the updated offset.
This also improves the log output of the snapshot process.
The mysql:5.7 docker image changed its output to be more like mysql/mysql-server:5.7, and this broke our build because of what our build is looking for while waiting to for the server to completely intialize. Simply changing the pattern corrects the problem.
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.
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.
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.
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.
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.
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.
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.
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
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.
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.
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.
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.
The `VerifyRecord` utility class has methods that will verify a `SourceRecord`, and is used in many of our integration tests to check whether records are constructed in a valid manner. The utility already checks whether the records can be serialized and deserialized using the JSON converter (provided with Kafka Connect); this change also checks with the Avro Converter (which produces much smaller records and is more suitable for production).
Note that version 3.0.0 of the Confluent Avro Converter is required; version 2.1.0-alpha1 could not properly handle complex Schema objects with optional fields (see https://github.com/confluentinc/schema-registry/pull/280).
Also, the names of the Kafka Connect schemas used in MySQL source records has changed.
# The record's envelope Schema used to be "<serverName>.<database>.<table>" but is now "<serverName>.<database>.<table>.Envelope".
# The Schema for record keys used to be named "<database>.<table>/pk", but the '/' character is not valid within a Avro name, and has been changed to "<serverName>.<database>.<table>.Key".
# The Schema for record values used to be named "<database>.<table>", but to better fit with the other Schema names it has been changed to "<serverName>.<database>.<table>.Value".
Thus, all of the Schemas for a single database table have the same Avro namespace "<serverName>.<database>.<table>" (or "<topicName>") with Avro schema names of "Envelope", "Key", and "Value".
All unit and integration tests pass.
Changed the MySQL connector to make use of MDC logging contexts, which allow thread-specific parameters that can be written out on every log line by simply changing the logging configuration (e.g., Log4J configuration file).
We adopt a convention for all Debezium connectors with the following MDC properties:
* `dbz.connectorType` - the type of connector, which would be a single well-known value for each connector (e.g., "MySQL" for the MySQL connector)
* `dbz.connectorName` - the name of the connector, which for the MySQL connector is simply the value of the `server.name` property (e.g., the logical name for the MySQL server/cluster). Unfortunately, Kafka Connect does not give us its name for the connector.
* `dbz.connectorContext` - the name of the thread, which is "main" for thread running the connector; the MySQL connector uses "snapshot" for the thread started by the snapshot reader, and "binlog" for the thread started by the binlog reader.
Different logging frameworks have their own way of using MDC properties. In a Log4J configuration, for example, simply use `%X{name}` in the logger's layout, where "name" is one of the properties listed above (or another MDC property).
Refactored the MySQL connector to break out the logic of reading the binlog into a separate class, added a similar class to read a full snapshot, and then updated the MySQL connector task class to use both. Added several test cases and updated the existing tests.
Several of the MySQL connector classes were fairly large and complicated, and to prepare for upcoming changes/enhancements these larger classes were refactored to pull out units of functionality. Currently all unit tests pass with these changes, with additional unit tests for these new components.
Previously, the DDL statements were being filtered and recorded based upon the name of the database that appeared in the binlog. However, that database name is actually the name of the database to which the client submitting the operation is connected, and is not necessarily the database _affected_ by the operation (e.g., when an operation includes a fully-qualified table name not in the connected-to database).
With these changes, the table/database affected by the DDL statements is now being used to filter the recording of the statements. The order of the DDL statements is still maintained, but since each DDL statement can apply to a separate database the DDL statements are batched (in the same original order) based upon the affected database. For example, two statements affecting "db1" will get batched together into one schema change record, followed by one statement affecting "db2" as a second schema change record, followed by another statement affecting "db1" as a third schema record.
Meanwhile, this change does not affect how the database history records the changes: it still records them as submitted using a single record for each separate binlog event/position. This is much safer as each binlog event (with specific position) is written atomically to the history stream. Also, since the database history stream is what the connector uses upon recovery, the database history records are now written _after_ any schema change records to ensure that, upon recovery after failure, no schema change records are lost (and instead have at-least-once delivery guarantees).
Added integration test logic to verify that UPDATE events include both 'before' and 'after' states (previously added as part of DBZ-52), to verify that altering a table does not generate events for the rows in that table, and that the 'before' and 'after' states (read from the binlog) are always defined in terms of the _current_ table schema. IOW, no special logic is needed to handle a 'before' state that has different columns than defined in the current table's definition.
Added to the Debezium event message's `source` information the MySQL server ID for the cluster process that recorded the event and the MySQL timestamp at which the event was recorded.
Added explicit support for handling `CHARSET` as an alias for `CHARACTER SET` in both tables and columns. `CREATE DATABASE` and `ALTER DATABASE` statements can also specify character sets, but the DDL parser handles but does not explicitly parse them so no modification is needed for them. Several unit tests were added to confirm the behavior.
The new envelope Struct contains fields for the local time at which the connector processed the event, the kind of operation (e.g., read, insert, update, or delete), the state of the record before and after the change, and the information about the event source. The latter two items are connector-specific. The timestamp is merely the time using the connector's process clock, and no guarantees are provided about accuracy, monotonicity, or relationship to the original source event.
The envelope structure is now used as the value for each event message in the MySQL connector; they keys of the event messages remain unchanged. Note that to facilitate Kafka log compaction (which requires a null value), a delete event containing the envelope with details about the deletion is followed by a "tombstone" event that contains the same key but null value.
An example of a message value with this new envelope is as follows:
{
"schema" : {
"type" : "struct",
"fields" : [ {
"type" : "struct",
"fields" : [ {
"type" : "int32",
"optional" : false,
"name" : "org.apache.kafka.connect.data.Date",
"version" : 1,
"field" : "order_date"
}, {
"type" : "int32",
"optional" : false,
"field" : "purchaser"
}, {
"type" : "int32",
"optional" : false,
"field" : "quantity"
}, {
"type" : "int32",
"optional" : false,
"field" : "product_id"
} ],
"optional" : true,
"name" : "connector_test.orders",
"field" : "before"
}, {
"type" : "struct",
"fields" : [ {
"type" : "int32",
"optional" : false,
"name" : "org.apache.kafka.connect.data.Date",
"version" : 1,
"field" : "order_date"
}, {
"type" : "int32",
"optional" : false,
"field" : "purchaser"
}, {
"type" : "int32",
"optional" : false,
"field" : "quantity"
}, {
"type" : "int32",
"optional" : false,
"field" : "product_id"
} ],
"optional" : true,
"name" : "connector_test.orders",
"field" : "after"
}, {
"type" : "struct",
"fields" : [ {
"type" : "string",
"optional" : false,
"field" : "server"
}, {
"type" : "string",
"optional" : false,
"field" : "file"
}, {
"type" : "int64",
"optional" : false,
"field" : "pos"
}, {
"type" : "int32",
"optional" : false,
"field" : "row"
} ],
"optional" : false,
"name" : "io.debezium.connector.mysql.Source",
"field" : "source"
}, {
"type" : "string",
"optional" : false,
"field" : "op"
}, {
"type" : "int64",
"optional" : true,
"field" : "ts"
} ],
"optional" : false,
"name" : "kafka-connect-2.connector_test.orders",
"version" : 1
},
"payload" : {
"before" : null,
"after" : {
"order_date" : 16852,
"purchaser" : 1003,
"quantity" : 1,
"product_id" : 107
},
"source" : {
"server" : "kafka-connect-2",
"file" : "mysql-bin.000002",
"pos" : 2887680,
"row" : 4
},
"op" : "c",
"ts" : 1463437199134
}
}
Notice how the Schema is significantly larger, since it must describe all of the envelope's fields even when those fields are not used. In this case, the event signifies that a record was created as the 4th record of a single event recorded in the binlog.
When a row is updated in the database and the primary/unique key for that table is changed, the MySQL connector continues to generate an update event with the new key and new value, but now also generates a tombstone event for the old key. This ensures that when a Kafka topic is compacted, all prior events with the old key will (eventually) be removed. It also ensures that consumers see that the row represented by the old key has been removed.
MySQL 5.6 using the MyISAM engine will create the `help_relation` system table using a CREATE TABLE statement that does not have in the columns' REFERENCE clause a list of columns in the referenced table. MySQL 5.7 using the InnoDB engine does not include the REFERENCE clauses.
Because Debezium's MySQL DDL parser is meant only to understand the statements recorded in the binlog, it does not have to validate the statements and therefore the DDL parser can be a bit more lenient by not requiring the list of columns in a REFERENCE clause in a CREATE TABLE statement's column definitions.
This commit also adds several unit tests that validate all of the DDL statements used by MySQL 5.6 and 5.7 during startup (in the configurations used in our integration tests).
Changed the MySQL connector to use comma-separated lists of regular expressions for the database
and table whitelist/blacklists. Literals are still accepted and will match fully-qualified table names,
although the '.' character used as a delimiter is also a special character in regular expressions and
therefore may need to be escaped with a double backslash ('\\') to more carefully match fully-qualified
table names.
Added several new configuration properties for the MySQL connector that instruct it to hide,
truncate, and/or mask certain columns. The properties' values are all lists of regular expressions
or literal fully-qualified column names. For example, the following configuration property:
column.blacklist=server.users.picture,server.users.other
will cause the connector to leave out of change event messages for the `server.users` table those
fields that correspond to the `picture` and `others` columns. This capability can be used to
This capability can be used to prevent dissemination of sensitive information in the change event
stream.
An alternative to blacklisting is masking. The following configuration property:
column.mask.with.10.chars=server\\.users\\.(\\w*email)
will cause the connector to mask in the change event messages for the `server.users` table
all values for columns whose name ends in `email`. The values will be replaced in this case with
a constant string of 10 asterisk ('*') characters, even when the email value is null.
This capability can also be used to prevent dissemination of sensitive information in the change event
stream.
Another option is to truncate string values for specific columns. The following configuration
property:
column.truncate.to.120.chars=server[.]users[.](description|biography)
will cause the connector to truncate to at most 120 characters the values of the `description` and
`biography` columns in the change event messages for the `server.users` table. Although this example
used a limit of 120 characters, any positive length can be specified; separate properties should
be used when different lengths are required. Note how the '.' delimiter in the fully-qualified names
is escaped since that same character is a special character in regular expressions. This capability
can be used to reduce the size of change event messages.
Drop table/view statements that involve more than one table generate one event for each table/view. Previously, each of those statements had the original multi-table/view statement. Now, each event has a statement that applies to only that table (generated from the original with all the same clauses).
The previous change did not correctly capture the statements for a `RENAME TO` that renamed multiple tables, so fixed the code so that it generates a single `RENAME TO` for each table rename.
Refactored the mechanism by which components can listen to the activities of a DDL parser. The new approach
should be significantly more flexible for additional types of DDL events while making it easier to maintain
backward compatibility. It also will enable passing event-specific information on each DDL event.
There is no point in sending a record that contains a null key and null value. While this may not be likely for insert or update cases (since at least the value should not be null), it is possible when a row is deleted (meaning the record value will be null) but the table has no primary/unique key (meaning the record key will be null).
Corrects a bug where a deleted row was written to Kafka in the same as an insert, making them indistinguishable. Now, a deleted row is written with the row's primary/unique key as the record key, and a null record value. Note that if the row has no primary/unique key, no record is written to Kafka.
Depending upon how the source MySQL database is configured, its binlog might not contain all the history for the database. In particular, it might not have the CREATE TABLE statements for some tables, which are then "unknown" to the connector. When the connector reads the binlog and comes across a change event for a row in one of those "unknown" tables, it previously resulted in a NPE. With this change, the condition results in a warning message in the log, and all subsequent change events on that table will be skipped.
Modified the 'docs' profile to build and attach JavaDoc JARs for each module's source and test source artifacts. The profile will be automatically used when releasing.
Changed the EmbeddedConnector framework to initialize all major components via configuration properties rather than through the public builder. This increases the size of the configurations, but it simplifies what embedding applications must do to obtain an EmbeddedConnector instance.
The DatabaseHistory framework was also changed to be configurable in similar ways to the OffsetBackingStore. Essentially, connectors that want to use it (like the MySqlConnector) will describe it as part of the connector's configuration, allowing more flexibility in which DatabaseHistory implementation is used and how it is configured whether in Kafka Connector or as part of the EmbeddedConnector.
Added a README.md to `debezium-embedded` to provide documentation and sample code showing how to use the EmbeddedConnector.