Understanding MySQL Source Connector schema_only snapshot.mode Behaviour

665 views
Skip to first unread message

Cliff Wheadon

unread,
Aug 22, 2018, 2:30:56 PM8/22/18
to debezium
Hello Everyone,

My team and I are currently developing a solution that uses a Debezium MySQL Source Connector to get data onto Kafka. Some of the downstream clients of our application are unable to handle the amount of data that a full data snapshot sends onto kafka, and none of them currently need to ensure that they've seen a message for every change in the source MySQL database. After picking around in the documentation we felt like setting the snapshot.mode to schema_only would alleviate our issues, as on initial load and any restart the Source Connector will not touch all of the data in whitelisted tables as part of the snapshot that it performs. This seems to work as expected on initial startup (we don't see a deluge of messages hit kafka), but when we run into situations where the connector restarts and there is no binlog available the connector completely dies (captured in a bug here: https://issues.jboss.org/browse/DBZ-414). We've enabled messages on a heartbeat topic to get around: https://issues.jboss.org/browse/DBZ-220 ... but it's still possible that in some situations the binlog won't be available on a Connector restart (after the connector has been down for a long time for example).

Why does the Connector need to fail when snapshot.mode mode is schema_only and the previously used binlog is no longer available? Can the connector use the exact same behaviour it does with the when_needed snapshot.mode but simply skip loading data? How would a "non-data" snapshot on a restart without a binlog differ from the one that is performed on the initial startup of the Source Connector?

Thanks much in advance for your help.

Cheers,

Cliff Wheadon

Gunnar Morling

unread,
Aug 23, 2018, 2:27:57 AM8/23/18
to debezium
Hi Cliff,

I think it's the best if you generally avoid the situation of not having the binlogs available. I.e. I'd recommend to set the log retention time to a value that's larger than the anticipated longest connector downtime.

That being said, you might have success by using the rather new "schema_only_recovery" mode. For that to work, you'd have to update the connector's offsets to some position of the binlogs you still have (e.g. either the oldest or the latest one, depending on your requirements). There must have no schema changes happened after that position. It will then take a new schema snapshot and continue streaming from that position.

Cheers,

--Gunnar

Cliff Wheadon

unread,
Aug 27, 2018, 6:52:04 PM8/27/18
to debezium
Hello Gunnar,

Thanks much for the reply, and I'm sorry it's taken so long to get back to you.

For now, we're setting the binlog retention period to some long period of time, and are doing what we can to ensure that our connector isn't down/idle for long enough to lose the binlog. I'm still very interested in the "schema_only" snapshot mode, as the semantics described in the documentation seem to match exactly what we want ("With the schema_only mode the connector reads the current binlog position, captures the current table schemas without reading any data, and then proceeds to read the binlog from its current position. "). Am I perhaps misinterpreting this? I'd assumed this meant that whenever the connector didn't have access to its previous binlog that it performed the steps outlined here: https://debezium.io/docs/connectors/mysql/#snapshots while skipping step 7 (and 6 unless include_schema_changes = true) and then continued reading the binlog from the earliest position that MySQL knew about.

I've picked around in the code for a bit, and I can see that there is an "includeData" flag that is set to false on the schema_only SnapshotMode, and that tells the SnapshotReader to skip touching every row in the source db when a Snapshot is performed. I can also see that in MySQLConnectorTask an exception is thrown explicitly when we don't have the binlog and SnapshotMode is not "when_needed". Would it be safe for the MySQLConnectorTask to go ahead and re-run the snapshot when SnapShotMode is schema_only as well? From what I can see the SnapShotReader would just skip the data load (Step 8) as it does when it first fires up.

For what it's worth, I amended the if-statement where the exception is thrown in schema_only mode to set startWithSnapshot = true when SnapshotMode is schema_only, and all of the tests still pass ... but I'm fairly naively poking around and could definitely be missing something. 

If I'm not out to lunch on the above I can create something on Jira and issue a PR so that there's a better forum to discuss the above-mentioned changes.

Cheers,

Cliff

Gunnar Morling

unread,
Aug 29, 2018, 3:17:28 AM8/29/18
to debezium
Hi Cliff,

So IIUC, you're after some snapshotting mode that could be described as "schema_only_when_needed"? I.e. it'd take a schema snapshot (but no data) every time, not just upon initial start-up? I suppose this could be done, although I'm a bit concerned about the proliferation of snapshotting modes, which already right now are quite complex and tough to understand.

I don't think that the behaviour of the existing "schema_only" mode should be changed: currently it guarantees a complete history of changes (or the connector will fail if it doesn't find the required binlogs after a restart). This wouldn't be the case when automatically taking a new schema snapshot automatically, which is why this should be made more explicit.

That being said, it seems you could achieve your desired semantics by simply dropping and re-creating the connector if the binlogs are missing, which will be reasonably fast in "schema_only" mode?

--Gunnar
Reply all
Reply to author
Forward
0 new messages